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);
}
}