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

[flink] branch master updated: [FLINK-12424] [table-planner-blink] Supports query optimization with multiple sinks

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

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


The following commit(s) were added to refs/heads/master by this push:
     new e038a80  [FLINK-12424] [table-planner-blink] Supports query optimization with multiple sinks
e038a80 is described below

commit e038a801a87f25b30a1b47ffe5710e5d9bd44c9b
Author: godfrey he <go...@163.com>
AuthorDate: Sat May 18 19:55:02 2019 +0800

    [FLINK-12424] [table-planner-blink] Supports query optimization with multiple sinks
    
    This closes #8356
---
 .../flink/table/api/PlannerConfigOptions.java      |  11 +
 .../flink/table/api/BatchTableEnvironment.scala    | 117 ++-
 .../flink/table/api/StreamTableEnvironment.scala   | 113 ++-
 .../apache/flink/table/api/TableEnvironment.scala  | 131 ++-
 .../nodes/common/CommonIntermediateTableScan.scala |  54 ++
 .../FlinkLogicalIntermediateTableScan.scala        |  92 ++
 .../nodes/physical/batch/BatchExecExchange.scala   |   4 +-
 .../batch/BatchExecIntermediateTableScan.scala     |  44 +
 .../stream/StreamExecIntermediateTableScan.scala   |  61 ++
 ...ala => BatchCommonSubGraphBasedOptimizer.scala} |  45 +-
 .../optimize/CommonSubGraphBasedOptimizer.scala    | 100 +++
 .../flink/table/plan/optimize/RelNodeBlock.scala   | 416 +++++++++
 .../StreamCommonSubGraphBasedOptimizer.scala       | 288 ++++++
 .../table/plan/optimize/StreamOptimizer.scala      |  81 --
 .../optimize/program/StreamOptimizeContext.scala   |   3 +-
 .../table/plan/rules/FlinkBatchRuleSets.scala      |   8 +-
 .../table/plan/rules/FlinkStreamRuleSets.scala     |   4 +-
 .../batch/BatchExecIntermediateTableScanRule.scala |  48 +
 .../StreamExecIntermediateTableScanRule.scala      |  48 +
 .../table/plan/schema/IntermediateRelTable.scala   |  68 ++
 .../apache/flink/table/plan/schema/RelTable.scala  |   2 +-
 .../flink/table/plan/stats/FlinkStatistic.scala    |  15 +-
 .../org/apache/flink/table/util/PlanUtil.scala     |  81 ++
 .../apache/flink/table/api/batch/ExplainTest.xml   | 648 +++++++++++++-
 .../apache/flink/table/api/stream/ExplainTest.xml  | 542 +++++++++++-
 .../table/plan/batch/sql/DagOptimizationTest.xml   | 863 ++++++++++++++++++
 .../apache/flink/table/plan/batch/sql/SinkTest.xml |  84 ++
 .../table/plan/stream/sql/DagOptimizationTest.xml  | 980 +++++++++++++++++++++
 .../flink/table/plan/stream/sql/SinkTest.xml       | 284 ++++++
 .../apache/flink/table/api/batch/ExplainTest.scala |  79 +-
 .../flink/table/api/stream/ExplainTest.scala       |  74 +-
 .../table/plan/batch/sql/DagOptimizationTest.scala | 464 ++++++++++
 .../flink/table/plan/batch/sql/SinkTest.scala      |  59 ++
 .../plan/stream/sql/DagOptimizationTest.scala      | 524 +++++++++++
 .../flink/table/plan/stream/sql/SinkTest.scala     | 148 ++++
 .../table/runtime/stream/sql/CalcITCase.scala      |   2 +-
 .../table/runtime/stream/sql/CorrelateITCase.scala |  20 +-
 .../table/runtime/stream/sql/RankITCase.scala      |  34 +-
 .../table/runtime/utils/BatchTableEnvUtil.scala    |   2 +-
 .../apache/flink/table/util/TableTestBase.scala    | 189 +++-
 .../flink/table/runtime/BinaryHashPartitioner.java |   8 +-
 41 files changed, 6562 insertions(+), 276 deletions(-)

diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/api/PlannerConfigOptions.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/api/PlannerConfigOptions.java
index 557913f..30dc16f 100644
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/api/PlannerConfigOptions.java
+++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/api/PlannerConfigOptions.java
@@ -108,4 +108,15 @@ public class PlannerConfigOptions {
 					.withDescription("When true, the optimizer will try to find out duplicated table-source and " +
 							"reuse them. This works only when " + SQL_OPTIMIZER_REUSE_SUB_PLAN_ENABLED + " is true.");
 
+	public static final ConfigOption<Boolean> SQL_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED =
+			key("sql.optimizer.reuse.optimize-block.with-digest.enabled")
+					.defaultValue(false)
+					.withDescription("When true, the optimizer will try to find out duplicated sub-plan by digest " +
+							"to build optimize block. Each optimize block will be optimized independently.");
+
+	public static final ConfigOption<Boolean> SQL_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED =
+			key("sql.optimizer.unionall-as-breakpoint.disabled")
+					.defaultValue(false)
+					.withDescription("Disable union all as breakpoint when constructing RelNodeBlock");
+
 }
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/api/BatchTableEnvironment.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/api/BatchTableEnvironment.scala
index 3668ecb..05ba98e 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/api/BatchTableEnvironment.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/api/BatchTableEnvironment.scala
@@ -17,20 +17,23 @@
  */
 package org.apache.flink.table.api
 
+import org.apache.flink.api.common.JobExecutionResult
 import org.apache.flink.configuration.Configuration
+import org.apache.flink.streaming.api.TimeCharacteristic
 import org.apache.flink.streaming.api.datastream.DataStream
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment
+import org.apache.flink.streaming.api.graph.{StreamGraph, StreamGraphGenerator}
 import org.apache.flink.streaming.api.transformations.StreamTransformation
 import org.apache.flink.table.plan.`trait`.FlinkRelDistributionTraitDef
-import org.apache.flink.table.plan.nodes.calcite.LogicalSink
 import org.apache.flink.table.plan.nodes.exec.{BatchExecNode, ExecNode}
-import org.apache.flink.table.plan.optimize.{BatchOptimizer, Optimizer}
+import org.apache.flink.table.plan.optimize.{BatchCommonSubGraphBasedOptimizer, Optimizer}
 import org.apache.flink.table.plan.reuse.DeadlockBreakupProcessor
 import org.apache.flink.table.plan.schema.{BatchTableSourceTable, TableSourceSinkTable, TableSourceTable}
 import org.apache.flink.table.plan.stats.FlinkStatistic
-import org.apache.flink.table.plan.util.FlinkRelOptUtil
+import org.apache.flink.table.plan.util.{ExecNodePlanDumper, FlinkRelOptUtil}
 import org.apache.flink.table.sinks._
 import org.apache.flink.table.sources._
+import org.apache.flink.table.util.PlanUtil
 
 import org.apache.calcite.plan.{ConventionTraitDef, RelTrait, RelTraitDef}
 import org.apache.calcite.rel.{RelCollationTraitDef, RelNode}
@@ -66,7 +69,7 @@ class BatchTableEnvironment(
       RelCollationTraitDef.INSTANCE)
   }
 
-  override protected def getOptimizer: Optimizer = new BatchOptimizer(this)
+  override protected def getOptimizer: Optimizer = new BatchCommonSubGraphBasedOptimizer(this)
 
   /**
     * Checks if the chosen table name is valid.
@@ -83,8 +86,40 @@ class BatchTableEnvironment(
     }
   }
 
-  override private[flink] def translateNodeDag(rels: Seq[RelNode]): Seq[ExecNode[_, _]] = {
-    val nodeDag = super.translateNodeDag(rels)
+  override def execute(jobName: String): JobExecutionResult = {
+    generateStreamGraph(jobName)
+    // TODO supports streamEnv.execute(streamGraph)
+    streamEnv.execute(jobName)
+  }
+
+  protected override def translateStreamGraph(
+      streamingTransformations: Seq[StreamTransformation[_]],
+      jobName: Option[String]): StreamGraph = {
+    mergeParameters()
+    streamEnv.getConfig
+      //.enableObjectReuse() // TODO add object reuse config in table config for batch and stream
+      .setLatencyTrackingInterval(-1L)
+    streamEnv.setStreamTimeCharacteristic(TimeCharacteristic.ProcessingTime)
+    streamEnv.setBufferTimeout(-1L)
+    if (streamEnv.getCheckpointConfig.isCheckpointingEnabled) {
+      throw new TableException("Checkpoint should be disabled on Batch job.")
+    }
+
+    // TODO introduce StreamGraphGenerator#Context to support following features:
+    // disable all CheckpointConfig
+    // setChainingEnabled
+    // setMultiHeadChainMode
+    // setSlotSharingEnabled
+    // setScheduleMode
+    // setChainEagerlyEnabled
+
+    val streamGraph = StreamGraphGenerator.generate(streamEnv, streamingTransformations.toList)
+    streamGraph.setJobName(jobName.getOrElse(DEFAULT_JOB_NAME))
+    streamGraph
+  }
+
+  override private[flink] def translateToExecNodeDag(rels: Seq[RelNode]): Seq[ExecNode[_, _]] = {
+    val nodeDag = super.translateToExecNodeDag(rels)
     // breakup deadlock
     new DeadlockBreakupProcessor().process(nodeDag)
   }
@@ -110,28 +145,8 @@ class BatchTableEnvironment(
     }
   }
 
-  /**
-    * Writes a [[Table]] to a [[TableSink]].
-    *
-    * Internally, the [[Table]] is translated into a [[DataStream]] and handed over to the
-    * [[TableSink]] to write it.
-    *
-    * @param table The [[Table]] to write.
-    * @param sink The [[TableSink]] to write the [[Table]] to.
-    * @tparam T The expected type of the [[DataStream]] which represents the [[Table]].
-    */
-  override private[table] def writeToSink[T](
-      table: Table,
-      sink: TableSink[T],
-      sinkName: String): Unit = {
-    mergeParameters()
-
-    val sinkNode = LogicalSink.create(table.asInstanceOf[TableImpl].getRelNode, sink, sinkName)
-    val optimizedPlan = optimize(sinkNode)
-    val optimizedNodes = translateNodeDag(Seq(optimizedPlan))
-    require(optimizedNodes.size() == 1)
-    translateToPlan(optimizedNodes.head)
-  }
+  override protected def translateToPlan(
+      sinks: Seq[ExecNode[_, _]]): Seq[StreamTransformation[_]] = sinks.map(translateToPlan)
 
   /**
     * Translates a [[BatchExecNode]] plan into a [[StreamTransformation]].
@@ -166,11 +181,10 @@ class BatchTableEnvironment(
     */
   def explain(table: Table, extended: Boolean): String = {
     val ast = table.asInstanceOf[TableImpl].getRelNode
-    val optimizedNode = optimize(ast)
-    // translate plan to physical operators
-    val optimizedNodes = translateNodeDag(Seq(optimizedNode))
-    require(optimizedNodes.size() == 1)
-    translateToPlan(optimizedNodes.head)
+    val execNodeDag = compileToExecNodePlan(ast)
+    val transformations = translateToPlan(execNodeDag)
+    val streamGraph = translateStreamGraph(transformations)
+    val executionPlan = PlanUtil.explainStreamGraph(streamGraph)
 
     val explainLevel = if (extended) {
       SqlExplainLevel.ALL_ATTRIBUTES
@@ -184,9 +198,11 @@ class BatchTableEnvironment(
       System.lineSeparator +
       s"== Optimized Logical Plan ==" +
       System.lineSeparator +
-      s"${FlinkRelOptUtil.toString(optimizedNode, explainLevel)}" +
-      System.lineSeparator
-    // TODO show Physical Execution Plan
+      s"${ExecNodePlanDumper.dagToString(execNodeDag, explainLevel)}" +
+      System.lineSeparator +
+      s"== Physical Execution Plan ==" +
+      System.lineSeparator +
+      s"$executionPlan"
   }
 
   /**
@@ -202,8 +218,33 @@ class BatchTableEnvironment(
     * @param extended Flag to include detailed optimizer estimates.
     */
   def explain(extended: Boolean): String = {
-    // TODO implements this method when supports multi-sinks
-    throw new TableException("Unsupported now")
+    val sinkExecNodes = compileToExecNodePlan(sinkNodes: _*)
+    val sinkTransformations = translateToPlan(sinkExecNodes)
+    val streamGraph = translateStreamGraph(sinkTransformations)
+    val sqlPlan = PlanUtil.explainStreamGraph(streamGraph)
+
+    val sb = new StringBuilder
+    sb.append("== Abstract Syntax Tree ==")
+    sb.append(System.lineSeparator)
+    sinkNodes.foreach { sink =>
+      sb.append(FlinkRelOptUtil.toString(sink))
+      sb.append(System.lineSeparator)
+    }
+
+    sb.append("== Optimized Logical Plan ==")
+    sb.append(System.lineSeparator)
+    val explainLevel = if (extended) {
+      SqlExplainLevel.ALL_ATTRIBUTES
+    } else {
+      SqlExplainLevel.EXPPLAN_ATTRIBUTES
+    }
+    sb.append(ExecNodePlanDumper.dagToString(sinkExecNodes, explainLevel))
+    sb.append(System.lineSeparator)
+
+    sb.append("== Physical Execution Plan ==")
+    sb.append(System.lineSeparator)
+    sb.append(sqlPlan)
+    sb.toString()
   }
 
   /**
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala
index 0f9d68c..87f40af 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/api/StreamTableEnvironment.scala
@@ -18,6 +18,7 @@
 
 package org.apache.flink.table.api
 
+import org.apache.flink.api.common.JobExecutionResult
 import org.apache.flink.api.common.typeinfo.TypeInformation
 import org.apache.flink.api.common.typeutils.CompositeType
 import org.apache.flink.configuration.Configuration
@@ -26,19 +27,21 @@ import org.apache.flink.runtime.state.memory.MemoryStateBackend
 import org.apache.flink.streaming.api.TimeCharacteristic
 import org.apache.flink.streaming.api.datastream.DataStream
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment
+import org.apache.flink.streaming.api.graph.{StreamGraph, StreamGraphGenerator}
 import org.apache.flink.streaming.api.transformations.StreamTransformation
-import org.apache.flink.table.`type`.{InternalType, InternalTypes, RowType, TypeConverters}
+import org.apache.flink.table.`type`.TypeConverters
 import org.apache.flink.table.dataformat.BaseRow
 import org.apache.flink.table.plan.`trait`.{AccModeTraitDef, FlinkRelDistributionTraitDef, MiniBatchIntervalTraitDef, UpdateAsRetractionTraitDef}
 import org.apache.flink.table.plan.nodes.calcite.LogicalSink
 import org.apache.flink.table.plan.nodes.exec.{ExecNode, StreamExecNode}
-import org.apache.flink.table.plan.optimize.{Optimizer, StreamOptimizer}
+import org.apache.flink.table.plan.optimize.{Optimizer, StreamCommonSubGraphBasedOptimizer}
 import org.apache.flink.table.plan.schema._
 import org.apache.flink.table.plan.stats.FlinkStatistic
-import org.apache.flink.table.plan.util.FlinkRelOptUtil
-import org.apache.flink.table.sinks.{DataStreamTableSink, TableSink}
+import org.apache.flink.table.plan.util.{ExecNodePlanDumper, FlinkRelOptUtil}
+import org.apache.flink.table.sinks.DataStreamTableSink
 import org.apache.flink.table.sources.{StreamTableSource, TableSource}
 import org.apache.flink.table.typeutils.{TimeIndicatorTypeInfo, TypeCheckUtils}
+import org.apache.flink.table.util.PlanUtil
 
 import org.apache.calcite.plan.{ConventionTraitDef, RelTrait, RelTraitDef}
 import org.apache.calcite.sql.SqlExplainLevel
@@ -84,7 +87,7 @@ abstract class StreamTableEnvironment(
       AccModeTraitDef.INSTANCE)
   }
 
-  override protected def getOptimizer: Optimizer = new StreamOptimizer(this)
+  override protected def getOptimizer: Optimizer = new StreamCommonSubGraphBasedOptimizer(this)
 
   /**
     * Checks if the chosen table name is valid.
@@ -101,6 +104,22 @@ abstract class StreamTableEnvironment(
     }
   }
 
+  override def execute(jobName: String): JobExecutionResult = {
+    generateStreamGraph(jobName)
+    // TODO supports execEnv.execute(streamGraph)
+    execEnv.execute(jobName)
+  }
+
+  protected override def translateStreamGraph(
+      streamingTransformations: Seq[StreamTransformation[_]],
+      jobName: Option[String] = None): StreamGraph = {
+    mergeParameters()
+
+    val streamGraph = StreamGraphGenerator.generate(execEnv, streamingTransformations.toList)
+    streamGraph.setJobName(jobName.getOrElse(DEFAULT_JOB_NAME))
+    streamGraph
+  }
+
   /**
     * Merge global job parameters and table config parameters,
     * and set the merged result to GlobalJobParameters
@@ -130,24 +149,6 @@ abstract class StreamTableEnvironment(
   }
 
   /**
-    * Writes a [[Table]] to a [[TableSink]].
-    *
-    * Internally, the [[Table]] is translated into a [[DataStream]] and handed over to the
-    * [[TableSink]] to write it.
-    *
-    * @param table The [[Table]] to write.
-    * @param sink The [[TableSink]] to write the [[Table]] to.
-    * @tparam T The expected type of the [[DataStream]] which represents the [[Table]].
-    */
-  override private[table] def writeToSink[T](
-      table: Table,
-      sink: TableSink[T],
-      sinkName: String): Unit = {
-    val sinkNode = LogicalSink.create(table.asInstanceOf[TableImpl].getRelNode, sink, sinkName)
-    translateSink(sinkNode)
-  }
-
-  /**
     * Translates a [[Table]] into a [[DataStream]].
     *
     * The transformation involves optimizing the relational expression tree as defined by
@@ -177,11 +178,14 @@ abstract class StreamTableEnvironment(
     mergeParameters()
 
     val optimizedPlan = optimize(sink)
-    val optimizedNodes = translateNodeDag(Seq(optimizedPlan))
+    val optimizedNodes = translateToExecNodeDag(Seq(optimizedPlan))
     require(optimizedNodes.size() == 1)
     translateToPlan(optimizedNodes.head)
   }
 
+  override protected def translateToPlan(
+      sinks: Seq[ExecNode[_, _]]): Seq[StreamTransformation[_]] = sinks.map(translateToPlan)
+
   /**
     * Translates a [[StreamExecNode]] plan into a [[StreamTransformation]].
     *
@@ -214,16 +218,15 @@ abstract class StreamTableEnvironment(
     */
   def explain(table: Table, extended: Boolean): String = {
     val ast = table.asInstanceOf[TableImpl].getRelNode
-    val optimizedNode = optimize(ast)
-    // translate plan to physical operators
-    val optimizedNodes = translateNodeDag(Seq(optimizedNode))
-    require(optimizedNodes.size() == 1)
-    translateToPlan(optimizedNodes.head)
+    val execNodeDag = compileToExecNodePlan(ast)
+    val transformations = translateToPlan(execNodeDag)
+    val streamGraph = translateStreamGraph(transformations)
+    val executionPlan = PlanUtil.explainStreamGraph(streamGraph)
 
-    val explainLevel = if (extended) {
-      SqlExplainLevel.ALL_ATTRIBUTES
+    val (explainLevel, withRetractTraits) = if (extended) {
+      (SqlExplainLevel.ALL_ATTRIBUTES, true)
     } else {
-      SqlExplainLevel.EXPPLAN_ATTRIBUTES
+      (SqlExplainLevel.EXPPLAN_ATTRIBUTES, false)
     }
 
     s"== Abstract Syntax Tree ==" +
@@ -232,9 +235,16 @@ abstract class StreamTableEnvironment(
       System.lineSeparator +
       s"== Optimized Logical Plan ==" +
       System.lineSeparator +
-      s"${FlinkRelOptUtil.toString(optimizedNode, explainLevel)}" +
-      System.lineSeparator
-    // TODO show Physical Execution Plan
+      s"${
+        ExecNodePlanDumper.dagToString(
+          execNodeDag,
+          explainLevel,
+          withRetractTraits = withRetractTraits)
+      }" +
+      System.lineSeparator +
+      s"== Physical Execution Plan ==" +
+      System.lineSeparator +
+      s"$executionPlan"
   }
 
   /**
@@ -250,8 +260,37 @@ abstract class StreamTableEnvironment(
     * @param extended Flag to include detailed optimizer estimates.
     */
   def explain(extended: Boolean): String = {
-    // TODO implements this method when supports multi-sinks
-    throw new TableException("Unsupported now")
+    val sinkExecNodes = compileToExecNodePlan(sinkNodes: _*)
+    // translate relNodes to StreamTransformations
+    val sinkTransformations = translateToPlan(sinkExecNodes)
+    val streamGraph = translateStreamGraph(sinkTransformations)
+    val sqlPlan = PlanUtil.explainStreamGraph(streamGraph)
+
+    val sb = new StringBuilder
+    sb.append("== Abstract Syntax Tree ==")
+    sb.append(System.lineSeparator)
+    sinkNodes.foreach { sink =>
+      sb.append(FlinkRelOptUtil.toString(sink))
+      sb.append(System.lineSeparator)
+    }
+
+    sb.append("== Optimized Logical Plan ==")
+    sb.append(System.lineSeparator)
+    val (explainLevel, withRetractTraits) = if (extended) {
+      (SqlExplainLevel.ALL_ATTRIBUTES, true)
+    } else {
+      (SqlExplainLevel.EXPPLAN_ATTRIBUTES, false)
+    }
+    sb.append(ExecNodePlanDumper.dagToString(
+      sinkExecNodes,
+      explainLevel,
+      withRetractTraits = withRetractTraits))
+    sb.append(System.lineSeparator)
+
+    sb.append("== Physical Execution Plan ==")
+    sb.append(System.lineSeparator)
+    sb.append(sqlPlan)
+    sb.toString()
   }
 
   /**
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala
index 317ea06..b135d1c 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/api/TableEnvironment.scala
@@ -19,12 +19,15 @@
 package org.apache.flink.table.api
 
 import org.apache.flink.annotation.VisibleForTesting
+import org.apache.flink.api.common.JobExecutionResult
 import org.apache.flink.api.common.typeinfo.TypeInformation
 import org.apache.flink.api.common.typeutils.CompositeType
 import org.apache.flink.api.java.typeutils._
 import org.apache.flink.api.scala.typeutils.CaseClassTypeInfo
 import org.apache.flink.streaming.api.environment.{StreamExecutionEnvironment => JavaStreamExecEnv}
+import org.apache.flink.streaming.api.graph.StreamGraph
 import org.apache.flink.streaming.api.scala.{StreamExecutionEnvironment => ScalaStreamExecEnv}
+import org.apache.flink.streaming.api.transformations.StreamTransformation
 import org.apache.flink.table.api.java.{BatchTableEnvironment => JavaBatchTableEnvironment, StreamTableEnvironment => JavaStreamTableEnv}
 import org.apache.flink.table.api.scala.{BatchTableEnvironment => ScalaBatchTableEnvironment, StreamTableEnvironment => ScalaStreamTableEnv}
 import org.apache.flink.table.calcite.{FlinkContextImpl, FlinkPlannerImpl, FlinkRelBuilder, FlinkTypeFactory, FlinkTypeSystem}
@@ -35,6 +38,7 @@ import org.apache.flink.table.functions.utils.UserDefinedFunctionUtils
 import org.apache.flink.table.functions.utils.UserDefinedFunctionUtils.{checkForInstantiation, checkNotSingleton, getAccumulatorTypeOfAggregateFunction, getResultTypeOfAggregateFunction}
 import org.apache.flink.table.functions.{AggregateFunction, ScalarFunction, TableFunction}
 import org.apache.flink.table.plan.cost.FlinkCostFactory
+import org.apache.flink.table.plan.nodes.calcite.{LogicalSink, Sink}
 import org.apache.flink.table.plan.nodes.exec.ExecNode
 import org.apache.flink.table.plan.nodes.physical.FlinkPhysicalRel
 import org.apache.flink.table.plan.optimize.Optimizer
@@ -67,6 +71,7 @@ import _root_.java.util.{Arrays => JArrays}
 import _root_.scala.annotation.varargs
 import _root_.scala.collection.JavaConversions._
 import _root_.scala.collection.JavaConverters._
+import _root_.scala.collection.mutable
 
 /**
   * The abstract base class for batch and stream TableEnvironments.
@@ -75,6 +80,8 @@ import _root_.scala.collection.JavaConverters._
   */
 abstract class TableEnvironment(val config: TableConfig) {
 
+  protected val DEFAULT_JOB_NAME = "Flink Exec Table Job"
+
   // the catalog to hold all registered and translated tables
   // we disable caching here to prevent side effects
   private val internalSchema: CalciteSchema = CalciteSchema.createRootSchema(false, false)
@@ -114,6 +121,12 @@ abstract class TableEnvironment(val config: TableConfig) {
 
   private[flink] val tableNamePrefix = "_TempTable_"
 
+  // sink nodes collection
+  // TODO use SinkNode(LogicalNode) instead of Sink(RelNode) after we introduce [Expression]
+  private[flink] var sinkNodes = new mutable.ArrayBuffer[Sink]
+
+  private[flink] val transformations = new mutable.ArrayBuffer[StreamTransformation[_]]
+
   /** Returns the table config to define the runtime behavior of the Table API. */
   def getConfig: TableConfig = config
 
@@ -174,6 +187,103 @@ abstract class TableEnvironment(val config: TableConfig) {
   protected def getOptimizer: Optimizer
 
   /**
+    * Triggers the program execution.
+    */
+  def execute(): JobExecutionResult = execute(DEFAULT_JOB_NAME)
+
+  /**
+    * Triggers the program execution with jobName.
+    */
+  def execute(jobName: String): JobExecutionResult
+
+  /**
+    * Generate a [[StreamGraph]] from this table environment, this will also clear sinkNodes.
+    * @return A [[StreamGraph]] describing the whole job.
+    */
+  def generateStreamGraph(): StreamGraph = generateStreamGraph(DEFAULT_JOB_NAME)
+
+  /**
+    * Generate a [[StreamGraph]] from this table environment, this will also clear sinkNodes.
+    * @return A [[StreamGraph]] describing the whole job.
+    */
+  def generateStreamGraph(jobName: String): StreamGraph = {
+    try {
+      compile()
+      if (transformations.isEmpty) {
+        throw new TableException("No table sinks have been created yet. " +
+          "A program needs at least one sink that consumes data. ")
+      }
+      translateStreamGraph(transformations, Some(jobName))
+    } finally {
+      sinkNodes.clear()
+      transformations.clear()
+    }
+  }
+
+  /**
+    * Translate a [[StreamGraph]] from Given streamingTransformations.
+    * @return A [[StreamGraph]] describing the given job.
+    */
+  protected def translateStreamGraph(
+      streamingTransformations: Seq[StreamTransformation[_]],
+      jobName: Option[String] = None): StreamGraph = ???
+
+  /**
+    * Compile the sinks to [[org.apache.flink.streaming.api.transformations.StreamTransformation]].
+    */
+  protected def compile(): Unit = {
+    if (sinkNodes.isEmpty) {
+      throw new TableException("Internal error in sql compile, SinkNode required here")
+    }
+
+    // translate to ExecNode
+    val nodeDag = compileToExecNodePlan(sinkNodes: _*)
+    // translate to transformation
+    val sinkTransformations = translateToPlan(nodeDag)
+    transformations.addAll(sinkTransformations)
+  }
+
+  /**
+    * Optimize [[RelNode]] tree (or DAG), and translate optimized result to ExecNode tree (or DAG).
+    */
+  @VisibleForTesting
+  private[flink] def compileToExecNodePlan(relNodes: RelNode*): Seq[ExecNode[_, _]] = {
+    if (relNodes.isEmpty) {
+      throw new TableException("Internal error in sql compile, SinkNode required here")
+    }
+
+    // optimize dag
+    val optRelNodes = optimize(relNodes)
+    // translate node dag
+    translateToExecNodeDag(optRelNodes)
+  }
+
+  /**
+    * Translate [[org.apache.flink.table.plan.nodes.physical.FlinkPhysicalRel]] DAG
+    * to [[ExecNode]] DAG.
+    */
+  @VisibleForTesting
+  private[flink] def translateToExecNodeDag(rels: Seq[RelNode]): Seq[ExecNode[_, _]] = {
+    require(rels.nonEmpty && rels.forall(_.isInstanceOf[FlinkPhysicalRel]))
+    // Rewrite same rel object to different rel objects
+    // in order to get the correct dag (dag reuse is based on object not digest)
+    val shuttle = new SameRelObjectShuttle()
+    val relsWithoutSameObj = rels.map(_.accept(shuttle))
+    // reuse subplan
+    val reusedPlan = SubplanReuser.reuseDuplicatedSubplan(relsWithoutSameObj, config)
+    // convert FlinkPhysicalRel DAG to ExecNode DAG
+    reusedPlan.map(_.asInstanceOf[ExecNode[_, _]])
+  }
+
+  /**
+    * Translates a [[ExecNode]] DAG into a [[StreamTransformation]] DAG.
+    *
+    * @param sinks The node DAG to translate.
+    * @return The [[StreamTransformation]] DAG that corresponds to the node DAG.
+    */
+  protected def translateToPlan(sinks: Seq[ExecNode[_, _]]): Seq[StreamTransformation[_]]
+
+  /**
     * Writes a [[Table]] to a [[TableSink]].
     *
     * @param table The [[Table]] to write.
@@ -183,7 +293,9 @@ abstract class TableEnvironment(val config: TableConfig) {
   private[table] def writeToSink[T](
       table: Table,
       sink: TableSink[T],
-      sinkName: String = null): Unit
+      sinkName: String = null): Unit = {
+    sinkNodes += LogicalSink.create(table.asInstanceOf[TableImpl].getRelNode, sink, sinkName)
+  }
 
   /**
     * Generates the optimized [[RelNode]] dag from the original relational nodes.
@@ -206,23 +318,6 @@ abstract class TableEnvironment(val config: TableConfig) {
   private[flink] def optimize(root: RelNode): RelNode = optimize(Seq(root)).head
 
   /**
-    * Convert [[org.apache.flink.table.plan.nodes.physical.FlinkPhysicalRel]] DAG
-    * to [[ExecNode]] DAG and translate them.
-    */
-  @VisibleForTesting
-  private[flink] def translateNodeDag(rels: Seq[RelNode]): Seq[ExecNode[_, _]] = {
-    require(rels.nonEmpty && rels.forall(_.isInstanceOf[FlinkPhysicalRel]))
-    // Rewrite same rel object to different rel objects
-    // in order to get the correct dag (dag reuse is based on object not digest)
-    val shuttle = new SameRelObjectShuttle()
-    val relsWithoutSameObj = rels.map(_.accept(shuttle))
-    // reuse subplan
-    val reusedPlan = SubplanReuser.reuseDuplicatedSubplan(relsWithoutSameObj, config)
-    // convert FlinkPhysicalRel DAG to ExecNode DAG
-    reusedPlan.map(_.asInstanceOf[ExecNode[_, _]])
-  }
-
-  /**
     * Registers a [[Table]] under a unique name in the TableEnvironment's catalog.
     * Registered tables can be referenced in SQL queries.
     *
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/common/CommonIntermediateTableScan.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/common/CommonIntermediateTableScan.scala
new file mode 100644
index 0000000..40fa33d
--- /dev/null
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/common/CommonIntermediateTableScan.scala
@@ -0,0 +1,54 @@
+/*
+ * 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.plan.nodes.common
+
+import org.apache.flink.table.plan.nodes.FlinkRelNode
+import org.apache.flink.table.plan.schema.IntermediateRelTable
+
+import org.apache.calcite.plan.{RelOptCluster, RelOptCost, RelOptPlanner, RelOptTable, RelTraitSet}
+import org.apache.calcite.rel.RelWriter
+import org.apache.calcite.rel.core.{Exchange, TableScan}
+import org.apache.calcite.rel.metadata.RelMetadataQuery
+
+import scala.collection.JavaConversions._
+
+/**
+  * Base class that wraps [[IntermediateRelTable]].
+  */
+abstract class CommonIntermediateTableScan(
+    cluster: RelOptCluster,
+    traitSet: RelTraitSet,
+    table: RelOptTable)
+  extends TableScan(cluster, traitSet, table)
+  with FlinkRelNode {
+
+  val intermediateTable: IntermediateRelTable = getTable.unwrap(classOf[IntermediateRelTable])
+
+  override def computeSelfCost(planner: RelOptPlanner, mq: RelMetadataQuery): RelOptCost = {
+    val rowCnt = mq.getRowCount(this)
+    val rowSize = mq.getAverageRowSize(this)
+    planner.getCostFactory.makeCost(rowCnt, rowCnt, rowCnt * rowSize)
+  }
+
+  override def explainTerms(pw: RelWriter): RelWriter = {
+    super.explainTerms(pw)
+      .item("fields", getRowType.getFieldNames.mkString(", "))
+  }
+}
+
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalIntermediateTableScan.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalIntermediateTableScan.scala
new file mode 100644
index 0000000..3c372a6
--- /dev/null
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalIntermediateTableScan.scala
@@ -0,0 +1,92 @@
+/*
+ * 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.plan.nodes.logical
+
+import org.apache.flink.table.plan.nodes.FlinkConventions
+import org.apache.flink.table.plan.nodes.common.CommonIntermediateTableScan
+import org.apache.flink.table.plan.schema.IntermediateRelTable
+
+import com.google.common.collect.ImmutableList
+import org.apache.calcite.plan._
+import org.apache.calcite.rel.convert.ConverterRule
+import org.apache.calcite.rel.core.TableScan
+import org.apache.calcite.rel.logical.LogicalTableScan
+import org.apache.calcite.rel.{RelCollation, RelCollationTraitDef, RelNode}
+
+import java.util
+import java.util.function.Supplier
+
+/**
+  * A flink TableScan that wraps [[IntermediateRelTable]].
+  */
+class FlinkLogicalIntermediateTableScan(
+    cluster: RelOptCluster,
+    traitSet: RelTraitSet,
+    table: RelOptTable)
+  extends CommonIntermediateTableScan(cluster, traitSet, table)
+  with FlinkLogicalRel {
+
+  override def copy(traitSet: RelTraitSet, inputs: util.List[RelNode]): RelNode = {
+    new FlinkLogicalIntermediateTableScan(cluster, traitSet, getTable)
+  }
+
+}
+
+class FlinkLogicalIntermediateTableScanConverter
+  extends ConverterRule(
+    classOf[LogicalTableScan],
+    Convention.NONE,
+    FlinkConventions.LOGICAL,
+    "FlinkLogicalIntermediateTableScanConverter") {
+
+  override def matches(call: RelOptRuleCall): Boolean = {
+    val scan: TableScan = call.rel(0)
+    val intermediateTable = scan.getTable.unwrap(classOf[IntermediateRelTable])
+    intermediateTable != null
+  }
+
+  def convert(rel: RelNode): RelNode = {
+    val scan = rel.asInstanceOf[TableScan]
+    FlinkLogicalIntermediateTableScan.create(rel.getCluster, scan.getTable)
+  }
+}
+
+object FlinkLogicalIntermediateTableScan {
+
+  val CONVERTER = new FlinkLogicalIntermediateTableScanConverter
+
+  def create(
+      cluster: RelOptCluster,
+      relOptTable: RelOptTable): FlinkLogicalIntermediateTableScan = {
+    val table: IntermediateRelTable = relOptTable.unwrap(classOf[IntermediateRelTable])
+    require(table != null)
+    val traitSet = cluster.traitSetOf(FlinkConventions.LOGICAL)
+      .replaceIfs(RelCollationTraitDef.INSTANCE, new Supplier[util.List[RelCollation]]() {
+        def get: util.List[RelCollation] = {
+          if (table != null) {
+            table.getStatistic.getCollations
+          } else {
+            ImmutableList.of[RelCollation]
+          }
+        }
+      }).simplify()
+
+    new FlinkLogicalIntermediateTableScan(cluster, traitSet, relOptTable)
+  }
+}
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecExchange.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecExchange.scala
index da2de93..8d98529 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecExchange.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecExchange.scala
@@ -205,7 +205,9 @@ class BatchExecExchange(
             CodeGeneratorContext(tableEnv.config),
             new RowType(inputType.getInternalTypes: _*),
             "HashPartitioner",
-            keys.map(_.intValue()).toArray))
+            keys.map(_.intValue()).toArray),
+          keys.map(getInput.getRowType.getFieldNames.get(_)).toArray
+        )
         val transformation = new PartitionTransformation(
           input,
           partitioner)
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecIntermediateTableScan.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecIntermediateTableScan.scala
new file mode 100644
index 0000000..34119ec
--- /dev/null
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecIntermediateTableScan.scala
@@ -0,0 +1,44 @@
+/*
+ * 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.plan.nodes.physical.batch
+
+import org.apache.flink.table.plan.nodes.common.CommonIntermediateTableScan
+
+import org.apache.calcite.plan._
+import org.apache.calcite.rel.RelNode
+import org.apache.calcite.rel.`type`.RelDataType
+
+/**
+  * Batch physical TableScan that wraps [[org.apache.flink.table.plan.schema.IntermediateRelTable]]
+  */
+class BatchExecIntermediateTableScan(
+    cluster: RelOptCluster,
+    traitSet: RelTraitSet,
+    table: RelOptTable,
+    outputRowType: RelDataType)
+  extends CommonIntermediateTableScan(cluster, traitSet, table)
+  with BatchPhysicalRel {
+
+  override def deriveRowType(): RelDataType = outputRowType
+
+  override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = {
+    new BatchExecIntermediateTableScan(cluster, traitSet, getTable, getRowType)
+  }
+
+}
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecIntermediateTableScan.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecIntermediateTableScan.scala
new file mode 100644
index 0000000..3e74f5f
--- /dev/null
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecIntermediateTableScan.scala
@@ -0,0 +1,61 @@
+/*
+ * 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.plan.nodes.physical.stream
+
+import org.apache.flink.table.plan.nodes.common.CommonIntermediateTableScan
+import org.apache.flink.table.plan.util.UpdatingPlanChecker
+
+import org.apache.calcite.plan.{RelOptCluster, RelOptTable, RelTraitSet}
+import org.apache.calcite.rel.RelNode
+import org.apache.calcite.rel.`type`.RelDataType
+
+import java.util
+
+/**
+  * Stream physical TableScan that wraps [[org.apache.flink.table.plan.schema.IntermediateRelTable]]
+  */
+class StreamExecIntermediateTableScan(
+    cluster: RelOptCluster,
+    traitSet: RelTraitSet,
+    table: RelOptTable,
+    outputRowType: RelDataType)
+  extends CommonIntermediateTableScan(cluster, traitSet, table)
+  with StreamPhysicalRel {
+
+  def isAccRetract: Boolean = intermediateTable.isAccRetract
+
+  override def producesUpdates: Boolean = {
+    !UpdatingPlanChecker.isAppendOnly(intermediateTable.relNode)
+  }
+
+  override def needsUpdatesAsRetraction(input: RelNode): Boolean = false
+
+  override def consumesRetractions: Boolean = false
+
+  override def producesRetractions: Boolean = producesUpdates && isAccRetract
+
+  override def requireWatermark: Boolean = false
+
+  override def deriveRowType(): RelDataType = outputRowType
+
+  override def copy(traitSet: RelTraitSet, inputs: util.List[RelNode]): RelNode = {
+    new StreamExecIntermediateTableScan(cluster, traitSet, getTable, outputRowType)
+  }
+
+}
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/optimize/BatchOptimizer.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/optimize/BatchCommonSubGraphBasedOptimizer.scala
similarity index 56%
rename from flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/optimize/BatchOptimizer.scala
rename to flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/optimize/BatchCommonSubGraphBasedOptimizer.scala
index 1be2340..f2d257b 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/optimize/BatchOptimizer.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/optimize/BatchCommonSubGraphBasedOptimizer.scala
@@ -18,21 +18,54 @@
 
 package org.apache.flink.table.plan.optimize
 
-import org.apache.flink.table.api.{BatchTableEnvironment, TableConfig}
+import org.apache.flink.table.api.{BatchTableEnvironment, TableConfig, TableImpl}
+import org.apache.flink.table.plan.nodes.physical.batch.BatchExecSink
 import org.apache.flink.table.plan.optimize.program.{BatchOptimizeContext, FlinkBatchProgram}
+import org.apache.flink.table.plan.schema.IntermediateRelTable
 import org.apache.flink.util.Preconditions
 
 import org.apache.calcite.plan.volcano.VolcanoPlanner
 import org.apache.calcite.rel.RelNode
 
 /**
-  * Query optimizer for Batch.
+  * A [[CommonSubGraphBasedOptimizer]] for Batch.
   */
-class BatchOptimizer(tEnv: BatchTableEnvironment) extends Optimizer {
+class BatchCommonSubGraphBasedOptimizer(tEnv: BatchTableEnvironment)
+  extends CommonSubGraphBasedOptimizer {
 
-  override def optimize(roots: Seq[RelNode]): Seq[RelNode] = {
-    // TODO optimize multi-roots as a whole DAG
-    roots.map(optimizeTree)
+  override protected def doOptimize(roots: Seq[RelNode]): Seq[RelNodeBlock] = {
+    // build RelNodeBlock plan
+    val rootBlocks = RelNodeBlockPlanBuilder.buildRelNodeBlockPlan(roots, tEnv)
+    // optimize recursively RelNodeBlock
+    rootBlocks.foreach(optimizeBlock)
+    rootBlocks
+  }
+
+  private def optimizeBlock(block: RelNodeBlock): Unit = {
+    block.children.foreach { child =>
+      if (child.getNewOutputNode.isEmpty) {
+        optimizeBlock(child)
+      }
+    }
+
+    val originTree = block.getPlan
+    val optimizedTree = optimizeTree(originTree)
+
+    optimizedTree match {
+      case _: BatchExecSink[_] => // ignore
+      case _ =>
+        val name = tEnv.createUniqueTableName()
+        registerIntermediateTable(name, optimizedTree)
+        val newTable = tEnv.scan(name)
+        block.setNewOutputNode(newTable.asInstanceOf[TableImpl].getRelNode)
+        block.setOutputTableName(name)
+    }
+    block.setOptimizedPlan(optimizedTree)
+  }
+
+  private def registerIntermediateTable(name: String, relNode: RelNode): Unit = {
+    val table = new IntermediateRelTable(relNode)
+    tEnv.registerTableInternal(name, table)
   }
 
   /**
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/optimize/CommonSubGraphBasedOptimizer.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/optimize/CommonSubGraphBasedOptimizer.scala
new file mode 100644
index 0000000..29c7e03
--- /dev/null
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/optimize/CommonSubGraphBasedOptimizer.scala
@@ -0,0 +1,100 @@
+/*
+ * 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.plan.optimize
+
+import org.apache.flink.table.plan.schema.IntermediateRelTable
+
+import org.apache.calcite.rel.core.TableScan
+import org.apache.calcite.rel.{RelNode, RelShuttleImpl}
+
+import java.util
+
+import scala.collection.JavaConversions._
+
+/**
+  * A [[Optimizer]] that optimizes [[RelNode]] DAG into semantically [[RelNode]] DAG
+  * based common sub-graph. Common sub-graph represents the common sub RelNode plan
+  * in multiple RelNode trees. Calcite planner does not support DAG (multiple roots) optimization,
+  * so a [[RelNode]] DAG should be decomposed into multiple common sub-graphs,
+  * and each sub-graph is a tree (which has only one root), and can be optimized independently
+  * by Calcite [[org.apache.calcite.plan.RelOptPlanner]].
+  * The algorithm works as follows:
+  * 1. Decompose [[RelNode]] DAG into multiple [[RelNodeBlock]]s, and build [[RelNodeBlock]] DAG.
+  * Each [[RelNodeBlock]] has only one sink output, and represents a common sub-graph.
+  * 2. optimize recursively each [[RelNodeBlock]] from leaf block to root(sink) block,
+  * and register the optimized result of non-root block as an [[IntermediateRelTable]].
+  * 3. expand [[IntermediateRelTable]] into RelNode tree in each [[RelNodeBlock]].
+  */
+abstract class CommonSubGraphBasedOptimizer extends Optimizer {
+
+  /**
+    * Generates the optimized [[RelNode]] DAG from the original relational nodes.
+    * NOTES: the reused node in result DAG will be converted to the same RelNode,
+    * and the result doesn't contain [[IntermediateRelTable]].
+    *
+    * @param roots the original relational nodes.
+    * @return a list of RelNode represents an optimized RelNode DAG.
+    */
+  override def optimize(roots: Seq[RelNode]): Seq[RelNode] = {
+    val sinkBlocks = doOptimize(roots)
+    val optimizedPlan = sinkBlocks.map { block =>
+      val plan = block.getOptimizedPlan
+      require(plan != null)
+      plan
+    }
+    expandIntermediateTableScan(optimizedPlan)
+  }
+
+  /**
+    * Decompose RelNode trees into multiple [[RelNodeBlock]]s, optimize recursively each
+    * [[RelNodeBlock]], return optimized [[RelNodeBlock]]s.
+    *
+    * @return optimized [[RelNodeBlock]]s.
+    */
+  protected def doOptimize(roots: Seq[RelNode]): Seq[RelNodeBlock]
+
+  /**
+    * Expand [[IntermediateRelTable]] in each RelNodeBlock.
+    */
+  private def expandIntermediateTableScan(nodes: Seq[RelNode]): Seq[RelNode] = {
+
+    class ExpandShuttle extends RelShuttleImpl {
+
+      // ensure the same intermediateTable would be expanded to the same RelNode tree.
+      private val expandedIntermediateTables =
+        new util.IdentityHashMap[IntermediateRelTable, RelNode]()
+
+      override def visit(scan: TableScan): RelNode = {
+        val intermediateTable = scan.getTable.unwrap(classOf[IntermediateRelTable])
+        if (intermediateTable != null) {
+          expandedIntermediateTables.getOrElseUpdate(intermediateTable, {
+            val underlyingRelNode = intermediateTable.relNode
+            underlyingRelNode.accept(this)
+          })
+        } else {
+          scan
+        }
+      }
+    }
+
+    val shuttle = new ExpandShuttle
+    nodes.map(_.accept(shuttle))
+  }
+
+}
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/optimize/RelNodeBlock.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/optimize/RelNodeBlock.scala
new file mode 100644
index 0000000..999c892
--- /dev/null
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/optimize/RelNodeBlock.scala
@@ -0,0 +1,416 @@
+/*
+ * 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.plan.optimize
+
+import org.apache.flink.table.api.{PlannerConfigOptions, TableConfig, TableEnvironment}
+import org.apache.flink.table.plan.nodes.calcite.Sink
+import org.apache.flink.table.plan.reuse.SubplanReuser.{SubplanReuseContext, SubplanReuseShuttle}
+import org.apache.flink.table.plan.rules.logical.WindowPropertiesRules
+import org.apache.flink.table.plan.util.{DefaultRelShuttle, ExpandTableScanShuttle}
+import org.apache.flink.util.Preconditions
+
+import com.google.common.collect.Sets
+import org.apache.calcite.rel._
+import org.apache.calcite.rel.core.{Aggregate, Project, Snapshot, TableFunctionScan, Union}
+import org.apache.calcite.rex.RexNode
+
+import java.util
+
+import scala.collection.JavaConversions._
+import scala.collection.mutable
+
+/**
+  * A [[RelNodeBlock]] is a sub-tree in the [[RelNode]] DAG, and represents common sub-graph
+  * in [[CommonSubGraphBasedOptimizer]]. All [[RelNode]]s in each block have
+  * only one [[Sink]] output.
+  *
+  * The algorithm works as follows:
+  * 1. If there is only one tree, the whole tree is in one block. (the next steps is needless.)
+  * 2. reuse common sub-plan in different RelNode tree, generate a RelNode DAG,
+  * 3. traverse each tree from root to leaf, and mark the sink RelNode of each RelNode
+  * 4. traverse each tree from root to leaf again, if meet a RelNode which has multiple sink
+  * RelNode, the RelNode is the output node of a new block (or named break-point).
+  * There are several special cases that a RelNode can not be a break-point.
+  * (1). UnionAll is not a break-point
+  * when [[PlannerConfigOptions.SQL_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED]] is true
+  * (2). [[TableFunctionScan]], [[Snapshot]] or window aggregate ([[Aggregate]] on a [[Project]]
+  * with window attribute) are not a break-point because their physical RelNodes are a composite
+  * RelNode, each of them cannot be optimized individually. e.g. FlinkLogicalTableFunctionScan and
+  * FlinkLogicalCorrelate will be combined into a BatchExecCorrelate or a StreamExecCorrelate.
+  *
+  * For example: (Table API)
+  *
+  * {{{-
+  *  val sourceTable = tEnv.scan("test_table").select('a, 'b, 'c)
+  *  val leftTable = sourceTable.filter('a > 0).select('a as 'a1, 'b as 'b1)
+  *  val rightTable = sourceTable.filter('c.isNotNull).select('b as 'b2, 'c as 'c2)
+  *  val joinTable = leftTable.join(rightTable, 'a1 === 'b2)
+  *  joinTable.where('a1 >= 70).select('a1, 'b1).writeToSink(sink1)
+  *  joinTable.where('a1 < 70 ).select('a1, 'c2).writeToSink(sink2)
+  * }}}
+  *
+  * the RelNode DAG is:
+  *
+  * {{{-
+  * Sink(sink1)     Sink(sink2)
+  *    |               |
+  * Project(a1,b1)  Project(a1,c2)
+  *    |               |
+  * Filter(a1>=70)  Filter(a1<70)
+  *       \          /
+  *        Join(a1=b2)
+  *       /           \
+  * Project(a1,b1)  Project(b2,c2)
+  *      |             |
+  * Filter(a>0)     Filter(c is not null)
+  *      \           /
+  *      Project(a,b,c)
+  *          |
+  *       TableScan
+  * }}}
+  *
+  * This [[RelNode]] DAG will be decomposed into three [[RelNodeBlock]]s, the break-point
+  * is the [[RelNode]](`Join(a1=b2)`) which data outputs to multiple [[Sink]]s.
+  * <p>Notes: Although `Project(a,b,c)` has two parents (outputs),
+  * they eventually merged at `Join(a1=b2)`. So `Project(a,b,c)` is not a break-point.
+  * <p>the first [[RelNodeBlock]] includes TableScan, Project(a,b,c), Filter(a>0),
+  * Filter(c is not null), Project(a1,b1), Project(b2,c2) and Join(a1=b2)
+  * <p>the second one includes Filter(a1>=70), Project(a1,b1) and Sink(sink1)
+  * <p>the third one includes Filter(a1<70), Project(a1,c2) and Sink(sink2)
+  * <p>And the first [[RelNodeBlock]] is the child of another two.
+  *
+  * The [[RelNodeBlock]] plan is:
+  * {{{-
+  * RelNodeBlock2  RelNodeBlock3
+  *        \            /
+  *        RelNodeBlock1
+  * }}}
+  *
+  * The optimizing order is from child block to parent. The optimized result (RelNode)
+  * will be registered into tables first, and then be converted to a new TableScan which is the
+  * new output node of current block and is also the input of its parent blocks.
+  *
+  * @param outputNode A RelNode of the output in the block, which could be a [[Sink]] or
+  * other RelNode which data outputs to multiple [[Sink]]s.
+  */
+class RelNodeBlock(val outputNode: RelNode, tEnv: TableEnvironment) {
+  // child (or input) blocks
+  private val childBlocks = mutable.LinkedHashSet[RelNodeBlock]()
+
+  // After this block has been optimized, the result will be converted to a new TableScan as
+  // new output node
+  private var newOutputNode: Option[RelNode] = None
+
+  private var outputTableName: Option[String] = None
+
+  private var optimizedPlan: Option[RelNode] = None
+
+  private var updateAsRetract: Boolean = false
+
+  def addChild(block: RelNodeBlock): Unit = childBlocks += block
+
+  def children: Seq[RelNodeBlock] = childBlocks.toSeq
+
+  def setNewOutputNode(newNode: RelNode): Unit = newOutputNode = Option(newNode)
+
+  def getNewOutputNode: Option[RelNode] = newOutputNode
+
+  def setOutputTableName(name: String): Unit = outputTableName = Option(name)
+
+  def getOutputTableName: String = outputTableName.orNull
+
+  def setOptimizedPlan(rel: RelNode): Unit = this.optimizedPlan = Option(rel)
+
+  def getOptimizedPlan: RelNode = optimizedPlan.orNull
+
+  def setUpdateAsRetraction(updateAsRetract: Boolean): Unit = {
+    // set child block updateAsRetract, a child may have multi father.
+    if (updateAsRetract) {
+      this.updateAsRetract = true
+    }
+  }
+
+  def isUpdateAsRetraction: Boolean = updateAsRetract
+
+  def getChildBlock(node: RelNode): Option[RelNodeBlock] = {
+    val find = children.filter(_.outputNode.equals(node))
+    if (find.isEmpty) {
+      None
+    } else {
+      Preconditions.checkArgument(find.size == 1)
+      Some(find.head)
+    }
+  }
+
+  /**
+    * Get new plan of this block. The child blocks (inputs) will be replace with new RelNodes (the
+    * optimized result of child block).
+    *
+    * @return New plan of this block
+    */
+  def getPlan: RelNode = {
+    val shuttle = new RelNodeBlockShuttle
+    outputNode.accept(shuttle)
+  }
+
+  private class RelNodeBlockShuttle extends DefaultRelShuttle {
+    override def visit(rel: RelNode): RelNode = {
+      val block = getChildBlock(rel)
+      block match {
+        case Some(b) => b.getNewOutputNode.get
+        case _ => super.visit(rel)
+      }
+    }
+  }
+
+}
+
+/**
+  * Holds information to build [[RelNodeBlock]].
+  */
+class RelNodeWrapper(relNode: RelNode) {
+  // parent nodes of `relNode`
+  private val parentNodes = Sets.newIdentityHashSet[RelNode]()
+  // output nodes of some blocks that data of `relNode` outputs to
+  private val blockOutputNodes = Sets.newIdentityHashSet[RelNode]()
+  // stores visited parent nodes when builds RelNodeBlock
+  private val visitedParentNodes = Sets.newIdentityHashSet[RelNode]()
+
+  def addParentNode(parent: Option[RelNode]): Unit = {
+    parent match {
+      case Some(p) => parentNodes.add(p)
+      case None => // Ignore
+    }
+  }
+
+  def addVisitedParentNode(parent: Option[RelNode]): Unit = {
+    parent match {
+      case Some(p) =>
+        require(parentNodes.contains(p))
+        visitedParentNodes.add(p)
+      case None => // Ignore
+    }
+  }
+
+  def addBlockOutputNode(blockOutputNode: RelNode): Unit = blockOutputNodes.add(blockOutputNode)
+
+  /**
+    * Returns true if all parent nodes had been visited, else false
+    */
+  def allParentNodesVisited: Boolean = parentNodes.size() == visitedParentNodes.size()
+
+  /**
+    * Returns true if number of `blockOutputNodes` is greater than 1, else false
+    */
+  def hasMultipleBlockOutputNodes: Boolean = blockOutputNodes.size() > 1
+
+  /**
+    * Returns the output node of the block that the `relNode` belongs to
+    */
+  def getBlockOutputNode: RelNode = {
+    if (hasMultipleBlockOutputNodes) {
+      // If has multiple block output nodes, the `relNode` is a break-point.
+      // So the `relNode` is the output node of the block that the `relNode` belongs to
+      relNode
+    } else {
+      // the `relNode` is not a break-point
+      require(blockOutputNodes.size == 1)
+      blockOutputNodes.head
+    }
+  }
+}
+
+/**
+  * Builds [[RelNodeBlock]] plan
+  */
+class RelNodeBlockPlanBuilder private(tEnv: TableEnvironment) {
+
+  private val node2Wrapper = new util.IdentityHashMap[RelNode, RelNodeWrapper]()
+  private val node2Block = new util.IdentityHashMap[RelNode, RelNodeBlock]()
+
+  private val isUnionAllAsBreakPointDisabled = tEnv.config.getConf.getBoolean(
+    PlannerConfigOptions.SQL_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED)
+
+
+  /**
+    * Decompose the [[RelNode]] plan into many [[RelNodeBlock]]s,
+    * and rebuild [[RelNodeBlock]] plan.
+    *
+    * @param  sinks RelNode DAG to decompose
+    * @return Sink-RelNodeBlocks, each Sink-RelNodeBlock is a tree.
+    */
+  def buildRelNodeBlockPlan(sinks: Seq[RelNode]): Seq[RelNodeBlock] = {
+    sinks.foreach(buildRelNodeWrappers(_, None))
+    buildBlockOutputNodes(sinks)
+    sinks.map(buildBlockPlan)
+  }
+
+  private def buildRelNodeWrappers(node: RelNode, parent: Option[RelNode]): Unit = {
+    node2Wrapper.getOrElseUpdate(node, new RelNodeWrapper(node)).addParentNode(parent)
+    node.getInputs.foreach(child => buildRelNodeWrappers(child, Some(node)))
+  }
+
+  private def buildBlockPlan(node: RelNode): RelNodeBlock = {
+    val currentBlock = new RelNodeBlock(node, tEnv)
+    buildBlock(node, currentBlock, createNewBlockWhenMeetValidBreakPoint = false)
+    currentBlock
+  }
+
+  private def buildBlock(
+      node: RelNode,
+      currentBlock: RelNodeBlock,
+      createNewBlockWhenMeetValidBreakPoint: Boolean): Unit = {
+    val hasDiffBlockOutputNodes = node2Wrapper(node).hasMultipleBlockOutputNodes
+    val validBreakPoint = isValidBreakPoint(node)
+
+    if (validBreakPoint && (createNewBlockWhenMeetValidBreakPoint || hasDiffBlockOutputNodes)) {
+      val childBlock = node2Block.getOrElseUpdate(node, new RelNodeBlock(node, tEnv))
+      currentBlock.addChild(childBlock)
+      node.getInputs.foreach {
+        child => buildBlock(child, childBlock, createNewBlockWhenMeetValidBreakPoint = false)
+      }
+    } else {
+      val newCreateNewBlockWhenMeetValidBreakPoint =
+        createNewBlockWhenMeetValidBreakPoint || hasDiffBlockOutputNodes && !validBreakPoint
+      node.getInputs.foreach {
+        child => buildBlock(child, currentBlock, newCreateNewBlockWhenMeetValidBreakPoint)
+      }
+    }
+  }
+
+  /**
+    * TableFunctionScan/Snapshot/Window Aggregate cannot be optimized individually,
+    * so TableFunctionScan/Snapshot/Window Aggregate is not a break-point
+    * even though it has multiple parents.
+    */
+  private def isValidBreakPoint(node: RelNode): Boolean = node match {
+    case _: TableFunctionScan | _: Snapshot => false
+    case union: Union if union.all => !isUnionAllAsBreakPointDisabled
+    case project: Project => project.getProjects.forall(p => !hasWindowGroup(p))
+    case agg: Aggregate =>
+      agg.getInput match {
+        case project: Project =>
+          agg.getGroupSet.forall { group =>
+            val p = project.getProjects.get(group)
+            !hasWindowGroup(p)
+          }
+        case _ => true
+      }
+    case _ => true
+  }
+
+  private def hasWindowGroup(rexNode: RexNode): Boolean = {
+    WindowPropertiesRules.hasGroupAuxiliaries(rexNode) ||
+      WindowPropertiesRules.hasGroupFunction(rexNode)
+  }
+
+  private def buildBlockOutputNodes(sinks: Seq[RelNode]): Unit = {
+    // init sink block output node
+    sinks.foreach(sink => node2Wrapper.get(sink).addBlockOutputNode(sink))
+
+    val unvisitedNodeQueue: util.Deque[RelNode] = new util.ArrayDeque[RelNode]()
+    unvisitedNodeQueue.addAll(sinks)
+    while (unvisitedNodeQueue.nonEmpty) {
+      val node = unvisitedNodeQueue.removeFirst()
+      val wrapper = node2Wrapper.get(node)
+      require(wrapper != null)
+      val blockOutputNode = wrapper.getBlockOutputNode
+      buildBlockOutputNodes(None, node, blockOutputNode, unvisitedNodeQueue)
+    }
+  }
+
+  private def buildBlockOutputNodes(
+      parent: Option[RelNode],
+      node: RelNode,
+      curBlockOutputNode: RelNode,
+      unvisitedNodeQueue: util.Deque[RelNode]): Unit = {
+    val wrapper = node2Wrapper.get(node)
+    require(wrapper != null)
+    wrapper.addBlockOutputNode(curBlockOutputNode)
+    wrapper.addVisitedParentNode(parent)
+
+    // the node can be visited only when its all parent nodes have been visited
+    if (wrapper.allParentNodesVisited) {
+      val newBlockOutputNode = if (wrapper.hasMultipleBlockOutputNodes) {
+        // if the node has different output node, the node is the output node of current block.
+        node
+      } else {
+        curBlockOutputNode
+      }
+      node.getInputs.foreach { input =>
+        buildBlockOutputNodes(Some(node), input, newBlockOutputNode, unvisitedNodeQueue)
+      }
+      unvisitedNodeQueue.remove(node)
+    } else {
+      // visit later
+      unvisitedNodeQueue.addLast(node)
+    }
+  }
+
+}
+
+object RelNodeBlockPlanBuilder {
+
+  /**
+    * Decompose the [[RelNode]] trees into [[RelNodeBlock]] trees. First, convert LogicalNode
+    * trees to RelNode trees. Second, reuse same sub-plan in different trees. Third, decompose the
+    * RelNode dag to [[RelNodeBlock]] trees.
+    *
+    * @param  sinkNodes SinkNodes belongs to a LogicalNode plan.
+    * @return Sink-RelNodeBlocks, each Sink-RelNodeBlock is a tree.
+    */
+  def buildRelNodeBlockPlan(
+      sinkNodes: Seq[RelNode],
+      tEnv: TableEnvironment): Seq[RelNodeBlock] = {
+    require(sinkNodes.nonEmpty)
+
+    if (sinkNodes.size == 1) {
+      Seq(new RelNodeBlock(sinkNodes.head, tEnv))
+    } else {
+      // merge multiple RelNode trees to RelNode dag
+      val relNodeDag = reuseRelNodes(sinkNodes, tEnv.getConfig)
+      val builder = new RelNodeBlockPlanBuilder(tEnv)
+      builder.buildRelNodeBlockPlan(relNodeDag)
+    }
+  }
+
+  /**
+    * Reuse common sub-plan in different RelNode tree, generate a RelNode dag
+    *
+    * @param relNodes RelNode trees
+    * @return RelNode dag which reuse common subPlan in each tree
+    */
+  private def reuseRelNodes(relNodes: Seq[RelNode], tableConfig: TableConfig): Seq[RelNode] = {
+    // expand RelTable in TableScan
+    val shuttle = new ExpandTableScanShuttle
+    val convertedRelNodes = relNodes.map(_.accept(shuttle))
+
+    val findOpBlockWithDigest = tableConfig.getConf.getBoolean(
+      PlannerConfigOptions.SQL_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED)
+    if (!findOpBlockWithDigest) {
+      return convertedRelNodes
+    }
+
+    // reuse sub-plan with same digest in input RelNode trees.
+    val context = new SubplanReuseContext(true, convertedRelNodes: _*)
+    val reuseShuttle = new SubplanReuseShuttle(context)
+    convertedRelNodes.map(_.accept(reuseShuttle))
+  }
+
+}
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/optimize/StreamCommonSubGraphBasedOptimizer.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/optimize/StreamCommonSubGraphBasedOptimizer.scala
new file mode 100644
index 0000000..e54965f
--- /dev/null
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/optimize/StreamCommonSubGraphBasedOptimizer.scala
@@ -0,0 +1,288 @@
+/*
+ * 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.plan.optimize
+
+import org.apache.flink.table.api.{StreamTableEnvironment, TableConfig, TableImpl}
+import org.apache.flink.table.plan.`trait`.{AccMode, AccModeTraitDef, UpdateAsRetractionTraitDef}
+import org.apache.flink.table.plan.metadata.FlinkRelMetadataQuery
+import org.apache.flink.table.plan.nodes.calcite.Sink
+import org.apache.flink.table.plan.nodes.physical.stream.{StreamExecDataStreamScan, StreamExecIntermediateTableScan, StreamPhysicalRel}
+import org.apache.flink.table.plan.optimize.program.{FlinkStreamProgram, StreamOptimizeContext}
+import org.apache.flink.table.plan.schema.IntermediateRelTable
+import org.apache.flink.table.plan.stats.FlinkStatistic
+import org.apache.flink.table.sinks.{DataStreamTableSink, RetractStreamTableSink}
+import org.apache.flink.util.Preconditions
+
+import org.apache.calcite.plan.volcano.VolcanoPlanner
+import org.apache.calcite.rel.RelNode
+import org.apache.calcite.rel.core.TableScan
+import org.apache.calcite.rex.RexBuilder
+
+import java.util
+
+import scala.collection.JavaConversions._
+
+/**
+  * A [[CommonSubGraphBasedOptimizer]] for Stream.
+  */
+class StreamCommonSubGraphBasedOptimizer(tEnv: StreamTableEnvironment)
+  extends CommonSubGraphBasedOptimizer {
+
+  override protected def doOptimize(roots: Seq[RelNode]): Seq[RelNodeBlock] = {
+    // build RelNodeBlock plan
+    val sinkBlocks = RelNodeBlockPlanBuilder.buildRelNodeBlockPlan(roots, tEnv)
+    // infer updateAsRetraction property for sink block
+    sinkBlocks.foreach { sinkBlock =>
+      val retractionFromRoot = sinkBlock.outputNode match {
+        case n: Sink =>
+          n.sink match {
+            case _: RetractStreamTableSink[_] => true
+            case s: DataStreamTableSink[_] => s.updatesAsRetraction
+            case _ => false
+          }
+        case o =>
+          o.getTraitSet.getTrait(UpdateAsRetractionTraitDef.INSTANCE).sendsUpdatesAsRetractions
+      }
+      sinkBlock.setUpdateAsRetraction(retractionFromRoot)
+    }
+
+    if (sinkBlocks.size == 1) {
+      // If there is only one sink block, the given relational expressions are a simple tree
+      // (only one root), not a dag. So many operations (e.g. `infer updateAsRetraction property`,
+      // `propagate updateAsRetraction property`) can be omitted to save optimization time.
+      val block = sinkBlocks.head
+      val optimizedTree = optimizeTree(
+        block.getPlan,
+        block.isUpdateAsRetraction,
+        isSinkBlock = true)
+      block.setOptimizedPlan(optimizedTree)
+      return sinkBlocks
+    }
+
+    // infer updateAsRetraction property for all input blocks
+    sinkBlocks.foreach(b => inferUpdateAsRetraction(
+      b, b.isUpdateAsRetraction, isSinkBlock = true))
+    // propagate updateAsRetraction to all input blocks
+    sinkBlocks.foreach(propagateTraits)
+    // clear the intermediate result
+    sinkBlocks.foreach(resetIntermediateResult)
+    // optimize recursively RelNodeBlock
+    sinkBlocks.foreach(b => optimizeBlock(b, isSinkBlock = true))
+    sinkBlocks
+  }
+
+  private def optimizeBlock(block: RelNodeBlock, isSinkBlock: Boolean): Unit = {
+    block.children.foreach {
+      child =>
+        if (child.getNewOutputNode.isEmpty) {
+          optimizeBlock(child, isSinkBlock = false)
+        }
+    }
+
+    val blockLogicalPlan = block.getPlan
+    blockLogicalPlan match {
+      case s: Sink =>
+        require(isSinkBlock)
+        val optimizedTree = optimizeTree(
+          s,
+          updatesAsRetraction = block.isUpdateAsRetraction,
+          isSinkBlock = true)
+        block.setOptimizedPlan(optimizedTree)
+
+      case o =>
+        val optimizedPlan = optimizeTree(
+          o,
+          updatesAsRetraction = block.isUpdateAsRetraction,
+          isSinkBlock = isSinkBlock)
+        val isAccRetract = optimizedPlan.getTraitSet
+          .getTrait(AccModeTraitDef.INSTANCE).getAccMode == AccMode.AccRetract
+        val name = tEnv.createUniqueTableName()
+        registerIntermediateTable(tEnv, name, optimizedPlan, isAccRetract)
+        val newTable = tEnv.scan(name)
+        block.setNewOutputNode(newTable.asInstanceOf[TableImpl].getRelNode)
+        block.setOutputTableName(name)
+        block.setOptimizedPlan(optimizedPlan)
+    }
+  }
+
+  /**
+    * Generates the optimized [[RelNode]] tree from the original relational node tree.
+    *
+    * @param relNode The root node of the relational expression tree.
+    * @param updatesAsRetraction True if request updates as retraction messages.
+    * @param isSinkBlock True if the given block is sink block.
+    * @return The optimized [[RelNode]] tree
+    */
+  private def optimizeTree(
+      relNode: RelNode,
+      updatesAsRetraction: Boolean,
+      isSinkBlock: Boolean): RelNode = {
+
+    val config = tEnv.getConfig
+    val programs = config.getCalciteConfig.getStreamProgram
+      .getOrElse(FlinkStreamProgram.buildProgram(config.getConf))
+    Preconditions.checkNotNull(programs)
+
+    programs.optimize(relNode, new StreamOptimizeContext() {
+
+      override def getRexBuilder: RexBuilder = tEnv.getRelBuilder.getRexBuilder
+
+      override def needFinalTimeIndicatorConversion: Boolean = true
+
+      override def getTableConfig: TableConfig = config
+
+      override def getVolcanoPlanner: VolcanoPlanner = tEnv.getPlanner.asInstanceOf[VolcanoPlanner]
+
+      override def updateAsRetraction: Boolean = updatesAsRetraction
+    })
+  }
+
+  /**
+    * Infer UpdateAsRetraction property for each block.
+    * NOTES: this method should not change the original RelNode tree.
+    *
+    * @param block              The [[RelNodeBlock]] instance.
+    * @param retractionFromRoot Whether the sink need update as retraction messages.
+    * @param isSinkBlock        True if the given block is sink block.
+    */
+  private def inferUpdateAsRetraction(
+      block: RelNodeBlock,
+      retractionFromRoot: Boolean,
+      isSinkBlock: Boolean): Unit = {
+
+    block.children.foreach {
+      child =>
+        if (child.getNewOutputNode.isEmpty) {
+          inferUpdateAsRetraction(child, retractionFromRoot = false, isSinkBlock = false)
+        }
+    }
+
+    val blockLogicalPlan = block.getPlan
+    blockLogicalPlan match {
+      case n: Sink =>
+        require(isSinkBlock)
+        val optimizedPlan = optimizeTree(
+          n, retractionFromRoot, isSinkBlock = true)
+        block.setOptimizedPlan(optimizedPlan)
+
+      case o =>
+        val optimizedPlan = optimizeTree(
+          o, retractionFromRoot, isSinkBlock = isSinkBlock)
+        val name = tEnv.createUniqueTableName()
+        registerIntermediateTable(tEnv, name, optimizedPlan, isAccRetract = false)
+        val newTable = tEnv.scan(name)
+        block.setNewOutputNode(newTable.asInstanceOf[TableImpl].getRelNode)
+        block.setOutputTableName(name)
+        block.setOptimizedPlan(optimizedPlan)
+    }
+  }
+
+  /**
+    * Propagate updateAsRetraction to all input blocks.
+    *
+    * @param block The [[RelNodeBlock]] instance.
+    */
+  private def propagateTraits(block: RelNodeBlock): Unit = {
+
+    // process current block
+    def shipTraits(rel: RelNode, updateAsRetraction: Boolean): Unit = {
+      rel match {
+        case _: StreamExecDataStreamScan | _: StreamExecIntermediateTableScan =>
+          val scan = rel.asInstanceOf[TableScan]
+          val retractionTrait = scan.getTraitSet.getTrait(UpdateAsRetractionTraitDef.INSTANCE)
+          val tableName = scan.getTable.getQualifiedName.last
+          val inputBlocks = block.children.filter(_.getOutputTableName eq tableName)
+          Preconditions.checkArgument(inputBlocks.size <= 1)
+          if (inputBlocks.size == 1) {
+            if (retractionTrait.sendsUpdatesAsRetractions || updateAsRetraction) {
+              inputBlocks.head.setUpdateAsRetraction(true)
+            }
+          }
+        case ser: StreamPhysicalRel => ser.getInputs.foreach { e =>
+          if (ser.needsUpdatesAsRetraction(e) || (updateAsRetraction && !ser.consumesRetractions)) {
+            shipTraits(e, updateAsRetraction = true)
+          } else {
+            shipTraits(e, updateAsRetraction = false)
+          }
+        }
+      }
+    }
+
+    shipTraits(block.getOptimizedPlan, block.isUpdateAsRetraction)
+    block.children.foreach(propagateTraits)
+  }
+
+  /**
+    * Reset the intermediate result including newOutputNode and outputTableName
+    *
+    * @param block the [[RelNodeBlock]] instance.
+    */
+  private def resetIntermediateResult(block: RelNodeBlock): Unit = {
+    block.setNewOutputNode(null)
+    block.setOutputTableName(null)
+
+    block.children.foreach {
+      child =>
+        if (child.getNewOutputNode.nonEmpty) {
+          resetIntermediateResult(child)
+        }
+    }
+  }
+
+  private def registerIntermediateTable(
+      tEnv: StreamTableEnvironment,
+      name: String,
+      relNode: RelNode,
+      isAccRetract: Boolean): Unit = {
+    val uniqueKeys = getUniqueKeys(tEnv, relNode)
+    val monotonicity = FlinkRelMetadataQuery
+      .reuseOrCreate(tEnv.getRelBuilder.getCluster.getMetadataQuery)
+      .getRelModifiedMonotonicity(relNode)
+    val statistic = FlinkStatistic.builder()
+      .uniqueKeys(uniqueKeys)
+      .relModifiedMonotonicity(monotonicity)
+      .build()
+
+    val table = new IntermediateRelTable(
+      relNode,
+      isAccRetract,
+      statistic)
+    tEnv.registerTableInternal(name, table)
+  }
+
+  private def getUniqueKeys(
+      tEnv: StreamTableEnvironment,
+      relNode: RelNode): util.Set[_ <: util.Set[String]] = {
+    val rowType = relNode.getRowType
+    val fmq = FlinkRelMetadataQuery.reuseOrCreate(tEnv.getRelBuilder.getCluster.getMetadataQuery)
+    val uniqueKeys = fmq.getUniqueKeys(relNode)
+    if (uniqueKeys != null) {
+      uniqueKeys.filter(_.nonEmpty).map { uniqueKey =>
+        val keys = new util.HashSet[String]()
+        uniqueKey.asList().foreach { idx =>
+          keys.add(rowType.getFieldNames.get(idx))
+        }
+        keys
+      }
+    } else {
+      null
+    }
+  }
+
+}
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/optimize/StreamOptimizer.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/optimize/StreamOptimizer.scala
deleted file mode 100644
index 189bf36..0000000
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/optimize/StreamOptimizer.scala
+++ /dev/null
@@ -1,81 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.table.plan.optimize
-
-import org.apache.flink.table.api.{StreamTableEnvironment, TableConfig}
-import org.apache.flink.table.plan.`trait`.UpdateAsRetractionTraitDef
-import org.apache.flink.table.plan.nodes.calcite.Sink
-import org.apache.flink.table.plan.optimize.program.{FlinkStreamProgram, StreamOptimizeContext}
-import org.apache.flink.table.sinks.{DataStreamTableSink, RetractStreamTableSink}
-import org.apache.flink.util.Preconditions
-
-import org.apache.calcite.plan.volcano.VolcanoPlanner
-import org.apache.calcite.rel.RelNode
-import org.apache.calcite.rex.RexBuilder
-
-/**
-  * Query optimizer for Stream.
-  */
-class StreamOptimizer(tEnv: StreamTableEnvironment) extends Optimizer {
-
-  override def optimize(roots: Seq[RelNode]): Seq[RelNode] = {
-    // TODO optimize multi-roots as a whole DAG
-    roots.map { root =>
-      val retractionFromRoot = root match {
-        case n: Sink =>
-          n.sink match {
-            case _: RetractStreamTableSink[_] => true
-            case s: DataStreamTableSink[_] => s.updatesAsRetraction
-            case _ => false
-          }
-        case o =>
-          o.getTraitSet.getTrait(UpdateAsRetractionTraitDef.INSTANCE).sendsUpdatesAsRetractions
-      }
-      optimizeTree(root, retractionFromRoot)
-    }
-  }
-
-  /**
-    * Generates the optimized [[RelNode]] tree from the original relational node tree.
-    *
-    * @param relNode The root node of the relational expression tree.
-    * @param updatesAsRetraction True if request updates as retraction messages.
-    * @return The optimized [[RelNode]] tree
-    */
-  private def optimizeTree(relNode: RelNode, updatesAsRetraction: Boolean): RelNode = {
-    val config = tEnv.getConfig
-    val programs = config.getCalciteConfig.getStreamProgram
-      .getOrElse(FlinkStreamProgram.buildProgram(config.getConf))
-    Preconditions.checkNotNull(programs)
-
-    programs.optimize(relNode, new StreamOptimizeContext() {
-
-      override def getRexBuilder: RexBuilder = tEnv.getRelBuilder.getRexBuilder
-
-      override def needFinalTimeIndicatorConversion: Boolean = true
-
-      override def getTableConfig: TableConfig = config
-
-      override def getVolcanoPlanner: VolcanoPlanner = tEnv.getPlanner.asInstanceOf[VolcanoPlanner]
-
-      override def updateAsRetraction: Boolean = updatesAsRetraction
-    })
-  }
-
-}
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/optimize/program/StreamOptimizeContext.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/optimize/program/StreamOptimizeContext.scala
index 6ea1661..2e2a722 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/optimize/program/StreamOptimizeContext.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/optimize/program/StreamOptimizeContext.scala
@@ -32,7 +32,8 @@ trait StreamOptimizeContext extends FlinkOptimizeContext {
 
   /**
     * Returns true if the sink requests updates as retraction messages
-    * defined in [[org.apache.flink.table.plan.optimize.StreamOptimizer.optimize]].
+    * defined in
+    * [[org.apache.flink.table.plan.optimize.StreamCommonSubGraphBasedOptimizer.optimize]].
     */
   def updateAsRetraction: Boolean
 
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/FlinkBatchRuleSets.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/FlinkBatchRuleSets.scala
index d0c7c9e..bee709a 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/FlinkBatchRuleSets.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/FlinkBatchRuleSets.scala
@@ -251,6 +251,7 @@ object FlinkBatchRuleSets {
     FlinkLogicalTableSourceScan.CONVERTER,
     FlinkLogicalTableFunctionScan.CONVERTER,
     FlinkLogicalDataStreamTableScan.CONVERTER,
+    FlinkLogicalIntermediateTableScan.CONVERTER,
     FlinkLogicalExpand.CONVERTER,
     FlinkLogicalRank.CONVERTER,
     FlinkLogicalWindowAggregate.CONVERTER,
@@ -269,11 +270,15 @@ object FlinkBatchRuleSets {
       LOGICAL_CONVERTERS.asScala
     ).asJava)
 
+  /**
+    * RuleSet to do rewrite on FlinkLogicalRel for batch
+    */
   val LOGICAL_REWRITE: RuleSet = RuleSets.ofList(
     // transpose calc past snapshot
     CalcSnapshotTransposeRule.INSTANCE,
     // merge calc after calc transpose
-    CalcMergeRule.INSTANCE)
+    FlinkCalcMergeRule.INSTANCE
+  )
 
   /**
     * RuleSet to do physical optimize for batch
@@ -282,6 +287,7 @@ object FlinkBatchRuleSets {
     FlinkExpandConversionRule.BATCH_INSTANCE,
     BatchExecBoundedStreamScanRule.INSTANCE,
     BatchExecScanTableSourceRule.INSTANCE,
+    BatchExecIntermediateTableScanRule.INSTANCE,
     BatchExecValuesRule.INSTANCE,
     BatchExecCalcRule.INSTANCE,
     BatchExecUnionRule.INSTANCE,
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/FlinkStreamRuleSets.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/FlinkStreamRuleSets.scala
index 4b5032a..fd8014b 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/FlinkStreamRuleSets.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/FlinkStreamRuleSets.scala
@@ -239,6 +239,7 @@ object FlinkStreamRuleSets {
     FlinkLogicalTableSourceScan.CONVERTER,
     FlinkLogicalTableFunctionScan.CONVERTER,
     FlinkLogicalDataStreamTableScan.CONVERTER,
+    FlinkLogicalIntermediateTableScan.CONVERTER,
     FlinkLogicalExpand.CONVERTER,
     FlinkLogicalWatermarkAssigner.CONVERTER,
     FlinkLogicalWindowAggregate.CONVERTER,
@@ -258,7 +259,7 @@ object FlinkStreamRuleSets {
     ).asJava)
 
   /**
-    * RuleSet to od rewrite on FlinkLogicalRel for Stream
+    * RuleSet to do rewrite on FlinkLogicalRel for Stream
     */
   val LOGICAL_REWRITE: RuleSet = RuleSets.ofList(
     // transform over window to topn node
@@ -278,6 +279,7 @@ object FlinkStreamRuleSets {
     FlinkExpandConversionRule.STREAM_INSTANCE,
     StreamExecDataStreamScanRule.INSTANCE,
     StreamExecTableSourceScanRule.INSTANCE,
+    StreamExecIntermediateTableScanRule.INSTANCE,
     StreamExecValuesRule.INSTANCE,
     StreamExecCalcRule.INSTANCE,
     StreamExecUnionRule.INSTANCE,
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecIntermediateTableScanRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecIntermediateTableScanRule.scala
new file mode 100644
index 0000000..5b6ff1a
--- /dev/null
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecIntermediateTableScanRule.scala
@@ -0,0 +1,48 @@
+/*
+ * 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.plan.rules.physical.batch
+
+import org.apache.flink.table.plan.nodes.FlinkConventions
+import org.apache.flink.table.plan.nodes.logical.FlinkLogicalIntermediateTableScan
+import org.apache.flink.table.plan.nodes.physical.batch.BatchExecIntermediateTableScan
+
+import org.apache.calcite.plan.RelOptRule
+import org.apache.calcite.rel.convert.ConverterRule
+import org.apache.calcite.rel.RelNode
+
+/**
+  * Rule that converts [[FlinkLogicalIntermediateTableScan]] to [[BatchExecIntermediateTableScan]].
+  */
+class BatchExecIntermediateTableScanRule
+  extends ConverterRule(
+    classOf[FlinkLogicalIntermediateTableScan],
+    FlinkConventions.LOGICAL,
+    FlinkConventions.BATCH_PHYSICAL,
+    "BatchExecIntermediateTableScanRule") {
+
+  def convert(rel: RelNode): RelNode = {
+    val scan = rel.asInstanceOf[FlinkLogicalIntermediateTableScan]
+    val newTrait = rel.getTraitSet.replace(FlinkConventions.BATCH_PHYSICAL)
+    new BatchExecIntermediateTableScan(rel.getCluster, newTrait, scan.getTable, rel.getRowType)
+  }
+}
+
+object BatchExecIntermediateTableScanRule {
+  val INSTANCE: RelOptRule = new BatchExecIntermediateTableScanRule
+}
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/stream/StreamExecIntermediateTableScanRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/stream/StreamExecIntermediateTableScanRule.scala
new file mode 100644
index 0000000..af87779
--- /dev/null
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/stream/StreamExecIntermediateTableScanRule.scala
@@ -0,0 +1,48 @@
+/*
+ * 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.plan.rules.physical.stream
+
+import org.apache.flink.table.plan.nodes.logical.FlinkLogicalIntermediateTableScan
+import org.apache.flink.table.plan.nodes.physical.stream.StreamExecIntermediateTableScan
+
+import org.apache.calcite.plan.RelOptRule
+import org.apache.calcite.rel.RelNode
+import org.apache.calcite.rel.convert.ConverterRule
+import org.apache.flink.table.plan.nodes.FlinkConventions
+
+/**
+  * Rule that converts [[FlinkLogicalIntermediateTableScan]] to [[StreamExecIntermediateTableScan]].
+  */
+class StreamExecIntermediateTableScanRule
+  extends ConverterRule(
+    classOf[FlinkLogicalIntermediateTableScan],
+    FlinkConventions.LOGICAL,
+    FlinkConventions.STREAM_PHYSICAL,
+    "StreamExecIntermediateTableScanRule") {
+
+  def convert(rel: RelNode): RelNode = {
+    val scan = rel.asInstanceOf[FlinkLogicalIntermediateTableScan]
+    val newTrait = rel.getTraitSet.replace(FlinkConventions.STREAM_PHYSICAL)
+    new StreamExecIntermediateTableScan(rel.getCluster, newTrait, scan.getTable, rel.getRowType)
+  }
+}
+
+object StreamExecIntermediateTableScanRule {
+  val INSTANCE: RelOptRule = new StreamExecIntermediateTableScanRule
+}
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/schema/IntermediateRelTable.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/schema/IntermediateRelTable.scala
new file mode 100644
index 0000000..e18fc27
--- /dev/null
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/schema/IntermediateRelTable.scala
@@ -0,0 +1,68 @@
+/*
+ * 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.plan.schema
+
+import org.apache.flink.annotation.Internal
+import org.apache.flink.table.plan.stats.FlinkStatistic
+
+import org.apache.calcite.rel.RelNode
+import org.apache.calcite.rel.`type`.{RelDataType, RelDataTypeFactory}
+
+/**
+  * An intermediate Table to wrap a optimized RelNode inside. The input data of this Table is
+  * generated by the underlying optimized RelNode.
+  *
+  * This table is very similar to [[RelTable]], they both wrap a RelNode, however they could not
+  * be unified. Because [[RelTable]] is treated as view which will be expanded before
+  * the optimization is started, this table will be expanded after the optimization is complete.
+  *
+  * @param relNode       underlying optimized RelNode
+  * @param isAccRetract  true if input data of table contain retraction messages.
+  * @param statistic     statistics of current Table
+  */
+@Internal
+class IntermediateRelTable(
+    val relNode: RelNode,
+    val isAccRetract: Boolean,
+    val statistic: FlinkStatistic = FlinkStatistic.UNKNOWN)
+  extends FlinkTable {
+
+  def this(relNode: RelNode) {
+    this(relNode, false)
+  }
+
+  override def getRowType(typeFactory: RelDataTypeFactory): RelDataType = relNode.getRowType
+
+  /**
+    * Creates a copy of this table, changing statistic.
+    *
+    * @param statistic A new FlinkStatistic.
+    * @return Copy of this table, substituting statistic.
+    */
+  override def copy(statistic: FlinkStatistic): FlinkTable =
+    new IntermediateRelTable(relNode, isAccRetract, statistic)
+
+  /**
+    * Returns statistics of current table
+    *
+    * @return statistics of current table
+    */
+  override def getStatistic: FlinkStatistic = statistic
+
+}
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/schema/RelTable.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/schema/RelTable.scala
index 57c725c..c2e7933 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/schema/RelTable.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/schema/RelTable.scala
@@ -23,8 +23,8 @@ import org.apache.calcite.plan.RelOptTable.ToRelContext
 import org.apache.calcite.rel.RelNode
 import org.apache.calcite.rel.`type`.{RelDataType, RelDataTypeFactory}
 import org.apache.calcite.schema.Schema.TableType
-import org.apache.calcite.schema.impl.AbstractTable
 import org.apache.calcite.schema.TranslatableTable
+import org.apache.calcite.schema.impl.AbstractTable
 
 /**
  * A [[org.apache.calcite.schema.Table]] implementation for registering
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/stats/FlinkStatistic.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/stats/FlinkStatistic.scala
index 8d6f049..30797b7 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/stats/FlinkStatistic.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/stats/FlinkStatistic.scala
@@ -35,7 +35,7 @@ import scala.collection.JavaConversions._
 class FlinkStatistic(
     tableStats: TableStats,
     uniqueKeys: util.Set[_ <: util.Set[String]] = null,
-    monotonicity: RelModifiedMonotonicity = null)
+    relModifiedMonotonicity: RelModifiedMonotonicity = null)
   extends Statistic {
 
   require(uniqueKeys == null || !uniqueKeys.exists(keys => keys == null || keys.isEmpty),
@@ -71,7 +71,7 @@ class FlinkStatistic(
   /**
     * Returns the modified monotonicity of the table
     */
-  def getRelModifiedMonotonicity: RelModifiedMonotonicity = monotonicity
+  def getRelModifiedMonotonicity: RelModifiedMonotonicity = relModifiedMonotonicity
 
   /**
     * Returns the number of rows of the table.
@@ -121,6 +121,7 @@ object FlinkStatistic {
 
     private var tableStats: TableStats = _
     private var uniqueKeys: util.Set[_ <: util.Set[String]] = _
+    private var relModifiedMonotonicity: RelModifiedMonotonicity = _
 
     def tableStats(tableStats: TableStats): Builder = {
       this.tableStats = tableStats
@@ -132,18 +133,24 @@ object FlinkStatistic {
       this
     }
 
+    def relModifiedMonotonicity(monotonicity: RelModifiedMonotonicity): Builder = {
+      this.relModifiedMonotonicity = monotonicity
+      this
+    }
+
     def statistic(statistic: FlinkStatistic): Builder = {
       require(statistic != null, "input statistic cannot be null!")
       this.tableStats = statistic.getTableStats
       this.uniqueKeys = statistic.getUniqueKeys
+      this.relModifiedMonotonicity = statistic.getRelModifiedMonotonicity
       this
     }
 
     def build(): FlinkStatistic = {
-      if (tableStats == null && uniqueKeys == null) {
+      if (tableStats == null && uniqueKeys == null && relModifiedMonotonicity == null) {
         UNKNOWN
       } else {
-        new FlinkStatistic(tableStats, uniqueKeys)
+        new FlinkStatistic(tableStats, uniqueKeys, relModifiedMonotonicity)
       }
     }
   }
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/util/PlanUtil.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/util/PlanUtil.scala
new file mode 100644
index 0000000..9efbcad
--- /dev/null
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/util/PlanUtil.scala
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.util
+
+import org.apache.flink.streaming.api.graph.StreamGraph
+
+import java.io.{PrintWriter, StringWriter}
+
+import scala.collection.JavaConversions._
+
+
+object PlanUtil extends Logging {
+
+  /**
+    * Converting an StreamGraph to a human-readable string.
+    *
+    * @param graph stream graph
+    */
+  def explainStreamGraph(graph: StreamGraph): String = {
+    def isSource(id: Int): Boolean = graph.getSourceIDs.contains(id)
+
+    def isSink(id: Int): Boolean = graph.getSinkIDs.contains(id)
+
+    // can not convert to single abstract method because it will throw compile error
+    implicit val order: Ordering[Int] = new Ordering[Int] {
+      override def compare(x: Int, y: Int): Int = (isSink(x), isSink(y)) match {
+        case (true, false) => 1
+        case (false, true) => -1
+        case (_, _) => x - y
+      }
+    }
+
+    val operatorIDs = graph.getStreamNodes.map(_.getId).toList.sorted(order)
+    val sw = new StringWriter()
+    val pw = new PrintWriter(sw)
+
+    var tabs = 0
+    operatorIDs.foreach { id =>
+      val op = graph.getStreamNode(id)
+      val (nodeType, content) = if (isSource(id)) {
+        tabs = 0
+        ("Data Source", "collect elements with CollectionInputFormat")
+      } else if (isSink(id)) {
+        ("Data Sink", op.getOperatorName)
+      } else {
+        ("Operator", op.getOperatorName)
+      }
+
+      pw.append("\t" * tabs).append(s"Stage $id : $nodeType\n")
+        .append("\t" * (tabs + 1)).append(s"content : $content\n")
+
+      if (!isSource(id)) {
+        val partition = op.getInEdges.head.getPartitioner.toString
+        pw.append("\t" * (tabs + 1)).append(s"ship_strategy : $partition\n")
+      }
+
+      pw.append("\n")
+      tabs += 1
+    }
+
+    pw.close()
+    sw.toString
+  }
+
+}
diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/api/batch/ExplainTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/api/batch/ExplainTest.xml
index bdd2453..3cd0486 100644
--- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/api/batch/ExplainTest.xml
+++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/api/batch/ExplainTest.xml
@@ -16,14 +16,656 @@ See the License for the specific language governing permissions and
 limitations under the License.
 -->
 <Root>
-  <TestCase name="testExplainTableSourceScan">
+  <TestCase name="testExplainWithAgg[extended=false]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalProject(EXPR$0=[$1])
++- LogicalAggregate(group=[{0}], EXPR$0=[COUNT()])
+   +- LogicalProject(a=[$0])
+      +- LogicalTableScan(table=[[MyTable1]])
+
+== Optimized Logical Plan ==
+Calc(select=[EXPR$0])
++- HashAggregate(isMerge=[true], groupBy=[a], select=[a, Final_COUNT(count1$0) AS EXPR$0])
+   +- Exchange(distribution=[hash[a]])
+      +- LocalHashAggregate(groupBy=[a], select=[a, Partial_COUNT(*) AS count1$0])
+         +- Calc(select=[a])
+            +- BoundedStreamScan(table=[[MyTable1]], fields=[a, b, c])
+
+== Physical Execution Plan ==
+ : Data Source
+	content : collect elements with CollectionInputFormat
+
+	 : Operator
+		content : SourceConversion(table:Buffer(MyTable1), fields:(a, b, c))
+		ship_strategy : FORWARD
+
+		 : Operator
+			content : Calc(select: (a))
+			ship_strategy : FORWARD
+
+			 : Operator
+				content : LocalHashAggregate(groupBy:(a),select:(a, Partial_COUNT(*) AS count1$0),)
+				ship_strategy : FORWARD
+
+				 : Operator
+					content : GlobalHashAggregate(groupBy:(a),select:(a, Final_COUNT(count1$0) AS EXPR$0),)
+					ship_strategy : HASH[a]
+
+					 : Operator
+						content : Calc(select: (EXPR$0))
+						ship_strategy : FORWARD
+
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testExplainWithAgg[extended=true]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalProject(EXPR$0=[$1])
++- LogicalAggregate(group=[{0}], EXPR$0=[COUNT()])
+   +- LogicalProject(a=[$0])
+      +- LogicalTableScan(table=[[MyTable1]])
+
+== Optimized Logical Plan ==
+Calc(select=[EXPR$0]): rowcount = , cumulative cost = {rows, cpu, io, network, memory}
++- HashAggregate(isMerge=[true], groupBy=[a], select=[a, Final_COUNT(count1$0) AS EXPR$0]): rowcount = , cumulative cost = {rows, cpu, io, network, memory}
+   +- Exchange(distribution=[hash[a]]): rowcount = , cumulative cost = {rows, cpu, io, network, memory}
+      +- LocalHashAggregate(groupBy=[a], select=[a, Partial_COUNT(*) AS count1$0]): rowcount = , cumulative cost = {rows, cpu, io, network, memory}
+         +- Calc(select=[a]): rowcount = , cumulative cost = {rows, cpu, io, network, memory}
+            +- BoundedStreamScan(table=[[MyTable1]], fields=[a, b, c]): rowcount = , cumulative cost = {rows, cpu, io, network, memory}
+
+== Physical Execution Plan ==
+ : Data Source
+	content : collect elements with CollectionInputFormat
+
+	 : Operator
+		content : SourceConversion(table:Buffer(MyTable1), fields:(a, b, c))
+		ship_strategy : FORWARD
+
+		 : Operator
+			content : Calc(select: (a))
+			ship_strategy : FORWARD
+
+			 : Operator
+				content : LocalHashAggregate(groupBy:(a),select:(a, Partial_COUNT(*) AS count1$0),)
+				ship_strategy : FORWARD
+
+				 : Operator
+					content : GlobalHashAggregate(groupBy:(a),select:(a, Final_COUNT(count1$0) AS EXPR$0),)
+					ship_strategy : HASH[a]
+
+					 : Operator
+						content : Calc(select: (EXPR$0))
+						ship_strategy : FORWARD
+
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testExplainWithDataStreamScan[extended=false]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalProject(a=[$0], b=[$1], c=[$2])
++- LogicalTableScan(table=[[MyTable1]])
+
+== Optimized Logical Plan ==
+BoundedStreamScan(table=[[MyTable1]], fields=[a, b, c])
+
+== Physical Execution Plan ==
+ : Data Source
+	content : collect elements with CollectionInputFormat
+
+	 : Operator
+		content : SourceConversion(table:Buffer(MyTable1), fields:(a, b, c))
+		ship_strategy : FORWARD
+
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testExplainWithDataStreamScan[extended=true]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalProject(a=[$0], b=[$1], c=[$2])
++- LogicalTableScan(table=[[MyTable1]])
+
+== Optimized Logical Plan ==
+BoundedStreamScan(table=[[MyTable1]], fields=[a, b, c]): rowcount = , cumulative cost = {rows, cpu, io, network, memory}
+
+== Physical Execution Plan ==
+ : Data Source
+	content : collect elements with CollectionInputFormat
+
+	 : Operator
+		content : SourceConversion(table:Buffer(MyTable1), fields:(a, b, c))
+		ship_strategy : FORWARD
+
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testExplainWithFilter[extended=false]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalProject(a=[$0], b=[$1], c=[$2])
++- LogicalFilter(condition=[=(MOD($0, 2), 0)])
+   +- LogicalTableScan(table=[[MyTable1]])
+
+== Optimized Logical Plan ==
+Calc(select=[a, b, c], where=[=(MOD(a, 2), 0)])
++- BoundedStreamScan(table=[[MyTable1]], fields=[a, b, c])
+
+== Physical Execution Plan ==
+ : Data Source
+	content : collect elements with CollectionInputFormat
+
+	 : Operator
+		content : SourceConversion(table:Buffer(MyTable1), fields:(a, b, c))
+		ship_strategy : FORWARD
+
+		 : Operator
+			content : Calc(where: ((a MOD 2) = 0), select: (a, b, c))
+			ship_strategy : FORWARD
+
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testExplainWithFilter[extended=true]">
     <Resource name="explain">
       <![CDATA[== Abstract Syntax Tree ==
 LogicalProject(a=[$0], b=[$1], c=[$2])
-+- LogicalTableScan(table=[[MyTable]])
++- LogicalFilter(condition=[=(MOD($0, 2), 0)])
+   +- LogicalTableScan(table=[[MyTable1]])
+
+== Optimized Logical Plan ==
+Calc(select=[a, b, c], where=[=(MOD(a, 2), 0)]): rowcount = , cumulative cost = {rows, cpu, io, network, memory}
++- BoundedStreamScan(table=[[MyTable1]], fields=[a, b, c]): rowcount = , cumulative cost = {rows, cpu, io, network, memory}
+
+== Physical Execution Plan ==
+ : Data Source
+	content : collect elements with CollectionInputFormat
+
+	 : Operator
+		content : SourceConversion(table:Buffer(MyTable1), fields:(a, b, c))
+		ship_strategy : FORWARD
+
+		 : Operator
+			content : Calc(where: ((a MOD 2) = 0), select: (a, b, c))
+			ship_strategy : FORWARD
+
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testExplainWithJoin[extended=false]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalProject(a=[$0], b=[$1], c=[$2], e=[$4], f=[$5])
++- LogicalFilter(condition=[=($0, $3)])
+   +- LogicalJoin(condition=[true], joinType=[inner])
+      :- LogicalTableScan(table=[[MyTable1]])
+      +- LogicalTableScan(table=[[MyTable2]])
+
+== Optimized Logical Plan ==
+Calc(select=[a, b, c, e, f])
++- SortMergeJoin(joinType=[InnerJoin], where=[=(a, d)], select=[a, b, c, d, e, f])
+   :- Exchange(distribution=[hash[a]])
+   :  +- BoundedStreamScan(table=[[MyTable1]], fields=[a, b, c])
+   +- Exchange(distribution=[hash[d]])
+      +- BoundedStreamScan(table=[[MyTable2]], fields=[d, e, f])
+
+== Physical Execution Plan ==
+ : Data Source
+	content : collect elements with CollectionInputFormat
+
+ : Data Source
+	content : collect elements with CollectionInputFormat
+
+	 : Operator
+		content : SourceConversion(table:Buffer(MyTable1), fields:(a, b, c))
+		ship_strategy : FORWARD
+
+		 : Operator
+			content : SourceConversion(table:Buffer(MyTable2), fields:(d, e, f))
+			ship_strategy : FORWARD
+
+			 : Operator
+				content : SortMergeJoin(where: (a = d))
+				ship_strategy : HASH[a]
+
+				 : Operator
+					content : Calc(select: (a, b, c, e, f))
+					ship_strategy : FORWARD
+
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testExplainWithJoin[extended=true]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalProject(a=[$0], b=[$1], c=[$2], e=[$4], f=[$5])
++- LogicalFilter(condition=[=($0, $3)])
+   +- LogicalJoin(condition=[true], joinType=[inner])
+      :- LogicalTableScan(table=[[MyTable1]])
+      +- LogicalTableScan(table=[[MyTable2]])
 
 == Optimized Logical Plan ==
-BoundedStreamScan(table=[[MyTable]], fields=[a, b, c])
+Calc(select=[a, b, c, e, f]): rowcount = , cumulative cost = {rows, cpu, io, network, memory}
++- SortMergeJoin(joinType=[InnerJoin], where=[=(a, d)], select=[a, b, c, d, e, f]): rowcount = , cumulative cost = {rows, cpu, io, network, memory}
+   :- Exchange(distribution=[hash[a]]): rowcount = , cumulative cost = {rows, cpu, io, network, memory}
+   :  +- BoundedStreamScan(table=[[MyTable1]], fields=[a, b, c]): rowcount = , cumulative cost = {rows, cpu, io, network, memory}
+   +- Exchange(distribution=[hash[d]]): rowcount = , cumulative cost = {rows, cpu, io, network, memory}
+      +- BoundedStreamScan(table=[[MyTable2]], fields=[d, e, f]): rowcount = , cumulative cost = {rows, cpu, io, network, memory}
+
+== Physical Execution Plan ==
+ : Data Source
+	content : collect elements with CollectionInputFormat
+
+ : Data Source
+	content : collect elements with CollectionInputFormat
+
+	 : Operator
+		content : SourceConversion(table:Buffer(MyTable1), fields:(a, b, c))
+		ship_strategy : FORWARD
+
+		 : Operator
+			content : SourceConversion(table:Buffer(MyTable2), fields:(d, e, f))
+			ship_strategy : FORWARD
+
+			 : Operator
+				content : SortMergeJoin(where: (a = d))
+				ship_strategy : HASH[a]
+
+				 : Operator
+					content : Calc(select: (a, b, c, e, f))
+					ship_strategy : FORWARD
+
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testExplainWithMultiSinks[extended=false]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalSink(name=[sink1], fields=[a, cnt])
++- LogicalProject(a=[$0], cnt=[$1])
+   +- LogicalFilter(condition=[>($1, 10)])
+      +- LogicalAggregate(group=[{0}], cnt=[COUNT()])
+         +- LogicalProject(a=[$0])
+            +- LogicalTableScan(table=[[MyTable1]])
+
+LogicalSink(name=[sink1], fields=[a, cnt])
++- LogicalProject(a=[$0], cnt=[$1])
+   +- LogicalFilter(condition=[<($1, 10)])
+      +- LogicalAggregate(group=[{0}], cnt=[COUNT()])
+         +- LogicalProject(a=[$0])
+            +- LogicalTableScan(table=[[MyTable1]])
+
+== Optimized Logical Plan ==
+HashAggregate(isMerge=[true], groupBy=[a], select=[a, Final_COUNT(count1$0) AS cnt], reuse_id=[1])
++- Exchange(distribution=[hash[a]])
+   +- LocalHashAggregate(groupBy=[a], select=[a, Partial_COUNT(*) AS count1$0])
+      +- Calc(select=[a])
+         +- BoundedStreamScan(table=[[MyTable1]], fields=[a, b, c])
+
+Sink(name=[sink1], fields=[a, cnt])
++- Calc(select=[a, cnt], where=[>(cnt, 10)])
+   +- Reused(reference_id=[1])
+
+Sink(name=[sink1], fields=[a, cnt])
++- Calc(select=[a, cnt], where=[<(cnt, 10)])
+   +- Reused(reference_id=[1])
+
+== Physical Execution Plan ==
+ : Data Source
+	content : collect elements with CollectionInputFormat
+
+	 : Operator
+		content : SourceConversion(table:Buffer(MyTable1), fields:(a, b, c))
+		ship_strategy : FORWARD
+
+		 : Operator
+			content : Calc(select: (a))
+			ship_strategy : FORWARD
+
+			 : Operator
+				content : LocalHashAggregate(groupBy:(a),select:(a, Partial_COUNT(*) AS count1$0),)
+				ship_strategy : FORWARD
+
+				 : Operator
+					content : GlobalHashAggregate(groupBy:(a),select:(a, Final_COUNT(count1$0) AS cnt),)
+					ship_strategy : HASH[a]
+
+					 : Operator
+						content : Calc(where: (cnt > 10), select: (a, cnt))
+						ship_strategy : FORWARD
+
+						 : Operator
+							content : SinkConversionToRow
+							ship_strategy : FORWARD
+
+							 : Operator
+								content : Calc(where: (cnt < 10), select: (a, cnt))
+								ship_strategy : FORWARD
+
+								 : Operator
+									content : SinkConversionToRow
+									ship_strategy : FORWARD
+
+									 : Data Sink
+										content : Sink: collect
+										ship_strategy : FORWARD
+
+										 : Data Sink
+											content : Sink: collect
+											ship_strategy : FORWARD
+
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testExplainWithMultiSinks[extended=true]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalSink(name=[sink1], fields=[a, cnt])
++- LogicalProject(a=[$0], cnt=[$1])
+   +- LogicalFilter(condition=[>($1, 10)])
+      +- LogicalAggregate(group=[{0}], cnt=[COUNT()])
+         +- LogicalProject(a=[$0])
+            +- LogicalTableScan(table=[[MyTable1]])
+
+LogicalSink(name=[sink1], fields=[a, cnt])
++- LogicalProject(a=[$0], cnt=[$1])
+   +- LogicalFilter(condition=[<($1, 10)])
+      +- LogicalAggregate(group=[{0}], cnt=[COUNT()])
+         +- LogicalProject(a=[$0])
+            +- LogicalTableScan(table=[[MyTable1]])
+
+== Optimized Logical Plan ==
+HashAggregate(isMerge=[true], groupBy=[a], select=[a, Final_COUNT(count1$0) AS cnt], reuse_id=[1]): rowcount = , cumulative cost = {rows, cpu, io, network, memory}
++- Exchange(distribution=[hash[a]]): rowcount = , cumulative cost = {rows, cpu, io, network, memory}
+   +- LocalHashAggregate(groupBy=[a], select=[a, Partial_COUNT(*) AS count1$0]): rowcount = , cumulative cost = {rows, cpu, io, network, memory}
+      +- Calc(select=[a]): rowcount = , cumulative cost = {rows, cpu, io, network, memory}
+         +- BoundedStreamScan(table=[[MyTable1]], fields=[a, b, c]): rowcount = , cumulative cost = {rows, cpu, io, network, memory}
+
+Sink(name=[sink1], fields=[a, cnt]): rowcount = , cumulative cost = {rows, cpu, io, network, memory}
++- Calc(select=[a, cnt], where=[>(cnt, 10)]): rowcount = , cumulative cost = {rows, cpu, io, network, memory}
+   +- Reused(reference_id=[1])
+
+Sink(name=[sink1], fields=[a, cnt]): rowcount = , cumulative cost = {rows, cpu, io, network, memory}
++- Calc(select=[a, cnt], where=[<(cnt, 10)]): rowcount = , cumulative cost = {rows, cpu, io, network, memory}
+   +- Reused(reference_id=[1])
+
+== Physical Execution Plan ==
+ : Data Source
+	content : collect elements with CollectionInputFormat
+
+	 : Operator
+		content : SourceConversion(table:Buffer(MyTable1), fields:(a, b, c))
+		ship_strategy : FORWARD
+
+		 : Operator
+			content : Calc(select: (a))
+			ship_strategy : FORWARD
+
+			 : Operator
+				content : LocalHashAggregate(groupBy:(a),select:(a, Partial_COUNT(*) AS count1$0),)
+				ship_strategy : FORWARD
+
+				 : Operator
+					content : GlobalHashAggregate(groupBy:(a),select:(a, Final_COUNT(count1$0) AS cnt),)
+					ship_strategy : HASH[a]
+
+					 : Operator
+						content : Calc(where: (cnt > 10), select: (a, cnt))
+						ship_strategy : FORWARD
+
+						 : Operator
+							content : SinkConversionToRow
+							ship_strategy : FORWARD
+
+							 : Operator
+								content : Calc(where: (cnt < 10), select: (a, cnt))
+								ship_strategy : FORWARD
+
+								 : Operator
+									content : SinkConversionToRow
+									ship_strategy : FORWARD
+
+									 : Data Sink
+										content : Sink: collect
+										ship_strategy : FORWARD
+
+										 : Data Sink
+											content : Sink: collect
+											ship_strategy : FORWARD
+
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testExplainWithSingleSink[extended=false]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalSink(fields=[a, b, c])
++- LogicalProject(a=[$0], b=[$1], c=[$2])
+   +- LogicalFilter(condition=[>($0, 10)])
+      +- LogicalTableScan(table=[[MyTable1]])
+
+== Optimized Logical Plan ==
+Sink(fields=[a, b, c])
++- Calc(select=[a, b, c], where=[>(a, 10)])
+   +- BoundedStreamScan(table=[[MyTable1]], fields=[a, b, c])
+
+== Physical Execution Plan ==
+ : Data Source
+	content : collect elements with CollectionInputFormat
+
+	 : Operator
+		content : SourceConversion(table:Buffer(MyTable1), fields:(a, b, c))
+		ship_strategy : FORWARD
+
+		 : Operator
+			content : Calc(where: (a > 10), select: (a, b, c))
+			ship_strategy : FORWARD
+
+			 : Operator
+				content : SinkConversionToRow
+				ship_strategy : FORWARD
+
+				 : Data Sink
+					content : Sink: collect
+					ship_strategy : FORWARD
+
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testExplainWithSingleSink[extended=true]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalSink(fields=[a, b, c])
++- LogicalProject(a=[$0], b=[$1], c=[$2])
+   +- LogicalFilter(condition=[>($0, 10)])
+      +- LogicalTableScan(table=[[MyTable1]])
+
+== Optimized Logical Plan ==
+Sink(fields=[a, b, c]): rowcount = , cumulative cost = {rows, cpu, io, network, memory}
++- Calc(select=[a, b, c], where=[>(a, 10)]): rowcount = , cumulative cost = {rows, cpu, io, network, memory}
+   +- BoundedStreamScan(table=[[MyTable1]], fields=[a, b, c]): rowcount = , cumulative cost = {rows, cpu, io, network, memory}
+
+== Physical Execution Plan ==
+ : Data Source
+	content : collect elements with CollectionInputFormat
+
+	 : Operator
+		content : SourceConversion(table:Buffer(MyTable1), fields:(a, b, c))
+		ship_strategy : FORWARD
+
+		 : Operator
+			content : Calc(where: (a > 10), select: (a, b, c))
+			ship_strategy : FORWARD
+
+			 : Operator
+				content : SinkConversionToRow
+				ship_strategy : FORWARD
+
+				 : Data Sink
+					content : Sink: collect
+					ship_strategy : FORWARD
+
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testExplainWithSort[extended=false]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalSort(sort0=[$0], dir0=[ASC-nulls-first], fetch=[5])
++- LogicalProject(a=[$0], b=[$1], c=[$2])
+   +- LogicalTableScan(table=[[MyTable1]])
+
+== Optimized Logical Plan ==
+SortLimit(orderBy=[a ASC], offset=[0], fetch=[5], global=[true])
++- Exchange(distribution=[single])
+   +- SortLimit(orderBy=[a ASC], offset=[0], fetch=[5], global=[false])
+      +- BoundedStreamScan(table=[[MyTable1]], fields=[a, b, c])
+
+== Physical Execution Plan ==
+ : Data Source
+	content : collect elements with CollectionInputFormat
+
+	 : Operator
+		content : SourceConversion(table:Buffer(MyTable1), fields:(a, b, c))
+		ship_strategy : FORWARD
+
+		 : Operator
+			content : LocalSortLimit(orderBy: [a ASC], offset: 0, fetch: 5)
+			ship_strategy : FORWARD
+
+			 : Operator
+				content : GlobalSortLimit(orderBy: [a ASC], offset: 0, fetch: 5)
+				ship_strategy : GLOBAL
+
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testExplainWithSort[extended=true]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalSort(sort0=[$0], dir0=[ASC-nulls-first], fetch=[5])
++- LogicalProject(a=[$0], b=[$1], c=[$2])
+   +- LogicalTableScan(table=[[MyTable1]])
+
+== Optimized Logical Plan ==
+SortLimit(orderBy=[a ASC], offset=[0], fetch=[5], global=[true]): rowcount = , cumulative cost = {rows, cpu, io, network, memory}
++- Exchange(distribution=[single]): rowcount = , cumulative cost = {rows, cpu, io, network, memory}
+   +- SortLimit(orderBy=[a ASC], offset=[0], fetch=[5], global=[false]): rowcount = , cumulative cost = {rows, cpu, io, network, memory}
+      +- BoundedStreamScan(table=[[MyTable1]], fields=[a, b, c]): rowcount = , cumulative cost = {rows, cpu, io, network, memory}
+
+== Physical Execution Plan ==
+ : Data Source
+	content : collect elements with CollectionInputFormat
+
+	 : Operator
+		content : SourceConversion(table:Buffer(MyTable1), fields:(a, b, c))
+		ship_strategy : FORWARD
+
+		 : Operator
+			content : LocalSortLimit(orderBy: [a ASC], offset: 0, fetch: 5)
+			ship_strategy : FORWARD
+
+			 : Operator
+				content : GlobalSortLimit(orderBy: [a ASC], offset: 0, fetch: 5)
+				ship_strategy : GLOBAL
+
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testExplainWithTableSourceScan[extended=false]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalProject(a=[$0], b=[$1], c=[$2])
++- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+
+== Optimized Logical Plan ==
+TableSourceScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+
+== Physical Execution Plan ==
+ : Data Source
+	content : collect elements with CollectionInputFormat
+
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testExplainWithTableSourceScan[extended=true]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalProject(a=[$0], b=[$1], c=[$2])
++- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+
+== Optimized Logical Plan ==
+TableSourceScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]): rowcount = , cumulative cost = {rows, cpu, io, network, memory}
+
+== Physical Execution Plan ==
+ : Data Source
+	content : collect elements with CollectionInputFormat
+
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testExplainWithUnion[extended=true]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalUnion(all=[true])
+:- LogicalProject(a=[$0], b=[$1], c=[$2])
+:  +- LogicalTableScan(table=[[MyTable1]])
++- LogicalProject(d=[$0], e=[$1], f=[$2])
+   +- LogicalTableScan(table=[[MyTable2]])
+
+== Optimized Logical Plan ==
+Union(all=[true], union=[a, b, c]): rowcount = , cumulative cost = {rows, cpu, io, network, memory}
+:- BoundedStreamScan(table=[[MyTable1]], fields=[a, b, c]): rowcount = , cumulative cost = {rows, cpu, io, network, memory}
++- BoundedStreamScan(table=[[MyTable2]], fields=[d, e, f]): rowcount = , cumulative cost = {rows, cpu, io, network, memory}
+
+== Physical Execution Plan ==
+ : Data Source
+	content : collect elements with CollectionInputFormat
+
+ : Data Source
+	content : collect elements with CollectionInputFormat
+
+	 : Operator
+		content : SourceConversion(table:Buffer(MyTable1), fields:(a, b, c))
+		ship_strategy : FORWARD
+
+		 : Operator
+			content : SourceConversion(table:Buffer(MyTable2), fields:(d, e, f))
+			ship_strategy : FORWARD
+
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testExplainWithUnion[extended=false]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalUnion(all=[true])
+:- LogicalProject(a=[$0], b=[$1], c=[$2])
+:  +- LogicalTableScan(table=[[MyTable1]])
++- LogicalProject(d=[$0], e=[$1], f=[$2])
+   +- LogicalTableScan(table=[[MyTable2]])
+
+== Optimized Logical Plan ==
+Union(all=[true], union=[a, b, c])
+:- BoundedStreamScan(table=[[MyTable1]], fields=[a, b, c])
++- BoundedStreamScan(table=[[MyTable2]], fields=[d, e, f])
+
+== Physical Execution Plan ==
+ : Data Source
+	content : collect elements with CollectionInputFormat
+
+ : Data Source
+	content : collect elements with CollectionInputFormat
+
+	 : Operator
+		content : SourceConversion(table:Buffer(MyTable1), fields:(a, b, c))
+		ship_strategy : FORWARD
+
+		 : Operator
+			content : SourceConversion(table:Buffer(MyTable2), fields:(d, e, f))
+			ship_strategy : FORWARD
 
 ]]>
     </Resource>
diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/api/stream/ExplainTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/api/stream/ExplainTest.xml
index 45ea294..fc4e019 100644
--- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/api/stream/ExplainTest.xml
+++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/api/stream/ExplainTest.xml
@@ -16,26 +16,556 @@ See the License for the specific language governing permissions and
 limitations under the License.
 -->
 <Root>
-  <TestCase name="testExplainDataStreamScan">
+  <TestCase name="testExplainDataStreamScan[extended=false]">
     <Resource name="explain">
       <![CDATA[== Abstract Syntax Tree ==
 LogicalProject(a=[$0], b=[$1], c=[$2])
-+- LogicalTableScan(table=[[DataStreamTable]])
++- LogicalTableScan(table=[[MyTable1]])
 
 == Optimized Logical Plan ==
-DataStreamScan(table=[[_DataStreamTable_1]], fields=[a, b, c])
+DataStreamScan(table=[[_DataStreamTable_0]], fields=[a, b, c])
+
+== Physical Execution Plan ==
+ : Data Source
+	content : collect elements with CollectionInputFormat
+
+	 : Operator
+		content : SourceConversion(table:Buffer(_DataStreamTable_0), fields:(a, b, c))
+		ship_strategy : FORWARD
 
 ]]>
     </Resource>
   </TestCase>
-  <TestCase name="testExplainTableSourceScan">
+  <TestCase name="testExplainWithAgg[extended=true]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalProject(EXPR$0=[$1])
++- LogicalAggregate(group=[{0}], EXPR$0=[COUNT()])
+   +- LogicalProject(a=[$0])
+      +- LogicalTableScan(table=[[MyTable1]])
+
+== Optimized Logical Plan ==
+Calc(select=[EXPR$0], updateAsRetraction=[false], accMode=[Acc]): rowcount = , cumulative cost = {rows, cpu, io, network, memory}
++- GroupAggregate(groupBy=[a], select=[a, COUNT(*) AS EXPR$0], updateAsRetraction=[false], accMode=[Acc]): rowcount = , cumulative cost = {rows, cpu, io, network, memory}
+   +- Exchange(distribution=[hash[a]], updateAsRetraction=[true], accMode=[Acc]): rowcount = , cumulative cost = {rows, cpu, io, network, memory}
+      +- Calc(select=[a], updateAsRetraction=[true], accMode=[Acc]): rowcount = , cumulative cost = {rows, cpu, io, network, memory}
+         +- DataStreamScan(table=[[_DataStreamTable_0]], fields=[a, b, c], updateAsRetraction=[true], accMode=[Acc]): rowcount = , cumulative cost = {rows, cpu, io, network, memory}
+
+== Physical Execution Plan ==
+ : Data Source
+	content : collect elements with CollectionInputFormat
+
+	 : Operator
+		content : SourceConversion(table:Buffer(_DataStreamTable_0), fields:(a, b, c))
+		ship_strategy : FORWARD
+
+		 : Operator
+			content : Calc(select: (a))
+			ship_strategy : FORWARD
+
+			 : Operator
+				content : GroupAggregate
+				ship_strategy : HASH
+
+				 : Operator
+					content : Calc(select: (EXPR$0))
+					ship_strategy : FORWARD
+
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testExplainDataStreamScan[extended=true]">
     <Resource name="explain">
       <![CDATA[== Abstract Syntax Tree ==
 LogicalProject(a=[$0], b=[$1], c=[$2])
-+- LogicalTableScan(table=[[MyTable]])
++- LogicalTableScan(table=[[MyTable1]])
 
 == Optimized Logical Plan ==
-DataStreamScan(table=[[_DataStreamTable_0]], fields=[a, b, c])
+DataStreamScan(table=[[_DataStreamTable_0]], fields=[a, b, c], updateAsRetraction=[false], accMode=[Acc]): rowcount = , cumulative cost = {rows, cpu, io, network, memory}
+
+== Physical Execution Plan ==
+ : Data Source
+	content : collect elements with CollectionInputFormat
+
+	 : Operator
+		content : SourceConversion(table:Buffer(_DataStreamTable_0), fields:(a, b, c))
+		ship_strategy : FORWARD
+
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testExplainTableSourceScan[extended=false]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalProject(a=[$0], b=[$1], c=[$2])
++- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+
+== Optimized Logical Plan ==
+TableSourceScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+
+== Physical Execution Plan ==
+ : Data Source
+	content : collect elements with CollectionInputFormat
+
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testExplainTableSourceScan[extended=true]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalProject(a=[$0], b=[$1], c=[$2])
++- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+
+== Optimized Logical Plan ==
+TableSourceScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], updateAsRetraction=[false], accMode=[Acc]): rowcount = , cumulative cost = {rows, cpu, io, network, memory}
+
+== Physical Execution Plan ==
+ : Data Source
+	content : collect elements with CollectionInputFormat
+
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testExplainWithAgg[extended=false]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalProject(EXPR$0=[$1])
++- LogicalAggregate(group=[{0}], EXPR$0=[COUNT()])
+   +- LogicalProject(a=[$0])
+      +- LogicalTableScan(table=[[MyTable1]])
+
+== Optimized Logical Plan ==
+Calc(select=[EXPR$0])
++- GroupAggregate(groupBy=[a], select=[a, COUNT(*) AS EXPR$0])
+   +- Exchange(distribution=[hash[a]])
+      +- Calc(select=[a])
+         +- DataStreamScan(table=[[_DataStreamTable_0]], fields=[a, b, c])
+
+== Physical Execution Plan ==
+ : Data Source
+	content : collect elements with CollectionInputFormat
+
+	 : Operator
+		content : SourceConversion(table:Buffer(_DataStreamTable_0), fields:(a, b, c))
+		ship_strategy : FORWARD
+
+		 : Operator
+			content : Calc(select: (a))
+			ship_strategy : FORWARD
+
+			 : Operator
+				content : GroupAggregate
+				ship_strategy : HASH
+
+				 : Operator
+					content : Calc(select: (EXPR$0))
+					ship_strategy : FORWARD
+
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testExplainWithFilter[extended=false]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalProject(a=[$0], b=[$1], c=[$2])
++- LogicalFilter(condition=[=(MOD($0, 2), 0)])
+   +- LogicalTableScan(table=[[MyTable1]])
+
+== Optimized Logical Plan ==
+Calc(select=[a, b, c], where=[=(MOD(a, 2), 0)])
++- DataStreamScan(table=[[_DataStreamTable_0]], fields=[a, b, c])
+
+== Physical Execution Plan ==
+ : Data Source
+	content : collect elements with CollectionInputFormat
+
+	 : Operator
+		content : SourceConversion(table:Buffer(_DataStreamTable_0), fields:(a, b, c))
+		ship_strategy : FORWARD
+
+		 : Operator
+			content : Calc(where: ((a MOD 2) = 0), select: (a, b, c))
+			ship_strategy : FORWARD
+
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testExplainWithFilter[extended=true]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalProject(a=[$0], b=[$1], c=[$2])
++- LogicalFilter(condition=[=(MOD($0, 2), 0)])
+   +- LogicalTableScan(table=[[MyTable1]])
+
+== Optimized Logical Plan ==
+Calc(select=[a, b, c], where=[=(MOD(a, 2), 0)], updateAsRetraction=[false], accMode=[Acc]): rowcount = , cumulative cost = {rows, cpu, io, network, memory}
++- DataStreamScan(table=[[_DataStreamTable_0]], fields=[a, b, c], updateAsRetraction=[false], accMode=[Acc]): rowcount = , cumulative cost = {rows, cpu, io, network, memory}
+
+== Physical Execution Plan ==
+ : Data Source
+	content : collect elements with CollectionInputFormat
+
+	 : Operator
+		content : SourceConversion(table:Buffer(_DataStreamTable_0), fields:(a, b, c))
+		ship_strategy : FORWARD
+
+		 : Operator
+			content : Calc(where: ((a MOD 2) = 0), select: (a, b, c))
+			ship_strategy : FORWARD
+
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testExplainWithMultiSinks[extended=false]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalSink(name=[sink1], fields=[a, cnt])
++- LogicalProject(a=[$0], cnt=[$1])
+   +- LogicalFilter(condition=[>($1, 10)])
+      +- LogicalAggregate(group=[{0}], cnt=[COUNT()])
+         +- LogicalProject(a=[$0])
+            +- LogicalTableScan(table=[[_DataStreamTable_0]])
+
+LogicalSink(name=[sink1], fields=[a, cnt])
++- LogicalProject(a=[$0], cnt=[$1])
+   +- LogicalFilter(condition=[<($1, 10)])
+      +- LogicalAggregate(group=[{0}], cnt=[COUNT()])
+         +- LogicalProject(a=[$0])
+            +- LogicalTableScan(table=[[_DataStreamTable_0]])
+
+== Optimized Logical Plan ==
+GroupAggregate(groupBy=[a], select=[a, COUNT(*) AS cnt], reuse_id=[1])
++- Exchange(distribution=[hash[a]])
+   +- Calc(select=[a])
+      +- DataStreamScan(table=[[_DataStreamTable_0]], fields=[a, b, c])
+
+Sink(name=[sink1], fields=[a, cnt])
++- Calc(select=[a, cnt], where=[>(cnt, 10)])
+   +- Reused(reference_id=[1])
+
+Sink(name=[sink1], fields=[a, cnt])
++- Calc(select=[a, cnt], where=[<(cnt, 10)])
+   +- Reused(reference_id=[1])
+
+== Physical Execution Plan ==
+ : Data Source
+	content : collect elements with CollectionInputFormat
+
+	 : Operator
+		content : SourceConversion(table:Buffer(_DataStreamTable_0), fields:(a, b, c))
+		ship_strategy : FORWARD
+
+		 : Operator
+			content : Calc(select: (a))
+			ship_strategy : FORWARD
+
+			 : Operator
+				content : GroupAggregate
+				ship_strategy : HASH
+
+				 : Operator
+					content : Calc(where: (cnt > 10), select: (a, cnt))
+					ship_strategy : FORWARD
+
+					 : Operator
+						content : SinkConversionToTuple2
+						ship_strategy : FORWARD
+
+						 : Operator
+							content : Map
+							ship_strategy : FORWARD
+
+							 : Operator
+								content : Calc(where: (cnt < 10), select: (a, cnt))
+								ship_strategy : FORWARD
+
+								 : Operator
+									content : SinkConversionToTuple2
+									ship_strategy : FORWARD
+
+									 : Operator
+										content : Map
+										ship_strategy : FORWARD
+
+										 : Data Sink
+											content : Sink: TestingUpsertTableSink(keys=(0))
+											ship_strategy : FORWARD
+
+											 : Data Sink
+												content : Sink: TestingUpsertTableSink(keys=(0))
+												ship_strategy : FORWARD
+
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testExplainWithMultiSinks[extended=true]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalSink(name=[sink1], fields=[a, cnt])
++- LogicalProject(a=[$0], cnt=[$1])
+   +- LogicalFilter(condition=[>($1, 10)])
+      +- LogicalAggregate(group=[{0}], cnt=[COUNT()])
+         +- LogicalProject(a=[$0])
+            +- LogicalTableScan(table=[[_DataStreamTable_0]])
+
+LogicalSink(name=[sink1], fields=[a, cnt])
++- LogicalProject(a=[$0], cnt=[$1])
+   +- LogicalFilter(condition=[<($1, 10)])
+      +- LogicalAggregate(group=[{0}], cnt=[COUNT()])
+         +- LogicalProject(a=[$0])
+            +- LogicalTableScan(table=[[_DataStreamTable_0]])
+
+== Optimized Logical Plan ==
+GroupAggregate(groupBy=[a], select=[a, COUNT(*) AS cnt], updateAsRetraction=[false], accMode=[Acc], reuse_id=[1]): rowcount = , cumulative cost = {rows, cpu, io, network, memory}
++- Exchange(distribution=[hash[a]], updateAsRetraction=[true], accMode=[Acc]): rowcount = , cumulative cost = {rows, cpu, io, network, memory}
+   +- Calc(select=[a], updateAsRetraction=[true], accMode=[Acc]): rowcount = , cumulative cost = {rows, cpu, io, network, memory}
+      +- DataStreamScan(table=[[_DataStreamTable_0]], fields=[a, b, c], updateAsRetraction=[true], accMode=[Acc]): rowcount = , cumulative cost = {rows, cpu, io, network, memory}
+
+Sink(name=[sink1], fields=[a, cnt], updateAsRetraction=[false], accMode=[Acc]): rowcount = , cumulative cost = {rows, cpu, io, network, memory}
++- Calc(select=[a, cnt], where=[>(cnt, 10)], updateAsRetraction=[false], accMode=[Acc]): rowcount = , cumulative cost = {rows, cpu, io, network, memory}
+   +- Reused(reference_id=[1])
+
+Sink(name=[sink1], fields=[a, cnt], updateAsRetraction=[false], accMode=[Acc]): rowcount = , cumulative cost = {rows, cpu, io, network, memory}
++- Calc(select=[a, cnt], where=[<(cnt, 10)], updateAsRetraction=[false], accMode=[Acc]): rowcount = , cumulative cost = {rows, cpu, io, network, memory}
+   +- Reused(reference_id=[1])
+
+== Physical Execution Plan ==
+ : Data Source
+	content : collect elements with CollectionInputFormat
+
+	 : Operator
+		content : SourceConversion(table:Buffer(_DataStreamTable_0), fields:(a, b, c))
+		ship_strategy : FORWARD
+
+		 : Operator
+			content : Calc(select: (a))
+			ship_strategy : FORWARD
+
+			 : Operator
+				content : GroupAggregate
+				ship_strategy : HASH
+
+				 : Operator
+					content : Calc(where: (cnt > 10), select: (a, cnt))
+					ship_strategy : FORWARD
+
+					 : Operator
+						content : SinkConversionToTuple2
+						ship_strategy : FORWARD
+
+						 : Operator
+							content : Map
+							ship_strategy : FORWARD
+
+							 : Operator
+								content : Calc(where: (cnt < 10), select: (a, cnt))
+								ship_strategy : FORWARD
+
+								 : Operator
+									content : SinkConversionToTuple2
+									ship_strategy : FORWARD
+
+									 : Operator
+										content : Map
+										ship_strategy : FORWARD
+
+										 : Data Sink
+											content : Sink: TestingUpsertTableSink(keys=(0))
+											ship_strategy : FORWARD
+
+											 : Data Sink
+												content : Sink: TestingUpsertTableSink(keys=(0))
+												ship_strategy : FORWARD
+
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testExplainWithSingleSink[extended=false]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalSink(fields=[a, b, c])
++- LogicalProject(a=[$0], b=[$1], c=[$2])
+   +- LogicalFilter(condition=[>($0, 10)])
+      +- LogicalTableScan(table=[[MyTable1]])
+
+== Optimized Logical Plan ==
+Sink(fields=[a, b, c])
++- Calc(select=[a, b, c], where=[>(a, 10)])
+   +- DataStreamScan(table=[[_DataStreamTable_0]], fields=[a, b, c])
+
+== Physical Execution Plan ==
+ : Data Source
+	content : collect elements with CollectionInputFormat
+
+	 : Operator
+		content : SourceConversion(table:Buffer(_DataStreamTable_0), fields:(a, b, c))
+		ship_strategy : FORWARD
+
+		 : Operator
+			content : Calc(where: (a > 10), select: (a, b, c))
+			ship_strategy : FORWARD
+
+			 : Operator
+				content : SinkConversionToRow
+				ship_strategy : FORWARD
+
+				 : Data Sink
+					content : Sink: TestingAppendTableSink
+					ship_strategy : FORWARD
+
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testExplainWithSingleSink[extended=true]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalSink(fields=[a, b, c])
++- LogicalProject(a=[$0], b=[$1], c=[$2])
+   +- LogicalFilter(condition=[>($0, 10)])
+      +- LogicalTableScan(table=[[MyTable1]])
+
+== Optimized Logical Plan ==
+Sink(fields=[a, b, c], updateAsRetraction=[false], accMode=[Acc]): rowcount = , cumulative cost = {rows, cpu, io, network, memory}
++- Calc(select=[a, b, c], where=[>(a, 10)], updateAsRetraction=[false], accMode=[Acc]): rowcount = , cumulative cost = {rows, cpu, io, network, memory}
+   +- DataStreamScan(table=[[_DataStreamTable_0]], fields=[a, b, c], updateAsRetraction=[false], accMode=[Acc]): rowcount = , cumulative cost = {rows, cpu, io, network, memory}
+
+== Physical Execution Plan ==
+ : Data Source
+	content : collect elements with CollectionInputFormat
+
+	 : Operator
+		content : SourceConversion(table:Buffer(_DataStreamTable_0), fields:(a, b, c))
+		ship_strategy : FORWARD
+
+		 : Operator
+			content : Calc(where: (a > 10), select: (a, b, c))
+			ship_strategy : FORWARD
+
+			 : Operator
+				content : SinkConversionToRow
+				ship_strategy : FORWARD
+
+				 : Data Sink
+					content : Sink: TestingAppendTableSink
+					ship_strategy : FORWARD
+
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testExplainWithSort[extended=false]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalSort(sort0=[$0], dir0=[ASC-nulls-first], fetch=[5])
++- LogicalProject(a=[$0], b=[$1], c=[$2])
+   +- LogicalTableScan(table=[[MyTable1]])
+
+== Optimized Logical Plan ==
+SortLimit(orderBy=[a ASC], offset=[0], fetch=[5])
++- Exchange(distribution=[single])
+   +- DataStreamScan(table=[[_DataStreamTable_0]], fields=[a, b, c])
+
+== Physical Execution Plan ==
+ : Data Source
+	content : collect elements with CollectionInputFormat
+
+	 : Operator
+		content : SourceConversion(table:Buffer(_DataStreamTable_0), fields:(a, b, c))
+		ship_strategy : FORWARD
+
+		 : Operator
+			content : SortLimit(orderBy: [a ASC], offset: 0, fetch: 5)
+			ship_strategy : GLOBAL
+
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testExplainWithUnion[extended=true]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalUnion(all=[true])
+:- LogicalProject(a=[$0], b=[$1], c=[$2])
+:  +- LogicalTableScan(table=[[MyTable1]])
++- LogicalProject(d=[$0], e=[$1], f=[$2])
+   +- LogicalTableScan(table=[[MyTable2]])
+
+== Optimized Logical Plan ==
+Union(all=[true], union=[a, b, c], updateAsRetraction=[false], accMode=[Acc]): rowcount = , cumulative cost = {rows, cpu, io, network, memory}
+:- DataStreamScan(table=[[_DataStreamTable_0]], fields=[a, b, c], updateAsRetraction=[false], accMode=[Acc]): rowcount = , cumulative cost = {rows, cpu, io, network, memory}
++- DataStreamScan(table=[[_DataStreamTable_1]], fields=[d, e, f], updateAsRetraction=[false], accMode=[Acc]): rowcount = , cumulative cost = {rows, cpu, io, network, memory}
+
+== Physical Execution Plan ==
+ : Data Source
+	content : collect elements with CollectionInputFormat
+
+ : Data Source
+	content : collect elements with CollectionInputFormat
+
+	 : Operator
+		content : SourceConversion(table:Buffer(_DataStreamTable_0), fields:(a, b, c))
+		ship_strategy : FORWARD
+
+		 : Operator
+			content : SourceConversion(table:Buffer(_DataStreamTable_1), fields:(d, e, f))
+			ship_strategy : FORWARD
+
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testExplainWithSort[extended=true]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalSort(sort0=[$0], dir0=[ASC-nulls-first], fetch=[5])
++- LogicalProject(a=[$0], b=[$1], c=[$2])
+   +- LogicalTableScan(table=[[MyTable1]])
+
+== Optimized Logical Plan ==
+SortLimit(orderBy=[a ASC], offset=[0], fetch=[5], updateAsRetraction=[false], accMode=[Acc]): rowcount = , cumulative cost = {rows, cpu, io, network, memory}
++- Exchange(distribution=[single], updateAsRetraction=[false], accMode=[Acc]): rowcount = , cumulative cost = {rows, cpu, io, network, memory}
+   +- DataStreamScan(table=[[_DataStreamTable_0]], fields=[a, b, c], updateAsRetraction=[false], accMode=[Acc]): rowcount = , cumulative cost = {rows, cpu, io, network, memory}
+
+== Physical Execution Plan ==
+ : Data Source
+	content : collect elements with CollectionInputFormat
+
+	 : Operator
+		content : SourceConversion(table:Buffer(_DataStreamTable_0), fields:(a, b, c))
+		ship_strategy : FORWARD
+
+		 : Operator
+			content : SortLimit(orderBy: [a ASC], offset: 0, fetch: 5)
+			ship_strategy : GLOBAL
+
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testExplainWithUnion[extended=false]">
+    <Resource name="explain">
+      <![CDATA[== Abstract Syntax Tree ==
+LogicalUnion(all=[true])
+:- LogicalProject(a=[$0], b=[$1], c=[$2])
+:  +- LogicalTableScan(table=[[MyTable1]])
++- LogicalProject(d=[$0], e=[$1], f=[$2])
+   +- LogicalTableScan(table=[[MyTable2]])
+
+== Optimized Logical Plan ==
+Union(all=[true], union=[a, b, c])
+:- DataStreamScan(table=[[_DataStreamTable_0]], fields=[a, b, c])
++- DataStreamScan(table=[[_DataStreamTable_1]], fields=[d, e, f])
+
+== Physical Execution Plan ==
+ : Data Source
+	content : collect elements with CollectionInputFormat
+
+ : Data Source
+	content : collect elements with CollectionInputFormat
+
+	 : Operator
+		content : SourceConversion(table:Buffer(_DataStreamTable_0), fields:(a, b, c))
+		ship_strategy : FORWARD
+
+		 : Operator
+			content : SourceConversion(table:Buffer(_DataStreamTable_1), fields:(d, e, f))
+			ship_strategy : FORWARD
 
 ]]>
     </Resource>
diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/DagOptimizationTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/DagOptimizationTest.xml
new file mode 100644
index 0000000..f4c3e12
--- /dev/null
+++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/DagOptimizationTest.xml
@@ -0,0 +1,863 @@
+<?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="testMultiLevelViews">
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalSink(fields=[a, b, c])
++- LogicalProject(a=[$0], b=[$1], c=[$2])
+   +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'%hello%')])
+      +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+
+LogicalSink(fields=[b, cnt])
++- LogicalProject(b=[$0], cnt=[$1])
+   +- LogicalFilter(condition=[<($0, 4)])
+      +- LogicalAggregate(group=[{0}], cnt=[COUNT($1)])
+         +- LogicalProject(b=[$1], a=[$0])
+            +- LogicalUnion(all=[true])
+               :- LogicalProject(a=[$0], b=[$1], c=[$2])
+               :  +- LogicalProject(a=[$0], b=[$1], c=[$2])
+               :     +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'%hello%')])
+               :        +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+               +- LogicalProject(a=[$0], b=[$1], c=[$2])
+                  +- LogicalProject(a=[$0], b=[$1], c=[$2])
+                     +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'%world%')])
+                        +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+
+LogicalSink(fields=[b, cnt])
++- LogicalProject(b=[$0], cnt=[$1])
+   +- LogicalFilter(condition=[AND(>=($0, 4), <($0, 6))])
+      +- LogicalAggregate(group=[{0}], cnt=[COUNT($1)])
+         +- LogicalProject(b=[$1], a=[$0])
+            +- LogicalUnion(all=[true])
+               :- LogicalProject(a=[$0], b=[$1], c=[$2])
+               :  +- LogicalProject(a=[$0], b=[$1], c=[$2])
+               :     +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'%hello%')])
+               :        +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+               +- LogicalProject(a=[$0], b=[$1], c=[$2])
+                  +- LogicalProject(a=[$0], b=[$1], c=[$2])
+                     +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'%world%')])
+                        +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+TableSourceScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], reuse_id=[2])
+
+Calc(select=[a, b, c], where=[LIKE(c, _UTF-16LE'%hello%')], reuse_id=[1])
++- Reused(reference_id=[2])
+
+Sink(fields=[a, b, c])
++- Reused(reference_id=[1])
+
+HashAggregate(isMerge=[true], groupBy=[b], select=[b, Final_COUNT(count$0) AS cnt], reuse_id=[3])
++- Exchange(distribution=[hash[b]])
+   +- LocalHashAggregate(groupBy=[b], select=[b, Partial_COUNT(a) AS count$0])
+      +- Union(all=[true], union=[b, a])
+         :- Calc(select=[b, a])
+         :  +- Reused(reference_id=[1])
+         +- Calc(select=[b, a], where=[LIKE(c, _UTF-16LE'%world%')])
+            +- Reused(reference_id=[2])
+
+Sink(fields=[b, cnt])
++- Calc(select=[b, cnt], where=[<(b, 4)])
+   +- Reused(reference_id=[3])
+
+Sink(fields=[b, cnt])
++- Calc(select=[b, cnt], where=[AND(>=(b, 4), <(b, 6))])
+   +- Reused(reference_id=[3])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testMultiSinks1">
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalSink(fields=[total_sum])
++- LogicalAggregate(group=[{}], total_sum=[SUM($0)])
+   +- LogicalProject(sum_a=[$0])
+      +- LogicalProject(sum_a=[$1], c=[$0])
+         +- LogicalAggregate(group=[{0}], sum_a=[SUM($1)])
+            +- LogicalProject(c=[$2], a=[$0])
+               +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+
+LogicalSink(fields=[total_min])
++- LogicalAggregate(group=[{}], total_min=[MIN($0)])
+   +- LogicalProject(sum_a=[$0])
+      +- LogicalProject(sum_a=[$1], c=[$0])
+         +- LogicalAggregate(group=[{0}], sum_a=[SUM($1)])
+            +- LogicalProject(c=[$2], a=[$0])
+               +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+Calc(select=[sum_a], reuse_id=[1])
++- HashAggregate(isMerge=[true], groupBy=[c], select=[c, Final_SUM(sum$0) AS sum_a])
+   +- Exchange(distribution=[hash[c]])
+      +- LocalHashAggregate(groupBy=[c], select=[c, Partial_SUM(a) AS sum$0])
+         +- Calc(select=[c, a])
+            +- TableSourceScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+
+Sink(fields=[total_sum])
++- HashAggregate(isMerge=[true], select=[Final_SUM(sum$0) AS total_sum])
+   +- Exchange(distribution=[single])
+      +- LocalHashAggregate(select=[Partial_SUM(sum_a) AS sum$0])
+         +- Reused(reference_id=[1])
+
+Sink(fields=[total_min])
++- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS total_min])
+   +- Exchange(distribution=[single])
+      +- LocalHashAggregate(select=[Partial_MIN(sum_a) AS min$0])
+         +- Reused(reference_id=[1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testMultiSinks2">
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalSink(fields=[a, b1])
++- LogicalUnion(all=[true])
+   :- LogicalProject(a1=[$0], b1=[$1])
+   :  +- LogicalProject(a1=[$0], b1=[$1])
+   :     +- LogicalFilter(condition=[<=($0, 10)])
+   :        +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+   +- LogicalProject(a=[$0], b1=[$1])
+      +- LogicalProject(a=[$2], b1=[$1])
+         +- LogicalFilter(condition=[=($2, $0)])
+            +- LogicalJoin(condition=[true], joinType=[inner])
+               :- LogicalProject(a1=[$0], b1=[$1])
+               :  +- LogicalFilter(condition=[<=($0, 10)])
+               :     +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+               +- LogicalTableScan(table=[[MyTable2, source: [TestTableSource(a, b, c, d, e)]]])
+
+LogicalSink(fields=[a, b1])
++- LogicalUnion(all=[true])
+   :- LogicalProject(a1=[$0], b1=[$1])
+   :  +- LogicalProject(a1=[$0], b1=[$1])
+   :     +- LogicalFilter(condition=[<=($0, 10)])
+   :        +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+   +- LogicalProject(a=[$0], b1=[$1])
+      +- LogicalProject(a=[$2], b1=[$1])
+         +- LogicalFilter(condition=[=($2, $0)])
+            +- LogicalJoin(condition=[true], joinType=[inner])
+               :- LogicalProject(a1=[$0], b1=[$1])
+               :  +- LogicalFilter(condition=[<=($0, 10)])
+               :     +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+               +- LogicalTableScan(table=[[MyTable2, source: [TestTableSource(a, b, c, d, e)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+Calc(select=[a, b], where=[<=(a, 10)], reuse_id=[1])
++- TableSourceScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+
+Union(all=[true], union=[a, b], reuse_id=[2])
+:- Reused(reference_id=[1])
++- Calc(select=[a0 AS a, b AS b1])
+   +- HashJoin(joinType=[InnerJoin], where=[=(a0, a)], select=[a, b, a0], build=[left])
+      :- Exchange(distribution=[hash[a]])
+      :  +- Reused(reference_id=[1])
+      +- Exchange(distribution=[hash[a]])
+         +- Calc(select=[a])
+            +- TableSourceScan(table=[[MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e])
+
+Sink(fields=[a, b1])
++- Reused(reference_id=[2])
+
+Sink(fields=[a, b1])
++- Reused(reference_id=[2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testMultiSinks3">
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalSink(fields=[a, b1])
++- LogicalProject(a=[$2], b1=[$1])
+   +- LogicalFilter(condition=[=($2, $0)])
+      +- LogicalJoin(condition=[true], joinType=[inner])
+         :- LogicalProject(a1=[$0], b1=[$1])
+         :  +- LogicalFilter(condition=[<=($0, 10)])
+         :     +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+         +- LogicalTableScan(table=[[MyTable2, source: [TestTableSource(a, b, c, d, e)]]])
+
+LogicalSink(fields=[a, b1])
++- LogicalUnion(all=[true])
+   :- LogicalProject(a1=[$0], b1=[$1])
+   :  +- LogicalProject(a1=[$0], b1=[$1])
+   :     +- LogicalFilter(condition=[<=($0, 10)])
+   :        +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+   +- LogicalProject(a=[$0], b1=[$1])
+      +- LogicalProject(a=[$2], b1=[$1])
+         +- LogicalFilter(condition=[=($2, $0)])
+            +- LogicalJoin(condition=[true], joinType=[inner])
+               :- LogicalProject(a1=[$0], b1=[$1])
+               :  +- LogicalFilter(condition=[<=($0, 10)])
+               :     +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+               +- LogicalTableScan(table=[[MyTable2, source: [TestTableSource(a, b, c, d, e)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+Calc(select=[a, b], where=[<=(a, 10)], reuse_id=[1])
++- TableSourceScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+
+Calc(select=[a0 AS a, b AS b1], reuse_id=[2])
++- HashJoin(joinType=[InnerJoin], where=[=(a0, a)], select=[a, b, a0], build=[right])
+   :- Exchange(distribution=[hash[a]])
+   :  +- Reused(reference_id=[1])
+   +- Exchange(distribution=[hash[a]])
+      +- Calc(select=[a])
+         +- TableSourceScan(table=[[MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e])
+
+Sink(fields=[a, b1])
++- Reused(reference_id=[2])
+
+Sink(fields=[a, b1])
++- Union(all=[true], union=[a, b])
+   :- Reused(reference_id=[1])
+   +- Reused(reference_id=[2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testMultiSinks4">
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalSink(fields=[a1, b, c2])
++- LogicalProject(a1=[$0], b=[$1], c2=[$3])
+   +- LogicalFilter(condition=[=($0, $2)])
+      +- LogicalJoin(condition=[true], joinType=[inner])
+         :- LogicalProject(a1=[$0], b=[$1])
+         :  +- LogicalFilter(condition=[<=($0, 10)])
+         :     +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+         +- LogicalProject(a2=[$0], c=[$2])
+            +- LogicalFilter(condition=[>=($1, 5)])
+               +- LogicalProject(a=[$0], b=[$1], c=[$2])
+                  +- LogicalFilter(condition=[>=($0, 0)])
+                     +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+
+LogicalSink(fields=[a1, b, c1])
++- LogicalProject(a1=[$2], b=[$3], c1=[$1])
+   +- LogicalFilter(condition=[=($2, $0)])
+      +- LogicalJoin(condition=[true], joinType=[inner])
+         :- LogicalProject(a3=[$0], c1=[$2])
+         :  +- LogicalFilter(condition=[<($1, 5)])
+         :     +- LogicalProject(a=[$0], b=[$1], c=[$2])
+         :        +- LogicalFilter(condition=[>=($0, 0)])
+         :           +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+         +- LogicalProject(a1=[$0], b=[$1], c2=[$3])
+            +- LogicalFilter(condition=[=($0, $2)])
+               +- LogicalJoin(condition=[true], joinType=[inner])
+                  :- LogicalProject(a1=[$0], b=[$1])
+                  :  +- LogicalFilter(condition=[<=($0, 10)])
+                  :     +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+                  +- LogicalProject(a2=[$0], c=[$2])
+                     +- LogicalFilter(condition=[>=($1, 5)])
+                        +- LogicalProject(a=[$0], b=[$1], c=[$2])
+                           +- LogicalFilter(condition=[>=($0, 0)])
+                              +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+TableSourceScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], reuse_id=[1])
+
+Calc(select=[a, b, c], where=[>=(a, 0)], reuse_id=[2])
++- Reused(reference_id=[1])
+
+Calc(select=[a AS a1, b, c AS c2], reuse_id=[3])
++- HashJoin(joinType=[InnerJoin], where=[=(a, a2)], select=[a, b, a2, c], build=[left])
+   :- Exchange(distribution=[hash[a]])
+   :  +- Calc(select=[a, b], where=[<=(a, 10)])
+   :     +- Reused(reference_id=[1])
+   +- Exchange(distribution=[hash[a2]], exchange_mode=[BATCH])
+      +- Calc(select=[a AS a2, c], where=[>=(b, 5)])
+         +- Reused(reference_id=[2])
+
+Sink(fields=[a1, b, c2])
++- Reused(reference_id=[3])
+
+Sink(fields=[a1, b, c1])
++- Calc(select=[a1, b, c1])
+   +- HashJoin(joinType=[InnerJoin], where=[=(a1, a3)], select=[a3, c1, a1, b], build=[left])
+      :- Exchange(distribution=[hash[a3]])
+      :  +- Calc(select=[a AS a3, c AS c1], where=[<(b, 5)])
+      :     +- Reused(reference_id=[2])
+      +- Exchange(distribution=[hash[a1]], exchange_mode=[BATCH])
+         +- Calc(select=[a1, b])
+            +- Reused(reference_id=[3])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testMultiSinks5">
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalSink(fields=[total_sum])
++- LogicalAggregate(group=[{}], total_sum=[SUM($0)])
+   +- LogicalProject(a=[$0])
+      +- LogicalProject(a=[random_udf($0)], c=[$2])
+         +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+
+LogicalSink(fields=[total_min])
++- LogicalAggregate(group=[{}], total_min=[MIN($0)])
+   +- LogicalProject(a=[$0])
+      +- LogicalProject(a=[random_udf($0)], c=[$2])
+         +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+Calc(select=[random_udf(a) AS a], reuse_id=[1])
++- TableSourceScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+
+Sink(fields=[total_sum])
++- HashAggregate(isMerge=[true], select=[Final_SUM(sum$0) AS total_sum])
+   +- Exchange(distribution=[single])
+      +- LocalHashAggregate(select=[Partial_SUM(a) AS sum$0])
+         +- Reused(reference_id=[1])
+
+Sink(fields=[total_min])
++- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS total_min])
+   +- Exchange(distribution=[single])
+      +- LocalHashAggregate(select=[Partial_MIN(a) AS min$0])
+         +- Reused(reference_id=[1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testMultiSinksSplitOnUnion1">
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalSink(fields=[total_sum])
++- LogicalAggregate(group=[{}], total_sum=[SUM($0)])
+   +- LogicalProject(a=[$0])
+      +- LogicalUnion(all=[true])
+         :- LogicalProject(a=[$0], c=[$2])
+         :  +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+         +- LogicalProject(d=[$0], f=[$2])
+            +- LogicalTableScan(table=[[MyTable1, source: [TestTableSource(d, e, f)]]])
+
+LogicalSink(fields=[total_min])
++- LogicalAggregate(group=[{}], total_min=[MIN($0)])
+   +- LogicalProject(a=[$0])
+      +- LogicalUnion(all=[true])
+         :- LogicalProject(a=[$0], c=[$2])
+         :  +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+         +- LogicalProject(d=[$0], f=[$2])
+            +- LogicalTableScan(table=[[MyTable1, source: [TestTableSource(d, e, f)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+Union(all=[true], union=[a], reuse_id=[1])
+:- Calc(select=[a])
+:  +- Calc(select=[a, c])
+:     +- TableSourceScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
++- Calc(select=[d AS a])
+   +- Calc(select=[d, f])
+      +- TableSourceScan(table=[[MyTable1, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+
+Sink(fields=[total_sum])
++- HashAggregate(isMerge=[true], select=[Final_SUM(sum$0) AS total_sum])
+   +- Exchange(distribution=[single])
+      +- LocalHashAggregate(select=[Partial_SUM(a) AS sum$0])
+         +- Reused(reference_id=[1])
+
+Sink(fields=[total_min])
++- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS total_min])
+   +- Exchange(distribution=[single])
+      +- LocalHashAggregate(select=[Partial_MIN(a) AS min$0])
+         +- Reused(reference_id=[1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testMultiSinksSplitOnUnion2">
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalSink(fields=[total_sum])
++- LogicalAggregate(group=[{}], total_sum=[SUM($0)])
+   +- LogicalProject(a=[$0])
+      +- LogicalUnion(all=[true])
+         :- LogicalUnion(all=[true])
+         :  :- LogicalProject(a=[$0], c=[$2])
+         :  :  +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+         :  +- LogicalProject(d=[$0], f=[$2])
+         :     +- LogicalTableScan(table=[[MyTable1, source: [TestTableSource(d, e, f)]]])
+         +- LogicalProject(a=[$0], c=[$2])
+            +- LogicalTableScan(table=[[MyTable2, source: [TestTableSource(a, b, c)]]])
+
+LogicalSink(fields=[total_min])
++- LogicalAggregate(group=[{}], total_min=[MIN($0)])
+   +- LogicalProject(a=[$0])
+      +- LogicalUnion(all=[true])
+         :- LogicalUnion(all=[true])
+         :  :- LogicalProject(a=[$0], c=[$2])
+         :  :  +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+         :  +- LogicalProject(d=[$0], f=[$2])
+         :     +- LogicalTableScan(table=[[MyTable1, source: [TestTableSource(d, e, f)]]])
+         +- LogicalProject(a=[$0], c=[$2])
+            +- LogicalTableScan(table=[[MyTable2, source: [TestTableSource(a, b, c)]]])
+
+LogicalSink(fields=[a])
++- LogicalProject(a=[$0])
+   +- LogicalUnion(all=[true])
+      :- LogicalProject(a=[$0], c=[$2])
+      :  +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+      +- LogicalProject(d=[$0], f=[$2])
+         +- LogicalTableScan(table=[[MyTable1, source: [TestTableSource(d, e, f)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+Union(all=[true], union=[a], reuse_id=[2])
+:- Calc(select=[a])
+:  +- Calc(select=[a, c])
+:     +- TableSourceScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
++- Calc(select=[d AS a])
+   +- Calc(select=[d, f])
+      +- TableSourceScan(table=[[MyTable1, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+
+Union(all=[true], union=[a], reuse_id=[1])
+:- Reused(reference_id=[2])
++- Calc(select=[a])
+   +- TableSourceScan(table=[[MyTable2, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+
+Sink(fields=[total_sum])
++- HashAggregate(isMerge=[true], select=[Final_SUM(sum$0) AS total_sum])
+   +- Exchange(distribution=[single])
+      +- LocalHashAggregate(select=[Partial_SUM(a) AS sum$0])
+         +- Reused(reference_id=[1])
+
+Sink(fields=[total_min])
++- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS total_min])
+   +- Exchange(distribution=[single])
+      +- LocalHashAggregate(select=[Partial_MIN(a) AS min$0])
+         +- Reused(reference_id=[1])
+
+Sink(fields=[a])
++- Reused(reference_id=[2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testMultiSinksSplitOnUnion3">
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalSink(fields=[a])
++- LogicalUnion(all=[true])
+   :- LogicalProject(a=[$0], c=[$2])
+   :  +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+   +- LogicalProject(d=[$0], f=[$2])
+      +- LogicalTableScan(table=[[MyTable1, source: [TestTableSource(d, e, f)]]])
+
+LogicalSink(fields=[total_sum])
++- LogicalAggregate(group=[{}], total_sum=[SUM($0)])
+   +- LogicalProject(a=[$0])
+      +- LogicalUnion(all=[true])
+         :- LogicalProject(a=[$0], c=[$1])
+         :  +- LogicalUnion(all=[true])
+         :     :- LogicalProject(a=[$0], c=[$2])
+         :     :  +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+         :     +- LogicalProject(d=[$0], f=[$2])
+         :        +- LogicalTableScan(table=[[MyTable1, source: [TestTableSource(d, e, f)]]])
+         +- LogicalProject(a=[$0], c=[$2])
+            +- LogicalTableScan(table=[[MyTable2, source: [TestTableSource(a, b, c)]]])
+
+LogicalSink(fields=[total_min])
++- LogicalAggregate(group=[{}], total_min=[MIN($0)])
+   +- LogicalProject(a=[$0])
+      +- LogicalUnion(all=[true])
+         :- LogicalProject(a=[$0], c=[$1])
+         :  +- LogicalUnion(all=[true])
+         :     :- LogicalProject(a=[$0], c=[$2])
+         :     :  +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+         :     +- LogicalProject(d=[$0], f=[$2])
+         :        +- LogicalTableScan(table=[[MyTable1, source: [TestTableSource(d, e, f)]]])
+         +- LogicalProject(a=[$0], c=[$2])
+            +- LogicalTableScan(table=[[MyTable2, source: [TestTableSource(a, b, c)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+Union(all=[true], union=[a, c], reuse_id=[1])
+:- Calc(select=[a, c])
+:  +- TableSourceScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
++- Calc(select=[d, f])
+   +- TableSourceScan(table=[[MyTable1, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+
+Sink(fields=[a])
++- Reused(reference_id=[1])
+
+Union(all=[true], union=[a], reuse_id=[2])
+:- Calc(select=[a])
+:  +- Reused(reference_id=[1])
++- Calc(select=[a])
+   +- Calc(select=[a, c])
+      +- TableSourceScan(table=[[MyTable2, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+
+Sink(fields=[total_sum])
++- HashAggregate(isMerge=[true], select=[Final_SUM(sum$0) AS total_sum])
+   +- Exchange(distribution=[single])
+      +- LocalHashAggregate(select=[Partial_SUM(a) AS sum$0])
+         +- Reused(reference_id=[2])
+
+Sink(fields=[total_min])
++- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS total_min])
+   +- Exchange(distribution=[single])
+      +- LocalHashAggregate(select=[Partial_MIN(a) AS min$0])
+         +- Reused(reference_id=[2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testMultiSinksSplitOnUnion4">
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalSink(fields=[total_sum])
++- LogicalAggregate(group=[{}], total_sum=[SUM($0)])
+   +- LogicalProject(a=[$0])
+      +- LogicalUnion(all=[true])
+         :- LogicalUnion(all=[true])
+         :  :- LogicalProject(a=[$0], c=[$2])
+         :  :  +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+         :  +- LogicalProject(d=[$0], f=[$2])
+         :     +- LogicalTableScan(table=[[MyTable1, source: [TestTableSource(d, e, f)]]])
+         +- LogicalProject(a=[$0], c=[$2])
+            +- LogicalTableScan(table=[[MyTable2, source: [TestTableSource(a, b, c)]]])
+
+LogicalSink(fields=[total_min])
++- LogicalAggregate(group=[{}], total_min=[MIN($0)])
+   +- LogicalProject(a=[$0])
+      +- LogicalUnion(all=[true])
+         :- LogicalUnion(all=[true])
+         :  :- LogicalProject(a=[$0], c=[$2])
+         :  :  +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+         :  +- LogicalProject(d=[$0], f=[$2])
+         :     +- LogicalTableScan(table=[[MyTable1, source: [TestTableSource(d, e, f)]]])
+         +- LogicalProject(a=[$0], c=[$2])
+            +- LogicalTableScan(table=[[MyTable2, source: [TestTableSource(a, b, c)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+Union(all=[true], union=[a], reuse_id=[1])
+:- Union(all=[true], union=[a])
+:  :- Calc(select=[a])
+:  :  +- Calc(select=[a, c])
+:  :     +- TableSourceScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+:  +- Calc(select=[d AS a])
+:     +- Calc(select=[d, f])
+:        +- TableSourceScan(table=[[MyTable1, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
++- Calc(select=[a])
+   +- Calc(select=[a, c])
+      +- TableSourceScan(table=[[MyTable2, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+
+Sink(fields=[total_sum])
++- HashAggregate(isMerge=[true], select=[Final_SUM(sum$0) AS total_sum])
+   +- Exchange(distribution=[single])
+      +- LocalHashAggregate(select=[Partial_SUM(a) AS sum$0])
+         +- Reused(reference_id=[1])
+
+Sink(fields=[total_min])
++- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS total_min])
+   +- Exchange(distribution=[single])
+      +- LocalHashAggregate(select=[Partial_MIN(a) AS min$0])
+         +- Reused(reference_id=[1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testMultiSinksWithWindow">
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalSink(fields=[a, sum_c, time, window_start, window_end])
++- LogicalProject(a=[$1], sum_c=[$2], time=[TUMBLE_END($0)], window_start=[TUMBLE_START($0)], window_end=[TUMBLE_END($0)])
+   +- LogicalAggregate(group=[{0, 1}], sum_c=[SUM($2)])
+      +- LogicalProject($f0=[TUMBLE($3, 15000:INTERVAL SECOND)], a=[$0], $f2=[CAST($2):DOUBLE])
+         +- LogicalTableScan(table=[[MyTable2, source: [TestTableSource(a, b, c, ts)]]])
+
+LogicalSink(fields=[a, sum_c, time])
++- LogicalProject(a=[$1], sum_c=[$2], time=[TUMBLE_END($0)])
+   +- LogicalAggregate(group=[{0, 1}], sum_c=[SUM($2)])
+      +- LogicalProject($f0=[TUMBLE($3, 15000:INTERVAL SECOND)], a=[$0], $f2=[CAST($2):DOUBLE])
+         +- LogicalTableScan(table=[[MyTable2, source: [TestTableSource(a, b, c, ts)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+HashWindowAggregate(groupBy=[a], window=[TumblingGroupWindow], properties=[w$start, w$end, w$rowtime], select=[a, SUM(c) AS sum_c], reuse_id=[1])
++- Exchange(distribution=[hash[a]])
+   +- Calc(select=[ts, a, CAST(c) AS c])
+      +- TableSourceScan(table=[[MyTable2, source: [TestTableSource(a, b, c, ts)]]], fields=[a, b, c, ts])
+
+Sink(fields=[a, sum_c, time, window_start, window_end])
++- Calc(select=[a, sum_c, w$end AS time, w$start AS window_start, w$end AS window_end])
+   +- Reused(reference_id=[1])
+
+Sink(fields=[a, sum_c, time])
++- Calc(select=[a, sum_c, w$end AS time])
+   +- Reused(reference_id=[1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testMultiSinksWithUDTF">
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalSink(fields=[a, total_c])
++- LogicalProject(a=[$0], total_c=[$1])
+   +- LogicalFilter(condition=[>($0, 50)])
+      +- LogicalUnion(all=[true])
+         :- LogicalProject(a=[$0], total_c=[$2])
+         :  +- LogicalAggregate(group=[{0, 1}], total_c=[COUNT(DISTINCT $2)])
+         :     +- LogicalProject(a=[$0], b=[$1], c=[$3])
+         :        +- LogicalFilter(condition=[<>($2, _UTF-16LE'')])
+         :           +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{2}])
+         :              :- LogicalProject(a=[$0], b=[-($1, MOD($1, 300))], c=[$2])
+         :              :  +- LogicalFilter(condition=[>=($1, UNIX_TIMESTAMP(_UTF-16LE'${startTime}'))])
+         :              :     +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+         :              +- LogicalTableFunctionScan(invocation=[split($cor0.c)], rowType=[RecordType(VARCHAR(65536) f0)], elementType=[class [Ljava.lang.Object;])
+         +- LogicalProject(a=[$0], total_c=[0])
+            +- LogicalProject(a=[$0], b=[-($1, MOD($1, 300))], c=[$2])
+               +- LogicalFilter(condition=[>=($1, UNIX_TIMESTAMP(_UTF-16LE'${startTime}'))])
+                  +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+
+LogicalSink(fields=[a, total_c])
++- LogicalProject(a=[$0], total_c=[$1])
+   +- LogicalFilter(condition=[<($0, 50)])
+      +- LogicalUnion(all=[true])
+         :- LogicalProject(a=[$0], total_c=[$2])
+         :  +- LogicalAggregate(group=[{0, 1}], total_c=[COUNT(DISTINCT $2)])
+         :     +- LogicalProject(a=[$0], b=[$1], c=[$3])
+         :        +- LogicalFilter(condition=[<>($2, _UTF-16LE'')])
+         :           +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{2}])
+         :              :- LogicalProject(a=[$0], b=[-($1, MOD($1, 300))], c=[$2])
+         :              :  +- LogicalFilter(condition=[>=($1, UNIX_TIMESTAMP(_UTF-16LE'${startTime}'))])
+         :              :     +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+         :              +- LogicalTableFunctionScan(invocation=[split($cor0.c)], rowType=[RecordType(VARCHAR(65536) f0)], elementType=[class [Ljava.lang.Object;])
+         +- LogicalProject(a=[$0], total_c=[0])
+            +- LogicalProject(a=[$0], b=[-($1, MOD($1, 300))], c=[$2])
+               +- LogicalFilter(condition=[>=($1, UNIX_TIMESTAMP(_UTF-16LE'${startTime}'))])
+                  +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+TableSourceScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], reuse_id=[1])
+
+Calc(select=[a, total_c], reuse_id=[2])
++- HashAggregate(isMerge=[true], groupBy=[a, b], select=[a, b, Final_COUNT(count$0) AS total_c])
+   +- Exchange(distribution=[hash[a, b]])
+      +- LocalHashAggregate(groupBy=[a, b], select=[a, b, Partial_COUNT(c) AS count$0])
+         +- HashAggregate(isMerge=[true], groupBy=[a, b, c], select=[a, b, c])
+            +- Exchange(distribution=[hash[a, b, c]])
+               +- LocalHashAggregate(groupBy=[a, b, c], select=[a, b, c])
+                  +- Calc(select=[a, b, f0 AS c])
+                     +- Correlate(invocation=[split($cor0.c)], correlate=[table(split($cor0.c))], select=[a,b,c,f0], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(65536) c, VARCHAR(65536) f0)], joinType=[INNER])
+                        +- Calc(select=[a, -(b, MOD(b, 300)) AS b, c], where=[AND(>=(b, UNIX_TIMESTAMP(_UTF-16LE'${startTime}')), <>(c, _UTF-16LE'':VARCHAR(65536) CHARACTER SET "UTF-16LE"))])
+                           +- Reused(reference_id=[1])
+
+Calc(select=[a, 0 AS total_c], where=[>=(b, UNIX_TIMESTAMP(_UTF-16LE'${startTime}'))], reuse_id=[3])
++- Reused(reference_id=[1])
+
+Sink(fields=[a, total_c])
++- Union(all=[true], union=[a, total_c])
+   :- Calc(select=[a, total_c], where=[>(a, 50)])
+   :  +- Reused(reference_id=[2])
+   +- Calc(select=[a, CAST(total_c) AS total_c], where=[>(a, 50)])
+      +- Reused(reference_id=[3])
+
+Sink(fields=[a, total_c])
++- Union(all=[true], union=[a, total_c])
+   :- Calc(select=[a, total_c], where=[<(a, 50)])
+   :  +- Reused(reference_id=[2])
+   +- Calc(select=[a, CAST(total_c) AS total_c], where=[<(a, 50)])
+      +- Reused(reference_id=[3])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testSingleSink1">
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalSink(fields=[c, cnt])
++- LogicalAggregate(group=[{0}], cnt=[COUNT($1)])
+   +- LogicalProject(c=[$2], a=[$0])
+      +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+Sink(fields=[c, cnt])
++- HashAggregate(isMerge=[true], groupBy=[c], select=[c, Final_COUNT(count$0) AS cnt])
+   +- Exchange(distribution=[hash[c]])
+      +- LocalHashAggregate(groupBy=[c], select=[c, Partial_COUNT(a) AS count$0])
+         +- Calc(select=[c, a])
+            +- TableSourceScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testSingleSink2">
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalSink(fields=[a1, b, c1])
++- LogicalProject(a1=[$2], b=[$3], c1=[$1])
+   +- LogicalFilter(condition=[=($2, $0)])
+      +- LogicalJoin(condition=[true], joinType=[inner])
+         :- LogicalTableScan(table=[[table4]])
+         +- LogicalTableScan(table=[[table5]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+Sink(fields=[a1, b, c1])
++- Calc(select=[a1, b, c1])
+   +- HashJoin(joinType=[InnerJoin], where=[=(a1, a3)], select=[a3, c1, a1, b], build=[right])
+      :- Exchange(distribution=[hash[a3]], exchange_mode=[BATCH])
+      :  +- Calc(select=[a AS a3, c AS c1], where=[AND(>=(a, 0), <(b, 5))])
+      :     +- TableSourceScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], reuse_id=[1])
+      +- Exchange(distribution=[hash[a1]])
+         +- Calc(select=[a AS a1, b])
+            +- HashJoin(joinType=[InnerJoin], where=[=(a, a2)], select=[a, b, a2], build=[right])
+               :- Exchange(distribution=[hash[a]], exchange_mode=[BATCH])
+               :  +- Calc(select=[a, b], where=[<=(a, 10)])
+               :     +- Reused(reference_id=[1])
+               +- Exchange(distribution=[hash[a2]])
+                  +- Calc(select=[a AS a2], where=[AND(>=(a, 0), >=(b, 5))])
+                     +- Reused(reference_id=[1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testSingleSink3">
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalSink(fields=[a1, b1])
++- LogicalUnion(all=[true])
+   :- LogicalProject(a1=[$0], b1=[$1])
+   :  +- LogicalTableScan(table=[[table1]])
+   +- LogicalProject(a=[$0], b1=[$1])
+      +- LogicalTableScan(table=[[table2]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+Sink(fields=[a1, b1])
++- Union(all=[true], union=[a, b])
+   :- Calc(select=[a, b], where=[<=(a, 10)], reuse_id=[1])
+   :  +- TableSourceScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   +- Calc(select=[a0 AS a, b AS b1])
+      +- HashJoin(joinType=[InnerJoin], where=[=(a0, a)], select=[a, b, a0], build=[left])
+         :- Exchange(distribution=[hash[a]])
+         :  +- Reused(reference_id=[1])
+         +- Exchange(distribution=[hash[a]])
+            +- Calc(select=[a])
+               +- BoundedStreamScan(table=[[MyTable2]], fields=[a, b, c, d, e])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testSingleSink4">
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalSink(fields=[a, b, c])
++- LogicalProject(a1=[$0], b1=[$3], c1=[$4])
+   +- LogicalFilter(condition=[=($0, $2)])
+      +- LogicalJoin(condition=[true], joinType=[inner])
+         :- LogicalTableScan(table=[[table1]])
+         +- LogicalTableScan(table=[[table6]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+Sink(fields=[a, b, c])
++- Calc(select=[a AS a1, b1, c1])
+   +- HashJoin(joinType=[InnerJoin], where=[=(a, a3)], select=[a, a3, b1, c1], build=[right])
+      :- Exchange(distribution=[hash[a]], exchange_mode=[BATCH])
+      :  +- Calc(select=[a], where=[<=(a, 10)])
+      :     +- TableSourceScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], reuse_id=[1])
+      +- Exchange(distribution=[hash[a3]])
+         +- Calc(select=[a3, b AS b1, c1])
+            +- HashJoin(joinType=[InnerJoin], where=[=(a1, a3)], select=[a3, c1, a1, b], build=[right])
+               :- Exchange(distribution=[hash[a3]], exchange_mode=[BATCH])
+               :  +- Calc(select=[a AS a3, c AS c1], where=[AND(>=(a, 0), <(b, 5))])
+               :     +- Reused(reference_id=[1])
+               +- Exchange(distribution=[hash[a1]])
+                  +- Calc(select=[a AS a1, b])
+                     +- HashJoin(joinType=[InnerJoin], where=[=(a, a2)], select=[a, b, a2], build=[right])
+                        :- Exchange(distribution=[hash[a]], exchange_mode=[BATCH])
+                        :  +- Calc(select=[a, b], where=[<=(a, 10)])
+                        :     +- Reused(reference_id=[1])
+                        +- Exchange(distribution=[hash[a2]])
+                           +- Calc(select=[a AS a2], where=[AND(>=(a, 0), >=(b, 5))])
+                              +- Reused(reference_id=[1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testSingleSinkWithUDTF">
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalSink(fields=[a, b, c, d, e, f, i, j, k, l, m, s])
++- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5], i=[$6], j=[$7], k=[$8], l=[$9], m=[$10], s=[$11])
+   +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{2}])
+      :- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5], i=[$6], j=[$7], k=[$8], l=[$9], m=[$10])
+      :  +- LogicalFilter(condition=[AND(=($1, $4), =($0, $6))])
+      :     +- LogicalJoin(condition=[true], joinType=[inner])
+      :        :- LogicalJoin(condition=[true], joinType=[inner])
+      :        :  :- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+      :        :  +- LogicalTableScan(table=[[MyTable1, source: [TestTableSource(d, e, f)]]])
+      :        +- LogicalTableScan(table=[[MyTable2, source: [TestTableSource(i, j, k, l, m)]]])
+      +- LogicalTableFunctionScan(invocation=[split($cor0.c)], rowType=[RecordType(VARCHAR(65536) f0)], elementType=[class [Ljava.lang.Object;])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+Sink(fields=[a, b, c, d, e, f, i, j, k, l, m, s])
++- Correlate(invocation=[split($cor0.c)], correlate=[table(split($cor0.c))], select=[a,b,c,d,e,f,i,j,k,l,m,f0], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(65536) c, INTEGER d, BIGINT e, VARCHAR(65536) f, INTEGER i, BIGINT j, INTEGER k, VARCHAR(65536) l, BIGINT m, VARCHAR(65536) f0)], joinType=[INNER])
+   +- HashJoin(joinType=[InnerJoin], where=[=(a, i)], select=[a, b, c, d, e, f, i, j, k, l, m], build=[left])
+      :- Exchange(distribution=[hash[a]])
+      :  +- HashJoin(joinType=[InnerJoin], where=[=(b, e)], select=[a, b, c, d, e, f], build=[right])
+      :     :- Exchange(distribution=[hash[b]])
+      :     :  +- TableSourceScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+      :     +- Exchange(distribution=[hash[e]])
+      :        +- TableSourceScan(table=[[MyTable1, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+      +- Exchange(distribution=[hash[i]])
+         +- TableSourceScan(table=[[MyTable2, source: [TestTableSource(i, j, k, l, m)]]], fields=[i, j, k, l, m])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testSingleSinkSplitOnUnion">
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalSink(fields=[total_sum])
++- LogicalAggregate(group=[{}], total_sum=[SUM($0)])
+   +- LogicalProject(a=[$0])
+      +- LogicalUnion(all=[true])
+         :- LogicalProject(a=[$0], c=[$2])
+         :  +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+         +- LogicalProject(d=[$0], f=[$2])
+            +- LogicalTableScan(table=[[MyTable1, source: [TestTableSource(d, e, f)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+Sink(fields=[total_sum])
++- HashAggregate(isMerge=[true], select=[Final_SUM(sum$0) AS total_sum])
+   +- Exchange(distribution=[single])
+      +- LocalHashAggregate(select=[Partial_SUM(a) AS sum$0])
+         +- Union(all=[true], union=[a])
+            :- Calc(select=[a])
+            :  +- TableSourceScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+            +- Calc(select=[d AS a])
+               +- TableSourceScan(table=[[MyTable1, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+]]>
+    </Resource>
+  </TestCase>
+</Root>
diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/SinkTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/SinkTest.xml
new file mode 100644
index 0000000..7652254
--- /dev/null
+++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/SinkTest.xml
@@ -0,0 +1,84 @@
+<?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="testSingleSink">
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalSink(fields=[a])
++- LogicalProject(cnt=[$1])
+   +- LogicalAggregate(group=[{0}], cnt=[COUNT()])
+      +- LogicalProject(a=[$0])
+         +- LogicalTableScan(table=[[MyTable]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+Sink(fields=[a])
++- Calc(select=[cnt])
+   +- HashAggregate(isMerge=[true], groupBy=[a], select=[a, Final_COUNT(count1$0) AS cnt])
+      +- Exchange(distribution=[hash[a]])
+         +- LocalHashAggregate(groupBy=[a], select=[a, Partial_COUNT(*) AS count1$0])
+            +- Calc(select=[a])
+               +- BoundedStreamScan(table=[[MyTable]], fields=[a, b, c])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testMultiSinks">
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalSink(fields=[total_sum])
++- LogicalAggregate(group=[{}], total_sum=[SUM($0)])
+   +- LogicalProject(sum_a=[$0])
+      +- LogicalProject(sum_a=[$1], c=[$0])
+         +- LogicalAggregate(group=[{0}], sum_a=[SUM($1)])
+            +- LogicalProject(c=[$2], a=[$0])
+               +- LogicalTableScan(table=[[MyTable]])
+
+LogicalSink(fields=[total_min])
++- LogicalAggregate(group=[{}], total_min=[MIN($0)])
+   +- LogicalProject(sum_a=[$0])
+      +- LogicalProject(sum_a=[$1], c=[$0])
+         +- LogicalAggregate(group=[{0}], sum_a=[SUM($1)])
+            +- LogicalProject(c=[$2], a=[$0])
+               +- LogicalTableScan(table=[[MyTable]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+Calc(select=[sum_a], reuse_id=[1])
++- HashAggregate(isMerge=[true], groupBy=[c], select=[c, Final_SUM(sum$0) AS sum_a])
+   +- Exchange(distribution=[hash[c]])
+      +- LocalHashAggregate(groupBy=[c], select=[c, Partial_SUM(a) AS sum$0])
+         +- Calc(select=[c, a])
+            +- BoundedStreamScan(table=[[MyTable]], fields=[a, b, c])
+
+Sink(fields=[total_sum])
++- HashAggregate(isMerge=[true], select=[Final_SUM(sum$0) AS total_sum])
+   +- Exchange(distribution=[single])
+      +- LocalHashAggregate(select=[Partial_SUM(sum_a) AS sum$0])
+         +- Reused(reference_id=[1])
+
+Sink(fields=[total_min])
++- HashAggregate(isMerge=[true], select=[Final_MIN(min$0) AS total_min])
+   +- Exchange(distribution=[single])
+      +- LocalHashAggregate(select=[Partial_MIN(sum_a) AS min$0])
+         +- Reused(reference_id=[1])
+]]>
+    </Resource>
+  </TestCase>
+</Root>
diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/stream/sql/DagOptimizationTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/stream/sql/DagOptimizationTest.xml
new file mode 100644
index 0000000..88931b0
--- /dev/null
+++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/stream/sql/DagOptimizationTest.xml
@@ -0,0 +1,980 @@
+<?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="testMultiLevelViews">
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalSink(fields=[a, b, c])
++- LogicalProject(a=[$0], b=[$1], c=[$2])
+   +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'%hello%')])
+      +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+
+LogicalSink(fields=[b, cnt])
++- LogicalProject(b=[$0], cnt=[$1])
+   +- LogicalFilter(condition=[<($0, 4)])
+      +- LogicalAggregate(group=[{0}], cnt=[COUNT($1)])
+         +- LogicalProject(b=[$1], a=[$0])
+            +- LogicalUnion(all=[true])
+               :- LogicalProject(a=[$0], b=[$1], c=[$2])
+               :  +- LogicalProject(a=[$0], b=[$1], c=[$2])
+               :     +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'%hello%')])
+               :        +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+               +- LogicalProject(a=[$0], b=[$1], c=[$2])
+                  +- LogicalProject(a=[$0], b=[$1], c=[$2])
+                     +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'%world%')])
+                        +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+
+LogicalSink(fields=[b, cnt])
++- LogicalProject(b=[$0], cnt=[$1])
+   +- LogicalFilter(condition=[AND(>=($0, 4), <($0, 6))])
+      +- LogicalAggregate(group=[{0}], cnt=[COUNT($1)])
+         +- LogicalProject(b=[$1], a=[$0])
+            +- LogicalUnion(all=[true])
+               :- LogicalProject(a=[$0], b=[$1], c=[$2])
+               :  +- LogicalProject(a=[$0], b=[$1], c=[$2])
+               :     +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'%hello%')])
+               :        +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+               +- LogicalProject(a=[$0], b=[$1], c=[$2])
+                  +- LogicalProject(a=[$0], b=[$1], c=[$2])
+                     +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'%world%')])
+                        +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+TableSourceScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], updateAsRetraction=[true], accMode=[Acc], reuse_id=[2])
+
+Calc(select=[a, b, c], where=[LIKE(c, _UTF-16LE'%hello%')], updateAsRetraction=[true], accMode=[Acc], reuse_id=[1])
++- Reused(reference_id=[2])
+
+Sink(fields=[a, b, c], updateAsRetraction=[false], accMode=[Acc])
++- Reused(reference_id=[1])
+
+GroupAggregate(groupBy=[b], select=[b, COUNT(a) AS cnt], updateAsRetraction=[true], accMode=[AccRetract], reuse_id=[3])
++- Exchange(distribution=[hash[b]], updateAsRetraction=[true], accMode=[Acc])
+   +- Union(all=[true], union=[b, a], updateAsRetraction=[true], accMode=[Acc])
+      :- Calc(select=[b, a], updateAsRetraction=[true], accMode=[Acc])
+      :  +- Reused(reference_id=[1])
+      +- Calc(select=[b, a], where=[LIKE(c, _UTF-16LE'%world%')], updateAsRetraction=[true], accMode=[Acc])
+         +- Reused(reference_id=[2])
+
+Sink(fields=[b, cnt], updateAsRetraction=[true], accMode=[AccRetract])
++- Calc(select=[b, cnt], where=[<(b, 4)], updateAsRetraction=[true], accMode=[AccRetract])
+   +- Reused(reference_id=[3])
+
+Sink(fields=[b, cnt], updateAsRetraction=[false], accMode=[AccRetract])
++- Calc(select=[b, cnt], where=[AND(>=(b, 4), <(b, 6))], updateAsRetraction=[false], accMode=[AccRetract])
+   +- Reused(reference_id=[3])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testMultiSinks1">
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalSink(fields=[total_sum])
++- LogicalAggregate(group=[{}], total_sum=[SUM($0)])
+   +- LogicalProject(sum_a=[$0])
+      +- LogicalProject(sum_a=[$1], c=[$0])
+         +- LogicalAggregate(group=[{0}], sum_a=[SUM($1)])
+            +- LogicalProject(c=[$2], a=[$0])
+               +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+
+LogicalSink(fields=[total_min])
++- LogicalAggregate(group=[{}], total_min=[MIN($0)])
+   +- LogicalProject(sum_a=[$0])
+      +- LogicalProject(sum_a=[$1], c=[$0])
+         +- LogicalAggregate(group=[{0}], sum_a=[SUM($1)])
+            +- LogicalProject(c=[$2], a=[$0])
+               +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+Exchange(distribution=[single], updateAsRetraction=[true], accMode=[AccRetract], reuse_id=[1])
++- Calc(select=[sum_a], updateAsRetraction=[true], accMode=[AccRetract])
+   +- GroupAggregate(groupBy=[c], select=[c, SUM(a) AS sum_a], updateAsRetraction=[true], accMode=[AccRetract])
+      +- Exchange(distribution=[hash[c]], updateAsRetraction=[true], accMode=[Acc])
+         +- Calc(select=[c, a], updateAsRetraction=[true], accMode=[Acc])
+            +- TableSourceScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], updateAsRetraction=[true], accMode=[Acc])
+
+Sink(fields=[total_sum], updateAsRetraction=[false], accMode=[Acc])
++- GroupAggregate(select=[SUM_RETRACT(sum_a) AS total_sum], updateAsRetraction=[false], accMode=[Acc])
+   +- Reused(reference_id=[1])
+
+Sink(fields=[total_min], updateAsRetraction=[false], accMode=[Acc])
++- GroupAggregate(select=[MIN_RETRACT(sum_a) AS total_min], updateAsRetraction=[false], accMode=[Acc])
+   +- Reused(reference_id=[1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testMultiSinks2">
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalSink(fields=[a, b1])
++- LogicalUnion(all=[true])
+   :- LogicalProject(a1=[$0], b1=[$1])
+   :  +- LogicalProject(a1=[$0], b1=[$1])
+   :     +- LogicalFilter(condition=[<=($0, 10)])
+   :        +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+   +- LogicalProject(a=[$0], b1=[$1])
+      +- LogicalProject(a=[$2], b1=[$1])
+         +- LogicalFilter(condition=[=($2, $0)])
+            +- LogicalJoin(condition=[true], joinType=[inner])
+               :- LogicalProject(a1=[$0], b1=[$1])
+               :  +- LogicalFilter(condition=[<=($0, 10)])
+               :     +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+               +- LogicalTableScan(table=[[MyTable2, source: [TestTableSource(a, b, c, d, e)]]])
+
+LogicalSink(fields=[a, b1])
++- LogicalUnion(all=[true])
+   :- LogicalProject(a1=[$0], b1=[$1])
+   :  +- LogicalProject(a1=[$0], b1=[$1])
+   :     +- LogicalFilter(condition=[<=($0, 10)])
+   :        +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+   +- LogicalProject(a=[$0], b1=[$1])
+      +- LogicalProject(a=[$2], b1=[$1])
+         +- LogicalFilter(condition=[=($2, $0)])
+            +- LogicalJoin(condition=[true], joinType=[inner])
+               :- LogicalProject(a1=[$0], b1=[$1])
+               :  +- LogicalFilter(condition=[<=($0, 10)])
+               :     +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+               +- LogicalTableScan(table=[[MyTable2, source: [TestTableSource(a, b, c, d, e)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+Calc(select=[a AS a1, b AS b1], where=[<=(a, 10)], updateAsRetraction=[false], accMode=[Acc], reuse_id=[1])
++- TableSourceScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], updateAsRetraction=[false], accMode=[Acc])
+
+Union(all=[true], union=[a1, b1], updateAsRetraction=[false], accMode=[Acc], reuse_id=[2])
+:- Reused(reference_id=[1])
++- Calc(select=[a, b1], updateAsRetraction=[false], accMode=[Acc])
+   +- Join(joinType=[InnerJoin], where=[=(a, a1)], select=[a1, b1, a, b, c, d, e], updateAsRetraction=[false], accMode=[Acc])
+      :- Exchange(distribution=[hash[a1]], updateAsRetraction=[true], accMode=[Acc])
+      :  +- Reused(reference_id=[1])
+      +- Exchange(distribution=[hash[a]], updateAsRetraction=[true], accMode=[Acc])
+         +- TableSourceScan(table=[[MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e], updateAsRetraction=[true], accMode=[Acc])
+
+Sink(fields=[a, b1], updateAsRetraction=[false], accMode=[Acc])
++- Reused(reference_id=[2])
+
+Sink(fields=[a, b1], updateAsRetraction=[false], accMode=[Acc])
++- Reused(reference_id=[2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testMultiSinks3">
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalSink(fields=[a, b1])
++- LogicalProject(a=[$2], b1=[$1])
+   +- LogicalFilter(condition=[=($2, $0)])
+      +- LogicalJoin(condition=[true], joinType=[inner])
+         :- LogicalProject(a1=[$0], b1=[$1])
+         :  +- LogicalFilter(condition=[<=($0, 10)])
+         :     +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+         +- LogicalTableScan(table=[[MyTable2, source: [TestTableSource(a, b, c, d, e)]]])
+
+LogicalSink(fields=[a, b1])
++- LogicalUnion(all=[true])
+   :- LogicalProject(a1=[$0], b1=[$1])
+   :  +- LogicalProject(a1=[$0], b1=[$1])
+   :     +- LogicalFilter(condition=[<=($0, 10)])
+   :        +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+   +- LogicalProject(a=[$0], b1=[$1])
+      +- LogicalProject(a=[$2], b1=[$1])
+         +- LogicalFilter(condition=[=($2, $0)])
+            +- LogicalJoin(condition=[true], joinType=[inner])
+               :- LogicalProject(a1=[$0], b1=[$1])
+               :  +- LogicalFilter(condition=[<=($0, 10)])
+               :     +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+               +- LogicalTableScan(table=[[MyTable2, source: [TestTableSource(a, b, c, d, e)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+Calc(select=[a, b], where=[<=(a, 10)], updateAsRetraction=[true], accMode=[Acc], reuse_id=[1])
++- TableSourceScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], updateAsRetraction=[true], accMode=[Acc])
+
+Calc(select=[a0 AS a, b AS b1], updateAsRetraction=[false], accMode=[Acc], reuse_id=[2])
++- Join(joinType=[InnerJoin], where=[=(a0, a)], select=[a, b, a0, b0, c, d, e], updateAsRetraction=[false], accMode=[Acc])
+   :- Exchange(distribution=[hash[a]], updateAsRetraction=[true], accMode=[Acc])
+   :  +- Reused(reference_id=[1])
+   +- Exchange(distribution=[hash[a]], updateAsRetraction=[true], accMode=[Acc])
+      +- TableSourceScan(table=[[MyTable2, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e], updateAsRetraction=[true], accMode=[Acc])
+
+Sink(fields=[a, b1], updateAsRetraction=[false], accMode=[Acc])
++- Reused(reference_id=[2])
+
+Sink(fields=[a, b1], updateAsRetraction=[false], accMode=[Acc])
++- Union(all=[true], union=[a, b], updateAsRetraction=[false], accMode=[Acc])
+   :- Reused(reference_id=[1])
+   +- Reused(reference_id=[2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testMultiSinks4">
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalSink(fields=[a1, b, c2])
++- LogicalProject(a1=[$0], b=[$1], c2=[$3])
+   +- LogicalFilter(condition=[=($0, $2)])
+      +- LogicalJoin(condition=[true], joinType=[inner])
+         :- LogicalProject(a1=[$0], b=[$1])
+         :  +- LogicalFilter(condition=[<=($0, 10)])
+         :     +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+         +- LogicalProject(a2=[$0], c=[$2])
+            +- LogicalFilter(condition=[>=($1, 5)])
+               +- LogicalProject(a=[$0], b=[$1], c=[$2])
+                  +- LogicalFilter(condition=[>=($0, 0)])
+                     +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+
+LogicalSink(fields=[a1, b, c1])
++- LogicalProject(a1=[$2], b=[$3], c1=[$1])
+   +- LogicalFilter(condition=[=($2, $0)])
+      +- LogicalJoin(condition=[true], joinType=[inner])
+         :- LogicalProject(a3=[$0], c1=[$2])
+         :  +- LogicalFilter(condition=[<($1, 5)])
+         :     +- LogicalProject(a=[$0], b=[$1], c=[$2])
+         :        +- LogicalFilter(condition=[>=($0, 0)])
+         :           +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+         +- LogicalProject(a1=[$0], b=[$1], c2=[$3])
+            +- LogicalFilter(condition=[=($0, $2)])
+               +- LogicalJoin(condition=[true], joinType=[inner])
+                  :- LogicalProject(a1=[$0], b=[$1])
+                  :  +- LogicalFilter(condition=[<=($0, 10)])
+                  :     +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+                  +- LogicalProject(a2=[$0], c=[$2])
+                     +- LogicalFilter(condition=[>=($1, 5)])
+                        +- LogicalProject(a=[$0], b=[$1], c=[$2])
+                           +- LogicalFilter(condition=[>=($0, 0)])
+                              +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+TableSourceScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], updateAsRetraction=[true], accMode=[Acc], reuse_id=[1])
+
+Calc(select=[a, b, c], where=[>=(a, 0)], updateAsRetraction=[true], accMode=[Acc], reuse_id=[2])
++- Reused(reference_id=[1])
+
+Calc(select=[a1, b, c AS c2], updateAsRetraction=[true], accMode=[Acc], reuse_id=[3])
++- Join(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b, a2, c], updateAsRetraction=[true], accMode=[Acc])
+   :- Exchange(distribution=[hash[a1]], updateAsRetraction=[true], accMode=[Acc])
+   :  +- Calc(select=[a AS a1, b], where=[<=(a, 10)], updateAsRetraction=[true], accMode=[Acc])
+   :     +- Reused(reference_id=[1])
+   +- Exchange(distribution=[hash[a2]], updateAsRetraction=[true], accMode=[Acc])
+      +- Calc(select=[a AS a2, c], where=[>=(b, 5)], updateAsRetraction=[true], accMode=[Acc])
+         +- Reused(reference_id=[2])
+
+Sink(fields=[a1, b, c2], updateAsRetraction=[false], accMode=[Acc])
++- Reused(reference_id=[3])
+
+Sink(fields=[a1, b, c1], updateAsRetraction=[false], accMode=[Acc])
++- Calc(select=[a1, b, c1], updateAsRetraction=[false], accMode=[Acc])
+   +- Join(joinType=[InnerJoin], where=[=(a1, a3)], select=[a3, c1, a1, b, c2], updateAsRetraction=[false], accMode=[Acc])
+      :- Exchange(distribution=[hash[a3]], updateAsRetraction=[true], accMode=[Acc])
+      :  +- Calc(select=[a AS a3, c AS c1], where=[<(b, 5)], updateAsRetraction=[true], accMode=[Acc])
+      :     +- Reused(reference_id=[2])
+      +- Exchange(distribution=[hash[a1]], updateAsRetraction=[true], accMode=[Acc])
+         +- Reused(reference_id=[3])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testMultiSinks5">
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalSink(fields=[total_sum])
++- LogicalAggregate(group=[{}], total_sum=[SUM($0)])
+   +- LogicalProject(a=[$0])
+      +- LogicalProject(a=[random_udf($0)], c=[$2])
+         +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+
+LogicalSink(fields=[total_min])
++- LogicalAggregate(group=[{}], total_min=[MIN($0)])
+   +- LogicalProject(a=[$0])
+      +- LogicalProject(a=[random_udf($0)], c=[$2])
+         +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+Exchange(distribution=[single], updateAsRetraction=[true], accMode=[Acc], reuse_id=[1])
++- Calc(select=[random_udf(a) AS a], updateAsRetraction=[true], accMode=[Acc])
+   +- TableSourceScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], updateAsRetraction=[true], accMode=[Acc])
+
+Sink(fields=[total_sum], updateAsRetraction=[false], accMode=[Acc])
++- GroupAggregate(select=[SUM(a) AS total_sum], updateAsRetraction=[false], accMode=[Acc])
+   +- Reused(reference_id=[1])
+
+Sink(fields=[total_min], updateAsRetraction=[false], accMode=[Acc])
++- GroupAggregate(select=[MIN(a) AS total_min], updateAsRetraction=[false], accMode=[Acc])
+   +- Reused(reference_id=[1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testMultiSinksWithUDTF">
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalSink(fields=[a, total_c])
++- LogicalProject(a=[$0], total_c=[$1])
+   +- LogicalFilter(condition=[>($0, 50)])
+      +- LogicalUnion(all=[true])
+         :- LogicalProject(a=[$0], total_c=[$2])
+         :  +- LogicalAggregate(group=[{0, 1}], total_c=[COUNT(DISTINCT $2)])
+         :     +- LogicalProject(a=[$0], b=[$1], c=[$3])
+         :        +- LogicalFilter(condition=[<>($2, _UTF-16LE'')])
+         :           +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{2}])
+         :              :- LogicalProject(a=[$0], b=[-($1, MOD($1, 300))], c=[$2])
+         :              :  +- LogicalFilter(condition=[>=($1, UNIX_TIMESTAMP(_UTF-16LE'${startTime}'))])
+         :              :     +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+         :              +- LogicalTableFunctionScan(invocation=[split($cor0.c)], rowType=[RecordType(VARCHAR(65536) f0)], elementType=[class [Ljava.lang.Object;])
+         +- LogicalProject(a=[$0], total_c=[0])
+            +- LogicalProject(a=[$0], b=[-($1, MOD($1, 300))], c=[$2])
+               +- LogicalFilter(condition=[>=($1, UNIX_TIMESTAMP(_UTF-16LE'${startTime}'))])
+                  +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+
+LogicalSink(fields=[a, total_c])
++- LogicalProject(a=[$0], total_c=[$1])
+   +- LogicalFilter(condition=[<($0, 50)])
+      +- LogicalUnion(all=[true])
+         :- LogicalProject(a=[$0], total_c=[$2])
+         :  +- LogicalAggregate(group=[{0, 1}], total_c=[COUNT(DISTINCT $2)])
+         :     +- LogicalProject(a=[$0], b=[$1], c=[$3])
+         :        +- LogicalFilter(condition=[<>($2, _UTF-16LE'')])
+         :           +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{2}])
+         :              :- LogicalProject(a=[$0], b=[-($1, MOD($1, 300))], c=[$2])
+         :              :  +- LogicalFilter(condition=[>=($1, UNIX_TIMESTAMP(_UTF-16LE'${startTime}'))])
+         :              :     +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+         :              +- LogicalTableFunctionScan(invocation=[split($cor0.c)], rowType=[RecordType(VARCHAR(65536) f0)], elementType=[class [Ljava.lang.Object;])
+         +- LogicalProject(a=[$0], total_c=[0])
+            +- LogicalProject(a=[$0], b=[-($1, MOD($1, 300))], c=[$2])
+               +- LogicalFilter(condition=[>=($1, UNIX_TIMESTAMP(_UTF-16LE'${startTime}'))])
+                  +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+TableSourceScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], updateAsRetraction=[true], accMode=[Acc], reuse_id=[1])
+
+Calc(select=[a, total_c], updateAsRetraction=[false], accMode=[Acc], reuse_id=[2])
++- GroupAggregate(groupBy=[a, b], select=[a, b, COUNT(DISTINCT c) AS total_c], updateAsRetraction=[false], accMode=[Acc])
+   +- Exchange(distribution=[hash[a, b]], updateAsRetraction=[true], accMode=[Acc])
+      +- Calc(select=[a, b, f0 AS c], updateAsRetraction=[true], accMode=[Acc])
+         +- Correlate(invocation=[split($cor0.c)], correlate=[table(split($cor0.c))], select=[a,b,c,f0], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(65536) c, VARCHAR(65536) f0)], joinType=[INNER], updateAsRetraction=[true], accMode=[Acc])
+            +- Calc(select=[a, -(b, MOD(b, 300)) AS b, c], where=[AND(>=(b, UNIX_TIMESTAMP(_UTF-16LE'${startTime}')), <>(c, _UTF-16LE'':VARCHAR(65536) CHARACTER SET "UTF-16LE"))], updateAsRetraction=[true], accMode=[Acc])
+               +- Reused(reference_id=[1])
+
+Calc(select=[a, 0 AS total_c], where=[>=(b, UNIX_TIMESTAMP(_UTF-16LE'${startTime}'))], updateAsRetraction=[false], accMode=[Acc], reuse_id=[3])
++- Reused(reference_id=[1])
+
+Sink(fields=[a, total_c], updateAsRetraction=[false], accMode=[Acc])
++- Union(all=[true], union=[a, total_c], updateAsRetraction=[false], accMode=[Acc])
+   :- Calc(select=[a, total_c], where=[>(a, 50)], updateAsRetraction=[false], accMode=[Acc])
+   :  +- Reused(reference_id=[2])
+   +- Calc(select=[a, CAST(total_c) AS total_c], where=[>(a, 50)], updateAsRetraction=[false], accMode=[Acc])
+      +- Reused(reference_id=[3])
+
+Sink(fields=[a, total_c], updateAsRetraction=[false], accMode=[Acc])
++- Union(all=[true], union=[a, total_c], updateAsRetraction=[false], accMode=[Acc])
+   :- Calc(select=[a, total_c], where=[<(a, 50)], updateAsRetraction=[false], accMode=[Acc])
+   :  +- Reused(reference_id=[2])
+   +- Calc(select=[a, CAST(total_c) AS total_c], where=[<(a, 50)], updateAsRetraction=[false], accMode=[Acc])
+      +- Reused(reference_id=[3])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testMultiSinksSplitOnUnion1">
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalSink(fields=[total_sum])
++- LogicalAggregate(group=[{}], total_sum=[SUM($0)])
+   +- LogicalProject(a=[$0])
+      +- LogicalUnion(all=[true])
+         :- LogicalProject(a=[$0], c=[$2])
+         :  +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+         +- LogicalProject(d=[$0], f=[$2])
+            +- LogicalTableScan(table=[[MyTable1, source: [TestTableSource(d, e, f)]]])
+
+LogicalSink(fields=[total_min])
++- LogicalAggregate(group=[{}], total_min=[MIN($0)])
+   +- LogicalProject(a=[$0])
+      +- LogicalUnion(all=[true])
+         :- LogicalProject(a=[$0], c=[$2])
+         :  +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+         +- LogicalProject(d=[$0], f=[$2])
+            +- LogicalTableScan(table=[[MyTable1, source: [TestTableSource(d, e, f)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+Exchange(distribution=[single], updateAsRetraction=[true], accMode=[Acc], reuse_id=[1])
++- Calc(select=[a], updateAsRetraction=[true], accMode=[Acc])
+   +- Union(all=[true], union=[a, c], updateAsRetraction=[true], accMode=[Acc])
+      :- Calc(select=[a, c], updateAsRetraction=[true], accMode=[Acc])
+      :  +- TableSourceScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], updateAsRetraction=[true], accMode=[Acc])
+      +- Calc(select=[d, f], updateAsRetraction=[true], accMode=[Acc])
+         +- TableSourceScan(table=[[MyTable1, source: [TestTableSource(d, e, f)]]], fields=[d, e, f], updateAsRetraction=[true], accMode=[Acc])
+
+Sink(fields=[total_sum], updateAsRetraction=[false], accMode=[Acc])
++- GroupAggregate(select=[SUM(a) AS total_sum], updateAsRetraction=[false], accMode=[Acc])
+   +- Reused(reference_id=[1])
+
+Sink(fields=[total_min], updateAsRetraction=[true], accMode=[AccRetract])
++- GroupAggregate(select=[MIN(a) AS total_min], updateAsRetraction=[true], accMode=[AccRetract])
+   +- Reused(reference_id=[1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testMultiSinksSplitOnUnion2">
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalSink(fields=[total_sum])
++- LogicalAggregate(group=[{}], total_sum=[SUM($0)])
+   +- LogicalProject(a=[$0])
+      +- LogicalUnion(all=[true])
+         :- LogicalUnion(all=[true])
+         :  :- LogicalProject(a=[$0], c=[$2])
+         :  :  +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+         :  +- LogicalProject(d=[$0], f=[$2])
+         :     +- LogicalTableScan(table=[[MyTable1, source: [TestTableSource(d, e, f)]]])
+         +- LogicalProject(a=[$0], c=[$2])
+            +- LogicalTableScan(table=[[MyTable2, source: [TestTableSource(a, b, c)]]])
+
+LogicalSink(fields=[total_min])
++- LogicalAggregate(group=[{}], total_min=[MIN($0)])
+   +- LogicalProject(a=[$0])
+      +- LogicalUnion(all=[true])
+         :- LogicalUnion(all=[true])
+         :  :- LogicalProject(a=[$0], c=[$2])
+         :  :  +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+         :  +- LogicalProject(d=[$0], f=[$2])
+         :     +- LogicalTableScan(table=[[MyTable1, source: [TestTableSource(d, e, f)]]])
+         +- LogicalProject(a=[$0], c=[$2])
+            +- LogicalTableScan(table=[[MyTable2, source: [TestTableSource(a, b, c)]]])
+
+LogicalSink(fields=[a])
++- LogicalProject(a=[$0])
+   +- LogicalUnion(all=[true])
+      :- LogicalProject(a=[$0], c=[$2])
+      :  +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+      +- LogicalProject(d=[$0], f=[$2])
+         +- LogicalTableScan(table=[[MyTable1, source: [TestTableSource(d, e, f)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+Calc(select=[a], reuse_id=[2])
++- Union(all=[true], union=[a, c])
+   :- Calc(select=[a, c])
+   :  +- TableSourceScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   +- Calc(select=[d, f])
+      +- TableSourceScan(table=[[MyTable1, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+
+Exchange(distribution=[single], reuse_id=[1])
++- Union(all=[true], union=[a])
+   :- Reused(reference_id=[2])
+   +- Calc(select=[a])
+      +- TableSourceScan(table=[[MyTable2, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+
+Sink(fields=[total_sum])
++- GroupAggregate(select=[SUM(a) AS total_sum])
+   +- Reused(reference_id=[1])
+
+Sink(fields=[total_min])
++- GroupAggregate(select=[MIN(a) AS total_min])
+   +- Reused(reference_id=[1])
+
+Sink(fields=[a])
++- Reused(reference_id=[2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testMultiSinksSplitOnUnion3">
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalSink(fields=[a])
++- LogicalUnion(all=[true])
+   :- LogicalProject(a=[$0], c=[$2])
+   :  +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+   +- LogicalProject(d=[$0], f=[$2])
+      +- LogicalTableScan(table=[[MyTable1, source: [TestTableSource(d, e, f)]]])
+
+LogicalSink(fields=[total_sum])
++- LogicalAggregate(group=[{}], total_sum=[SUM($0)])
+   +- LogicalProject(a=[$0])
+      +- LogicalUnion(all=[true])
+         :- LogicalProject(a=[$0], c=[$1])
+         :  +- LogicalUnion(all=[true])
+         :     :- LogicalProject(a=[$0], c=[$2])
+         :     :  +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+         :     +- LogicalProject(d=[$0], f=[$2])
+         :        +- LogicalTableScan(table=[[MyTable1, source: [TestTableSource(d, e, f)]]])
+         +- LogicalProject(a=[$0], c=[$2])
+            +- LogicalTableScan(table=[[MyTable2, source: [TestTableSource(a, b, c)]]])
+
+LogicalSink(fields=[total_min])
++- LogicalAggregate(group=[{}], total_min=[MIN($0)])
+   +- LogicalProject(a=[$0])
+      +- LogicalUnion(all=[true])
+         :- LogicalProject(a=[$0], c=[$1])
+         :  +- LogicalUnion(all=[true])
+         :     :- LogicalProject(a=[$0], c=[$2])
+         :     :  +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+         :     +- LogicalProject(d=[$0], f=[$2])
+         :        +- LogicalTableScan(table=[[MyTable1, source: [TestTableSource(d, e, f)]]])
+         +- LogicalProject(a=[$0], c=[$2])
+            +- LogicalTableScan(table=[[MyTable2, source: [TestTableSource(a, b, c)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+Union(all=[true], union=[a, c], updateAsRetraction=[false], accMode=[Acc], reuse_id=[1])
+:- Calc(select=[a, c], updateAsRetraction=[true], accMode=[Acc])
+:  +- TableSourceScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], updateAsRetraction=[true], accMode=[Acc])
++- Calc(select=[d, f], updateAsRetraction=[true], accMode=[Acc])
+   +- TableSourceScan(table=[[MyTable1, source: [TestTableSource(d, e, f)]]], fields=[d, e, f], updateAsRetraction=[true], accMode=[Acc])
+
+Sink(fields=[a], updateAsRetraction=[false], accMode=[Acc])
++- Reused(reference_id=[1])
+
+Exchange(distribution=[single], updateAsRetraction=[true], accMode=[Acc], reuse_id=[2])
++- Calc(select=[a], updateAsRetraction=[true], accMode=[Acc])
+   +- Union(all=[true], union=[a, c], updateAsRetraction=[true], accMode=[Acc])
+      :- Reused(reference_id=[1])
+      +- Calc(select=[a, c], updateAsRetraction=[true], accMode=[Acc])
+         +- TableSourceScan(table=[[MyTable2, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], updateAsRetraction=[true], accMode=[Acc])
+
+Sink(fields=[total_sum], updateAsRetraction=[true], accMode=[AccRetract])
++- GroupAggregate(select=[SUM(a) AS total_sum], updateAsRetraction=[true], accMode=[AccRetract])
+   +- Reused(reference_id=[2])
+
+Sink(fields=[total_min], updateAsRetraction=[false], accMode=[Acc])
++- GroupAggregate(select=[MIN(a) AS total_min], updateAsRetraction=[false], accMode=[Acc])
+   +- Reused(reference_id=[2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testMultiSinksSplitOnUnion4">
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalSink(fields=[total_sum])
++- LogicalAggregate(group=[{}], total_sum=[SUM($0)])
+   +- LogicalProject(a=[$0])
+      +- LogicalUnion(all=[true])
+         :- LogicalUnion(all=[true])
+         :  :- LogicalProject(a=[$0], c=[$2])
+         :  :  +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+         :  +- LogicalProject(d=[$0], f=[$2])
+         :     +- LogicalTableScan(table=[[MyTable1, source: [TestTableSource(d, e, f)]]])
+         +- LogicalProject(a=[$0], c=[$2])
+            +- LogicalTableScan(table=[[MyTable2, source: [TestTableSource(a, b, c)]]])
+
+LogicalSink(fields=[total_min])
++- LogicalAggregate(group=[{}], total_min=[MIN($0)])
+   +- LogicalProject(a=[$0])
+      +- LogicalUnion(all=[true])
+         :- LogicalUnion(all=[true])
+         :  :- LogicalProject(a=[$0], c=[$2])
+         :  :  +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+         :  +- LogicalProject(d=[$0], f=[$2])
+         :     +- LogicalTableScan(table=[[MyTable1, source: [TestTableSource(d, e, f)]]])
+         +- LogicalProject(a=[$0], c=[$2])
+            +- LogicalTableScan(table=[[MyTable2, source: [TestTableSource(a, b, c)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+Exchange(distribution=[single], updateAsRetraction=[true], accMode=[Acc], reuse_id=[1])
++- Calc(select=[a], updateAsRetraction=[true], accMode=[Acc])
+   +- Union(all=[true], union=[a, c], updateAsRetraction=[true], accMode=[Acc])
+      :- Union(all=[true], union=[a, c], updateAsRetraction=[true], accMode=[Acc])
+      :  :- Calc(select=[a, c], updateAsRetraction=[true], accMode=[Acc])
+      :  :  +- TableSourceScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], updateAsRetraction=[true], accMode=[Acc])
+      :  +- Calc(select=[d, f], updateAsRetraction=[true], accMode=[Acc])
+      :     +- TableSourceScan(table=[[MyTable1, source: [TestTableSource(d, e, f)]]], fields=[d, e, f], updateAsRetraction=[true], accMode=[Acc])
+      +- Calc(select=[a, c], updateAsRetraction=[true], accMode=[Acc])
+         +- TableSourceScan(table=[[MyTable2, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], updateAsRetraction=[true], accMode=[Acc])
+
+Sink(fields=[total_sum], updateAsRetraction=[false], accMode=[Acc])
++- GroupAggregate(select=[SUM(a) AS total_sum], updateAsRetraction=[false], accMode=[Acc])
+   +- Reused(reference_id=[1])
+
+Sink(fields=[total_min], updateAsRetraction=[true], accMode=[AccRetract])
++- GroupAggregate(select=[MIN(a) AS total_min], updateAsRetraction=[true], accMode=[AccRetract])
+   +- Reused(reference_id=[1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testSharedUnionNode">
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalSink(fields=[a, b, c])
++- LogicalProject(a=[$0], b=[$1], c=[$2])
+   +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'%hello%')])
+      +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+
+LogicalSink(fields=[a, b, c])
++- LogicalProject(a=[$0], b=[$1], c=[$2])
+   +- LogicalFilter(condition=[>=($1, 5)])
+      +- LogicalUnion(all=[true])
+         :- LogicalProject(a=[$0], b=[$1], c=[$2])
+         :  +- LogicalProject(a=[$0], b=[$1], c=[$2])
+         :     +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'%hello%')])
+         :        +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+         +- LogicalProject(a=[$0], b=[$1], c=[$2])
+            +- LogicalProject(a=[$0], b=[$1], c=[$2])
+               +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'%world%')])
+                  +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+
+LogicalSink(fields=[b, cnt])
++- LogicalProject(b=[$0], cnt=[$1])
+   +- LogicalFilter(condition=[<($0, 4)])
+      +- LogicalAggregate(group=[{0}], cnt=[COUNT($1)])
+         +- LogicalProject(b=[$1], a=[$0])
+            +- LogicalUnion(all=[true])
+               :- LogicalProject(a=[$0], b=[$1], c=[$2])
+               :  +- LogicalProject(a=[$0], b=[$1], c=[$2])
+               :     +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'%hello%')])
+               :        +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+               +- LogicalProject(a=[$0], b=[$1], c=[$2])
+                  +- LogicalProject(a=[$0], b=[$1], c=[$2])
+                     +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'%world%')])
+                        +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+
+LogicalSink(fields=[b, cnt])
++- LogicalProject(b=[$0], cnt=[$1])
+   +- LogicalFilter(condition=[<($0, 4)])
+      +- LogicalAggregate(group=[{0}], cnt=[COUNT($1)])
+         +- LogicalProject(b=[$1], a=[$0])
+            +- LogicalUnion(all=[true])
+               :- LogicalProject(a=[$0], b=[$1], c=[$2])
+               :  +- LogicalProject(a=[$0], b=[$1], c=[$2])
+               :     +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'%hello%')])
+               :        +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+               +- LogicalProject(a=[$0], b=[$1], c=[$2])
+                  +- LogicalProject(a=[$0], b=[$1], c=[$2])
+                     +- LogicalFilter(condition=[LIKE($2, _UTF-16LE'%world%')])
+                        +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+TableSourceScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], updateAsRetraction=[true], accMode=[Acc], reuse_id=[2])
+
+Calc(select=[a, b, c], where=[LIKE(c, _UTF-16LE'%hello%')], updateAsRetraction=[true], accMode=[Acc], reuse_id=[1])
++- Reused(reference_id=[2])
+
+Sink(fields=[a, b, c], updateAsRetraction=[false], accMode=[Acc])
++- Reused(reference_id=[1])
+
+Calc(select=[a, b, c], where=[LIKE(c, _UTF-16LE'%world%')], updateAsRetraction=[true], accMode=[Acc], reuse_id=[3])
++- Reused(reference_id=[2])
+
+Sink(fields=[a, b, c], updateAsRetraction=[true], accMode=[Acc])
++- Union(all=[true], union=[a, b, c], updateAsRetraction=[true], accMode=[Acc])
+   :- Calc(select=[a, b, c], where=[>=(b, 5)], updateAsRetraction=[true], accMode=[Acc])
+   :  +- Reused(reference_id=[1])
+   +- Calc(select=[a, b, c], where=[>=(b, 5)], updateAsRetraction=[true], accMode=[Acc])
+      +- Reused(reference_id=[3])
+
+GroupAggregate(groupBy=[b], select=[b, COUNT(a) AS cnt], updateAsRetraction=[true], accMode=[AccRetract], reuse_id=[4])
++- Exchange(distribution=[hash[b]], updateAsRetraction=[true], accMode=[Acc])
+   +- Union(all=[true], union=[b, a], updateAsRetraction=[true], accMode=[Acc])
+      :- Calc(select=[b, a], where=[<(b, 4)], updateAsRetraction=[true], accMode=[Acc])
+      :  +- Reused(reference_id=[1])
+      +- Calc(select=[b, a], where=[<(b, 4)], updateAsRetraction=[true], accMode=[Acc])
+         +- Reused(reference_id=[3])
+
+Sink(fields=[b, cnt], updateAsRetraction=[true], accMode=[AccRetract])
++- Reused(reference_id=[4])
+
+Sink(fields=[b, cnt], updateAsRetraction=[false], accMode=[AccRetract])
++- Reused(reference_id=[4])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testSingleSink1">
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalSink(fields=[c, cnt])
++- LogicalAggregate(group=[{0}], cnt=[COUNT($1)])
+   +- LogicalProject(c=[$2], a=[$0])
+      +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+Sink(fields=[c, cnt], updateAsRetraction=[false], accMode=[Acc])
++- GroupAggregate(groupBy=[c], select=[c, COUNT(a) AS cnt], updateAsRetraction=[false], accMode=[Acc])
+   +- Exchange(distribution=[hash[c]], updateAsRetraction=[true], accMode=[Acc])
+      +- Calc(select=[c, a], updateAsRetraction=[true], accMode=[Acc])
+         +- TableSourceScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], updateAsRetraction=[true], accMode=[Acc])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testSingleSink2">
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalSink(fields=[a1, b, c1])
++- LogicalProject(a1=[$2], b=[$3], c1=[$1])
+   +- LogicalFilter(condition=[=($2, $0)])
+      +- LogicalJoin(condition=[true], joinType=[inner])
+         :- LogicalTableScan(table=[[table4]])
+         +- LogicalTableScan(table=[[table5]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+Sink(fields=[a1, b, c1], updateAsRetraction=[false], accMode=[Acc])
++- Calc(select=[a1, b, c1], updateAsRetraction=[false], accMode=[Acc])
+   +- Join(joinType=[InnerJoin], where=[=(a1, a3)], select=[a3, c1, a1, b, c2], updateAsRetraction=[false], accMode=[Acc])
+      :- Exchange(distribution=[hash[a3]], updateAsRetraction=[true], accMode=[Acc])
+      :  +- Calc(select=[a AS a3, c AS c1], where=[AND(>=(a, 0), <(b, 5))], updateAsRetraction=[true], accMode=[Acc])
+      :     +- TableSourceScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], updateAsRetraction=[true], accMode=[Acc], reuse_id=[1])
+      +- Exchange(distribution=[hash[a1]], updateAsRetraction=[true], accMode=[Acc])
+         +- Calc(select=[a1, b, c AS c2], updateAsRetraction=[true], accMode=[Acc])
+            +- Join(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b, a2, c], updateAsRetraction=[true], accMode=[Acc])
+               :- Exchange(distribution=[hash[a1]], updateAsRetraction=[true], accMode=[Acc])
+               :  +- Calc(select=[a AS a1, b], where=[<=(a, 10)], updateAsRetraction=[true], accMode=[Acc])
+               :     +- Reused(reference_id=[1])
+               +- Exchange(distribution=[hash[a2]], updateAsRetraction=[true], accMode=[Acc])
+                  +- Calc(select=[a AS a2, c], where=[AND(>=(a, 0), >=(b, 5))], updateAsRetraction=[true], accMode=[Acc])
+                     +- Reused(reference_id=[1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testSingleSink3">
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalSink(fields=[a1, b1])
++- LogicalUnion(all=[true])
+   :- LogicalProject(a1=[$0], b1=[$1])
+   :  +- LogicalTableScan(table=[[table1]])
+   +- LogicalProject(a=[$0], b1=[$1])
+      +- LogicalTableScan(table=[[table2]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+Sink(fields=[a1, b1], updateAsRetraction=[false], accMode=[Acc])
++- Union(all=[true], union=[a1, b1], updateAsRetraction=[false], accMode=[Acc])
+   :- Calc(select=[a AS a1, b AS b1], where=[<=(a, 10)], updateAsRetraction=[false], accMode=[Acc], reuse_id=[1])
+   :  +- TableSourceScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], updateAsRetraction=[false], accMode=[Acc])
+   +- Calc(select=[a, b1], updateAsRetraction=[false], accMode=[Acc])
+      +- Join(joinType=[InnerJoin], where=[=(a, a1)], select=[a1, b1, a, b, c, d, e], updateAsRetraction=[false], accMode=[Acc])
+         :- Exchange(distribution=[hash[a1]], updateAsRetraction=[true], accMode=[Acc])
+         :  +- Reused(reference_id=[1])
+         +- Exchange(distribution=[hash[a]], updateAsRetraction=[true], accMode=[Acc])
+            +- DataStreamScan(table=[[_DataStreamTable_0]], fields=[a, b, c, d, e], updateAsRetraction=[true], accMode=[Acc])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testSingleSink4">
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalSink(fields=[a, b, c])
++- LogicalProject(a1=[$0], b1=[$3], c1=[$4])
+   +- LogicalFilter(condition=[=($0, $2)])
+      +- LogicalJoin(condition=[true], joinType=[inner])
+         :- LogicalTableScan(table=[[table1]])
+         +- LogicalTableScan(table=[[table6]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+Sink(fields=[a, b, c], updateAsRetraction=[false], accMode=[Acc])
++- Calc(select=[a1, b1, c1], updateAsRetraction=[false], accMode=[Acc])
+   +- Join(joinType=[InnerJoin], where=[=(a1, a3)], select=[a1, a3, b1, c1], updateAsRetraction=[false], accMode=[Acc])
+      :- Exchange(distribution=[hash[a1]], updateAsRetraction=[true], accMode=[Acc])
+      :  +- Calc(select=[a AS a1], where=[<=(a, 10)], updateAsRetraction=[true], accMode=[Acc])
+      :     +- TableSourceScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], updateAsRetraction=[true], accMode=[Acc], reuse_id=[1])
+      +- Exchange(distribution=[hash[a3]], updateAsRetraction=[true], accMode=[Acc])
+         +- Calc(select=[a3, b AS b1, c1], updateAsRetraction=[true], accMode=[Acc])
+            +- Join(joinType=[InnerJoin], where=[=(a1, a3)], select=[a3, c1, a1, b, c2], updateAsRetraction=[true], accMode=[Acc])
+               :- Exchange(distribution=[hash[a3]], updateAsRetraction=[true], accMode=[Acc])
+               :  +- Calc(select=[a AS a3, c AS c1], where=[AND(>=(a, 0), <(b, 5))], updateAsRetraction=[true], accMode=[Acc])
+               :     +- Reused(reference_id=[1])
+               +- Exchange(distribution=[hash[a1]], updateAsRetraction=[true], accMode=[Acc])
+                  +- Calc(select=[a1, b, c AS c2], updateAsRetraction=[true], accMode=[Acc])
+                     +- Join(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b, a2, c], updateAsRetraction=[true], accMode=[Acc])
+                        :- Exchange(distribution=[hash[a1]], updateAsRetraction=[true], accMode=[Acc])
+                        :  +- Calc(select=[a AS a1, b], where=[<=(a, 10)], updateAsRetraction=[true], accMode=[Acc])
+                        :     +- Reused(reference_id=[1])
+                        +- Exchange(distribution=[hash[a2]], updateAsRetraction=[true], accMode=[Acc])
+                           +- Calc(select=[a AS a2, c], where=[AND(>=(a, 0), >=(b, 5))], updateAsRetraction=[true], accMode=[Acc])
+                              +- Reused(reference_id=[1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testSingleSinkWithUDTF">
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalSink(fields=[a, b, c, d, e, f, i, j, k, l, m, s])
++- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5], i=[$6], j=[$7], k=[$8], l=[$9], m=[$10], s=[$11])
+   +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{2}])
+      :- LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5], i=[$6], j=[$7], k=[$8], l=[$9], m=[$10])
+      :  +- LogicalFilter(condition=[AND(=($1, $4), =($0, $6))])
+      :     +- LogicalJoin(condition=[true], joinType=[inner])
+      :        :- LogicalJoin(condition=[true], joinType=[inner])
+      :        :  :- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+      :        :  +- LogicalTableScan(table=[[MyTable1, source: [TestTableSource(d, e, f)]]])
+      :        +- LogicalTableScan(table=[[MyTable2, source: [TestTableSource(i, j, k, l, m)]]])
+      +- LogicalTableFunctionScan(invocation=[split($cor0.c)], rowType=[RecordType(VARCHAR(65536) f0)], elementType=[class [Ljava.lang.Object;])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+Sink(fields=[a, b, c, d, e, f, i, j, k, l, m, s], updateAsRetraction=[false], accMode=[Acc])
++- Correlate(invocation=[split($cor0.c)], correlate=[table(split($cor0.c))], select=[a,b,c,d,e,f,i,j,k,l,m,f0], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(65536) c, INTEGER d, BIGINT e, VARCHAR(65536) f, INTEGER i, BIGINT j, INTEGER k, VARCHAR(65536) l, BIGINT m, VARCHAR(65536) f0)], joinType=[INNER], updateAsRetraction=[false], accMode=[Acc])
+   +- Join(joinType=[InnerJoin], where=[=(a, i)], select=[a, b, c, d, e, f, i, j, k, l, m], updateAsRetraction=[false], accMode=[Acc])
+      :- Exchange(distribution=[hash[a]], updateAsRetraction=[true], accMode=[Acc])
+      :  +- Join(joinType=[InnerJoin], where=[=(b, e)], select=[a, b, c, d, e, f], updateAsRetraction=[true], accMode=[Acc])
+      :     :- Exchange(distribution=[hash[b]], updateAsRetraction=[true], accMode=[Acc])
+      :     :  +- TableSourceScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], updateAsRetraction=[true], accMode=[Acc])
+      :     +- Exchange(distribution=[hash[e]], updateAsRetraction=[true], accMode=[Acc])
+      :        +- TableSourceScan(table=[[MyTable1, source: [TestTableSource(d, e, f)]]], fields=[d, e, f], updateAsRetraction=[true], accMode=[Acc])
+      +- Exchange(distribution=[hash[i]], updateAsRetraction=[true], accMode=[Acc])
+         +- TableSourceScan(table=[[MyTable2, source: [TestTableSource(i, j, k, l, m)]]], fields=[i, j, k, l, m], updateAsRetraction=[true], accMode=[Acc])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testSingleSinkSplitOnUnion">
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalSink(fields=[total_sum])
++- LogicalAggregate(group=[{}], total_sum=[SUM($0)])
+   +- LogicalProject(a=[$0])
+      +- LogicalUnion(all=[true])
+         :- LogicalProject(a=[$0], c=[$2])
+         :  +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+         +- LogicalProject(d=[$0], f=[$2])
+            +- LogicalTableScan(table=[[MyTable1, source: [TestTableSource(d, e, f)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+Sink(fields=[total_sum], updateAsRetraction=[true], accMode=[AccRetract])
++- GroupAggregate(select=[SUM(a) AS total_sum], updateAsRetraction=[true], accMode=[AccRetract])
+   +- Exchange(distribution=[single], updateAsRetraction=[true], accMode=[Acc])
+      +- Union(all=[true], union=[a], updateAsRetraction=[true], accMode=[Acc])
+         :- Calc(select=[a], updateAsRetraction=[true], accMode=[Acc])
+         :  +- TableSourceScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], updateAsRetraction=[true], accMode=[Acc])
+         +- Calc(select=[d AS a], updateAsRetraction=[true], accMode=[Acc])
+            +- TableSourceScan(table=[[MyTable1, source: [TestTableSource(d, e, f)]]], fields=[d, e, f], updateAsRetraction=[true], accMode=[Acc])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testUpdateAsRetractConsumedAtSourceBlock">
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalSink(fields=[a])
++- LogicalProject(a=[$0])
+   +- LogicalFilter(condition=[>($0, 6)])
+      +- LogicalProject(a=[$0], b=[$1], c=[$2], rank_num=[$3])
+         +- LogicalFilter(condition=[<=($3, 10)])
+            +- LogicalProject(a=[$0], b=[$1], c=[$2], rank_num=[ROW_NUMBER() OVER (PARTITION BY $1 ORDER BY $2 DESC NULLS LAST ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)])
+               +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+
+LogicalSink(fields=[a, b])
++- LogicalProject(a=[$0], b=[$1])
+   +- LogicalFilter(condition=[<($0, 6)])
+      +- LogicalProject(a=[$0], b=[$1], c=[$2], rank_num=[$3])
+         +- LogicalFilter(condition=[<=($3, 10)])
+            +- LogicalProject(a=[$0], b=[$1], c=[$2], rank_num=[ROW_NUMBER() OVER (PARTITION BY $1 ORDER BY $2 DESC NULLS LAST ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)])
+               +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+Rank(strategy=[AppendFastStrategy], rankType=[ROW_NUMBER], rankRange=[rankStart=1, rankEnd=10], partitionBy=[b], orderBy=[c DESC], select=[a, b, c, w0$o0], updateAsRetraction=[true], accMode=[AccRetract], reuse_id=[1])
++- Exchange(distribution=[hash[b]], updateAsRetraction=[false], accMode=[Acc])
+   +- TableSourceScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], updateAsRetraction=[false], accMode=[Acc])
+
+Sink(fields=[a], updateAsRetraction=[true], accMode=[AccRetract])
++- Calc(select=[a], where=[>(a, 6)], updateAsRetraction=[true], accMode=[AccRetract])
+   +- Reused(reference_id=[1])
+
+Sink(fields=[a, b], updateAsRetraction=[false], accMode=[AccRetract])
++- Calc(select=[a, b], where=[<(a, 6)], updateAsRetraction=[false], accMode=[AccRetract])
+   +- Reused(reference_id=[1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testUnionAndAggWithDifferentGroupings">
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalSink(fields=[b, c, a_sum])
++- LogicalUnion(all=[true])
+   :- LogicalAggregate(group=[{0, 1}], a_sum=[SUM($2)])
+   :  +- LogicalProject(b=[$1], c=[$2], a=[$0])
+   :     +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+   +- LogicalProject(b=[1], c=[$0], a_sum=[$1])
+      +- LogicalAggregate(group=[{0}], a_sum=[SUM($1)])
+         +- LogicalProject(c=[$2], a=[$0])
+            +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+Sink(fields=[b, c, a_sum], updateAsRetraction=[false], accMode=[Acc])
++- Union(all=[true], union=[b, c, a_sum], updateAsRetraction=[false], accMode=[Acc])
+   :- GroupAggregate(groupBy=[b, c], select=[b, c, SUM(a) AS a_sum], updateAsRetraction=[false], accMode=[Acc])
+   :  +- Exchange(distribution=[hash[b, c]], updateAsRetraction=[true], accMode=[Acc])
+   :     +- TableSourceScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], updateAsRetraction=[true], accMode=[Acc], reuse_id=[1])
+   +- Calc(select=[CAST(1:BIGINT) AS b, c, a_sum], updateAsRetraction=[false], accMode=[Acc])
+      +- GroupAggregate(groupBy=[c], select=[c, SUM(a) AS a_sum], updateAsRetraction=[false], accMode=[Acc])
+         +- Exchange(distribution=[hash[c]], updateAsRetraction=[true], accMode=[Acc])
+            +- Calc(select=[c, a], updateAsRetraction=[true], accMode=[Acc])
+               +- Reused(reference_id=[1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testUpdateAsRetractConsumedAtSinkBlock">
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalSink(fields=[a, b, c])
++- LogicalProject(a=[$0], b=[$1], c=[$2], rank_num=[$3])
+   +- LogicalFilter(condition=[<=($3, 10)])
+      +- LogicalProject(a=[$0], b=[$1], c=[$2], rank_num=[ROW_NUMBER() OVER (PARTITION BY $1 ORDER BY $2 DESC NULLS LAST ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)])
+         +- LogicalProject(a=[$0], b=[$1], c=[$2])
+            +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+
+LogicalSink(fields=[a, b])
++- LogicalProject(a=[$0], b=[$1])
+   +- LogicalFilter(condition=[<($0, 6)])
+      +- LogicalProject(a=[$0], b=[$1], c=[$2])
+         +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+TableSourceScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], updateAsRetraction=[false], accMode=[Acc], reuse_id=[1])
+
+Sink(fields=[a, b, c], updateAsRetraction=[true], accMode=[AccRetract])
++- Rank(strategy=[AppendFastStrategy], rankType=[ROW_NUMBER], rankRange=[rankStart=1, rankEnd=10], partitionBy=[b], orderBy=[c DESC], select=[a, b, c, w0$o0], updateAsRetraction=[true], accMode=[AccRetract])
+   +- Exchange(distribution=[hash[b]], updateAsRetraction=[false], accMode=[Acc])
+      +- Reused(reference_id=[1])
+
+Sink(fields=[a, b], updateAsRetraction=[false], accMode=[Acc])
++- Calc(select=[a, b], where=[<(a, 6)], updateAsRetraction=[false], accMode=[Acc])
+   +- Reused(reference_id=[1])
+]]>
+    </Resource>
+  </TestCase>
+</Root>
diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/stream/sql/SinkTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/stream/sql/SinkTest.xml
new file mode 100644
index 0000000..22e014d
--- /dev/null
+++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/stream/sql/SinkTest.xml
@@ -0,0 +1,284 @@
+<?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="testAppendSink">
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalSink(fields=[a])
++- LogicalProject(cnt=[$1])
+   +- LogicalAggregate(group=[{0}], cnt=[COUNT()])
+      +- LogicalProject(a=[$0])
+         +- LogicalTableScan(table=[[MyTable]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+Sink(fields=[a], updateAsRetraction=[false], accMode=[Acc])
++- Calc(select=[cnt], updateAsRetraction=[false], accMode=[Acc])
+   +- GroupAggregate(groupBy=[a], select=[a, COUNT(*) AS cnt], updateAsRetraction=[false], accMode=[Acc])
+      +- Exchange(distribution=[hash[a]], updateAsRetraction=[true], accMode=[Acc])
+         +- Calc(select=[a], updateAsRetraction=[true], accMode=[Acc])
+            +- DataStreamScan(table=[[_DataStreamTable_0]], fields=[a, b, c], updateAsRetraction=[true], accMode=[Acc])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testUpsertAndUpsertSink">
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalSink(fields=[b, cnt])
++- LogicalAggregate(group=[{0}], frequency=[COUNT($1)])
+   +- LogicalProject(cnt=[$1], b=[$0])
+      +- LogicalFilter(condition=[<($0, 4)])
+         +- LogicalAggregate(group=[{0}], cnt=[COUNT($1)])
+            +- LogicalProject(b=[$1], a=[$0])
+               +- LogicalTableScan(table=[[_DataStreamTable_0]])
+
+LogicalSink(fields=[b, cnt])
++- LogicalProject(b=[$0], cnt=[$1])
+   +- LogicalFilter(condition=[AND(>=($0, 4), <($0, 6))])
+      +- LogicalAggregate(group=[{0}], cnt=[COUNT($1)])
+         +- LogicalProject(b=[$1], a=[$0])
+            +- LogicalTableScan(table=[[_DataStreamTable_0]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+GroupAggregate(groupBy=[b], select=[b, COUNT(a) AS cnt], updateAsRetraction=[true], accMode=[AccRetract], reuse_id=[1])
++- Exchange(distribution=[hash[b]], updateAsRetraction=[true], accMode=[Acc])
+   +- Calc(select=[b, a], updateAsRetraction=[true], accMode=[Acc])
+      +- DataStreamScan(table=[[_DataStreamTable_0]], fields=[a, b, c], updateAsRetraction=[true], accMode=[Acc])
+
+Sink(fields=[b, cnt], updateAsRetraction=[false], accMode=[Acc])
++- GroupAggregate(groupBy=[cnt], select=[cnt, COUNT_RETRACT(b) AS frequency], updateAsRetraction=[false], accMode=[Acc])
+   +- Exchange(distribution=[hash[cnt]], updateAsRetraction=[true], accMode=[AccRetract])
+      +- Calc(select=[b, cnt], where=[<(b, 4)], updateAsRetraction=[true], accMode=[AccRetract])
+         +- Reused(reference_id=[1])
+
+Sink(fields=[b, cnt], updateAsRetraction=[false], accMode=[AccRetract])
++- Calc(select=[b, cnt], where=[AND(>=(b, 4), <(b, 6))], updateAsRetraction=[false], accMode=[AccRetract])
+   +- Reused(reference_id=[1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testAppendUpsertAndRetractSink">
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalSink(fields=[a, b])
++- LogicalUnion(all=[true])
+   :- LogicalProject(a=[$0], b=[$1])
+   :  +- LogicalTableScan(table=[[_DataStreamTable_0]])
+   +- LogicalProject(d=[$0], e=[$1])
+      +- LogicalTableScan(table=[[_DataStreamTable_1]])
+
+LogicalSink(fields=[total_sum])
++- LogicalAggregate(group=[{}], total_sum=[SUM($0)])
+   +- LogicalProject(a=[$0])
+      +- LogicalUnion(all=[true])
+         :- LogicalProject(a=[$0], b=[$1])
+         :  +- LogicalUnion(all=[true])
+         :     :- LogicalProject(a=[$0], b=[$1])
+         :     :  +- LogicalTableScan(table=[[_DataStreamTable_0]])
+         :     +- LogicalProject(d=[$0], e=[$1])
+         :        +- LogicalTableScan(table=[[_DataStreamTable_1]])
+         +- LogicalProject(i=[$0], j=[$1])
+            +- LogicalTableScan(table=[[_DataStreamTable_2]])
+
+LogicalSink(fields=[total_min])
++- LogicalAggregate(group=[{}], total_min=[MIN($0)])
+   +- LogicalProject(a=[$0])
+      +- LogicalUnion(all=[true])
+         :- LogicalProject(a=[$0], b=[$1])
+         :  +- LogicalUnion(all=[true])
+         :     :- LogicalProject(a=[$0], b=[$1])
+         :     :  +- LogicalTableScan(table=[[_DataStreamTable_0]])
+         :     +- LogicalProject(d=[$0], e=[$1])
+         :        +- LogicalTableScan(table=[[_DataStreamTable_1]])
+         +- LogicalProject(i=[$0], j=[$1])
+            +- LogicalTableScan(table=[[_DataStreamTable_2]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+Union(all=[true], union=[a, b], updateAsRetraction=[true], accMode=[Acc], reuse_id=[1])
+:- Calc(select=[a, b], updateAsRetraction=[true], accMode=[Acc])
+:  +- DataStreamScan(table=[[_DataStreamTable_0]], fields=[a, b, c], updateAsRetraction=[true], accMode=[Acc])
++- Calc(select=[d, e], updateAsRetraction=[true], accMode=[Acc])
+   +- DataStreamScan(table=[[_DataStreamTable_1]], fields=[d, e, f], updateAsRetraction=[true], accMode=[Acc])
+
+Sink(fields=[a, b], updateAsRetraction=[false], accMode=[Acc])
++- Reused(reference_id=[1])
+
+Exchange(distribution=[single], updateAsRetraction=[true], accMode=[Acc], reuse_id=[2])
++- Calc(select=[a], updateAsRetraction=[true], accMode=[Acc])
+   +- Union(all=[true], union=[a, b], updateAsRetraction=[true], accMode=[Acc])
+      :- Reused(reference_id=[1])
+      +- Calc(select=[i, j], updateAsRetraction=[true], accMode=[Acc])
+         +- DataStreamScan(table=[[_DataStreamTable_2]], fields=[i, j, k], updateAsRetraction=[true], accMode=[Acc])
+
+Sink(fields=[total_sum], updateAsRetraction=[true], accMode=[AccRetract])
++- GroupAggregate(select=[SUM(a) AS total_sum], updateAsRetraction=[true], accMode=[AccRetract])
+   +- Reused(reference_id=[2])
+
+Sink(fields=[total_min], updateAsRetraction=[false], accMode=[Acc])
++- GroupAggregate(select=[MIN(a) AS total_min], updateAsRetraction=[false], accMode=[Acc])
+   +- Reused(reference_id=[2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testRetractAndUpsertSink">
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalSink(fields=[b, cnt])
++- LogicalProject(b=[$0], cnt=[$1])
+   +- LogicalFilter(condition=[<($0, 4)])
+      +- LogicalAggregate(group=[{0}], cnt=[COUNT($1)])
+         +- LogicalProject(b=[$1], a=[$0])
+            +- LogicalTableScan(table=[[_DataStreamTable_0]])
+
+LogicalSink(fields=[b, cnt])
++- LogicalProject(b=[$0], cnt=[$1])
+   +- LogicalFilter(condition=[AND(>=($0, 4), <($0, 6))])
+      +- LogicalAggregate(group=[{0}], cnt=[COUNT($1)])
+         +- LogicalProject(b=[$1], a=[$0])
+            +- LogicalTableScan(table=[[_DataStreamTable_0]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+GroupAggregate(groupBy=[b], select=[b, COUNT(a) AS cnt], updateAsRetraction=[true], accMode=[AccRetract], reuse_id=[1])
++- Exchange(distribution=[hash[b]], updateAsRetraction=[true], accMode=[Acc])
+   +- Calc(select=[b, a], updateAsRetraction=[true], accMode=[Acc])
+      +- DataStreamScan(table=[[_DataStreamTable_0]], fields=[a, b, c], updateAsRetraction=[true], accMode=[Acc])
+
+Sink(fields=[b, cnt], updateAsRetraction=[true], accMode=[AccRetract])
++- Calc(select=[b, cnt], where=[<(b, 4)], updateAsRetraction=[true], accMode=[AccRetract])
+   +- Reused(reference_id=[1])
+
+Sink(fields=[b, cnt], updateAsRetraction=[false], accMode=[AccRetract])
++- Calc(select=[b, cnt], where=[AND(>=(b, 4), <(b, 6))], updateAsRetraction=[false], accMode=[AccRetract])
+   +- Reused(reference_id=[1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testRetractSink1">
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalSink(fields=[a, cnt])
++- LogicalAggregate(group=[{0}], cnt=[COUNT()])
+   +- LogicalProject(a=[$0])
+      +- LogicalTableScan(table=[[MyTable]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+Sink(fields=[a, cnt], updateAsRetraction=[true], accMode=[AccRetract])
++- GroupAggregate(groupBy=[a], select=[a, COUNT(*) AS cnt], updateAsRetraction=[true], accMode=[AccRetract])
+   +- Exchange(distribution=[hash[a]], updateAsRetraction=[true], accMode=[Acc])
+      +- Calc(select=[a], updateAsRetraction=[true], accMode=[Acc])
+         +- DataStreamScan(table=[[_DataStreamTable_0]], fields=[a, b, c], updateAsRetraction=[true], accMode=[Acc])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testRetractSink2">
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalSink(fields=[cnt, a])
++- LogicalAggregate(group=[{0}], a=[COUNT($1)])
+   +- LogicalProject(cnt=[$1], a=[$0])
+      +- LogicalAggregate(group=[{0}], cnt=[COUNT()])
+         +- LogicalProject(a=[$0])
+            +- LogicalTableScan(table=[[MyTable]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+Sink(fields=[cnt, a], updateAsRetraction=[true], accMode=[AccRetract])
++- GroupAggregate(groupBy=[cnt], select=[cnt, COUNT_RETRACT(a) AS a], updateAsRetraction=[true], accMode=[AccRetract])
+   +- Exchange(distribution=[hash[cnt]], updateAsRetraction=[true], accMode=[AccRetract])
+      +- GroupAggregate(groupBy=[a], select=[a, COUNT(*) AS cnt], updateAsRetraction=[true], accMode=[AccRetract])
+         +- Exchange(distribution=[hash[a]], updateAsRetraction=[true], accMode=[Acc])
+            +- Calc(select=[a], updateAsRetraction=[true], accMode=[Acc])
+               +- DataStreamScan(table=[[_DataStreamTable_0]], fields=[a, b, c], updateAsRetraction=[true], accMode=[Acc])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testUpsertSink1">
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalSink(fields=[a, cnt])
++- LogicalAggregate(group=[{0}], cnt=[COUNT()])
+   +- LogicalProject(a=[$0])
+      +- LogicalTableScan(table=[[MyTable]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+Sink(fields=[a, cnt], updateAsRetraction=[false], accMode=[Acc])
++- GroupAggregate(groupBy=[a], select=[a, COUNT(*) AS cnt], updateAsRetraction=[false], accMode=[Acc])
+   +- Exchange(distribution=[hash[a]], updateAsRetraction=[true], accMode=[Acc])
+      +- Calc(select=[a], updateAsRetraction=[true], accMode=[Acc])
+         +- DataStreamScan(table=[[_DataStreamTable_0]], fields=[a, b, c], updateAsRetraction=[true], accMode=[Acc])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testUpsertSink2">
+    <Resource name="planBefore">
+      <![CDATA[
+LogicalSink(fields=[a1, b, c1])
++- LogicalProject(a1=[$2], b=[$3], c1=[$1])
+   +- LogicalFilter(condition=[=($2, $0)])
+      +- LogicalJoin(condition=[true], joinType=[inner])
+         :- LogicalProject(a3=[$0], c1=[$2])
+         :  +- LogicalFilter(condition=[<($1, 5)])
+         :     +- LogicalProject(a=[$0], b=[$1], c=[$2])
+         :        +- LogicalFilter(condition=[>=($0, 0)])
+         :           +- LogicalTableScan(table=[[MyTable]])
+         +- LogicalProject(a1=[$0], b=[$1], c2=[$3])
+            +- LogicalFilter(condition=[=($0, $2)])
+               +- LogicalJoin(condition=[true], joinType=[inner])
+                  :- LogicalProject(a1=[$0], b=[$1])
+                  :  +- LogicalFilter(condition=[<=($0, 10)])
+                  :     +- LogicalTableScan(table=[[MyTable]])
+                  +- LogicalProject(a2=[$0], c=[$2])
+                     +- LogicalFilter(condition=[>=($1, 5)])
+                        +- LogicalProject(a=[$0], b=[$1], c=[$2])
+                           +- LogicalFilter(condition=[>=($0, 0)])
+                              +- LogicalTableScan(table=[[MyTable]])
+]]>
+    </Resource>
+    <Resource name="planAfter">
+      <![CDATA[
+Sink(fields=[a1, b, c1], updateAsRetraction=[false], accMode=[Acc])
++- Calc(select=[a1, b, c1], updateAsRetraction=[false], accMode=[Acc])
+   +- Join(joinType=[InnerJoin], where=[=(a1, a3)], select=[a3, c1, a1, b, c2], updateAsRetraction=[false], accMode=[Acc])
+      :- Exchange(distribution=[hash[a3]], updateAsRetraction=[true], accMode=[Acc])
+      :  +- Calc(select=[a AS a3, c AS c1], where=[AND(>=(a, 0), <(b, 5))], updateAsRetraction=[true], accMode=[Acc])
+      :     +- DataStreamScan(table=[[_DataStreamTable_0]], fields=[a, b, c], updateAsRetraction=[true], accMode=[Acc], reuse_id=[1])
+      +- Exchange(distribution=[hash[a1]], updateAsRetraction=[true], accMode=[Acc])
+         +- Calc(select=[a1, b, c AS c2], updateAsRetraction=[true], accMode=[Acc])
+            +- Join(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b, a2, c], updateAsRetraction=[true], accMode=[Acc])
+               :- Exchange(distribution=[hash[a1]], updateAsRetraction=[true], accMode=[Acc])
+               :  +- Calc(select=[a AS a1, b], where=[<=(a, 10)], updateAsRetraction=[true], accMode=[Acc])
+               :     +- Reused(reference_id=[1])
+               +- Exchange(distribution=[hash[a2]], updateAsRetraction=[true], accMode=[Acc])
+                  +- Calc(select=[a AS a2, c], where=[AND(>=(a, 0), >=(b, 5))], updateAsRetraction=[true], accMode=[Acc])
+                     +- Reused(reference_id=[1])
+]]>
+    </Resource>
+  </TestCase>
+</Root>
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/batch/ExplainTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/batch/ExplainTest.scala
index abb87e1..f7e6c40 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/batch/ExplainTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/batch/ExplainTest.scala
@@ -19,18 +19,89 @@
 package org.apache.flink.table.api.batch
 
 import org.apache.flink.api.scala._
+import org.apache.flink.table.`type`.InternalTypes.{INT, LONG, STRING}
+import org.apache.flink.table.api.TableConfigOptions
 import org.apache.flink.table.util.TableTestBase
 
 import org.junit.Test
+import org.junit.runner.RunWith
+import org.junit.runners.Parameterized
 
-class ExplainTest extends TableTestBase {
+@RunWith(classOf[Parameterized])
+class ExplainTest(extended: Boolean) extends TableTestBase {
 
   private val util = batchTestUtil()
-  util.addDataStream[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
+  util.addTableSource[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
+  util.addDataStream[(Int, Long, String)]("MyTable1", 'a, 'b, 'c)
+  util.addDataStream[(Int, Long, String)]("MyTable2", 'd, 'e, 'f)
 
   @Test
-  def testExplainTableSourceScan(): Unit = {
-    util.verifyExplain("SELECT * FROM MyTable")
+  def testExplainWithTableSourceScan(): Unit = {
+    util.verifyExplain("SELECT * FROM MyTable", extended)
   }
 
+  @Test
+  def testExplainWithDataStreamScan(): Unit = {
+    util.verifyExplain("SELECT * FROM MyTable1", extended)
+  }
+
+  @Test
+  def testExplainWithFilter(): Unit = {
+    util.verifyExplain("SELECT * FROM MyTable1 WHERE mod(a, 2) = 0", extended)
+  }
+
+  @Test
+  def testExplainWithAgg(): Unit = {
+    util.verifyExplain("SELECT COUNT(*) FROM MyTable1 GROUP BY a", extended)
+  }
+
+  @Test
+  def testExplainWithJoin(): Unit = {
+    // TODO support other join operators when them are supported
+    util.tableEnv.getConfig.getConf.setString(
+      TableConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin, NestedLoopJoin")
+    util.verifyExplain("SELECT a, b, c, e, f FROM MyTable1, MyTable2 WHERE a = d", extended)
+  }
+
+  @Test
+  def testExplainWithUnion(): Unit = {
+    util.verifyExplain("SELECT * FROM MyTable1 UNION ALL SELECT * FROM MyTable2", extended)
+  }
+
+  @Test
+  def testExplainWithSort(): Unit = {
+    util.verifyExplain("SELECT * FROM MyTable1 ORDER BY a LIMIT 5", extended)
+  }
+
+  @Test
+  def testExplainWithSingleSink(): Unit = {
+    val table = util.tableEnv.sqlQuery("SELECT * FROM MyTable1 WHERE a > 10")
+    val sink = util.createCollectTableSink(Array("a", "b", "c"), Array(INT, LONG, STRING))
+    util.tableEnv.writeToSink(table, sink)
+    util.verifyExplain(extended)
+  }
+
+  @Test
+  def testExplainWithMultiSinks(): Unit = {
+    val table = util.tableEnv.sqlQuery("SELECT a, COUNT(*) AS cnt FROM MyTable1 GROUP BY a")
+    util.tableEnv.registerTable("TempTable", table)
+
+    val table1 = util.tableEnv.sqlQuery("SELECT * FROM TempTable WHERE cnt > 10")
+    val sink1 = util.createCollectTableSink(Array("a", "cnt"), Array(INT, LONG))
+    util.tableEnv.writeToSink(table1, sink1, "sink1")
+
+    val table2 = util.tableEnv.sqlQuery("SELECT * FROM TempTable WHERE cnt < 10")
+    val sink2 = util.createCollectTableSink(Array("a", "cnt"), Array(INT, LONG))
+    util.tableEnv.writeToSink(table2, sink2, "sink1")
+
+    util.verifyExplain(extended)
+  }
+
+}
+
+object ExplainTest {
+  @Parameterized.Parameters(name = "extended={0}")
+  def parameters(): java.util.Collection[Boolean] = {
+    java.util.Arrays.asList(true, false)
+  }
 }
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/stream/ExplainTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/stream/ExplainTest.scala
index 96f7a3a..aaac061 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/stream/ExplainTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/stream/ExplainTest.scala
@@ -19,24 +19,88 @@
 package org.apache.flink.table.api.stream
 
 import org.apache.flink.api.scala._
+import org.apache.flink.table.`type`.InternalTypes.{INT, LONG, STRING}
+import org.apache.flink.table.api.TableException
 import org.apache.flink.table.util.TableTestBase
 
 import org.junit.Test
+import org.junit.runner.RunWith
+import org.junit.runners.Parameterized
 
-class ExplainTest extends TableTestBase {
+@RunWith(classOf[Parameterized])
+class ExplainTest(extended: Boolean) extends TableTestBase {
 
   private val util = streamTestUtil()
-  util.addDataStream[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
+  util.addTableSource[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
+  util.addDataStream[(Int, Long, String)]("MyTable1", 'a, 'b, 'c)
+  util.addDataStream[(Int, Long, String)]("MyTable2", 'd, 'e, 'f)
 
   @Test
   def testExplainTableSourceScan(): Unit = {
-    util.verifyExplain("SELECT * FROM MyTable")
+    util.verifyExplain("SELECT * FROM MyTable", extended)
   }
 
   @Test
   def testExplainDataStreamScan(): Unit = {
-    util.addDataStream[(Int, Long, String)]("DataStreamTable", 'a, 'b, 'c)
-    util.verifyExplain("SELECT * FROM DataStreamTable")
+    util.verifyExplain("SELECT * FROM MyTable1", extended)
   }
 
+  @Test
+  def testExplainWithFilter(): Unit = {
+    util.verifyExplain("SELECT * FROM MyTable1 WHERE mod(a, 2) = 0", extended)
+  }
+
+  @Test
+  def testExplainWithAgg(): Unit = {
+    util.verifyExplain("SELECT COUNT(*) FROM MyTable1 GROUP BY a", extended)
+  }
+
+  @Test(expected = classOf[TableException])
+  // TODO remove expected TableException after translateToPlanInternal method is implemented
+  //  in StreamExecJoin
+  def testExplainWithJoin(): Unit = {
+    util.verifyExplain("SELECT a, b, c, e, f FROM MyTable1, MyTable2 WHERE a = d", extended)
+  }
+
+  @Test
+  def testExplainWithUnion(): Unit = {
+    util.verifyExplain("SELECT * FROM MyTable1 UNION ALL SELECT * FROM MyTable2", extended)
+  }
+
+  @Test
+  def testExplainWithSort(): Unit = {
+    util.verifyExplain("SELECT * FROM MyTable1 ORDER BY a LIMIT 5", extended)
+  }
+
+  @Test
+  def testExplainWithSingleSink(): Unit = {
+    val table = util.tableEnv.sqlQuery("SELECT * FROM MyTable1 WHERE a > 10")
+    val appendSink = util.createAppendTableSink(Array("a", "b", "c"), Array(INT, LONG, STRING))
+    util.tableEnv.writeToSink(table, appendSink)
+    util.verifyExplain(extended)
+  }
+
+  @Test
+  def testExplainWithMultiSinks(): Unit = {
+    val table = util.tableEnv.sqlQuery("SELECT a, COUNT(*) AS cnt FROM MyTable1 GROUP BY a")
+    util.tableEnv.registerTable("TempTable", table)
+
+    val table1 = util.tableEnv.sqlQuery("SELECT * FROM TempTable WHERE cnt > 10")
+    val upsertSink1 = util.createUpsertTableSink(Array(0), Array("a", "cnt"), Array(INT, LONG))
+    util.tableEnv.writeToSink(table1, upsertSink1, "sink1")
+
+    val table2 = util.tableEnv.sqlQuery("SELECT * FROM TempTable WHERE cnt < 10")
+    val upsertSink2 = util.createUpsertTableSink(Array(0), Array("a", "cnt"), Array(INT, LONG))
+    util.tableEnv.writeToSink(table2, upsertSink2, "sink1")
+
+    util.verifyExplain(extended)
+  }
+
+}
+
+object ExplainTest {
+  @Parameterized.Parameters(name = "extended={0}")
+  def parameters(): java.util.Collection[Boolean] = {
+    java.util.Arrays.asList(true, false)
+  }
 }
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/DagOptimizationTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/DagOptimizationTest.scala
new file mode 100644
index 0000000..3b75370
--- /dev/null
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/DagOptimizationTest.scala
@@ -0,0 +1,464 @@
+/*
+ * 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.plan.batch.sql
+
+import org.apache.flink.api.scala._
+import org.apache.flink.table.`type`.InternalTypes.{DOUBLE, INT, LONG, STRING, TIMESTAMP}
+import org.apache.flink.table.api.PlannerConfigOptions
+import org.apache.flink.table.runtime.utils.JavaUserDefinedScalarFunctions.NonDeterministicUdf
+import org.apache.flink.table.util.{TableFunc1, TableTestBase}
+
+import org.junit.Test
+
+import java.sql.Timestamp
+
+class DagOptimizationTest extends TableTestBase {
+  private val util = batchTestUtil()
+  util.addTableSource[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
+  util.addTableSource[(Int, Long, String)]("MyTable1", 'd, 'e, 'f)
+
+  @Test
+  def testSingleSink1(): Unit = {
+    val table = util.tableEnv.sqlQuery("SELECT c, COUNT(a) AS cnt FROM MyTable GROUP BY c")
+    val appendSink = util.createCollectTableSink(Array("c", "cnt"), Array(STRING, LONG))
+    util.tableEnv.writeToSink(table, appendSink)
+    util.verifyPlan()
+  }
+
+  @Test
+  def testSingleSink2(): Unit = {
+    val table1 = util.tableEnv.sqlQuery("SELECT a as a1, b FROM MyTable WHERE a <= 10")
+    util.tableEnv.registerTable("table1", table1)
+    val table2 = util.tableEnv.sqlQuery("SELECT a, b, c FROM MyTable WHERE a >= 0")
+    util.tableEnv.registerTable("table2", table2)
+    val table3 = util.tableEnv.sqlQuery("SELECT a AS a2, c FROM table2 WHERE b >= 5")
+    util.tableEnv.registerTable("table3", table3)
+    val table4 = util.tableEnv.sqlQuery("SELECT a AS a3, c as c1 FROM table2 WHERE b < 5")
+    util.tableEnv.registerTable("table4", table4)
+    val table5 = util.tableEnv.sqlQuery("SELECT a1, b, c as c2 FROM table1, table3 WHERE a1 = a2")
+    util.tableEnv.registerTable("table5", table5)
+    val table6 = util.tableEnv.sqlQuery("SELECT a1, b, c1 FROM table4, table5 WHERE a1 = a3")
+
+    val appendSink = util.createCollectTableSink(Array("a1", "b", "c1"), Array(INT, LONG, STRING))
+    util.tableEnv.writeToSink(table6, appendSink)
+    util.verifyPlan()
+  }
+
+  @Test
+  def testSingleSink3(): Unit = {
+    util.addDataStream[(Int, Long, String, Double, Boolean)]("MyTable2", 'a, 'b, 'c, 'd, 'e)
+    val table1 = util.tableEnv.sqlQuery("SELECT a AS a1, b as b1 FROM MyTable WHERE a <= 10")
+    util.tableEnv.registerTable("table1", table1)
+    val table2 = util.tableEnv.sqlQuery("SELECT a, b1 FROM table1, MyTable2 WHERE a = a1")
+    util.tableEnv.registerTable("table2", table2)
+    val table3 = util.tableEnv.sqlQuery("SELECT * FROM table1 UNION ALL SELECT * FROM table2")
+
+    val appendSink = util.createCollectTableSink(Array("a1", "b1"), Array(INT, LONG))
+    util.tableEnv.writeToSink(table3, appendSink)
+    util.verifyPlan()
+  }
+
+  @Test
+  def testSingleSink4(): Unit = {
+    val table1 = util.tableEnv.sqlQuery("SELECT a as a1, b FROM MyTable WHERE a <= 10")
+    util.tableEnv.registerTable("table1", table1)
+    val table2 = util.tableEnv.sqlQuery("SELECT a, b, c FROM MyTable WHERE a >= 0")
+    util.tableEnv.registerTable("table2", table2)
+    val table3 = util.tableEnv.sqlQuery("SELECT a AS a2, c FROM table2 WHERE b >= 5")
+    util.tableEnv.registerTable("table3", table3)
+    val table4 = util.tableEnv.sqlQuery("SELECT a AS a3, c AS c1 FROM table2 WHERE b < 5")
+    util.tableEnv.registerTable("table4", table4)
+    val table5 = util.tableEnv.sqlQuery("SELECT a1, b, c AS c2 from table1, table3 WHERE a1 = a2")
+    util.tableEnv.registerTable("table5", table5)
+    val table6 = util.tableEnv.sqlQuery("SELECT a3, b as b1, c1 FROM table4, table5 WHERE a1 = a3")
+    util.tableEnv.registerTable("table6", table6)
+    val table7 = util.tableEnv.sqlQuery("SELECT a1, b1, c1 FROM table1, table6 WHERE a1 = a3")
+
+    val sink = util.createCollectTableSink(Array("a", "b", "c"), Array(INT, LONG, STRING))
+    util.tableEnv.writeToSink(table7, sink)
+    util.verifyPlan()
+  }
+
+  @Test
+  def testSingleSinkWithUDTF(): Unit = {
+    util.addTableSource[(Int, Long, Int, String, Long)]("MyTable2", 'i, 'j, 'k, 'l, 'm)
+    util.addFunction("split", new TableFunc1)
+
+    val sqlQuery =
+      """
+        |select * from
+        |    (SELECT * FROM MyTable, MyTable1, MyTable2 WHERE b = e AND a = i) t,
+        |    LATERAL TABLE(split(c)) as T(s)
+      """.stripMargin
+
+    val table = util.tableEnv.sqlQuery(sqlQuery)
+    val sink = util.createCollectTableSink(
+      Array("a", "b", "c", "d", "e", "f", "i", "j", "k", "l", "m", "s"),
+      Array(INT, LONG, STRING, INT, LONG, STRING, INT, LONG, INT, STRING, LONG, STRING))
+    util.tableEnv.writeToSink(table, sink)
+
+    util.verifyPlan()
+  }
+
+  @Test
+  def testSingleSinkSplitOnUnion(): Unit = {
+    val sqlQuery = "SELECT SUM(a) AS total_sum FROM " +
+      "(SELECT a, c FROM MyTable UNION ALL SELECT d, f FROM MyTable1)"
+    val table = util.tableEnv.sqlQuery(sqlQuery)
+    val sink = util.createCollectTableSink(Array("total_sum"), Array(INT))
+    util.tableEnv.writeToSink(table, sink)
+
+    util.verifyPlan()
+  }
+
+  @Test
+  def testMultiSinks1(): Unit = {
+    util.tableEnv.getConfig.getConf.setBoolean(
+      PlannerConfigOptions.SQL_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED, true)
+    val table1 = util.tableEnv.sqlQuery("SELECT SUM(a) AS sum_a, c FROM MyTable GROUP BY c")
+    util.tableEnv.registerTable("table1", table1)
+    val table2 = util.tableEnv.sqlQuery("SELECT SUM(sum_a) AS total_sum FROM table1")
+    val table3 = util.tableEnv.sqlQuery("SELECT MIN(sum_a) AS total_min FROM table1")
+
+    val sink1 = util.createCollectTableSink(Array("total_sum"), Array(INT))
+    util.tableEnv.writeToSink(table2, sink1)
+
+    val sink2 = util.createCollectTableSink(Array("total_min"), Array(INT))
+    util.tableEnv.writeToSink(table3, sink2)
+
+    util.verifyPlan()
+  }
+
+  @Test
+  def testMultiSinks2(): Unit = {
+    util.tableEnv.getConfig.getConf.setBoolean(
+      PlannerConfigOptions.SQL_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, false)
+    util.addTableSource[(Int, Long, String, Double, Boolean)]("MyTable2", 'a, 'b, 'c, 'd, 'e)
+
+    val table1 = util.tableEnv.sqlQuery("SELECT a as a1, b as b1 FROM MyTable WHERE a <= 10")
+    util.tableEnv.registerTable("table1", table1)
+    val table2 = util.tableEnv.sqlQuery("SELECT a, b1 from table1, MyTable2 where a = a1")
+    util.tableEnv.registerTable("table2", table2)
+    val table3 = util.tableEnv.sqlQuery("SELECT * FROM table1 UNION ALL SELECT * FROM table2")
+
+    val sink1 = util.createCollectTableSink(Array("a", "b1"), Array(INT, LONG))
+    util.tableEnv.writeToSink(table3, sink1)
+
+    val sink2 = util.createCollectTableSink(Array("a", "b1"), Array(INT, LONG))
+    util.tableEnv.writeToSink(table3, sink2)
+
+    util.verifyPlan()
+  }
+
+  @Test
+  def testMultiSinks3(): Unit = {
+    util.tableEnv.getConfig.getConf.setBoolean(
+      PlannerConfigOptions.SQL_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, true)
+    util.addTableSource[(Int, Long, String, Double, Boolean)]("MyTable2", 'a, 'b, 'c, 'd, 'e)
+
+    val table1 = util.tableEnv.sqlQuery("SELECT a AS a1, b AS b1 FROM MyTable WHERE a <= 10")
+    util.tableEnv.registerTable("table1", table1)
+    val table2 = util.tableEnv.sqlQuery("SELECT a, b1 FROM table1, MyTable2 WHERE a = a1")
+    util.tableEnv.registerTable("table2", table2)
+    val table3 = util.tableEnv.sqlQuery("SELECT * FROM table1 UNION ALL SELECT * FROM table2")
+
+    val sink1 = util.createCollectTableSink(Array("a", "b1"), Array(INT, LONG))
+    util.tableEnv.writeToSink(table2, sink1)
+
+    val sink2 = util.createCollectTableSink(Array("a", "b1"), Array(INT, LONG))
+    util.tableEnv.writeToSink(table3, sink2)
+
+    util.verifyPlan()
+  }
+
+  @Test
+  def testMultiSinks4(): Unit = {
+    util.tableEnv.getConfig.getConf.setBoolean(
+      PlannerConfigOptions.SQL_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED, true)
+
+    val table1 = util.tableEnv.sqlQuery("SELECT a as a1, b FROM MyTable WHERE a <= 10")
+    util.tableEnv.registerTable("table1", table1)
+    val table2 = util.tableEnv.sqlQuery("SELECT a, b, c FROM MyTable WHERE a >= 0")
+    util.tableEnv.registerTable("table2", table2)
+    val table3 = util.tableEnv.sqlQuery("SELECT a as a2, c FROM table2 WHERE b >= 5")
+    util.tableEnv.registerTable("table3", table3)
+    val table4 = util.tableEnv.sqlQuery("SELECT a as a3, c as c1 FROM table2 WHERE b < 5")
+    util.tableEnv.registerTable("table4", table4)
+    val table5 = util.tableEnv.sqlQuery("SELECT a1, b, c as c2 FROM table1, table3 WHERE a1 = a2")
+    util.tableEnv.registerTable("table5", table5)
+    val table6 = util.tableEnv.sqlQuery("SELECT a1, b, c1 FROM table4, table5 WHERE a1 = a3")
+
+    val sink1 = util.createCollectTableSink(Array("a1", "b", "c2"), Array(INT, LONG, STRING))
+    util.tableEnv.writeToSink(table5, sink1)
+
+    val sink2 = util.createCollectTableSink(Array("a1", "b", "c1"), Array(INT, LONG, STRING))
+    util.tableEnv.writeToSink(table6, sink2)
+
+    util.verifyPlan()
+  }
+
+  @Test
+  def testMultiSinks5(): Unit = {
+    util.tableEnv.getConfig.getConf.setBoolean(
+      PlannerConfigOptions.SQL_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED, true)
+    // test with non-deterministic udf
+    util.tableEnv.registerFunction("random_udf", new NonDeterministicUdf())
+    val table1 = util.tableEnv.sqlQuery("SELECT random_udf(a) AS a, c FROM MyTable")
+    util.tableEnv.registerTable("table1", table1)
+    val table2 = util.tableEnv.sqlQuery("SELECT SUM(a) AS total_sum FROM table1")
+    val table3 = util.tableEnv.sqlQuery("SELECT MIN(a) AS total_min FROM table1")
+
+    val sink1 = util.createCollectTableSink(Array("total_sum"), Array(INT))
+    util.tableEnv.writeToSink(table2, sink1)
+
+    val sink2 = util.createCollectTableSink(Array("total_min"), Array(INT))
+    util.tableEnv.writeToSink(table3, sink2)
+
+    util.verifyPlan()
+  }
+
+  @Test
+  def testMultiLevelViews(): Unit = {
+    util.tableEnv.getConfig.getConf.setBoolean(
+      PlannerConfigOptions.SQL_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, true)
+
+    val table1 = util.tableEnv.sqlQuery("SELECT a, b, c FROM MyTable WHERE c LIKE '%hello%'")
+    util.tableEnv.registerTable("TempTable1", table1)
+    val sink1 = util.createCollectTableSink(Array("a", "b", "c"), Array(INT, LONG, STRING))
+    util.tableEnv.writeToSink(table1, sink1)
+
+    val table2 = util.tableEnv.sqlQuery("SELECT a, b, c FROM MyTable WHERE c LIKE '%world%'")
+    util.tableEnv.registerTable("TempTable2", table2)
+
+    val sqlQuery =
+      """
+        |SELECT b, COUNT(a) AS cnt FROM (
+        | (SELECT * FROM TempTable1)
+        | UNION ALL
+        | (SELECT * FROM TempTable2)
+        |) t
+        |GROUP BY b
+      """.stripMargin
+    val table3 = util.tableEnv.sqlQuery(sqlQuery)
+    util.tableEnv.registerTable("TempTable3", table3)
+
+    val table4 = util.tableEnv.sqlQuery("SELECT b, cnt FROM TempTable3 WHERE b < 4")
+    val sink2 = util.createCollectTableSink(Array("b", "cnt"), Array(LONG, LONG))
+    util.tableEnv.writeToSink(table4, sink2)
+
+    val table5 = util.tableEnv.sqlQuery("SELECT b, cnt FROM TempTable3 WHERE b >=4 AND b < 6")
+    val sink3 = util.createCollectTableSink(Array("b", "cnt"), Array(LONG, LONG))
+    util.tableEnv.writeToSink(table5, sink3)
+
+    util.verifyPlan()
+  }
+
+  @Test
+  def testMultiSinksWithUDTF(): Unit = {
+    util.tableEnv.getConfig.getConf.setBoolean(
+      PlannerConfigOptions.SQL_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, true)
+    util.tableEnv.registerFunction("split", new TableFunc1)
+    val sqlQuery1 =
+      """
+        |SELECT  a, b - MOD(b, 300) AS b, c FROM MyTable
+        |WHERE b >= UNIX_TIMESTAMP('${startTime}')
+      """.stripMargin
+    val table1 = util.tableEnv.sqlQuery(sqlQuery1)
+    util.tableEnv.registerTable("table1", table1)
+
+    val sqlQuery2 =
+      "SELECT a, b, c1 AS c FROM table1, LATERAL TABLE(split(c)) AS T(c1) WHERE c <> '' "
+    val table2 = util.tableEnv.sqlQuery(sqlQuery2)
+    util.tableEnv.registerTable("table2", table2)
+
+    val sqlQuery3 = "SELECT a, b, COUNT(DISTINCT c) AS total_c FROM table2 GROUP BY a, b"
+    val table3 = util.tableEnv.sqlQuery(sqlQuery3)
+    util.tableEnv.registerTable("table3", table3)
+
+    val sqlQuery4 = "SELECT a, total_c FROM table3 UNION ALL SELECT a, 0 AS total_c FROM table1"
+    val table4 = util.tableEnv.sqlQuery(sqlQuery4)
+    util.tableEnv.registerTable("table4", table4)
+
+    val sqlQuery5 = "SELECT * FROM table4 WHERE a > 50"
+    val table5 = util.tableEnv.sqlQuery(sqlQuery5)
+    val sink1 = util.createCollectTableSink(Array("a", "total_c"), Array(INT, LONG))
+    util.tableEnv.writeToSink(table5, sink1)
+
+    val sqlQuery6 = "SELECT * FROM table4 WHERE a < 50"
+    val table6 = util.tableEnv.sqlQuery(sqlQuery6)
+    val sink2 = util.createCollectTableSink(Array("a", "total_c"), Array(INT, LONG))
+    util.tableEnv.writeToSink(table6, sink2)
+
+    util.verifyPlan()
+  }
+
+  @Test
+  def testMultiSinksSplitOnUnion1(): Unit = {
+    util.tableEnv.getConfig.getConf.setBoolean(
+      PlannerConfigOptions.SQL_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, true)
+
+    val table = util.tableEnv.sqlQuery(
+      "SELECT a, c FROM MyTable UNION ALL SELECT d, f FROM MyTable1")
+    util.tableEnv.registerTable("TempTable", table)
+
+    val table1 = util.tableEnv.sqlQuery("SELECT SUM(a) AS total_sum FROM TempTable")
+    val sink1 = util.createCollectTableSink( Array("total_sum"), Array(INT))
+    util.tableEnv.writeToSink(table1, sink1)
+
+    val table3 = util.tableEnv.sqlQuery("SELECT MIN(a) AS total_min FROM TempTable")
+    val sink2 = util.createCollectTableSink(Array("total_min"), Array(INT))
+    util.tableEnv.writeToSink(table3, sink2)
+
+    util.verifyPlan()
+  }
+
+  @Test
+  def testMultiSinksSplitOnUnion2(): Unit = {
+    util.tableEnv.getConfig.getConf.setBoolean(
+      PlannerConfigOptions.SQL_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED, true)
+    util.tableEnv.getConfig.getConf.setBoolean(
+      PlannerConfigOptions.SQL_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, true)
+    util.addTableSource[(Int, Long, String)]("MyTable2", 'a, 'b, 'c)
+
+    val sqlQuery1 =
+      """
+        |SELECT a, c FROM MyTable
+        |UNION ALL
+        |SELECT d, f FROM MyTable1
+        |UNION ALL
+        |SELECT a, c FROM MyTable2
+      """.stripMargin
+    val table = util.tableEnv.sqlQuery(sqlQuery1)
+    util.tableEnv.registerTable("TempTable", table)
+
+    val table1 = util.tableEnv.sqlQuery("SELECT SUM(a) AS total_sum FROM TempTable")
+    val sink1 = util.createCollectTableSink(Array("total_sum"), Array(INT))
+    util.tableEnv.writeToSink(table1, sink1)
+
+    val table2 = util.tableEnv.sqlQuery("SELECT MIN(a) AS total_min FROM TempTable")
+    val sink2 = util.createCollectTableSink(Array("total_min"), Array(INT))
+    util.tableEnv.writeToSink(table2, sink2)
+
+    val sqlQuery2 = "SELECT a FROM (SELECT a, c FROM MyTable UNION ALL SELECT d, f FROM MyTable1)"
+    val table3 = util.tableEnv.sqlQuery(sqlQuery2)
+    val sink3 = util.createCollectTableSink(Array("a"), Array(INT))
+    util.tableEnv.writeToSink(table3, sink3)
+
+    util.verifyPlan()
+  }
+
+  @Test
+  def testMultiSinksSplitOnUnion3(): Unit = {
+    util.tableEnv.getConfig.getConf.setBoolean(
+      PlannerConfigOptions.SQL_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, true)
+    util.addTableSource[(Int, Long, String)]("MyTable2", 'a, 'b, 'c)
+
+    val sqlQuery1 = "SELECT a, c FROM MyTable UNION ALL SELECT d, f FROM MyTable1"
+    val table = util.tableEnv.sqlQuery(sqlQuery1)
+    util.tableEnv.registerTable("TempTable", table)
+
+    val sink1 = util.createCollectTableSink(Array("a"), Array(INT))
+    util.tableEnv.writeToSink(table, sink1)
+
+    val sqlQuery2 = "SELECT a, c FROM TempTable UNION ALL SELECT a, c FROM MyTable2"
+    val table1 = util.tableEnv.sqlQuery(sqlQuery2)
+    util.tableEnv.registerTable("TempTable1", table1)
+
+    val table2 = util.tableEnv.sqlQuery("SELECT SUM(a) AS total_sum FROM TempTable1")
+    val sink2 = util.createCollectTableSink(Array("total_sum"), Array(INT))
+    util.tableEnv.writeToSink(table2, sink2)
+
+    val table3 = util.tableEnv.sqlQuery("SELECT MIN(a) AS total_min FROM TempTable1")
+    val sink3 = util.createCollectTableSink(Array("total_min"), Array(INT))
+    util.tableEnv.writeToSink(table3, sink3)
+
+    util.verifyPlan()
+  }
+
+  @Test
+  def testMultiSinksSplitOnUnion4(): Unit = {
+    util.tableEnv.getConfig.getConf.setBoolean(
+      PlannerConfigOptions.SQL_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, true)
+    util.addTableSource[(Int, Long, String)]("MyTable2", 'a, 'b, 'c)
+
+    val sqlQuery =
+      """
+        |SELECT a, c FROM MyTable
+        |UNION ALL
+        |SELECT d, f FROM MyTable1
+        |UNION ALL
+        |SELECT a, c FROM MyTable2
+      """.stripMargin
+    val table = util.tableEnv.sqlQuery(sqlQuery)
+    util.tableEnv.registerTable("TempTable", table)
+
+    val table1 = util.tableEnv.sqlQuery("SELECT SUM(a) AS total_sum FROM TempTable")
+    val sink1 = util.createCollectTableSink(Array("total_sum"), Array(INT))
+    util.tableEnv.writeToSink(table1, sink1)
+
+    val table2 = util.tableEnv.sqlQuery("SELECT MIN(a) AS total_min FROM TempTable")
+    val sink2 = util.createCollectTableSink(Array("total_min"), Array(INT))
+    util.tableEnv.writeToSink(table2, sink2)
+
+    util.verifyPlan()
+  }
+
+  @Test
+  def testMultiSinksWithWindow(): Unit = {
+    util.addTableSource[(Int, Double, Int, Timestamp)]("MyTable2", 'a, 'b, 'c, 'ts)
+    val sqlQuery1 =
+      """
+        |SELECT
+        |    a,
+        |    SUM (CAST (c AS DOUBLE)) AS sum_c,
+        |    TUMBLE_END(ts, INTERVAL '15' SECOND) AS `time`,
+        |    TUMBLE_START(ts, INTERVAL '15' SECOND) AS window_start,
+        |    TUMBLE_END (ts, INTERVAL '15' SECOND) AS window_end
+        |FROM
+        |    MyTable2
+        |GROUP BY
+        |    TUMBLE (ts, INTERVAL '15' SECOND), a
+      """.stripMargin
+
+    val sqlQuery2 =
+      """
+        |SELECT
+        |    a,
+        |    SUM (CAST (c AS DOUBLE)) AS sum_c,
+        |    TUMBLE_END(ts, INTERVAL '15' SECOND) AS `time`
+        |FROM
+        |    MyTable2
+        |GROUP BY
+        |    TUMBLE (ts, INTERVAL '15' SECOND), a
+      """.stripMargin
+
+    val table1 = util.tableEnv.sqlQuery(sqlQuery1)
+    val sink1 = util.createCollectTableSink(
+      Array("a", "sum_c", "time", "window_start", "window_end"),
+      Array(INT, DOUBLE, TIMESTAMP, TIMESTAMP, TIMESTAMP))
+    util.tableEnv.writeToSink(table1, sink1)
+
+    val table2 = util.tableEnv.sqlQuery(sqlQuery2)
+    val sink2 = util.createCollectTableSink(Array("a", "sum_c", "time"),
+      Array(INT, DOUBLE, TIMESTAMP))
+    util.tableEnv.writeToSink(table2, sink2)
+
+    util.verifyPlan()
+  }
+
+}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/SinkTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/SinkTest.scala
new file mode 100644
index 0000000..4c98dc0
--- /dev/null
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/SinkTest.scala
@@ -0,0 +1,59 @@
+/*
+ * 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.plan.batch.sql
+
+import org.apache.flink.api.scala._
+import org.apache.flink.table.`type`.InternalTypes.{INT, LONG}
+import org.apache.flink.table.api.PlannerConfigOptions
+import org.apache.flink.table.util.TableTestBase
+
+import org.junit.Test
+
+class SinkTest extends TableTestBase {
+
+  private val util = batchTestUtil()
+  util.addDataStream[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
+
+  @Test
+  def testSingleSink(): Unit = {
+    val table = util.tableEnv.sqlQuery("SELECT COUNT(*) AS cnt FROM MyTable GROUP BY a")
+    val sink = util.createCollectTableSink(Array("a"), Array(LONG))
+    util.tableEnv.writeToSink(table, sink)
+    util.verifyPlan()
+  }
+
+  @Test
+  def testMultiSinks(): Unit = {
+    util.tableEnv.getConfig.getConf.setBoolean(
+      PlannerConfigOptions.SQL_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED, true)
+    val table1 = util.tableEnv.sqlQuery("SELECT SUM(a) AS sum_a, c FROM MyTable GROUP BY c")
+    util.tableEnv.registerTable("table1", table1)
+    val table2 = util.tableEnv.sqlQuery("SELECT SUM(sum_a) AS total_sum FROM table1")
+    val table3 = util.tableEnv.sqlQuery("SELECT MIN(sum_a) AS total_min FROM table1")
+
+    val sink1 = util.createCollectTableSink(Array("total_sum"), Array(INT))
+    util.tableEnv.writeToSink(table2, sink1)
+
+    val sink2 = util.createCollectTableSink(Array("total_min"), Array(INT))
+    util.tableEnv.writeToSink(table3, sink2)
+
+    util.verifyPlan()
+  }
+
+}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/DagOptimizationTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/DagOptimizationTest.scala
new file mode 100644
index 0000000..0efdf7a
--- /dev/null
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/DagOptimizationTest.scala
@@ -0,0 +1,524 @@
+/*
+ * 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.plan.stream.sql
+
+import org.apache.flink.api.scala._
+import org.apache.flink.table.`type`.InternalTypes.{INT, LONG, STRING}
+import org.apache.flink.table.api.PlannerConfigOptions
+import org.apache.flink.table.runtime.utils.JavaUserDefinedScalarFunctions.NonDeterministicUdf
+import org.apache.flink.table.util.{TableFunc1, TableTestBase}
+
+import org.junit.Test
+
+class DagOptimizationTest extends TableTestBase {
+  private val util = streamTestUtil()
+  util.addTableSource[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
+  util.addTableSource[(Int, Long, String)]("MyTable1", 'd, 'e, 'f)
+
+  @Test
+  def testSingleSink1(): Unit = {
+    val table = util.tableEnv.sqlQuery("SELECT c, COUNT(a) AS cnt FROM MyTable GROUP BY c")
+    val appendSink = util.createAppendTableSink(Array("c", "cnt"), Array(STRING, LONG))
+    util.tableEnv.writeToSink(table, appendSink)
+    util.verifyPlanWithTrait()
+  }
+
+  @Test
+  def testSingleSink2(): Unit = {
+    val table1 = util.tableEnv.sqlQuery("SELECT a as a1, b FROM MyTable WHERE a <= 10")
+    util.tableEnv.registerTable("table1", table1)
+    val table2 = util.tableEnv.sqlQuery("SELECT a, b, c FROM MyTable WHERE a >= 0")
+    util.tableEnv.registerTable("table2", table2)
+    val table3 = util.tableEnv.sqlQuery("SELECT a AS a2, c FROM table2 WHERE b >= 5")
+    util.tableEnv.registerTable("table3", table3)
+    val table4 = util.tableEnv.sqlQuery("SELECT a AS a3, c as c1 FROM table2 WHERE b < 5")
+    util.tableEnv.registerTable("table4", table4)
+    val table5 = util.tableEnv.sqlQuery("SELECT a1, b, c as c2 FROM table1, table3 WHERE a1 = a2")
+    util.tableEnv.registerTable("table5", table5)
+    val table6 = util.tableEnv.sqlQuery("SELECT a1, b, c1 FROM table4, table5 WHERE a1 = a3")
+
+    val appendSink = util.createAppendTableSink(Array("a1", "b", "c1"), Array(INT, LONG, STRING))
+    util.tableEnv.writeToSink(table6, appendSink)
+    util.verifyPlanWithTrait()
+  }
+
+  @Test
+  def testSingleSink3(): Unit = {
+    util.addDataStream[(Int, Long, String, Double, Boolean)]("MyTable2", 'a, 'b, 'c, 'd, 'e)
+    val table1 = util.tableEnv.sqlQuery("SELECT a AS a1, b as b1 FROM MyTable WHERE a <= 10")
+    util.tableEnv.registerTable("table1", table1)
+    val table2 = util.tableEnv.sqlQuery("SELECT a, b1 FROM table1, MyTable2 WHERE a = a1")
+    util.tableEnv.registerTable("table2", table2)
+    val table3 = util.tableEnv.sqlQuery("SELECT * FROM table1 UNION ALL SELECT * FROM table2")
+
+    val appendSink = util.createAppendTableSink(Array("a1", "b1"), Array(INT, LONG))
+    util.tableEnv.writeToSink(table3, appendSink)
+    util.verifyPlanWithTrait()
+  }
+
+  @Test
+  def testSingleSink4(): Unit = {
+    val table1 = util.tableEnv.sqlQuery("SELECT a as a1, b FROM MyTable WHERE a <= 10")
+    util.tableEnv.registerTable("table1", table1)
+    val table2 = util.tableEnv.sqlQuery("SELECT a, b, c FROM MyTable WHERE a >= 0")
+    util.tableEnv.registerTable("table2", table2)
+    val table3 = util.tableEnv.sqlQuery("SELECT a AS a2, c FROM table2 WHERE b >= 5")
+    util.tableEnv.registerTable("table3", table3)
+    val table4 = util.tableEnv.sqlQuery("SELECT a AS a3, c AS c1 FROM table2 WHERE b < 5")
+    util.tableEnv.registerTable("table4", table4)
+    val table5 = util.tableEnv.sqlQuery("SELECT a1, b, c AS c2 from table1, table3 WHERE a1 = a2")
+    util.tableEnv.registerTable("table5", table5)
+    val table6 = util.tableEnv.sqlQuery("SELECT a3, b as b1, c1 FROM table4, table5 WHERE a1 = a3")
+    util.tableEnv.registerTable("table6", table6)
+    val table7 = util.tableEnv.sqlQuery("SELECT a1, b1, c1 FROM table1, table6 WHERE a1 = a3")
+
+    val appendSink = util.createAppendTableSink(Array("a", "b", "c"), Array(INT, LONG, STRING))
+    util.tableEnv.writeToSink(table7, appendSink)
+    util.verifyPlanWithTrait()
+  }
+
+  @Test
+  def testSingleSinkWithUDTF(): Unit = {
+    util.addTableSource[(Int, Long, Int, String, Long)]("MyTable2", 'i, 'j, 'k, 'l, 'm)
+    util.addFunction("split", new TableFunc1)
+
+    val sqlQuery =
+      """
+        |select * from
+        |    (SELECT * FROM MyTable, MyTable1, MyTable2 WHERE b = e AND a = i) t,
+        |    LATERAL TABLE(split(c)) as T(s)
+      """.stripMargin
+
+    val table = util.tableEnv.sqlQuery(sqlQuery)
+    val appendSink = util.createAppendTableSink(
+      Array("a", "b", "c", "d", "e", "f", "i", "j", "k", "l", "m", "s"),
+      Array(INT, LONG, STRING, INT, LONG, STRING, INT, LONG, INT, STRING, LONG, STRING))
+    util.tableEnv.writeToSink(table, appendSink)
+
+    util.verifyPlanWithTrait()
+  }
+
+  @Test
+  def testSingleSinkSplitOnUnion(): Unit = {
+    util.tableEnv.getConfig.getConf.setBoolean(
+      PlannerConfigOptions.SQL_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, true)
+
+    val sqlQuery = "SELECT SUM(a) AS total_sum FROM " +
+      "(SELECT a, c FROM MyTable UNION ALL SELECT d, f FROM MyTable1)"
+    val table = util.tableEnv.sqlQuery(sqlQuery)
+    val retractSink = util.createRetractTableSink(Array("total_sum"), Array(INT))
+    util.tableEnv.writeToSink(table, retractSink)
+
+    util.verifyPlanWithTrait()
+  }
+
+  @Test
+  def testMultiSinks1(): Unit = {
+    util.tableEnv.getConfig.getConf.setBoolean(
+      PlannerConfigOptions.SQL_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED, true)
+    val table1 = util.tableEnv.sqlQuery("SELECT SUM(a) AS sum_a, c FROM MyTable GROUP BY c")
+    util.tableEnv.registerTable("table1", table1)
+    val table2 = util.tableEnv.sqlQuery("SELECT SUM(sum_a) AS total_sum FROM table1")
+    val table3 = util.tableEnv.sqlQuery("SELECT MIN(sum_a) AS total_min FROM table1")
+
+    val appendSink1 = util.createAppendTableSink(Array("total_sum"), Array(INT))
+    util.tableEnv.writeToSink(table2, appendSink1)
+
+    val appendSink2 = util.createAppendTableSink(Array("total_min"), Array(INT))
+    util.tableEnv.writeToSink(table3, appendSink2)
+
+    util.verifyPlanWithTrait()
+  }
+
+  @Test
+  def testMultiSinks2(): Unit = {
+    util.tableEnv.getConfig.getConf.setBoolean(
+      PlannerConfigOptions.SQL_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, false)
+    util.addTableSource[(Int, Long, String, Double, Boolean)]("MyTable2", 'a, 'b, 'c, 'd, 'e)
+
+    val table1 = util.tableEnv.sqlQuery("SELECT a as a1, b as b1 FROM MyTable WHERE a <= 10")
+    util.tableEnv.registerTable("table1", table1)
+    val table2 = util.tableEnv.sqlQuery("SELECT a, b1 from table1, MyTable2 where a = a1")
+    util.tableEnv.registerTable("table2", table2)
+    val table3 = util.tableEnv.sqlQuery("SELECT * FROM table1 UNION ALL SELECT * FROM table2")
+
+    val appendSink1 = util.createAppendTableSink(Array("a", "b1"), Array(INT, LONG))
+    util.tableEnv.writeToSink(table3, appendSink1)
+
+    val appendSink2 = util.createAppendTableSink(Array("a", "b1"), Array(INT, LONG))
+    util.tableEnv.writeToSink(table3, appendSink2)
+
+    util.verifyPlanWithTrait()
+  }
+
+  @Test
+  def testMultiSinks3(): Unit = {
+    util.tableEnv.getConfig.getConf.setBoolean(
+      PlannerConfigOptions.SQL_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, false)
+    util.addTableSource[(Int, Long, String, Double, Boolean)]("MyTable2", 'a, 'b, 'c, 'd, 'e)
+
+    val table1 = util.tableEnv.sqlQuery("SELECT a AS a1, b AS b1 FROM MyTable WHERE a <= 10")
+    util.tableEnv.registerTable("table1", table1)
+    val table2 = util.tableEnv.sqlQuery("SELECT a, b1 FROM table1, MyTable2 WHERE a = a1")
+    util.tableEnv.registerTable("table2", table2)
+    val table3 = util.tableEnv.sqlQuery("SELECT * FROM table1 UNION ALL SELECT * FROM table2")
+
+    val appendSink1 = util.createAppendTableSink(Array("a", "b1"), Array(INT, LONG))
+    util.tableEnv.writeToSink(table2, appendSink1)
+
+    val appendSink2 = util.createAppendTableSink(Array("a", "b1"), Array(INT, LONG))
+    util.tableEnv.writeToSink(table3, appendSink2)
+
+    util.verifyPlanWithTrait()
+  }
+
+  @Test
+  def testMultiSinks4(): Unit = {
+    val table1 = util.tableEnv.sqlQuery("SELECT a as a1, b FROM MyTable WHERE a <= 10")
+    util.tableEnv.registerTable("table1", table1)
+    val table2 = util.tableEnv.sqlQuery("SELECT a, b, c FROM MyTable WHERE a >= 0")
+    util.tableEnv.registerTable("table2", table2)
+    val table3 = util.tableEnv.sqlQuery("SELECT a as a2, c FROM table2 WHERE b >= 5")
+    util.tableEnv.registerTable("table3", table3)
+    val table4 = util.tableEnv.sqlQuery("SELECT a as a3, c as c1 FROM table2 WHERE b < 5")
+    util.tableEnv.registerTable("table4", table4)
+    val table5 = util.tableEnv.sqlQuery("SELECT a1, b, c as c2 FROM table1, table3 WHERE a1 = a2")
+    util.tableEnv.registerTable("table5", table5)
+    val table6 = util.tableEnv.sqlQuery("SELECT a1, b, c1 FROM table4, table5 WHERE a1 = a3")
+
+    val appendSink1 = util.createAppendTableSink(Array("a1", "b", "c2"), Array(INT, LONG, STRING))
+    util.tableEnv.writeToSink(table5, appendSink1)
+
+    val appendSink2 = util.createAppendTableSink(Array("a1", "b", "c1"), Array(INT, LONG, STRING))
+    util.tableEnv.writeToSink(table6, appendSink2)
+
+    util.verifyPlanWithTrait()
+  }
+
+  @Test
+  def testMultiSinks5(): Unit = {
+    util.tableEnv.getConfig.getConf.setBoolean(
+      PlannerConfigOptions.SQL_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED, true)
+    // test with non-deterministic udf
+    util.tableEnv.registerFunction("random_udf", new NonDeterministicUdf())
+    val table1 = util.tableEnv.sqlQuery("SELECT random_udf(a) AS a, c FROM MyTable")
+    util.tableEnv.registerTable("table1", table1)
+    val table2 = util.tableEnv.sqlQuery("SELECT SUM(a) AS total_sum FROM table1")
+    val table3 = util.tableEnv.sqlQuery("SELECT MIN(a) AS total_min FROM table1")
+
+    val appendSink1 = util.createAppendTableSink(Array("total_sum"), Array(INT))
+    util.tableEnv.writeToSink(table2, appendSink1)
+
+    val appendSink2 = util.createAppendTableSink(Array("total_min"), Array(INT))
+    util.tableEnv.writeToSink(table3, appendSink2)
+
+    util.verifyPlanWithTrait()
+  }
+
+  @Test
+  def testMultiSinksWithUDTF(): Unit = {
+    util.tableEnv.getConfig.getConf.setBoolean(
+      PlannerConfigOptions.SQL_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, true)
+    util.tableEnv.registerFunction("split", new TableFunc1)
+    val sqlQuery1 =
+      """
+        |SELECT  a, b - MOD(b, 300) AS b, c FROM MyTable
+        |WHERE b >= UNIX_TIMESTAMP('${startTime}')
+      """.stripMargin
+    val table1 = util.tableEnv.sqlQuery(sqlQuery1)
+    util.tableEnv.registerTable("table1", table1)
+
+    val sqlQuery2 =
+      "SELECT a, b, c1 AS c FROM table1, LATERAL TABLE(split(c)) AS T(c1) WHERE c <> '' "
+    val table2 = util.tableEnv.sqlQuery(sqlQuery2)
+    util.tableEnv.registerTable("table2", table2)
+
+    val sqlQuery3 = "SELECT a, b, COUNT(DISTINCT c) AS total_c FROM table2 GROUP BY a, b"
+    val table3 = util.tableEnv.sqlQuery(sqlQuery3)
+    util.tableEnv.registerTable("table3", table3)
+
+    val sqlQuery4 = "SELECT a, total_c FROM table3 UNION ALL SELECT a, 0 AS total_c FROM table1"
+    val table4 = util.tableEnv.sqlQuery(sqlQuery4)
+    util.tableEnv.registerTable("table4", table4)
+
+    val sqlQuery5 = "SELECT * FROM table4 WHERE a > 50"
+    val table5 = util.tableEnv.sqlQuery(sqlQuery5)
+    val appendSink1 = util.createAppendTableSink(Array("a", "total_c"), Array(INT, LONG))
+    util.tableEnv.writeToSink(table5, appendSink1)
+
+    val sqlQuery6 = "SELECT * FROM table4 WHERE a < 50"
+    val table6 = util.tableEnv.sqlQuery(sqlQuery6)
+    val appendSink2 = util.createAppendTableSink(Array("a", "total_c"), Array(INT, LONG))
+    util.tableEnv.writeToSink(table6, appendSink2)
+
+    util.verifyPlanWithTrait()
+  }
+
+  @Test
+  def testMultiSinksSplitOnUnion1(): Unit = {
+    util.tableEnv.getConfig.getConf.setBoolean(
+      PlannerConfigOptions.SQL_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, true)
+
+    val table = util.tableEnv.sqlQuery(
+      "SELECT a, c FROM MyTable UNION ALL SELECT d, f FROM MyTable1")
+    util.tableEnv.registerTable("TempTable", table)
+
+    val table1 = util.tableEnv.sqlQuery("SELECT SUM(a) AS total_sum FROM TempTable")
+    val upsertSink = util.createUpsertTableSink(Array(), Array("total_sum"), Array(INT))
+    util.tableEnv.writeToSink(table1, upsertSink)
+
+    val table3 = util.tableEnv.sqlQuery("SELECT MIN(a) AS total_min FROM TempTable")
+    val retractSink = util.createRetractTableSink(Array("total_min"), Array(INT))
+    util.tableEnv.writeToSink(table3, retractSink)
+
+    util.verifyPlanWithTrait()
+  }
+
+  @Test
+  def testMultiSinksSplitOnUnion2(): Unit = {
+    util.tableEnv.getConfig.getConf.setBoolean(
+      PlannerConfigOptions.SQL_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED, true)
+    util.tableEnv.getConfig.getConf.setBoolean(
+      PlannerConfigOptions.SQL_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, true)
+    util.addTableSource[(Int, Long, String)]("MyTable2", 'a, 'b, 'c)
+
+    val sqlQuery1 =
+      """
+        |SELECT a, c FROM MyTable
+        |UNION ALL
+        |SELECT d, f FROM MyTable1
+        |UNION ALL
+        |SELECT a, c FROM MyTable2
+      """.stripMargin
+    val table = util.tableEnv.sqlQuery(sqlQuery1)
+    util.tableEnv.registerTable("TempTable", table)
+
+    val table1 = util.tableEnv.sqlQuery("SELECT SUM(a) AS total_sum FROM TempTable")
+    val appendSink1 = util.createAppendTableSink(Array("total_sum"), Array(INT))
+    util.tableEnv.writeToSink(table1, appendSink1)
+
+    val table2 = util.tableEnv.sqlQuery("SELECT MIN(a) AS total_min FROM TempTable")
+    val appendSink2 = util.createAppendTableSink(Array("total_min"), Array(INT))
+    util.tableEnv.writeToSink(table2, appendSink2)
+
+    val sqlQuery2 = "SELECT a FROM (SELECT a, c FROM MyTable UNION ALL SELECT d, f FROM MyTable1)"
+    val table3 = util.tableEnv.sqlQuery(sqlQuery2)
+    val appendSink3 = util.createAppendTableSink(Array("a"), Array(INT))
+    util.tableEnv.writeToSink(table3, appendSink3)
+
+    util.verifyPlan()
+  }
+
+  @Test
+  def testMultiSinksSplitOnUnion3(): Unit = {
+    util.tableEnv.getConfig.getConf.setBoolean(
+      PlannerConfigOptions.SQL_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, true)
+    util.addTableSource[(Int, Long, String)]("MyTable2", 'a, 'b, 'c)
+
+    val sqlQuery1 = "SELECT a, c FROM MyTable UNION ALL SELECT d, f FROM MyTable1"
+    val table = util.tableEnv.sqlQuery(sqlQuery1)
+    util.tableEnv.registerTable("TempTable", table)
+
+    val appendSink = util.createAppendTableSink(Array("a"), Array(INT))
+    util.tableEnv.writeToSink(table, appendSink)
+
+    val sqlQuery2 = "SELECT a, c FROM TempTable UNION ALL SELECT a, c FROM MyTable2"
+    val table1 = util.tableEnv.sqlQuery(sqlQuery2)
+    util.tableEnv.registerTable("TempTable1", table1)
+
+    val table2 = util.tableEnv.sqlQuery("SELECT SUM(a) AS total_sum FROM TempTable1")
+    val retractSink = util.createRetractTableSink(Array("total_sum"), Array(INT))
+    util.tableEnv.writeToSink(table2, retractSink)
+
+    val table3 = util.tableEnv.sqlQuery("SELECT MIN(a) AS total_min FROM TempTable1")
+    val upsertSink = util.createUpsertTableSink(Array(), Array("total_min"), Array(INT))
+    util.tableEnv.writeToSink(table3, upsertSink)
+
+    util.verifyPlanWithTrait()
+  }
+
+  @Test
+  def testMultiSinksSplitOnUnion4(): Unit = {
+    util.tableEnv.getConfig.getConf.setBoolean(
+      PlannerConfigOptions.SQL_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, true)
+    util.addTableSource[(Int, Long, String)]("MyTable2", 'a, 'b, 'c)
+
+    val sqlQuery =
+      """
+        |SELECT a, c FROM MyTable
+        |UNION ALL
+        |SELECT d, f FROM MyTable1
+        |UNION ALL
+        |SELECT a, c FROM MyTable2
+      """.stripMargin
+    val table = util.tableEnv.sqlQuery(sqlQuery)
+    util.tableEnv.registerTable("TempTable", table)
+
+    val table1 = util.tableEnv.sqlQuery("SELECT SUM(a) AS total_sum FROM TempTable")
+    val upsertSink = util.createUpsertTableSink(Array(), Array("total_sum"), Array(INT))
+    util.tableEnv.writeToSink(table1, upsertSink)
+
+    val table2 = util.tableEnv.sqlQuery("SELECT MIN(a) AS total_min FROM TempTable")
+    val retractSink = util.createRetractTableSink(Array("total_min"), Array(INT))
+    util.tableEnv.writeToSink(table2, retractSink)
+
+    util.verifyPlanWithTrait()
+  }
+
+  @Test
+  def testUnionAndAggWithDifferentGroupings(): Unit = {
+    val sqlQuery =
+      """
+        |SELECT b, c, SUM(a) AS a_sum FROM MyTable GROUP BY b, c
+        |UNION ALL
+        |SELECT 1 AS b, c, SUM(a) AS a_sum FROM MyTable GROUP BY c
+      """.stripMargin
+    val table = util.tableEnv.sqlQuery(sqlQuery)
+
+    val upsertSink = util.createUpsertTableSink(Array(), Array("b", "c", "a_sum"),
+      Array(LONG, STRING, INT))
+    util.tableEnv.writeToSink(table, upsertSink)
+
+    util.verifyPlanWithTrait()
+  }
+
+  @Test
+  def testUpdateAsRetractConsumedAtSinkBlock(): Unit = {
+    val table = util.tableEnv.sqlQuery("SELECT a, b, c FROM MyTable")
+    util.tableEnv.registerTable("TempTable", table)
+
+    val sqlQuery =
+      s"""
+         |SELECT * FROM (
+         |  SELECT a, b, c,
+         |      ROW_NUMBER() OVER (PARTITION BY b ORDER BY c DESC) as rank_num
+         |  FROM TempTable)
+         |WHERE rank_num <= 10
+      """.stripMargin
+    val table1 = util.tableEnv.sqlQuery(sqlQuery)
+    val retractSink = util.createRetractTableSink(Array("a", "b", "c"), Array(INT, LONG, STRING))
+    util.tableEnv.writeToSink(table1, retractSink)
+
+    val upsertSink = util.createUpsertTableSink(Array(), Array("a", "b"), Array(INT, LONG))
+    val table2 = util.tableEnv.sqlQuery("SELECT a, b FROM TempTable WHERE a < 6")
+    util.tableEnv.writeToSink(table2, upsertSink)
+
+    util.verifyPlanWithTrait()
+  }
+
+  @Test
+  def testUpdateAsRetractConsumedAtSourceBlock(): Unit = {
+    val sqlQuery =
+      s"""
+         |SELECT * FROM (
+         |   SELECT a, b, c,
+         |      ROW_NUMBER() OVER (PARTITION BY b ORDER BY c DESC) as rank_num
+         |  FROM MyTable)
+         |WHERE rank_num <= 10
+      """.stripMargin
+    val table = util.tableEnv.sqlQuery(sqlQuery)
+    util.tableEnv.registerTable("TempTable", table)
+
+    val table1 = util.tableEnv.sqlQuery("SELECT a FROM TempTable WHERE a > 6")
+    val retractSink = util.createRetractTableSink(Array("a"), Array(INT))
+    util.tableEnv.writeToSink(table1, retractSink)
+
+    val table2 = util.tableEnv.sqlQuery("SELECT a, b FROM TempTable WHERE a < 6")
+    val upsertSink = util.createUpsertTableSink(Array(), Array("a", "b"), Array(INT, LONG))
+    util.tableEnv.writeToSink(table2, upsertSink)
+
+    util.verifyPlanWithTrait()
+  }
+
+  @Test
+  def testMultiLevelViews(): Unit = {
+    util.tableEnv.getConfig.getConf.setBoolean(
+      PlannerConfigOptions.SQL_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, true)
+
+    val table1 = util.tableEnv.sqlQuery("SELECT a, b, c FROM MyTable WHERE c LIKE '%hello%'")
+    util.tableEnv.registerTable("TempTable1", table1)
+    val appendSink = util.createAppendTableSink(Array("a", "b", "c"), Array(INT, LONG, STRING))
+    util.tableEnv.writeToSink(table1, appendSink)
+
+    val table2 = util.tableEnv.sqlQuery("SELECT a, b, c FROM MyTable WHERE c LIKE '%world%'")
+    util.tableEnv.registerTable("TempTable2", table2)
+
+    val sqlQuery =
+      """
+        |SELECT b, COUNT(a) AS cnt FROM (
+        | (SELECT * FROM TempTable1)
+        | UNION ALL
+        | (SELECT * FROM TempTable2)
+        |) t
+        |GROUP BY b
+      """.stripMargin
+    val table3 = util.tableEnv.sqlQuery(sqlQuery)
+    util.tableEnv.registerTable("TempTable3", table3)
+
+    val table4 = util.tableEnv.sqlQuery("SELECT b, cnt FROM TempTable3 WHERE b < 4")
+    val retractSink = util.createRetractTableSink(Array("b", "cnt"), Array(LONG, LONG))
+    util.tableEnv.writeToSink(table4, retractSink)
+
+    val table5 = util.tableEnv.sqlQuery("SELECT b, cnt FROM TempTable3 WHERE b >=4 AND b < 6")
+    val upsertSink = util.createUpsertTableSink(Array(), Array("b", "cnt"), Array(LONG, LONG))
+    util.tableEnv.writeToSink(table5, upsertSink)
+
+    util.verifyPlanWithTrait()
+  }
+
+  @Test
+  def testSharedUnionNode(): Unit = {
+    util.tableEnv.getConfig.getConf.setBoolean(
+      PlannerConfigOptions.SQL_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, true)
+
+    val table1 = util.tableEnv.sqlQuery("SELECT a, b, c FROM MyTable WHERE c LIKE '%hello%'")
+    util.tableEnv.registerTable("TempTable1", table1)
+    val appendSink = util.createAppendTableSink(Array("a", "b", "c"), Array(INT, LONG, STRING))
+    util.tableEnv.writeToSink(table1, appendSink)
+
+    val table2 = util.tableEnv.sqlQuery("SELECT a, b, c FROM MyTable WHERE c LIKE '%world%'")
+    util.tableEnv.registerTable("TempTable2", table2)
+
+    val sqlQuery1 =
+      """
+        |SELECT * FROM TempTable1
+        |UNION ALL
+        |SELECT * FROM TempTable2
+      """.stripMargin
+    val table3 = util.tableEnv.sqlQuery(sqlQuery1)
+    util.tableEnv.registerTable("TempTable3", table3)
+
+    val table4 = util.tableEnv.sqlQuery("SELECT * FROM TempTable3 WHERE b >= 5")
+    val retractSink1 = util.createRetractTableSink(Array("a", "b", "c"), Array(INT, LONG, STRING))
+    util.tableEnv.writeToSink(table4, retractSink1)
+
+    val table5 = util.tableEnv.sqlQuery("SELECT b, count(a) as cnt FROM TempTable3 GROUP BY b")
+    util.tableEnv.registerTable("TempTable4", table5)
+
+    val table6 = util.tableEnv.sqlQuery("SELECT b, cnt FROM TempTable4 WHERE b < 4")
+    val retractSink2 = util.createRetractTableSink(Array("b", "cnt"), Array(LONG, LONG))
+    util.tableEnv.writeToSink(table6, retractSink2)
+
+    util.tableEnv.sqlQuery("SELECT b, cnt FROM TempTable4 WHERE b >=4 AND b < 6")
+    val upsertSink = util.createUpsertTableSink(Array(), Array("b", "cnt"), Array(LONG, LONG))
+    util.tableEnv.writeToSink(table6, upsertSink)
+
+    util.verifyPlanWithTrait()
+  }
+
+}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/SinkTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/SinkTest.scala
new file mode 100644
index 0000000..a24e8de
--- /dev/null
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/SinkTest.scala
@@ -0,0 +1,148 @@
+/*
+ * 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.plan.stream.sql
+
+import org.apache.flink.api.scala._
+import org.apache.flink.table.`type`.InternalTypes.{INT, LONG, STRING}
+import org.apache.flink.table.util.TableTestBase
+
+import org.junit.Test
+
+class SinkTest extends TableTestBase {
+
+  private val util = streamTestUtil()
+  util.addDataStream[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
+
+  @Test
+  def testAppendSink(): Unit = {
+    val table = util.tableEnv.sqlQuery("SELECT COUNT(*) AS cnt FROM MyTable GROUP BY a")
+    val appendSink = util.createAppendTableSink(Array("a"), Array(LONG))
+    util.tableEnv.writeToSink(table, appendSink)
+    util.verifyPlanWithTrait()
+  }
+
+  @Test
+  def testRetractSink1(): Unit = {
+    val table = util.tableEnv.sqlQuery("SELECT a, COUNT(*) AS cnt FROM MyTable GROUP BY a")
+    val retractSink = util.createRetractTableSink(Array("a", "cnt"), Array(LONG, LONG))
+    util.tableEnv.writeToSink(table, retractSink)
+    util.verifyPlanWithTrait()
+  }
+
+  @Test
+  def testRetractSink2(): Unit = {
+    val sqlQuery =
+      """
+        |SELECT cnt, COUNT(a) AS a FROM (
+        |    SELECT a, COUNT(*) AS cnt FROM MyTable GROUP BY a) t
+        |GROUP BY cnt
+      """.stripMargin
+    val table = util.tableEnv.sqlQuery(sqlQuery)
+    val retractSink = util.createRetractTableSink(Array("cnt", "a"), Array(LONG, LONG))
+    util.tableEnv.writeToSink(table, retractSink)
+    util.verifyPlanWithTrait()
+  }
+
+  @Test
+  def testUpsertSink1(): Unit = {
+    val table = util.tableEnv.sqlQuery("SELECT a, COUNT(*) AS cnt FROM MyTable GROUP BY a")
+    val upsertSink = util.createUpsertTableSink(Array(0), Array("a", "cnt"), Array(INT, LONG))
+    util.tableEnv.writeToSink(table, upsertSink)
+    util.verifyPlanWithTrait()
+  }
+
+  @Test
+  def testUpsertSink2(): Unit = {
+    val sqlQuery =
+      """
+        |with t1 AS (SELECT a AS a1, b FROM MyTable WHERE a <= 10),
+        |     t2 AS (SELECT * from MyTable WHERE a >= 0),
+        |     t3 AS (SELECT a AS a2, c from t2 where b >= 5),
+        |     t4 AS (SELECT a AS a3, c AS c1 FROM t2 WHERE b < 5),
+        |     t5 AS (SELECT a1, b, c AS c2 FROM t1, t3 where a1 = a2)
+        |SELECT a1, b, c1 FROM t4, t5 WHERE a1 = a3
+      """.stripMargin
+    val table = util.tableEnv.sqlQuery(sqlQuery)
+    val upsertSink = util.createUpsertTableSink(Array(), Array("a1", "b", "c1"),
+      Array(INT, LONG, STRING))
+    util.tableEnv.writeToSink(table, upsertSink)
+    util.verifyPlanWithTrait()
+  }
+
+  @Test
+  def testRetractAndUpsertSink(): Unit = {
+    val table = util.tableEnv.sqlQuery("SELECT b, COUNT(a) AS cnt FROM MyTable GROUP BY b")
+    util.tableEnv.registerTable("TempTable", table)
+
+    val table1 = util.tableEnv.sqlQuery("SELECT b, cnt FROM TempTable WHERE b < 4")
+    val retractSink = util.createRetractTableSink(Array("b", "cnt"), Array(LONG, LONG))
+    util.tableEnv.writeToSink(table1, retractSink)
+
+    val table2 = util.tableEnv.sqlQuery("SELECT b, cnt FROM TempTable WHERE b >= 4 AND b < 6")
+    val upsertSink = util.createUpsertTableSink(Array(), Array("b", "cnt"), Array(LONG, LONG))
+    util.tableEnv.writeToSink(table2, upsertSink)
+
+    util.verifyPlanWithTrait()
+  }
+
+  @Test
+  def testUpsertAndUpsertSink(): Unit = {
+    val table = util.tableEnv.sqlQuery("SELECT b, COUNT(a) AS cnt FROM MyTable GROUP BY b")
+    util.tableEnv.registerTable("TempTable", table)
+
+    val table1 = util.tableEnv.sqlQuery(
+      "SELECT cnt, COUNT(b) AS frequency FROM TempTable WHERE b < 4 GROUP BY cnt")
+    val upsertSink1 = util.createUpsertTableSink(Array(0), Array("b", "cnt"), Array(LONG, LONG))
+    util.tableEnv.writeToSink(table1, upsertSink1)
+
+    val table2 = util.tableEnv.sqlQuery("SELECT b, cnt FROM TempTable WHERE b >= 4 AND b < 6")
+    val upsertSink2 = util.createUpsertTableSink(Array(), Array("b", "cnt"), Array(LONG, LONG))
+    util.tableEnv.writeToSink(table2, upsertSink2)
+
+    util.verifyPlanWithTrait()
+  }
+
+  @Test
+  def testAppendUpsertAndRetractSink(): Unit = {
+    util.addDataStream[(Int, Long, String)]("MyTable2", 'd, 'e, 'f)
+    util.addDataStream[(Int, Long, String)]("MyTable3", 'i, 'j, 'k)
+
+    val table = util.tableEnv.sqlQuery(
+      "SELECT a, b FROM MyTable UNION ALL SELECT d, e FROM MyTable2")
+    util.tableEnv.registerTable("TempTable", table)
+
+    val appendSink = util.createAppendTableSink(Array("a", "b"), Array(INT, LONG))
+    util.tableEnv.writeToSink(table, appendSink)
+
+    val table1 = util.tableEnv.sqlQuery(
+      "SELECT a, b FROM TempTable UNION ALL SELECT i, j FROM MyTable3")
+    util.tableEnv.registerTable("TempTable1", table1)
+
+    val table2 = util.tableEnv.sqlQuery("SELECT SUM(a) AS total_sum FROM TempTable1")
+    val retractSink = util.createRetractTableSink(Array("total_sum"), Array(INT))
+    util.tableEnv.writeToSink(table2, retractSink)
+
+    val table3 = util.tableEnv.sqlQuery("SELECT MIN(a) AS total_min FROM TempTable1")
+    val upsertSink = util.createUpsertTableSink(Array(), Array("total_min"), Array(INT))
+    util.tableEnv.writeToSink(table3, upsertSink)
+
+    util.verifyPlanWithTrait()
+  }
+
+}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/CalcITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/CalcITCase.scala
index ed8f22b..d141c28 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/CalcITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/CalcITCase.scala
@@ -217,7 +217,7 @@ class CalcITCase extends StreamingTestBase {
     val result = tEnv.sqlQuery(sqlQuery)
     val sink = TestSinkUtil.configureSink(result, new TestingAppendTableSink())
     tEnv.writeToSink(result, sink)
-    env.execute()
+    tEnv.execute()
 
     val expected = List("0,0,0", "1,1,1", "2,2,2")
     assertEquals(expected.sorted, sink.getAppendResults.sorted)
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/CorrelateITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/CorrelateITCase.scala
index 8ea664e..be8cd26 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/CorrelateITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/CorrelateITCase.scala
@@ -189,7 +189,7 @@ class CorrelateITCase extends StreamingTestBase {
     val result = tEnv.sqlQuery(sql)
     val sink = TestSinkUtil.configureSink(result, new TestingAppendTableSink)
     tEnv.writeToSink(result, sink)
-    env.execute()
+    tEnv.execute()
 
     val expected = List("1,2,,null", "1,3,,null")
     assertEquals(expected.sorted, sink.getAppendResults.sorted)
@@ -210,7 +210,7 @@ class CorrelateITCase extends StreamingTestBase {
     val result = tEnv.sqlQuery(sql)
     val sink = TestSinkUtil.configureSink(result, new TestingAppendTableSink)
     tEnv.writeToSink(result, sink)
-    env.execute()
+    tEnv.execute()
 
     val expected = List("3018-06-10", "2018-06-03", "2018-06-01", "2018-06-02")
     assertEquals(expected.sorted, sink.getAppendResults.sorted)
@@ -231,7 +231,7 @@ class CorrelateITCase extends StreamingTestBase {
     val result = tEnv.sqlQuery(sql)
     val sink = TestSinkUtil.configureSink(result, new TestingAppendTableSink)
     tEnv.writeToSink(result, sink)
-    env.execute()
+    tEnv.execute()
 
     val expected = List("1,3018-06-10", "1,2018-06-03", "1,2018-06-01", "1,2018-06-02")
     assertEquals(expected.sorted, sink.getAppendResults.sorted)
@@ -251,7 +251,7 @@ class CorrelateITCase extends StreamingTestBase {
     val result = tEnv.sqlQuery(sql)
     val sink = TestSinkUtil.configureSink(result, new TestingAppendTableSink)
     tEnv.writeToSink(result, sink)
-    env.execute()
+    tEnv.execute()
 
     val expected = List("a")
     assertEquals(expected.sorted, sink.getAppendResults.sorted)
@@ -271,7 +271,7 @@ class CorrelateITCase extends StreamingTestBase {
     val result = tEnv.sqlQuery(sql)
     val sink = TestSinkUtil.configureSink(result, new TestingAppendTableSink)
     tEnv.writeToSink(result, sink)
-    env.execute()
+    tEnv.execute()
 
     // output two null
     val expected = List("null", "null")
@@ -292,7 +292,7 @@ class CorrelateITCase extends StreamingTestBase {
     val result = tEnv.sqlQuery(sql)
     val sink = TestSinkUtil.configureSink(result, new TestingAppendTableSink)
     tEnv.writeToSink(result, sink)
-    env.execute()
+    tEnv.execute()
 
     val expected = List("1,a")
     assertEquals(expected.sorted, sink.getAppendResults.sorted)
@@ -312,7 +312,7 @@ class CorrelateITCase extends StreamingTestBase {
     val result = tEnv.sqlQuery(sql)
     val sink = TestSinkUtil.configureSink(result, new TestingAppendTableSink)
     tEnv.writeToSink(result, sink)
-    env.execute()
+    tEnv.execute()
 
     val expected = List("2,null", "3,null")
     assertEquals(expected.sorted, sink.getAppendResults.sorted)
@@ -336,8 +336,7 @@ class CorrelateITCase extends StreamingTestBase {
 //
 //    val sink = new TestingUpsertTableSink(Array(0))
 //    tEnv.sqlQuery(sql).writeToSink(sink)
-//    tEnv.compile()
-//    env.execute()
+//    tEnv.execute()
 //
 //    val expected = List("1", "2", "3", "4")
 //    assertEquals(expected.sorted, sink.getUpsertResults.sorted)
@@ -361,8 +360,7 @@ class CorrelateITCase extends StreamingTestBase {
 //
 //    val sink = new TestingUpsertTableSink(Array(0))
 //    tEnv.sqlQuery(sql).writeToSink(sink)
-//    tEnv.compile()
-//    env.execute()
+//    tEnv.execute()
 //
 //    val expected = List("1", "2")
 //    assertEquals(expected.sorted, sink.getUpsertResults.sorted)
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/RankITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/RankITCase.scala
index 6a5e111..610bd3a 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/RankITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/RankITCase.scala
@@ -130,7 +130,7 @@ class RankITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode
     val sink = new TestingUpsertTableSink(Array(0, 3)).
       configure(schema.getFieldNames, schema.getFieldTypes)
     tEnv.writeToSink(table, sink)
-    env.execute()
+    tEnv.execute()
 
     val expected = List(
       "book,4,11,1",
@@ -191,7 +191,7 @@ class RankITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode
     val sink = new TestingUpsertTableSink(Array(0, 3)).
       configure(schema.getFieldNames, schema.getFieldTypes)
     tEnv.writeToSink(table, sink)
-    env.execute()
+    tEnv.execute()
 
     val updatedExpected = List(
       "book,5,800,1",
@@ -252,7 +252,7 @@ class RankITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode
     val sink = new TestingUpsertTableSink(Array(0, 3)).
       configure(schema.getFieldNames, schema.getFieldTypes)
     tEnv.writeToSink(table, sink)
-    env.execute()
+    tEnv.execute()
 
     val updatedExpected = List(
       "book,3,110,1",
@@ -296,7 +296,7 @@ class RankITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode
     val sink = new TestingUpsertTableSink(Array(0, 3)).
       configure(schema.getFieldNames, schema.getFieldTypes)
     tEnv.writeToSink(table, sink)
-    env.execute()
+    tEnv.execute()
 
     val updatedExpected = List(
       "book,1,22,1",
@@ -348,7 +348,7 @@ class RankITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode
     val sink = new TestingUpsertTableSink(Array(0, 3)).
       configure(schema.getFieldNames, schema.getFieldTypes)
     tEnv.writeToSink(table, sink)
-    env.execute()
+    tEnv.execute()
 
     val updatedExpected = List(
       "book,10,1300.0,1",
@@ -391,7 +391,7 @@ class RankITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode
     val sink = new TestingUpsertTableSink(Array(0, 3)).
       configure(schema.getFieldNames, schema.getFieldTypes)
     tEnv.writeToSink(table, sink)
-    env.execute()
+    tEnv.execute()
 
     val updatedExpected = List(
       "book,2,19,2",
@@ -524,7 +524,7 @@ class RankITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode
     val sink = new TestingUpsertTableSink(Array(0, 1)).
       configure(schema.getFieldNames, schema.getFieldTypes)
     tEnv.writeToSink(table, sink)
-    env.execute()
+    tEnv.execute()
 
     val expected = List(
       "book,1,5,4",
@@ -581,7 +581,7 @@ class RankITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode
     val sink = new TestingUpsertTableSink(Array(0, 1)).
       configure(schema.getFieldNames, schema.getFieldTypes)
     tEnv.writeToSink(table, sink)
-    env.execute()
+    tEnv.execute()
 
     val expected = List(
       "book,3,2,2",
@@ -639,7 +639,7 @@ class RankITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode
     val sink = new TestingUpsertTableSink(Array(0)).
       configure(schema.getFieldNames, schema.getFieldTypes)
     tEnv.writeToSink(table, sink)
-    env.execute()
+    tEnv.execute()
 
     val expected = List(
       "(true,1,book,a,1,1)", "(true,2,book,b,1,1)", "(true,3,book,c,1,1)",
@@ -701,7 +701,7 @@ class RankITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode
     val sink = new TestingUpsertTableSink(Array(0)).
       configure(schema.getFieldNames, schema.getFieldTypes)
     tEnv.writeToSink(table, sink)
-    env.execute()
+    tEnv.execute()
 
     val expected = List(
       "(true,1,book,a,1,1)",
@@ -770,7 +770,7 @@ class RankITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode
     val sink = new TestingUpsertTableSink(Array(0, 1)).
       configure(schema.getFieldNames, schema.getFieldTypes)
     tEnv.writeToSink(table, sink)
-    env.execute()
+    tEnv.execute()
 
     val expected = List(
       "book,1,5,4",
@@ -832,7 +832,7 @@ class RankITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode
     val sink = new TestingUpsertTableSink(Array(0, 3)).
       configure(schema.getFieldNames, schema.getFieldTypes)
     tEnv.writeToSink(table, sink)
-    env.execute()
+    tEnv.execute()
 
     val expected = List(
       "(true,book,1,11,1)",
@@ -910,7 +910,7 @@ class RankITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode
     val sink = new TestingUpsertTableSink(Array(0, 1)).
       configure(schema.getFieldNames, schema.getFieldTypes)
     tEnv.writeToSink(table, sink)
-    env.execute()
+    tEnv.execute()
 
     val expected = List(
       "(true,book,1,100.0)",
@@ -981,7 +981,7 @@ class RankITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode
     val sink = new TestingUpsertTableSink(Array(0, 1)).
       configure(schema.getFieldNames, schema.getFieldTypes)
     tEnv.writeToSink(table, sink)
-    env.execute()
+    tEnv.execute()
 
     val expected = List(
       "(true,book,1,1)",
@@ -1046,7 +1046,7 @@ class RankITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode
     val sink = new TestingUpsertTableSink(Array(0, 2)).
       configure(schema.getFieldNames, schema.getFieldTypes)
     tEnv.writeToSink(table, sink)
-    env.execute()
+    tEnv.execute()
 
     val expected = List(
       "(true,book,12,1)",
@@ -1189,8 +1189,8 @@ class RankITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode
       configure(schema2.getFieldNames, schema2
       .getFieldTypes)
     tEnv.writeToSink(table2, sink2)
+    tEnv.execute()
 
-    env.execute()
     val expected1 = List(
       "book,1,25,1",
       "book,2,19,2",
@@ -1258,7 +1258,7 @@ class RankITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode
     val sink2 = new TestingRetractTableSink().
       configure(schema2.getFieldNames, schema2.getFieldTypes)
     tEnv.writeToSink(table2, sink2)
-    env.execute()
+    tEnv.execute()
 
     val expected1 = List(
       "book,1,2,1",
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/BatchTableEnvUtil.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/BatchTableEnvUtil.scala
index eb026a8..be32bff 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/BatchTableEnvUtil.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/BatchTableEnvUtil.scala
@@ -46,7 +46,7 @@ object BatchTableEnvUtil {
     sink.init(typeSerializer.asInstanceOf[TypeSerializer[T]], id)
     tEnv.writeToSink(table, sink)
 
-    val res = tEnv.streamEnv.execute()
+    val res = tEnv.execute()
     val accResult: JArrayList[Array[Byte]] = res.getAccumulatorResult(id)
     SerializedListAccumulator.deserializeList(accResult, typeSerializer)
   }
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/util/TableTestBase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/util/TableTestBase.scala
index f766135..e304407 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/util/TableTestBase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/util/TableTestBase.scala
@@ -24,9 +24,9 @@ import org.apache.flink.streaming.api.datastream.DataStream
 import org.apache.flink.streaming.api.environment.LocalStreamEnvironment
 import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
 import org.apache.flink.streaming.api.{TimeCharacteristic, environment}
-import org.apache.flink.table.`type`.TypeConverters
+import org.apache.flink.table.`type`.{InternalType, TypeConverters}
 import org.apache.flink.table.api._
-import org.apache.flink.table.api.java.{BatchTableEnvironment => JavaBatchTableEnv, StreamTableEnvironment => JavaStreamTableEnv}
+import java.{BatchTableEnvironment => JavaBatchTableEnv, StreamTableEnvironment => JavaStreamTableEnv}
 import org.apache.flink.table.api.scala.{BatchTableEnvironment => ScalaBatchTableEnv, StreamTableEnvironment => ScalaStreamTableEnv, _}
 import org.apache.flink.table.calcite.CalciteConfig
 import org.apache.flink.table.dataformat.BaseRow
@@ -34,12 +34,15 @@ import org.apache.flink.table.functions.{AggregateFunction, ScalarFunction, Tabl
 import org.apache.flink.table.plan.nodes.exec.ExecNode
 import org.apache.flink.table.plan.optimize.program.{FlinkBatchProgram, FlinkStreamProgram}
 import org.apache.flink.table.plan.util.{ExecNodePlanDumper, FlinkRelOptUtil}
+import org.apache.flink.table.runtime.utils.{BatchTableEnvUtil, TestingAppendTableSink, TestingRetractTableSink, TestingUpsertTableSink}
+import org.apache.flink.table.sinks.{AppendStreamTableSink, CollectRowTableSink, RetractStreamTableSink, TableSink, UpsertStreamTableSink}
 import org.apache.flink.table.sources.{BatchTableSource, StreamTableSource}
 import org.apache.flink.table.typeutils.BaseRowTypeInfo
+import org.apache.flink.types.Row
+
 import org.apache.calcite.rel.RelNode
 import org.apache.calcite.sql.SqlExplainLevel
 import org.apache.commons.lang3.SystemUtils
-import org.apache.flink.table.runtime.utils.BatchTableEnvUtil
 import org.junit.Assert.{assertEquals, assertTrue}
 import org.junit.Rule
 import org.junit.rules.{ExpectedException, TestName}
@@ -183,7 +186,11 @@ abstract class TableTestUtil(test: TableTestBase) {
       function: AggregateFunction[T, ACC]): Unit = getTableEnv.registerFunction(name, function)
 
   def verifyPlan(): Unit = {
-    // TODO implements this method when supporting multi-sinks
+    doVerifyPlan(
+      SqlExplainLevel.EXPPLAN_ATTRIBUTES,
+      withRowType = false,
+      withRetractTraits = false,
+      printPlanBefore = true)
   }
 
   def verifyPlan(sql: String): Unit = {
@@ -226,7 +233,7 @@ abstract class TableTestUtil(test: TableTestBase) {
     require(notExpected.nonEmpty)
     val relNode = table.asInstanceOf[TableImpl].getRelNode
     val optimizedPlan = getOptimizedPlan(
-      relNode,
+      Array(relNode),
       explainLevel = SqlExplainLevel.EXPPLAN_ATTRIBUTES,
       withRetractTraits = false,
       withRowType = false)
@@ -235,15 +242,21 @@ abstract class TableTestUtil(test: TableTestBase) {
     assertTrue(message, result)
   }
 
-  def verifyExplain(): Unit = doVerifyExplain()
+  def verifyExplain(): Unit = verifyExplain(extended = false)
+
+  def verifyExplain(extended: Boolean): Unit = doVerifyExplain(extended = extended)
+
+  def verifyExplain(sql: String): Unit = verifyExplain(sql, extended = false)
 
-  def verifyExplain(sql: String): Unit = {
+  def verifyExplain(sql: String, extended: Boolean): Unit = {
     val table = getTableEnv.sqlQuery(sql)
-    verifyExplain(table)
+    verifyExplain(table, extended)
   }
 
-  def verifyExplain(table: Table): Unit = {
-    doVerifyExplain(Some(table))
+  def verifyExplain(table: Table): Unit = verifyExplain(table, extended = false)
+
+  def verifyExplain(table: Table, extended: Boolean): Unit = {
+    doVerifyExplain(Some(table), extended = extended)
   }
 
   def doVerifyPlan(
@@ -268,7 +281,7 @@ abstract class TableTestUtil(test: TableTestBase) {
     val table = getTableEnv.sqlQuery(sql)
     val relNode = table.asInstanceOf[TableImpl].getRelNode
     val optimizedPlan = getOptimizedPlan(
-      relNode,
+      Array(relNode),
       explainLevel,
       withRetractTraits = withRetractTraits,
       withRowType = withRowType)
@@ -309,7 +322,7 @@ abstract class TableTestUtil(test: TableTestBase) {
       printPlanBefore: Boolean): Unit = {
     val relNode = table.asInstanceOf[TableImpl].getRelNode
     val optimizedPlan = getOptimizedPlan(
-      relNode,
+      Array(relNode),
       explainLevel,
       withRetractTraits = withRetractTraits,
       withRowType = withRowType)
@@ -327,46 +340,102 @@ abstract class TableTestUtil(test: TableTestBase) {
     assertEqualsOrExpand("planAfter", actual.toString, expand = false)
   }
 
-  private def doVerifyExplain(table: Option[Table] = None): Unit = {
+  def doVerifyPlan(
+      explainLevel: SqlExplainLevel,
+      withRowType: Boolean,
+      withRetractTraits: Boolean,
+      printPlanBefore: Boolean): Unit = {
+    val tableEnv = getTableEnv
+    if (tableEnv.sinkNodes.isEmpty) {
+      throw new TableException("No output table have been created yet. " +
+        "A program needs at least one output table that consumes data.\n" +
+        "Please create output table(s) for your program")
+    }
+    val relNodes = tableEnv.sinkNodes.toArray
+    val optimizedPlan = getOptimizedPlan(
+      relNodes.toArray,
+      explainLevel,
+      withRetractTraits = withRetractTraits,
+      withRowType = withRowType)
+    tableEnv.sinkNodes.clear()
+
+    if (printPlanBefore) {
+      val planBefore = new StringBuilder
+      relNodes.foreach { sink =>
+        planBefore.append(System.lineSeparator)
+        planBefore.append(FlinkRelOptUtil.toString(sink, SqlExplainLevel.EXPPLAN_ATTRIBUTES))
+      }
+      assertEqualsOrExpand("planBefore", planBefore.toString())
+    }
+
+    val actual = SystemUtils.LINE_SEPARATOR + optimizedPlan
+    assertEqualsOrExpand("planAfter", actual.toString, expand = false)
+  }
+
+  private def doVerifyExplain(table: Option[Table] = None, extended: Boolean = false): Unit = {
     val explainResult = table match {
-      case Some(t) => getTableEnv.explain(t)
-      case _ => getTableEnv.explain()
+      case Some(t) => getTableEnv.explain(t, extended = extended)
+      case _ => getTableEnv.explain(extended = extended)
+    }
+    val actual = if (extended) {
+      replaceEstimatedCost(explainResult)
+    } else {
+      explainResult
     }
-    assertEqualsOrExpand("explain", replaceStageId(explainResult), expand = false)
+    assertEqualsOrExpand("explain", replaceStageId(actual), expand = false)
   }
 
   private def getOptimizedPlan(
-      relNode: RelNode,
+      relNodes: Array[RelNode],
       explainLevel: SqlExplainLevel,
       withRetractTraits: Boolean,
       withRowType: Boolean): String = {
+    require(relNodes.nonEmpty)
     val tEnv = getTableEnv
-    val optimized = tEnv.optimize(relNode)
-    optimized match {
-      case execNode: ExecNode[_, _] =>
-        val optimizedNodes = tEnv.translateNodeDag(Seq(execNode))
-        require(optimizedNodes.length == 1)
-        ExecNodePlanDumper.treeToString(
-          optimizedNodes.head,
+    val optimizedRels = tEnv.optimize(relNodes)
+    optimizedRels.head match {
+      case _: ExecNode[_, _] =>
+        val optimizedNodes = tEnv.translateToExecNodeDag(optimizedRels)
+        require(optimizedNodes.length == optimizedRels.length)
+        ExecNodePlanDumper.dagToString(
+          optimizedNodes,
           detailLevel = explainLevel,
           withRetractTraits = withRetractTraits,
           withOutputType = withRowType)
       case _ =>
-        FlinkRelOptUtil.toString(
-          optimized,
-          detailLevel = explainLevel,
-          withRetractTraits = withRetractTraits,
-          withRowType = withRowType)
+        optimizedRels.map { rel =>
+          FlinkRelOptUtil.toString(
+            rel,
+            detailLevel = explainLevel,
+            withRetractTraits = withRetractTraits,
+            withRowType = withRowType)
+        }.mkString("\n")
     }
   }
 
-  /* Stage {id} is ignored, because id keeps incrementing in test class
-     * while StreamExecutionEnvironment is up
-     */
+  /**
+    * Stage {id} is ignored, because id keeps incrementing in test class
+    * while StreamExecutionEnvironment is up
+    */
   protected def replaceStageId(s: String): String = {
     s.replaceAll("\\r\\n", "\n").replaceAll("Stage \\d+", "")
   }
 
+  /**
+    * ignore estimated cost, because it may be unstable.
+    */
+  protected def replaceEstimatedCost(s: String): String = {
+    var str = s.replaceAll("\\r\\n", "\n")
+    val scientificFormRegExpr = "[+-]?[\\d]+([\\.][\\d]*)?([Ee][+-]?[0-9]{0,2})?"
+    str = str.replaceAll(s"rowcount = $scientificFormRegExpr", "rowcount = ")
+    str = str.replaceAll(s"$scientificFormRegExpr rows", "rows")
+    str = str.replaceAll(s"$scientificFormRegExpr cpu", "cpu")
+    str = str.replaceAll(s"$scientificFormRegExpr io", "io")
+    str = str.replaceAll(s"$scientificFormRegExpr network", "network")
+    str = str.replaceAll(s"$scientificFormRegExpr memory", "memory")
+    str
+  }
+
   private def assertEqualsOrExpand(tag: String, actual: String, expand: Boolean = true): Unit = {
     val expected = s"$${$tag}"
     if (!expand) {
@@ -397,13 +466,20 @@ case class StreamTableTestUtil(test: TableTestBase) extends TableTestUtil(test)
 
   override def getTableEnv: TableEnvironment = tableEnv
 
-  override def addDataStream[T: TypeInformation](
-      name: String, fields: Symbol*): Table = {
+  override def addDataStream[T: TypeInformation](name: String, fields: Symbol*): Table = {
     val table = env.fromElements[T]().toTable(tableEnv, fields: _*)
     tableEnv.registerTable(name, table)
     tableEnv.scan(name)
   }
 
+  def verifyPlanWithTrait(): Unit = {
+    doVerifyPlan(
+      SqlExplainLevel.EXPPLAN_ATTRIBUTES,
+      withRetractTraits = true,
+      withRowType = false,
+      printPlanBefore = true)
+  }
+
   def verifyPlanWithTrait(sql: String): Unit = {
     doVerifyPlan(
       sql,
@@ -444,6 +520,31 @@ case class StreamTableTestUtil(test: TableTestBase) extends TableTestUtil(test)
       TableConfigOptions.SQL_EXEC_MINIBATCH_ALLOW_LATENCY, 1000L)
     tableEnv.getConfig.getConf.setLong(TableConfigOptions.SQL_EXEC_MINIBATCH_SIZE, 3L)
   }
+
+  def createAppendTableSink(
+      fieldNames: Array[String],
+      fieldTypes: Array[InternalType]): AppendStreamTableSink[Row] = {
+    require(fieldNames.length == fieldTypes.length)
+    val typeInfos = fieldTypes.map(TypeConverters.createInternalTypeInfoFromInternalType)
+    new TestingAppendTableSink().configure(fieldNames, typeInfos)
+  }
+
+  def createUpsertTableSink(
+      keys: Array[Int],
+      fieldNames: Array[String],
+      fieldTypes: Array[InternalType]): UpsertStreamTableSink[BaseRow] = {
+    require(fieldNames.length == fieldTypes.length)
+    val typeInfos = fieldTypes.map(TypeConverters.createInternalTypeInfoFromInternalType)
+    new TestingUpsertTableSink(keys).configure(fieldNames, typeInfos)
+  }
+
+  def createRetractTableSink(
+      fieldNames: Array[String],
+      fieldTypes: Array[InternalType]): RetractStreamTableSink[Row] = {
+    require(fieldNames.length == fieldTypes.length)
+    val typeInfos = fieldTypes.map(TypeConverters.createInternalTypeInfoFromInternalType)
+    new TestingRetractTableSink().configure(fieldNames, typeInfos)
+  }
 }
 
 /**
@@ -459,6 +560,8 @@ case class BatchTableTestUtil(test: TableTestBase) extends TableTestUtil(test) {
 
   override def addDataStream[T: TypeInformation](
       name: String, fields: Symbol*): Table = {
+    // TODO use BatchTableEnvironment#fromBoundedStream when it's introduced
+
     val typeInfo = implicitly[TypeInformation[T]]
     BatchTableEnvUtil.registerCollection(
       tableEnv,
@@ -485,6 +588,14 @@ case class BatchTableTestUtil(test: TableTestBase) extends TableTestUtil(test) {
       .replaceBatchProgram(program).build()
     tableEnv.getConfig.setCalciteConfig(calciteConfig)
   }
+
+  def createCollectTableSink(
+      fieldNames: Array[String],
+      fieldTypes: Array[InternalType]): TableSink[Row] = {
+    require(fieldNames.length == fieldTypes.length)
+    val typeInfos = fieldTypes.map(TypeConverters.createInternalTypeInfoFromInternalType)
+    new CollectRowTableSink().configure(fieldNames, typeInfos)
+  }
 }
 
 /**
@@ -495,14 +606,18 @@ class TestTableSource(schema: TableSchema)
   with StreamTableSource[BaseRow] {
 
   override def getBoundedStream(
-      streamEnv: environment.StreamExecutionEnvironment): DataStream[BaseRow] = ???
+      streamEnv: environment.StreamExecutionEnvironment): DataStream[BaseRow] = {
+    streamEnv.fromCollection(List[BaseRow](), getReturnType)
+  }
 
   override def getDataStream(
-      execEnv: environment.StreamExecutionEnvironment): DataStream[BaseRow] = ???
+      execEnv: environment.StreamExecutionEnvironment): DataStream[BaseRow] = {
+    execEnv.fromCollection(List[BaseRow](), getReturnType)
+  }
 
   override def getReturnType: TypeInformation[BaseRow] = {
     val internalTypes = schema.getFieldTypes.map(TypeConverters.createInternalTypeFromTypeInfo)
-    new BaseRowTypeInfo(internalTypes: _*)
+    new BaseRowTypeInfo(internalTypes, schema.getFieldNames)
   }
 
   override def getTableSchema: TableSchema = schema
diff --git a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/BinaryHashPartitioner.java b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/BinaryHashPartitioner.java
index 9ebc76d..198047a 100644
--- a/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/BinaryHashPartitioner.java
+++ b/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/BinaryHashPartitioner.java
@@ -27,6 +27,8 @@ import org.apache.flink.table.generated.GeneratedHashFunction;
 import org.apache.flink.table.generated.HashFunction;
 import org.apache.flink.util.MathUtils;
 
+import java.util.Arrays;
+
 /**
  * Hash partitioner for {@link BinaryRow}.
  */
@@ -35,9 +37,11 @@ public class BinaryHashPartitioner extends StreamPartitioner<BaseRow> {
 	private GeneratedHashFunction genHashFunc;
 
 	private transient HashFunction hashFunc;
+	private String[] hashFieldNames;
 
-	public BinaryHashPartitioner(GeneratedHashFunction genHashFunc) {
+	public BinaryHashPartitioner(GeneratedHashFunction genHashFunc, String[] hashFieldNames) {
 		this.genHashFunc = genHashFunc;
+		this.hashFieldNames = hashFieldNames;
 	}
 
 	@Override
@@ -65,6 +69,6 @@ public class BinaryHashPartitioner extends StreamPartitioner<BaseRow> {
 
 	@Override
 	public String toString() {
-		return "HASH(" + genHashFunc.getClassName() + ")";
+		return "HASH" + Arrays.toString(hashFieldNames);
 	}
 }