You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by go...@apache.org on 2021/12/15 09:49:46 UTC

[flink] branch master updated: [FLINK-25076][table-planner] Improve vertex name for sql job

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

godfrey 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 1255c19  [FLINK-25076][table-planner] Improve vertex name for sql job
1255c19 is described below

commit 1255c190b750da5a560204dfe9ff283082d0d424
Author: 龙三 <we...@alibaba-inc.com>
AuthorDate: Fri Nov 5 17:43:56 2021 +0800

    [FLINK-25076][table-planner] Improve vertex name for sql job
    
    This closes #18042
---
 .../generated/optimizer_config_configuration.html  |    6 +
 .../flink/connectors/hive/HiveTableSinkITCase.java |    5 +-
 .../testHiveTableSinkWithParallelismInBatch.out    |   24 +-
 ...testHiveTableSinkWithParallelismInStreaming.out |   24 +-
 .../flink/table/client/cli/CliClientITCase.java    |    6 +-
 .../src/test/resources/sql/table.q                 |   48 +-
 .../table/api/config/OptimizerConfigOptions.java   |    7 +
 .../planner/connectors/ExternalDynamicSink.java    |    9 +-
 .../planner/connectors/ExternalDynamicSource.java  |    9 +-
 .../planner/plan/nodes/exec/ExecNodeBase.java      |   41 +
 .../exec/batch/BatchExecBoundedStreamScan.java     |    5 +
 .../nodes/exec/batch/BatchExecHashAggregate.java   |    3 +-
 .../plan/nodes/exec/batch/BatchExecHashJoin.java   |    3 +-
 .../exec/batch/BatchExecHashWindowAggregate.java   |    3 +-
 .../exec/batch/BatchExecLegacyTableSourceScan.java |    7 +-
 .../plan/nodes/exec/batch/BatchExecLimit.java      |    7 +-
 .../nodes/exec/batch/BatchExecMultipleInput.java   |    3 +-
 .../nodes/exec/batch/BatchExecNestedLoopJoin.java  |    3 +-
 .../nodes/exec/batch/BatchExecOverAggregate.java   |    3 +-
 .../exec/batch/BatchExecPythonGroupAggregate.java  |    6 +-
 .../batch/BatchExecPythonGroupWindowAggregate.java |    6 +-
 .../exec/batch/BatchExecPythonOverAggregate.java   |    6 +-
 .../plan/nodes/exec/batch/BatchExecRank.java       |    7 +-
 .../plan/nodes/exec/batch/BatchExecSort.java       |    3 +-
 .../nodes/exec/batch/BatchExecSortAggregate.java   |    7 +-
 .../plan/nodes/exec/batch/BatchExecSortLimit.java  |    7 +-
 .../nodes/exec/batch/BatchExecSortMergeJoin.java   |    3 +-
 .../exec/batch/BatchExecSortWindowAggregate.java   |    7 +-
 .../plan/nodes/exec/common/CommonExecCalc.java     |    7 +-
 .../nodes/exec/common/CommonExecCorrelate.java     |    3 +-
 .../plan/nodes/exec/common/CommonExecExpand.java   |    7 +-
 .../nodes/exec/common/CommonExecLegacySink.java    |   11 +-
 .../nodes/exec/common/CommonExecLookupJoin.java    |    7 +-
 .../nodes/exec/common/CommonExecPythonCalc.java    |   10 +-
 .../exec/common/CommonExecPythonCorrelate.java     |    6 +-
 .../plan/nodes/exec/common/CommonExecSink.java     |  112 +-
 .../exec/common/CommonExecTableSourceScan.java     |   41 +-
 .../plan/nodes/exec/common/CommonExecValues.java   |    3 +-
 .../exec/common/CommonExecWindowTableFunction.java |    7 +-
 .../exec/stream/StreamExecChangelogNormalize.java  |    6 +-
 .../exec/stream/StreamExecDataStreamScan.java      |    7 +-
 .../nodes/exec/stream/StreamExecDeduplicate.java   |    6 +-
 .../exec/stream/StreamExecDropUpdateBefore.java    |    7 +-
 .../stream/StreamExecGlobalGroupAggregate.java     |    6 +-
 .../stream/StreamExecGlobalWindowAggregate.java    |    3 +-
 .../exec/stream/StreamExecGroupAggregate.java      |    6 +-
 .../exec/stream/StreamExecGroupTableAggregate.java |    6 +-
 .../stream/StreamExecGroupWindowAggregate.java     |    6 +-
 .../StreamExecIncrementalGroupAggregate.java       |    6 +-
 .../nodes/exec/stream/StreamExecIntervalJoin.java  |   59 +-
 .../plan/nodes/exec/stream/StreamExecJoin.java     |    6 +-
 .../stream/StreamExecLegacyTableSourceScan.java    |    9 +-
 .../exec/stream/StreamExecLocalGroupAggregate.java |    7 +-
 .../stream/StreamExecLocalWindowAggregate.java     |    3 +-
 .../plan/nodes/exec/stream/StreamExecMatch.java    |    9 +-
 .../exec/stream/StreamExecMiniBatchAssigner.java   |    7 +-
 .../nodes/exec/stream/StreamExecOverAggregate.java |    6 +-
 .../stream/StreamExecPythonGroupAggregate.java     |    6 +-
 .../StreamExecPythonGroupTableAggregate.java       |    6 +-
 .../StreamExecPythonGroupWindowAggregate.java      |   11 +-
 .../exec/stream/StreamExecPythonOverAggregate.java |    6 +-
 .../plan/nodes/exec/stream/StreamExecRank.java     |    6 +-
 .../plan/nodes/exec/stream/StreamExecSort.java     |    7 +-
 .../nodes/exec/stream/StreamExecTemporalJoin.java  |    6 +-
 .../nodes/exec/stream/StreamExecTemporalSort.java  |   11 +-
 .../exec/stream/StreamExecWatermarkAssigner.java   |    7 +-
 .../exec/stream/StreamExecWindowAggregate.java     |    3 +-
 .../exec/stream/StreamExecWindowDeduplicate.java   |    3 +-
 .../nodes/exec/stream/StreamExecWindowJoin.java    |    6 +-
 .../nodes/exec/stream/StreamExecWindowRank.java    |    3 +-
 .../plan/nodes/exec/utils/ExecNodeUtil.java        |  106 +-
 .../planner/codegen/CorrelateCodeGenerator.scala   |    6 +-
 .../flink/table/planner/plan/utils/ScanUtil.scala  |   10 +-
 .../file/table/FileSystemTableSinkTest.java        |   13 +-
 .../api/internal/TableEnvironmentInternalTest.java |    4 +-
 .../nodes/exec/operator/BatchOperatorNameTest.java |  159 +
 .../nodes/exec/operator/OperatorNameTestBase.java  |  195 +
 .../exec/operator/StreamOperatorNameTest.java      |  404 ++
 ...stFileSystemTableSinkWithParallelismInBatch.out |    4 +-
 ...stemTableSinkWithParallelismInStreamingSql0.out |    4 +-
 ...stemTableSinkWithParallelismInStreamingSql1.out |    4 +-
 .../testExecuteSqlWithExplainDetailsAndUnion.out   |   10 +-
 .../testExecuteSqlWithExplainDetailsInsert.out     |   14 +-
 .../testExecuteSqlWithExplainDetailsSelect.out     |   10 +-
 .../test/resources/explain/testExplainJsonPlan.out |    8 +-
 .../explain/testStatementSetExecutionExplain.out   |   10 +-
 .../testStreamTableEnvironmentExecutionExplain.out |   10 +-
 .../nodes/exec/operator/BatchOperatorNameTest.xml  | 2099 ++++++++++
 .../nodes/exec/operator/StreamOperatorNameTest.xml | 4191 ++++++++++++++++++++
 .../planner/plan/stream/sql/TableSinkTest.xml      |   56 +-
 .../flink/table/api/TableEnvironmentTest.scala     |   16 +-
 .../planner/runtime/harness/HarnessTestBase.scala  |   11 +-
 .../flink/table/planner/utils/TableTestBase.scala  |   12 +-
 93 files changed, 7759 insertions(+), 325 deletions(-)

diff --git a/docs/layouts/shortcodes/generated/optimizer_config_configuration.html b/docs/layouts/shortcodes/generated/optimizer_config_configuration.html
index 61fc8ce..fd3a4dc 100644
--- a/docs/layouts/shortcodes/generated/optimizer_config_configuration.html
+++ b/docs/layouts/shortcodes/generated/optimizer_config_configuration.html
@@ -60,6 +60,12 @@ ONE_PHASE: Enforce to use one stage aggregate which only has CompleteGlobalAggre
             <td>When it is true, the optimizer will try to find out duplicated sub-plans and reuse them.</td>
         </tr>
         <tr>
+            <td><h5>table.optimizer.simplify-operator-name-enabled</h5><br> <span class="label label-primary">Batch</span> <span class="label label-primary">Streaming</span></td>
+            <td style="word-wrap: break-word;">true</td>
+            <td>Boolean</td>
+            <td>When it is true, the optimizer will simplify the operator name with id and type of ExecNode and keep detail in description. Default value is true.</td>
+        </tr>
+        <tr>
             <td><h5>table.optimizer.source.aggregate-pushdown-enabled</h5><br> <span class="label label-primary">Batch</span></td>
             <td style="word-wrap: break-word;">true</td>
             <td>Boolean</td>
diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveTableSinkITCase.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveTableSinkITCase.java
index 0cd9ff2..1d2b4f5 100644
--- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveTableSinkITCase.java
+++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveTableSinkITCase.java
@@ -69,6 +69,7 @@ import static org.apache.flink.connector.file.table.FileSystemConnectorOptions.S
 import static org.apache.flink.connector.file.table.FileSystemConnectorOptions.SINK_PARTITION_COMMIT_SUCCESS_FILE_NAME;
 import static org.apache.flink.table.api.Expressions.$;
 import static org.apache.flink.table.planner.utils.TableTestUtil.readFromResource;
+import static org.apache.flink.table.planner.utils.TableTestUtil.replaceNodeIdInOperator;
 import static org.apache.flink.table.planner.utils.TableTestUtil.replaceStageId;
 import static org.apache.flink.table.planner.utils.TableTestUtil.replaceStreamNodeId;
 import static org.junit.Assert.assertEquals;
@@ -131,8 +132,8 @@ public class HiveTableSinkITCase {
         final String expected = readFromResource(expectedResourceFileName);
 
         assertEquals(
-                replaceStreamNodeId(replaceStageId(expected)),
-                replaceStreamNodeId(replaceStageId(actual)));
+                replaceNodeIdInOperator(replaceStreamNodeId(replaceStageId(expected))),
+                replaceNodeIdInOperator(replaceStreamNodeId(replaceStageId(actual))));
 
         tEnv.executeSql("drop database db1 cascade");
     }
diff --git a/flink-connectors/flink-connector-hive/src/test/resources/explain/testHiveTableSinkWithParallelismInBatch.out b/flink-connectors/flink-connector-hive/src/test/resources/explain/testHiveTableSinkWithParallelismInBatch.out
index 1dac561..5858300 100644
--- a/flink-connectors/flink-connector-hive/src/test/resources/explain/testHiveTableSinkWithParallelismInBatch.out
+++ b/flink-connectors/flink-connector-hive/src/test/resources/explain/testHiveTableSinkWithParallelismInBatch.out
@@ -16,43 +16,43 @@ Sink(table=[test-catalog.db1.test_table], fields=[EXPR$0, EXPR$1])
 == Physical Execution Plan ==
 {
   "nodes" : [ {
-    "id" : 1,
-    "type" : "Source: Values(tuples=[[{ 0 }]], values=[ZERO])",
+    "id" : ,
+    "type" : "Source: Values[]",
     "pact" : "Data Source",
-    "contents" : "Source: Values(tuples=[[{ 0 }]], values=[ZERO])",
+    "contents" : "[]:Values(tuples=[[{ 0 }]], values=[ZERO])",
     "parallelism" : 1
   }, {
-    "id" : 2,
-    "type" : "Calc(select=[1 AS EXPR$0, 1 AS EXPR$1])",
+    "id" : ,
+    "type" : "Calc[]",
     "pact" : "Operator",
-    "contents" : "Calc(select=[1 AS EXPR$0, 1 AS EXPR$1])",
+    "contents" : "[]:Calc(select=[1 AS EXPR$0, 1 AS EXPR$1])",
     "parallelism" : 1,
     "predecessors" : [ {
-      "id" : 1,
+      "id" : ,
       "ship_strategy" : "FORWARD",
       "side" : "second"
     } ]
   }, {
-    "id" : 3,
+    "id" : ,
     "type" : "Map",
     "pact" : "Operator",
     "contents" : "Map",
     "parallelism" : 1,
     "predecessors" : [ {
-      "id" : 2,
+      "id" : ,
       "ship_strategy" : "FORWARD",
       "side" : "second"
     } ]
   }, {
-    "id" : 4,
+    "id" : ,
     "type" : "Sink: Unnamed",
     "pact" : "Data Sink",
     "contents" : "Sink: Unnamed",
     "parallelism" : 8,
     "predecessors" : [ {
-      "id" : 3,
+      "id" : ,
       "ship_strategy" : "REBALANCE",
       "side" : "second"
     } ]
   } ]
-}
+}
\ No newline at end of file
diff --git a/flink-connectors/flink-connector-hive/src/test/resources/explain/testHiveTableSinkWithParallelismInStreaming.out b/flink-connectors/flink-connector-hive/src/test/resources/explain/testHiveTableSinkWithParallelismInStreaming.out
index 7e42154..e072969 100644
--- a/flink-connectors/flink-connector-hive/src/test/resources/explain/testHiveTableSinkWithParallelismInStreaming.out
+++ b/flink-connectors/flink-connector-hive/src/test/resources/explain/testHiveTableSinkWithParallelismInStreaming.out
@@ -16,43 +16,43 @@ Sink(table=[test-catalog.db1.test_table], fields=[EXPR$0, EXPR$1])
 == Physical Execution Plan ==
 {
   "nodes" : [ {
-    "id" : 1,
-    "type" : "Source: Values(tuples=[[{ 0 }]])",
+    "id" : ,
+    "type" : "Source: Values[]",
     "pact" : "Data Source",
-    "contents" : "Source: Values(tuples=[[{ 0 }]])",
+    "contents" : "[]:Values(tuples=[[{ 0 }]])",
     "parallelism" : 1
   }, {
-    "id" : 2,
-    "type" : "Calc(select=[1 AS EXPR$0, 1 AS EXPR$1])",
+    "id" : ,
+    "type" : "Calc[]",
     "pact" : "Operator",
-    "contents" : "Calc(select=[1 AS EXPR$0, 1 AS EXPR$1])",
+    "contents" : "[]:Calc(select=[1 AS EXPR$0, 1 AS EXPR$1])",
     "parallelism" : 1,
     "predecessors" : [ {
-      "id" : 1,
+      "id" : ,
       "ship_strategy" : "FORWARD",
       "side" : "second"
     } ]
   }, {
-    "id" : 3,
+    "id" : ,
     "type" : "StreamingFileWriter",
     "pact" : "Operator",
     "contents" : "StreamingFileWriter",
     "parallelism" : 8,
     "predecessors" : [ {
-      "id" : 2,
+      "id" : ,
       "ship_strategy" : "REBALANCE",
       "side" : "second"
     } ]
   }, {
-    "id" : 4,
+    "id" : ,
     "type" : "Sink: end",
     "pact" : "Data Sink",
     "contents" : "Sink: end",
     "parallelism" : 1,
     "predecessors" : [ {
-      "id" : 3,
+      "id" : ,
       "ship_strategy" : "REBALANCE",
       "side" : "second"
     } ]
   } ]
-}
+}
\ No newline at end of file
diff --git a/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/CliClientITCase.java b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/CliClientITCase.java
index 4292bc9..f0bde83 100644
--- a/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/CliClientITCase.java
+++ b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/CliClientITCase.java
@@ -25,6 +25,7 @@ import org.apache.flink.table.client.gateway.Executor;
 import org.apache.flink.table.client.gateway.context.DefaultContext;
 import org.apache.flink.table.client.gateway.local.LocalExecutor;
 import org.apache.flink.table.client.gateway.utils.UserDefinedFunctions;
+import org.apache.flink.table.planner.utils.TableTestUtil;
 import org.apache.flink.table.utils.TestUserClassLoaderJar;
 import org.apache.flink.test.util.AbstractTestBase;
 
@@ -304,7 +305,8 @@ public class CliClientITCase extends AbstractTestBase {
             out.append(sqlScript.comment).append(sqlScript.sql);
             if (i < results.size()) {
                 Result result = results.get(i);
-                String content = removeStreamNodeId(result.content);
+                String content =
+                        TableTestUtil.replaceNodeIdInOperator(removeExecNodeId(result.content));
                 out.append(content).append(result.highestTag.tag).append("\n");
             }
         }
@@ -312,7 +314,7 @@ public class CliClientITCase extends AbstractTestBase {
         return out.toString();
     }
 
-    private static String removeStreamNodeId(String s) {
+    private static String removeExecNodeId(String s) {
         return s.replaceAll("\"id\" : \\d+", "\"id\" : ");
     }
 
diff --git a/flink-table/flink-sql-client/src/test/resources/sql/table.q b/flink-table/flink-sql-client/src/test/resources/sql/table.q
index 74048e7..a016087 100644
--- a/flink-table/flink-sql-client/src/test/resources/sql/table.q
+++ b/flink-table/flink-sql-client/src/test/resources/sql/table.q
@@ -665,15 +665,15 @@ Sink(table=[default_catalog.default_database.orders2], fields=[user, product, am
 {
   "nodes" : [ {
     "id" : ,
-    "type" : "Source: TableSourceScan(table=[[default_catalog, default_database, orders]], fields=[user, product, amount, ts])",
+    "type" : "Source: orders[]",
     "pact" : "Data Source",
-    "contents" : "Source: TableSourceScan(table=[[default_catalog, default_database, orders]], fields=[user, product, amount, ts])",
+    "contents" : "[]:TableSourceScan(table=[[default_catalog, default_database, orders]], fields=[user, product, amount, ts])",
     "parallelism" : 1
   }, {
     "id" : ,
-    "type" : "WatermarkAssigner(rowtime=[ts], watermark=[(ts - 1000:INTERVAL SECOND)])",
+    "type" : "WatermarkAssigner[]",
     "pact" : "Operator",
-    "contents" : "WatermarkAssigner(rowtime=[ts], watermark=[(ts - 1000:INTERVAL SECOND)])",
+    "contents" : "[]:WatermarkAssigner(rowtime=[ts], watermark=[(ts - 1000:INTERVAL SECOND)])",
     "parallelism" : 1,
     "predecessors" : [ {
       "id" : ,
@@ -682,9 +682,9 @@ Sink(table=[default_catalog.default_database.orders2], fields=[user, product, am
     } ]
   }, {
     "id" : ,
-    "type" : "ConstraintEnforcer[NotNullEnforcer(fields=[user])]",
+    "type" : "ConstraintEnforcer[]",
     "pact" : "Operator",
-    "contents" : "ConstraintEnforcer[NotNullEnforcer(fields=[user])]",
+    "contents" : "[]:ConstraintEnforcer[NotNullEnforcer(fields=[user])]",
     "parallelism" : 1,
     "predecessors" : [ {
       "id" : ,
@@ -693,9 +693,9 @@ Sink(table=[default_catalog.default_database.orders2], fields=[user, product, am
     } ]
   }, {
     "id" : ,
-    "type" : "Sink: Sink(table=[default_catalog.default_database.orders2], fields=[user, product, amount, ts])",
+    "type" : "Sink: orders2[]",
     "pact" : "Data Sink",
-    "contents" : "Sink: Sink(table=[default_catalog.default_database.orders2], fields=[user, product, amount, ts])",
+    "contents" : "[]:Sink(table=[default_catalog.default_database.orders2], fields=[user, product, amount, ts])",
     "parallelism" : 1,
     "predecessors" : [ {
       "id" : ,
@@ -730,15 +730,15 @@ Calc(select=[user, product])
 {
   "nodes" : [ {
     "id" : ,
-    "type" : "Source: TableSourceScan(table=[[default_catalog, default_database, orders]], fields=[user, product, amount, ts])",
+    "type" : "Source: orders[]",
     "pact" : "Data Source",
-    "contents" : "Source: TableSourceScan(table=[[default_catalog, default_database, orders]], fields=[user, product, amount, ts])",
+    "contents" : "[]:TableSourceScan(table=[[default_catalog, default_database, orders]], fields=[user, product, amount, ts])",
     "parallelism" : 1
   }, {
     "id" : ,
-    "type" : "Calc(select=[user, product, ts])",
+    "type" : "Calc[]",
     "pact" : "Operator",
-    "contents" : "Calc(select=[user, product, ts])",
+    "contents" : "[]:Calc(select=[user, product, ts])",
     "parallelism" : 1,
     "predecessors" : [ {
       "id" : ,
@@ -747,9 +747,9 @@ Calc(select=[user, product])
     } ]
   }, {
     "id" : ,
-    "type" : "WatermarkAssigner(rowtime=[ts], watermark=[(ts - 1000:INTERVAL SECOND)])",
+    "type" : "WatermarkAssigner[]",
     "pact" : "Operator",
-    "contents" : "WatermarkAssigner(rowtime=[ts], watermark=[(ts - 1000:INTERVAL SECOND)])",
+    "contents" : "[]:WatermarkAssigner(rowtime=[ts], watermark=[(ts - 1000:INTERVAL SECOND)])",
     "parallelism" : 1,
     "predecessors" : [ {
       "id" : ,
@@ -758,9 +758,9 @@ Calc(select=[user, product])
     } ]
   }, {
     "id" : ,
-    "type" : "Calc(select=[user, product])",
+    "type" : "Calc[]",
     "pact" : "Operator",
-    "contents" : "Calc(select=[user, product])",
+    "contents" : "[]:Calc(select=[user, product])",
     "parallelism" : 1,
     "predecessors" : [ {
       "id" : ,
@@ -839,15 +839,15 @@ Calc(select=[user, product])
 {
   "nodes" : [ {
     "id" : ,
-    "type" : "Source: TableSourceScan(table=[[default_catalog, default_database, orders]], fields=[user, product, amount, ts])",
+    "type" : "Source: orders[]",
     "pact" : "Data Source",
-    "contents" : "Source: TableSourceScan(table=[[default_catalog, default_database, orders]], fields=[user, product, amount, ts])",
+    "contents" : "[]:TableSourceScan(table=[[default_catalog, default_database, orders]], fields=[user, product, amount, ts])",
     "parallelism" : 1
   }, {
     "id" : ,
-    "type" : "Calc(select=[user, product, ts])",
+    "type" : "Calc[]",
     "pact" : "Operator",
-    "contents" : "Calc(select=[user, product, ts])",
+    "contents" : "[]:Calc(select=[user, product, ts])",
     "parallelism" : 1,
     "predecessors" : [ {
       "id" : ,
@@ -856,9 +856,9 @@ Calc(select=[user, product])
     } ]
   }, {
     "id" : ,
-    "type" : "WatermarkAssigner(rowtime=[ts], watermark=[(ts - 1000:INTERVAL SECOND)])",
+    "type" : "WatermarkAssigner[]",
     "pact" : "Operator",
-    "contents" : "WatermarkAssigner(rowtime=[ts], watermark=[(ts - 1000:INTERVAL SECOND)])",
+    "contents" : "[]:WatermarkAssigner(rowtime=[ts], watermark=[(ts - 1000:INTERVAL SECOND)])",
     "parallelism" : 1,
     "predecessors" : [ {
       "id" : ,
@@ -867,9 +867,9 @@ Calc(select=[user, product])
     } ]
   }, {
     "id" : ,
-    "type" : "Calc(select=[user, product])",
+    "type" : "Calc[]",
     "pact" : "Operator",
-    "contents" : "Calc(select=[user, product])",
+    "contents" : "[]:Calc(select=[user, product])",
     "parallelism" : 1,
     "predecessors" : [ {
       "id" : ,
diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/config/OptimizerConfigOptions.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/config/OptimizerConfigOptions.java
index 1e01256..2108616 100644
--- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/config/OptimizerConfigOptions.java
+++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/config/OptimizerConfigOptions.java
@@ -123,4 +123,11 @@ public class OptimizerConfigOptions {
                     .withDescription(
                             "When it is true, the optimizer will merge the operators with pipelined shuffling "
                                     + "into a multiple input operator to reduce shuffling and improve performance. Default value is true.");
+
+    @Documentation.TableOption(execMode = Documentation.ExecMode.BATCH_STREAMING)
+    public static final ConfigOption<Boolean> TABLE_OPTIMIZER_SIMPLIFY_OPERATOR_NAME_ENABLED =
+            key("table.optimizer.simplify-operator-name-enabled")
+                    .defaultValue(true)
+                    .withDescription(
+                            "When it is true, the optimizer will simplify the operator name with id and type of ExecNode and keep detail in description. Default value is true.");
 }
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/connectors/ExternalDynamicSink.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/connectors/ExternalDynamicSink.java
index e916744..2f9f2ef 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/connectors/ExternalDynamicSink.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/connectors/ExternalDynamicSink.java
@@ -21,12 +21,12 @@ package org.apache.flink.table.planner.connectors;
 import org.apache.flink.annotation.Internal;
 import org.apache.flink.api.dag.Transformation;
 import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.transformations.OneInputTransformation;
 import org.apache.flink.table.api.DataTypes;
 import org.apache.flink.table.connector.ChangelogMode;
 import org.apache.flink.table.connector.sink.DynamicTableSink;
 import org.apache.flink.table.connector.sink.abilities.SupportsWritingMetadata;
 import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
 import org.apache.flink.table.runtime.operators.sink.OutputConversionOperator;
 import org.apache.flink.table.runtime.typeutils.ExternalTypeInfo;
 import org.apache.flink.table.types.DataType;
@@ -86,9 +86,10 @@ final class ExternalDynamicSink implements DynamicTableSink, SupportsWritingMeta
                         atomicFieldGetter = RowData.createFieldGetter(physicalType, 0);
                     }
 
-                    return new OneInputTransformation<>(
+                    return ExecNodeUtil.createOneInputTransformation(
                             input,
                             generateOperatorName(),
+                            generateOperatorDesc(),
                             new OutputConversionOperator(
                                     atomicFieldGetter,
                                     physicalConverter,
@@ -100,6 +101,10 @@ final class ExternalDynamicSink implements DynamicTableSink, SupportsWritingMeta
     }
 
     private String generateOperatorName() {
+        return "TableToDataSteam";
+    }
+
+    private String generateOperatorDesc() {
         return String.format(
                 "TableToDataSteam(type=%s, rowtime=%s)",
                 physicalDataType.toString(), consumeRowtimeMetadata);
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/connectors/ExternalDynamicSource.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/connectors/ExternalDynamicSource.java
index 78d9fb2..2645b27 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/connectors/ExternalDynamicSource.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/connectors/ExternalDynamicSource.java
@@ -21,7 +21,6 @@ package org.apache.flink.table.planner.connectors;
 import org.apache.flink.annotation.Internal;
 import org.apache.flink.api.dag.Transformation;
 import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.transformations.OneInputTransformation;
 import org.apache.flink.table.api.DataTypes;
 import org.apache.flink.table.catalog.ObjectIdentifier;
 import org.apache.flink.table.connector.ChangelogMode;
@@ -30,6 +29,7 @@ import org.apache.flink.table.connector.source.ScanTableSource;
 import org.apache.flink.table.connector.source.abilities.SupportsReadingMetadata;
 import org.apache.flink.table.connector.source.abilities.SupportsSourceWatermark;
 import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
 import org.apache.flink.table.runtime.operators.source.InputConversionOperator;
 import org.apache.flink.table.types.DataType;
 import org.apache.flink.types.RowKind;
@@ -104,9 +104,10 @@ final class ExternalDynamicSource<E>
         final Transformation<E> externalTransformation = dataStream.getTransformation();
 
         final Transformation<RowData> conversionTransformation =
-                new OneInputTransformation<>(
+                ExecNodeUtil.createOneInputTransformation(
                         externalTransformation,
                         generateOperatorName(),
+                        generateOperatorDesc(),
                         new InputConversionOperator<>(
                                 physicalConverter,
                                 !isTopLevelRecord,
@@ -120,6 +121,10 @@ final class ExternalDynamicSource<E>
     }
 
     private String generateOperatorName() {
+        return "DataSteamToTable";
+    }
+
+    private String generateOperatorDesc() {
         return String.format(
                 "DataSteamToTable(stream=%s, type=%s, rowtime=%s, watermark=%s)",
                 identifier.asSummaryString(),
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeBase.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeBase.java
index 1807eac..9781ab4 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeBase.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeBase.java
@@ -20,6 +20,9 @@ package org.apache.flink.table.planner.plan.nodes.exec;
 
 import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.api.dag.Transformation;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.table.api.TableConfig;
+import org.apache.flink.table.api.config.OptimizerConfigOptions;
 import org.apache.flink.table.delegation.Planner;
 import org.apache.flink.table.planner.delegation.PlannerBase;
 import org.apache.flink.table.planner.plan.nodes.exec.common.CommonExecExchange;
@@ -163,4 +166,42 @@ public abstract class ExecNodeBase<T> implements ExecNode<T> {
                                                         .getType()
                                                 == InputProperty.DistributionType.SINGLETON);
     }
+
+    public String getOperatorName(TableConfig config) {
+        return getOperatorName(config.getConfiguration());
+    }
+
+    public String getOperatorName(Configuration config) {
+        return getFormattedOperatorName(getDescription(), getSimplifiedName(), config);
+    }
+
+    @JsonIgnore
+    protected String getSimplifiedName() {
+        return getClass().getSimpleName().replace("StreamExec", "").replace("BatchExec", "");
+    }
+
+    protected String getOperatorDescription(TableConfig config) {
+        return getOperatorDescription(config.getConfiguration());
+    }
+
+    protected String getOperatorDescription(Configuration config) {
+        return getFormattedOperatorDescription(getDescription(), config);
+    }
+
+    protected String getFormattedOperatorDescription(String description, Configuration config) {
+        if (config.getBoolean(
+                OptimizerConfigOptions.TABLE_OPTIMIZER_SIMPLIFY_OPERATOR_NAME_ENABLED)) {
+            return String.format("[%d]:%s", id, description);
+        }
+        return description;
+    }
+
+    protected String getFormattedOperatorName(
+            String detailName, String simplifiedName, Configuration config) {
+        if (config.getBoolean(
+                OptimizerConfigOptions.TABLE_OPTIMIZER_SIMPLIFY_OPERATOR_NAME_ENABLED)) {
+            return String.format("%s[%d]", simplifiedName, id);
+        }
+        return detailName;
+    }
 }
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecBoundedStreamScan.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecBoundedStreamScan.java
index edc628c..7e89e09 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecBoundedStreamScan.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecBoundedStreamScan.java
@@ -19,6 +19,7 @@
 package org.apache.flink.table.planner.plan.nodes.exec.batch;
 
 import org.apache.flink.api.dag.Transformation;
+import org.apache.flink.configuration.Configuration;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.table.data.RowData;
 import org.apache.flink.table.planner.codegen.CodeGeneratorContext;
@@ -63,6 +64,7 @@ public class BatchExecBoundedStreamScan extends ExecNodeBase<RowData>
     @Override
     protected Transformation<RowData> translateToPlanInternal(PlannerBase planner) {
         final Transformation<?> sourceTransform = dataStream.getTransformation();
+        final Configuration config = planner.getTableConfig().getConfiguration();
         if (needInternalConversion()) {
             return ScanUtil.convertToInternalRow(
                     new CodeGeneratorContext(planner.getTableConfig()),
@@ -71,6 +73,9 @@ public class BatchExecBoundedStreamScan extends ExecNodeBase<RowData>
                     sourceType,
                     (RowType) getOutputType(),
                     qualifiedName,
+                    (detailName, simplifyName) ->
+                            getFormattedOperatorName(detailName, simplifyName, config),
+                    (description) -> getFormattedOperatorDescription(description, config),
                     JavaScalaConversionUtil.toScala(Optional.empty()),
                     "",
                     "");
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecHashAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecHashAggregate.java
index 16c13ca..95c3b1f 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecHashAggregate.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecHashAggregate.java
@@ -131,7 +131,8 @@ public class BatchExecHashAggregate extends ExecNodeBase<RowData>
 
         return ExecNodeUtil.createOneInputTransformation(
                 inputTransform,
-                getDescription(),
+                getOperatorName(planner.getTableConfig()),
+                getOperatorDescription(planner.getTableConfig()),
                 new CodeGenOperatorFactory<>(generatedOperator),
                 InternalTypeInfo.of(outputRowType),
                 inputTransform.getParallelism(),
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecHashJoin.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecHashJoin.java
index 3da4a67..36a7540 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecHashJoin.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecHashJoin.java
@@ -213,7 +213,8 @@ public class BatchExecHashJoin extends ExecNodeBase<RowData>
         return ExecNodeUtil.createTwoInputTransformation(
                 buildTransform,
                 probeTransform,
-                getDescription(),
+                getOperatorName(config),
+                getOperatorDescription(config),
                 operator,
                 InternalTypeInfo.of(getOutputType()),
                 probeTransform.getParallelism(),
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecHashWindowAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecHashWindowAggregate.java
index af02476..59bf088 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecHashWindowAggregate.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecHashWindowAggregate.java
@@ -145,7 +145,8 @@ public class BatchExecHashWindowAggregate extends ExecNodeBase<RowData>
                         .getBytes();
         return ExecNodeUtil.createOneInputTransformation(
                 inputTransform,
-                getDescription(),
+                getOperatorName(tableConfig),
+                getOperatorDescription(tableConfig),
                 new CodeGenOperatorFactory<>(generatedOperator),
                 InternalTypeInfo.of(getOutputType()),
                 inputTransform.getParallelism(),
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecLegacyTableSourceScan.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecLegacyTableSourceScan.java
index 167cba0..076382c 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecLegacyTableSourceScan.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecLegacyTableSourceScan.java
@@ -21,6 +21,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch;
 import org.apache.flink.api.common.io.InputFormat;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.dag.Transformation;
+import org.apache.flink.configuration.Configuration;
 import org.apache.flink.core.io.InputSplit;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.functions.source.InputFormatSourceFunction;
@@ -80,9 +81,10 @@ public class BatchExecLegacyTableSourceScan extends CommonExecLegacyTableSourceS
             // the produced type may not carry the correct precision user defined in DDL, because
             // it may be converted from legacy type. Fix precision using logical schema from DDL.
             // code generation requires the correct precision of input fields.
-            DataType fixedProducedDataType =
+            final DataType fixedProducedDataType =
                     TableSourceUtil.fixPrecisionForProducedDataType(
                             tableSource, (RowType) getOutputType());
+            final Configuration config = planner.getTableConfig().getConfiguration();
             return ScanUtil.convertToInternalRow(
                     new CodeGeneratorContext(planner.getTableConfig()),
                     (Transformation<Object>) sourceTransform,
@@ -90,6 +92,9 @@ public class BatchExecLegacyTableSourceScan extends CommonExecLegacyTableSourceS
                     fixedProducedDataType,
                     (RowType) getOutputType(),
                     qualifiedName,
+                    (detailName, simplifyName) ->
+                            getFormattedOperatorName(detailName, simplifyName, config),
+                    (description) -> getFormattedOperatorDescription(description, config),
                     JavaScalaConversionUtil.toScala(Optional.ofNullable(rowtimeExpression)),
                     "",
                     "");
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecLimit.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecLimit.java
index ab37a00..b1e3a43c 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecLimit.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecLimit.java
@@ -20,12 +20,12 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch;
 
 import org.apache.flink.api.dag.Transformation;
 import org.apache.flink.streaming.api.operators.SimpleOperatorFactory;
-import org.apache.flink.streaming.api.transformations.OneInputTransformation;
 import org.apache.flink.table.data.RowData;
 import org.apache.flink.table.planner.delegation.PlannerBase;
 import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
 import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
 import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
+import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
 import org.apache.flink.table.runtime.operators.sort.LimitOperator;
 import org.apache.flink.table.types.logical.LogicalType;
 
@@ -57,9 +57,10 @@ public class BatchExecLimit extends ExecNodeBase<RowData> implements BatchExecNo
         Transformation<RowData> inputTransform =
                 (Transformation<RowData>) getInputEdges().get(0).translateToPlan(planner);
         LimitOperator operator = new LimitOperator(isGlobal, limitStart, limitEnd);
-        return new OneInputTransformation<>(
+        return ExecNodeUtil.createOneInputTransformation(
                 inputTransform,
-                getDescription(),
+                getOperatorName(planner.getTableConfig()),
+                getOperatorDescription(planner.getTableConfig()),
                 SimpleOperatorFactory.of(operator),
                 inputTransform.getOutputType(),
                 inputTransform.getParallelism());
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecMultipleInput.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecMultipleInput.java
index b04e252..2536889 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecMultipleInput.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecMultipleInput.java
@@ -100,7 +100,7 @@ public class BatchExecMultipleInput extends ExecNodeBase<RowData>
 
         final MultipleInputTransformation<RowData> multipleInputTransform =
                 new MultipleInputTransformation<>(
-                        getDescription(),
+                        getOperatorName(planner.getTableConfig()),
                         new BatchMultipleInputStreamOperatorFactory(
                                 inputTransformAndInputSpecPairs.stream()
                                         .map(Pair::getValue)
@@ -109,6 +109,7 @@ public class BatchExecMultipleInput extends ExecNodeBase<RowData>
                                 generator.getTailWrapper()),
                         InternalTypeInfo.of(getOutputType()),
                         generator.getParallelism());
+        multipleInputTransform.setDescription(getOperatorDescription(planner.getTableConfig()));
         inputTransformAndInputSpecPairs.forEach(
                 input -> multipleInputTransform.addInput(input.getKey()));
 
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecNestedLoopJoin.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecNestedLoopJoin.java
index 22c97be..6595974 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecNestedLoopJoin.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecNestedLoopJoin.java
@@ -108,7 +108,8 @@ public class BatchExecNestedLoopJoin extends ExecNodeBase<RowData>
         return ExecNodeUtil.createTwoInputTransformation(
                 leftInputTransform,
                 rightInputTransform,
-                getDescription(),
+                getOperatorName(config),
+                getOperatorDescription(config),
                 operator,
                 InternalTypeInfo.of(getOutputType()),
                 parallelism,
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecOverAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecOverAggregate.java
index 50feefd..74143fb 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecOverAggregate.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecOverAggregate.java
@@ -165,7 +165,8 @@ public class BatchExecOverAggregate extends BatchExecOverAggregateBase {
         }
         return ExecNodeUtil.createOneInputTransformation(
                 inputTransform,
-                getDescription(),
+                getOperatorName(planner.getTableConfig()),
+                getOperatorDescription(planner.getTableConfig()),
                 SimpleOperatorFactory.of(operator),
                 InternalTypeInfo.of(getOutputType()),
                 inputTransform.getParallelism(),
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonGroupAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonGroupAggregate.java
index ab52ac4..0550d72 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonGroupAggregate.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonGroupAggregate.java
@@ -34,6 +34,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
 import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
 import org.apache.flink.table.planner.plan.nodes.exec.SingleTransformationTranslator;
 import org.apache.flink.table.planner.plan.nodes.exec.utils.CommonPythonUtil;
+import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
 import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
 import org.apache.flink.table.types.logical.RowType;
 
@@ -104,9 +105,10 @@ public class BatchExecPythonGroupAggregate extends ExecNodeBase<RowData>
                         outputRowType,
                         pythonUdafInputOffsets,
                         pythonFunctionInfos);
-        return new OneInputTransformation<>(
+        return ExecNodeUtil.createOneInputTransformation(
                 inputTransform,
-                getDescription(),
+                getOperatorName(config),
+                getOperatorDescription(config),
                 pythonOperator,
                 InternalTypeInfo.of(outputRowType),
                 inputTransform.getParallelism());
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonGroupWindowAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonGroupWindowAggregate.java
index 34cc289..4b3fa18 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonGroupWindowAggregate.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonGroupWindowAggregate.java
@@ -43,6 +43,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
 import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
 import org.apache.flink.table.planner.plan.nodes.exec.SingleTransformationTranslator;
 import org.apache.flink.table.planner.plan.nodes.exec.utils.CommonPythonUtil;
+import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
 import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
 import org.apache.flink.table.types.logical.RowType;
 
@@ -158,9 +159,10 @@ public class BatchExecPythonGroupWindowAggregate extends ExecNodeBase<RowData>
                         namePropertyTypeArray,
                         pythonUdafInputOffsets,
                         pythonFunctionInfos);
-        return new OneInputTransformation<>(
+        return ExecNodeUtil.createOneInputTransformation(
                 inputTransform,
-                getDescription(),
+                getOperatorName(config),
+                getOperatorDescription(config),
                 pythonOperator,
                 InternalTypeInfo.of(outputRowType),
                 inputTransform.getParallelism());
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonOverAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonOverAggregate.java
index 2be035f..3eec757 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonOverAggregate.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonOverAggregate.java
@@ -35,6 +35,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.spec.OverSpec;
 import org.apache.flink.table.planner.plan.nodes.exec.spec.PartitionSpec;
 import org.apache.flink.table.planner.plan.nodes.exec.spec.SortSpec;
 import org.apache.flink.table.planner.plan.nodes.exec.utils.CommonPythonUtil;
+import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
 import org.apache.flink.table.planner.plan.utils.OverAggregateUtil;
 import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
 import org.apache.flink.table.types.logical.RowType;
@@ -169,9 +170,10 @@ public class BatchExecPythonOverAggregate extends BatchExecOverAggregateBase {
                         isRangeWindows,
                         pythonUdafInputOffsets,
                         pythonFunctionInfos);
-        return new OneInputTransformation<>(
+        return ExecNodeUtil.createOneInputTransformation(
                 inputTransform,
-                getDescription(),
+                getOperatorName(config),
+                getOperatorDescription(config),
                 pythonOperator,
                 InternalTypeInfo.of(outputRowType),
                 inputTransform.getParallelism());
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecRank.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecRank.java
index 60cddf8..bbe5006 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecRank.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecRank.java
@@ -21,13 +21,13 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch;
 
 import org.apache.flink.api.dag.Transformation;
 import org.apache.flink.streaming.api.operators.SimpleOperatorFactory;
-import org.apache.flink.streaming.api.transformations.OneInputTransformation;
 import org.apache.flink.table.data.RowData;
 import org.apache.flink.table.planner.codegen.sort.ComparatorCodeGenerator;
 import org.apache.flink.table.planner.delegation.PlannerBase;
 import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
 import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
 import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
+import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
 import org.apache.flink.table.planner.plan.utils.SortUtil;
 import org.apache.flink.table.runtime.operators.sort.RankOperator;
 import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
@@ -93,9 +93,10 @@ public class BatchExecRank extends ExecNodeBase<RowData> implements BatchExecNod
                         rankEnd,
                         outputRankNumber);
 
-        return new OneInputTransformation<>(
+        return ExecNodeUtil.createOneInputTransformation(
                 inputTransform,
-                getDescription(),
+                getOperatorName(planner.getTableConfig()),
+                getOperatorDescription(planner.getTableConfig()),
                 SimpleOperatorFactory.of(operator),
                 InternalTypeInfo.of((RowType) getOutputType()),
                 inputTransform.getParallelism());
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSort.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSort.java
index 00e0d75..c793a38 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSort.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSort.java
@@ -75,7 +75,8 @@ public class BatchExecSort extends ExecNodeBase<RowData> implements BatchExecNod
                         .getBytes();
         return ExecNodeUtil.createOneInputTransformation(
                 inputTransform,
-                getDescription(),
+                getOperatorName(config),
+                getOperatorDescription(config),
                 SimpleOperatorFactory.of(operator),
                 InternalTypeInfo.of((RowType) getOutputType()),
                 inputTransform.getParallelism(),
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSortAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSortAggregate.java
index d96d8fa..6d47224 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSortAggregate.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSortAggregate.java
@@ -20,7 +20,6 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch;
 
 import org.apache.flink.api.dag.Transformation;
 import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
-import org.apache.flink.streaming.api.transformations.OneInputTransformation;
 import org.apache.flink.table.data.RowData;
 import org.apache.flink.table.planner.codegen.CodeGeneratorContext;
 import org.apache.flink.table.planner.codegen.agg.batch.AggWithoutKeysCodeGenerator;
@@ -31,6 +30,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
 import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
 import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
 import org.apache.flink.table.planner.plan.nodes.exec.SingleTransformationTranslator;
+import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
 import org.apache.flink.table.planner.plan.utils.AggregateInfoList;
 import org.apache.flink.table.planner.plan.utils.AggregateUtil;
 import org.apache.flink.table.planner.utils.JavaScalaConversionUtil;
@@ -118,9 +118,10 @@ public class BatchExecSortAggregate extends ExecNodeBase<RowData>
                             isFinal);
         }
 
-        return new OneInputTransformation<>(
+        return ExecNodeUtil.createOneInputTransformation(
                 inputTransform,
-                getDescription(),
+                getOperatorName(planner.getTableConfig()),
+                getOperatorDescription(planner.getTableConfig()),
                 new CodeGenOperatorFactory<>(generatedOperator),
                 InternalTypeInfo.of(outputRowType),
                 inputTransform.getParallelism());
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSortLimit.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSortLimit.java
index 2b16eb7..56cb80c 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSortLimit.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSortLimit.java
@@ -20,7 +20,6 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch;
 
 import org.apache.flink.api.dag.Transformation;
 import org.apache.flink.streaming.api.operators.SimpleOperatorFactory;
-import org.apache.flink.streaming.api.transformations.OneInputTransformation;
 import org.apache.flink.table.api.TableException;
 import org.apache.flink.table.data.RowData;
 import org.apache.flink.table.planner.codegen.sort.ComparatorCodeGenerator;
@@ -30,6 +29,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
 import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
 import org.apache.flink.table.planner.plan.nodes.exec.SingleTransformationTranslator;
 import org.apache.flink.table.planner.plan.nodes.exec.spec.SortSpec;
+import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
 import org.apache.flink.table.runtime.generated.GeneratedRecordComparator;
 import org.apache.flink.table.runtime.operators.sort.SortLimitOperator;
 import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
@@ -86,9 +86,10 @@ public class BatchExecSortLimit extends ExecNodeBase<RowData>
         SortLimitOperator operator =
                 new SortLimitOperator(isGlobal, limitStart, limitEnd, genComparator);
 
-        return new OneInputTransformation<>(
+        return ExecNodeUtil.createOneInputTransformation(
                 inputTransform,
-                getDescription(),
+                getOperatorName(planner.getTableConfig()),
+                getOperatorDescription(planner.getTableConfig()),
                 SimpleOperatorFactory.of(operator),
                 InternalTypeInfo.of(inputType),
                 inputTransform.getParallelism());
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSortMergeJoin.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSortMergeJoin.java
index b8c85cc..4bc8262 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSortMergeJoin.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSortMergeJoin.java
@@ -156,7 +156,8 @@ public class BatchExecSortMergeJoin extends ExecNodeBase<RowData>
         return ExecNodeUtil.createTwoInputTransformation(
                 leftInputTransform,
                 rightInputTransform,
-                getDescription(),
+                getOperatorName(planner.getTableConfig()),
+                getOperatorDescription(planner.getTableConfig()),
                 SimpleOperatorFactory.of(operator),
                 InternalTypeInfo.of(getOutputType()),
                 rightInputTransform.getParallelism(),
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSortWindowAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSortWindowAggregate.java
index 1d8e39f..87f5aeb 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSortWindowAggregate.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSortWindowAggregate.java
@@ -21,7 +21,6 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch;
 import org.apache.flink.api.dag.Transformation;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
-import org.apache.flink.streaming.api.transformations.OneInputTransformation;
 import org.apache.flink.table.api.TableConfig;
 import org.apache.flink.table.api.config.ExecutionConfigOptions;
 import org.apache.flink.table.data.RowData;
@@ -36,6 +35,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
 import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
 import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
 import org.apache.flink.table.planner.plan.nodes.exec.SingleTransformationTranslator;
+import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
 import org.apache.flink.table.planner.plan.utils.AggregateInfoList;
 import org.apache.flink.table.planner.plan.utils.AggregateUtil;
 import org.apache.flink.table.planner.utils.JavaScalaConversionUtil;
@@ -143,9 +143,10 @@ public class BatchExecSortWindowAggregate extends ExecNodeBase<RowData>
             generatedOperator = windowCodeGenerator.genWithKeys();
         }
 
-        return new OneInputTransformation<>(
+        return ExecNodeUtil.createOneInputTransformation(
                 inputTransform,
-                getDescription(),
+                getOperatorName(tableConfig),
+                getOperatorDescription(tableConfig),
                 new CodeGenOperatorFactory<>(generatedOperator),
                 InternalTypeInfo.of(getOutputType()),
                 inputTransform.getParallelism());
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecCalc.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecCalc.java
index 27b502a..73ba163 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecCalc.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecCalc.java
@@ -19,7 +19,6 @@
 package org.apache.flink.table.planner.plan.nodes.exec.common;
 
 import org.apache.flink.api.dag.Transformation;
-import org.apache.flink.streaming.api.transformations.OneInputTransformation;
 import org.apache.flink.table.data.RowData;
 import org.apache.flink.table.planner.codegen.CalcCodeGenerator;
 import org.apache.flink.table.planner.codegen.CodeGeneratorContext;
@@ -28,6 +27,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
 import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
 import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
 import org.apache.flink.table.planner.plan.nodes.exec.SingleTransformationTranslator;
+import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
 import org.apache.flink.table.planner.utils.JavaScalaConversionUtil;
 import org.apache.flink.table.runtime.operators.CodeGenOperatorFactory;
 import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
@@ -99,9 +99,10 @@ public abstract class CommonExecCalc extends ExecNodeBase<RowData>
                         JavaScalaConversionUtil.toScala(Optional.ofNullable(this.condition)),
                         retainHeader,
                         getClass().getSimpleName());
-        return new OneInputTransformation<>(
+        return ExecNodeUtil.createOneInputTransformation(
                 inputTransform,
-                getDescription(),
+                getOperatorName(planner.getTableConfig()),
+                getOperatorDescription(planner.getTableConfig()),
                 substituteStreamOperator,
                 InternalTypeInfo.of(getOutputType()),
                 inputTransform.getParallelism());
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecCorrelate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecCorrelate.java
index 614c5c8..69dcb45 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecCorrelate.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecCorrelate.java
@@ -108,6 +108,7 @@ public abstract class CommonExecCorrelate extends ExecNodeBase<RowData>
                 inputTransform.getParallelism(),
                 retainHeader,
                 getClass().getSimpleName(),
-                getDescription());
+                getOperatorName(planner.getTableConfig()),
+                getOperatorDescription(planner.getTableConfig()));
     }
 }
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecExpand.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecExpand.java
index 17ee1d0..de28b37 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecExpand.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecExpand.java
@@ -19,7 +19,6 @@
 package org.apache.flink.table.planner.plan.nodes.exec.common;
 
 import org.apache.flink.api.dag.Transformation;
-import org.apache.flink.streaming.api.transformations.OneInputTransformation;
 import org.apache.flink.table.data.RowData;
 import org.apache.flink.table.planner.codegen.CodeGeneratorContext;
 import org.apache.flink.table.planner.codegen.ExpandCodeGenerator;
@@ -29,6 +28,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
 import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
 import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
 import org.apache.flink.table.planner.plan.nodes.exec.SingleTransformationTranslator;
+import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
 import org.apache.flink.table.runtime.operators.CodeGenOperatorFactory;
 import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
 import org.apache.flink.table.types.logical.RowType;
@@ -87,9 +87,10 @@ public abstract class CommonExecExpand extends ExecNodeBase<RowData>
                         retainHeader,
                         getClass().getSimpleName());
 
-        return new OneInputTransformation<>(
+        return ExecNodeUtil.createOneInputTransformation(
                 inputTransform,
-                getDescription(),
+                getOperatorName(planner.getTableConfig()),
+                getOperatorDescription(planner.getTableConfig()),
                 operatorFactory,
                 InternalTypeInfo.of(getOutputType()),
                 inputTransform.getParallelism());
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecLegacySink.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecLegacySink.java
index ca29944..af89f0a 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecLegacySink.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecLegacySink.java
@@ -20,9 +20,9 @@ package org.apache.flink.table.planner.plan.nodes.exec.common;
 
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.dag.Transformation;
+import org.apache.flink.configuration.Configuration;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.datastream.DataStreamSink;
-import org.apache.flink.streaming.api.transformations.OneInputTransformation;
 import org.apache.flink.table.api.TableException;
 import org.apache.flink.table.data.RowData;
 import org.apache.flink.table.planner.codegen.CodeGenUtils;
@@ -34,6 +34,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
 import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
 import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
 import org.apache.flink.table.planner.plan.nodes.exec.MultipleTransformationTranslator;
+import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
 import org.apache.flink.table.planner.sinks.DataStreamTableSink;
 import org.apache.flink.table.planner.sinks.TableSinkUtils;
 import org.apache.flink.table.runtime.operators.CodeGenOperatorFactory;
@@ -195,9 +196,13 @@ public abstract class CommonExecLegacySink<T> extends ExecNodeBase<T>
                             withChangeFlag,
                             "SinkConversion",
                             rowtimeIndex);
-            return new OneInputTransformation<>(
+            final Configuration config = planner.getTableConfig().getConfiguration();
+            final String description =
+                    "SinkConversion To " + resultDataType.getConversionClass().getSimpleName();
+            return ExecNodeUtil.createOneInputTransformation(
                     inputTransform,
-                    "SinkConversionTo" + resultDataType.getConversionClass().getSimpleName(),
+                    getFormattedOperatorName(description, "SinkConversion", config),
+                    getFormattedOperatorDescription(description, config),
                     converterOperator,
                     outputTypeInfo,
                     inputTransform.getParallelism());
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecLookupJoin.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecLookupJoin.java
index 8f80151..bcb0379 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecLookupJoin.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecLookupJoin.java
@@ -29,7 +29,6 @@ import org.apache.flink.streaming.api.operators.ProcessOperator;
 import org.apache.flink.streaming.api.operators.SimpleOperatorFactory;
 import org.apache.flink.streaming.api.operators.StreamOperatorFactory;
 import org.apache.flink.streaming.api.operators.async.AsyncWaitOperatorFactory;
-import org.apache.flink.streaming.api.transformations.OneInputTransformation;
 import org.apache.flink.table.api.TableConfig;
 import org.apache.flink.table.api.TableException;
 import org.apache.flink.table.api.config.ExecutionConfigOptions;
@@ -52,6 +51,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
 import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
 import org.apache.flink.table.planner.plan.nodes.exec.SingleTransformationTranslator;
 import org.apache.flink.table.planner.plan.nodes.exec.spec.TemporalTableSourceSpec;
+import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
 import org.apache.flink.table.planner.plan.schema.LegacyTableSourceTable;
 import org.apache.flink.table.planner.plan.schema.TableSourceTable;
 import org.apache.flink.table.planner.plan.utils.LookupJoinUtil;
@@ -263,9 +263,10 @@ public abstract class CommonExecLookupJoin extends ExecNodeBase<RowData>
 
         Transformation<RowData> inputTransformation =
                 (Transformation<RowData>) inputEdge.translateToPlan(planner);
-        return new OneInputTransformation<>(
+        return ExecNodeUtil.createOneInputTransformation(
                 inputTransformation,
-                getDescription(),
+                getOperatorName(planner.getTableConfig()),
+                getOperatorDescription(planner.getTableConfig()),
                 operatorFactory,
                 InternalTypeInfo.of(resultRowType),
                 inputTransformation.getParallelism());
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecPythonCalc.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecPythonCalc.java
index ddd4645..3ac4271 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecPythonCalc.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecPythonCalc.java
@@ -35,6 +35,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
 import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
 import org.apache.flink.table.planner.plan.nodes.exec.SingleTransformationTranslator;
 import org.apache.flink.table.planner.plan.nodes.exec.utils.CommonPythonUtil;
+import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
 import org.apache.flink.table.planner.plan.utils.PythonUtil;
 import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
 import org.apache.flink.table.types.logical.LogicalType;
@@ -95,7 +96,7 @@ public abstract class CommonExecPythonCalc extends ExecNodeBase<RowData>
         final Configuration config =
                 CommonPythonUtil.getMergedConfig(planner.getExecEnv(), planner.getTableConfig());
         OneInputTransformation<RowData, RowData> ret =
-                createPythonOneInputTransformation(inputTransform, getDescription(), config);
+                createPythonOneInputTransformation(inputTransform, config);
         if (CommonPythonUtil.isPythonWorkerUsingManagedMemory(config)) {
             ret.declareManagedMemoryUseCaseAtSlotScope(ManagedMemoryUseCase.PYTHON);
         }
@@ -103,7 +104,7 @@ public abstract class CommonExecPythonCalc extends ExecNodeBase<RowData>
     }
 
     private OneInputTransformation<RowData, RowData> createPythonOneInputTransformation(
-            Transformation<RowData> inputTransform, String name, Configuration config) {
+            Transformation<RowData> inputTransform, Configuration config) {
         List<RexCall> pythonRexCalls =
                 projection.stream()
                         .filter(x -> x instanceof RexCall)
@@ -152,9 +153,10 @@ public abstract class CommonExecPythonCalc extends ExecNodeBase<RowData>
                                                 PythonUtil.containsPythonCall(
                                                         x, PythonFunctionKind.PANDAS)));
 
-        return new OneInputTransformation<>(
+        return ExecNodeUtil.createOneInputTransformation(
                 inputTransform,
-                name,
+                getOperatorName(config),
+                getOperatorDescription(config),
                 pythonOperator,
                 pythonOperatorResultTyeInfo,
                 inputTransform.getParallelism());
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecPythonCorrelate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecPythonCorrelate.java
index c990259..da4f0bd 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecPythonCorrelate.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecPythonCorrelate.java
@@ -34,6 +34,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
 import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
 import org.apache.flink.table.planner.plan.nodes.exec.SingleTransformationTranslator;
 import org.apache.flink.table.planner.plan.nodes.exec.utils.CommonPythonUtil;
+import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
 import org.apache.flink.table.runtime.operators.join.FlinkJoinType;
 import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
 import org.apache.flink.table.types.logical.RowType;
@@ -113,9 +114,10 @@ public abstract class CommonExecPythonCorrelate extends ExecNodeBase<RowData>
                         pythonOperatorOutputRowType,
                         pythonFunctionInfo,
                         pythonUdtfInputOffsets);
-        return new OneInputTransformation<>(
+        return ExecNodeUtil.createOneInputTransformation(
                 inputTransform,
-                getDescription(),
+                getOperatorName(config),
+                getOperatorDescription(config),
                 pythonOperator,
                 pythonOperatorOutputRowType,
                 inputTransform.getParallelism());
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecSink.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecSink.java
index 091c09a..cb73eaf 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecSink.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecSink.java
@@ -21,6 +21,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.common;
 import org.apache.flink.api.common.io.OutputFormat;
 import org.apache.flink.api.dag.Transformation;
 import org.apache.flink.api.java.typeutils.InputTypeConfigurable;
+import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.state.KeyGroupRangeAssignment;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
@@ -53,6 +54,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
 import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
 import org.apache.flink.table.planner.plan.nodes.exec.MultipleTransformationTranslator;
 import org.apache.flink.table.planner.plan.nodes.exec.spec.DynamicTableSinkSpec;
+import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
 import org.apache.flink.table.planner.plan.utils.KeySelectorUtil;
 import org.apache.flink.table.runtime.connector.sink.SinkRuntimeProviderContext;
 import org.apache.flink.table.runtime.generated.GeneratedRecordEqualiser;
@@ -109,6 +111,11 @@ public abstract class CommonExecSink extends ExecNodeBase<Object>
         this.isBounded = isBounded;
     }
 
+    @Override
+    public String getSimplifiedName() {
+        return tableSinkSpec.getObjectIdentifier().getObjectName();
+    }
+
     public DynamicTableSinkSpec getTableSinkSpec() {
         return tableSinkSpec;
     }
@@ -177,7 +184,8 @@ public abstract class CommonExecSink extends ExecNodeBase<Object>
                         planner.getExecEnv(),
                         runtimeProvider,
                         rowtimeFieldIndex,
-                        sinkParallelism);
+                        sinkParallelism,
+                        planner.getTableConfig().getConfiguration());
     }
 
     /**
@@ -221,10 +229,19 @@ public abstract class CommonExecSink extends ExecNodeBase<Object>
 
         ConstraintEnforcer constraintEnforcer = validatorBuilder.build();
         if (constraintEnforcer != null) {
-            return new OneInputTransformation<>(
+            final String operatorDesc =
+                    getFormattedOperatorDescription(
+                            constraintEnforcer.getOperatorName(), config.getConfiguration());
+            final String operatorName =
+                    getFormattedOperatorName(
+                            constraintEnforcer.getOperatorName(),
+                            "ConstraintEnforcer",
+                            config.getConfiguration());
+            return ExecNodeUtil.createOneInputTransformation(
                     inputTransform,
-                    constraintEnforcer.getOperatorName(),
-                    validatorBuilder.build(),
+                    operatorName,
+                    operatorDesc,
+                    constraintEnforcer,
                     getInputTypeInfo(),
                     inputTransform.getParallelism());
         } else {
@@ -348,10 +365,23 @@ public abstract class CommonExecSink extends ExecNodeBase<Object>
                                 tableConfig.getIdleStateRetention().toMillis()),
                         InternalSerializers.create(physicalRowType),
                         equaliser);
+        final String[] fieldNames = physicalRowType.getFieldNames().toArray(new String[0]);
+        final List<String> pkFieldNames =
+                Arrays.stream(primaryKeys)
+                        .mapToObj(idx -> fieldNames[idx])
+                        .collect(Collectors.toList());
+        final String operatorDesc =
+                getFormattedOperatorDescription(
+                        String.format("SinkMaterializer(pk=[%s])", String.join(", ", pkFieldNames)),
+                        tableConfig.getConfiguration());
+        final String operatorName =
+                getFormattedOperatorName(
+                        operatorDesc, "SinkMaterializer", tableConfig.getConfiguration());
         OneInputTransformation<RowData, RowData> materializeTransform =
-                new OneInputTransformation<>(
+                ExecNodeUtil.createOneInputTransformation(
                         inputTransform,
-                        "SinkMaterializer",
+                        operatorName,
+                        operatorDesc,
                         operator,
                         inputTransform.getOutputType(),
                         sinkParallelism);
@@ -368,10 +398,14 @@ public abstract class CommonExecSink extends ExecNodeBase<Object>
             StreamExecutionEnvironment env,
             SinkRuntimeProvider runtimeProvider,
             int rowtimeFieldIndex,
-            int sinkParallelism) {
+            int sinkParallelism,
+            Configuration config) {
+        String sinkName = getOperatorName(config);
+        String sinkDescription = getOperatorDescription(config);
         if (runtimeProvider instanceof DataStreamSinkProvider) {
             Transformation<RowData> sinkTransformation =
-                    applyRowtimeTransformation(inputTransform, rowtimeFieldIndex, sinkParallelism);
+                    applyRowtimeTransformation(
+                            inputTransform, rowtimeFieldIndex, sinkParallelism, config);
             final DataStream<RowData> dataStream = new DataStream<>(env, sinkTransformation);
             final DataStreamSinkProvider provider = (DataStreamSinkProvider) runtimeProvider;
             return provider.consumeDataStream(dataStream).getTransformation();
@@ -384,19 +418,35 @@ public abstract class CommonExecSink extends ExecNodeBase<Object>
             final SinkFunction<RowData> sinkFunction =
                     ((SinkFunctionProvider) runtimeProvider).createSinkFunction();
             return createSinkFunctionTransformation(
-                    sinkFunction, env, inputTransform, rowtimeFieldIndex, sinkParallelism);
+                    sinkFunction,
+                    env,
+                    inputTransform,
+                    rowtimeFieldIndex,
+                    sinkName,
+                    sinkDescription,
+                    sinkParallelism);
         } else if (runtimeProvider instanceof OutputFormatProvider) {
             OutputFormat<RowData> outputFormat =
                     ((OutputFormatProvider) runtimeProvider).createOutputFormat();
             final SinkFunction<RowData> sinkFunction = new OutputFormatSinkFunction<>(outputFormat);
             return createSinkFunctionTransformation(
-                    sinkFunction, env, inputTransform, rowtimeFieldIndex, sinkParallelism);
-        } else if (runtimeProvider instanceof SinkProvider) {
-            return new SinkTransformation<>(
-                    applyRowtimeTransformation(inputTransform, rowtimeFieldIndex, sinkParallelism),
-                    ((SinkProvider) runtimeProvider).createSink(),
-                    getDescription(),
+                    sinkFunction,
+                    env,
+                    inputTransform,
+                    rowtimeFieldIndex,
+                    sinkName,
+                    sinkDescription,
                     sinkParallelism);
+        } else if (runtimeProvider instanceof SinkProvider) {
+            Transformation<?> transformation =
+                    new SinkTransformation<>(
+                            applyRowtimeTransformation(
+                                    inputTransform, rowtimeFieldIndex, sinkParallelism, config),
+                            ((SinkProvider) runtimeProvider).createSink(),
+                            sinkName,
+                            sinkParallelism);
+            transformation.setDescription(sinkDescription);
+            return transformation;
         } else {
             throw new TableException("Unsupported sink runtime provider.");
         }
@@ -407,6 +457,8 @@ public abstract class CommonExecSink extends ExecNodeBase<Object>
             StreamExecutionEnvironment env,
             Transformation<RowData> inputTransformation,
             int rowtimeFieldIndex,
+            String sinkName,
+            String sinkDescription,
             int sinkParallelism) {
         final SinkOperator operator = new SinkOperator(env.clean(sinkFunction), rowtimeFieldIndex);
 
@@ -415,23 +467,35 @@ public abstract class CommonExecSink extends ExecNodeBase<Object>
                     .setInputType(getInputTypeInfo(), env.getConfig());
         }
 
-        return new LegacySinkTransformation<>(
-                inputTransformation,
-                getDescription(),
-                SimpleOperatorFactory.of(operator),
-                sinkParallelism);
+        final Transformation<?> transformation =
+                new LegacySinkTransformation<>(
+                        inputTransformation,
+                        sinkName,
+                        SimpleOperatorFactory.of(operator),
+                        sinkParallelism);
+        transformation.setDescription(sinkDescription);
+        return transformation;
     }
 
     private Transformation<RowData> applyRowtimeTransformation(
-            Transformation<RowData> inputTransform, int rowtimeFieldIndex, int sinkParallelism) {
+            Transformation<RowData> inputTransform,
+            int rowtimeFieldIndex,
+            int sinkParallelism,
+            Configuration config) {
         // Don't apply the transformation/operator if there is no rowtimeFieldIndex
         if (rowtimeFieldIndex == -1) {
             return inputTransform;
         }
-        return new OneInputTransformation<>(
+        final String description =
+                getFormattedOperatorDescription(
+                        String.format(
+                                "StreamRecordTimestampInserter(rowtime field: %s)",
+                                rowtimeFieldIndex),
+                        config);
+        return ExecNodeUtil.createOneInputTransformation(
                 inputTransform,
-                String.format(
-                        "StreamRecordTimestampInserter(rowtime field: %s)", rowtimeFieldIndex),
+                getFormattedOperatorName(description, "StreamRecordTimestampInserter", config),
+                description,
                 new StreamRecordTimestampInserter(rowtimeFieldIndex),
                 inputTransform.getOutputType(),
                 sinkParallelism);
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecTableSourceScan.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecTableSourceScan.java
index cee9709..713a522 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecTableSourceScan.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecTableSourceScan.java
@@ -69,6 +69,11 @@ public abstract class CommonExecTableSourceScan extends ExecNodeBase<RowData>
         this.tableSourceSpec = tableSourceSpec;
     }
 
+    @Override
+    public String getSimplifiedName() {
+        return tableSourceSpec.getObjectIdentifier().getObjectName();
+    }
+
     public DynamicTableSourceSpec getTableSourceSpec() {
         return tableSourceSpec;
     }
@@ -76,7 +81,7 @@ public abstract class CommonExecTableSourceScan extends ExecNodeBase<RowData>
     @Override
     protected Transformation<RowData> translateToPlanInternal(PlannerBase planner) {
         final StreamExecutionEnvironment env = planner.getExecEnv();
-        final String operatorName = getDescription();
+        final String operatorName = getOperatorName(planner.getTableConfig());
         final InternalTypeInfo<RowData> outputTypeInfo =
                 InternalTypeInfo.of((RowType) getOutputType());
         final ScanTableSource tableSource =
@@ -86,22 +91,34 @@ public abstract class CommonExecTableSourceScan extends ExecNodeBase<RowData>
         if (provider instanceof SourceFunctionProvider) {
             final SourceFunctionProvider sourceFunctionProvider = (SourceFunctionProvider) provider;
             final SourceFunction<RowData> function = sourceFunctionProvider.createSourceFunction();
-            return createSourceFunctionTransformation(
-                    env,
-                    function,
-                    sourceFunctionProvider.isBounded(),
-                    operatorName,
-                    outputTypeInfo);
+            final Transformation<RowData> transformation =
+                    createSourceFunctionTransformation(
+                            env,
+                            function,
+                            sourceFunctionProvider.isBounded(),
+                            operatorName,
+                            outputTypeInfo);
+            transformation.setDescription(getOperatorDescription(planner.getTableConfig()));
+            return transformation;
         } else if (provider instanceof InputFormatProvider) {
             final InputFormat<RowData, ?> inputFormat =
                     ((InputFormatProvider) provider).createInputFormat();
-            return createInputFormatTransformation(env, inputFormat, outputTypeInfo, operatorName);
+            final Transformation<RowData> transformation =
+                    createInputFormatTransformation(env, inputFormat, outputTypeInfo, operatorName);
+            transformation.setDescription(getOperatorDescription(planner.getTableConfig()));
+            return transformation;
         } else if (provider instanceof SourceProvider) {
-            Source<RowData, ?, ?> source = ((SourceProvider) provider).createSource();
+            final Source<RowData, ?, ?> source = ((SourceProvider) provider).createSource();
             // TODO: Push down watermark strategy to source scan
-            return env.fromSource(
-                            source, WatermarkStrategy.noWatermarks(), operatorName, outputTypeInfo)
-                    .getTransformation();
+            final Transformation<RowData> transformation =
+                    env.fromSource(
+                                    source,
+                                    WatermarkStrategy.noWatermarks(),
+                                    operatorName,
+                                    outputTypeInfo)
+                            .getTransformation();
+            transformation.setDescription(getOperatorDescription(planner.getTableConfig()));
+            return transformation;
         } else if (provider instanceof DataStreamScanProvider) {
             Transformation<RowData> transformation =
                     ((DataStreamScanProvider) provider).produceDataStream(env).getTransformation();
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecValues.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecValues.java
index 1106496..65f6869 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecValues.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecValues.java
@@ -64,7 +64,8 @@ public abstract class CommonExecValues extends ExecNodeBase<RowData>
                 planner.getExecEnv()
                         .createInput(inputFormat, inputFormat.getProducedType())
                         .getTransformation();
-        transformation.setName(getDescription());
+        transformation.setName(getOperatorName(planner.getTableConfig()));
+        transformation.setDescription(getOperatorDescription(planner.getTableConfig()));
         transformation.setParallelism(1);
         transformation.setMaxParallelism(1);
         return transformation;
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecWindowTableFunction.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecWindowTableFunction.java
index a6a05e4..1059d5f 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecWindowTableFunction.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecWindowTableFunction.java
@@ -19,7 +19,6 @@
 package org.apache.flink.table.planner.plan.nodes.exec.common;
 
 import org.apache.flink.api.dag.Transformation;
-import org.apache.flink.streaming.api.transformations.OneInputTransformation;
 import org.apache.flink.table.api.TableException;
 import org.apache.flink.table.data.RowData;
 import org.apache.flink.table.planner.delegation.PlannerBase;
@@ -31,6 +30,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
 import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
 import org.apache.flink.table.planner.plan.nodes.exec.SingleTransformationTranslator;
 import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecNode;
+import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
 import org.apache.flink.table.runtime.operators.window.TimeWindow;
 import org.apache.flink.table.runtime.operators.window.WindowTableFunctionOperator;
 import org.apache.flink.table.runtime.operators.window.assigners.WindowAssigner;
@@ -86,9 +86,10 @@ public abstract class CommonExecWindowTableFunction extends ExecNodeBase<RowData
         WindowTableFunctionOperator windowTableFunctionOperator =
                 new WindowTableFunctionOperator(
                         windowAssigner, windowingStrategy.getTimeAttributeIndex(), shiftTimeZone);
-        return new OneInputTransformation<>(
+        return ExecNodeUtil.createOneInputTransformation(
                 inputTransform,
-                getDescription(),
+                getOperatorName(planner.getTableConfig()),
+                getOperatorDescription(planner.getTableConfig()),
                 windowTableFunctionOperator,
                 InternalTypeInfo.of(getOutputType()),
                 inputTransform.getParallelism());
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecChangelogNormalize.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecChangelogNormalize.java
index bcc1cf3..2fd7a47 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecChangelogNormalize.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecChangelogNormalize.java
@@ -33,6 +33,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
 import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
 import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
 import org.apache.flink.table.planner.plan.nodes.exec.SingleTransformationTranslator;
+import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
 import org.apache.flink.table.planner.plan.utils.AggregateUtil;
 import org.apache.flink.table.planner.plan.utils.KeySelectorUtil;
 import org.apache.flink.table.runtime.generated.GeneratedRecordEqualiser;
@@ -146,9 +147,10 @@ public class StreamExecChangelogNormalize extends ExecNodeBase<RowData>
         }
 
         final OneInputTransformation<RowData, RowData> transform =
-                new OneInputTransformation<>(
+                ExecNodeUtil.createOneInputTransformation(
                         inputTransform,
-                        getDescription(),
+                        getOperatorName(tableConfig),
+                        getOperatorDescription(tableConfig),
                         operator,
                         rowTypeInfo,
                         inputTransform.getParallelism());
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecDataStreamScan.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecDataStreamScan.java
index 468ba1f..04fde22 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecDataStreamScan.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecDataStreamScan.java
@@ -19,6 +19,7 @@
 package org.apache.flink.table.planner.plan.nodes.exec.stream;
 
 import org.apache.flink.api.dag.Transformation;
+import org.apache.flink.configuration.Configuration;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.table.data.RowData;
 import org.apache.flink.table.planner.calcite.FlinkRelBuilder;
@@ -95,9 +96,10 @@ public class StreamExecDataStreamScan extends ExecNodeBase<RowData>
                 extractElement = "";
                 resetElement = "";
             }
-            CodeGeneratorContext ctx =
+            final CodeGeneratorContext ctx =
                     new CodeGeneratorContext(planner.getTableConfig())
                             .setOperatorBaseClass(TableStreamOperator.class);
+            final Configuration config = planner.getTableConfig().getConfiguration();
             transformation =
                     ScanUtil.convertToInternalRow(
                             ctx,
@@ -106,6 +108,9 @@ public class StreamExecDataStreamScan extends ExecNodeBase<RowData>
                             sourceType,
                             (RowType) getOutputType(),
                             qualifiedName,
+                            (detailName, simplifyName) ->
+                                    getFormattedOperatorName(detailName, simplifyName, config),
+                            (description) -> getFormattedOperatorDescription(description, config),
                             JavaScalaConversionUtil.toScala(rowtimeExpr),
                             extractElement,
                             resetElement);
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecDeduplicate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecDeduplicate.java
index 309fce0..dbf9767 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecDeduplicate.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecDeduplicate.java
@@ -36,6 +36,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
 import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
 import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
 import org.apache.flink.table.planner.plan.nodes.exec.SingleTransformationTranslator;
+import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
 import org.apache.flink.table.planner.plan.utils.KeySelectorUtil;
 import org.apache.flink.table.runtime.generated.GeneratedRecordEqualiser;
 import org.apache.flink.table.runtime.keyselector.RowDataKeySelector;
@@ -188,9 +189,10 @@ public class StreamExecDeduplicate extends ExecNodeBase<RowData>
         }
 
         final OneInputTransformation<RowData, RowData> transform =
-                new OneInputTransformation<>(
+                ExecNodeUtil.createOneInputTransformation(
                         inputTransform,
-                        getDescription(),
+                        getOperatorName(planner.getTableConfig()),
+                        getOperatorDescription(planner.getTableConfig()),
                         operator,
                         rowTypeInfo,
                         inputTransform.getParallelism());
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecDropUpdateBefore.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecDropUpdateBefore.java
index c8cf699..2f26635 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecDropUpdateBefore.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecDropUpdateBefore.java
@@ -20,13 +20,13 @@ package org.apache.flink.table.planner.plan.nodes.exec.stream;
 
 import org.apache.flink.api.dag.Transformation;
 import org.apache.flink.streaming.api.operators.StreamFilter;
-import org.apache.flink.streaming.api.transformations.OneInputTransformation;
 import org.apache.flink.table.data.RowData;
 import org.apache.flink.table.planner.delegation.PlannerBase;
 import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
 import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
 import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
 import org.apache.flink.table.planner.plan.nodes.exec.SingleTransformationTranslator;
+import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
 import org.apache.flink.table.runtime.operators.misc.DropUpdateBeforeFunction;
 import org.apache.flink.table.types.logical.RowType;
 
@@ -67,9 +67,10 @@ public class StreamExecDropUpdateBefore extends ExecNodeBase<RowData>
                 (Transformation<RowData>) getInputEdges().get(0).translateToPlan(planner);
         final StreamFilter<RowData> operator = new StreamFilter<>(new DropUpdateBeforeFunction());
 
-        return new OneInputTransformation<>(
+        return ExecNodeUtil.createOneInputTransformation(
                 inputTransform,
-                getDescription(),
+                getOperatorName(planner.getTableConfig()),
+                getOperatorDescription(planner.getTableConfig()),
                 operator,
                 inputTransform.getOutputType(),
                 inputTransform.getParallelism());
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGlobalGroupAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGlobalGroupAggregate.java
index 67ccfab..1e26443 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGlobalGroupAggregate.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGlobalGroupAggregate.java
@@ -34,6 +34,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
 import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
 import org.apache.flink.table.planner.plan.nodes.exec.serde.LogicalTypeJsonDeserializer;
 import org.apache.flink.table.planner.plan.nodes.exec.serde.LogicalTypeJsonSerializer;
+import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
 import org.apache.flink.table.planner.plan.utils.AggregateInfoList;
 import org.apache.flink.table.planner.plan.utils.AggregateUtil;
 import org.apache.flink.table.planner.plan.utils.KeySelectorUtil;
@@ -248,9 +249,10 @@ public class StreamExecGlobalGroupAggregate extends StreamExecAggregateBase {
 
         // partitioned aggregation
         final OneInputTransformation<RowData, RowData> transform =
-                new OneInputTransformation<>(
+                ExecNodeUtil.createOneInputTransformation(
                         inputTransform,
-                        getDescription(),
+                        getOperatorName(tableConfig),
+                        getOperatorDescription(tableConfig),
                         operator,
                         InternalTypeInfo.of(getOutputType()),
                         inputTransform.getParallelism());
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGlobalWindowAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGlobalWindowAggregate.java
index 41ab7a2..b6505cd 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGlobalWindowAggregate.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGlobalWindowAggregate.java
@@ -222,7 +222,8 @@ public class StreamExecGlobalWindowAggregate extends StreamExecWindowAggregateBa
         final OneInputTransformation<RowData, RowData> transform =
                 ExecNodeUtil.createOneInputTransformation(
                         inputTransform,
-                        getDescription(),
+                        getOperatorName(planner.getTableConfig()),
+                        getOperatorDescription(planner.getTableConfig()),
                         SimpleOperatorFactory.of(windowOperator),
                         InternalTypeInfo.of(getOutputType()),
                         inputTransform.getParallelism(),
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGroupAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGroupAggregate.java
index b672544..cac5b7d 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGroupAggregate.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGroupAggregate.java
@@ -32,6 +32,7 @@ import org.apache.flink.table.planner.delegation.PlannerBase;
 import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
 import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
 import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
+import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
 import org.apache.flink.table.planner.plan.utils.AggregateInfoList;
 import org.apache.flink.table.planner.plan.utils.AggregateUtil;
 import org.apache.flink.table.planner.plan.utils.KeySelectorUtil;
@@ -219,9 +220,10 @@ public class StreamExecGroupAggregate extends StreamExecAggregateBase {
 
         // partitioned aggregation
         final OneInputTransformation<RowData, RowData> transform =
-                new OneInputTransformation<>(
+                ExecNodeUtil.createOneInputTransformation(
                         inputTransform,
-                        getDescription(),
+                        getOperatorName(tableConfig),
+                        getOperatorDescription(tableConfig),
                         operator,
                         InternalTypeInfo.of(getOutputType()),
                         inputTransform.getParallelism());
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGroupTableAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGroupTableAggregate.java
index fdddf98..fce5dae 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGroupTableAggregate.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGroupTableAggregate.java
@@ -32,6 +32,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
 import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
 import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
 import org.apache.flink.table.planner.plan.nodes.exec.SingleTransformationTranslator;
+import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
 import org.apache.flink.table.planner.plan.utils.AggregateInfoList;
 import org.apache.flink.table.planner.plan.utils.AggregateUtil;
 import org.apache.flink.table.planner.plan.utils.KeySelectorUtil;
@@ -146,9 +147,10 @@ public class StreamExecGroupTableAggregate extends ExecNodeBase<RowData>
 
         // partitioned aggregation
         final OneInputTransformation<RowData, RowData> transform =
-                new OneInputTransformation<>(
+                ExecNodeUtil.createOneInputTransformation(
                         inputTransform,
-                        "GroupTableAggregate",
+                        getOperatorName(planner.getTableConfig()),
+                        getOperatorDescription(planner.getTableConfig()),
                         operator,
                         InternalTypeInfo.of(getOutputType()),
                         inputTransform.getParallelism());
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGroupWindowAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGroupWindowAggregate.java
index ef96b94..bb8cdb5 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGroupWindowAggregate.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGroupWindowAggregate.java
@@ -41,6 +41,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
 import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
 import org.apache.flink.table.planner.plan.nodes.exec.serde.LogicalWindowJsonDeserializer;
 import org.apache.flink.table.planner.plan.nodes.exec.serde.LogicalWindowJsonSerializer;
+import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
 import org.apache.flink.table.planner.plan.utils.AggregateInfoList;
 import org.apache.flink.table.planner.plan.utils.KeySelectorUtil;
 import org.apache.flink.table.planner.plan.utils.WindowEmitStrategy;
@@ -261,9 +262,10 @@ public class StreamExecGroupWindowAggregate extends StreamExecAggregateBase {
                         inputCountIndex);
 
         final OneInputTransformation<RowData, RowData> transform =
-                new OneInputTransformation<>(
+                ExecNodeUtil.createOneInputTransformation(
                         inputTransform,
-                        getDescription(),
+                        getOperatorName(planner.getTableConfig()),
+                        getOperatorDescription(planner.getTableConfig()),
                         operator,
                         InternalTypeInfo.of(getOutputType()),
                         inputTransform.getParallelism());
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecIncrementalGroupAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecIncrementalGroupAggregate.java
index 11c2c12..76fcc33 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecIncrementalGroupAggregate.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecIncrementalGroupAggregate.java
@@ -31,6 +31,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
 import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
 import org.apache.flink.table.planner.plan.nodes.exec.serde.LogicalTypeJsonDeserializer;
 import org.apache.flink.table.planner.plan.nodes.exec.serde.LogicalTypeJsonSerializer;
+import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
 import org.apache.flink.table.planner.plan.utils.AggregateInfoList;
 import org.apache.flink.table.planner.plan.utils.AggregateUtil;
 import org.apache.flink.table.planner.plan.utils.KeySelectorUtil;
@@ -208,9 +209,10 @@ public class StreamExecIncrementalGroupAggregate extends StreamExecAggregateBase
 
         // partitioned aggregation
         final OneInputTransformation<RowData, RowData> transform =
-                new OneInputTransformation<>(
+                ExecNodeUtil.createOneInputTransformation(
                         inputTransform,
-                        getDescription(),
+                        getOperatorName(planner.getTableConfig()),
+                        getOperatorDescription(planner.getTableConfig()),
                         operator,
                         InternalTypeInfo.of(getOutputType()),
                         inputTransform.getParallelism());
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecIntervalJoin.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecIntervalJoin.java
index 792a03f..b5a8163 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecIntervalJoin.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecIntervalJoin.java
@@ -23,12 +23,14 @@ import org.apache.flink.api.common.functions.MapFunction;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.dag.Transformation;
 import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
+import org.apache.flink.configuration.Configuration;
 import org.apache.flink.streaming.api.operators.StreamFlatMap;
 import org.apache.flink.streaming.api.operators.StreamMap;
 import org.apache.flink.streaming.api.operators.co.KeyedCoProcessOperator;
 import org.apache.flink.streaming.api.transformations.OneInputTransformation;
 import org.apache.flink.streaming.api.transformations.TwoInputTransformation;
 import org.apache.flink.streaming.api.transformations.UnionTransformation;
+import org.apache.flink.table.api.TableConfig;
 import org.apache.flink.table.api.TableException;
 import org.apache.flink.table.data.RowData;
 import org.apache.flink.table.planner.delegation.PlannerBase;
@@ -38,6 +40,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
 import org.apache.flink.table.planner.plan.nodes.exec.MultipleTransformationTranslator;
 import org.apache.flink.table.planner.plan.nodes.exec.spec.IntervalJoinSpec;
 import org.apache.flink.table.planner.plan.nodes.exec.spec.JoinSpec;
+import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
 import org.apache.flink.table.planner.plan.utils.JoinUtil;
 import org.apache.flink.table.planner.plan.utils.KeySelectorUtil;
 import org.apache.flink.table.runtime.generated.GeneratedJoinCondition;
@@ -134,7 +137,8 @@ public class StreamExecIntervalJoin extends ExecNodeBase<RowData>
                             rightInputTransform,
                             leftRowType.getFieldCount(),
                             rightRowType.getFieldCount(),
-                            returnTypeInfo);
+                            returnTypeInfo,
+                            planner.getTableConfig().getConfiguration());
                 } else {
                     GeneratedJoinCondition joinCondition =
                             JoinUtil.generateConditionFunction(
@@ -152,7 +156,8 @@ public class StreamExecIntervalJoin extends ExecNodeBase<RowData>
                                         returnTypeInfo,
                                         joinFunction,
                                         joinSpec,
-                                        windowBounds);
+                                        windowBounds,
+                                        planner.getTableConfig());
                     } else {
                         transform =
                                 createProcTimeJoin(
@@ -161,7 +166,8 @@ public class StreamExecIntervalJoin extends ExecNodeBase<RowData>
                                         returnTypeInfo,
                                         joinFunction,
                                         joinSpec,
-                                        windowBounds);
+                                        windowBounds,
+                                        planner.getTableConfig());
                     }
 
                     if (inputsContainSingleton()) {
@@ -257,14 +263,14 @@ public class StreamExecIntervalJoin extends ExecNodeBase<RowData>
         }
     }
 
-    @SuppressWarnings("unchecked")
     private Transformation<RowData> createNegativeWindowSizeJoin(
             JoinSpec joinSpec,
             Transformation<RowData> leftInputTransform,
             Transformation<RowData> rightInputTransform,
             int leftArity,
             int rightArity,
-            InternalTypeInfo<RowData> returnTypeInfo) {
+            InternalTypeInfo<RowData> returnTypeInfo,
+            Configuration config) {
         // We filter all records instead of adding an empty source to preserve the watermarks.
         FilterAllFlatMapFunction allFilter = new FilterAllFlatMapFunction(returnTypeInfo);
 
@@ -280,34 +286,53 @@ public class StreamExecIntervalJoin extends ExecNodeBase<RowData>
         OneInputTransformation<RowData, RowData> filterAllLeftStream =
                 new OneInputTransformation<>(
                         leftInputTransform,
-                        "filter all left input transformation",
+                        "FilterLeft",
                         new StreamFlatMap<>(allFilter),
                         returnTypeInfo,
                         leftParallelism);
+        filterAllLeftStream.setDescription(
+                getFormattedOperatorDescription("filter all left input transformation", config));
+        filterAllLeftStream.setName(
+                getFormattedOperatorName(
+                        filterAllLeftStream.getDescription(), "FilterLeft", config));
 
         OneInputTransformation<RowData, RowData> filterAllRightStream =
                 new OneInputTransformation<>(
                         rightInputTransform,
-                        "filter all right input transformation",
+                        "FilterRight",
                         new StreamFlatMap<>(allFilter),
                         returnTypeInfo,
                         rightParallelism);
+        filterAllRightStream.setDescription(
+                getFormattedOperatorDescription("filter all right input transformation", config));
+        filterAllRightStream.setName(
+                getFormattedOperatorName(
+                        filterAllRightStream.getDescription(), "FilterRight", config));
 
         OneInputTransformation<RowData, RowData> padLeftStream =
                 new OneInputTransformation<>(
                         leftInputTransform,
-                        "pad left input transformation",
+                        "PadLeft",
                         new StreamMap<>(leftPadder),
                         returnTypeInfo,
                         leftParallelism);
+        padLeftStream.setDescription(
+                getFormattedOperatorDescription("pad left input transformation", config));
+        padLeftStream.setName(
+                getFormattedOperatorName(padLeftStream.getDescription(), "PadLeft", config));
 
         OneInputTransformation<RowData, RowData> padRightStream =
                 new OneInputTransformation<>(
                         rightInputTransform,
-                        "pad right input transformation",
+                        "PadRight",
                         new StreamMap<>(rightPadder),
                         returnTypeInfo,
                         rightParallelism);
+        padRightStream.setDescription(
+                getFormattedOperatorDescription("pad right input transformation", config));
+        padRightStream.setName(
+                getFormattedOperatorName(padRightStream.getDescription(), "PadRight", config));
+
         switch (joinSpec.getJoinType()) {
             case INNER:
                 return new UnionTransformation<>(
@@ -331,7 +356,8 @@ public class StreamExecIntervalJoin extends ExecNodeBase<RowData>
             InternalTypeInfo<RowData> returnTypeInfo,
             IntervalJoinFunction joinFunction,
             JoinSpec joinSpec,
-            IntervalJoinSpec.WindowBounds windowBounds) {
+            IntervalJoinSpec.WindowBounds windowBounds,
+            TableConfig config) {
         InternalTypeInfo<RowData> leftTypeInfo =
                 (InternalTypeInfo<RowData>) leftInputTransform.getOutputType();
         InternalTypeInfo<RowData> rightTypeInfo =
@@ -345,10 +371,11 @@ public class StreamExecIntervalJoin extends ExecNodeBase<RowData>
                         rightTypeInfo,
                         joinFunction);
 
-        return new TwoInputTransformation<>(
+        return ExecNodeUtil.createTwoInputTransformation(
                 leftInputTransform,
                 rightInputTransform,
-                getDescription(),
+                getOperatorName(config),
+                getOperatorDescription(config),
                 new KeyedCoProcessOperator<>(procJoinFunc),
                 returnTypeInfo,
                 leftInputTransform.getParallelism());
@@ -360,7 +387,8 @@ public class StreamExecIntervalJoin extends ExecNodeBase<RowData>
             InternalTypeInfo<RowData> returnTypeInfo,
             IntervalJoinFunction joinFunction,
             JoinSpec joinSpec,
-            IntervalJoinSpec.WindowBounds windowBounds) {
+            IntervalJoinSpec.WindowBounds windowBounds,
+            TableConfig config) {
 
         InternalTypeInfo<RowData> leftTypeInfo =
                 (InternalTypeInfo<RowData>) leftInputTransform.getOutputType();
@@ -378,10 +406,11 @@ public class StreamExecIntervalJoin extends ExecNodeBase<RowData>
                         windowBounds.getLeftTimeIdx(),
                         windowBounds.getRightTimeIdx());
 
-        return new TwoInputTransformation<>(
+        return ExecNodeUtil.createTwoInputTransformation(
                 leftInputTransform,
                 rightInputTransform,
-                getDescription(),
+                getOperatorName(config),
+                getOperatorDescription(config),
                 new KeyedCoProcessOperatorWithWatermarkDelay<>(
                         rowJoinFunc, rowJoinFunc.getMaxOutputDelay()),
                 returnTypeInfo,
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecJoin.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecJoin.java
index c767bc3..c6461ae 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecJoin.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecJoin.java
@@ -29,6 +29,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
 import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
 import org.apache.flink.table.planner.plan.nodes.exec.SingleTransformationTranslator;
 import org.apache.flink.table.planner.plan.nodes.exec.spec.JoinSpec;
+import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
 import org.apache.flink.table.planner.plan.utils.JoinUtil;
 import org.apache.flink.table.planner.plan.utils.KeySelectorUtil;
 import org.apache.flink.table.runtime.generated.GeneratedJoinCondition;
@@ -171,10 +172,11 @@ public class StreamExecJoin extends ExecNodeBase<RowData>
 
         final RowType returnType = (RowType) getOutputType();
         final TwoInputTransformation<RowData, RowData, RowData> transform =
-                new TwoInputTransformation<>(
+                ExecNodeUtil.createTwoInputTransformation(
                         leftTransform,
                         rightTransform,
-                        getDescription(),
+                        getOperatorName(tableConfig),
+                        getOperatorDescription(tableConfig),
                         operator,
                         InternalTypeInfo.of(returnType),
                         leftTransform.getParallelism());
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLegacyTableSourceScan.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLegacyTableSourceScan.java
index a64aa3f..50b87ab 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLegacyTableSourceScan.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLegacyTableSourceScan.java
@@ -21,6 +21,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.stream;
 import org.apache.flink.api.common.io.InputFormat;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.dag.Transformation;
+import org.apache.flink.configuration.Configuration;
 import org.apache.flink.core.io.InputSplit;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
@@ -95,14 +96,15 @@ public class StreamExecLegacyTableSourceScan extends CommonExecLegacyTableSource
                 resetElement = "";
             }
 
-            CodeGeneratorContext ctx =
+            final CodeGeneratorContext ctx =
                     new CodeGeneratorContext(planner.getTableConfig())
                             .setOperatorBaseClass(TableStreamOperator.class);
             // the produced type may not carry the correct precision user defined in DDL, because
             // it may be converted from legacy type. Fix precision using logical schema from DDL.
             // Code generation requires the correct precision of input fields.
-            DataType fixedProducedDataType =
+            final DataType fixedProducedDataType =
                     TableSourceUtil.fixPrecisionForProducedDataType(tableSource, outputType);
+            final Configuration config = planner.getTableConfig().getConfiguration();
             transformation =
                     ScanUtil.convertToInternalRow(
                             ctx,
@@ -111,6 +113,9 @@ public class StreamExecLegacyTableSourceScan extends CommonExecLegacyTableSource
                             fixedProducedDataType,
                             outputType,
                             qualifiedName,
+                            (detailName, simplifyName) ->
+                                    getFormattedOperatorName(detailName, simplifyName, config),
+                            (description) -> getFormattedOperatorDescription(description, config),
                             JavaScalaConversionUtil.toScala(Optional.ofNullable(rowtimeExpression)),
                             extractElement,
                             resetElement);
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLocalGroupAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLocalGroupAggregate.java
index d711b95..26ddb89 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLocalGroupAggregate.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLocalGroupAggregate.java
@@ -19,7 +19,6 @@
 package org.apache.flink.table.planner.plan.nodes.exec.stream;
 
 import org.apache.flink.api.dag.Transformation;
-import org.apache.flink.streaming.api.transformations.OneInputTransformation;
 import org.apache.flink.table.data.RowData;
 import org.apache.flink.table.planner.codegen.CodeGeneratorContext;
 import org.apache.flink.table.planner.codegen.agg.AggsHandlerCodeGenerator;
@@ -27,6 +26,7 @@ import org.apache.flink.table.planner.delegation.PlannerBase;
 import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
 import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
 import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
+import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
 import org.apache.flink.table.planner.plan.utils.AggregateInfoList;
 import org.apache.flink.table.planner.plan.utils.AggregateUtil;
 import org.apache.flink.table.planner.plan.utils.KeySelectorUtil;
@@ -147,9 +147,10 @@ public class StreamExecLocalGroupAggregate extends StreamExecAggregateBase {
                         AggregateUtil.createMiniBatchTrigger(planner.getTableConfig()),
                         selector);
 
-        return new OneInputTransformation<>(
+        return ExecNodeUtil.createOneInputTransformation(
                 inputTransform,
-                getDescription(),
+                getOperatorName(planner.getTableConfig()),
+                getOperatorDescription(planner.getTableConfig()),
                 operator,
                 InternalTypeInfo.of(getOutputType()),
                 inputTransform.getParallelism());
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLocalWindowAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLocalWindowAggregate.java
index e58e7f1..67b8803 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLocalWindowAggregate.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLocalWindowAggregate.java
@@ -156,7 +156,8 @@ public class StreamExecLocalWindowAggregate extends StreamExecWindowAggregateBas
 
         return ExecNodeUtil.createOneInputTransformation(
                 inputTransform,
-                getDescription(),
+                getOperatorName(planner.getTableConfig()),
+                getOperatorDescription(planner.getTableConfig()),
                 SimpleOperatorFactory.of(localAggOperator),
                 InternalTypeInfo.of(getOutputType()),
                 inputTransform.getParallelism(),
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecMatch.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecMatch.java
index e246c7a..0e529dc 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecMatch.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecMatch.java
@@ -47,6 +47,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
 import org.apache.flink.table.planner.plan.nodes.exec.MultipleTransformationTranslator;
 import org.apache.flink.table.planner.plan.nodes.exec.spec.MatchSpec;
 import org.apache.flink.table.planner.plan.nodes.exec.spec.SortSpec;
+import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
 import org.apache.flink.table.planner.plan.utils.KeySelectorUtil;
 import org.apache.flink.table.planner.plan.utils.RexDefaultVisitor;
 import org.apache.flink.table.planner.utils.JavaScalaConversionUtil;
@@ -192,9 +193,10 @@ public class StreamExecMatch extends ExecNodeBase<RowData>
                         patternProcessFunction,
                         null);
         final OneInputTransformation<RowData, RowData> transform =
-                new OneInputTransformation<>(
+                ExecNodeUtil.createOneInputTransformation(
                         timestampedInputTransform,
-                        getDescription(),
+                        getOperatorName(config),
+                        getOperatorDescription(config),
                         operator,
                         InternalTypeInfo.of(getOutputType()),
                         timestampedInputTransform.getParallelism());
@@ -256,8 +258,9 @@ public class StreamExecMatch extends ExecNodeBase<RowData>
             // copy the rowtime field into the StreamRecord timestamp field
             int precision = getPrecision(timeOrderFieldType);
             Transformation<RowData> transform =
-                    new OneInputTransformation<>(
+                    ExecNodeUtil.createOneInputTransformation(
                             inputTransform,
+                            "StreamRecordTimestampInserter",
                             String.format(
                                     "StreamRecordTimestampInserter(rowtime field: %s)",
                                     timeOrderFieldIdx),
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecMiniBatchAssigner.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecMiniBatchAssigner.java
index bcb51db..cbaf8cf 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecMiniBatchAssigner.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecMiniBatchAssigner.java
@@ -20,7 +20,6 @@ package org.apache.flink.table.planner.plan.nodes.exec.stream;
 
 import org.apache.flink.api.dag.Transformation;
 import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
-import org.apache.flink.streaming.api.transformations.OneInputTransformation;
 import org.apache.flink.table.api.TableException;
 import org.apache.flink.table.data.RowData;
 import org.apache.flink.table.planner.delegation.PlannerBase;
@@ -28,6 +27,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
 import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
 import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
 import org.apache.flink.table.planner.plan.nodes.exec.SingleTransformationTranslator;
+import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
 import org.apache.flink.table.planner.plan.trait.MiniBatchInterval;
 import org.apache.flink.table.planner.plan.trait.MiniBatchMode;
 import org.apache.flink.table.runtime.operators.wmassigners.ProcTimeMiniBatchAssignerOperator;
@@ -102,9 +102,10 @@ public class StreamExecMiniBatchAssigner extends ExecNodeBase<RowData>
                             miniBatchInterval.getMode()));
         }
 
-        return new OneInputTransformation<>(
+        return ExecNodeUtil.createOneInputTransformation(
                 inputTransform,
-                getDescription(),
+                getOperatorName(planner.getTableConfig()),
+                getOperatorDescription(planner.getTableConfig()),
                 operator,
                 InternalTypeInfo.of(getOutputType()),
                 inputTransform.getParallelism());
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecOverAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecOverAggregate.java
index d0fd7f3..5d984fb 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecOverAggregate.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecOverAggregate.java
@@ -35,6 +35,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
 import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
 import org.apache.flink.table.planner.plan.nodes.exec.SingleTransformationTranslator;
 import org.apache.flink.table.planner.plan.nodes.exec.spec.OverSpec;
+import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
 import org.apache.flink.table.planner.plan.utils.AggregateInfoList;
 import org.apache.flink.table.planner.plan.utils.AggregateUtil;
 import org.apache.flink.table.planner.plan.utils.KeySelectorUtil;
@@ -219,9 +220,10 @@ public class StreamExecOverAggregate extends ExecNodeBase<RowData>
                 new KeyedProcessOperator<>(overProcessFunction);
 
         OneInputTransformation<RowData, RowData> transform =
-                new OneInputTransformation<>(
+                ExecNodeUtil.createOneInputTransformation(
                         inputTransform,
-                        getDescription(),
+                        getOperatorName(tableConfig),
+                        getOperatorDescription(tableConfig),
                         operator,
                         InternalTypeInfo.of(getOutputType()),
                         inputTransform.getParallelism());
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonGroupAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonGroupAggregate.java
index 6a8a25a..799eae4 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonGroupAggregate.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonGroupAggregate.java
@@ -33,6 +33,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
 import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
 import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
 import org.apache.flink.table.planner.plan.nodes.exec.utils.CommonPythonUtil;
+import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
 import org.apache.flink.table.planner.plan.utils.AggregateInfoList;
 import org.apache.flink.table.planner.plan.utils.AggregateUtil;
 import org.apache.flink.table.planner.plan.utils.KeySelectorUtil;
@@ -166,9 +167,10 @@ public class StreamExecPythonGroupAggregate extends StreamExecAggregateBase {
                         countStarInserted);
         // partitioned aggregation
         OneInputTransformation<RowData, RowData> transform =
-                new OneInputTransformation<>(
+                ExecNodeUtil.createOneInputTransformation(
                         inputTransform,
-                        getDescription(),
+                        getOperatorName(config),
+                        getOperatorDescription(config),
                         operator,
                         InternalTypeInfo.of(getOutputType()),
                         inputTransform.getParallelism());
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonGroupTableAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonGroupTableAggregate.java
index 584994d..0ca93cc 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonGroupTableAggregate.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonGroupTableAggregate.java
@@ -35,6 +35,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
 import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
 import org.apache.flink.table.planner.plan.nodes.exec.SingleTransformationTranslator;
 import org.apache.flink.table.planner.plan.nodes.exec.utils.CommonPythonUtil;
+import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
 import org.apache.flink.table.planner.plan.utils.AggregateInfoList;
 import org.apache.flink.table.planner.plan.utils.AggregateUtil;
 import org.apache.flink.table.planner.plan.utils.KeySelectorUtil;
@@ -130,9 +131,10 @@ public class StreamExecPythonGroupTableAggregate extends ExecNodeBase<RowData>
                         inputCountIndex);
 
         OneInputTransformation<RowData, RowData> transform =
-                new OneInputTransformation<>(
+                ExecNodeUtil.createOneInputTransformation(
                         inputTransform,
-                        getDescription(),
+                        getOperatorName(config),
+                        getOperatorDescription(config),
                         pythonOperator,
                         InternalTypeInfo.of(getOutputType()),
                         inputTransform.getParallelism());
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonGroupWindowAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonGroupWindowAggregate.java
index c8a44a2..a609bd4 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonGroupWindowAggregate.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonGroupWindowAggregate.java
@@ -45,6 +45,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
 import org.apache.flink.table.planner.plan.nodes.exec.serde.LogicalWindowJsonDeserializer;
 import org.apache.flink.table.planner.plan.nodes.exec.serde.LogicalWindowJsonSerializer;
 import org.apache.flink.table.planner.plan.nodes.exec.utils.CommonPythonUtil;
+import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
 import org.apache.flink.table.planner.plan.utils.AggregateInfoList;
 import org.apache.flink.table.planner.plan.utils.KeySelectorUtil;
 import org.apache.flink.table.planner.plan.utils.PythonUtil;
@@ -377,9 +378,10 @@ public class StreamExecPythonGroupWindowAggregate extends StreamExecAggregateBas
                         pythonUdafInputOffsets,
                         pythonFunctionInfos,
                         shiftTimeZone);
-        return new OneInputTransformation<>(
+        return ExecNodeUtil.createOneInputTransformation(
                 inputTransform,
-                getDescription(),
+                getOperatorName(config),
+                getOperatorDescription(config),
                 pythonOperator,
                 InternalTypeInfo.of(outputRowType),
                 inputTransform.getParallelism());
@@ -418,9 +420,10 @@ public class StreamExecPythonGroupWindowAggregate extends StreamExecAggregateBas
                         allowance,
                         shiftTimeZone);
 
-        return new OneInputTransformation<>(
+        return ExecNodeUtil.createOneInputTransformation(
                 inputTransform,
-                getDescription(),
+                getOperatorName(config),
+                getOperatorDescription(config),
                 pythonOperator,
                 InternalTypeInfo.of(outputRowType),
                 inputTransform.getParallelism());
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonOverAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonOverAggregate.java
index 5f9bd11..e537497 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonOverAggregate.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonOverAggregate.java
@@ -36,6 +36,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
 import org.apache.flink.table.planner.plan.nodes.exec.SingleTransformationTranslator;
 import org.apache.flink.table.planner.plan.nodes.exec.spec.OverSpec;
 import org.apache.flink.table.planner.plan.nodes.exec.utils.CommonPythonUtil;
+import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
 import org.apache.flink.table.planner.plan.utils.KeySelectorUtil;
 import org.apache.flink.table.planner.plan.utils.OverAggregateUtil;
 import org.apache.flink.table.runtime.keyselector.RowDataKeySelector;
@@ -228,9 +229,10 @@ public class StreamExecPythonOverAggregate extends ExecNodeBase<RowData>
                         minIdleStateRetentionTime,
                         maxIdleStateRetentionTime);
 
-        return new OneInputTransformation<>(
+        return ExecNodeUtil.createOneInputTransformation(
                 inputTransform,
-                getDescription(),
+                getOperatorName(config),
+                getOperatorDescription(config),
                 pythonOperator,
                 InternalTypeInfo.of(outputRowType),
                 inputTransform.getParallelism());
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecRank.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecRank.java
index b3261d8..40ad72b 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecRank.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecRank.java
@@ -38,6 +38,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
 import org.apache.flink.table.planner.plan.nodes.exec.SingleTransformationTranslator;
 import org.apache.flink.table.planner.plan.nodes.exec.spec.PartitionSpec;
 import org.apache.flink.table.planner.plan.nodes.exec.spec.SortSpec;
+import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
 import org.apache.flink.table.planner.plan.utils.KeySelectorUtil;
 import org.apache.flink.table.planner.plan.utils.RankProcessStrategy;
 import org.apache.flink.table.planner.plan.utils.RankUtil;
@@ -320,9 +321,10 @@ public class StreamExecRank extends ExecNodeBase<RowData>
         processFunction.setKeyContext(operator);
 
         OneInputTransformation<RowData, RowData> transform =
-                new OneInputTransformation<>(
+                ExecNodeUtil.createOneInputTransformation(
                         inputTransform,
-                        getDescription(),
+                        getOperatorName(tableConfig),
+                        getOperatorDescription(tableConfig),
                         operator,
                         InternalTypeInfo.of((RowType) getOutputType()),
                         inputTransform.getParallelism());
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecSort.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecSort.java
index a7f86e3..3ff0a5b 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecSort.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecSort.java
@@ -22,7 +22,6 @@ import org.apache.flink.annotation.Experimental;
 import org.apache.flink.api.dag.Transformation;
 import org.apache.flink.configuration.ConfigOption;
 import org.apache.flink.configuration.ConfigOptions;
-import org.apache.flink.streaming.api.transformations.OneInputTransformation;
 import org.apache.flink.table.api.TableConfig;
 import org.apache.flink.table.api.TableException;
 import org.apache.flink.table.data.RowData;
@@ -32,6 +31,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
 import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
 import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
 import org.apache.flink.table.planner.plan.nodes.exec.spec.SortSpec;
+import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
 import org.apache.flink.table.runtime.generated.GeneratedRecordComparator;
 import org.apache.flink.table.runtime.operators.sort.StreamSortOperator;
 import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
@@ -86,9 +86,10 @@ public class StreamExecSort extends ExecNodeBase<RowData> implements StreamExecN
         Transformation<RowData> inputTransform =
                 (Transformation<RowData>) inputEdge.translateToPlan(planner);
 
-        return new OneInputTransformation<>(
+        return ExecNodeUtil.createOneInputTransformation(
                 inputTransform,
-                getDescription(),
+                getOperatorName(config),
+                getOperatorDescription(config),
                 sortOperator,
                 InternalTypeInfo.of(inputType),
                 inputTransform.getParallelism());
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecTemporalJoin.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecTemporalJoin.java
index 7fa736e..f244522 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecTemporalJoin.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecTemporalJoin.java
@@ -36,6 +36,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
 import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
 import org.apache.flink.table.planner.plan.nodes.exec.SingleTransformationTranslator;
 import org.apache.flink.table.planner.plan.nodes.exec.spec.JoinSpec;
+import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
 import org.apache.flink.table.planner.plan.utils.JoinUtil;
 import org.apache.flink.table.planner.plan.utils.KeySelectorUtil;
 import org.apache.flink.table.planner.utils.JavaScalaConversionUtil;
@@ -165,10 +166,11 @@ public class StreamExecTemporalJoin extends ExecNodeBase<RowData>
                 (Transformation<RowData>) rightInputEdge.translateToPlan(planner);
 
         TwoInputTransformation<RowData, RowData, RowData> ret =
-                new TwoInputTransformation<>(
+                ExecNodeUtil.createTwoInputTransformation(
                         leftTransform,
                         rightTransform,
-                        getDescription(),
+                        getOperatorName(planner.getTableConfig()),
+                        getOperatorDescription(planner.getTableConfig()),
                         joinOperator,
                         InternalTypeInfo.of(returnType),
                         leftTransform.getParallelism());
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecTemporalSort.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecTemporalSort.java
index a177011..37bde9c 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecTemporalSort.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecTemporalSort.java
@@ -30,6 +30,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
 import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
 import org.apache.flink.table.planner.plan.nodes.exec.MultipleTransformationTranslator;
 import org.apache.flink.table.planner.plan.nodes.exec.spec.SortSpec;
+import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
 import org.apache.flink.table.runtime.generated.GeneratedRecordComparator;
 import org.apache.flink.table.runtime.keyselector.EmptyRowDataKeySelector;
 import org.apache.flink.table.runtime.operators.sort.ProcTimeSortOperator;
@@ -128,9 +129,10 @@ public class StreamExecTemporalSort extends ExecNodeBase<RowData>
                     new ProcTimeSortOperator(InternalTypeInfo.of(inputType), rowComparator);
 
             OneInputTransformation<RowData, RowData> transform =
-                    new OneInputTransformation<>(
+                    ExecNodeUtil.createOneInputTransformation(
                             inputTransform,
-                            getDescription(),
+                            getOperatorName(tableConfig),
+                            getOperatorDescription(tableConfig),
                             sortOperator,
                             InternalTypeInfo.of(inputType),
                             inputTransform.getParallelism());
@@ -169,9 +171,10 @@ public class StreamExecTemporalSort extends ExecNodeBase<RowData>
                         rowComparator);
 
         OneInputTransformation<RowData, RowData> transform =
-                new OneInputTransformation<>(
+                ExecNodeUtil.createOneInputTransformation(
                         inputTransform,
-                        getDescription(),
+                        getOperatorName(tableConfig),
+                        getOperatorDescription(tableConfig),
                         sortOperator,
                         InternalTypeInfo.of(inputType),
                         inputTransform.getParallelism());
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWatermarkAssigner.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWatermarkAssigner.java
index 633c183..b70e071 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWatermarkAssigner.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWatermarkAssigner.java
@@ -19,7 +19,6 @@
 package org.apache.flink.table.planner.plan.nodes.exec.stream;
 
 import org.apache.flink.api.dag.Transformation;
-import org.apache.flink.streaming.api.transformations.OneInputTransformation;
 import org.apache.flink.table.api.TableConfig;
 import org.apache.flink.table.api.config.ExecutionConfigOptions;
 import org.apache.flink.table.data.RowData;
@@ -30,6 +29,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
 import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
 import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
 import org.apache.flink.table.planner.plan.nodes.exec.SingleTransformationTranslator;
+import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
 import org.apache.flink.table.planner.utils.JavaScalaConversionUtil;
 import org.apache.flink.table.runtime.generated.GeneratedWatermarkGenerator;
 import org.apache.flink.table.runtime.operators.wmassigners.WatermarkAssignerOperatorFactory;
@@ -114,9 +114,10 @@ public class StreamExecWatermarkAssigner extends ExecNodeBase<RowData>
                 new WatermarkAssignerOperatorFactory(
                         rowtimeFieldIndex, idleTimeout, watermarkGenerator);
 
-        return new OneInputTransformation<>(
+        return ExecNodeUtil.createOneInputTransformation(
                 inputTransform,
-                getDescription(),
+                getOperatorName(planner.getTableConfig()),
+                getOperatorDescription(planner.getTableConfig()),
                 operatorFactory,
                 InternalTypeInfo.of(getOutputType()),
                 inputTransform.getParallelism());
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowAggregate.java
index 3229441..7d0298d 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowAggregate.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowAggregate.java
@@ -177,7 +177,8 @@ public class StreamExecWindowAggregate extends StreamExecWindowAggregateBase {
         final OneInputTransformation<RowData, RowData> transform =
                 ExecNodeUtil.createOneInputTransformation(
                         inputTransform,
-                        getDescription(),
+                        getOperatorName(planner.getTableConfig()),
+                        getOperatorDescription(planner.getTableConfig()),
                         SimpleOperatorFactory.of(windowOperator),
                         InternalTypeInfo.of(getOutputType()),
                         inputTransform.getParallelism(),
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowDeduplicate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowDeduplicate.java
index ad9e5bd..dd7b5a8 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowDeduplicate.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowDeduplicate.java
@@ -157,7 +157,8 @@ public class StreamExecWindowDeduplicate extends ExecNodeBase<RowData>
         OneInputTransformation<RowData, RowData> transform =
                 ExecNodeUtil.createOneInputTransformation(
                         inputTransform,
-                        getDescription(),
+                        getOperatorName(tableConfig),
+                        getOperatorDescription(tableConfig),
                         SimpleOperatorFactory.of(operator),
                         InternalTypeInfo.of(getOutputType()),
                         inputTransform.getParallelism(),
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowJoin.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowJoin.java
index 0490f86..aee6206 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowJoin.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowJoin.java
@@ -32,6 +32,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
 import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
 import org.apache.flink.table.planner.plan.nodes.exec.SingleTransformationTranslator;
 import org.apache.flink.table.planner.plan.nodes.exec.spec.JoinSpec;
+import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
 import org.apache.flink.table.planner.plan.utils.JoinUtil;
 import org.apache.flink.table.planner.plan.utils.KeySelectorUtil;
 import org.apache.flink.table.runtime.generated.GeneratedJoinCondition;
@@ -160,10 +161,11 @@ public class StreamExecWindowJoin extends ExecNodeBase<RowData>
 
         final RowType returnType = (RowType) getOutputType();
         final TwoInputTransformation<RowData, RowData, RowData> transform =
-                new TwoInputTransformation<>(
+                ExecNodeUtil.createTwoInputTransformation(
                         leftTransform,
                         rightTransform,
-                        getDescription(),
+                        getOperatorName(planner.getTableConfig()),
+                        getOperatorDescription(planner.getTableConfig()),
                         operator,
                         InternalTypeInfo.of(returnType),
                         leftTransform.getParallelism());
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowRank.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowRank.java
index 6f20e90..150bc69 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowRank.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowRank.java
@@ -234,7 +234,8 @@ public class StreamExecWindowRank extends ExecNodeBase<RowData>
         OneInputTransformation<RowData, RowData> transform =
                 ExecNodeUtil.createOneInputTransformation(
                         inputTransform,
-                        getDescription(),
+                        getOperatorName(planner.getTableConfig()),
+                        getOperatorDescription(planner.getTableConfig()),
                         SimpleOperatorFactory.of(operator),
                         InternalTypeInfo.of(getOutputType()),
                         inputTransform.getParallelism(),
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/utils/ExecNodeUtil.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/utils/ExecNodeUtil.java
index 37c6a7f..2e059ad 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/utils/ExecNodeUtil.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/utils/ExecNodeUtil.java
@@ -22,7 +22,10 @@ import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.connector.source.Boundedness;
 import org.apache.flink.api.dag.Transformation;
 import org.apache.flink.core.memory.ManagedMemoryUseCase;
+import org.apache.flink.streaming.api.operators.SimpleOperatorFactory;
+import org.apache.flink.streaming.api.operators.StreamOperator;
 import org.apache.flink.streaming.api.operators.StreamOperatorFactory;
+import org.apache.flink.streaming.api.operators.TwoInputStreamOperator;
 import org.apache.flink.streaming.api.transformations.LegacySourceTransformation;
 import org.apache.flink.streaming.api.transformations.OneInputTransformation;
 import org.apache.flink.streaming.api.transformations.TwoInputTransformation;
@@ -54,33 +57,114 @@ public class ExecNodeUtil {
         }
     }
 
+    /** Create a {@link OneInputTransformation}. */
+    public static <I, O> OneInputTransformation<I, O> createOneInputTransformation(
+            Transformation<I> input,
+            String name,
+            String desc,
+            StreamOperator<O> operator,
+            TypeInformation<O> outputType,
+            int parallelism) {
+        return createOneInputTransformation(
+                input, name, desc, operator, outputType, parallelism, 0);
+    }
+
+    /** Create a {@link OneInputTransformation} with memoryBytes. */
+    public static <I, O> OneInputTransformation<I, O> createOneInputTransformation(
+            Transformation<I> input,
+            String name,
+            String desc,
+            StreamOperator<O> operator,
+            TypeInformation<O> outputType,
+            int parallelism,
+            long memoryBytes) {
+        return createOneInputTransformation(
+                input,
+                name,
+                desc,
+                SimpleOperatorFactory.of(operator),
+                outputType,
+                parallelism,
+                memoryBytes);
+    }
+
+    /** Create a {@link OneInputTransformation}. */
+    public static <I, O> OneInputTransformation<I, O> createOneInputTransformation(
+            Transformation<I> input,
+            String name,
+            String desc,
+            StreamOperatorFactory<O> operatorFactory,
+            TypeInformation<O> outputType,
+            int parallelism) {
+        return createOneInputTransformation(
+                input, name, desc, operatorFactory, outputType, parallelism, 0);
+    }
+
     /** Create a {@link OneInputTransformation} with memoryBytes. */
-    public static <T> OneInputTransformation<T, T> createOneInputTransformation(
-            Transformation<T> input,
+    public static <I, O> OneInputTransformation<I, O> createOneInputTransformation(
+            Transformation<I> input,
             String name,
-            StreamOperatorFactory<T> operatorFactory,
-            TypeInformation<T> outputType,
+            String desc,
+            StreamOperatorFactory<O> operatorFactory,
+            TypeInformation<O> outputType,
             int parallelism,
             long memoryBytes) {
-        OneInputTransformation<T, T> transformation =
+        OneInputTransformation<I, O> transformation =
                 new OneInputTransformation<>(input, name, operatorFactory, outputType, parallelism);
         setManagedMemoryWeight(transformation, memoryBytes);
+        transformation.setDescription(desc);
         return transformation;
     }
 
     /** Create a {@link TwoInputTransformation} with memoryBytes. */
-    public static <T> TwoInputTransformation<T, T, T> createTwoInputTransformation(
-            Transformation<T> input1,
-            Transformation<T> input2,
+    public static <IN1, IN2, O> TwoInputTransformation<IN1, IN2, O> createTwoInputTransformation(
+            Transformation<IN1> input1,
+            Transformation<IN2> input2,
+            String name,
+            String desc,
+            TwoInputStreamOperator<IN1, IN2, O> operator,
+            TypeInformation<O> outputType,
+            int parallelism) {
+        return createTwoInputTransformation(
+                input1, input2, name, desc, operator, outputType, parallelism, 0);
+    }
+
+    /** Create a {@link TwoInputTransformation} with memoryBytes. */
+    public static <IN1, IN2, O> TwoInputTransformation<IN1, IN2, O> createTwoInputTransformation(
+            Transformation<IN1> input1,
+            Transformation<IN2> input2,
+            String name,
+            String desc,
+            TwoInputStreamOperator<IN1, IN2, O> operator,
+            TypeInformation<O> outputType,
+            int parallelism,
+            long memoryBytes) {
+        return createTwoInputTransformation(
+                input1,
+                input2,
+                name,
+                desc,
+                SimpleOperatorFactory.of(operator),
+                outputType,
+                parallelism,
+                memoryBytes);
+    }
+
+    /** Create a {@link TwoInputTransformation} with memoryBytes. */
+    public static <I1, I2, O> TwoInputTransformation<I1, I2, O> createTwoInputTransformation(
+            Transformation<I1> input1,
+            Transformation<I2> input2,
             String name,
-            StreamOperatorFactory<T> operatorFactory,
-            TypeInformation<T> outputType,
+            String desc,
+            StreamOperatorFactory<O> operatorFactory,
+            TypeInformation<O> outputType,
             int parallelism,
             long memoryBytes) {
-        TwoInputTransformation<T, T, T> transformation =
+        TwoInputTransformation<I1, I2, O> transformation =
                 new TwoInputTransformation<>(
                         input1, input2, name, operatorFactory, outputType, parallelism);
         setManagedMemoryWeight(transformation, memoryBytes);
+        transformation.setDescription(desc);
         return transformation;
     }
 
diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/CorrelateCodeGenerator.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/CorrelateCodeGenerator.scala
index 79bf788..5b25e6a 100644
--- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/CorrelateCodeGenerator.scala
+++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/CorrelateCodeGenerator.scala
@@ -51,8 +51,9 @@ object CorrelateCodeGenerator {
       parallelism: Int,
       retainHeader: Boolean,
       opName: String,
-      transformationName: String)
-    : Transformation[RowData] = {
+      transformationName: String,
+      transformationDescription: String)
+  : Transformation[RowData] = {
 
     // according to the SQL standard, every scalar function should also be a table function
     // but we don't allow that for now
@@ -86,6 +87,7 @@ object CorrelateCodeGenerator {
     ExecNodeUtil.createOneInputTransformation(
       inputTransformation,
       transformationName,
+      transformationDescription,
       substituteStreamOperator,
       InternalTypeInfo.of(outputType),
       parallelism,
diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/ScanUtil.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/ScanUtil.scala
index 6a16b90..e070564 100644
--- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/ScanUtil.scala
+++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/ScanUtil.scala
@@ -68,6 +68,8 @@ object ScanUtil {
       inputType: DataType,
       outputRowType: RowType,
       qualifiedName: util.List[String],
+      nameFormatter: (String, String) => String,
+      descriptionFormatter: String => String,
       rowtimeExpr: Option[RexNode] = None,
       beforeConvert: String = "",
       afterConvert: String = ""): Transformation[RowData] = {
@@ -115,9 +117,12 @@ object ScanUtil {
 
     val substituteStreamOperator = new CodeGenOperatorFactory[RowData](generatedOperator)
 
+    val description = descriptionFormatter(getOperatorDescription(qualifiedName, outputRowType))
+    val name = nameFormatter(description, "SourceConversion");
     ExecNodeUtil.createOneInputTransformation(
       input.asInstanceOf[Transformation[RowData]],
-      getOperatorName(qualifiedName, outputRowType),
+      name,
+      description,
       substituteStreamOperator,
       InternalTypeInfo.of(outputRowType),
       input.getParallelism,
@@ -127,7 +132,8 @@ object ScanUtil {
   /**
     * @param qualifiedName qualified name for table
     */
-  private[flink] def getOperatorName(qualifiedName: Seq[String], rowType: RowType): String = {
+  private[flink] def getOperatorDescription(
+      qualifiedName: Seq[String], rowType: RowType): String = {
     val tableQualifiedName = qualifiedName.mkString(".")
     val fieldNames = rowType.getFieldNames.mkString(", ")
     s"SourceConversion(table=[$tableQualifiedName], fields=[$fieldNames])"
diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/connector/file/table/FileSystemTableSinkTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/connector/file/table/FileSystemTableSinkTest.java
index 864bdfc..9c9d28b 100644
--- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/connector/file/table/FileSystemTableSinkTest.java
+++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/connector/file/table/FileSystemTableSinkTest.java
@@ -28,6 +28,7 @@ import org.junit.Test;
 
 import static org.apache.flink.core.testutils.CommonTestUtils.assertThrows;
 import static org.apache.flink.table.planner.utils.TableTestUtil.readFromResource;
+import static org.apache.flink.table.planner.utils.TableTestUtil.replaceNodeIdInOperator;
 import static org.apache.flink.table.planner.utils.TableTestUtil.replaceStageId;
 import static org.apache.flink.table.planner.utils.TableTestUtil.replaceStreamNodeId;
 import static org.junit.Assert.assertEquals;
@@ -77,8 +78,8 @@ public class FileSystemTableSinkTest {
                 readFromResource(
                         "/explain/filesystem/testFileSystemTableSinkWithParallelismInStreamingSql0.out");
         assertEquals(
-                replaceStreamNodeId(replaceStageId(expectedNormal)),
-                replaceStreamNodeId(replaceStageId(actualNormal)));
+                replaceNodeIdInOperator(replaceStreamNodeId(replaceStageId(expectedNormal))),
+                replaceNodeIdInOperator(replaceStreamNodeId(replaceStageId(actualNormal))));
 
         // verify operator parallelisms when compaction is enabled
         final String testCompactSinkTableName = "test_compact_sink_table";
@@ -89,8 +90,8 @@ public class FileSystemTableSinkTest {
                 readFromResource(
                         "/explain/filesystem/testFileSystemTableSinkWithParallelismInStreamingSql1.out");
         assertEquals(
-                replaceStreamNodeId(replaceStageId(expectedCompact)),
-                replaceStreamNodeId(replaceStageId(actualCompact)));
+                replaceNodeIdInOperator(replaceStreamNodeId(replaceStageId(expectedCompact))),
+                replaceNodeIdInOperator(replaceStreamNodeId(replaceStageId(actualCompact))));
     }
 
     @Test
@@ -113,8 +114,8 @@ public class FileSystemTableSinkTest {
                         "/explain/filesystem/testFileSystemTableSinkWithParallelismInBatch.out");
 
         assertEquals(
-                replaceStreamNodeId(replaceStageId(expected)),
-                replaceStreamNodeId(replaceStageId(actual)));
+                replaceNodeIdInOperator(replaceStreamNodeId(replaceStageId(expected))),
+                replaceNodeIdInOperator(replaceStreamNodeId(replaceStageId(actual))));
     }
 
     private static String buildSourceTableSql(String testSourceTableName, boolean bounded) {
diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/api/internal/TableEnvironmentInternalTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/api/internal/TableEnvironmentInternalTest.java
index eb1cc3a..d4714e4 100644
--- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/api/internal/TableEnvironmentInternalTest.java
+++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/api/internal/TableEnvironmentInternalTest.java
@@ -93,7 +93,9 @@ public class TableEnvironmentInternalTest extends JsonPlanTestBase {
         String jsonPlan = TableTestUtil.readFromResource("/jsonplan/testGetJsonPlan.out");
         String actual = tableEnv.explainJsonPlan(jsonPlan, ExplainDetail.JSON_EXECUTION_PLAN);
         String expected = TableTestUtil.readFromResource("/explain/testExplainJsonPlan.out");
-        assertEquals(expected, TableTestUtil.replaceStreamNodeId(actual));
+        assertEquals(
+                expected,
+                TableTestUtil.replaceNodeIdInOperator(TableTestUtil.replaceStreamNodeId(actual)));
     }
 
     @Test
diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/operator/BatchOperatorNameTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/operator/BatchOperatorNameTest.java
new file mode 100644
index 0000000..1d55870
--- /dev/null
+++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/operator/BatchOperatorNameTest.java
@@ -0,0 +1,159 @@
+/*
+ * 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.plan.nodes.exec.operator;
+
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.table.api.TableConfig;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.config.ExecutionConfigOptions;
+import org.apache.flink.table.planner.utils.BatchTableTestUtil;
+import org.apache.flink.table.planner.utils.JavaScalaConversionUtil;
+import org.apache.flink.table.planner.utils.TableTestUtil;
+import org.apache.flink.table.planner.utils.TestLegacyFilterableTableSource;
+import org.apache.flink.table.sinks.TableSink;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.types.Row;
+
+import org.junit.Test;
+
+import java.util.Optional;
+
+/** Tests for verifying name and description of batch sql operator. */
+public class BatchOperatorNameTest extends OperatorNameTestBase {
+
+    @Override
+    protected TableTestUtil getTableTestUtil() {
+        return batchTestUtil(TableConfig.getDefault());
+    }
+
+    @Test
+    public void testBoundedStreamScan() {
+        final DataStream<Integer> dataStream = util.getStreamEnv().fromElements(1, 2, 3, 4, 5);
+        TableTestUtil.createTemporaryView(
+                tEnv,
+                "MyTable",
+                dataStream,
+                JavaScalaConversionUtil.toScala(Optional.empty()),
+                JavaScalaConversionUtil.toScala(Optional.empty()),
+                JavaScalaConversionUtil.toScala(Optional.empty()));
+        verifyQuery("SELECT * FROM MyTable");
+    }
+
+    /** Verify Expand, HashAggregate. */
+    @Test
+    public void testHashAggregate() {
+        createTestSource();
+        verifyQuery("SELECT a, " + "count(distinct b) as b " + "FROM MyTable GROUP BY a");
+    }
+
+    /** Verify Sort, SortAggregate. */
+    @Test
+    public void testSortAggregate() {
+        tEnv.getConfig()
+                .getConfiguration()
+                .setString(ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashAgg");
+        createTestSource();
+        verifyQuery("SELECT a, " + "count(distinct b) as b " + "FROM MyTable GROUP BY a");
+    }
+
+    /** Verify SortWindowAggregate. */
+    @Test
+    public void testSortWindowAggregate() {
+        createSourceWithTimeAttribute();
+        verifyQuery(
+                "SELECT\n"
+                        + "  b,\n"
+                        + "  TUMBLE_END(rowtime, INTERVAL '15' MINUTE) as window_end,\n"
+                        + "  FIRST_VALUE(a)\n"
+                        + "FROM MyTable\n"
+                        + "GROUP BY b, TUMBLE(rowtime, INTERVAL '15' MINUTE)");
+    }
+
+    /** Verify HashJoin. */
+    @Test
+    public void testHashJoin() {
+        testJoinInternal();
+    }
+
+    /** Verify NestedLoopJoin. */
+    @Test
+    public void testNestedLoopJoin() {
+        tEnv.getConfig()
+                .getConfiguration()
+                .setString(
+                        ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS,
+                        "HashJoin, SortMergeJoin");
+        testJoinInternal();
+    }
+
+    /** Verify SortMergeJoin. */
+    @Test
+    public void testSortMergeJoin() {
+        tEnv.getConfig()
+                .getConfiguration()
+                .setString(
+                        ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS,
+                        "HashJoin, NestedLoopJoin");
+        testJoinInternal();
+    }
+
+    /** Verify MultiInput. */
+    @Test
+    public void testMultiInput() {
+        createTestSource("A");
+        createTestSource("B");
+        createTestSource("C");
+        verifyQuery("SELECT * FROM A, B, C where A.a = B.a and A.a = C.a");
+    }
+
+    /** Verify Limit. */
+    @Test
+    public void testLimit() {
+        createTestSource();
+        verifyQuery("select * from MyTable limit 10");
+    }
+
+    /** Verify SortLimit. */
+    @Test
+    public void testSortLimit() {
+        createTestSource();
+        verifyQuery("select * from MyTable order by a limit 10");
+    }
+
+    @Test
+    public void testLegacySourceSink() {
+        TableSchema schema = TestLegacyFilterableTableSource.defaultSchema();
+        TestLegacyFilterableTableSource.createTemporaryTable(
+                tEnv,
+                schema,
+                "MySource",
+                true,
+                TestLegacyFilterableTableSource.defaultRows().toList(),
+                TestLegacyFilterableTableSource.defaultFilterableFields());
+        TableSink<Row> sink =
+                ((BatchTableTestUtil) util)
+                        .createCollectTableSink(
+                                schema.getFieldNames(),
+                                schema.getTableColumns().stream()
+                                        .map(col -> col.getType().getLogicalType())
+                                        .toArray(LogicalType[]::new));
+        util.testingTableEnv().registerTableSinkInternal("MySink", sink);
+        verifyInsert("insert into MySink select * from MySource");
+    }
+}
diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/operator/OperatorNameTestBase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/operator/OperatorNameTestBase.java
new file mode 100644
index 0000000..0332826
--- /dev/null
+++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/operator/OperatorNameTestBase.java
@@ -0,0 +1,195 @@
+/*
+ * 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.plan.nodes.exec.operator;
+
+import org.apache.flink.table.api.ExplainDetail;
+import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.table.api.config.OptimizerConfigOptions;
+import org.apache.flink.table.planner.utils.JavaScalaConversionUtil;
+import org.apache.flink.table.planner.utils.TableFunc1;
+import org.apache.flink.table.planner.utils.TableTestBase;
+import org.apache.flink.table.planner.utils.TableTestUtil;
+
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+/** Base class for verifying name and description of SQL operators. */
+@RunWith(Parameterized.class)
+public abstract class OperatorNameTestBase extends TableTestBase {
+    @Parameterized.Parameter public boolean isNameSimplifyEnabled;
+    protected TableTestUtil util;
+    protected TableEnvironment tEnv;
+
+    @Parameterized.Parameters(name = "isNameSimplifyEnabled={0}")
+    public static List<Boolean> testData() {
+        return Arrays.asList(true, false);
+    }
+
+    @Before
+    public void setup() {
+        util = getTableTestUtil();
+        util.getStreamEnv().setParallelism(2);
+        tEnv = util.getTableEnv();
+        tEnv.getConfig()
+                .getConfiguration()
+                .set(
+                        OptimizerConfigOptions.TABLE_OPTIMIZER_SIMPLIFY_OPERATOR_NAME_ENABLED,
+                        isNameSimplifyEnabled);
+    }
+
+    protected void verifyQuery(String query) {
+        util.verifyExplain(
+                query,
+                JavaScalaConversionUtil.toScala(
+                        Collections.singletonList(ExplainDetail.JSON_EXECUTION_PLAN)));
+    }
+
+    protected void verifyInsert(String statement) {
+        util.verifyExplainInsert(
+                statement,
+                JavaScalaConversionUtil.toScala(
+                        Collections.singletonList(ExplainDetail.JSON_EXECUTION_PLAN)));
+    }
+
+    /** Verify Correlate and Calc. */
+    @Test
+    public void testCorrelate() {
+        createTestSource();
+        util.addTemporarySystemFunction("func1", new TableFunc1());
+        verifyQuery("SELECT s FROM MyTable, LATERAL TABLE(func1(c)) AS T(s)");
+    }
+
+    /** Verify LookUpJoin. */
+    @Test
+    public void testLookupJoin() {
+        createSourceWithTimeAttribute();
+        String srcTableB =
+                "CREATE TABLE LookupTable (\n"
+                        + "  id int,\n"
+                        + "  name varchar,\n"
+                        + "  age int \n"
+                        + ") with (\n"
+                        + "  'connector' = 'values',\n"
+                        + "  'bounded' = 'true')";
+        tEnv.executeSql(srcTableB);
+        verifyQuery(
+                "SELECT * FROM MyTable AS T JOIN LookupTable "
+                        + "FOR SYSTEM_TIME AS OF T.proctime AS D ON T.b = D.id");
+    }
+
+    /** Verify GroupWindowAggregate. */
+    @Test
+    public void testGroupWindowAggregate() {
+        createSourceWithTimeAttribute();
+        verifyQuery(
+                "SELECT\n"
+                        + "  b,\n"
+                        + "  TUMBLE_END(rowtime, INTERVAL '15' MINUTE) as window_end,\n"
+                        + "  COUNT(*)\n"
+                        + "FROM MyTable\n"
+                        + "GROUP BY b, TUMBLE(rowtime, INTERVAL '15' MINUTE)");
+    }
+
+    /** Verify OverAggregate. */
+    @Test
+    public void testOverAggregate() {
+        createSourceWithTimeAttribute();
+        String sql =
+                "SELECT b,\n"
+                        + "    COUNT(a) OVER (PARTITION BY b ORDER BY rowtime\n"
+                        + "        ROWS BETWEEN 2 PRECEDING AND CURRENT ROW) AS cnt1\n"
+                        + "FROM MyTable";
+        verifyQuery(sql);
+    }
+
+    /** Verify Rank. */
+    @Test
+    public void testRank() {
+        createTestSource();
+        String sql =
+                "SELECT a, row_num\n"
+                        + "FROM (\n"
+                        + "  SELECT a, ROW_NUMBER() OVER (PARTITION BY b ORDER BY a) as row_num\n"
+                        + "  FROM MyTable)\n"
+                        + "WHERE row_num <= a";
+        verifyQuery(sql);
+    }
+
+    /** Verify GroupAggregate. */
+    protected void testGroupAggregateInternal() {
+        createTestSource();
+        verifyQuery("SELECT a, " + "max(b) as b " + "FROM MyTable GROUP BY a");
+    }
+
+    /** Verify Join. */
+    protected void testJoinInternal() {
+        createTestSource("A");
+        createTestSource("B");
+        verifyQuery("SELECT * from A, B where A.a = B.d");
+    }
+
+    protected void createTestSource() {
+        createTestSource("MyTable");
+    }
+
+    protected void createTestSource(String tableName) {
+        String srcTableDdl =
+                String.format(
+                        "CREATE TABLE %s (\n"
+                                + "  a bigint,\n"
+                                + "  b int not null,\n"
+                                + "  c varchar,\n"
+                                + "  d bigint not null\n"
+                                + ") with (\n"
+                                + "  'connector' = 'values',\n"
+                                + "  'bounded' = 'true')",
+                        tableName);
+        tEnv.executeSql(srcTableDdl);
+    }
+
+    protected void createSourceWithTimeAttribute() {
+        createSourceWithTimeAttribute("MyTable");
+    }
+
+    protected void createSourceWithTimeAttribute(String name) {
+        String srcTableDdl =
+                String.format(
+                        "CREATE TABLE %s (\n"
+                                + "  a bigint,\n"
+                                + "  b int not null,\n"
+                                + "  c varchar,\n"
+                                + "  d bigint not null,\n"
+                                + "  rowtime timestamp(3),\n"
+                                + "  proctime as proctime(),\n"
+                                + "  watermark for rowtime AS rowtime - interval '1' second\n"
+                                + ") with (\n"
+                                + "  'connector' = 'values',\n"
+                                + "  'bounded' = 'true')",
+                        name);
+        tEnv.executeSql(srcTableDdl);
+    }
+
+    protected abstract TableTestUtil getTableTestUtil();
+}
diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/operator/StreamOperatorNameTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/operator/StreamOperatorNameTest.java
new file mode 100644
index 0000000..fb4fad5
--- /dev/null
+++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/operator/StreamOperatorNameTest.java
@@ -0,0 +1,404 @@
+/*
+ * 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.plan.nodes.exec.operator;
+
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.table.api.TableConfig;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.api.config.OptimizerConfigOptions;
+import org.apache.flink.table.functions.TemporalTableFunction;
+import org.apache.flink.table.planner.utils.JavaScalaConversionUtil;
+import org.apache.flink.table.planner.utils.StreamTableTestUtil;
+import org.apache.flink.table.planner.utils.TableTestUtil;
+import org.apache.flink.table.planner.utils.TestLegacyFilterableTableSource;
+import org.apache.flink.table.planner.utils.Top3;
+import org.apache.flink.table.sinks.TableSink;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.types.Row;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.Optional;
+
+import static org.apache.flink.table.api.Expressions.$;
+import static org.apache.flink.table.api.Expressions.call;
+
+/** Tests for verifying name and description of stream sql operator name. */
+public class StreamOperatorNameTest extends OperatorNameTestBase {
+
+    private StreamTableTestUtil util;
+
+    @Override
+    protected TableTestUtil getTableTestUtil() {
+        return streamTestUtil(TableConfig.getDefault());
+    }
+
+    @Before
+    public void setup() {
+        super.setup();
+        util = (StreamTableTestUtil) super.util;
+    }
+
+    /** Verify DropUpdateBefore. */
+    @Test
+    public void testDropUpdateBefore() throws Exception {
+
+        util.getStreamEnv().setParallelism(2);
+
+        String srcTableDdl =
+                "CREATE TABLE MyTable (\n"
+                        + "  a bigint,\n"
+                        + "  b int not null,\n"
+                        + "  c varchar,\n"
+                        + "  d bigint not null,\n"
+                        + "  primary key(a, b) NOT ENFORCED\n"
+                        + ") with (\n"
+                        + "  'connector' = 'values',\n"
+                        + "  'changelog-mode' = 'I,UA,UB,D',\n"
+                        + "  'bounded' = 'false')";
+        tEnv.executeSql(srcTableDdl);
+
+        String sinkTableDdl =
+                "CREATE TABLE MySink (\n"
+                        + "  c varchar,\n"
+                        + "  a bigint,\n"
+                        + "  b int not null,\n"
+                        + "  primary key(a, b) NOT ENFORCED\n"
+                        + ") with (\n"
+                        + "  'connector' = 'values',\n"
+                        + "  'sink-insert-only' = 'false',\n"
+                        + "  'sink-changelog-mode-enforced' = 'I,UA,D',"
+                        + "  'table-sink-class' = 'DEFAULT')";
+        tEnv.executeSql(sinkTableDdl);
+
+        verifyInsert("insert into MySink select c, a, b from MyTable");
+    }
+
+    /** Verify ChangelogNormalize and SinkMaterialize. */
+    @Test
+    public void testChangelogNormalize() throws Exception {
+
+        util.getStreamEnv().setParallelism(2);
+
+        String srcTableDdl =
+                "CREATE TABLE MyTable (\n"
+                        + "  a bigint,\n"
+                        + "  b int not null,\n"
+                        + "  c varchar,\n"
+                        + "  d bigint not null,\n"
+                        + "  primary key(a, b) NOT ENFORCED\n"
+                        + ") with (\n"
+                        + "  'connector' = 'values',\n"
+                        + "  'changelog-mode' = 'I,UA,D',\n"
+                        + "  'bounded' = 'false')";
+        tEnv.executeSql(srcTableDdl);
+
+        String sinkTableDdl =
+                "CREATE TABLE MySink (\n"
+                        + "  c varchar,\n"
+                        + "  a bigint,\n"
+                        + "  b int not null,\n"
+                        + "  primary key(a) NOT ENFORCED\n"
+                        + ") with (\n"
+                        + "  'connector' = 'values',\n"
+                        + "  'sink-insert-only' = 'false',\n"
+                        + "  'sink-changelog-mode-enforced' = 'I,UA,D',"
+                        + "  'table-sink-class' = 'DEFAULT')";
+        tEnv.executeSql(sinkTableDdl);
+
+        verifyInsert("insert into MySink select c, a, b from MyTable");
+    }
+
+    /** Verify Deduplicate. */
+    @Test
+    public void testDeduplicate() {
+        createSourceWithTimeAttribute();
+        verifyQuery(
+                "SELECT a, b, c FROM "
+                        + "(SELECT *, "
+                        + "    ROW_NUMBER() OVER (PARTITION BY a ORDER BY rowtime ASC) AS rk"
+                        + " FROM MyTable) t "
+                        + "WHERE rk = 1");
+    }
+
+    /**
+     * Verify Expand, MiniBatchAssigner, LocalGroupAggregate, GlobalGroupAggregate,
+     * IncrementalAggregate.
+     */
+    @Test
+    public void testIncrementalAggregate() {
+        util.enableMiniBatch();
+        tEnv.getConfig()
+                .getConfiguration()
+                .set(OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, true);
+        createTestSource();
+        verifyQuery("SELECT a, " + "count(distinct b) as b " + "FROM MyTable GROUP BY a");
+    }
+
+    /** Verify GroupAggregate. */
+    @Test
+    public void testGroupAggregate() {
+        testGroupAggregateInternal();
+    }
+
+    /** Verify RowConversion, TableGroupAggregate. */
+    @Test
+    public void testTableGroupAggregate() {
+        final DataStream<Integer> dataStream = util.getStreamEnv().fromElements(1, 2, 3, 4, 5);
+        TableTestUtil.createTemporaryView(
+                tEnv,
+                "MySource",
+                dataStream,
+                JavaScalaConversionUtil.toScala(Optional.empty()),
+                JavaScalaConversionUtil.toScala(Optional.empty()),
+                JavaScalaConversionUtil.toScala(Optional.empty()));
+        tEnv.createTemporaryFunction("top3", new Top3());
+        tEnv.createTemporaryView(
+                "MyTable",
+                tEnv.from("MySource")
+                        .flatAggregate(call(Top3.class, $("f0")))
+                        .select($("f0"), $("f1")));
+        verifyQuery("SELECT * FROM MyTable");
+    }
+
+    /** Verify IntervalJoin. */
+    @Test
+    public void testIntervalJoin() {
+        createSourceWithTimeAttribute("A");
+        createSourceWithTimeAttribute("B");
+        verifyQuery(
+                "SELECT t1.a, t2.b FROM A t1 JOIN B t2 ON\n"
+                        + "    t1.a = t2.a AND \n"
+                        + "    t1.proctime BETWEEN t2.proctime - INTERVAL '1' HOUR AND t2.proctime + INTERVAL '1' HOUR");
+    }
+
+    /** Verify IntervalJoin. */
+    @Test
+    public void testIntervalJoinNegativeWindow() {
+        createSourceWithTimeAttribute("A");
+        createSourceWithTimeAttribute("B");
+        verifyQuery(
+                "SELECT t1.a, t2.b FROM A t1 LEFT JOIN B t2 ON\n"
+                        + "    t1.a = t2.a AND \n"
+                        + "    t1.proctime BETWEEN t2.proctime + INTERVAL '2' HOUR AND t2.proctime + INTERVAL '1' HOUR");
+    }
+
+    /** Verify Join. */
+    @Test
+    public void testJoin() {
+        testJoinInternal();
+    }
+
+    @Test
+    public void testMatch() {
+        createSourceWithTimeAttribute();
+        String sql =
+                "SELECT T.aid, T.bid, T.cid\n"
+                        + "     FROM MyTable MATCH_RECOGNIZE (\n"
+                        + "             ORDER BY proctime\n"
+                        + "             MEASURES\n"
+                        + "             `A\"`.a AS aid,\n"
+                        + "             \u006C.a AS bid,\n"
+                        + "             C.a AS cid\n"
+                        + "             PATTERN (`A\"` \u006C C)\n"
+                        + "             DEFINE\n"
+                        + "                 `A\"` AS a = 1,\n"
+                        + "                 \u006C AS b = 2,\n"
+                        + "                 C AS c = 'c'\n"
+                        + "     ) AS T";
+        verifyQuery(sql);
+    }
+
+    @Test
+    public void testTemporalJoin() {
+        tEnv.executeSql(
+                "CREATE TABLE Orders (\n"
+                        + " amount INT,\n"
+                        + " currency STRING,\n"
+                        + " rowtime TIMESTAMP(3),\n"
+                        + " proctime AS PROCTIME(),\n"
+                        + " WATERMARK FOR rowtime AS rowtime\n"
+                        + ") WITH (\n"
+                        + " 'connector' = 'values'\n"
+                        + ")");
+        tEnv.executeSql(
+                "CREATE TABLE RatesHistory (\n"
+                        + " currency STRING,\n"
+                        + " rate INT,\n"
+                        + " rowtime TIMESTAMP(3),\n"
+                        + " WATERMARK FOR rowtime AS rowtime,\n"
+                        + " PRIMARY KEY(currency) NOT ENFORCED\n"
+                        + ") WITH (\n"
+                        + " 'connector' = 'values'\n"
+                        + ")");
+        TemporalTableFunction ratesHistory =
+                tEnv.from("RatesHistory").createTemporalTableFunction("rowtime", "currency");
+        tEnv.createTemporarySystemFunction("Rates", ratesHistory);
+        verifyQuery(
+                "SELECT amount * r.rate "
+                        + "FROM Orders AS o,  "
+                        + "LATERAL TABLE (Rates(o.rowtime)) AS r "
+                        + "WHERE o.currency = r.currency ");
+    }
+
+    @Test
+    public void testTemporalSortOnProcTime() {
+        createSourceWithTimeAttribute();
+        verifyQuery("SELECT a FROM MyTable order by proctime, c");
+    }
+
+    @Test
+    public void testTemporalSortOnEventTime() {
+        createSourceWithTimeAttribute();
+        verifyQuery("SELECT a FROM MyTable order by rowtime, c");
+    }
+
+    /** Verify WindowJoin, WindowRank, WindowAggregate, WindowDeduplicate. */
+    @Test
+    public void testWindowAggregate() {
+        tEnv.getConfig()
+                .getConfiguration()
+                .set(OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY, "ONE_PHASE");
+        createSourceWithTimeAttribute();
+        verifyQuery(
+                "SELECT\n"
+                        + "  b,\n"
+                        + "  window_start,\n"
+                        + "  window_end,\n"
+                        + "  COUNT(*),\n"
+                        + "  SUM(a)\n"
+                        + "FROM TABLE(\n"
+                        + "   TUMBLE(TABLE MyTable, DESCRIPTOR(rowtime), INTERVAL '5' SECOND))\n"
+                        + "GROUP BY b, window_start, window_end");
+    }
+
+    /** Verify LocalWindowAggregate, GlobalWindowAggregate. */
+    @Test
+    public void testLocalGlobalWindowAggregate() {
+        createSourceWithTimeAttribute();
+        verifyQuery(
+                "SELECT\n"
+                        + "  b,\n"
+                        + "  window_start,\n"
+                        + "  window_end,\n"
+                        + "  COUNT(*),\n"
+                        + "  SUM(a)\n"
+                        + "FROM TABLE(\n"
+                        + "   TUMBLE(TABLE MyTable, DESCRIPTOR(rowtime), INTERVAL '5' SECOND))\n"
+                        + "GROUP BY b, window_start, window_end");
+    }
+
+    /** Verify WindowJoin. */
+    @Test
+    public void testWindowJoin() {
+        createSourceWithTimeAttribute("MyTable");
+        createSourceWithTimeAttribute("MyTable2");
+        verifyQuery(
+                "select\n"
+                        + "  L.a,\n"
+                        + "  L.window_start,\n"
+                        + "  L.window_end,\n"
+                        + "  L.cnt,\n"
+                        + "  L.uv,\n"
+                        + "  R.a,\n"
+                        + "  R.cnt,\n"
+                        + "  R.uv\n"
+                        + "FROM (\n"
+                        + "  SELECT\n"
+                        + "    a,\n"
+                        + "    window_start,\n"
+                        + "    window_end,\n"
+                        + "    count(*) as cnt,\n"
+                        + "    count(distinct c) AS uv\n"
+                        + "  FROM TABLE(TUMBLE(TABLE MyTable, DESCRIPTOR(rowtime), INTERVAL '15' MINUTE))\n"
+                        + "  GROUP BY a, window_start, window_end, window_time\n"
+                        + ") L\n"
+                        + "JOIN (\n"
+                        + "  SELECT\n"
+                        + "    a,\n"
+                        + "    window_start,\n"
+                        + "    window_end,\n"
+                        + "    count(*) as cnt,\n"
+                        + "    count(distinct c) AS uv\n"
+                        + "  FROM TABLE(TUMBLE(TABLE MyTable2, DESCRIPTOR(rowtime), INTERVAL '15' MINUTE))\n"
+                        + "  GROUP BY a, window_start, window_end, window_time\n"
+                        + ") R\n"
+                        + "ON L.window_start = R.window_start AND L.window_end = R.window_end AND L.a = R.a");
+    }
+
+    /** Verify WindowTableFunction and WindowRank. */
+    @Test
+    public void testWindowRank() {
+        createSourceWithTimeAttribute();
+        verifyQuery(
+                "select\n"
+                        + "  window_start,\n"
+                        + "  window_end,\n"
+                        + "  a,\n"
+                        + "  b,\n"
+                        + "  c\n"
+                        + "FROM (\n"
+                        + "  SELECT\n"
+                        + "    *,\n"
+                        + "   ROW_NUMBER() OVER(PARTITION BY a, window_start, window_end ORDER BY b DESC) as rownum\n"
+                        + "  FROM TABLE(TUMBLE(TABLE MyTable, DESCRIPTOR(rowtime), INTERVAL '15' MINUTE)))\n"
+                        + "WHERE rownum <= 3");
+    }
+
+    /** Verify WindowDeduplicate. */
+    @Test
+    public void testWindowDeduplicate() {
+        createSourceWithTimeAttribute();
+        verifyQuery(
+                "select\n"
+                        + "  window_start,\n"
+                        + "  window_end,\n"
+                        + "  a,\n"
+                        + "  b,\n"
+                        + "  c\n"
+                        + "FROM (\n"
+                        + "  SELECT\n"
+                        + "    *,\n"
+                        + "   ROW_NUMBER() OVER(PARTITION BY a, window_start, window_end ORDER BY rowtime DESC) as rownum\n"
+                        + "  FROM TABLE(TUMBLE(TABLE MyTable, DESCRIPTOR(rowtime), INTERVAL '15' MINUTE)))\n"
+                        + "WHERE rownum <= 1");
+    }
+
+    /** Verify LegacySource and LegacySink. */
+    @Test
+    public void testLegacySourceSink() {
+        TableSchema schema = TestLegacyFilterableTableSource.defaultSchema();
+        TestLegacyFilterableTableSource.createTemporaryTable(
+                tEnv,
+                schema,
+                "MySource",
+                true,
+                TestLegacyFilterableTableSource.defaultRows().toList(),
+                TestLegacyFilterableTableSource.defaultFilterableFields());
+        TableSink<Row> sink =
+                util.createAppendTableSink(
+                        schema.getFieldNames(),
+                        schema.getTableColumns().stream()
+                                .map(col -> col.getType().getLogicalType())
+                                .toArray(LogicalType[]::new));
+        util.testingTableEnv().registerTableSinkInternal("MySink", sink);
+        verifyInsert("insert into MySink select * from MySource");
+    }
+}
diff --git a/flink-table/flink-table-planner/src/test/resources/explain/filesystem/testFileSystemTableSinkWithParallelismInBatch.out b/flink-table/flink-table-planner/src/test/resources/explain/filesystem/testFileSystemTableSinkWithParallelismInBatch.out
index 81df7d7..8f4c10f 100644
--- a/flink-table/flink-table-planner/src/test/resources/explain/filesystem/testFileSystemTableSinkWithParallelismInBatch.out
+++ b/flink-table/flink-table-planner/src/test/resources/explain/filesystem/testFileSystemTableSinkWithParallelismInBatch.out
@@ -15,9 +15,9 @@ Sink(table=[default_catalog.default_database.test_sink_table], fields=[id, real_
 {
   "nodes" : [ {
     "id" : ,
-    "type" : "Source: TableSourceScan(table=[[default_catalog, default_database, test_source_table]], fields=[id, real_col, double_col, decimal_col])",
+    "type" : "Source: test_source_table[]",
     "pact" : "Data Source",
-    "contents" : "Source: TableSourceScan(table=[[default_catalog, default_database, test_source_table]], fields=[id, real_col, double_col, decimal_col])",
+    "contents" : "[]:TableSourceScan(table=[[default_catalog, default_database, test_source_table]], fields=[id, real_col, double_col, decimal_col])",
     "parallelism" : 1
   }, {
     "id" : ,
diff --git a/flink-table/flink-table-planner/src/test/resources/explain/filesystem/testFileSystemTableSinkWithParallelismInStreamingSql0.out b/flink-table/flink-table-planner/src/test/resources/explain/filesystem/testFileSystemTableSinkWithParallelismInStreamingSql0.out
index 6ab8bc2..56ba6f7 100644
--- a/flink-table/flink-table-planner/src/test/resources/explain/filesystem/testFileSystemTableSinkWithParallelismInStreamingSql0.out
+++ b/flink-table/flink-table-planner/src/test/resources/explain/filesystem/testFileSystemTableSinkWithParallelismInStreamingSql0.out
@@ -15,9 +15,9 @@ Sink(table=[default_catalog.default_database.test_sink_table], fields=[id, real_
 {
   "nodes" : [ {
     "id" : ,
-    "type" : "Source: TableSourceScan(table=[[default_catalog, default_database, test_source_table]], fields=[id, real_col, double_col, decimal_col])",
+    "type" : "Source: test_source_table[]",
     "pact" : "Data Source",
-    "contents" : "Source: TableSourceScan(table=[[default_catalog, default_database, test_source_table]], fields=[id, real_col, double_col, decimal_col])",
+    "contents" : "[]:TableSourceScan(table=[[default_catalog, default_database, test_source_table]], fields=[id, real_col, double_col, decimal_col])",
     "parallelism" : 1
   }, {
     "id" : ,
diff --git a/flink-table/flink-table-planner/src/test/resources/explain/filesystem/testFileSystemTableSinkWithParallelismInStreamingSql1.out b/flink-table/flink-table-planner/src/test/resources/explain/filesystem/testFileSystemTableSinkWithParallelismInStreamingSql1.out
index fcfe456..a9842af 100644
--- a/flink-table/flink-table-planner/src/test/resources/explain/filesystem/testFileSystemTableSinkWithParallelismInStreamingSql1.out
+++ b/flink-table/flink-table-planner/src/test/resources/explain/filesystem/testFileSystemTableSinkWithParallelismInStreamingSql1.out
@@ -15,9 +15,9 @@ Sink(table=[default_catalog.default_database.test_compact_sink_table], fields=[i
 {
   "nodes" : [ {
     "id" : ,
-    "type" : "Source: TableSourceScan(table=[[default_catalog, default_database, test_source_table]], fields=[id, real_col, double_col, decimal_col])",
+    "type" : "Source: test_source_table[]",
     "pact" : "Data Source",
-    "contents" : "Source: TableSourceScan(table=[[default_catalog, default_database, test_source_table]], fields=[id, real_col, double_col, decimal_col])",
+    "contents" : "[]:TableSourceScan(table=[[default_catalog, default_database, test_source_table]], fields=[id, real_col, double_col, decimal_col])",
     "parallelism" : 1
   }, {
     "id" : ,
diff --git a/flink-table/flink-table-planner/src/test/resources/explain/testExecuteSqlWithExplainDetailsAndUnion.out b/flink-table/flink-table-planner/src/test/resources/explain/testExecuteSqlWithExplainDetailsAndUnion.out
index f787da2..6e11a0b 100644
--- a/flink-table/flink-table-planner/src/test/resources/explain/testExecuteSqlWithExplainDetailsAndUnion.out
+++ b/flink-table/flink-table-planner/src/test/resources/explain/testExecuteSqlWithExplainDetailsAndUnion.out
@@ -25,9 +25,9 @@ Union(all=[true], union=[a, b, c])
     "parallelism" : 1
   }, {
     "id" : ,
-    "type" : "SourceConversion(table=[default_catalog.default_database.MyTable, source: [CollectionTableSource(a, b, c)]], fields=[a, b, c])",
+    "type" : "SourceConversion[]",
     "pact" : "Operator",
-    "contents" : "SourceConversion(table=[default_catalog.default_database.MyTable, source: [CollectionTableSource(a, b, c)]], fields=[a, b, c])",
+    "contents" : "[]:SourceConversion(table=[default_catalog.default_database.MyTable, source: [CollectionTableSource(a, b, c)]], fields=[a, b, c])",
     "parallelism" : 1,
     "predecessors" : [ {
       "id" : ,
@@ -42,9 +42,9 @@ Union(all=[true], union=[a, b, c])
     "parallelism" : 1
   }, {
     "id" : ,
-    "type" : "SourceConversion(table=[default_catalog.default_database.MyTable2, source: [CollectionTableSource(a, b, c)]], fields=[a, b, c])",
+    "type" : "SourceConversion[]",
     "pact" : "Operator",
-    "contents" : "SourceConversion(table=[default_catalog.default_database.MyTable2, source: [CollectionTableSource(a, b, c)]], fields=[a, b, c])",
+    "contents" : "[]:SourceConversion(table=[default_catalog.default_database.MyTable2, source: [CollectionTableSource(a, b, c)]], fields=[a, b, c])",
     "parallelism" : 1,
     "predecessors" : [ {
       "id" : ,
@@ -52,4 +52,4 @@ Union(all=[true], union=[a, b, c])
       "side" : "second"
     } ]
   } ]
-}
+}
\ No newline at end of file
diff --git a/flink-table/flink-table-planner/src/test/resources/explain/testExecuteSqlWithExplainDetailsInsert.out b/flink-table/flink-table-planner/src/test/resources/explain/testExecuteSqlWithExplainDetailsInsert.out
index 3c2be6d..17e5174 100644
--- a/flink-table/flink-table-planner/src/test/resources/explain/testExecuteSqlWithExplainDetailsInsert.out
+++ b/flink-table/flink-table-planner/src/test/resources/explain/testExecuteSqlWithExplainDetailsInsert.out
@@ -24,9 +24,9 @@ LegacySink(name=[`default_catalog`.`default_database`.`MySink`], fields=[d, e])
     "parallelism" : 1
   }, {
     "id" : ,
-    "type" : "SourceConversion(table=[default_catalog.default_database.MyTable, source: [CollectionTableSource(a, b, c)]], fields=[a, b, c])",
+    "type" : "SourceConversion[]",
     "pact" : "Operator",
-    "contents" : "SourceConversion(table=[default_catalog.default_database.MyTable, source: [CollectionTableSource(a, b, c)]], fields=[a, b, c])",
+    "contents" : "[]:SourceConversion(table=[default_catalog.default_database.MyTable, source: [CollectionTableSource(a, b, c)]], fields=[a, b, c])",
     "parallelism" : 1,
     "predecessors" : [ {
       "id" : ,
@@ -35,9 +35,9 @@ LegacySink(name=[`default_catalog`.`default_database`.`MySink`], fields=[d, e])
     } ]
   }, {
     "id" : ,
-    "type" : "Calc(select=[a, b], where=[(a > 10)])",
+    "type" : "Calc[]",
     "pact" : "Operator",
-    "contents" : "Calc(select=[a, b], where=[(a > 10)])",
+    "contents" : "[]:Calc(select=[a, b], where=[(a > 10)])",
     "parallelism" : 1,
     "predecessors" : [ {
       "id" : ,
@@ -46,9 +46,9 @@ LegacySink(name=[`default_catalog`.`default_database`.`MySink`], fields=[d, e])
     } ]
   }, {
     "id" : ,
-    "type" : "SinkConversionToRow",
+    "type" : "SinkConversion[]",
     "pact" : "Operator",
-    "contents" : "SinkConversionToRow",
+    "contents" : "[]:SinkConversion To Row",
     "parallelism" : 1,
     "predecessors" : [ {
       "id" : ,
@@ -67,4 +67,4 @@ LegacySink(name=[`default_catalog`.`default_database`.`MySink`], fields=[d, e])
       "side" : "second"
     } ]
   } ]
-}
+}
\ No newline at end of file
diff --git a/flink-table/flink-table-planner/src/test/resources/explain/testExecuteSqlWithExplainDetailsSelect.out b/flink-table/flink-table-planner/src/test/resources/explain/testExecuteSqlWithExplainDetailsSelect.out
index 385f2f5..43e80a7 100644
--- a/flink-table/flink-table-planner/src/test/resources/explain/testExecuteSqlWithExplainDetailsSelect.out
+++ b/flink-table/flink-table-planner/src/test/resources/explain/testExecuteSqlWithExplainDetailsSelect.out
@@ -21,9 +21,9 @@ Calc(select=[a, b, c], where=[(a > 10)])
     "parallelism" : 1
   }, {
     "id" : ,
-    "type" : "SourceConversion(table=[default_catalog.default_database.MyTable, source: [CollectionTableSource(a, b, c)]], fields=[a, b, c])",
+    "type" : "SourceConversion[]",
     "pact" : "Operator",
-    "contents" : "SourceConversion(table=[default_catalog.default_database.MyTable, source: [CollectionTableSource(a, b, c)]], fields=[a, b, c])",
+    "contents" : "[]:SourceConversion(table=[default_catalog.default_database.MyTable, source: [CollectionTableSource(a, b, c)]], fields=[a, b, c])",
     "parallelism" : 1,
     "predecessors" : [ {
       "id" : ,
@@ -32,9 +32,9 @@ Calc(select=[a, b, c], where=[(a > 10)])
     } ]
   }, {
     "id" : ,
-    "type" : "Calc(select=[a, b, c], where=[(a > 10)])",
+    "type" : "Calc[]",
     "pact" : "Operator",
-    "contents" : "Calc(select=[a, b, c], where=[(a > 10)])",
+    "contents" : "[]:Calc(select=[a, b, c], where=[(a > 10)])",
     "parallelism" : 1,
     "predecessors" : [ {
       "id" : ,
@@ -42,4 +42,4 @@ Calc(select=[a, b, c], where=[(a > 10)])
       "side" : "second"
     } ]
   } ]
-}
+}
\ No newline at end of file
diff --git a/flink-table/flink-table-planner/src/test/resources/explain/testExplainJsonPlan.out b/flink-table/flink-table-planner/src/test/resources/explain/testExplainJsonPlan.out
index 55f99b4..ed21d1a 100644
--- a/flink-table/flink-table-planner/src/test/resources/explain/testExplainJsonPlan.out
+++ b/flink-table/flink-table-planner/src/test/resources/explain/testExplainJsonPlan.out
@@ -6,15 +6,15 @@ Sink(table=[default_catalog.default_database.MySink], fields=[a, b, c])
 {
   "nodes" : [ {
     "id" : ,
-    "type" : "Source: TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c])",
+    "type" : "Source: MyTable[]",
     "pact" : "Data Source",
-    "contents" : "Source: TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c])",
+    "contents" : "[]:TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c])",
     "parallelism" : 1
   }, {
     "id" : ,
-    "type" : "Sink: Sink(table=[default_catalog.default_database.MySink], fields=[a, b, c])",
+    "type" : "Sink: MySink[]",
     "pact" : "Data Sink",
-    "contents" : "Sink: Sink(table=[default_catalog.default_database.MySink], fields=[a, b, c])",
+    "contents" : "[]:Sink(table=[default_catalog.default_database.MySink], fields=[a, b, c])",
     "parallelism" : 1,
     "predecessors" : [ {
       "id" : ,
diff --git a/flink-table/flink-table-planner/src/test/resources/explain/testStatementSetExecutionExplain.out b/flink-table/flink-table-planner/src/test/resources/explain/testStatementSetExecutionExplain.out
index c73081f..625d6f0 100644
--- a/flink-table/flink-table-planner/src/test/resources/explain/testStatementSetExecutionExplain.out
+++ b/flink-table/flink-table-planner/src/test/resources/explain/testStatementSetExecutionExplain.out
@@ -32,9 +32,9 @@ LegacySink(name=[`default_catalog`.`default_database`.`MySink`], fields=[first])
     } ]
   }, {
     "id" : ,
-    "type" : "SourceConversion(table=[default_catalog.default_database.MyTable, source: [CsvTableSource(read fields: last)]], fields=[last])",
+    "type" : "SourceConversion[]",
     "pact" : "Operator",
-    "contents" : "SourceConversion(table=[default_catalog.default_database.MyTable, source: [CsvTableSource(read fields: last)]], fields=[last])",
+    "contents" : "[]:SourceConversion(table=[default_catalog.default_database.MyTable, source: [CsvTableSource(read fields: last)]], fields=[last])",
     "parallelism" : 1,
     "predecessors" : [ {
       "id" : ,
@@ -43,9 +43,9 @@ LegacySink(name=[`default_catalog`.`default_database`.`MySink`], fields=[first])
     } ]
   }, {
     "id" : ,
-    "type" : "SinkConversionToRow",
+    "type" : "SinkConversion[]",
     "pact" : "Operator",
-    "contents" : "SinkConversionToRow",
+    "contents" : "[]:SinkConversion To Row",
     "parallelism" : 1,
     "predecessors" : [ {
       "id" : ,
@@ -75,4 +75,4 @@ LegacySink(name=[`default_catalog`.`default_database`.`MySink`], fields=[first])
       "side" : "second"
     } ]
   } ]
-}
+}
\ No newline at end of file
diff --git a/flink-table/flink-table-planner/src/test/resources/explain/testStreamTableEnvironmentExecutionExplain.out b/flink-table/flink-table-planner/src/test/resources/explain/testStreamTableEnvironmentExecutionExplain.out
index 4623a72..ab99c0d 100644
--- a/flink-table/flink-table-planner/src/test/resources/explain/testStreamTableEnvironmentExecutionExplain.out
+++ b/flink-table/flink-table-planner/src/test/resources/explain/testStreamTableEnvironmentExecutionExplain.out
@@ -32,9 +32,9 @@ LegacySink(name=[`default_catalog`.`default_database`.`MySink`], fields=[first])
     } ]
   }, {
     "id" : ,
-    "type" : "SourceConversion(table=[default_catalog.default_database.MyTable, source: [CsvTableSource(read fields: first)]], fields=[first])",
+    "type" : "SourceConversion[]",
     "pact" : "Operator",
-    "contents" : "SourceConversion(table=[default_catalog.default_database.MyTable, source: [CsvTableSource(read fields: first)]], fields=[first])",
+    "contents" : "[]:SourceConversion(table=[default_catalog.default_database.MyTable, source: [CsvTableSource(read fields: first)]], fields=[first])",
     "parallelism" : 4,
     "predecessors" : [ {
       "id" : ,
@@ -43,9 +43,9 @@ LegacySink(name=[`default_catalog`.`default_database`.`MySink`], fields=[first])
     } ]
   }, {
     "id" : ,
-    "type" : "SinkConversionToRow",
+    "type" : "SinkConversion[]",
     "pact" : "Operator",
-    "contents" : "SinkConversionToRow",
+    "contents" : "[]:SinkConversion To Row",
     "parallelism" : 4,
     "predecessors" : [ {
       "id" : ,
@@ -75,4 +75,4 @@ LegacySink(name=[`default_catalog`.`default_database`.`MySink`], fields=[first])
       "side" : "second"
     } ]
   } ]
-}
+}
\ No newline at end of file
diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/operator/BatchOperatorNameTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/operator/BatchOperatorNameTest.xml
new file mode 100644
index 0000000..9e8ac59
--- /dev/null
+++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/operator/BatchOperatorNameTest.xml
@@ -0,0 +1,2099 @@
+<?xml version="1.0" ?>
+<!--
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to you under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+-->
+<Root>
+  <TestCase name="testBoundedStreamScan[isNameSimplifyEnabled=false]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalProject(f0=[$0])
++- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
+
+== Optimized Physical Plan ==
+BoundedStreamScan(table=[[default_catalog, default_database, MyTable]], fields=[f0])
+
+== Optimized Execution Plan ==
+BoundedStreamScan(table=[[default_catalog, default_database, MyTable]], fields=[f0])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: Collection Source",
+    "pact" : "Data Source",
+    "contents" : "Source: Collection Source",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "SourceConversion(table=[default_catalog.default_database.MyTable], fields=[f0])",
+    "pact" : "Operator",
+    "contents" : "SourceConversion(table=[default_catalog.default_database.MyTable], fields=[f0])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testBoundedStreamScan[isNameSimplifyEnabled=true]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalProject(f0=[$0])
++- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
+
+== Optimized Physical Plan ==
+BoundedStreamScan(table=[[default_catalog, default_database, MyTable]], fields=[f0])
+
+== Optimized Execution Plan ==
+BoundedStreamScan(table=[[default_catalog, default_database, MyTable]], fields=[f0])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: Collection Source",
+    "pact" : "Data Source",
+    "contents" : "Source: Collection Source",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "SourceConversion[]",
+    "pact" : "Operator",
+    "contents" : "[]:SourceConversion(table=[default_catalog.default_database.MyTable], fields=[f0])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testCorrelate[isNameSimplifyEnabled=false]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalProject(s=[$4])
++- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{2}])
+   :- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
+   +- LogicalTableFunctionScan(invocation=[func1($cor0.c)], rowType=[RecordType(VARCHAR(2147483647) EXPR$0)])
+
+== Optimized Physical Plan ==
+Calc(select=[EXPR$0 AS s])
++- Correlate(invocation=[func1($cor0.c)], correlate=[table(func1($cor0.c))], select=[a,b,c,d,EXPR$0], rowType=[RecordType(BIGINT a, INTEGER b, VARCHAR(2147483647) c, BIGINT d, VARCHAR(2147483647) EXPR$0)], joinType=[INNER])
+   +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d])
+
+== Optimized Execution Plan ==
+Calc(select=[EXPR$0 AS s])
++- Correlate(invocation=[func1($cor0.c)], correlate=[table(func1($cor0.c))], select=[a,b,c,d,EXPR$0], rowType=[RecordType(BIGINT a, INTEGER b, VARCHAR(2147483647) c, BIGINT d, VARCHAR(2147483647) EXPR$0)], joinType=[INNER])
+   +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d])",
+    "pact" : "Data Source",
+    "contents" : "TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "Correlate(invocation=[func1($cor0.c)], correlate=[table(func1($cor0.c))], select=[a,b,c,d,EXPR$0], rowType=[RecordType(BIGINT a, INTEGER b, VARCHAR(2147483647) c, BIGINT d, VARCHAR(2147483647) EXPR$0)], joinType=[INNER])",
+    "pact" : "Operator",
+    "contents" : "Correlate(invocation=[func1($cor0.c)], correlate=[table(func1($cor0.c))], select=[a,b,c,d,EXPR$0], rowType=[RecordType(BIGINT a, INTEGER b, VARCHAR(2147483647) c, BIGINT d, VARCHAR(2147483647) EXPR$0)], joinType=[INNER])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Calc(select=[EXPR$0 AS s])",
+    "pact" : "Operator",
+    "contents" : "Calc(select=[EXPR$0 AS s])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testCorrelate[isNameSimplifyEnabled=true]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalProject(s=[$4])
++- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{2}])
+   :- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
+   +- LogicalTableFunctionScan(invocation=[func1($cor0.c)], rowType=[RecordType(VARCHAR(2147483647) EXPR$0)])
+
+== Optimized Physical Plan ==
+Calc(select=[EXPR$0 AS s])
++- Correlate(invocation=[func1($cor0.c)], correlate=[table(func1($cor0.c))], select=[a,b,c,d,EXPR$0], rowType=[RecordType(BIGINT a, INTEGER b, VARCHAR(2147483647) c, BIGINT d, VARCHAR(2147483647) EXPR$0)], joinType=[INNER])
+   +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d])
+
+== Optimized Execution Plan ==
+Calc(select=[EXPR$0 AS s])
++- Correlate(invocation=[func1($cor0.c)], correlate=[table(func1($cor0.c))], select=[a,b,c,d,EXPR$0], rowType=[RecordType(BIGINT a, INTEGER b, VARCHAR(2147483647) c, BIGINT d, VARCHAR(2147483647) EXPR$0)], joinType=[INNER])
+   +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: MyTable[]",
+    "pact" : "Data Source",
+    "contents" : "[]:TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "Correlate[]",
+    "pact" : "Operator",
+    "contents" : "[]:Correlate(invocation=[func1($cor0.c)], correlate=[table(func1($cor0.c))], select=[a,b,c,d,EXPR$0], rowType=[RecordType(BIGINT a, INTEGER b, VARCHAR(2147483647) c, BIGINT d, VARCHAR(2147483647) EXPR$0)], joinType=[INNER])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Calc[]",
+    "pact" : "Operator",
+    "contents" : "[]:Calc(select=[EXPR$0 AS s])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testGroupWindowAggregate[isNameSimplifyEnabled=false]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalProject(b=[$0], window_end=[TUMBLE_END($1)], EXPR$2=[$2])
++- LogicalAggregate(group=[{0, 1}], EXPR$2=[COUNT()])
+   +- LogicalProject(b=[$1], $f1=[$TUMBLE($4, 900000:INTERVAL MINUTE)])
+      +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
+
+== Optimized Physical Plan ==
+Calc(select=[b, w$end AS window_end, EXPR$2])
++- HashWindowAggregate(groupBy=[b], window=[TumblingGroupWindow('w$, rowtime, 900000)], properties=[w$start, w$end, w$rowtime], select=[b, Final_COUNT(count1$0) AS EXPR$2])
+   +- Exchange(distribution=[hash[b]])
+      +- LocalHashWindowAggregate(groupBy=[b], window=[TumblingGroupWindow('w$, rowtime, 900000)], properties=[w$start, w$end, w$rowtime], select=[b, Partial_COUNT(*) AS count1$0])
+         +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[b, rowtime], metadata=[]]], fields=[b, rowtime])
+
+== Optimized Execution Plan ==
+Calc(select=[b, w$end AS window_end, EXPR$2])
++- HashWindowAggregate(groupBy=[b], window=[TumblingGroupWindow('w$, rowtime, 900000)], properties=[w$start, w$end, w$rowtime], select=[b, Final_COUNT(count1$0) AS EXPR$2])
+   +- Exchange(distribution=[hash[b]])
+      +- LocalHashWindowAggregate(groupBy=[b], window=[TumblingGroupWindow('w$, rowtime, 900000)], properties=[w$start, w$end, w$rowtime], select=[b, Partial_COUNT(*) AS count1$0])
+         +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[b, rowtime], metadata=[]]], fields=[b, rowtime])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[b, rowtime], metadata=[]]], fields=[b, rowtime])",
+    "pact" : "Data Source",
+    "contents" : "TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[b, rowtime], metadata=[]]], fields=[b, rowtime])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "LocalHashWindowAggregate(groupBy=[b], window=[TumblingGroupWindow('w$, rowtime, 900000)], properties=[w$start, w$end, w$rowtime], select=[b, Partial_COUNT(*) AS count1$0])",
+    "pact" : "Operator",
+    "contents" : "LocalHashWindowAggregate(groupBy=[b], window=[TumblingGroupWindow('w$, rowtime, 900000)], properties=[w$start, w$end, w$rowtime], select=[b, Partial_COUNT(*) AS count1$0])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "HashWindowAggregate(groupBy=[b], window=[TumblingGroupWindow('w$, rowtime, 900000)], properties=[w$start, w$end, w$rowtime], select=[b, Final_COUNT(count1$0) AS EXPR$2])",
+    "pact" : "Operator",
+    "contents" : "HashWindowAggregate(groupBy=[b], window=[TumblingGroupWindow('w$, rowtime, 900000)], properties=[w$start, w$end, w$rowtime], select=[b, Final_COUNT(count1$0) AS EXPR$2])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "HASH[b]",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Calc(select=[b, w$end AS window_end, EXPR$2])",
+    "pact" : "Operator",
+    "contents" : "Calc(select=[b, w$end AS window_end, EXPR$2])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testGroupWindowAggregate[isNameSimplifyEnabled=true]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalProject(b=[$0], window_end=[TUMBLE_END($1)], EXPR$2=[$2])
++- LogicalAggregate(group=[{0, 1}], EXPR$2=[COUNT()])
+   +- LogicalProject(b=[$1], $f1=[$TUMBLE($4, 900000:INTERVAL MINUTE)])
+      +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
+
+== Optimized Physical Plan ==
+Calc(select=[b, w$end AS window_end, EXPR$2])
++- HashWindowAggregate(groupBy=[b], window=[TumblingGroupWindow('w$, rowtime, 900000)], properties=[w$start, w$end, w$rowtime], select=[b, Final_COUNT(count1$0) AS EXPR$2])
+   +- Exchange(distribution=[hash[b]])
+      +- LocalHashWindowAggregate(groupBy=[b], window=[TumblingGroupWindow('w$, rowtime, 900000)], properties=[w$start, w$end, w$rowtime], select=[b, Partial_COUNT(*) AS count1$0])
+         +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[b, rowtime], metadata=[]]], fields=[b, rowtime])
+
+== Optimized Execution Plan ==
+Calc(select=[b, w$end AS window_end, EXPR$2])
++- HashWindowAggregate(groupBy=[b], window=[TumblingGroupWindow('w$, rowtime, 900000)], properties=[w$start, w$end, w$rowtime], select=[b, Final_COUNT(count1$0) AS EXPR$2])
+   +- Exchange(distribution=[hash[b]])
+      +- LocalHashWindowAggregate(groupBy=[b], window=[TumblingGroupWindow('w$, rowtime, 900000)], properties=[w$start, w$end, w$rowtime], select=[b, Partial_COUNT(*) AS count1$0])
+         +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[b, rowtime], metadata=[]]], fields=[b, rowtime])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: MyTable[]",
+    "pact" : "Data Source",
+    "contents" : "[]:TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[b, rowtime], metadata=[]]], fields=[b, rowtime])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "HashWindowAggregate[]",
+    "pact" : "Operator",
+    "contents" : "[]:LocalHashWindowAggregate(groupBy=[b], window=[TumblingGroupWindow('w$, rowtime, 900000)], properties=[w$start, w$end, w$rowtime], select=[b, Partial_COUNT(*) AS count1$0])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "HashWindowAggregate[]",
+    "pact" : "Operator",
+    "contents" : "[]:HashWindowAggregate(groupBy=[b], window=[TumblingGroupWindow('w$, rowtime, 900000)], properties=[w$start, w$end, w$rowtime], select=[b, Final_COUNT(count1$0) AS EXPR$2])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "HASH[b]",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Calc[]",
+    "pact" : "Operator",
+    "contents" : "[]:Calc(select=[b, w$end AS window_end, EXPR$2])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testHashAggregate[isNameSimplifyEnabled=false]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalAggregate(group=[{0}], b=[COUNT(DISTINCT $1)])
++- LogicalProject(a=[$0], b=[$1])
+   +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
+
+== Optimized Physical Plan ==
+HashAggregate(isMerge=[true], groupBy=[a], select=[a, Final_COUNT(count$0) AS b])
++- Exchange(distribution=[hash[a]])
+   +- LocalHashAggregate(groupBy=[a], select=[a, Partial_COUNT(b) AS count$0])
+      +- HashAggregate(isMerge=[true], groupBy=[a, b], select=[a, b])
+         +- Exchange(distribution=[hash[a, b]])
+            +- LocalHashAggregate(groupBy=[a, b], select=[a, b])
+               +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b], metadata=[]]], fields=[a, b])
+
+== Optimized Execution Plan ==
+HashAggregate(isMerge=[true], groupBy=[a], select=[a, Final_COUNT(count$0) AS b])
++- Exchange(distribution=[hash[a]])
+   +- LocalHashAggregate(groupBy=[a], select=[a, Partial_COUNT(b) AS count$0])
+      +- HashAggregate(isMerge=[true], groupBy=[a, b], select=[a, b])
+         +- Exchange(distribution=[hash[a, b]])
+            +- LocalHashAggregate(groupBy=[a, b], select=[a, b])
+               +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b], metadata=[]]], fields=[a, b])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b], metadata=[]]], fields=[a, b])",
+    "pact" : "Data Source",
+    "contents" : "TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b], metadata=[]]], fields=[a, b])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "LocalHashAggregate(groupBy=[a, b], select=[a, b])",
+    "pact" : "Operator",
+    "contents" : "LocalHashAggregate(groupBy=[a, b], select=[a, b])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "HashAggregate(isMerge=[true], groupBy=[a, b], select=[a, b])",
+    "pact" : "Operator",
+    "contents" : "HashAggregate(isMerge=[true], groupBy=[a, b], select=[a, b])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "HASH[a, b]",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "LocalHashAggregate(groupBy=[a], select=[a, Partial_COUNT(b) AS count$0])",
+    "pact" : "Operator",
+    "contents" : "LocalHashAggregate(groupBy=[a], select=[a, Partial_COUNT(b) AS count$0])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "HashAggregate(isMerge=[true], groupBy=[a], select=[a, Final_COUNT(count$0) AS b])",
+    "pact" : "Operator",
+    "contents" : "HashAggregate(isMerge=[true], groupBy=[a], select=[a, Final_COUNT(count$0) AS b])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "HASH[a]",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testHashAggregate[isNameSimplifyEnabled=true]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalAggregate(group=[{0}], b=[COUNT(DISTINCT $1)])
++- LogicalProject(a=[$0], b=[$1])
+   +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
+
+== Optimized Physical Plan ==
+HashAggregate(isMerge=[true], groupBy=[a], select=[a, Final_COUNT(count$0) AS b])
++- Exchange(distribution=[hash[a]])
+   +- LocalHashAggregate(groupBy=[a], select=[a, Partial_COUNT(b) AS count$0])
+      +- HashAggregate(isMerge=[true], groupBy=[a, b], select=[a, b])
+         +- Exchange(distribution=[hash[a, b]])
+            +- LocalHashAggregate(groupBy=[a, b], select=[a, b])
+               +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b], metadata=[]]], fields=[a, b])
+
+== Optimized Execution Plan ==
+HashAggregate(isMerge=[true], groupBy=[a], select=[a, Final_COUNT(count$0) AS b])
++- Exchange(distribution=[hash[a]])
+   +- LocalHashAggregate(groupBy=[a], select=[a, Partial_COUNT(b) AS count$0])
+      +- HashAggregate(isMerge=[true], groupBy=[a, b], select=[a, b])
+         +- Exchange(distribution=[hash[a, b]])
+            +- LocalHashAggregate(groupBy=[a, b], select=[a, b])
+               +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b], metadata=[]]], fields=[a, b])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: MyTable[]",
+    "pact" : "Data Source",
+    "contents" : "[]:TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b], metadata=[]]], fields=[a, b])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "HashAggregate[]",
+    "pact" : "Operator",
+    "contents" : "[]:LocalHashAggregate(groupBy=[a, b], select=[a, b])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "HashAggregate[]",
+    "pact" : "Operator",
+    "contents" : "[]:HashAggregate(isMerge=[true], groupBy=[a, b], select=[a, b])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "HASH[a, b]",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "HashAggregate[]",
+    "pact" : "Operator",
+    "contents" : "[]:LocalHashAggregate(groupBy=[a], select=[a, Partial_COUNT(b) AS count$0])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "HashAggregate[]",
+    "pact" : "Operator",
+    "contents" : "[]:HashAggregate(isMerge=[true], groupBy=[a], select=[a, Final_COUNT(count$0) AS b])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "HASH[a]",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testHashJoin[isNameSimplifyEnabled=false]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], a0=[$4], b0=[$5], c0=[$6], d0=[$7])
++- LogicalFilter(condition=[=($0, $7)])
+   +- LogicalJoin(condition=[true], joinType=[inner])
+      :- LogicalTableScan(table=[[default_catalog, default_database, A]])
+      +- LogicalTableScan(table=[[default_catalog, default_database, B]])
+
+== Optimized Physical Plan ==
+HashJoin(joinType=[InnerJoin], where=[=(a, d0)], select=[a, b, c, d, a0, b0, c0, d0], build=[right])
+:- Exchange(distribution=[hash[a]])
+:  +- TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a, b, c, d])
++- Exchange(distribution=[hash[d]])
+   +- TableSourceScan(table=[[default_catalog, default_database, B]], fields=[a, b, c, d])
+
+== Optimized Execution Plan ==
+HashJoin(joinType=[InnerJoin], where=[(a = d0)], select=[a, b, c, d, a0, b0, c0, d0], build=[right])
+:- Exchange(distribution=[hash[a]])
+:  +- TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a, b, c, d])
++- Exchange(distribution=[hash[d]])
+   +- TableSourceScan(table=[[default_catalog, default_database, B]], fields=[a, b, c, d])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a, b, c, d])",
+    "pact" : "Data Source",
+    "contents" : "TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a, b, c, d])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "Source: TableSourceScan(table=[[default_catalog, default_database, B]], fields=[a, b, c, d])",
+    "pact" : "Data Source",
+    "contents" : "TableSourceScan(table=[[default_catalog, default_database, B]], fields=[a, b, c, d])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "HashJoin(joinType=[InnerJoin], where=[(a = d0)], select=[a, b, c, d, a0, b0, c0, d0], build=[right])",
+    "pact" : "Operator",
+    "contents" : "HashJoin(joinType=[InnerJoin], where=[(a = d0)], select=[a, b, c, d, a0, b0, c0, d0], build=[right])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "HASH[d]",
+      "side" : "second"
+    }, {
+      "id" : ,
+      "ship_strategy" : "HASH[a]",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testHashJoin[isNameSimplifyEnabled=true]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], a0=[$4], b0=[$5], c0=[$6], d0=[$7])
++- LogicalFilter(condition=[=($0, $7)])
+   +- LogicalJoin(condition=[true], joinType=[inner])
+      :- LogicalTableScan(table=[[default_catalog, default_database, A]])
+      +- LogicalTableScan(table=[[default_catalog, default_database, B]])
+
+== Optimized Physical Plan ==
+HashJoin(joinType=[InnerJoin], where=[=(a, d0)], select=[a, b, c, d, a0, b0, c0, d0], build=[right])
+:- Exchange(distribution=[hash[a]])
+:  +- TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a, b, c, d])
++- Exchange(distribution=[hash[d]])
+   +- TableSourceScan(table=[[default_catalog, default_database, B]], fields=[a, b, c, d])
+
+== Optimized Execution Plan ==
+HashJoin(joinType=[InnerJoin], where=[(a = d0)], select=[a, b, c, d, a0, b0, c0, d0], build=[right])
+:- Exchange(distribution=[hash[a]])
+:  +- TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a, b, c, d])
++- Exchange(distribution=[hash[d]])
+   +- TableSourceScan(table=[[default_catalog, default_database, B]], fields=[a, b, c, d])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: A[]",
+    "pact" : "Data Source",
+    "contents" : "[]:TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a, b, c, d])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "Source: B[]",
+    "pact" : "Data Source",
+    "contents" : "[]:TableSourceScan(table=[[default_catalog, default_database, B]], fields=[a, b, c, d])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "HashJoin[]",
+    "pact" : "Operator",
+    "contents" : "[]:HashJoin(joinType=[InnerJoin], where=[(a = d0)], select=[a, b, c, d, a0, b0, c0, d0], build=[right])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "HASH[d]",
+      "side" : "second"
+    }, {
+      "id" : ,
+      "ship_strategy" : "HASH[a]",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testLegacySourceSink[isNameSimplifyEnabled=false]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalLegacySink(name=[`default_catalog`.`default_database`.`MySink`], fields=[name, id, amount, price])
++- LogicalProject(name=[$0], id=[$1], amount=[$2], price=[$3])
+   +- LogicalTableScan(table=[[default_catalog, default_database, MySource, source: [filterPushedDown=[false], filter=[]]]])
+
+== Optimized Physical Plan ==
+LegacySink(name=[`default_catalog`.`default_database`.`MySink`], fields=[name, id, amount, price])
++- LegacyTableSourceScan(table=[[default_catalog, default_database, MySource, source: [filterPushedDown=[false], filter=[]]]], fields=[name, id, amount, price])
+
+== Optimized Execution Plan ==
+LegacySink(name=[`default_catalog`.`default_database`.`MySink`], fields=[name, id, amount, price])
++- LegacyTableSourceScan(table=[[default_catalog, default_database, MySource, source: [filterPushedDown=[false], filter=[]]]], fields=[name, id, amount, price])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: Collection Source",
+    "pact" : "Data Source",
+    "contents" : "Source: Collection Source",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "SourceConversion(table=[default_catalog.default_database.MySource, source: [filterPushedDown=[false], filter=[]]], fields=[name, id, amount, price])",
+    "pact" : "Operator",
+    "contents" : "SourceConversion(table=[default_catalog.default_database.MySource, source: [filterPushedDown=[false], filter=[]]], fields=[name, id, amount, price])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "SinkConversion To Row",
+    "pact" : "Operator",
+    "contents" : "SinkConversion To Row",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Sink: collect",
+    "pact" : "Data Sink",
+    "contents" : "Sink: collect",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testLegacySourceSink[isNameSimplifyEnabled=true]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalLegacySink(name=[`default_catalog`.`default_database`.`MySink`], fields=[name, id, amount, price])
++- LogicalProject(name=[$0], id=[$1], amount=[$2], price=[$3])
+   +- LogicalTableScan(table=[[default_catalog, default_database, MySource, source: [filterPushedDown=[false], filter=[]]]])
+
+== Optimized Physical Plan ==
+LegacySink(name=[`default_catalog`.`default_database`.`MySink`], fields=[name, id, amount, price])
++- LegacyTableSourceScan(table=[[default_catalog, default_database, MySource, source: [filterPushedDown=[false], filter=[]]]], fields=[name, id, amount, price])
+
+== Optimized Execution Plan ==
+LegacySink(name=[`default_catalog`.`default_database`.`MySink`], fields=[name, id, amount, price])
++- LegacyTableSourceScan(table=[[default_catalog, default_database, MySource, source: [filterPushedDown=[false], filter=[]]]], fields=[name, id, amount, price])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: Collection Source",
+    "pact" : "Data Source",
+    "contents" : "Source: Collection Source",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "SourceConversion[]",
+    "pact" : "Operator",
+    "contents" : "[]:SourceConversion(table=[default_catalog.default_database.MySource, source: [filterPushedDown=[false], filter=[]]], fields=[name, id, amount, price])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "SinkConversion[]",
+    "pact" : "Operator",
+    "contents" : "[]:SinkConversion To Row",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Sink: collect",
+    "pact" : "Data Sink",
+    "contents" : "Sink: collect",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testLimit[isNameSimplifyEnabled=false]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalSort(fetch=[10])
++- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3])
+   +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
+
+== Optimized Physical Plan ==
+Limit(offset=[0], fetch=[10], global=[true])
++- Exchange(distribution=[single])
+   +- Limit(offset=[0], fetch=[10], global=[false])
+      +- TableSourceScan(table=[[default_catalog, default_database, MyTable, limit=[10]]], fields=[a, b, c, d])
+
+== Optimized Execution Plan ==
+Limit(offset=[0], fetch=[10], global=[true])
++- Exchange(distribution=[single])
+   +- Limit(offset=[0], fetch=[10], global=[false])
+      +- TableSourceScan(table=[[default_catalog, default_database, MyTable, limit=[10]]], fields=[a, b, c, d])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: TableSourceScan(table=[[default_catalog, default_database, MyTable, limit=[]]], fields=[a, b, c, d])",
+    "pact" : "Data Source",
+    "contents" : "TableSourceScan(table=[[default_catalog, default_database, MyTable, limit=[10]]], fields=[a, b, c, d])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "Limit(offset=[], fetch=[10], global=[false])",
+    "pact" : "Operator",
+    "contents" : "Limit(offset=[0], fetch=[10], global=[false])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Limit(offset=[], fetch=[10], global=[true])",
+    "pact" : "Operator",
+    "contents" : "Limit(offset=[0], fetch=[10], global=[true])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "GLOBAL",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testLimit[isNameSimplifyEnabled=true]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalSort(fetch=[10])
++- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3])
+   +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
+
+== Optimized Physical Plan ==
+Limit(offset=[0], fetch=[10], global=[true])
++- Exchange(distribution=[single])
+   +- Limit(offset=[0], fetch=[10], global=[false])
+      +- TableSourceScan(table=[[default_catalog, default_database, MyTable, limit=[10]]], fields=[a, b, c, d])
+
+== Optimized Execution Plan ==
+Limit(offset=[0], fetch=[10], global=[true])
++- Exchange(distribution=[single])
+   +- Limit(offset=[0], fetch=[10], global=[false])
+      +- TableSourceScan(table=[[default_catalog, default_database, MyTable, limit=[10]]], fields=[a, b, c, d])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: MyTable[]",
+    "pact" : "Data Source",
+    "contents" : "[]:TableSourceScan(table=[[default_catalog, default_database, MyTable, limit=[10]]], fields=[a, b, c, d])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "Limit[]",
+    "pact" : "Operator",
+    "contents" : "[]:Limit(offset=[0], fetch=[10], global=[false])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Limit[]",
+    "pact" : "Operator",
+    "contents" : "[]:Limit(offset=[0], fetch=[10], global=[true])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "GLOBAL",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testLookupJoin[isNameSimplifyEnabled=false]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], rowtime=[$4], proctime=[$5], id=[$6], name=[$7], age=[$8])
++- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{1, 5}])
+   :- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], rowtime=[$4], proctime=[PROCTIME()])
+   :  +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
+   +- LogicalFilter(condition=[=($cor0.b, $0)])
+      +- LogicalSnapshot(period=[$cor0.proctime])
+         +- LogicalTableScan(table=[[default_catalog, default_database, LookupTable]])
+
+== Optimized Physical Plan ==
+Calc(select=[a, b, c, d, rowtime, PROCTIME_MATERIALIZE(proctime) AS proctime, id, name, age])
++- LookupJoin(table=[default_catalog.default_database.LookupTable], joinType=[InnerJoin], async=[false], lookup=[id=b], select=[a, b, c, d, rowtime, proctime, id, name, age])
+   +- Calc(select=[a, b, c, d, rowtime, PROCTIME() AS proctime])
+      +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, rowtime])
+
+== Optimized Execution Plan ==
+Calc(select=[a, b, c, d, rowtime, PROCTIME_MATERIALIZE(proctime) AS proctime, id, name, age])
++- LookupJoin(table=[default_catalog.default_database.LookupTable], joinType=[InnerJoin], async=[false], lookup=[id=b], select=[a, b, c, d, rowtime, proctime, id, name, age])
+   +- Calc(select=[a, b, c, d, rowtime, PROCTIME() AS proctime])
+      +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, rowtime])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, rowtime])",
+    "pact" : "Data Source",
+    "contents" : "TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, rowtime])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "Calc(select=[a, b, c, d, rowtime, PROCTIME() AS proctime])",
+    "pact" : "Operator",
+    "contents" : "Calc(select=[a, b, c, d, rowtime, PROCTIME() AS proctime])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "LookupJoin(table=[default_catalog.default_database.LookupTable], joinType=[InnerJoin], async=[false], lookup=[id=b], select=[a, b, c, d, rowtime, proctime, id, name, age])",
+    "pact" : "Operator",
+    "contents" : "LookupJoin(table=[default_catalog.default_database.LookupTable], joinType=[InnerJoin], async=[false], lookup=[id=b], select=[a, b, c, d, rowtime, proctime, id, name, age])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Calc(select=[a, b, c, d, rowtime, PROCTIME_MATERIALIZE(proctime) AS proctime, id, name, age])",
+    "pact" : "Operator",
+    "contents" : "Calc(select=[a, b, c, d, rowtime, PROCTIME_MATERIALIZE(proctime) AS proctime, id, name, age])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testLookupJoin[isNameSimplifyEnabled=true]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], rowtime=[$4], proctime=[$5], id=[$6], name=[$7], age=[$8])
++- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{1, 5}])
+   :- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], rowtime=[$4], proctime=[PROCTIME()])
+   :  +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
+   +- LogicalFilter(condition=[=($cor0.b, $0)])
+      +- LogicalSnapshot(period=[$cor0.proctime])
+         +- LogicalTableScan(table=[[default_catalog, default_database, LookupTable]])
+
+== Optimized Physical Plan ==
+Calc(select=[a, b, c, d, rowtime, PROCTIME_MATERIALIZE(proctime) AS proctime, id, name, age])
++- LookupJoin(table=[default_catalog.default_database.LookupTable], joinType=[InnerJoin], async=[false], lookup=[id=b], select=[a, b, c, d, rowtime, proctime, id, name, age])
+   +- Calc(select=[a, b, c, d, rowtime, PROCTIME() AS proctime])
+      +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, rowtime])
+
+== Optimized Execution Plan ==
+Calc(select=[a, b, c, d, rowtime, PROCTIME_MATERIALIZE(proctime) AS proctime, id, name, age])
++- LookupJoin(table=[default_catalog.default_database.LookupTable], joinType=[InnerJoin], async=[false], lookup=[id=b], select=[a, b, c, d, rowtime, proctime, id, name, age])
+   +- Calc(select=[a, b, c, d, rowtime, PROCTIME() AS proctime])
+      +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, rowtime])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: MyTable[]",
+    "pact" : "Data Source",
+    "contents" : "[]:TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, rowtime])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "Calc[]",
+    "pact" : "Operator",
+    "contents" : "[]:Calc(select=[a, b, c, d, rowtime, PROCTIME() AS proctime])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "LookupJoin[]",
+    "pact" : "Operator",
+    "contents" : "[]:LookupJoin(table=[default_catalog.default_database.LookupTable], joinType=[InnerJoin], async=[false], lookup=[id=b], select=[a, b, c, d, rowtime, proctime, id, name, age])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Calc[]",
+    "pact" : "Operator",
+    "contents" : "[]:Calc(select=[a, b, c, d, rowtime, PROCTIME_MATERIALIZE(proctime) AS proctime, id, name, age])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testMultiInput[isNameSimplifyEnabled=false]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], a0=[$4], b0=[$5], c0=[$6], d0=[$7], a1=[$8], b1=[$9], c1=[$10], d1=[$11])
++- LogicalFilter(condition=[AND(=($0, $4), =($0, $8))])
+   +- LogicalJoin(condition=[true], joinType=[inner])
+      :- LogicalJoin(condition=[true], joinType=[inner])
+      :  :- LogicalTableScan(table=[[default_catalog, default_database, A]])
+      :  +- LogicalTableScan(table=[[default_catalog, default_database, B]])
+      +- LogicalTableScan(table=[[default_catalog, default_database, C]])
+
+== Optimized Physical Plan ==
+HashJoin(joinType=[InnerJoin], where=[=(a, a1)], select=[a, b, c, d, a0, b0, c0, d0, a1, b1, c1, d1], build=[left])
+:- HashJoin(joinType=[InnerJoin], where=[=(a, a0)], select=[a, b, c, d, a0, b0, c0, d0], build=[right])
+:  :- Exchange(distribution=[hash[a]])
+:  :  +- TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a, b, c, d])
+:  +- Exchange(distribution=[hash[a]])
+:     +- TableSourceScan(table=[[default_catalog, default_database, B]], fields=[a, b, c, d])
++- Exchange(distribution=[hash[a]])
+   +- TableSourceScan(table=[[default_catalog, default_database, C]], fields=[a, b, c, d])
+
+== Optimized Execution Plan ==
+MultipleInput(readOrder=[2,1,0], members=[\nHashJoin(joinType=[InnerJoin], where=[(a = a1)], select=[a, b, c, d, a0, b0, c0, d0, a1, b1, c1, d1], build=[left])\n:- HashJoin(joinType=[InnerJoin], where=[(a = a0)], select=[a, b, c, d, a0, b0, c0, d0], build=[right])\n:  :- [#2] Exchange(distribution=[hash[a]])\n:  +- [#3] Exchange(distribution=[hash[a]])\n+- [#1] Exchange(distribution=[hash[a]])\n])
+:- Exchange(distribution=[hash[a]])
+:  +- TableSourceScan(table=[[default_catalog, default_database, C]], fields=[a, b, c, d])
+:- Exchange(distribution=[hash[a]])
+:  +- TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a, b, c, d])
++- Exchange(distribution=[hash[a]])
+   +- TableSourceScan(table=[[default_catalog, default_database, B]], fields=[a, b, c, d])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: TableSourceScan(table=[[default_catalog, default_database, C]], fields=[a, b, c, d])",
+    "pact" : "Data Source",
+    "contents" : "TableSourceScan(table=[[default_catalog, default_database, C]], fields=[a, b, c, d])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "Source: TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a, b, c, d])",
+    "pact" : "Data Source",
+    "contents" : "TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a, b, c, d])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "Source: TableSourceScan(table=[[default_catalog, default_database, B]], fields=[a, b, c, d])",
+    "pact" : "Data Source",
+    "contents" : "TableSourceScan(table=[[default_catalog, default_database, B]], fields=[a, b, c, d])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "MultipleInput(readOrder=[2,1,0], members=[\\nHashJoin(joinType=[InnerJoin], where=[(a = a1)], select=[a, b, c, d, a0, b0, c0, d0, a1, b1, c1, d1], build=[left])\\n:- HashJoin(joinType=[InnerJoin], where=[(a = a0)], select=[a, b, c, d, a0, b0, c0, d0], build=[right])\\n:  :- [#2] Exchange(distribution=[hash[a]])\\n:  +- [#3] Exchange(distribution=[hash[a]])\\n+- [#1] Exchange(distribution=[hash[a]])\\n])",
+    "pact" : "Operator",
+    "contents" : "MultipleInput(readOrder=[2,1,0], members=[\\nHashJoin(joinType=[InnerJoin], where=[(a = a1)], select=[a, b, c, d, a0, b0, c0, d0, a1, b1, c1, d1], build=[left])\\n:- HashJoin(joinType=[InnerJoin], where=[(a = a0)], select=[a, b, c, d, a0, b0, c0, d0], build=[right])\\n:  :- [#2] Exchange(distribution=[hash[a]])\\n:  +- [#3] Exchange(distribution=[hash[a]])\\n+- [#1] Exchange(distribution=[hash[a]])\\n])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "HASH[a]",
+      "side" : "second"
+    }, {
+      "id" : ,
+      "ship_strategy" : "HASH[a]",
+      "side" : "second"
+    }, {
+      "id" : ,
+      "ship_strategy" : "HASH[a]",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testMultiInput[isNameSimplifyEnabled=true]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], a0=[$4], b0=[$5], c0=[$6], d0=[$7], a1=[$8], b1=[$9], c1=[$10], d1=[$11])
++- LogicalFilter(condition=[AND(=($0, $4), =($0, $8))])
+   +- LogicalJoin(condition=[true], joinType=[inner])
+      :- LogicalJoin(condition=[true], joinType=[inner])
+      :  :- LogicalTableScan(table=[[default_catalog, default_database, A]])
+      :  +- LogicalTableScan(table=[[default_catalog, default_database, B]])
+      +- LogicalTableScan(table=[[default_catalog, default_database, C]])
+
+== Optimized Physical Plan ==
+HashJoin(joinType=[InnerJoin], where=[=(a, a1)], select=[a, b, c, d, a0, b0, c0, d0, a1, b1, c1, d1], build=[left])
+:- HashJoin(joinType=[InnerJoin], where=[=(a, a0)], select=[a, b, c, d, a0, b0, c0, d0], build=[right])
+:  :- Exchange(distribution=[hash[a]])
+:  :  +- TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a, b, c, d])
+:  +- Exchange(distribution=[hash[a]])
+:     +- TableSourceScan(table=[[default_catalog, default_database, B]], fields=[a, b, c, d])
++- Exchange(distribution=[hash[a]])
+   +- TableSourceScan(table=[[default_catalog, default_database, C]], fields=[a, b, c, d])
+
+== Optimized Execution Plan ==
+MultipleInput(readOrder=[2,1,0], members=[\nHashJoin(joinType=[InnerJoin], where=[(a = a1)], select=[a, b, c, d, a0, b0, c0, d0, a1, b1, c1, d1], build=[left])\n:- HashJoin(joinType=[InnerJoin], where=[(a = a0)], select=[a, b, c, d, a0, b0, c0, d0], build=[right])\n:  :- [#2] Exchange(distribution=[hash[a]])\n:  +- [#3] Exchange(distribution=[hash[a]])\n+- [#1] Exchange(distribution=[hash[a]])\n])
+:- Exchange(distribution=[hash[a]])
+:  +- TableSourceScan(table=[[default_catalog, default_database, C]], fields=[a, b, c, d])
+:- Exchange(distribution=[hash[a]])
+:  +- TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a, b, c, d])
++- Exchange(distribution=[hash[a]])
+   +- TableSourceScan(table=[[default_catalog, default_database, B]], fields=[a, b, c, d])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: C[]",
+    "pact" : "Data Source",
+    "contents" : "[]:TableSourceScan(table=[[default_catalog, default_database, C]], fields=[a, b, c, d])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "Source: A[]",
+    "pact" : "Data Source",
+    "contents" : "[]:TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a, b, c, d])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "Source: B[]",
+    "pact" : "Data Source",
+    "contents" : "[]:TableSourceScan(table=[[default_catalog, default_database, B]], fields=[a, b, c, d])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "MultipleInput[]",
+    "pact" : "Operator",
+    "contents" : "[]:MultipleInput(readOrder=[2,1,0], members=[\\nHashJoin(joinType=[InnerJoin], where=[(a = a1)], select=[a, b, c, d, a0, b0, c0, d0, a1, b1, c1, d1], build=[left])\\n:- HashJoin(joinType=[InnerJoin], where=[(a = a0)], select=[a, b, c, d, a0, b0, c0, d0], build=[right])\\n:  :- [#2] Exchange(distribution=[hash[a]])\\n:  +- [#3] Exchange(distribution=[hash[a]])\\n+- [#1] Exchange(distribution=[hash[a]])\\n])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "HASH[a]",
+      "side" : "second"
+    }, {
+      "id" : ,
+      "ship_strategy" : "HASH[a]",
+      "side" : "second"
+    }, {
+      "id" : ,
+      "ship_strategy" : "HASH[a]",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testNestedLoopJoin[isNameSimplifyEnabled=false]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], a0=[$4], b0=[$5], c0=[$6], d0=[$7])
++- LogicalFilter(condition=[=($0, $7)])
+   +- LogicalJoin(condition=[true], joinType=[inner])
+      :- LogicalTableScan(table=[[default_catalog, default_database, A]])
+      +- LogicalTableScan(table=[[default_catalog, default_database, B]])
+
+== Optimized Physical Plan ==
+NestedLoopJoin(joinType=[InnerJoin], where=[=(a, d0)], select=[a, b, c, d, a0, b0, c0, d0], build=[left])
+:- Exchange(distribution=[broadcast])
+:  +- TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a, b, c, d])
++- TableSourceScan(table=[[default_catalog, default_database, B]], fields=[a, b, c, d])
+
+== Optimized Execution Plan ==
+NestedLoopJoin(joinType=[InnerJoin], where=[(a = d0)], select=[a, b, c, d, a0, b0, c0, d0], build=[left])
+:- Exchange(distribution=[broadcast])
+:  +- TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a, b, c, d])
++- TableSourceScan(table=[[default_catalog, default_database, B]], fields=[a, b, c, d])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: TableSourceScan(table=[[default_catalog, default_database, B]], fields=[a, b, c, d])",
+    "pact" : "Data Source",
+    "contents" : "TableSourceScan(table=[[default_catalog, default_database, B]], fields=[a, b, c, d])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "Source: TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a, b, c, d])",
+    "pact" : "Data Source",
+    "contents" : "TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a, b, c, d])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "NestedLoopJoin(joinType=[InnerJoin], where=[(a = d0)], select=[a, b, c, d, a0, b0, c0, d0], build=[left])",
+    "pact" : "Operator",
+    "contents" : "NestedLoopJoin(joinType=[InnerJoin], where=[(a = d0)], select=[a, b, c, d, a0, b0, c0, d0], build=[left])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "BROADCAST",
+      "side" : "second"
+    }, {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testNestedLoopJoin[isNameSimplifyEnabled=true]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], a0=[$4], b0=[$5], c0=[$6], d0=[$7])
++- LogicalFilter(condition=[=($0, $7)])
+   +- LogicalJoin(condition=[true], joinType=[inner])
+      :- LogicalTableScan(table=[[default_catalog, default_database, A]])
+      +- LogicalTableScan(table=[[default_catalog, default_database, B]])
+
+== Optimized Physical Plan ==
+NestedLoopJoin(joinType=[InnerJoin], where=[=(a, d0)], select=[a, b, c, d, a0, b0, c0, d0], build=[left])
+:- Exchange(distribution=[broadcast])
+:  +- TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a, b, c, d])
++- TableSourceScan(table=[[default_catalog, default_database, B]], fields=[a, b, c, d])
+
+== Optimized Execution Plan ==
+NestedLoopJoin(joinType=[InnerJoin], where=[(a = d0)], select=[a, b, c, d, a0, b0, c0, d0], build=[left])
+:- Exchange(distribution=[broadcast])
+:  +- TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a, b, c, d])
++- TableSourceScan(table=[[default_catalog, default_database, B]], fields=[a, b, c, d])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: B[]",
+    "pact" : "Data Source",
+    "contents" : "[]:TableSourceScan(table=[[default_catalog, default_database, B]], fields=[a, b, c, d])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "Source: A[]",
+    "pact" : "Data Source",
+    "contents" : "[]:TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a, b, c, d])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "NestedLoopJoin[]",
+    "pact" : "Operator",
+    "contents" : "[]:NestedLoopJoin(joinType=[InnerJoin], where=[(a = d0)], select=[a, b, c, d, a0, b0, c0, d0], build=[left])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "BROADCAST",
+      "side" : "second"
+    }, {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testOverAggregate[isNameSimplifyEnabled=false]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalProject(b=[$1], cnt1=[COUNT($0) OVER (PARTITION BY $1 ORDER BY $4 NULLS FIRST ROWS 2 PRECEDING)])
++- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
+
+== Optimized Physical Plan ==
+Calc(select=[b, w0$o0 AS $1])
++- OverAggregate(partitionBy=[b], orderBy=[rowtime ASC], window#0=[COUNT(a) AS w0$o0 ROWS BETWEEN 2 PRECEDING AND CURRENT ROW], select=[a, b, rowtime, w0$o0])
+   +- Sort(orderBy=[b ASC, rowtime ASC])
+      +- Exchange(distribution=[hash[b]])
+         +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b, rowtime], metadata=[]]], fields=[a, b, rowtime])
+
+== Optimized Execution Plan ==
+Calc(select=[b, w0$o0 AS $1])
++- OverAggregate(partitionBy=[b], orderBy=[rowtime ASC], window#0=[COUNT(a) AS w0$o0 ROWS BETWEEN 2 PRECEDING AND CURRENT ROW], select=[a, b, rowtime, w0$o0])
+   +- Sort(orderBy=[b ASC, rowtime ASC])
+      +- Exchange(distribution=[hash[b]])
+         +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b, rowtime], metadata=[]]], fields=[a, b, rowtime])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b, rowtime], metadata=[]]], fields=[a, b, rowtime])",
+    "pact" : "Data Source",
+    "contents" : "TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b, rowtime], metadata=[]]], fields=[a, b, rowtime])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "Sort(orderBy=[b ASC, rowtime ASC])",
+    "pact" : "Operator",
+    "contents" : "Sort(orderBy=[b ASC, rowtime ASC])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "HASH[b]",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "OverAggregate(partitionBy=[b], orderBy=[rowtime ASC], window#0=[COUNT(a) AS w0$o0 ROWS BETWEEN 2 PRECEDING AND CURRENT ROW], select=[a, b, rowtime, w0$o0])",
+    "pact" : "Operator",
+    "contents" : "OverAggregate(partitionBy=[b], orderBy=[rowtime ASC], window#0=[COUNT(a) AS w0$o0 ROWS BETWEEN 2 PRECEDING AND CURRENT ROW], select=[a, b, rowtime, w0$o0])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Calc(select=[b, w0$o0 AS $1])",
+    "pact" : "Operator",
+    "contents" : "Calc(select=[b, w0$o0 AS $1])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testOverAggregate[isNameSimplifyEnabled=true]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalProject(b=[$1], cnt1=[COUNT($0) OVER (PARTITION BY $1 ORDER BY $4 NULLS FIRST ROWS 2 PRECEDING)])
++- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
+
+== Optimized Physical Plan ==
+Calc(select=[b, w0$o0 AS $1])
++- OverAggregate(partitionBy=[b], orderBy=[rowtime ASC], window#0=[COUNT(a) AS w0$o0 ROWS BETWEEN 2 PRECEDING AND CURRENT ROW], select=[a, b, rowtime, w0$o0])
+   +- Sort(orderBy=[b ASC, rowtime ASC])
+      +- Exchange(distribution=[hash[b]])
+         +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b, rowtime], metadata=[]]], fields=[a, b, rowtime])
+
+== Optimized Execution Plan ==
+Calc(select=[b, w0$o0 AS $1])
++- OverAggregate(partitionBy=[b], orderBy=[rowtime ASC], window#0=[COUNT(a) AS w0$o0 ROWS BETWEEN 2 PRECEDING AND CURRENT ROW], select=[a, b, rowtime, w0$o0])
+   +- Sort(orderBy=[b ASC, rowtime ASC])
+      +- Exchange(distribution=[hash[b]])
+         +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b, rowtime], metadata=[]]], fields=[a, b, rowtime])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: MyTable[]",
+    "pact" : "Data Source",
+    "contents" : "[]:TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b, rowtime], metadata=[]]], fields=[a, b, rowtime])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "Sort[]",
+    "pact" : "Operator",
+    "contents" : "[]:Sort(orderBy=[b ASC, rowtime ASC])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "HASH[b]",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "OverAggregate[]",
+    "pact" : "Operator",
+    "contents" : "[]:OverAggregate(partitionBy=[b], orderBy=[rowtime ASC], window#0=[COUNT(a) AS w0$o0 ROWS BETWEEN 2 PRECEDING AND CURRENT ROW], select=[a, b, rowtime, w0$o0])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Calc[]",
+    "pact" : "Operator",
+    "contents" : "[]:Calc(select=[b, w0$o0 AS $1])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testRank[isNameSimplifyEnabled=false]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalProject(a=[$0], row_num=[$1])
++- LogicalFilter(condition=[<=($1, $0)])
+   +- LogicalProject(a=[$0], row_num=[ROW_NUMBER() OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST)])
+      +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
+
+== Optimized Physical Plan ==
+Calc(select=[a, w0$o0], where=[<=(w0$o0, a)])
++- OverAggregate(partitionBy=[b], orderBy=[a ASC], window#0=[ROW_NUMBER(*) AS w0$o0 ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[a, b, w0$o0])
+   +- Sort(orderBy=[b ASC, a ASC])
+      +- Exchange(distribution=[hash[b]])
+         +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b], metadata=[]]], fields=[a, b])
+
+== Optimized Execution Plan ==
+Calc(select=[a, w0$o0], where=[(w0$o0 <= a)])
++- OverAggregate(partitionBy=[b], orderBy=[a ASC], window#0=[ROW_NUMBER(*) AS w0$o0 ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[a, b, w0$o0])
+   +- Sort(orderBy=[b ASC, a ASC])
+      +- Exchange(distribution=[hash[b]])
+         +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b], metadata=[]]], fields=[a, b])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b], metadata=[]]], fields=[a, b])",
+    "pact" : "Data Source",
+    "contents" : "TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b], metadata=[]]], fields=[a, b])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "Sort(orderBy=[b ASC, a ASC])",
+    "pact" : "Operator",
+    "contents" : "Sort(orderBy=[b ASC, a ASC])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "HASH[b]",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "OverAggregate(partitionBy=[b], orderBy=[a ASC], window#0=[ROW_NUMBER(*) AS w0$o0 ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[a, b, w0$o0])",
+    "pact" : "Operator",
+    "contents" : "OverAggregate(partitionBy=[b], orderBy=[a ASC], window#0=[ROW_NUMBER(*) AS w0$o0 ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[a, b, w0$o0])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Calc(select=[a, w0$o0], where=[(w0$o0 <= a)])",
+    "pact" : "Operator",
+    "contents" : "Calc(select=[a, w0$o0], where=[(w0$o0 <= a)])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testRank[isNameSimplifyEnabled=true]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalProject(a=[$0], row_num=[$1])
++- LogicalFilter(condition=[<=($1, $0)])
+   +- LogicalProject(a=[$0], row_num=[ROW_NUMBER() OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST)])
+      +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
+
+== Optimized Physical Plan ==
+Calc(select=[a, w0$o0], where=[<=(w0$o0, a)])
++- OverAggregate(partitionBy=[b], orderBy=[a ASC], window#0=[ROW_NUMBER(*) AS w0$o0 ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[a, b, w0$o0])
+   +- Sort(orderBy=[b ASC, a ASC])
+      +- Exchange(distribution=[hash[b]])
+         +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b], metadata=[]]], fields=[a, b])
+
+== Optimized Execution Plan ==
+Calc(select=[a, w0$o0], where=[(w0$o0 <= a)])
++- OverAggregate(partitionBy=[b], orderBy=[a ASC], window#0=[ROW_NUMBER(*) AS w0$o0 ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[a, b, w0$o0])
+   +- Sort(orderBy=[b ASC, a ASC])
+      +- Exchange(distribution=[hash[b]])
+         +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b], metadata=[]]], fields=[a, b])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: MyTable[]",
+    "pact" : "Data Source",
+    "contents" : "[]:TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b], metadata=[]]], fields=[a, b])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "Sort[]",
+    "pact" : "Operator",
+    "contents" : "[]:Sort(orderBy=[b ASC, a ASC])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "HASH[b]",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "OverAggregate[]",
+    "pact" : "Operator",
+    "contents" : "[]:OverAggregate(partitionBy=[b], orderBy=[a ASC], window#0=[ROW_NUMBER(*) AS w0$o0 ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[a, b, w0$o0])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Calc[]",
+    "pact" : "Operator",
+    "contents" : "[]:Calc(select=[a, w0$o0], where=[(w0$o0 <= a)])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testSortAggregate[isNameSimplifyEnabled=false]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalAggregate(group=[{0}], b=[COUNT(DISTINCT $1)])
++- LogicalProject(a=[$0], b=[$1])
+   +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
+
+== Optimized Physical Plan ==
+SortAggregate(isMerge=[true], groupBy=[a], select=[a, Final_COUNT(count$0) AS b])
++- Sort(orderBy=[a ASC])
+   +- Exchange(distribution=[hash[a]])
+      +- LocalSortAggregate(groupBy=[a], select=[a, Partial_COUNT(b) AS count$0])
+         +- Sort(orderBy=[a ASC])
+            +- SortAggregate(isMerge=[true], groupBy=[a, b], select=[a, b])
+               +- Sort(orderBy=[a ASC, b ASC])
+                  +- Exchange(distribution=[hash[a, b]])
+                     +- LocalSortAggregate(groupBy=[a, b], select=[a, b])
+                        +- Sort(orderBy=[a ASC, b ASC])
+                           +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b], metadata=[]]], fields=[a, b])
+
+== Optimized Execution Plan ==
+SortAggregate(isMerge=[true], groupBy=[a], select=[a, Final_COUNT(count$0) AS b])
++- Sort(orderBy=[a ASC])
+   +- Exchange(distribution=[hash[a]])
+      +- LocalSortAggregate(groupBy=[a], select=[a, Partial_COUNT(b) AS count$0])
+         +- Sort(orderBy=[a ASC])
+            +- SortAggregate(isMerge=[true], groupBy=[a, b], select=[a, b])
+               +- Sort(orderBy=[a ASC, b ASC])
+                  +- Exchange(distribution=[hash[a, b]])
+                     +- LocalSortAggregate(groupBy=[a, b], select=[a, b])
+                        +- Sort(orderBy=[a ASC, b ASC])
+                           +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b], metadata=[]]], fields=[a, b])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b], metadata=[]]], fields=[a, b])",
+    "pact" : "Data Source",
+    "contents" : "TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b], metadata=[]]], fields=[a, b])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "Sort(orderBy=[a ASC, b ASC])",
+    "pact" : "Operator",
+    "contents" : "Sort(orderBy=[a ASC, b ASC])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "LocalSortAggregate(groupBy=[a, b], select=[a, b])",
+    "pact" : "Operator",
+    "contents" : "LocalSortAggregate(groupBy=[a, b], select=[a, b])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Sort(orderBy=[a ASC, b ASC])",
+    "pact" : "Operator",
+    "contents" : "Sort(orderBy=[a ASC, b ASC])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "HASH[a, b]",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "SortAggregate(isMerge=[true], groupBy=[a, b], select=[a, b])",
+    "pact" : "Operator",
+    "contents" : "SortAggregate(isMerge=[true], groupBy=[a, b], select=[a, b])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Sort(orderBy=[a ASC])",
+    "pact" : "Operator",
+    "contents" : "Sort(orderBy=[a ASC])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "LocalSortAggregate(groupBy=[a], select=[a, Partial_COUNT(b) AS count$0])",
+    "pact" : "Operator",
+    "contents" : "LocalSortAggregate(groupBy=[a], select=[a, Partial_COUNT(b) AS count$0])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Sort(orderBy=[a ASC])",
+    "pact" : "Operator",
+    "contents" : "Sort(orderBy=[a ASC])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "HASH[a]",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "SortAggregate(isMerge=[true], groupBy=[a], select=[a, Final_COUNT(count$0) AS b])",
+    "pact" : "Operator",
+    "contents" : "SortAggregate(isMerge=[true], groupBy=[a], select=[a, Final_COUNT(count$0) AS b])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testSortAggregate[isNameSimplifyEnabled=true]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalAggregate(group=[{0}], b=[COUNT(DISTINCT $1)])
++- LogicalProject(a=[$0], b=[$1])
+   +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
+
+== Optimized Physical Plan ==
+SortAggregate(isMerge=[true], groupBy=[a], select=[a, Final_COUNT(count$0) AS b])
++- Sort(orderBy=[a ASC])
+   +- Exchange(distribution=[hash[a]])
+      +- LocalSortAggregate(groupBy=[a], select=[a, Partial_COUNT(b) AS count$0])
+         +- Sort(orderBy=[a ASC])
+            +- SortAggregate(isMerge=[true], groupBy=[a, b], select=[a, b])
+               +- Sort(orderBy=[a ASC, b ASC])
+                  +- Exchange(distribution=[hash[a, b]])
+                     +- LocalSortAggregate(groupBy=[a, b], select=[a, b])
+                        +- Sort(orderBy=[a ASC, b ASC])
+                           +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b], metadata=[]]], fields=[a, b])
+
+== Optimized Execution Plan ==
+SortAggregate(isMerge=[true], groupBy=[a], select=[a, Final_COUNT(count$0) AS b])
++- Sort(orderBy=[a ASC])
+   +- Exchange(distribution=[hash[a]])
+      +- LocalSortAggregate(groupBy=[a], select=[a, Partial_COUNT(b) AS count$0])
+         +- Sort(orderBy=[a ASC])
+            +- SortAggregate(isMerge=[true], groupBy=[a, b], select=[a, b])
+               +- Sort(orderBy=[a ASC, b ASC])
+                  +- Exchange(distribution=[hash[a, b]])
+                     +- LocalSortAggregate(groupBy=[a, b], select=[a, b])
+                        +- Sort(orderBy=[a ASC, b ASC])
+                           +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b], metadata=[]]], fields=[a, b])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: MyTable[]",
+    "pact" : "Data Source",
+    "contents" : "[]:TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b], metadata=[]]], fields=[a, b])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "Sort[]",
+    "pact" : "Operator",
+    "contents" : "[]:Sort(orderBy=[a ASC, b ASC])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "SortAggregate[]",
+    "pact" : "Operator",
+    "contents" : "[]:LocalSortAggregate(groupBy=[a, b], select=[a, b])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Sort[]",
+    "pact" : "Operator",
+    "contents" : "[]:Sort(orderBy=[a ASC, b ASC])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "HASH[a, b]",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "SortAggregate[]",
+    "pact" : "Operator",
+    "contents" : "[]:SortAggregate(isMerge=[true], groupBy=[a, b], select=[a, b])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Sort[]",
+    "pact" : "Operator",
+    "contents" : "[]:Sort(orderBy=[a ASC])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "SortAggregate[]",
+    "pact" : "Operator",
+    "contents" : "[]:LocalSortAggregate(groupBy=[a], select=[a, Partial_COUNT(b) AS count$0])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Sort[]",
+    "pact" : "Operator",
+    "contents" : "[]:Sort(orderBy=[a ASC])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "HASH[a]",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "SortAggregate[]",
+    "pact" : "Operator",
+    "contents" : "[]:SortAggregate(isMerge=[true], groupBy=[a], select=[a, Final_COUNT(count$0) AS b])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testSortLimit[isNameSimplifyEnabled=false]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalSort(sort0=[$0], dir0=[ASC-nulls-first], fetch=[10])
++- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3])
+   +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
+
+== Optimized Physical Plan ==
+SortLimit(orderBy=[a ASC], offset=[0], fetch=[10], global=[true])
++- Exchange(distribution=[single])
+   +- SortLimit(orderBy=[a ASC], offset=[0], fetch=[10], global=[false])
+      +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d])
+
+== Optimized Execution Plan ==
+SortLimit(orderBy=[a ASC], offset=[0], fetch=[10], global=[true])
++- Exchange(distribution=[single])
+   +- SortLimit(orderBy=[a ASC], offset=[0], fetch=[10], global=[false])
+      +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d])",
+    "pact" : "Data Source",
+    "contents" : "TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "SortLimit(orderBy=[a ASC], offset=[], fetch=[10], global=[false])",
+    "pact" : "Operator",
+    "contents" : "SortLimit(orderBy=[a ASC], offset=[0], fetch=[10], global=[false])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "SortLimit(orderBy=[a ASC], offset=[], fetch=[10], global=[true])",
+    "pact" : "Operator",
+    "contents" : "SortLimit(orderBy=[a ASC], offset=[0], fetch=[10], global=[true])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "GLOBAL",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testSortLimit[isNameSimplifyEnabled=true]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalSort(sort0=[$0], dir0=[ASC-nulls-first], fetch=[10])
++- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3])
+   +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
+
+== Optimized Physical Plan ==
+SortLimit(orderBy=[a ASC], offset=[0], fetch=[10], global=[true])
++- Exchange(distribution=[single])
+   +- SortLimit(orderBy=[a ASC], offset=[0], fetch=[10], global=[false])
+      +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d])
+
+== Optimized Execution Plan ==
+SortLimit(orderBy=[a ASC], offset=[0], fetch=[10], global=[true])
++- Exchange(distribution=[single])
+   +- SortLimit(orderBy=[a ASC], offset=[0], fetch=[10], global=[false])
+      +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: MyTable[]",
+    "pact" : "Data Source",
+    "contents" : "[]:TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "SortLimit[]",
+    "pact" : "Operator",
+    "contents" : "[]:SortLimit(orderBy=[a ASC], offset=[0], fetch=[10], global=[false])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "SortLimit[]",
+    "pact" : "Operator",
+    "contents" : "[]:SortLimit(orderBy=[a ASC], offset=[0], fetch=[10], global=[true])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "GLOBAL",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testSortMergeJoin[isNameSimplifyEnabled=false]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], a0=[$4], b0=[$5], c0=[$6], d0=[$7])
++- LogicalFilter(condition=[=($0, $7)])
+   +- LogicalJoin(condition=[true], joinType=[inner])
+      :- LogicalTableScan(table=[[default_catalog, default_database, A]])
+      +- LogicalTableScan(table=[[default_catalog, default_database, B]])
+
+== Optimized Physical Plan ==
+SortMergeJoin(joinType=[InnerJoin], where=[=(a, d0)], select=[a, b, c, d, a0, b0, c0, d0])
+:- Exchange(distribution=[hash[a]])
+:  +- TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a, b, c, d])
++- Exchange(distribution=[hash[d]])
+   +- TableSourceScan(table=[[default_catalog, default_database, B]], fields=[a, b, c, d])
+
+== Optimized Execution Plan ==
+SortMergeJoin(joinType=[InnerJoin], where=[(a = d0)], select=[a, b, c, d, a0, b0, c0, d0])
+:- Exchange(distribution=[hash[a]])
+:  +- TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a, b, c, d])
++- Exchange(distribution=[hash[d]])
+   +- TableSourceScan(table=[[default_catalog, default_database, B]], fields=[a, b, c, d])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a, b, c, d])",
+    "pact" : "Data Source",
+    "contents" : "TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a, b, c, d])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "Source: TableSourceScan(table=[[default_catalog, default_database, B]], fields=[a, b, c, d])",
+    "pact" : "Data Source",
+    "contents" : "TableSourceScan(table=[[default_catalog, default_database, B]], fields=[a, b, c, d])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "SortMergeJoin(joinType=[InnerJoin], where=[(a = d0)], select=[a, b, c, d, a0, b0, c0, d0])",
+    "pact" : "Operator",
+    "contents" : "SortMergeJoin(joinType=[InnerJoin], where=[(a = d0)], select=[a, b, c, d, a0, b0, c0, d0])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "HASH[a]",
+      "side" : "second"
+    }, {
+      "id" : ,
+      "ship_strategy" : "HASH[d]",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testSortWindowAggregate[isNameSimplifyEnabled=true]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalProject(b=[$0], window_end=[TUMBLE_END($1)], EXPR$2=[$2])
++- LogicalAggregate(group=[{0, 1}], EXPR$2=[FIRST_VALUE($2)])
+   +- LogicalProject(b=[$1], $f1=[$TUMBLE($4, 900000:INTERVAL MINUTE)], a=[$0])
+      +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
+
+== Optimized Physical Plan ==
+Calc(select=[b, w$end AS window_end, EXPR$2])
++- SortWindowAggregate(groupBy=[b], window=[TumblingGroupWindow('w$, rowtime, 900000)], properties=[w$start, w$end, w$rowtime], select=[b, FIRST_VALUE(a) AS EXPR$2])
+   +- Sort(orderBy=[b ASC, rowtime ASC])
+      +- Exchange(distribution=[hash[b]])
+         +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[b, rowtime, a], metadata=[]]], fields=[b, rowtime, a])
+
+== Optimized Execution Plan ==
+Calc(select=[b, w$end AS window_end, EXPR$2])
++- SortWindowAggregate(groupBy=[b], window=[TumblingGroupWindow('w$, rowtime, 900000)], properties=[w$start, w$end, w$rowtime], select=[b, FIRST_VALUE(a) AS EXPR$2])
+   +- Sort(orderBy=[b ASC, rowtime ASC])
+      +- Exchange(distribution=[hash[b]])
+         +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[b, rowtime, a], metadata=[]]], fields=[b, rowtime, a])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: MyTable[]",
+    "pact" : "Data Source",
+    "contents" : "[]:TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[b, rowtime, a], metadata=[]]], fields=[b, rowtime, a])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "Sort[]",
+    "pact" : "Operator",
+    "contents" : "[]:Sort(orderBy=[b ASC, rowtime ASC])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "HASH[b]",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "SortWindowAggregate[]",
+    "pact" : "Operator",
+    "contents" : "[]:SortWindowAggregate(groupBy=[b], window=[TumblingGroupWindow('w$, rowtime, 900000)], properties=[w$start, w$end, w$rowtime], select=[b, FIRST_VALUE(a) AS EXPR$2])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Calc[]",
+    "pact" : "Operator",
+    "contents" : "[]:Calc(select=[b, w$end AS window_end, EXPR$2])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testSortMergeJoin[isNameSimplifyEnabled=true]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], a0=[$4], b0=[$5], c0=[$6], d0=[$7])
++- LogicalFilter(condition=[=($0, $7)])
+   +- LogicalJoin(condition=[true], joinType=[inner])
+      :- LogicalTableScan(table=[[default_catalog, default_database, A]])
+      +- LogicalTableScan(table=[[default_catalog, default_database, B]])
+
+== Optimized Physical Plan ==
+SortMergeJoin(joinType=[InnerJoin], where=[=(a, d0)], select=[a, b, c, d, a0, b0, c0, d0])
+:- Exchange(distribution=[hash[a]])
+:  +- TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a, b, c, d])
++- Exchange(distribution=[hash[d]])
+   +- TableSourceScan(table=[[default_catalog, default_database, B]], fields=[a, b, c, d])
+
+== Optimized Execution Plan ==
+SortMergeJoin(joinType=[InnerJoin], where=[(a = d0)], select=[a, b, c, d, a0, b0, c0, d0])
+:- Exchange(distribution=[hash[a]])
+:  +- TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a, b, c, d])
++- Exchange(distribution=[hash[d]])
+   +- TableSourceScan(table=[[default_catalog, default_database, B]], fields=[a, b, c, d])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: A[]",
+    "pact" : "Data Source",
+    "contents" : "[]:TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a, b, c, d])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "Source: B[]",
+    "pact" : "Data Source",
+    "contents" : "[]:TableSourceScan(table=[[default_catalog, default_database, B]], fields=[a, b, c, d])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "SortMergeJoin[]",
+    "pact" : "Operator",
+    "contents" : "[]:SortMergeJoin(joinType=[InnerJoin], where=[(a = d0)], select=[a, b, c, d, a0, b0, c0, d0])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "HASH[a]",
+      "side" : "second"
+    }, {
+      "id" : ,
+      "ship_strategy" : "HASH[d]",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testSortWindowAggregate[isNameSimplifyEnabled=false]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalProject(b=[$0], window_end=[TUMBLE_END($1)], EXPR$2=[$2])
++- LogicalAggregate(group=[{0, 1}], EXPR$2=[FIRST_VALUE($2)])
+   +- LogicalProject(b=[$1], $f1=[$TUMBLE($4, 900000:INTERVAL MINUTE)], a=[$0])
+      +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
+
+== Optimized Physical Plan ==
+Calc(select=[b, w$end AS window_end, EXPR$2])
++- SortWindowAggregate(groupBy=[b], window=[TumblingGroupWindow('w$, rowtime, 900000)], properties=[w$start, w$end, w$rowtime], select=[b, FIRST_VALUE(a) AS EXPR$2])
+   +- Sort(orderBy=[b ASC, rowtime ASC])
+      +- Exchange(distribution=[hash[b]])
+         +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[b, rowtime, a], metadata=[]]], fields=[b, rowtime, a])
+
+== Optimized Execution Plan ==
+Calc(select=[b, w$end AS window_end, EXPR$2])
++- SortWindowAggregate(groupBy=[b], window=[TumblingGroupWindow('w$, rowtime, 900000)], properties=[w$start, w$end, w$rowtime], select=[b, FIRST_VALUE(a) AS EXPR$2])
+   +- Sort(orderBy=[b ASC, rowtime ASC])
+      +- Exchange(distribution=[hash[b]])
+         +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[b, rowtime, a], metadata=[]]], fields=[b, rowtime, a])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[b, rowtime, a], metadata=[]]], fields=[b, rowtime, a])",
+    "pact" : "Data Source",
+    "contents" : "TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[b, rowtime, a], metadata=[]]], fields=[b, rowtime, a])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "Sort(orderBy=[b ASC, rowtime ASC])",
+    "pact" : "Operator",
+    "contents" : "Sort(orderBy=[b ASC, rowtime ASC])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "HASH[b]",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "SortWindowAggregate(groupBy=[b], window=[TumblingGroupWindow('w$, rowtime, 900000)], properties=[w$start, w$end, w$rowtime], select=[b, FIRST_VALUE(a) AS EXPR$2])",
+    "pact" : "Operator",
+    "contents" : "SortWindowAggregate(groupBy=[b], window=[TumblingGroupWindow('w$, rowtime, 900000)], properties=[w$start, w$end, w$rowtime], select=[b, FIRST_VALUE(a) AS EXPR$2])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Calc(select=[b, w$end AS window_end, EXPR$2])",
+    "pact" : "Operator",
+    "contents" : "Calc(select=[b, w$end AS window_end, EXPR$2])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+</Root>
diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/operator/StreamOperatorNameTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/operator/StreamOperatorNameTest.xml
new file mode 100644
index 0000000..16ac03f
--- /dev/null
+++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/operator/StreamOperatorNameTest.xml
@@ -0,0 +1,4191 @@
+<?xml version="1.0" ?>
+<!--
+Licensed to the Apache Software Foundation (ASF) under one or more
+contributor license agreements.  See the NOTICE file distributed with
+this work for additional information regarding copyright ownership.
+The ASF licenses this file to you under the Apache License, Version 2.0
+(the "License"); you may not use this file except in compliance with
+the License.  You may obtain a copy of the License at
+
+http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+-->
+<Root>
+  <TestCase name="testChangelogNormalize[isNameSimplifyEnabled=false]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalSink(table=[default_catalog.default_database.MySink], fields=[c, a, b])
++- LogicalProject(c=[$2], a=[$0], b=[$1])
+   +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
+
+== Optimized Physical Plan ==
+Sink(table=[default_catalog.default_database.MySink], fields=[c, a, b])
++- ChangelogNormalize(key=[a, b])
+   +- Exchange(distribution=[hash[a, b]])
+      +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[c, a, b], metadata=[]]], fields=[c, a, b])
+
+== Optimized Execution Plan ==
+Sink(table=[default_catalog.default_database.MySink], fields=[c, a, b], upsertMaterialize=[true])
++- ChangelogNormalize(key=[a, b])
+   +- Exchange(distribution=[hash[a, b]])
+      +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[c, a, b], metadata=[]]], fields=[c, a, b])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[c, a, b], metadata=[]]], fields=[c, a, b])",
+    "pact" : "Data Source",
+    "contents" : "TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[c, a, b], metadata=[]]], fields=[c, a, b])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "ChangelogNormalize(key=[a, b])",
+    "pact" : "Operator",
+    "contents" : "ChangelogNormalize(key=[a, b])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "HASH",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "ConstraintEnforcer[NotNullEnforcer(fields=[a, b])]",
+    "pact" : "Operator",
+    "contents" : "ConstraintEnforcer[NotNullEnforcer(fields=[a, b])]",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "SinkMaterializer(pk=[a])",
+    "pact" : "Operator",
+    "contents" : "SinkMaterializer(pk=[a])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "HASH",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Sink: Sink(table=[default_catalog.default_database.MySink], fields=[c, a, b], upsertMaterialize=[true])",
+    "pact" : "Data Sink",
+    "contents" : "Sink(table=[default_catalog.default_database.MySink], fields=[c, a, b], upsertMaterialize=[true])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testChangelogNormalize[isNameSimplifyEnabled=true]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalSink(table=[default_catalog.default_database.MySink], fields=[c, a, b])
++- LogicalProject(c=[$2], a=[$0], b=[$1])
+   +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
+
+== Optimized Physical Plan ==
+Sink(table=[default_catalog.default_database.MySink], fields=[c, a, b])
++- ChangelogNormalize(key=[a, b])
+   +- Exchange(distribution=[hash[a, b]])
+      +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[c, a, b], metadata=[]]], fields=[c, a, b])
+
+== Optimized Execution Plan ==
+Sink(table=[default_catalog.default_database.MySink], fields=[c, a, b], upsertMaterialize=[true])
++- ChangelogNormalize(key=[a, b])
+   +- Exchange(distribution=[hash[a, b]])
+      +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[c, a, b], metadata=[]]], fields=[c, a, b])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: MyTable[]",
+    "pact" : "Data Source",
+    "contents" : "[]:TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[c, a, b], metadata=[]]], fields=[c, a, b])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "ChangelogNormalize[]",
+    "pact" : "Operator",
+    "contents" : "[]:ChangelogNormalize(key=[a, b])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "HASH",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "ConstraintEnforcer[]",
+    "pact" : "Operator",
+    "contents" : "[]:ConstraintEnforcer[NotNullEnforcer(fields=[a, b])]",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "SinkMaterializer[]",
+    "pact" : "Operator",
+    "contents" : "[]:SinkMaterializer(pk=[a])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "HASH",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Sink: MySink[]",
+    "pact" : "Data Sink",
+    "contents" : "[]:Sink(table=[default_catalog.default_database.MySink], fields=[c, a, b], upsertMaterialize=[true])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testCorrelate[isNameSimplifyEnabled=false]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalProject(s=[$4])
++- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{2}])
+   :- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
+   +- LogicalTableFunctionScan(invocation=[func1($cor0.c)], rowType=[RecordType(VARCHAR(2147483647) EXPR$0)])
+
+== Optimized Physical Plan ==
+Calc(select=[EXPR$0 AS s])
++- Correlate(invocation=[func1($cor0.c)], correlate=[table(func1($cor0.c))], select=[a,b,c,d,EXPR$0], rowType=[RecordType(BIGINT a, INTEGER b, VARCHAR(2147483647) c, BIGINT d, VARCHAR(2147483647) EXPR$0)], joinType=[INNER])
+   +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d])
+
+== Optimized Execution Plan ==
+Calc(select=[EXPR$0 AS s])
++- Correlate(invocation=[func1($cor0.c)], correlate=[table(func1($cor0.c))], select=[a,b,c,d,EXPR$0], rowType=[RecordType(BIGINT a, INTEGER b, VARCHAR(2147483647) c, BIGINT d, VARCHAR(2147483647) EXPR$0)], joinType=[INNER])
+   +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d])",
+    "pact" : "Data Source",
+    "contents" : "TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "Correlate(invocation=[func1($cor0.c)], correlate=[table(func1($cor0.c))], select=[a,b,c,d,EXPR$0], rowType=[RecordType(BIGINT a, INTEGER b, VARCHAR(2147483647) c, BIGINT d, VARCHAR(2147483647) EXPR$0)], joinType=[INNER])",
+    "pact" : "Operator",
+    "contents" : "Correlate(invocation=[func1($cor0.c)], correlate=[table(func1($cor0.c))], select=[a,b,c,d,EXPR$0], rowType=[RecordType(BIGINT a, INTEGER b, VARCHAR(2147483647) c, BIGINT d, VARCHAR(2147483647) EXPR$0)], joinType=[INNER])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Calc(select=[EXPR$0 AS s])",
+    "pact" : "Operator",
+    "contents" : "Calc(select=[EXPR$0 AS s])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testCorrelate[isNameSimplifyEnabled=true]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalProject(s=[$4])
++- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{2}])
+   :- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
+   +- LogicalTableFunctionScan(invocation=[func1($cor0.c)], rowType=[RecordType(VARCHAR(2147483647) EXPR$0)])
+
+== Optimized Physical Plan ==
+Calc(select=[EXPR$0 AS s])
++- Correlate(invocation=[func1($cor0.c)], correlate=[table(func1($cor0.c))], select=[a,b,c,d,EXPR$0], rowType=[RecordType(BIGINT a, INTEGER b, VARCHAR(2147483647) c, BIGINT d, VARCHAR(2147483647) EXPR$0)], joinType=[INNER])
+   +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d])
+
+== Optimized Execution Plan ==
+Calc(select=[EXPR$0 AS s])
++- Correlate(invocation=[func1($cor0.c)], correlate=[table(func1($cor0.c))], select=[a,b,c,d,EXPR$0], rowType=[RecordType(BIGINT a, INTEGER b, VARCHAR(2147483647) c, BIGINT d, VARCHAR(2147483647) EXPR$0)], joinType=[INNER])
+   +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: MyTable[]",
+    "pact" : "Data Source",
+    "contents" : "[]:TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "Correlate[]",
+    "pact" : "Operator",
+    "contents" : "[]:Correlate(invocation=[func1($cor0.c)], correlate=[table(func1($cor0.c))], select=[a,b,c,d,EXPR$0], rowType=[RecordType(BIGINT a, INTEGER b, VARCHAR(2147483647) c, BIGINT d, VARCHAR(2147483647) EXPR$0)], joinType=[INNER])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Calc[]",
+    "pact" : "Operator",
+    "contents" : "[]:Calc(select=[EXPR$0 AS s])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testDeduplicate[isNameSimplifyEnabled=false]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalProject(a=[$0], b=[$1], c=[$2])
++- LogicalFilter(condition=[=($6, 1)])
+   +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], rowtime=[$4], proctime=[$5], rk=[ROW_NUMBER() OVER (PARTITION BY $0 ORDER BY $4 NULLS FIRST)])
+      +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($4, 1000:INTERVAL SECOND)])
+         +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], rowtime=[$4], proctime=[PROCTIME()])
+            +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
+
+== Optimized Physical Plan ==
+Calc(select=[a, b, c])
++- Deduplicate(keep=[FirstRow], key=[a], order=[ROWTIME])
+   +- Exchange(distribution=[hash[a]])
+      +- Calc(select=[a, b, c, rowtime])
+         +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)])
+            +- Calc(select=[a, b, c, d, rowtime, PROCTIME() AS proctime])
+               +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, rowtime])
+
+== Optimized Execution Plan ==
+Calc(select=[a, b, c])
++- Deduplicate(keep=[FirstRow], key=[a], order=[ROWTIME])
+   +- Exchange(distribution=[hash[a]])
+      +- Calc(select=[a, b, c, rowtime])
+         +- WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])
+            +- Calc(select=[a, b, c, d, rowtime, PROCTIME() AS proctime])
+               +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, rowtime])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, rowtime])",
+    "pact" : "Data Source",
+    "contents" : "TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, rowtime])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "Calc(select=[a, b, c, d, rowtime, PROCTIME() AS proctime])",
+    "pact" : "Operator",
+    "contents" : "Calc(select=[a, b, c, d, rowtime, PROCTIME() AS proctime])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])",
+    "pact" : "Operator",
+    "contents" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Calc(select=[a, b, c, rowtime])",
+    "pact" : "Operator",
+    "contents" : "Calc(select=[a, b, c, rowtime])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Deduplicate(keep=[FirstRow], key=[a], order=[ROWTIME])",
+    "pact" : "Operator",
+    "contents" : "Deduplicate(keep=[FirstRow], key=[a], order=[ROWTIME])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "HASH",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Calc(select=[a, b, c])",
+    "pact" : "Operator",
+    "contents" : "Calc(select=[a, b, c])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testDeduplicate[isNameSimplifyEnabled=true]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalProject(a=[$0], b=[$1], c=[$2])
++- LogicalFilter(condition=[=($6, 1)])
+   +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], rowtime=[$4], proctime=[$5], rk=[ROW_NUMBER() OVER (PARTITION BY $0 ORDER BY $4 NULLS FIRST)])
+      +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($4, 1000:INTERVAL SECOND)])
+         +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], rowtime=[$4], proctime=[PROCTIME()])
+            +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
+
+== Optimized Physical Plan ==
+Calc(select=[a, b, c])
++- Deduplicate(keep=[FirstRow], key=[a], order=[ROWTIME])
+   +- Exchange(distribution=[hash[a]])
+      +- Calc(select=[a, b, c, rowtime])
+         +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)])
+            +- Calc(select=[a, b, c, d, rowtime, PROCTIME() AS proctime])
+               +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, rowtime])
+
+== Optimized Execution Plan ==
+Calc(select=[a, b, c])
++- Deduplicate(keep=[FirstRow], key=[a], order=[ROWTIME])
+   +- Exchange(distribution=[hash[a]])
+      +- Calc(select=[a, b, c, rowtime])
+         +- WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])
+            +- Calc(select=[a, b, c, d, rowtime, PROCTIME() AS proctime])
+               +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, rowtime])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: MyTable[]",
+    "pact" : "Data Source",
+    "contents" : "[]:TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, rowtime])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "Calc[]",
+    "pact" : "Operator",
+    "contents" : "[]:Calc(select=[a, b, c, d, rowtime, PROCTIME() AS proctime])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "WatermarkAssigner[]",
+    "pact" : "Operator",
+    "contents" : "[]:WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Calc[]",
+    "pact" : "Operator",
+    "contents" : "[]:Calc(select=[a, b, c, rowtime])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Deduplicate[]",
+    "pact" : "Operator",
+    "contents" : "[]:Deduplicate(keep=[FirstRow], key=[a], order=[ROWTIME])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "HASH",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Calc[]",
+    "pact" : "Operator",
+    "contents" : "[]:Calc(select=[a, b, c])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testDropUpdateBefore[isNameSimplifyEnabled=false]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalSink(table=[default_catalog.default_database.MySink], fields=[c, a, b])
++- LogicalProject(c=[$2], a=[$0], b=[$1])
+   +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
+
+== Optimized Physical Plan ==
+Sink(table=[default_catalog.default_database.MySink], fields=[c, a, b])
++- DropUpdateBefore
+   +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[c, a, b], metadata=[]]], fields=[c, a, b])
+
+== Optimized Execution Plan ==
+Sink(table=[default_catalog.default_database.MySink], fields=[c, a, b])
++- DropUpdateBefore
+   +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[c, a, b], metadata=[]]], fields=[c, a, b])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[c, a, b], metadata=[]]], fields=[c, a, b])",
+    "pact" : "Data Source",
+    "contents" : "TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[c, a, b], metadata=[]]], fields=[c, a, b])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "DropUpdateBefore",
+    "pact" : "Operator",
+    "contents" : "DropUpdateBefore",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "ConstraintEnforcer[NotNullEnforcer(fields=[a, b])]",
+    "pact" : "Operator",
+    "contents" : "ConstraintEnforcer[NotNullEnforcer(fields=[a, b])]",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Sink: Sink(table=[default_catalog.default_database.MySink], fields=[c, a, b])",
+    "pact" : "Data Sink",
+    "contents" : "Sink(table=[default_catalog.default_database.MySink], fields=[c, a, b])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testDropUpdateBefore[isNameSimplifyEnabled=true]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalSink(table=[default_catalog.default_database.MySink], fields=[c, a, b])
++- LogicalProject(c=[$2], a=[$0], b=[$1])
+   +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
+
+== Optimized Physical Plan ==
+Sink(table=[default_catalog.default_database.MySink], fields=[c, a, b])
++- DropUpdateBefore
+   +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[c, a, b], metadata=[]]], fields=[c, a, b])
+
+== Optimized Execution Plan ==
+Sink(table=[default_catalog.default_database.MySink], fields=[c, a, b])
++- DropUpdateBefore
+   +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[c, a, b], metadata=[]]], fields=[c, a, b])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: MyTable[]",
+    "pact" : "Data Source",
+    "contents" : "[]:TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[c, a, b], metadata=[]]], fields=[c, a, b])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "DropUpdateBefore[]",
+    "pact" : "Operator",
+    "contents" : "[]:DropUpdateBefore",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "ConstraintEnforcer[]",
+    "pact" : "Operator",
+    "contents" : "[]:ConstraintEnforcer[NotNullEnforcer(fields=[a, b])]",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Sink: MySink[]",
+    "pact" : "Data Sink",
+    "contents" : "[]:Sink(table=[default_catalog.default_database.MySink], fields=[c, a, b])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testGroupAggregate[isNameSimplifyEnabled=false]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalAggregate(group=[{0}], b=[MAX($1)])
++- LogicalProject(a=[$0], b=[$1])
+   +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
+
+== Optimized Physical Plan ==
+GroupAggregate(groupBy=[a], select=[a, MAX(b) AS b])
++- Exchange(distribution=[hash[a]])
+   +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b], metadata=[]]], fields=[a, b])
+
+== Optimized Execution Plan ==
+GroupAggregate(groupBy=[a], select=[a, MAX(b) AS b])
++- Exchange(distribution=[hash[a]])
+   +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b], metadata=[]]], fields=[a, b])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b], metadata=[]]], fields=[a, b])",
+    "pact" : "Data Source",
+    "contents" : "TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b], metadata=[]]], fields=[a, b])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "GroupAggregate(groupBy=[a], select=[a, MAX(b) AS b])",
+    "pact" : "Operator",
+    "contents" : "GroupAggregate(groupBy=[a], select=[a, MAX(b) AS b])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "HASH",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testWindowDeduplicate[isNameSimplifyEnabled=true]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalProject(window_start=[$6], window_end=[$7], a=[$0], b=[$1], c=[$2])
++- LogicalFilter(condition=[<=($9, 1)])
+   +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], rowtime=[$4], proctime=[$5], window_start=[$6], window_end=[$7], window_time=[$8], rownum=[ROW_NUMBER() OVER (PARTITION BY $0, $6, $7 ORDER BY $4 DESC NULLS LAST)])
+      +- LogicalTableFunctionScan(invocation=[TUMBLE($5, DESCRIPTOR($4), 900000:INTERVAL MINUTE)], rowType=[RecordType(BIGINT a, INTEGER b, VARCHAR(2147483647) c, BIGINT d, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)])
+         +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], rowtime=[$4], proctime=[$5])
+            +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($4, 1000:INTERVAL SECOND)])
+               +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], rowtime=[$4], proctime=[PROCTIME()])
+                  +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
+
+== Optimized Physical Plan ==
+Calc(select=[window_start, window_end, a, b, c])
++- WindowDeduplicate(window=[TUMBLE(win_start=[window_start], win_end=[window_end], size=[15 min])], keep=[LastRow], partitionKeys=[a], orderKey=[rowtime], order=[ROWTIME])
+   +- Exchange(distribution=[hash[a]])
+      +- Calc(select=[a, b, c, rowtime, window_start, window_end])
+         +- WindowTableFunction(window=[TUMBLE(time_col=[rowtime], size=[15 min])], emitPerRecord=[true])
+            +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)])
+               +- Calc(select=[a, b, c, d, rowtime, PROCTIME() AS proctime])
+                  +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, rowtime])
+
+== Optimized Execution Plan ==
+Calc(select=[window_start, window_end, a, b, c])
++- WindowDeduplicate(window=[TUMBLE(win_start=[window_start], win_end=[window_end], size=[15 min])], keep=[LastRow], partitionKeys=[a], orderKey=[rowtime], order=[ROWTIME])
+   +- Exchange(distribution=[hash[a]])
+      +- Calc(select=[a, b, c, rowtime, window_start, window_end])
+         +- WindowTableFunction(window=[TUMBLE(time_col=[rowtime], size=[15 min])], emitPerRecord=[true])
+            +- WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])
+               +- Calc(select=[a, b, c, d, rowtime, PROCTIME() AS proctime])
+                  +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, rowtime])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: MyTable[]",
+    "pact" : "Data Source",
+    "contents" : "[]:TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, rowtime])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "Calc[]",
+    "pact" : "Operator",
+    "contents" : "[]:Calc(select=[a, b, c, d, rowtime, PROCTIME() AS proctime])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "WatermarkAssigner[]",
+    "pact" : "Operator",
+    "contents" : "[]:WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "WindowTableFunction[]",
+    "pact" : "Operator",
+    "contents" : "[]:WindowTableFunction(window=[TUMBLE(time_col=[rowtime], size=[15 min])], emitPerRecord=[true])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Calc[]",
+    "pact" : "Operator",
+    "contents" : "[]:Calc(select=[a, b, c, rowtime, window_start, window_end])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "WindowDeduplicate[]",
+    "pact" : "Operator",
+    "contents" : "[]:WindowDeduplicate(window=[TUMBLE(win_start=[window_start], win_end=[window_end], size=[15 min])], keep=[LastRow], partitionKeys=[a], orderKey=[rowtime], order=[ROWTIME])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "HASH",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Calc[]",
+    "pact" : "Operator",
+    "contents" : "[]:Calc(select=[window_start, window_end, a, b, c])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testGroupAggregate[isNameSimplifyEnabled=true]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalAggregate(group=[{0}], b=[MAX($1)])
++- LogicalProject(a=[$0], b=[$1])
+   +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
+
+== Optimized Physical Plan ==
+GroupAggregate(groupBy=[a], select=[a, MAX(b) AS b])
++- Exchange(distribution=[hash[a]])
+   +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b], metadata=[]]], fields=[a, b])
+
+== Optimized Execution Plan ==
+GroupAggregate(groupBy=[a], select=[a, MAX(b) AS b])
++- Exchange(distribution=[hash[a]])
+   +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b], metadata=[]]], fields=[a, b])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: MyTable[]",
+    "pact" : "Data Source",
+    "contents" : "[]:TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b], metadata=[]]], fields=[a, b])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "GroupAggregate[]",
+    "pact" : "Operator",
+    "contents" : "[]:GroupAggregate(groupBy=[a], select=[a, MAX(b) AS b])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "HASH",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testGroupWindowAggregate[isNameSimplifyEnabled=false]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalProject(b=[$0], window_end=[TUMBLE_END($1)], EXPR$2=[$2])
++- LogicalAggregate(group=[{0, 1}], EXPR$2=[COUNT()])
+   +- LogicalProject(b=[$1], $f1=[$TUMBLE($4, 900000:INTERVAL MINUTE)])
+      +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($4, 1000:INTERVAL SECOND)])
+         +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], rowtime=[$4], proctime=[PROCTIME()])
+            +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
+
+== Optimized Physical Plan ==
+Calc(select=[b, w$end AS window_end, EXPR$2])
++- GroupWindowAggregate(groupBy=[b], window=[TumblingGroupWindow('w$, rowtime, 900000)], properties=[w$start, w$end, w$rowtime, w$proctime], select=[b, COUNT(*) AS EXPR$2, start('w$) AS w$start, end('w$) AS w$end, rowtime('w$) AS w$rowtime, proctime('w$) AS w$proctime])
+   +- Exchange(distribution=[hash[b]])
+      +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)])
+         +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[b, rowtime], metadata=[]]], fields=[b, rowtime])
+
+== Optimized Execution Plan ==
+Calc(select=[b, w$end AS window_end, EXPR$2])
++- GroupWindowAggregate(groupBy=[b], window=[TumblingGroupWindow('w$, rowtime, 900000)], properties=[w$start, w$end, w$rowtime, w$proctime], select=[b, COUNT(*) AS EXPR$2, start('w$) AS w$start, end('w$) AS w$end, rowtime('w$) AS w$rowtime, proctime('w$) AS w$proctime])
+   +- Exchange(distribution=[hash[b]])
+      +- WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])
+         +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[b, rowtime], metadata=[]]], fields=[b, rowtime])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[b, rowtime], metadata=[]]], fields=[b, rowtime])",
+    "pact" : "Data Source",
+    "contents" : "TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[b, rowtime], metadata=[]]], fields=[b, rowtime])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])",
+    "pact" : "Operator",
+    "contents" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "GroupWindowAggregate(groupBy=[b], window=[TumblingGroupWindow('w$, rowtime, 900000)], properties=[w$start, w$end, w$rowtime, w$proctime], select=[b, COUNT(*) AS EXPR$2, start('w$) AS w$start, end('w$) AS w$end, rowtime('w$) AS w$rowtime, proctime('w$) AS w$proctime])",
+    "pact" : "Operator",
+    "contents" : "GroupWindowAggregate(groupBy=[b], window=[TumblingGroupWindow('w$, rowtime, 900000)], properties=[w$start, w$end, w$rowtime, w$proctime], select=[b, COUNT(*) AS EXPR$2, start('w$) AS w$start, end('w$) AS w$end, rowtime('w$) AS w$rowtime, proctime('w$) AS w$proctime])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "HASH",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Calc(select=[b, w$end AS window_end, EXPR$2])",
+    "pact" : "Operator",
+    "contents" : "Calc(select=[b, w$end AS window_end, EXPR$2])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testGroupWindowAggregate[isNameSimplifyEnabled=true]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalProject(b=[$0], window_end=[TUMBLE_END($1)], EXPR$2=[$2])
++- LogicalAggregate(group=[{0, 1}], EXPR$2=[COUNT()])
+   +- LogicalProject(b=[$1], $f1=[$TUMBLE($4, 900000:INTERVAL MINUTE)])
+      +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($4, 1000:INTERVAL SECOND)])
+         +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], rowtime=[$4], proctime=[PROCTIME()])
+            +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
+
+== Optimized Physical Plan ==
+Calc(select=[b, w$end AS window_end, EXPR$2])
++- GroupWindowAggregate(groupBy=[b], window=[TumblingGroupWindow('w$, rowtime, 900000)], properties=[w$start, w$end, w$rowtime, w$proctime], select=[b, COUNT(*) AS EXPR$2, start('w$) AS w$start, end('w$) AS w$end, rowtime('w$) AS w$rowtime, proctime('w$) AS w$proctime])
+   +- Exchange(distribution=[hash[b]])
+      +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)])
+         +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[b, rowtime], metadata=[]]], fields=[b, rowtime])
+
+== Optimized Execution Plan ==
+Calc(select=[b, w$end AS window_end, EXPR$2])
++- GroupWindowAggregate(groupBy=[b], window=[TumblingGroupWindow('w$, rowtime, 900000)], properties=[w$start, w$end, w$rowtime, w$proctime], select=[b, COUNT(*) AS EXPR$2, start('w$) AS w$start, end('w$) AS w$end, rowtime('w$) AS w$rowtime, proctime('w$) AS w$proctime])
+   +- Exchange(distribution=[hash[b]])
+      +- WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])
+         +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[b, rowtime], metadata=[]]], fields=[b, rowtime])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: MyTable[]",
+    "pact" : "Data Source",
+    "contents" : "[]:TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[b, rowtime], metadata=[]]], fields=[b, rowtime])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "WatermarkAssigner[]",
+    "pact" : "Operator",
+    "contents" : "[]:WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "GroupWindowAggregate[]",
+    "pact" : "Operator",
+    "contents" : "[]:GroupWindowAggregate(groupBy=[b], window=[TumblingGroupWindow('w$, rowtime, 900000)], properties=[w$start, w$end, w$rowtime, w$proctime], select=[b, COUNT(*) AS EXPR$2, start('w$) AS w$start, end('w$) AS w$end, rowtime('w$) AS w$rowtime, proctime('w$) AS w$proctime])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "HASH",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Calc[]",
+    "pact" : "Operator",
+    "contents" : "[]:Calc(select=[b, w$end AS window_end, EXPR$2])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testIncrementalAggregate[isNameSimplifyEnabled=false]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalAggregate(group=[{0}], b=[COUNT(DISTINCT $1)])
++- LogicalProject(a=[$0], b=[$1])
+   +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
+
+== Optimized Physical Plan ==
+GlobalGroupAggregate(groupBy=[a], partialFinalType=[FINAL], select=[a, $SUM0(count$0) AS $f1])
++- Exchange(distribution=[hash[a]])
+   +- IncrementalGroupAggregate(partialAggGrouping=[a, $f2], finalAggGrouping=[a], select=[a, COUNT(distinct$0 count$0) AS count$0])
+      +- Exchange(distribution=[hash[a, $f2]])
+         +- LocalGroupAggregate(groupBy=[a, $f2], partialFinalType=[PARTIAL], select=[a, $f2, COUNT(distinct$0 b) AS count$0, DISTINCT(b) AS distinct$0])
+            +- Calc(select=[a, b, MOD(HASH_CODE(b), 1024) AS $f2])
+               +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime])
+                  +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b], metadata=[]]], fields=[a, b])
+
+== Optimized Execution Plan ==
+GlobalGroupAggregate(groupBy=[a], partialFinalType=[FINAL], select=[a, $SUM0(count$0) AS $f1])
++- Exchange(distribution=[hash[a]])
+   +- IncrementalGroupAggregate(partialAggGrouping=[a, $f2], finalAggGrouping=[a], select=[a, COUNT(distinct$0 count$0) AS count$0])
+      +- Exchange(distribution=[hash[a, $f2]])
+         +- LocalGroupAggregate(groupBy=[a, $f2], partialFinalType=[PARTIAL], select=[a, $f2, COUNT(distinct$0 b) AS count$0, DISTINCT(b) AS distinct$0])
+            +- Calc(select=[a, b, MOD(HASH_CODE(b), 1024) AS $f2])
+               +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime])
+                  +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b], metadata=[]]], fields=[a, b])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b], metadata=[]]], fields=[a, b])",
+    "pact" : "Data Source",
+    "contents" : "TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b], metadata=[]]], fields=[a, b])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "MiniBatchAssigner(interval=[1000ms], mode=[ProcTime])",
+    "pact" : "Operator",
+    "contents" : "MiniBatchAssigner(interval=[1000ms], mode=[ProcTime])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Calc(select=[a, b, MOD(HASH_CODE(b), 1024) AS $f2])",
+    "pact" : "Operator",
+    "contents" : "Calc(select=[a, b, MOD(HASH_CODE(b), 1024) AS $f2])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "LocalGroupAggregate(groupBy=[a, $f2], partialFinalType=[PARTIAL], select=[a, $f2, COUNT(distinct$0 b) AS count$0, DISTINCT(b) AS distinct$0])",
+    "pact" : "Operator",
+    "contents" : "LocalGroupAggregate(groupBy=[a, $f2], partialFinalType=[PARTIAL], select=[a, $f2, COUNT(distinct$0 b) AS count$0, DISTINCT(b) AS distinct$0])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "IncrementalGroupAggregate(partialAggGrouping=[a, $f2], finalAggGrouping=[a], select=[a, COUNT(distinct$0 count$0) AS count$0])",
+    "pact" : "Operator",
+    "contents" : "IncrementalGroupAggregate(partialAggGrouping=[a, $f2], finalAggGrouping=[a], select=[a, COUNT(distinct$0 count$0) AS count$0])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "HASH",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "GlobalGroupAggregate(groupBy=[a], partialFinalType=[FINAL], select=[a, $SUM0(count$0) AS $f1])",
+    "pact" : "Operator",
+    "contents" : "GlobalGroupAggregate(groupBy=[a], partialFinalType=[FINAL], select=[a, $SUM0(count$0) AS $f1])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "HASH",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testIncrementalAggregate[isNameSimplifyEnabled=true]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalAggregate(group=[{0}], b=[COUNT(DISTINCT $1)])
++- LogicalProject(a=[$0], b=[$1])
+   +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
+
+== Optimized Physical Plan ==
+GlobalGroupAggregate(groupBy=[a], partialFinalType=[FINAL], select=[a, $SUM0(count$0) AS $f1])
++- Exchange(distribution=[hash[a]])
+   +- IncrementalGroupAggregate(partialAggGrouping=[a, $f2], finalAggGrouping=[a], select=[a, COUNT(distinct$0 count$0) AS count$0])
+      +- Exchange(distribution=[hash[a, $f2]])
+         +- LocalGroupAggregate(groupBy=[a, $f2], partialFinalType=[PARTIAL], select=[a, $f2, COUNT(distinct$0 b) AS count$0, DISTINCT(b) AS distinct$0])
+            +- Calc(select=[a, b, MOD(HASH_CODE(b), 1024) AS $f2])
+               +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime])
+                  +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b], metadata=[]]], fields=[a, b])
+
+== Optimized Execution Plan ==
+GlobalGroupAggregate(groupBy=[a], partialFinalType=[FINAL], select=[a, $SUM0(count$0) AS $f1])
++- Exchange(distribution=[hash[a]])
+   +- IncrementalGroupAggregate(partialAggGrouping=[a, $f2], finalAggGrouping=[a], select=[a, COUNT(distinct$0 count$0) AS count$0])
+      +- Exchange(distribution=[hash[a, $f2]])
+         +- LocalGroupAggregate(groupBy=[a, $f2], partialFinalType=[PARTIAL], select=[a, $f2, COUNT(distinct$0 b) AS count$0, DISTINCT(b) AS distinct$0])
+            +- Calc(select=[a, b, MOD(HASH_CODE(b), 1024) AS $f2])
+               +- MiniBatchAssigner(interval=[1000ms], mode=[ProcTime])
+                  +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b], metadata=[]]], fields=[a, b])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: MyTable[]",
+    "pact" : "Data Source",
+    "contents" : "[]:TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b], metadata=[]]], fields=[a, b])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "MiniBatchAssigner[]",
+    "pact" : "Operator",
+    "contents" : "[]:MiniBatchAssigner(interval=[1000ms], mode=[ProcTime])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Calc[]",
+    "pact" : "Operator",
+    "contents" : "[]:Calc(select=[a, b, MOD(HASH_CODE(b), 1024) AS $f2])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "LocalGroupAggregate[]",
+    "pact" : "Operator",
+    "contents" : "[]:LocalGroupAggregate(groupBy=[a, $f2], partialFinalType=[PARTIAL], select=[a, $f2, COUNT(distinct$0 b) AS count$0, DISTINCT(b) AS distinct$0])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "IncrementalGroupAggregate[]",
+    "pact" : "Operator",
+    "contents" : "[]:IncrementalGroupAggregate(partialAggGrouping=[a, $f2], finalAggGrouping=[a], select=[a, COUNT(distinct$0 count$0) AS count$0])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "HASH",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "GlobalGroupAggregate[]",
+    "pact" : "Operator",
+    "contents" : "[]:GlobalGroupAggregate(groupBy=[a], partialFinalType=[FINAL], select=[a, $SUM0(count$0) AS $f1])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "HASH",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testIntervalJoin[isNameSimplifyEnabled=false]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalProject(a=[$0], b=[$7])
++- LogicalJoin(condition=[AND(=($0, $6), >=($5, -($11, 3600000:INTERVAL HOUR)), <=($5, +($11, 3600000:INTERVAL HOUR)))], joinType=[inner])
+   :- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($4, 1000:INTERVAL SECOND)])
+   :  +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], rowtime=[$4], proctime=[PROCTIME()])
+   :     +- LogicalTableScan(table=[[default_catalog, default_database, A]])
+   +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($4, 1000:INTERVAL SECOND)])
+      +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], rowtime=[$4], proctime=[PROCTIME()])
+         +- LogicalTableScan(table=[[default_catalog, default_database, B]])
+
+== Optimized Physical Plan ==
+Calc(select=[a, b])
++- IntervalJoin(joinType=[InnerJoin], windowBounds=[isRowTime=false, leftLowerBound=-3600000, leftUpperBound=3600000, leftTimeIndex=1, rightTimeIndex=2], where=[AND(=(a, a0), >=(proctime, -(proctime0, 3600000:INTERVAL HOUR)), <=(proctime, +(proctime0, 3600000:INTERVAL HOUR)))], select=[a, proctime, a0, b, proctime0])
+   :- Exchange(distribution=[hash[a]])
+   :  +- Calc(select=[a, proctime])
+   :     +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)])
+   :        +- Calc(select=[a, PROCTIME() AS proctime, rowtime])
+   :           +- TableSourceScan(table=[[default_catalog, default_database, A, project=[a, rowtime], metadata=[]]], fields=[a, rowtime])
+   +- Exchange(distribution=[hash[a]])
+      +- Calc(select=[a, b, proctime])
+         +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)])
+            +- Calc(select=[a, b, PROCTIME() AS proctime, rowtime])
+               +- TableSourceScan(table=[[default_catalog, default_database, B, project=[a, b, rowtime], metadata=[]]], fields=[a, b, rowtime])
+
+== Optimized Execution Plan ==
+Calc(select=[a, b])
++- IntervalJoin(joinType=[InnerJoin], windowBounds=[isRowTime=false, leftLowerBound=-3600000, leftUpperBound=3600000, leftTimeIndex=1, rightTimeIndex=2], where=[((a = a0) AND (proctime >= (proctime0 - 3600000:INTERVAL HOUR)) AND (proctime <= (proctime0 + 3600000:INTERVAL HOUR)))], select=[a, proctime, a0, b, proctime0])
+   :- Exchange(distribution=[hash[a]])
+   :  +- Calc(select=[a, proctime])
+   :     +- WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])
+   :        +- Calc(select=[a, PROCTIME() AS proctime, rowtime])
+   :           +- TableSourceScan(table=[[default_catalog, default_database, A, project=[a, rowtime], metadata=[]]], fields=[a, rowtime])
+   +- Exchange(distribution=[hash[a]])
+      +- Calc(select=[a, b, proctime])
+         +- WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])
+            +- Calc(select=[a, b, PROCTIME() AS proctime, rowtime])
+               +- TableSourceScan(table=[[default_catalog, default_database, B, project=[a, b, rowtime], metadata=[]]], fields=[a, b, rowtime])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: TableSourceScan(table=[[default_catalog, default_database, A, project=[a, rowtime], metadata=[]]], fields=[a, rowtime])",
+    "pact" : "Data Source",
+    "contents" : "TableSourceScan(table=[[default_catalog, default_database, A, project=[a, rowtime], metadata=[]]], fields=[a, rowtime])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "Calc(select=[a, PROCTIME() AS proctime, rowtime])",
+    "pact" : "Operator",
+    "contents" : "Calc(select=[a, PROCTIME() AS proctime, rowtime])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])",
+    "pact" : "Operator",
+    "contents" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Calc(select=[a, proctime])",
+    "pact" : "Operator",
+    "contents" : "Calc(select=[a, proctime])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Source: TableSourceScan(table=[[default_catalog, default_database, B, project=[a, b, rowtime], metadata=[]]], fields=[a, b, rowtime])",
+    "pact" : "Data Source",
+    "contents" : "TableSourceScan(table=[[default_catalog, default_database, B, project=[a, b, rowtime], metadata=[]]], fields=[a, b, rowtime])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "Calc(select=[a, b, PROCTIME() AS proctime, rowtime])",
+    "pact" : "Operator",
+    "contents" : "Calc(select=[a, b, PROCTIME() AS proctime, rowtime])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])",
+    "pact" : "Operator",
+    "contents" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Calc(select=[a, b, proctime])",
+    "pact" : "Operator",
+    "contents" : "Calc(select=[a, b, proctime])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "IntervalJoin(joinType=[InnerJoin], windowBounds=[isRowTime=false, leftLowerBound=-3600000, leftUpperBound=3600000, leftTimeIndex=1, rightTimeIndex=2], where=[((a = a0) AND (proctime >= (proctime0 - 3600000:INTERVAL HOUR)) AND (proctime <= (proctime0 + 3600000:INTERVAL HOUR)))], select=[a, proctime, a0, b, proctime0])",
+    "pact" : "Operator",
+    "contents" : "IntervalJoin(joinType=[InnerJoin], windowBounds=[isRowTime=false, leftLowerBound=-3600000, leftUpperBound=3600000, leftTimeIndex=1, rightTimeIndex=2], where=[((a = a0) AND (proctime >= (proctime0 - 3600000:INTERVAL HOUR)) AND (proctime <= (proctime0 + 3600000:INTERVAL HOUR)))], select=[a, proctime, a0, b, proctime0])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "HASH",
+      "side" : "second"
+    }, {
+      "id" : ,
+      "ship_strategy" : "HASH",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Calc(select=[a, b])",
+    "pact" : "Operator",
+    "contents" : "Calc(select=[a, b])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testIntervalJoin[isNameSimplifyEnabled=true]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalProject(a=[$0], b=[$7])
++- LogicalJoin(condition=[AND(=($0, $6), >=($5, -($11, 3600000:INTERVAL HOUR)), <=($5, +($11, 3600000:INTERVAL HOUR)))], joinType=[inner])
+   :- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($4, 1000:INTERVAL SECOND)])
+   :  +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], rowtime=[$4], proctime=[PROCTIME()])
+   :     +- LogicalTableScan(table=[[default_catalog, default_database, A]])
+   +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($4, 1000:INTERVAL SECOND)])
+      +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], rowtime=[$4], proctime=[PROCTIME()])
+         +- LogicalTableScan(table=[[default_catalog, default_database, B]])
+
+== Optimized Physical Plan ==
+Calc(select=[a, b])
++- IntervalJoin(joinType=[InnerJoin], windowBounds=[isRowTime=false, leftLowerBound=-3600000, leftUpperBound=3600000, leftTimeIndex=1, rightTimeIndex=2], where=[AND(=(a, a0), >=(proctime, -(proctime0, 3600000:INTERVAL HOUR)), <=(proctime, +(proctime0, 3600000:INTERVAL HOUR)))], select=[a, proctime, a0, b, proctime0])
+   :- Exchange(distribution=[hash[a]])
+   :  +- Calc(select=[a, proctime])
+   :     +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)])
+   :        +- Calc(select=[a, PROCTIME() AS proctime, rowtime])
+   :           +- TableSourceScan(table=[[default_catalog, default_database, A, project=[a, rowtime], metadata=[]]], fields=[a, rowtime])
+   +- Exchange(distribution=[hash[a]])
+      +- Calc(select=[a, b, proctime])
+         +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)])
+            +- Calc(select=[a, b, PROCTIME() AS proctime, rowtime])
+               +- TableSourceScan(table=[[default_catalog, default_database, B, project=[a, b, rowtime], metadata=[]]], fields=[a, b, rowtime])
+
+== Optimized Execution Plan ==
+Calc(select=[a, b])
++- IntervalJoin(joinType=[InnerJoin], windowBounds=[isRowTime=false, leftLowerBound=-3600000, leftUpperBound=3600000, leftTimeIndex=1, rightTimeIndex=2], where=[((a = a0) AND (proctime >= (proctime0 - 3600000:INTERVAL HOUR)) AND (proctime <= (proctime0 + 3600000:INTERVAL HOUR)))], select=[a, proctime, a0, b, proctime0])
+   :- Exchange(distribution=[hash[a]])
+   :  +- Calc(select=[a, proctime])
+   :     +- WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])
+   :        +- Calc(select=[a, PROCTIME() AS proctime, rowtime])
+   :           +- TableSourceScan(table=[[default_catalog, default_database, A, project=[a, rowtime], metadata=[]]], fields=[a, rowtime])
+   +- Exchange(distribution=[hash[a]])
+      +- Calc(select=[a, b, proctime])
+         +- WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])
+            +- Calc(select=[a, b, PROCTIME() AS proctime, rowtime])
+               +- TableSourceScan(table=[[default_catalog, default_database, B, project=[a, b, rowtime], metadata=[]]], fields=[a, b, rowtime])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: A[]",
+    "pact" : "Data Source",
+    "contents" : "[]:TableSourceScan(table=[[default_catalog, default_database, A, project=[a, rowtime], metadata=[]]], fields=[a, rowtime])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "Calc[]",
+    "pact" : "Operator",
+    "contents" : "[]:Calc(select=[a, PROCTIME() AS proctime, rowtime])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "WatermarkAssigner[]",
+    "pact" : "Operator",
+    "contents" : "[]:WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Calc[]",
+    "pact" : "Operator",
+    "contents" : "[]:Calc(select=[a, proctime])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Source: B[]",
+    "pact" : "Data Source",
+    "contents" : "[]:TableSourceScan(table=[[default_catalog, default_database, B, project=[a, b, rowtime], metadata=[]]], fields=[a, b, rowtime])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "Calc[]",
+    "pact" : "Operator",
+    "contents" : "[]:Calc(select=[a, b, PROCTIME() AS proctime, rowtime])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "WatermarkAssigner[]",
+    "pact" : "Operator",
+    "contents" : "[]:WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Calc[]",
+    "pact" : "Operator",
+    "contents" : "[]:Calc(select=[a, b, proctime])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "IntervalJoin[]",
+    "pact" : "Operator",
+    "contents" : "[]:IntervalJoin(joinType=[InnerJoin], windowBounds=[isRowTime=false, leftLowerBound=-3600000, leftUpperBound=3600000, leftTimeIndex=1, rightTimeIndex=2], where=[((a = a0) AND (proctime >= (proctime0 - 3600000:INTERVAL HOUR)) AND (proctime <= (proctime0 + 3600000:INTERVAL HOUR)))], select=[a, proctime, a0, b, proctime0])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "HASH",
+      "side" : "second"
+    }, {
+      "id" : ,
+      "ship_strategy" : "HASH",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Calc[]",
+    "pact" : "Operator",
+    "contents" : "[]:Calc(select=[a, b])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testIntervalJoinNegativeWindow[isNameSimplifyEnabled=false]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalProject(a=[$0], b=[$7])
++- LogicalJoin(condition=[AND(=($0, $6), >=($5, +($11, 7200000:INTERVAL HOUR)), <=($5, +($11, 3600000:INTERVAL HOUR)))], joinType=[left])
+   :- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($4, 1000:INTERVAL SECOND)])
+   :  +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], rowtime=[$4], proctime=[PROCTIME()])
+   :     +- LogicalTableScan(table=[[default_catalog, default_database, A]])
+   +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($4, 1000:INTERVAL SECOND)])
+      +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], rowtime=[$4], proctime=[PROCTIME()])
+         +- LogicalTableScan(table=[[default_catalog, default_database, B]])
+
+== Optimized Physical Plan ==
+Calc(select=[a, b])
++- IntervalJoin(joinType=[LeftOuterJoin], windowBounds=[isRowTime=false, leftLowerBound=7200000, leftUpperBound=3600000, leftTimeIndex=1, rightTimeIndex=2], where=[AND(=(a, a0), >=(proctime, +(proctime0, 7200000:INTERVAL HOUR)), <=(proctime, +(proctime0, 3600000:INTERVAL HOUR)))], select=[a, proctime, a0, b, proctime0])
+   :- Exchange(distribution=[hash[a]])
+   :  +- Calc(select=[a, proctime])
+   :     +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)])
+   :        +- Calc(select=[a, PROCTIME() AS proctime, rowtime])
+   :           +- TableSourceScan(table=[[default_catalog, default_database, A, project=[a, rowtime], metadata=[]]], fields=[a, rowtime])
+   +- Exchange(distribution=[hash[a]])
+      +- Calc(select=[a, b, proctime])
+         +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)])
+            +- Calc(select=[a, b, PROCTIME() AS proctime, rowtime])
+               +- TableSourceScan(table=[[default_catalog, default_database, B, project=[a, b, rowtime], metadata=[]]], fields=[a, b, rowtime])
+
+== Optimized Execution Plan ==
+Calc(select=[a, b])
++- IntervalJoin(joinType=[LeftOuterJoin], windowBounds=[isRowTime=false, leftLowerBound=7200000, leftUpperBound=3600000, leftTimeIndex=1, rightTimeIndex=2], where=[((a = a0) AND (proctime >= (proctime0 + 7200000:INTERVAL HOUR)) AND (proctime <= (proctime0 + 3600000:INTERVAL HOUR)))], select=[a, proctime, a0, b, proctime0])
+   :- Exchange(distribution=[hash[a]])
+   :  +- Calc(select=[a, proctime])
+   :     +- WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])
+   :        +- Calc(select=[a, PROCTIME() AS proctime, rowtime])
+   :           +- TableSourceScan(table=[[default_catalog, default_database, A, project=[a, rowtime], metadata=[]]], fields=[a, rowtime])
+   +- Exchange(distribution=[hash[a]])
+      +- Calc(select=[a, b, proctime])
+         +- WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])
+            +- Calc(select=[a, b, PROCTIME() AS proctime, rowtime])
+               +- TableSourceScan(table=[[default_catalog, default_database, B, project=[a, b, rowtime], metadata=[]]], fields=[a, b, rowtime])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: TableSourceScan(table=[[default_catalog, default_database, A, project=[a, rowtime], metadata=[]]], fields=[a, rowtime])",
+    "pact" : "Data Source",
+    "contents" : "TableSourceScan(table=[[default_catalog, default_database, A, project=[a, rowtime], metadata=[]]], fields=[a, rowtime])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "Calc(select=[a, PROCTIME() AS proctime, rowtime])",
+    "pact" : "Operator",
+    "contents" : "Calc(select=[a, PROCTIME() AS proctime, rowtime])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])",
+    "pact" : "Operator",
+    "contents" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Calc(select=[a, proctime])",
+    "pact" : "Operator",
+    "contents" : "Calc(select=[a, proctime])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Source: TableSourceScan(table=[[default_catalog, default_database, B, project=[a, b, rowtime], metadata=[]]], fields=[a, b, rowtime])",
+    "pact" : "Data Source",
+    "contents" : "TableSourceScan(table=[[default_catalog, default_database, B, project=[a, b, rowtime], metadata=[]]], fields=[a, b, rowtime])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "Calc(select=[a, b, PROCTIME() AS proctime, rowtime])",
+    "pact" : "Operator",
+    "contents" : "Calc(select=[a, b, PROCTIME() AS proctime, rowtime])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])",
+    "pact" : "Operator",
+    "contents" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Calc(select=[a, b, proctime])",
+    "pact" : "Operator",
+    "contents" : "Calc(select=[a, b, proctime])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "filter all right input transformation",
+    "pact" : "Operator",
+    "contents" : "filter all right input transformation",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "HASH",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "pad left input transformation",
+    "pact" : "Operator",
+    "contents" : "pad left input transformation",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "HASH",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Calc(select=[a, b])",
+    "pact" : "Operator",
+    "contents" : "Calc(select=[a, b])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    }, {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testIntervalJoinNegativeWindow[isNameSimplifyEnabled=true]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalProject(a=[$0], b=[$7])
++- LogicalJoin(condition=[AND(=($0, $6), >=($5, +($11, 7200000:INTERVAL HOUR)), <=($5, +($11, 3600000:INTERVAL HOUR)))], joinType=[left])
+   :- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($4, 1000:INTERVAL SECOND)])
+   :  +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], rowtime=[$4], proctime=[PROCTIME()])
+   :     +- LogicalTableScan(table=[[default_catalog, default_database, A]])
+   +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($4, 1000:INTERVAL SECOND)])
+      +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], rowtime=[$4], proctime=[PROCTIME()])
+         +- LogicalTableScan(table=[[default_catalog, default_database, B]])
+
+== Optimized Physical Plan ==
+Calc(select=[a, b])
++- IntervalJoin(joinType=[LeftOuterJoin], windowBounds=[isRowTime=false, leftLowerBound=7200000, leftUpperBound=3600000, leftTimeIndex=1, rightTimeIndex=2], where=[AND(=(a, a0), >=(proctime, +(proctime0, 7200000:INTERVAL HOUR)), <=(proctime, +(proctime0, 3600000:INTERVAL HOUR)))], select=[a, proctime, a0, b, proctime0])
+   :- Exchange(distribution=[hash[a]])
+   :  +- Calc(select=[a, proctime])
+   :     +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)])
+   :        +- Calc(select=[a, PROCTIME() AS proctime, rowtime])
+   :           +- TableSourceScan(table=[[default_catalog, default_database, A, project=[a, rowtime], metadata=[]]], fields=[a, rowtime])
+   +- Exchange(distribution=[hash[a]])
+      +- Calc(select=[a, b, proctime])
+         +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)])
+            +- Calc(select=[a, b, PROCTIME() AS proctime, rowtime])
+               +- TableSourceScan(table=[[default_catalog, default_database, B, project=[a, b, rowtime], metadata=[]]], fields=[a, b, rowtime])
+
+== Optimized Execution Plan ==
+Calc(select=[a, b])
++- IntervalJoin(joinType=[LeftOuterJoin], windowBounds=[isRowTime=false, leftLowerBound=7200000, leftUpperBound=3600000, leftTimeIndex=1, rightTimeIndex=2], where=[((a = a0) AND (proctime >= (proctime0 + 7200000:INTERVAL HOUR)) AND (proctime <= (proctime0 + 3600000:INTERVAL HOUR)))], select=[a, proctime, a0, b, proctime0])
+   :- Exchange(distribution=[hash[a]])
+   :  +- Calc(select=[a, proctime])
+   :     +- WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])
+   :        +- Calc(select=[a, PROCTIME() AS proctime, rowtime])
+   :           +- TableSourceScan(table=[[default_catalog, default_database, A, project=[a, rowtime], metadata=[]]], fields=[a, rowtime])
+   +- Exchange(distribution=[hash[a]])
+      +- Calc(select=[a, b, proctime])
+         +- WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])
+            +- Calc(select=[a, b, PROCTIME() AS proctime, rowtime])
+               +- TableSourceScan(table=[[default_catalog, default_database, B, project=[a, b, rowtime], metadata=[]]], fields=[a, b, rowtime])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: A[]",
+    "pact" : "Data Source",
+    "contents" : "[]:TableSourceScan(table=[[default_catalog, default_database, A, project=[a, rowtime], metadata=[]]], fields=[a, rowtime])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "Calc[]",
+    "pact" : "Operator",
+    "contents" : "[]:Calc(select=[a, PROCTIME() AS proctime, rowtime])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "WatermarkAssigner[]",
+    "pact" : "Operator",
+    "contents" : "[]:WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Calc[]",
+    "pact" : "Operator",
+    "contents" : "[]:Calc(select=[a, proctime])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Source: B[]",
+    "pact" : "Data Source",
+    "contents" : "[]:TableSourceScan(table=[[default_catalog, default_database, B, project=[a, b, rowtime], metadata=[]]], fields=[a, b, rowtime])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "Calc[]",
+    "pact" : "Operator",
+    "contents" : "[]:Calc(select=[a, b, PROCTIME() AS proctime, rowtime])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "WatermarkAssigner[]",
+    "pact" : "Operator",
+    "contents" : "[]:WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Calc[]",
+    "pact" : "Operator",
+    "contents" : "[]:Calc(select=[a, b, proctime])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "FilterRight[]",
+    "pact" : "Operator",
+    "contents" : "[]:filter all right input transformation",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "HASH",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "PadLeft[]",
+    "pact" : "Operator",
+    "contents" : "[]:pad left input transformation",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "HASH",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Calc[]",
+    "pact" : "Operator",
+    "contents" : "[]:Calc(select=[a, b])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    }, {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoin[isNameSimplifyEnabled=false]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], a0=[$4], b0=[$5], c0=[$6], d0=[$7])
++- LogicalFilter(condition=[=($0, $7)])
+   +- LogicalJoin(condition=[true], joinType=[inner])
+      :- LogicalTableScan(table=[[default_catalog, default_database, A]])
+      +- LogicalTableScan(table=[[default_catalog, default_database, B]])
+
+== Optimized Physical Plan ==
+Join(joinType=[InnerJoin], where=[=(a, d0)], select=[a, b, c, d, a0, b0, c0, d0], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey])
+:- Exchange(distribution=[hash[a]])
+:  +- TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a, b, c, d])
++- Exchange(distribution=[hash[d]])
+   +- TableSourceScan(table=[[default_catalog, default_database, B]], fields=[a, b, c, d])
+
+== Optimized Execution Plan ==
+Join(joinType=[InnerJoin], where=[(a = d0)], select=[a, b, c, d, a0, b0, c0, d0], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey])
+:- Exchange(distribution=[hash[a]])
+:  +- TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a, b, c, d])
++- Exchange(distribution=[hash[d]])
+   +- TableSourceScan(table=[[default_catalog, default_database, B]], fields=[a, b, c, d])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a, b, c, d])",
+    "pact" : "Data Source",
+    "contents" : "TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a, b, c, d])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "Source: TableSourceScan(table=[[default_catalog, default_database, B]], fields=[a, b, c, d])",
+    "pact" : "Data Source",
+    "contents" : "TableSourceScan(table=[[default_catalog, default_database, B]], fields=[a, b, c, d])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "Join(joinType=[InnerJoin], where=[(a = d0)], select=[a, b, c, d, a0, b0, c0, d0], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey])",
+    "pact" : "Operator",
+    "contents" : "Join(joinType=[InnerJoin], where=[(a = d0)], select=[a, b, c, d, a0, b0, c0, d0], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "HASH",
+      "side" : "second"
+    }, {
+      "id" : ,
+      "ship_strategy" : "HASH",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoin[isNameSimplifyEnabled=true]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], a0=[$4], b0=[$5], c0=[$6], d0=[$7])
++- LogicalFilter(condition=[=($0, $7)])
+   +- LogicalJoin(condition=[true], joinType=[inner])
+      :- LogicalTableScan(table=[[default_catalog, default_database, A]])
+      +- LogicalTableScan(table=[[default_catalog, default_database, B]])
+
+== Optimized Physical Plan ==
+Join(joinType=[InnerJoin], where=[=(a, d0)], select=[a, b, c, d, a0, b0, c0, d0], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey])
+:- Exchange(distribution=[hash[a]])
+:  +- TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a, b, c, d])
++- Exchange(distribution=[hash[d]])
+   +- TableSourceScan(table=[[default_catalog, default_database, B]], fields=[a, b, c, d])
+
+== Optimized Execution Plan ==
+Join(joinType=[InnerJoin], where=[(a = d0)], select=[a, b, c, d, a0, b0, c0, d0], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey])
+:- Exchange(distribution=[hash[a]])
+:  +- TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a, b, c, d])
++- Exchange(distribution=[hash[d]])
+   +- TableSourceScan(table=[[default_catalog, default_database, B]], fields=[a, b, c, d])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: A[]",
+    "pact" : "Data Source",
+    "contents" : "[]:TableSourceScan(table=[[default_catalog, default_database, A]], fields=[a, b, c, d])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "Source: B[]",
+    "pact" : "Data Source",
+    "contents" : "[]:TableSourceScan(table=[[default_catalog, default_database, B]], fields=[a, b, c, d])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "Join[]",
+    "pact" : "Operator",
+    "contents" : "[]:Join(joinType=[InnerJoin], where=[(a = d0)], select=[a, b, c, d, a0, b0, c0, d0], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "HASH",
+      "side" : "second"
+    }, {
+      "id" : ,
+      "ship_strategy" : "HASH",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testLegacySourceSink[isNameSimplifyEnabled=false]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalLegacySink(name=[`default_catalog`.`default_database`.`MySink`], fields=[name, id, amount, price])
++- LogicalProject(name=[$0], id=[$1], amount=[$2], price=[$3])
+   +- LogicalTableScan(table=[[default_catalog, default_database, MySource, source: [filterPushedDown=[false], filter=[]]]])
+
+== Optimized Physical Plan ==
+LegacySink(name=[`default_catalog`.`default_database`.`MySink`], fields=[name, id, amount, price])
++- LegacyTableSourceScan(table=[[default_catalog, default_database, MySource, source: [filterPushedDown=[false], filter=[]]]], fields=[name, id, amount, price])
+
+== Optimized Execution Plan ==
+LegacySink(name=[`default_catalog`.`default_database`.`MySink`], fields=[name, id, amount, price])
++- LegacyTableSourceScan(table=[[default_catalog, default_database, MySource, source: [filterPushedDown=[false], filter=[]]]], fields=[name, id, amount, price])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: Collection Source",
+    "pact" : "Data Source",
+    "contents" : "Source: Collection Source",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "SourceConversion(table=[default_catalog.default_database.MySource, source: [filterPushedDown=[false], filter=[]]], fields=[name, id, amount, price])",
+    "pact" : "Operator",
+    "contents" : "SourceConversion(table=[default_catalog.default_database.MySource, source: [filterPushedDown=[false], filter=[]]], fields=[name, id, amount, price])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "SinkConversion To Row",
+    "pact" : "Operator",
+    "contents" : "SinkConversion To Row",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Sink: TestingAppendTableSink",
+    "pact" : "Data Sink",
+    "contents" : "Sink: TestingAppendTableSink",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testLegacySourceSink[isNameSimplifyEnabled=true]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalLegacySink(name=[`default_catalog`.`default_database`.`MySink`], fields=[name, id, amount, price])
++- LogicalProject(name=[$0], id=[$1], amount=[$2], price=[$3])
+   +- LogicalTableScan(table=[[default_catalog, default_database, MySource, source: [filterPushedDown=[false], filter=[]]]])
+
+== Optimized Physical Plan ==
+LegacySink(name=[`default_catalog`.`default_database`.`MySink`], fields=[name, id, amount, price])
++- LegacyTableSourceScan(table=[[default_catalog, default_database, MySource, source: [filterPushedDown=[false], filter=[]]]], fields=[name, id, amount, price])
+
+== Optimized Execution Plan ==
+LegacySink(name=[`default_catalog`.`default_database`.`MySink`], fields=[name, id, amount, price])
++- LegacyTableSourceScan(table=[[default_catalog, default_database, MySource, source: [filterPushedDown=[false], filter=[]]]], fields=[name, id, amount, price])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: Collection Source",
+    "pact" : "Data Source",
+    "contents" : "Source: Collection Source",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "SourceConversion[]",
+    "pact" : "Operator",
+    "contents" : "[]:SourceConversion(table=[default_catalog.default_database.MySource, source: [filterPushedDown=[false], filter=[]]], fields=[name, id, amount, price])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "SinkConversion[]",
+    "pact" : "Operator",
+    "contents" : "[]:SinkConversion To Row",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Sink: TestingAppendTableSink",
+    "pact" : "Data Sink",
+    "contents" : "Sink: TestingAppendTableSink",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testLocalGlobalWindowAggregate[isNameSimplifyEnabled=false]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalAggregate(group=[{0, 1, 2}], EXPR$3=[COUNT()], EXPR$4=[SUM($3)])
++- LogicalProject(b=[$1], window_start=[$6], window_end=[$7], a=[$0])
+   +- LogicalTableFunctionScan(invocation=[TUMBLE($5, DESCRIPTOR($4), 5000:INTERVAL SECOND)], rowType=[RecordType(BIGINT a, INTEGER b, VARCHAR(2147483647) c, BIGINT d, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)])
+      +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], rowtime=[$4], proctime=[$5])
+         +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($4, 1000:INTERVAL SECOND)])
+            +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], rowtime=[$4], proctime=[PROCTIME()])
+               +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
+
+== Optimized Physical Plan ==
+Calc(select=[b, window_start, window_end, EXPR$3, EXPR$4])
++- GlobalWindowAggregate(groupBy=[b], window=[TUMBLE(slice_end=[$slice_end], size=[5 s])], select=[b, COUNT(count1$0) AS EXPR$3, SUM(sum$1) AS EXPR$4, start('w$) AS window_start, end('w$) AS window_end])
+   +- Exchange(distribution=[hash[b]])
+      +- LocalWindowAggregate(groupBy=[b], window=[TUMBLE(time_col=[rowtime], size=[5 s])], select=[b, COUNT(*) AS count1$0, SUM(a) AS sum$1, slice_end('w$) AS $slice_end])
+         +- Calc(select=[b, a, rowtime])
+            +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)])
+               +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b, rowtime], metadata=[]]], fields=[a, b, rowtime])
+
+== Optimized Execution Plan ==
+Calc(select=[b, window_start, window_end, EXPR$3, EXPR$4])
++- GlobalWindowAggregate(groupBy=[b], window=[TUMBLE(slice_end=[$slice_end], size=[5 s])], select=[b, COUNT(count1$0) AS EXPR$3, SUM(sum$1) AS EXPR$4, start('w$) AS window_start, end('w$) AS window_end])
+   +- Exchange(distribution=[hash[b]])
+      +- LocalWindowAggregate(groupBy=[b], window=[TUMBLE(time_col=[rowtime], size=[5 s])], select=[b, COUNT(*) AS count1$0, SUM(a) AS sum$1, slice_end('w$) AS $slice_end])
+         +- Calc(select=[b, a, rowtime])
+            +- WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])
+               +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b, rowtime], metadata=[]]], fields=[a, b, rowtime])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b, rowtime], metadata=[]]], fields=[a, b, rowtime])",
+    "pact" : "Data Source",
+    "contents" : "TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b, rowtime], metadata=[]]], fields=[a, b, rowtime])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])",
+    "pact" : "Operator",
+    "contents" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Calc(select=[b, a, rowtime])",
+    "pact" : "Operator",
+    "contents" : "Calc(select=[b, a, rowtime])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "LocalWindowAggregate(groupBy=[b], window=[TUMBLE(time_col=[rowtime], size=[5 s])], select=[b, COUNT(*) AS count1$0, SUM(a) AS sum$1, slice_end('w$) AS $slice_end])",
+    "pact" : "Operator",
+    "contents" : "LocalWindowAggregate(groupBy=[b], window=[TUMBLE(time_col=[rowtime], size=[5 s])], select=[b, COUNT(*) AS count1$0, SUM(a) AS sum$1, slice_end('w$) AS $slice_end])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "GlobalWindowAggregate(groupBy=[b], window=[TUMBLE(slice_end=[$slice_end], size=[5 s])], select=[b, COUNT(count1$0) AS EXPR$3, SUM(sum$1) AS EXPR$4, start('w$) AS window_start, end('w$) AS window_end])",
+    "pact" : "Operator",
+    "contents" : "GlobalWindowAggregate(groupBy=[b], window=[TUMBLE(slice_end=[$slice_end], size=[5 s])], select=[b, COUNT(count1$0) AS EXPR$3, SUM(sum$1) AS EXPR$4, start('w$) AS window_start, end('w$) AS window_end])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "HASH",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Calc(select=[b, window_start, window_end, EXPR$3, EXPR$4])",
+    "pact" : "Operator",
+    "contents" : "Calc(select=[b, window_start, window_end, EXPR$3, EXPR$4])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testLocalGlobalWindowAggregate[isNameSimplifyEnabled=true]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalAggregate(group=[{0, 1, 2}], EXPR$3=[COUNT()], EXPR$4=[SUM($3)])
++- LogicalProject(b=[$1], window_start=[$6], window_end=[$7], a=[$0])
+   +- LogicalTableFunctionScan(invocation=[TUMBLE($5, DESCRIPTOR($4), 5000:INTERVAL SECOND)], rowType=[RecordType(BIGINT a, INTEGER b, VARCHAR(2147483647) c, BIGINT d, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)])
+      +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], rowtime=[$4], proctime=[$5])
+         +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($4, 1000:INTERVAL SECOND)])
+            +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], rowtime=[$4], proctime=[PROCTIME()])
+               +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
+
+== Optimized Physical Plan ==
+Calc(select=[b, window_start, window_end, EXPR$3, EXPR$4])
++- GlobalWindowAggregate(groupBy=[b], window=[TUMBLE(slice_end=[$slice_end], size=[5 s])], select=[b, COUNT(count1$0) AS EXPR$3, SUM(sum$1) AS EXPR$4, start('w$) AS window_start, end('w$) AS window_end])
+   +- Exchange(distribution=[hash[b]])
+      +- LocalWindowAggregate(groupBy=[b], window=[TUMBLE(time_col=[rowtime], size=[5 s])], select=[b, COUNT(*) AS count1$0, SUM(a) AS sum$1, slice_end('w$) AS $slice_end])
+         +- Calc(select=[b, a, rowtime])
+            +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)])
+               +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b, rowtime], metadata=[]]], fields=[a, b, rowtime])
+
+== Optimized Execution Plan ==
+Calc(select=[b, window_start, window_end, EXPR$3, EXPR$4])
++- GlobalWindowAggregate(groupBy=[b], window=[TUMBLE(slice_end=[$slice_end], size=[5 s])], select=[b, COUNT(count1$0) AS EXPR$3, SUM(sum$1) AS EXPR$4, start('w$) AS window_start, end('w$) AS window_end])
+   +- Exchange(distribution=[hash[b]])
+      +- LocalWindowAggregate(groupBy=[b], window=[TUMBLE(time_col=[rowtime], size=[5 s])], select=[b, COUNT(*) AS count1$0, SUM(a) AS sum$1, slice_end('w$) AS $slice_end])
+         +- Calc(select=[b, a, rowtime])
+            +- WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])
+               +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b, rowtime], metadata=[]]], fields=[a, b, rowtime])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: MyTable[]",
+    "pact" : "Data Source",
+    "contents" : "[]:TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b, rowtime], metadata=[]]], fields=[a, b, rowtime])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "WatermarkAssigner[]",
+    "pact" : "Operator",
+    "contents" : "[]:WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Calc[]",
+    "pact" : "Operator",
+    "contents" : "[]:Calc(select=[b, a, rowtime])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "LocalWindowAggregate[]",
+    "pact" : "Operator",
+    "contents" : "[]:LocalWindowAggregate(groupBy=[b], window=[TUMBLE(time_col=[rowtime], size=[5 s])], select=[b, COUNT(*) AS count1$0, SUM(a) AS sum$1, slice_end('w$) AS $slice_end])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "GlobalWindowAggregate[]",
+    "pact" : "Operator",
+    "contents" : "[]:GlobalWindowAggregate(groupBy=[b], window=[TUMBLE(slice_end=[$slice_end], size=[5 s])], select=[b, COUNT(count1$0) AS EXPR$3, SUM(sum$1) AS EXPR$4, start('w$) AS window_start, end('w$) AS window_end])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "HASH",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Calc[]",
+    "pact" : "Operator",
+    "contents" : "[]:Calc(select=[b, window_start, window_end, EXPR$3, EXPR$4])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testLookupJoin[isNameSimplifyEnabled=false]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], rowtime=[$4], proctime=[$5], id=[$6], name=[$7], age=[$8])
++- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{1, 5}])
+   :- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($4, 1000:INTERVAL SECOND)])
+   :  +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], rowtime=[$4], proctime=[PROCTIME()])
+   :     +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
+   +- LogicalFilter(condition=[=($cor0.b, $0)])
+      +- LogicalSnapshot(period=[$cor0.proctime])
+         +- LogicalTableScan(table=[[default_catalog, default_database, LookupTable]])
+
+== Optimized Physical Plan ==
+Calc(select=[a, b, c, d, rowtime, PROCTIME_MATERIALIZE(proctime) AS proctime, id, name, age])
++- LookupJoin(table=[default_catalog.default_database.LookupTable], joinType=[InnerJoin], async=[false], lookup=[id=b], select=[a, b, c, d, rowtime, proctime, id, name, age])
+   +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)])
+      +- Calc(select=[a, b, c, d, rowtime, PROCTIME() AS proctime])
+         +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, rowtime])
+
+== Optimized Execution Plan ==
+Calc(select=[a, b, c, d, rowtime, PROCTIME_MATERIALIZE(proctime) AS proctime, id, name, age])
++- LookupJoin(table=[default_catalog.default_database.LookupTable], joinType=[InnerJoin], async=[false], lookup=[id=b], select=[a, b, c, d, rowtime, proctime, id, name, age])
+   +- WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])
+      +- Calc(select=[a, b, c, d, rowtime, PROCTIME() AS proctime])
+         +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, rowtime])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, rowtime])",
+    "pact" : "Data Source",
+    "contents" : "TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, rowtime])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "Calc(select=[a, b, c, d, rowtime, PROCTIME() AS proctime])",
+    "pact" : "Operator",
+    "contents" : "Calc(select=[a, b, c, d, rowtime, PROCTIME() AS proctime])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])",
+    "pact" : "Operator",
+    "contents" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "LookupJoin(table=[default_catalog.default_database.LookupTable], joinType=[InnerJoin], async=[false], lookup=[id=b], select=[a, b, c, d, rowtime, proctime, id, name, age])",
+    "pact" : "Operator",
+    "contents" : "LookupJoin(table=[default_catalog.default_database.LookupTable], joinType=[InnerJoin], async=[false], lookup=[id=b], select=[a, b, c, d, rowtime, proctime, id, name, age])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Calc(select=[a, b, c, d, rowtime, PROCTIME_MATERIALIZE(proctime) AS proctime, id, name, age])",
+    "pact" : "Operator",
+    "contents" : "Calc(select=[a, b, c, d, rowtime, PROCTIME_MATERIALIZE(proctime) AS proctime, id, name, age])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testLookupJoin[isNameSimplifyEnabled=true]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], rowtime=[$4], proctime=[$5], id=[$6], name=[$7], age=[$8])
++- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{1, 5}])
+   :- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($4, 1000:INTERVAL SECOND)])
+   :  +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], rowtime=[$4], proctime=[PROCTIME()])
+   :     +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
+   +- LogicalFilter(condition=[=($cor0.b, $0)])
+      +- LogicalSnapshot(period=[$cor0.proctime])
+         +- LogicalTableScan(table=[[default_catalog, default_database, LookupTable]])
+
+== Optimized Physical Plan ==
+Calc(select=[a, b, c, d, rowtime, PROCTIME_MATERIALIZE(proctime) AS proctime, id, name, age])
++- LookupJoin(table=[default_catalog.default_database.LookupTable], joinType=[InnerJoin], async=[false], lookup=[id=b], select=[a, b, c, d, rowtime, proctime, id, name, age])
+   +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)])
+      +- Calc(select=[a, b, c, d, rowtime, PROCTIME() AS proctime])
+         +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, rowtime])
+
+== Optimized Execution Plan ==
+Calc(select=[a, b, c, d, rowtime, PROCTIME_MATERIALIZE(proctime) AS proctime, id, name, age])
++- LookupJoin(table=[default_catalog.default_database.LookupTable], joinType=[InnerJoin], async=[false], lookup=[id=b], select=[a, b, c, d, rowtime, proctime, id, name, age])
+   +- WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])
+      +- Calc(select=[a, b, c, d, rowtime, PROCTIME() AS proctime])
+         +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, rowtime])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: MyTable[]",
+    "pact" : "Data Source",
+    "contents" : "[]:TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, rowtime])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "Calc[]",
+    "pact" : "Operator",
+    "contents" : "[]:Calc(select=[a, b, c, d, rowtime, PROCTIME() AS proctime])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "WatermarkAssigner[]",
+    "pact" : "Operator",
+    "contents" : "[]:WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "LookupJoin[]",
+    "pact" : "Operator",
+    "contents" : "[]:LookupJoin(table=[default_catalog.default_database.LookupTable], joinType=[InnerJoin], async=[false], lookup=[id=b], select=[a, b, c, d, rowtime, proctime, id, name, age])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Calc[]",
+    "pact" : "Operator",
+    "contents" : "[]:Calc(select=[a, b, c, d, rowtime, PROCTIME_MATERIALIZE(proctime) AS proctime, id, name, age])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testMatch[isNameSimplifyEnabled=false]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalProject(aid=[$0], bid=[$1], cid=[$2])
++- LogicalMatch(partition=[[]], order=[[5 ASC-nulls-first]], outputFields=[[aid, bid, cid]], allRows=[false], after=[FLAG(SKIP TO NEXT ROW)], pattern=[((_UTF-16LE'A"', _UTF-16LE'l'), _UTF-16LE'C')], isStrictStarts=[false], isStrictEnds=[false], subsets=[[]], patternDefinitions=[[=(LAST(*.$0, 0), 1), =(LAST(*.$1, 0), 2), =(LAST(*.$2, 0), _UTF-16LE'c')]], inputFields=[[a, b, c, d, rowtime, proctime]])
+   +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($4, 1000:INTERVAL SECOND)])
+      +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], rowtime=[$4], proctime=[PROCTIME()])
+         +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
+
+== Optimized Physical Plan ==
+Match(orderBy=[proctime ASC], measures=[FINAL(A".a) AS aid, FINAL(l.a) AS bid, FINAL(C.a) AS cid], rowsPerMatch=[ONE ROW PER MATCH], after=[SKIP TO NEXT ROW], pattern=[((_UTF-16LE'A"', _UTF-16LE'l'), _UTF-16LE'C')], define=[{A"==(LAST(*.$0, 0), 1), l==(LAST(*.$1, 0), 2), C==(LAST(*.$2, 0), _UTF-16LE'c')}])
++- Exchange(distribution=[single])
+   +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)])
+      +- Calc(select=[a, b, c, d, rowtime, PROCTIME() AS proctime])
+         +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, rowtime])
+
+== Optimized Execution Plan ==
+Match(orderBy=[proctime ASC], measures=[FINAL(A".a) AS aid, FINAL(l.a) AS bid, FINAL(C.a) AS cid], rowsPerMatch=[ONE ROW PER MATCH], after=[SKIP TO NEXT ROW], pattern=[((_UTF-16LE'A"', _UTF-16LE'l'), _UTF-16LE'C')], define=[{A"==(LAST(*.$0, 0), 1), l==(LAST(*.$1, 0), 2), C==(LAST(*.$2, 0), _UTF-16LE'c')}])
++- Exchange(distribution=[single])
+   +- WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])
+      +- Calc(select=[a, b, c, d, rowtime, PROCTIME() AS proctime])
+         +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, rowtime])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, rowtime])",
+    "pact" : "Data Source",
+    "contents" : "TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, rowtime])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "Calc(select=[a, b, c, d, rowtime, PROCTIME() AS proctime])",
+    "pact" : "Operator",
+    "contents" : "Calc(select=[a, b, c, d, rowtime, PROCTIME() AS proctime])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])",
+    "pact" : "Operator",
+    "contents" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Match(orderBy=[proctime ASC], measures=[FINAL(A\".a) AS aid, FINAL(l.a) AS bid, FINAL(C.a) AS cid], rowsPerMatch=[ONE ROW PER MATCH], after=[SKIP TO NEXT ROW], pattern=[((_UTF-16LE'A\"', _UTF-16LE'l'), _UTF-16LE'C')], define=[{A\"==(LAST(*.$0, 0), 1), l==(LAST(*.$1, 0), 2), C==(LAST(*.$2, 0), _UTF-16LE'c')}])",
+    "pact" : "Operator",
+    "contents" : "Match(orderBy=[proctime ASC], measures=[FINAL(A\".a) AS aid, FINAL(l.a) AS bid, FINAL(C.a) AS cid], rowsPerMatch=[ONE ROW PER MATCH], after=[SKIP TO NEXT ROW], pattern=[((_UTF-16LE'A\"', _UTF-16LE'l'), _UTF-16LE'C')], define=[{A\"==(LAST(*.$0, 0), 1), l==(LAST(*.$1, 0), 2), C==(LAST(*.$2, 0), _UTF-16LE'c')}])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "GLOBAL",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testOverAggregate[isNameSimplifyEnabled=false]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalProject(b=[$1], cnt1=[COUNT($0) OVER (PARTITION BY $1 ORDER BY $4 NULLS FIRST ROWS 2 PRECEDING)])
++- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($4, 1000:INTERVAL SECOND)])
+   +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], rowtime=[$4], proctime=[PROCTIME()])
+      +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
+
+== Optimized Physical Plan ==
+Calc(select=[b, w0$o0 AS $1])
++- OverAggregate(partitionBy=[b], orderBy=[rowtime ASC], window=[ ROWS BETWEEN 2 PRECEDING AND CURRENT ROW], select=[a, b, rowtime, COUNT(a) AS w0$o0])
+   +- Exchange(distribution=[hash[b]])
+      +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)])
+         +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b, rowtime], metadata=[]]], fields=[a, b, rowtime])
+
+== Optimized Execution Plan ==
+Calc(select=[b, w0$o0 AS $1])
++- OverAggregate(partitionBy=[b], orderBy=[rowtime ASC], window=[ ROWS BETWEEN 2 PRECEDING AND CURRENT ROW], select=[a, b, rowtime, COUNT(a) AS w0$o0])
+   +- Exchange(distribution=[hash[b]])
+      +- WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])
+         +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b, rowtime], metadata=[]]], fields=[a, b, rowtime])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b, rowtime], metadata=[]]], fields=[a, b, rowtime])",
+    "pact" : "Data Source",
+    "contents" : "TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b, rowtime], metadata=[]]], fields=[a, b, rowtime])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])",
+    "pact" : "Operator",
+    "contents" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "OverAggregate(partitionBy=[b], orderBy=[rowtime ASC], window=[ ROWS BETWEEN 2 PRECEDING AND CURRENT ROW], select=[a, b, rowtime, COUNT(a) AS w0$o0])",
+    "pact" : "Operator",
+    "contents" : "OverAggregate(partitionBy=[b], orderBy=[rowtime ASC], window=[ ROWS BETWEEN 2 PRECEDING AND CURRENT ROW], select=[a, b, rowtime, COUNT(a) AS w0$o0])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "HASH",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Calc(select=[b, w0$o0 AS $1])",
+    "pact" : "Operator",
+    "contents" : "Calc(select=[b, w0$o0 AS $1])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testOverAggregate[isNameSimplifyEnabled=true]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalProject(b=[$1], cnt1=[COUNT($0) OVER (PARTITION BY $1 ORDER BY $4 NULLS FIRST ROWS 2 PRECEDING)])
++- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($4, 1000:INTERVAL SECOND)])
+   +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], rowtime=[$4], proctime=[PROCTIME()])
+      +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
+
+== Optimized Physical Plan ==
+Calc(select=[b, w0$o0 AS $1])
++- OverAggregate(partitionBy=[b], orderBy=[rowtime ASC], window=[ ROWS BETWEEN 2 PRECEDING AND CURRENT ROW], select=[a, b, rowtime, COUNT(a) AS w0$o0])
+   +- Exchange(distribution=[hash[b]])
+      +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)])
+         +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b, rowtime], metadata=[]]], fields=[a, b, rowtime])
+
+== Optimized Execution Plan ==
+Calc(select=[b, w0$o0 AS $1])
++- OverAggregate(partitionBy=[b], orderBy=[rowtime ASC], window=[ ROWS BETWEEN 2 PRECEDING AND CURRENT ROW], select=[a, b, rowtime, COUNT(a) AS w0$o0])
+   +- Exchange(distribution=[hash[b]])
+      +- WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])
+         +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b, rowtime], metadata=[]]], fields=[a, b, rowtime])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: MyTable[]",
+    "pact" : "Data Source",
+    "contents" : "[]:TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b, rowtime], metadata=[]]], fields=[a, b, rowtime])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "WatermarkAssigner[]",
+    "pact" : "Operator",
+    "contents" : "[]:WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "OverAggregate[]",
+    "pact" : "Operator",
+    "contents" : "[]:OverAggregate(partitionBy=[b], orderBy=[rowtime ASC], window=[ ROWS BETWEEN 2 PRECEDING AND CURRENT ROW], select=[a, b, rowtime, COUNT(a) AS w0$o0])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "HASH",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Calc[]",
+    "pact" : "Operator",
+    "contents" : "[]:Calc(select=[b, w0$o0 AS $1])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testRank[isNameSimplifyEnabled=false]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalProject(a=[$0], row_num=[$1])
++- LogicalFilter(condition=[<=($1, $0)])
+   +- LogicalProject(a=[$0], row_num=[ROW_NUMBER() OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST)])
+      +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
+
+== Optimized Physical Plan ==
+Calc(select=[a, w0$o0])
++- Rank(strategy=[AppendFastStrategy], rankType=[ROW_NUMBER], rankRange=[rankEnd=a], partitionBy=[b], orderBy=[a ASC], select=[a, b, w0$o0])
+   +- Exchange(distribution=[hash[b]])
+      +- Calc(select=[a, b])
+         +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d])
+
+== Optimized Execution Plan ==
+Calc(select=[a, w0$o0])
++- Rank(strategy=[AppendFastStrategy], rankType=[ROW_NUMBER], rankRange=[rankEnd=a], partitionBy=[b], orderBy=[a ASC], select=[a, b, w0$o0])
+   +- Exchange(distribution=[hash[b]])
+      +- Calc(select=[a, b])
+         +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d])",
+    "pact" : "Data Source",
+    "contents" : "TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "Calc(select=[a, b])",
+    "pact" : "Operator",
+    "contents" : "Calc(select=[a, b])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Rank(strategy=[AppendFastStrategy], rankType=[ROW_NUMBER], rankRange=[rankEnd=a], partitionBy=[b], orderBy=[a ASC], select=[a, b, w0$o0])",
+    "pact" : "Operator",
+    "contents" : "Rank(strategy=[AppendFastStrategy], rankType=[ROW_NUMBER], rankRange=[rankEnd=a], partitionBy=[b], orderBy=[a ASC], select=[a, b, w0$o0])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "HASH",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Calc(select=[a, w0$o0])",
+    "pact" : "Operator",
+    "contents" : "Calc(select=[a, w0$o0])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testRank[isNameSimplifyEnabled=true]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalProject(a=[$0], row_num=[$1])
++- LogicalFilter(condition=[<=($1, $0)])
+   +- LogicalProject(a=[$0], row_num=[ROW_NUMBER() OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST)])
+      +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
+
+== Optimized Physical Plan ==
+Calc(select=[a, w0$o0])
++- Rank(strategy=[AppendFastStrategy], rankType=[ROW_NUMBER], rankRange=[rankEnd=a], partitionBy=[b], orderBy=[a ASC], select=[a, b, w0$o0])
+   +- Exchange(distribution=[hash[b]])
+      +- Calc(select=[a, b])
+         +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d])
+
+== Optimized Execution Plan ==
+Calc(select=[a, w0$o0])
++- Rank(strategy=[AppendFastStrategy], rankType=[ROW_NUMBER], rankRange=[rankEnd=a], partitionBy=[b], orderBy=[a ASC], select=[a, b, w0$o0])
+   +- Exchange(distribution=[hash[b]])
+      +- Calc(select=[a, b])
+         +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: MyTable[]",
+    "pact" : "Data Source",
+    "contents" : "[]:TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "Calc[]",
+    "pact" : "Operator",
+    "contents" : "[]:Calc(select=[a, b])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Rank[]",
+    "pact" : "Operator",
+    "contents" : "[]:Rank(strategy=[AppendFastStrategy], rankType=[ROW_NUMBER], rankRange=[rankEnd=a], partitionBy=[b], orderBy=[a ASC], select=[a, b, w0$o0])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "HASH",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Calc[]",
+    "pact" : "Operator",
+    "contents" : "[]:Calc(select=[a, w0$o0])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testTableGroupAggregate[isNameSimplifyEnabled=false]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalProject(f0=[$0], f1=[$1])
++- LogicalTableAggregate(group=[{}], tableAggregate=[[org$apache$flink$table$planner$utils$Top3$3b7f784b4c3a67151e3462287356c734($0)]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, MySource]])
+
+== Optimized Physical Plan ==
+GroupTableAggregate(select=[org$apache$flink$table$planner$utils$Top3$3b7f784b4c3a67151e3462287356c734(f0) AS (f0, f1)])
++- Exchange(distribution=[single])
+   +- DataStreamScan(table=[[default_catalog, default_database, MySource]], fields=[f0])
+
+== Optimized Execution Plan ==
+GroupTableAggregate(select=[org$apache$flink$table$planner$utils$Top3$3b7f784b4c3a67151e3462287356c734(f0) AS (f0, f1)])
++- Exchange(distribution=[single])
+   +- DataStreamScan(table=[[default_catalog, default_database, MySource]], fields=[f0])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: Collection Source",
+    "pact" : "Data Source",
+    "contents" : "Source: Collection Source",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "SourceConversion(table=[default_catalog.default_database.MySource], fields=[f0])",
+    "pact" : "Operator",
+    "contents" : "SourceConversion(table=[default_catalog.default_database.MySource], fields=[f0])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "GroupTableAggregate(select=[org$apache$flink$table$planner$utils$Top3$3b7f784b4c3a67151e3462287356c734(f0) AS (f0, f1)])",
+    "pact" : "Operator",
+    "contents" : "GroupTableAggregate(select=[org$apache$flink$table$planner$utils$Top3$3b7f784b4c3a67151e3462287356c734(f0) AS (f0, f1)])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "GLOBAL",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testTableGroupAggregate[isNameSimplifyEnabled=true]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalProject(f0=[$0], f1=[$1])
++- LogicalTableAggregate(group=[{}], tableAggregate=[[org$apache$flink$table$planner$utils$Top3$3b7f784b4c3a67151e3462287356c734($0)]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, MySource]])
+
+== Optimized Physical Plan ==
+GroupTableAggregate(select=[org$apache$flink$table$planner$utils$Top3$3b7f784b4c3a67151e3462287356c734(f0) AS (f0, f1)])
++- Exchange(distribution=[single])
+   +- DataStreamScan(table=[[default_catalog, default_database, MySource]], fields=[f0])
+
+== Optimized Execution Plan ==
+GroupTableAggregate(select=[org$apache$flink$table$planner$utils$Top3$3b7f784b4c3a67151e3462287356c734(f0) AS (f0, f1)])
++- Exchange(distribution=[single])
+   +- DataStreamScan(table=[[default_catalog, default_database, MySource]], fields=[f0])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: Collection Source",
+    "pact" : "Data Source",
+    "contents" : "Source: Collection Source",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "SourceConversion[]",
+    "pact" : "Operator",
+    "contents" : "[]:SourceConversion(table=[default_catalog.default_database.MySource], fields=[f0])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "GroupTableAggregate[]",
+    "pact" : "Operator",
+    "contents" : "[]:GroupTableAggregate(select=[org$apache$flink$table$planner$utils$Top3$3b7f784b4c3a67151e3462287356c734(f0) AS (f0, f1)])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "GLOBAL",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testTemporalJoin[isNameSimplifyEnabled=false]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalProject(EXPR$0=[*($0, $5)])
++- LogicalFilter(condition=[=($1, $4)])
+   +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{2}])
+      :- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[$2])
+      :  +- LogicalProject(amount=[$0], currency=[$1], rowtime=[$2], proctime=[PROCTIME()])
+      :     +- LogicalTableScan(table=[[default_catalog, default_database, Orders]])
+      +- LogicalTableFunctionScan(invocation=[Rates($cor0.rowtime)], rowType=[RecordType:peek_no_expand(VARCHAR(2147483647) currency, INTEGER rate, TIMESTAMP(3) *ROWTIME* rowtime)])
+
+== Optimized Physical Plan ==
+Calc(select=[*(amount, rate) AS EXPR$0])
++- TemporalJoin(joinType=[InnerJoin], where=[AND(__TEMPORAL_JOIN_CONDITION(rowtime, rowtime0, __TEMPORAL_JOIN_CONDITION_PRIMARY_KEY(currency0)), =(currency, currency0))], select=[amount, currency, rowtime, currency0, rate, rowtime0])
+   :- Exchange(distribution=[hash[currency]])
+   :  +- WatermarkAssigner(rowtime=[rowtime], watermark=[rowtime])
+   :     +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[amount, currency, rowtime])
+   +- Exchange(distribution=[hash[currency]])
+      +- WatermarkAssigner(rowtime=[rowtime], watermark=[rowtime])
+         +- TableSourceScan(table=[[default_catalog, default_database, RatesHistory]], fields=[currency, rate, rowtime])
+
+== Optimized Execution Plan ==
+Calc(select=[(amount * rate) AS EXPR$0])
++- TemporalJoin(joinType=[InnerJoin], where=[(__TEMPORAL_JOIN_CONDITION(rowtime, rowtime0, __TEMPORAL_JOIN_CONDITION_PRIMARY_KEY(currency0)) AND (currency = currency0))], select=[amount, currency, rowtime, currency0, rate, rowtime0])
+   :- Exchange(distribution=[hash[currency]])
+   :  +- WatermarkAssigner(rowtime=[rowtime], watermark=[rowtime])
+   :     +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[amount, currency, rowtime])
+   +- Exchange(distribution=[hash[currency]])
+      +- WatermarkAssigner(rowtime=[rowtime], watermark=[rowtime])
+         +- TableSourceScan(table=[[default_catalog, default_database, RatesHistory]], fields=[currency, rate, rowtime])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[amount, currency, rowtime])",
+    "pact" : "Data Source",
+    "contents" : "TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[amount, currency, rowtime])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "WatermarkAssigner(rowtime=[rowtime], watermark=[rowtime])",
+    "pact" : "Operator",
+    "contents" : "WatermarkAssigner(rowtime=[rowtime], watermark=[rowtime])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Source: TableSourceScan(table=[[default_catalog, default_database, RatesHistory]], fields=[currency, rate, rowtime])",
+    "pact" : "Data Source",
+    "contents" : "TableSourceScan(table=[[default_catalog, default_database, RatesHistory]], fields=[currency, rate, rowtime])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "WatermarkAssigner(rowtime=[rowtime], watermark=[rowtime])",
+    "pact" : "Operator",
+    "contents" : "WatermarkAssigner(rowtime=[rowtime], watermark=[rowtime])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "TemporalJoin(joinType=[InnerJoin], where=[(__TEMPORAL_JOIN_CONDITION(rowtime, rowtime0, __TEMPORAL_JOIN_CONDITION_PRIMARY_KEY(currency0)) AND (currency = currency0))], select=[amount, currency, rowtime, currency0, rate, rowtime0])",
+    "pact" : "Operator",
+    "contents" : "TemporalJoin(joinType=[InnerJoin], where=[(__TEMPORAL_JOIN_CONDITION(rowtime, rowtime0, __TEMPORAL_JOIN_CONDITION_PRIMARY_KEY(currency0)) AND (currency = currency0))], select=[amount, currency, rowtime, currency0, rate, rowtime0])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "HASH",
+      "side" : "second"
+    }, {
+      "id" : ,
+      "ship_strategy" : "HASH",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Calc(select=[(amount * rate) AS EXPR$0])",
+    "pact" : "Operator",
+    "contents" : "Calc(select=[(amount * rate) AS EXPR$0])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testTemporalJoin[isNameSimplifyEnabled=true]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalProject(EXPR$0=[*($0, $5)])
++- LogicalFilter(condition=[=($1, $4)])
+   +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{2}])
+      :- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[$2])
+      :  +- LogicalProject(amount=[$0], currency=[$1], rowtime=[$2], proctime=[PROCTIME()])
+      :     +- LogicalTableScan(table=[[default_catalog, default_database, Orders]])
+      +- LogicalTableFunctionScan(invocation=[Rates($cor0.rowtime)], rowType=[RecordType:peek_no_expand(VARCHAR(2147483647) currency, INTEGER rate, TIMESTAMP(3) *ROWTIME* rowtime)])
+
+== Optimized Physical Plan ==
+Calc(select=[*(amount, rate) AS EXPR$0])
++- TemporalJoin(joinType=[InnerJoin], where=[AND(__TEMPORAL_JOIN_CONDITION(rowtime, rowtime0, __TEMPORAL_JOIN_CONDITION_PRIMARY_KEY(currency0)), =(currency, currency0))], select=[amount, currency, rowtime, currency0, rate, rowtime0])
+   :- Exchange(distribution=[hash[currency]])
+   :  +- WatermarkAssigner(rowtime=[rowtime], watermark=[rowtime])
+   :     +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[amount, currency, rowtime])
+   +- Exchange(distribution=[hash[currency]])
+      +- WatermarkAssigner(rowtime=[rowtime], watermark=[rowtime])
+         +- TableSourceScan(table=[[default_catalog, default_database, RatesHistory]], fields=[currency, rate, rowtime])
+
+== Optimized Execution Plan ==
+Calc(select=[(amount * rate) AS EXPR$0])
++- TemporalJoin(joinType=[InnerJoin], where=[(__TEMPORAL_JOIN_CONDITION(rowtime, rowtime0, __TEMPORAL_JOIN_CONDITION_PRIMARY_KEY(currency0)) AND (currency = currency0))], select=[amount, currency, rowtime, currency0, rate, rowtime0])
+   :- Exchange(distribution=[hash[currency]])
+   :  +- WatermarkAssigner(rowtime=[rowtime], watermark=[rowtime])
+   :     +- TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[amount, currency, rowtime])
+   +- Exchange(distribution=[hash[currency]])
+      +- WatermarkAssigner(rowtime=[rowtime], watermark=[rowtime])
+         +- TableSourceScan(table=[[default_catalog, default_database, RatesHistory]], fields=[currency, rate, rowtime])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: Orders[]",
+    "pact" : "Data Source",
+    "contents" : "[]:TableSourceScan(table=[[default_catalog, default_database, Orders]], fields=[amount, currency, rowtime])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "WatermarkAssigner[]",
+    "pact" : "Operator",
+    "contents" : "[]:WatermarkAssigner(rowtime=[rowtime], watermark=[rowtime])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Source: RatesHistory[]",
+    "pact" : "Data Source",
+    "contents" : "[]:TableSourceScan(table=[[default_catalog, default_database, RatesHistory]], fields=[currency, rate, rowtime])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "WatermarkAssigner[]",
+    "pact" : "Operator",
+    "contents" : "[]:WatermarkAssigner(rowtime=[rowtime], watermark=[rowtime])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "TemporalJoin[]",
+    "pact" : "Operator",
+    "contents" : "[]:TemporalJoin(joinType=[InnerJoin], where=[(__TEMPORAL_JOIN_CONDITION(rowtime, rowtime0, __TEMPORAL_JOIN_CONDITION_PRIMARY_KEY(currency0)) AND (currency = currency0))], select=[amount, currency, rowtime, currency0, rate, rowtime0])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "HASH",
+      "side" : "second"
+    }, {
+      "id" : ,
+      "ship_strategy" : "HASH",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Calc[]",
+    "pact" : "Operator",
+    "contents" : "[]:Calc(select=[(amount * rate) AS EXPR$0])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testMatch[isNameSimplifyEnabled=true]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalProject(aid=[$0], bid=[$1], cid=[$2])
++- LogicalMatch(partition=[[]], order=[[5 ASC-nulls-first]], outputFields=[[aid, bid, cid]], allRows=[false], after=[FLAG(SKIP TO NEXT ROW)], pattern=[((_UTF-16LE'A"', _UTF-16LE'l'), _UTF-16LE'C')], isStrictStarts=[false], isStrictEnds=[false], subsets=[[]], patternDefinitions=[[=(LAST(*.$0, 0), 1), =(LAST(*.$1, 0), 2), =(LAST(*.$2, 0), _UTF-16LE'c')]], inputFields=[[a, b, c, d, rowtime, proctime]])
+   +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($4, 1000:INTERVAL SECOND)])
+      +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], rowtime=[$4], proctime=[PROCTIME()])
+         +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
+
+== Optimized Physical Plan ==
+Match(orderBy=[proctime ASC], measures=[FINAL(A".a) AS aid, FINAL(l.a) AS bid, FINAL(C.a) AS cid], rowsPerMatch=[ONE ROW PER MATCH], after=[SKIP TO NEXT ROW], pattern=[((_UTF-16LE'A"', _UTF-16LE'l'), _UTF-16LE'C')], define=[{A"==(LAST(*.$0, 0), 1), l==(LAST(*.$1, 0), 2), C==(LAST(*.$2, 0), _UTF-16LE'c')}])
++- Exchange(distribution=[single])
+   +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)])
+      +- Calc(select=[a, b, c, d, rowtime, PROCTIME() AS proctime])
+         +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, rowtime])
+
+== Optimized Execution Plan ==
+Match(orderBy=[proctime ASC], measures=[FINAL(A".a) AS aid, FINAL(l.a) AS bid, FINAL(C.a) AS cid], rowsPerMatch=[ONE ROW PER MATCH], after=[SKIP TO NEXT ROW], pattern=[((_UTF-16LE'A"', _UTF-16LE'l'), _UTF-16LE'C')], define=[{A"==(LAST(*.$0, 0), 1), l==(LAST(*.$1, 0), 2), C==(LAST(*.$2, 0), _UTF-16LE'c')}])
++- Exchange(distribution=[single])
+   +- WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])
+      +- Calc(select=[a, b, c, d, rowtime, PROCTIME() AS proctime])
+         +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, rowtime])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: MyTable[]",
+    "pact" : "Data Source",
+    "contents" : "[]:TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, rowtime])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "Calc[]",
+    "pact" : "Operator",
+    "contents" : "[]:Calc(select=[a, b, c, d, rowtime, PROCTIME() AS proctime])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "WatermarkAssigner[]",
+    "pact" : "Operator",
+    "contents" : "[]:WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Match[]",
+    "pact" : "Operator",
+    "contents" : "[]:Match(orderBy=[proctime ASC], measures=[FINAL(A\".a) AS aid, FINAL(l.a) AS bid, FINAL(C.a) AS cid], rowsPerMatch=[ONE ROW PER MATCH], after=[SKIP TO NEXT ROW], pattern=[((_UTF-16LE'A\"', _UTF-16LE'l'), _UTF-16LE'C')], define=[{A\"==(LAST(*.$0, 0), 1), l==(LAST(*.$1, 0), 2), C==(LAST(*.$2, 0), _UTF-16LE'c')}])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "GLOBAL",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testTemporalSortOnEventTime[isNameSimplifyEnabled=false]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalProject(a=[$0])
++- LogicalSort(sort0=[$1], sort1=[$2], dir0=[ASC-nulls-first], dir1=[ASC-nulls-first])
+   +- LogicalProject(a=[$0], rowtime=[$4], c=[$2])
+      +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($4, 1000:INTERVAL SECOND)])
+         +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], rowtime=[$4], proctime=[PROCTIME()])
+            +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
+
+== Optimized Physical Plan ==
+Calc(select=[a])
++- TemporalSort(orderBy=[rowtime ASC, c ASC])
+   +- Exchange(distribution=[single])
+      +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)])
+         +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, rowtime, c], metadata=[]]], fields=[a, rowtime, c])
+
+== Optimized Execution Plan ==
+Calc(select=[a])
++- TemporalSort(orderBy=[rowtime ASC, c ASC])
+   +- Exchange(distribution=[single])
+      +- WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])
+         +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, rowtime, c], metadata=[]]], fields=[a, rowtime, c])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, rowtime, c], metadata=[]]], fields=[a, rowtime, c])",
+    "pact" : "Data Source",
+    "contents" : "TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, rowtime, c], metadata=[]]], fields=[a, rowtime, c])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])",
+    "pact" : "Operator",
+    "contents" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "TemporalSort(orderBy=[rowtime ASC, c ASC])",
+    "pact" : "Operator",
+    "contents" : "TemporalSort(orderBy=[rowtime ASC, c ASC])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "GLOBAL",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Calc(select=[a])",
+    "pact" : "Operator",
+    "contents" : "Calc(select=[a])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testTemporalSortOnEventTime[isNameSimplifyEnabled=true]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalProject(a=[$0])
++- LogicalSort(sort0=[$1], sort1=[$2], dir0=[ASC-nulls-first], dir1=[ASC-nulls-first])
+   +- LogicalProject(a=[$0], rowtime=[$4], c=[$2])
+      +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($4, 1000:INTERVAL SECOND)])
+         +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], rowtime=[$4], proctime=[PROCTIME()])
+            +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
+
+== Optimized Physical Plan ==
+Calc(select=[a])
++- TemporalSort(orderBy=[rowtime ASC, c ASC])
+   +- Exchange(distribution=[single])
+      +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)])
+         +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, rowtime, c], metadata=[]]], fields=[a, rowtime, c])
+
+== Optimized Execution Plan ==
+Calc(select=[a])
++- TemporalSort(orderBy=[rowtime ASC, c ASC])
+   +- Exchange(distribution=[single])
+      +- WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])
+         +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, rowtime, c], metadata=[]]], fields=[a, rowtime, c])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: MyTable[]",
+    "pact" : "Data Source",
+    "contents" : "[]:TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, rowtime, c], metadata=[]]], fields=[a, rowtime, c])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "WatermarkAssigner[]",
+    "pact" : "Operator",
+    "contents" : "[]:WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "TemporalSort[]",
+    "pact" : "Operator",
+    "contents" : "[]:TemporalSort(orderBy=[rowtime ASC, c ASC])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "GLOBAL",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Calc[]",
+    "pact" : "Operator",
+    "contents" : "[]:Calc(select=[a])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testTemporalSortOnProcTime[isNameSimplifyEnabled=false]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalProject(a=[$0])
++- LogicalSort(sort0=[$1], sort1=[$2], dir0=[ASC-nulls-first], dir1=[ASC-nulls-first])
+   +- LogicalProject(a=[$0], proctime=[$5], c=[$2])
+      +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($4, 1000:INTERVAL SECOND)])
+         +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], rowtime=[$4], proctime=[PROCTIME()])
+            +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
+
+== Optimized Physical Plan ==
+Calc(select=[a])
++- TemporalSort(orderBy=[proctime ASC, c ASC])
+   +- Exchange(distribution=[single])
+      +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)])
+         +- Calc(select=[a, PROCTIME() AS proctime, c, rowtime])
+            +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, c, rowtime], metadata=[]]], fields=[a, c, rowtime])
+
+== Optimized Execution Plan ==
+Calc(select=[a])
++- TemporalSort(orderBy=[proctime ASC, c ASC])
+   +- Exchange(distribution=[single])
+      +- WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])
+         +- Calc(select=[a, PROCTIME() AS proctime, c, rowtime])
+            +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, c, rowtime], metadata=[]]], fields=[a, c, rowtime])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, c, rowtime], metadata=[]]], fields=[a, c, rowtime])",
+    "pact" : "Data Source",
+    "contents" : "TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, c, rowtime], metadata=[]]], fields=[a, c, rowtime])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "Calc(select=[a, PROCTIME() AS proctime, c, rowtime])",
+    "pact" : "Operator",
+    "contents" : "Calc(select=[a, PROCTIME() AS proctime, c, rowtime])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])",
+    "pact" : "Operator",
+    "contents" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "TemporalSort(orderBy=[proctime ASC, c ASC])",
+    "pact" : "Operator",
+    "contents" : "TemporalSort(orderBy=[proctime ASC, c ASC])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "GLOBAL",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Calc(select=[a])",
+    "pact" : "Operator",
+    "contents" : "Calc(select=[a])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testTemporalSortOnProcTime[isNameSimplifyEnabled=true]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalProject(a=[$0])
++- LogicalSort(sort0=[$1], sort1=[$2], dir0=[ASC-nulls-first], dir1=[ASC-nulls-first])
+   +- LogicalProject(a=[$0], proctime=[$5], c=[$2])
+      +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($4, 1000:INTERVAL SECOND)])
+         +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], rowtime=[$4], proctime=[PROCTIME()])
+            +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
+
+== Optimized Physical Plan ==
+Calc(select=[a])
++- TemporalSort(orderBy=[proctime ASC, c ASC])
+   +- Exchange(distribution=[single])
+      +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)])
+         +- Calc(select=[a, PROCTIME() AS proctime, c, rowtime])
+            +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, c, rowtime], metadata=[]]], fields=[a, c, rowtime])
+
+== Optimized Execution Plan ==
+Calc(select=[a])
++- TemporalSort(orderBy=[proctime ASC, c ASC])
+   +- Exchange(distribution=[single])
+      +- WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])
+         +- Calc(select=[a, PROCTIME() AS proctime, c, rowtime])
+            +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, c, rowtime], metadata=[]]], fields=[a, c, rowtime])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: MyTable[]",
+    "pact" : "Data Source",
+    "contents" : "[]:TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, c, rowtime], metadata=[]]], fields=[a, c, rowtime])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "Calc[]",
+    "pact" : "Operator",
+    "contents" : "[]:Calc(select=[a, PROCTIME() AS proctime, c, rowtime])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "WatermarkAssigner[]",
+    "pact" : "Operator",
+    "contents" : "[]:WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "TemporalSort[]",
+    "pact" : "Operator",
+    "contents" : "[]:TemporalSort(orderBy=[proctime ASC, c ASC])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "GLOBAL",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Calc[]",
+    "pact" : "Operator",
+    "contents" : "[]:Calc(select=[a])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testWindowAggregate[isNameSimplifyEnabled=false]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalAggregate(group=[{0, 1, 2}], EXPR$3=[COUNT()], EXPR$4=[SUM($3)])
++- LogicalProject(b=[$1], window_start=[$6], window_end=[$7], a=[$0])
+   +- LogicalTableFunctionScan(invocation=[TUMBLE($5, DESCRIPTOR($4), 5000:INTERVAL SECOND)], rowType=[RecordType(BIGINT a, INTEGER b, VARCHAR(2147483647) c, BIGINT d, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)])
+      +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], rowtime=[$4], proctime=[$5])
+         +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($4, 1000:INTERVAL SECOND)])
+            +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], rowtime=[$4], proctime=[PROCTIME()])
+               +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
+
+== Optimized Physical Plan ==
+Calc(select=[b, window_start, window_end, EXPR$3, EXPR$4])
++- WindowAggregate(groupBy=[b], window=[TUMBLE(time_col=[rowtime], size=[5 s])], select=[b, COUNT(*) AS EXPR$3, SUM(a) AS EXPR$4, start('w$) AS window_start, end('w$) AS window_end])
+   +- Exchange(distribution=[hash[b]])
+      +- Calc(select=[b, a, rowtime])
+         +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)])
+            +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b, rowtime], metadata=[]]], fields=[a, b, rowtime])
+
+== Optimized Execution Plan ==
+Calc(select=[b, window_start, window_end, EXPR$3, EXPR$4])
++- WindowAggregate(groupBy=[b], window=[TUMBLE(time_col=[rowtime], size=[5 s])], select=[b, COUNT(*) AS EXPR$3, SUM(a) AS EXPR$4, start('w$) AS window_start, end('w$) AS window_end])
+   +- Exchange(distribution=[hash[b]])
+      +- Calc(select=[b, a, rowtime])
+         +- WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])
+            +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b, rowtime], metadata=[]]], fields=[a, b, rowtime])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b, rowtime], metadata=[]]], fields=[a, b, rowtime])",
+    "pact" : "Data Source",
+    "contents" : "TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b, rowtime], metadata=[]]], fields=[a, b, rowtime])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])",
+    "pact" : "Operator",
+    "contents" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Calc(select=[b, a, rowtime])",
+    "pact" : "Operator",
+    "contents" : "Calc(select=[b, a, rowtime])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "WindowAggregate(groupBy=[b], window=[TUMBLE(time_col=[rowtime], size=[5 s])], select=[b, COUNT(*) AS EXPR$3, SUM(a) AS EXPR$4, start('w$) AS window_start, end('w$) AS window_end])",
+    "pact" : "Operator",
+    "contents" : "WindowAggregate(groupBy=[b], window=[TUMBLE(time_col=[rowtime], size=[5 s])], select=[b, COUNT(*) AS EXPR$3, SUM(a) AS EXPR$4, start('w$) AS window_start, end('w$) AS window_end])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "HASH",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Calc(select=[b, window_start, window_end, EXPR$3, EXPR$4])",
+    "pact" : "Operator",
+    "contents" : "Calc(select=[b, window_start, window_end, EXPR$3, EXPR$4])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testWindowAggregate[isNameSimplifyEnabled=true]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalAggregate(group=[{0, 1, 2}], EXPR$3=[COUNT()], EXPR$4=[SUM($3)])
++- LogicalProject(b=[$1], window_start=[$6], window_end=[$7], a=[$0])
+   +- LogicalTableFunctionScan(invocation=[TUMBLE($5, DESCRIPTOR($4), 5000:INTERVAL SECOND)], rowType=[RecordType(BIGINT a, INTEGER b, VARCHAR(2147483647) c, BIGINT d, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)])
+      +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], rowtime=[$4], proctime=[$5])
+         +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($4, 1000:INTERVAL SECOND)])
+            +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], rowtime=[$4], proctime=[PROCTIME()])
+               +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
+
+== Optimized Physical Plan ==
+Calc(select=[b, window_start, window_end, EXPR$3, EXPR$4])
++- WindowAggregate(groupBy=[b], window=[TUMBLE(time_col=[rowtime], size=[5 s])], select=[b, COUNT(*) AS EXPR$3, SUM(a) AS EXPR$4, start('w$) AS window_start, end('w$) AS window_end])
+   +- Exchange(distribution=[hash[b]])
+      +- Calc(select=[b, a, rowtime])
+         +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)])
+            +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b, rowtime], metadata=[]]], fields=[a, b, rowtime])
+
+== Optimized Execution Plan ==
+Calc(select=[b, window_start, window_end, EXPR$3, EXPR$4])
++- WindowAggregate(groupBy=[b], window=[TUMBLE(time_col=[rowtime], size=[5 s])], select=[b, COUNT(*) AS EXPR$3, SUM(a) AS EXPR$4, start('w$) AS window_start, end('w$) AS window_end])
+   +- Exchange(distribution=[hash[b]])
+      +- Calc(select=[b, a, rowtime])
+         +- WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])
+            +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b, rowtime], metadata=[]]], fields=[a, b, rowtime])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: MyTable[]",
+    "pact" : "Data Source",
+    "contents" : "[]:TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b, rowtime], metadata=[]]], fields=[a, b, rowtime])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "WatermarkAssigner[]",
+    "pact" : "Operator",
+    "contents" : "[]:WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Calc[]",
+    "pact" : "Operator",
+    "contents" : "[]:Calc(select=[b, a, rowtime])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "WindowAggregate[]",
+    "pact" : "Operator",
+    "contents" : "[]:WindowAggregate(groupBy=[b], window=[TUMBLE(time_col=[rowtime], size=[5 s])], select=[b, COUNT(*) AS EXPR$3, SUM(a) AS EXPR$4, start('w$) AS window_start, end('w$) AS window_end])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "HASH",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Calc[]",
+    "pact" : "Operator",
+    "contents" : "[]:Calc(select=[b, window_start, window_end, EXPR$3, EXPR$4])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testWindowDeduplicate[isNameSimplifyEnabled=false]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalProject(window_start=[$6], window_end=[$7], a=[$0], b=[$1], c=[$2])
++- LogicalFilter(condition=[<=($9, 1)])
+   +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], rowtime=[$4], proctime=[$5], window_start=[$6], window_end=[$7], window_time=[$8], rownum=[ROW_NUMBER() OVER (PARTITION BY $0, $6, $7 ORDER BY $4 DESC NULLS LAST)])
+      +- LogicalTableFunctionScan(invocation=[TUMBLE($5, DESCRIPTOR($4), 900000:INTERVAL MINUTE)], rowType=[RecordType(BIGINT a, INTEGER b, VARCHAR(2147483647) c, BIGINT d, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)])
+         +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], rowtime=[$4], proctime=[$5])
+            +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($4, 1000:INTERVAL SECOND)])
+               +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], rowtime=[$4], proctime=[PROCTIME()])
+                  +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
+
+== Optimized Physical Plan ==
+Calc(select=[window_start, window_end, a, b, c])
++- WindowDeduplicate(window=[TUMBLE(win_start=[window_start], win_end=[window_end], size=[15 min])], keep=[LastRow], partitionKeys=[a], orderKey=[rowtime], order=[ROWTIME])
+   +- Exchange(distribution=[hash[a]])
+      +- Calc(select=[a, b, c, rowtime, window_start, window_end])
+         +- WindowTableFunction(window=[TUMBLE(time_col=[rowtime], size=[15 min])], emitPerRecord=[true])
+            +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)])
+               +- Calc(select=[a, b, c, d, rowtime, PROCTIME() AS proctime])
+                  +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, rowtime])
+
+== Optimized Execution Plan ==
+Calc(select=[window_start, window_end, a, b, c])
++- WindowDeduplicate(window=[TUMBLE(win_start=[window_start], win_end=[window_end], size=[15 min])], keep=[LastRow], partitionKeys=[a], orderKey=[rowtime], order=[ROWTIME])
+   +- Exchange(distribution=[hash[a]])
+      +- Calc(select=[a, b, c, rowtime, window_start, window_end])
+         +- WindowTableFunction(window=[TUMBLE(time_col=[rowtime], size=[15 min])], emitPerRecord=[true])
+            +- WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])
+               +- Calc(select=[a, b, c, d, rowtime, PROCTIME() AS proctime])
+                  +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, rowtime])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, rowtime])",
+    "pact" : "Data Source",
+    "contents" : "TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, rowtime])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "Calc(select=[a, b, c, d, rowtime, PROCTIME() AS proctime])",
+    "pact" : "Operator",
+    "contents" : "Calc(select=[a, b, c, d, rowtime, PROCTIME() AS proctime])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])",
+    "pact" : "Operator",
+    "contents" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "WindowTableFunction(window=[TUMBLE(time_col=[rowtime], size=[15 min])], emitPerRecord=[true])",
+    "pact" : "Operator",
+    "contents" : "WindowTableFunction(window=[TUMBLE(time_col=[rowtime], size=[15 min])], emitPerRecord=[true])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Calc(select=[a, b, c, rowtime, window_start, window_end])",
+    "pact" : "Operator",
+    "contents" : "Calc(select=[a, b, c, rowtime, window_start, window_end])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "WindowDeduplicate(window=[TUMBLE(win_start=[window_start], win_end=[window_end], size=[15 min])], keep=[LastRow], partitionKeys=[a], orderKey=[rowtime], order=[ROWTIME])",
+    "pact" : "Operator",
+    "contents" : "WindowDeduplicate(window=[TUMBLE(win_start=[window_start], win_end=[window_end], size=[15 min])], keep=[LastRow], partitionKeys=[a], orderKey=[rowtime], order=[ROWTIME])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "HASH",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Calc(select=[window_start, window_end, a, b, c])",
+    "pact" : "Operator",
+    "contents" : "Calc(select=[window_start, window_end, a, b, c])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testWindowJoin[isNameSimplifyEnabled=false]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalProject(a=[$0], window_start=[$1], window_end=[$2], cnt=[$3], uv=[$4], a0=[$5], cnt0=[$8], uv0=[$9])
++- LogicalJoin(condition=[AND(=($1, $6), =($2, $7), =($0, $5))], joinType=[inner])
+   :- LogicalProject(a=[$0], window_start=[$1], window_end=[$2], cnt=[$4], uv=[$5])
+   :  +- LogicalAggregate(group=[{0, 1, 2, 3}], cnt=[COUNT()], uv=[COUNT(DISTINCT $4)])
+   :     +- LogicalProject(a=[$0], window_start=[$6], window_end=[$7], window_time=[$8], c=[$2])
+   :        +- LogicalTableFunctionScan(invocation=[TUMBLE($5, DESCRIPTOR($4), 900000:INTERVAL MINUTE)], rowType=[RecordType(BIGINT a, INTEGER b, VARCHAR(2147483647) c, BIGINT d, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)])
+   :           +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], rowtime=[$4], proctime=[$5])
+   :              +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($4, 1000:INTERVAL SECOND)])
+   :                 +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], rowtime=[$4], proctime=[PROCTIME()])
+   :                    +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
+   +- LogicalProject(a=[$0], window_start=[$1], window_end=[$2], cnt=[$4], uv=[$5])
+      +- LogicalAggregate(group=[{0, 1, 2, 3}], cnt=[COUNT()], uv=[COUNT(DISTINCT $4)])
+         +- LogicalProject(a=[$0], window_start=[$6], window_end=[$7], window_time=[$8], c=[$2])
+            +- LogicalTableFunctionScan(invocation=[TUMBLE($5, DESCRIPTOR($4), 900000:INTERVAL MINUTE)], rowType=[RecordType(BIGINT a, INTEGER b, VARCHAR(2147483647) c, BIGINT d, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)])
+               +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], rowtime=[$4], proctime=[$5])
+                  +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($4, 1000:INTERVAL SECOND)])
+                     +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], rowtime=[$4], proctime=[PROCTIME()])
+                        +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]])
+
+== Optimized Physical Plan ==
+Calc(select=[a, window_start, window_end, cnt, uv, a0, cnt0, uv0])
++- WindowJoin(leftWindow=[TUMBLE(win_start=[window_start], win_end=[window_end], size=[15 min])], rightWindow=[TUMBLE(win_start=[window_start], win_end=[window_end], size=[15 min])], joinType=[InnerJoin], where=[=(a, a0)], select=[a, window_start, window_end, cnt, uv, a0, window_start0, window_end0, cnt0, uv0])
+   :- Exchange(distribution=[hash[a]])
+   :  +- Calc(select=[a, window_start, window_end, cnt, uv])
+   :     +- GlobalWindowAggregate(groupBy=[a], window=[TUMBLE(slice_end=[$slice_end], size=[15 min])], select=[a, COUNT(count1$0) AS cnt, COUNT(distinct$0 count$1) AS uv, start('w$) AS window_start, end('w$) AS window_end, rowtime('w$) AS window_time])
+   :        +- Exchange(distribution=[hash[a]])
+   :           +- LocalWindowAggregate(groupBy=[a], window=[TUMBLE(time_col=[rowtime], size=[15 min])], select=[a, COUNT(*) AS count1$0, COUNT(distinct$0 c) AS count$1, DISTINCT(c) AS distinct$0, slice_end('w$) AS $slice_end])
+   :              +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)])
+   :                 +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, c, rowtime], metadata=[]]], fields=[a, c, rowtime])
+   +- Exchange(distribution=[hash[a]])
+      +- Calc(select=[a, window_start, window_end, cnt, uv])
+         +- GlobalWindowAggregate(groupBy=[a], window=[TUMBLE(slice_end=[$slice_end], size=[15 min])], select=[a, COUNT(count1$0) AS cnt, COUNT(distinct$0 count$1) AS uv, start('w$) AS window_start, end('w$) AS window_end, rowtime('w$) AS window_time])
+            +- Exchange(distribution=[hash[a]])
+               +- LocalWindowAggregate(groupBy=[a], window=[TUMBLE(time_col=[rowtime], size=[15 min])], select=[a, COUNT(*) AS count1$0, COUNT(distinct$0 c) AS count$1, DISTINCT(c) AS distinct$0, slice_end('w$) AS $slice_end])
+                  +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)])
+                     +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, project=[a, c, rowtime], metadata=[]]], fields=[a, c, rowtime])
+
+== Optimized Execution Plan ==
+Calc(select=[a, window_start, window_end, cnt, uv, a0, cnt0, uv0])
++- WindowJoin(leftWindow=[TUMBLE(win_start=[window_start], win_end=[window_end], size=[15 min])], rightWindow=[TUMBLE(win_start=[window_start], win_end=[window_end], size=[15 min])], joinType=[InnerJoin], where=[(a = a0)], select=[a, window_start, window_end, cnt, uv, a0, window_start0, window_end0, cnt0, uv0])
+   :- Exchange(distribution=[hash[a]])
+   :  +- Calc(select=[a, window_start, window_end, cnt, uv])
+   :     +- GlobalWindowAggregate(groupBy=[a], window=[TUMBLE(slice_end=[$slice_end], size=[15 min])], select=[a, COUNT(count1$0) AS cnt, COUNT(distinct$0 count$1) AS uv, start('w$) AS window_start, end('w$) AS window_end, rowtime('w$) AS window_time])
+   :        +- Exchange(distribution=[hash[a]])
+   :           +- LocalWindowAggregate(groupBy=[a], window=[TUMBLE(time_col=[rowtime], size=[15 min])], select=[a, COUNT(*) AS count1$0, COUNT(distinct$0 c) AS count$1, DISTINCT(c) AS distinct$0, slice_end('w$) AS $slice_end])
+   :              +- WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])
+   :                 +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, c, rowtime], metadata=[]]], fields=[a, c, rowtime])
+   +- Exchange(distribution=[hash[a]])
+      +- Calc(select=[a, window_start, window_end, cnt, uv])
+         +- GlobalWindowAggregate(groupBy=[a], window=[TUMBLE(slice_end=[$slice_end], size=[15 min])], select=[a, COUNT(count1$0) AS cnt, COUNT(distinct$0 count$1) AS uv, start('w$) AS window_start, end('w$) AS window_end, rowtime('w$) AS window_time])
+            +- Exchange(distribution=[hash[a]])
+               +- LocalWindowAggregate(groupBy=[a], window=[TUMBLE(time_col=[rowtime], size=[15 min])], select=[a, COUNT(*) AS count1$0, COUNT(distinct$0 c) AS count$1, DISTINCT(c) AS distinct$0, slice_end('w$) AS $slice_end])
+                  +- WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])
+                     +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, project=[a, c, rowtime], metadata=[]]], fields=[a, c, rowtime])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, c, rowtime], metadata=[]]], fields=[a, c, rowtime])",
+    "pact" : "Data Source",
+    "contents" : "TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, c, rowtime], metadata=[]]], fields=[a, c, rowtime])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])",
+    "pact" : "Operator",
+    "contents" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "LocalWindowAggregate(groupBy=[a], window=[TUMBLE(time_col=[rowtime], size=[15 min])], select=[a, COUNT(*) AS count1$0, COUNT(distinct$0 c) AS count$1, DISTINCT(c) AS distinct$0, slice_end('w$) AS $slice_end])",
+    "pact" : "Operator",
+    "contents" : "LocalWindowAggregate(groupBy=[a], window=[TUMBLE(time_col=[rowtime], size=[15 min])], select=[a, COUNT(*) AS count1$0, COUNT(distinct$0 c) AS count$1, DISTINCT(c) AS distinct$0, slice_end('w$) AS $slice_end])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "GlobalWindowAggregate(groupBy=[a], window=[TUMBLE(slice_end=[$slice_end], size=[15 min])], select=[a, COUNT(count1$0) AS cnt, COUNT(distinct$0 count$1) AS uv, start('w$) AS window_start, end('w$) AS window_end, rowtime('w$) AS window_time])",
+    "pact" : "Operator",
+    "contents" : "GlobalWindowAggregate(groupBy=[a], window=[TUMBLE(slice_end=[$slice_end], size=[15 min])], select=[a, COUNT(count1$0) AS cnt, COUNT(distinct$0 count$1) AS uv, start('w$) AS window_start, end('w$) AS window_end, rowtime('w$) AS window_time])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "HASH",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Calc(select=[a, window_start, window_end, cnt, uv])",
+    "pact" : "Operator",
+    "contents" : "Calc(select=[a, window_start, window_end, cnt, uv])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Source: TableSourceScan(table=[[default_catalog, default_database, MyTable2, project=[a, c, rowtime], metadata=[]]], fields=[a, c, rowtime])",
+    "pact" : "Data Source",
+    "contents" : "TableSourceScan(table=[[default_catalog, default_database, MyTable2, project=[a, c, rowtime], metadata=[]]], fields=[a, c, rowtime])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])",
+    "pact" : "Operator",
+    "contents" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "LocalWindowAggregate(groupBy=[a], window=[TUMBLE(time_col=[rowtime], size=[15 min])], select=[a, COUNT(*) AS count1$0, COUNT(distinct$0 c) AS count$1, DISTINCT(c) AS distinct$0, slice_end('w$) AS $slice_end])",
+    "pact" : "Operator",
+    "contents" : "LocalWindowAggregate(groupBy=[a], window=[TUMBLE(time_col=[rowtime], size=[15 min])], select=[a, COUNT(*) AS count1$0, COUNT(distinct$0 c) AS count$1, DISTINCT(c) AS distinct$0, slice_end('w$) AS $slice_end])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "GlobalWindowAggregate(groupBy=[a], window=[TUMBLE(slice_end=[$slice_end], size=[15 min])], select=[a, COUNT(count1$0) AS cnt, COUNT(distinct$0 count$1) AS uv, start('w$) AS window_start, end('w$) AS window_end, rowtime('w$) AS window_time])",
+    "pact" : "Operator",
+    "contents" : "GlobalWindowAggregate(groupBy=[a], window=[TUMBLE(slice_end=[$slice_end], size=[15 min])], select=[a, COUNT(count1$0) AS cnt, COUNT(distinct$0 count$1) AS uv, start('w$) AS window_start, end('w$) AS window_end, rowtime('w$) AS window_time])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "HASH",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Calc(select=[a, window_start, window_end, cnt, uv])",
+    "pact" : "Operator",
+    "contents" : "Calc(select=[a, window_start, window_end, cnt, uv])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "WindowJoin(leftWindow=[TUMBLE(win_start=[window_start], win_end=[window_end], size=[15 min])], rightWindow=[TUMBLE(win_start=[window_start], win_end=[window_end], size=[15 min])], joinType=[InnerJoin], where=[(a = a0)], select=[a, window_start, window_end, cnt, uv, a0, window_start0, window_end0, cnt0, uv0])",
+    "pact" : "Operator",
+    "contents" : "WindowJoin(leftWindow=[TUMBLE(win_start=[window_start], win_end=[window_end], size=[15 min])], rightWindow=[TUMBLE(win_start=[window_start], win_end=[window_end], size=[15 min])], joinType=[InnerJoin], where=[(a = a0)], select=[a, window_start, window_end, cnt, uv, a0, window_start0, window_end0, cnt0, uv0])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "HASH",
+      "side" : "second"
+    }, {
+      "id" : ,
+      "ship_strategy" : "HASH",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Calc(select=[a, window_start, window_end, cnt, uv, a0, cnt0, uv0])",
+    "pact" : "Operator",
+    "contents" : "Calc(select=[a, window_start, window_end, cnt, uv, a0, cnt0, uv0])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testWindowJoin[isNameSimplifyEnabled=true]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalProject(a=[$0], window_start=[$1], window_end=[$2], cnt=[$3], uv=[$4], a0=[$5], cnt0=[$8], uv0=[$9])
++- LogicalJoin(condition=[AND(=($1, $6), =($2, $7), =($0, $5))], joinType=[inner])
+   :- LogicalProject(a=[$0], window_start=[$1], window_end=[$2], cnt=[$4], uv=[$5])
+   :  +- LogicalAggregate(group=[{0, 1, 2, 3}], cnt=[COUNT()], uv=[COUNT(DISTINCT $4)])
+   :     +- LogicalProject(a=[$0], window_start=[$6], window_end=[$7], window_time=[$8], c=[$2])
+   :        +- LogicalTableFunctionScan(invocation=[TUMBLE($5, DESCRIPTOR($4), 900000:INTERVAL MINUTE)], rowType=[RecordType(BIGINT a, INTEGER b, VARCHAR(2147483647) c, BIGINT d, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)])
+   :           +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], rowtime=[$4], proctime=[$5])
+   :              +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($4, 1000:INTERVAL SECOND)])
+   :                 +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], rowtime=[$4], proctime=[PROCTIME()])
+   :                    +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
+   +- LogicalProject(a=[$0], window_start=[$1], window_end=[$2], cnt=[$4], uv=[$5])
+      +- LogicalAggregate(group=[{0, 1, 2, 3}], cnt=[COUNT()], uv=[COUNT(DISTINCT $4)])
+         +- LogicalProject(a=[$0], window_start=[$6], window_end=[$7], window_time=[$8], c=[$2])
+            +- LogicalTableFunctionScan(invocation=[TUMBLE($5, DESCRIPTOR($4), 900000:INTERVAL MINUTE)], rowType=[RecordType(BIGINT a, INTEGER b, VARCHAR(2147483647) c, BIGINT d, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)])
+               +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], rowtime=[$4], proctime=[$5])
+                  +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($4, 1000:INTERVAL SECOND)])
+                     +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], rowtime=[$4], proctime=[PROCTIME()])
+                        +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]])
+
+== Optimized Physical Plan ==
+Calc(select=[a, window_start, window_end, cnt, uv, a0, cnt0, uv0])
++- WindowJoin(leftWindow=[TUMBLE(win_start=[window_start], win_end=[window_end], size=[15 min])], rightWindow=[TUMBLE(win_start=[window_start], win_end=[window_end], size=[15 min])], joinType=[InnerJoin], where=[=(a, a0)], select=[a, window_start, window_end, cnt, uv, a0, window_start0, window_end0, cnt0, uv0])
+   :- Exchange(distribution=[hash[a]])
+   :  +- Calc(select=[a, window_start, window_end, cnt, uv])
+   :     +- GlobalWindowAggregate(groupBy=[a], window=[TUMBLE(slice_end=[$slice_end], size=[15 min])], select=[a, COUNT(count1$0) AS cnt, COUNT(distinct$0 count$1) AS uv, start('w$) AS window_start, end('w$) AS window_end, rowtime('w$) AS window_time])
+   :        +- Exchange(distribution=[hash[a]])
+   :           +- LocalWindowAggregate(groupBy=[a], window=[TUMBLE(time_col=[rowtime], size=[15 min])], select=[a, COUNT(*) AS count1$0, COUNT(distinct$0 c) AS count$1, DISTINCT(c) AS distinct$0, slice_end('w$) AS $slice_end])
+   :              +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)])
+   :                 +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, c, rowtime], metadata=[]]], fields=[a, c, rowtime])
+   +- Exchange(distribution=[hash[a]])
+      +- Calc(select=[a, window_start, window_end, cnt, uv])
+         +- GlobalWindowAggregate(groupBy=[a], window=[TUMBLE(slice_end=[$slice_end], size=[15 min])], select=[a, COUNT(count1$0) AS cnt, COUNT(distinct$0 count$1) AS uv, start('w$) AS window_start, end('w$) AS window_end, rowtime('w$) AS window_time])
+            +- Exchange(distribution=[hash[a]])
+               +- LocalWindowAggregate(groupBy=[a], window=[TUMBLE(time_col=[rowtime], size=[15 min])], select=[a, COUNT(*) AS count1$0, COUNT(distinct$0 c) AS count$1, DISTINCT(c) AS distinct$0, slice_end('w$) AS $slice_end])
+                  +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)])
+                     +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, project=[a, c, rowtime], metadata=[]]], fields=[a, c, rowtime])
+
+== Optimized Execution Plan ==
+Calc(select=[a, window_start, window_end, cnt, uv, a0, cnt0, uv0])
++- WindowJoin(leftWindow=[TUMBLE(win_start=[window_start], win_end=[window_end], size=[15 min])], rightWindow=[TUMBLE(win_start=[window_start], win_end=[window_end], size=[15 min])], joinType=[InnerJoin], where=[(a = a0)], select=[a, window_start, window_end, cnt, uv, a0, window_start0, window_end0, cnt0, uv0])
+   :- Exchange(distribution=[hash[a]])
+   :  +- Calc(select=[a, window_start, window_end, cnt, uv])
+   :     +- GlobalWindowAggregate(groupBy=[a], window=[TUMBLE(slice_end=[$slice_end], size=[15 min])], select=[a, COUNT(count1$0) AS cnt, COUNT(distinct$0 count$1) AS uv, start('w$) AS window_start, end('w$) AS window_end, rowtime('w$) AS window_time])
+   :        +- Exchange(distribution=[hash[a]])
+   :           +- LocalWindowAggregate(groupBy=[a], window=[TUMBLE(time_col=[rowtime], size=[15 min])], select=[a, COUNT(*) AS count1$0, COUNT(distinct$0 c) AS count$1, DISTINCT(c) AS distinct$0, slice_end('w$) AS $slice_end])
+   :              +- WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])
+   :                 +- TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, c, rowtime], metadata=[]]], fields=[a, c, rowtime])
+   +- Exchange(distribution=[hash[a]])
+      +- Calc(select=[a, window_start, window_end, cnt, uv])
+         +- GlobalWindowAggregate(groupBy=[a], window=[TUMBLE(slice_end=[$slice_end], size=[15 min])], select=[a, COUNT(count1$0) AS cnt, COUNT(distinct$0 count$1) AS uv, start('w$) AS window_start, end('w$) AS window_end, rowtime('w$) AS window_time])
+            +- Exchange(distribution=[hash[a]])
+               +- LocalWindowAggregate(groupBy=[a], window=[TUMBLE(time_col=[rowtime], size=[15 min])], select=[a, COUNT(*) AS count1$0, COUNT(distinct$0 c) AS count$1, DISTINCT(c) AS distinct$0, slice_end('w$) AS $slice_end])
+                  +- WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])
+                     +- TableSourceScan(table=[[default_catalog, default_database, MyTable2, project=[a, c, rowtime], metadata=[]]], fields=[a, c, rowtime])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: MyTable[]",
+    "pact" : "Data Source",
+    "contents" : "[]:TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, c, rowtime], metadata=[]]], fields=[a, c, rowtime])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "WatermarkAssigner[]",
+    "pact" : "Operator",
+    "contents" : "[]:WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "LocalWindowAggregate[]",
+    "pact" : "Operator",
+    "contents" : "[]:LocalWindowAggregate(groupBy=[a], window=[TUMBLE(time_col=[rowtime], size=[15 min])], select=[a, COUNT(*) AS count1$0, COUNT(distinct$0 c) AS count$1, DISTINCT(c) AS distinct$0, slice_end('w$) AS $slice_end])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "GlobalWindowAggregate[]",
+    "pact" : "Operator",
+    "contents" : "[]:GlobalWindowAggregate(groupBy=[a], window=[TUMBLE(slice_end=[$slice_end], size=[15 min])], select=[a, COUNT(count1$0) AS cnt, COUNT(distinct$0 count$1) AS uv, start('w$) AS window_start, end('w$) AS window_end, rowtime('w$) AS window_time])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "HASH",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Calc[]",
+    "pact" : "Operator",
+    "contents" : "[]:Calc(select=[a, window_start, window_end, cnt, uv])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Source: MyTable2[]",
+    "pact" : "Data Source",
+    "contents" : "[]:TableSourceScan(table=[[default_catalog, default_database, MyTable2, project=[a, c, rowtime], metadata=[]]], fields=[a, c, rowtime])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "WatermarkAssigner[]",
+    "pact" : "Operator",
+    "contents" : "[]:WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "LocalWindowAggregate[]",
+    "pact" : "Operator",
+    "contents" : "[]:LocalWindowAggregate(groupBy=[a], window=[TUMBLE(time_col=[rowtime], size=[15 min])], select=[a, COUNT(*) AS count1$0, COUNT(distinct$0 c) AS count$1, DISTINCT(c) AS distinct$0, slice_end('w$) AS $slice_end])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "GlobalWindowAggregate[]",
+    "pact" : "Operator",
+    "contents" : "[]:GlobalWindowAggregate(groupBy=[a], window=[TUMBLE(slice_end=[$slice_end], size=[15 min])], select=[a, COUNT(count1$0) AS cnt, COUNT(distinct$0 count$1) AS uv, start('w$) AS window_start, end('w$) AS window_end, rowtime('w$) AS window_time])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "HASH",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Calc[]",
+    "pact" : "Operator",
+    "contents" : "[]:Calc(select=[a, window_start, window_end, cnt, uv])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "WindowJoin[]",
+    "pact" : "Operator",
+    "contents" : "[]:WindowJoin(leftWindow=[TUMBLE(win_start=[window_start], win_end=[window_end], size=[15 min])], rightWindow=[TUMBLE(win_start=[window_start], win_end=[window_end], size=[15 min])], joinType=[InnerJoin], where=[(a = a0)], select=[a, window_start, window_end, cnt, uv, a0, window_start0, window_end0, cnt0, uv0])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "HASH",
+      "side" : "second"
+    }, {
+      "id" : ,
+      "ship_strategy" : "HASH",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Calc[]",
+    "pact" : "Operator",
+    "contents" : "[]:Calc(select=[a, window_start, window_end, cnt, uv, a0, cnt0, uv0])",
+    "parallelism" : 2,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  } ]
+}]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testWindowRank[isNameSimplifyEnabled=true]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalProject(window_start=[$6], window_end=[$7], a=[$0], b=[$1], c=[$2])
++- LogicalFilter(condition=[<=($9, 3)])
+   +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], rowtime=[$4], proctime=[$5], window_start=[$6], window_end=[$7], window_time=[$8], rownum=[ROW_NUMBER() OVER (PARTITION BY $0, $6, $7 ORDER BY $1 DESC NULLS LAST)])
+      +- LogicalTableFunctionScan(invocation=[TUMBLE($5, DESCRIPTOR($4), 900000:INTERVAL MINUTE)], rowType=[RecordType(BIGINT a, INTEGER b, VARCHAR(2147483647) c, BIGINT d, TIMESTAMP(3) *ROWTIME* rowtime, TIMESTAMP_LTZ(3) *PROCTIME* proctime, TIMESTAMP(3) window_start, TIMESTAMP(3) window_end, TIMESTAMP(3) *ROWTIME* window_time)])
+         +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], rowtime=[$4], proctime=[$5])
+            +- LogicalWatermarkAssigner(rowtime=[rowtime], watermark=[-($4, 1000:INTERVAL SECOND)])
+               +- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], rowtime=[$4], proctime=[PROCTIME()])
+                  +- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
+
+== Optimized Physical Plan ==
+Calc(select=[window_start, window_end, a, b, c])
++- WindowRank(window=[TUMBLE(win_start=[window_start], win_end=[window_end], size=[15 min])], rankType=[ROW_NUMBER], rankRange=[rankStart=1, rankEnd=3], partitionBy=[a], orderBy=[b DESC], select=[a, b, c, window_start, window_end])
+   +- Exchange(distribution=[hash[a]])
+      +- Calc(select=[a, b, c, window_start, window_end])
+         +- WindowTableFunction(window=[TUMBLE(time_col=[rowtime], size=[15 min])], emitPerRecord=[true])
+            +- WatermarkAssigner(rowtime=[rowtime], watermark=[-(rowtime, 1000:INTERVAL SECOND)])
+               +- Calc(select=[a, b, c, d, rowtime, PROCTIME() AS proctime])
+                  +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, rowtime])
+
+== Optimized Execution Plan ==
+Calc(select=[window_start, window_end, a, b, c])
++- WindowRank(window=[TUMBLE(win_start=[window_start], win_end=[window_end], size=[15 min])], rankType=[ROW_NUMBER], rankRange=[rankStart=1, rankEnd=3], partitionBy=[a], orderBy=[b DESC], select=[a, b, c, window_start, window_end])
+   +- Exchange(distribution=[hash[a]])
+      +- Calc(select=[a, b, c, window_start, window_end])
+         +- WindowTableFunction(window=[TUMBLE(time_col=[rowtime], size=[15 min])], emitPerRecord=[true])
+            +- WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])
+               +- Calc(select=[a, b, c, d, rowtime, PROCTIME() AS proctime])
+                  +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, rowtime])
+
+== Physical Execution Plan ==
+{
+  "nodes" : [ {
+    "id" : ,
+    "type" : "Source: MyTable[]",
+    "pact" : "Data Source",
+    "contents" : "[]:TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d, rowtime])",
+    "parallelism" : 1
+  }, {
+    "id" : ,
+    "type" : "Calc[]",
+    "pact" : "Operator",
+    "contents" : "[]:Calc(select=[a, b, c, d, rowtime, PROCTIME() AS proctime])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "WatermarkAssigner[]",
+    "pact" : "Operator",
+    "contents" : "[]:WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "WindowTableFunction[]",
+    "pact" : "Operator",
+    "contents" : "[]:WindowTableFunction(window=[TUMBLE(time_col=[rowtime], size=[15 min])], emitPerRecord=[true])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "Calc[]",
+    "pact" : "Operator",
+    "contents" : "[]:Calc(select=[a, b, c, window_start, window_end])",
+    "parallelism" : 1,
+    "predecessors" : [ {
+      "id" : ,
+      "ship_strategy" : "FORWARD",
+      "side" : "second"
+    } ]
+  }, {
+    "id" : ,
+    "type" : "WindowRank[]",
+    "pact" : "Operator",
+    "contents" : "[]:WindowRank(window=[TUMBLE(win_start=[window_start], win_end=[window_end], size=[15 min])], rankType=[ROW_NUMBER], rankRange=[rankStart=1, rankEnd=3], partitionBy=[a], orderBy=[b DESC], select=[a, b, c, window_start, window_end])",
... 410 lines suppressed ...