From 237e3a2df4b027995491b67adbbe77f8f3648704 Mon Sep 17 00:00:00 2001 From: godfreyhe Date: Tue, 21 May 2019 20:30:48 +0800 Subject: [PATCH] [FLINK-12575][table-planner-blink] Introduce planner rules to remove redundant shuffle and collation This closes #8499 i#refactor satisfyTraits method --- .../nodes/physical/FlinkPhysicalRel.scala | 4 +- .../nodes/physical/batch/BatchExecCalc.scala | 63 +- .../physical/batch/BatchExecCorrelate.scala | 83 +- .../batch/BatchExecGroupAggregateBase.scala | 11 +- .../batch/BatchExecHashAggregate.scala | 58 +- .../physical/batch/BatchExecHashJoin.scala | 33 + .../physical/batch/BatchExecJoinBase.scala | 156 +- .../batch/BatchExecLocalHashAggregate.scala | 40 +- .../batch/BatchExecLocalSortAggregate.scala | 48 +- .../batch/BatchExecNestedLoopJoin.scala | 6 + .../batch/BatchExecOverAggregate.scala | 105 +- .../nodes/physical/batch/BatchExecRank.scala | 125 +- .../batch/BatchExecSortAggregate.scala | 69 +- .../batch/BatchExecSortMergeJoin.scala | 54 +- .../nodes/physical/batch/BatchExecUnion.scala | 39 +- .../table/plan/rules/FlinkBatchRuleSets.scala | 25 +- .../plan/rules/FlinkStreamRuleSets.scala | 18 +- .../logical/WindowGroupReorderRule.scala | 137 ++ .../physical/FlinkExpandConversionRule.scala | 18 +- .../RemoveRedundantLocalHashAggRule.scala | 60 + .../batch/RemoveRedundantLocalRankRule.scala | 60 + .../RemoveRedundantLocalSortAggRule.scala | 110 ++ .../plan/batch/sql/DagOptimizationTest.xml | 45 +- .../plan/batch/sql/DeadlockBreakupTest.xml | 58 +- .../plan/batch/sql/RemoveCollationTest.xml | 708 +++++++++ .../plan/batch/sql/RemoveShuffleTest.xml | 1264 +++++++++++++++++ .../table/plan/batch/sql/SubplanReuseTest.xml | 63 +- .../plan/batch/sql/agg/GroupingSetsTest.xml | 18 +- .../plan/batch/sql/agg/OverAggregateTest.xml | 271 ++-- .../join/BroadcastHashSemiAntiJoinTest.xml | 13 +- .../batch/sql/join/NestedLoopJoinTest.xml | 4 +- .../sql/join/NestedLoopSemiAntiJoinTest.xml | 13 +- .../plan/batch/sql/join/SemiAntiJoinTest.xml | 197 ++- .../batch/sql/join/ShuffledHashJoinTest.xml | 8 +- .../sql/join/ShuffledHashSemiAntiJoinTest.xml | 140 +- .../plan/batch/sql/join/SingleRowJoinTest.xml | 43 +- .../plan/batch/sql/join/SortMergeJoinTest.xml | 8 +- .../sql/join/SortMergeSemiAntiJoinTest.xml | 140 +- .../logical/WindowGroupReorderRuleTest.xml | 280 ++++ .../RemoveRedundantLocalHashAggRuleTest.xml | 84 ++ .../RemoveRedundantLocalRankRuleTest.xml | 114 ++ .../RemoveRedundantLocalSortAggRuleTest.xml | 86 ++ .../plan/batch/sql/RemoveCollationTest.scala | 384 +++++ .../plan/batch/sql/RemoveShuffleTest.scala | 547 +++++++ .../batch/sql/agg/OverAggregateTest.scala | 51 +- .../logical/WindowGroupReorderRuleTest.scala | 179 +++ .../RemoveRedundantLocalHashAggRuleTest.scala | 71 + .../RemoveRedundantLocalRankRuleTest.scala | 72 + .../RemoveRedundantLocalSortAggRuleTest.scala | 66 + 49 files changed, 5693 insertions(+), 556 deletions(-) create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/WindowGroupReorderRule.scala create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/RemoveRedundantLocalHashAggRule.scala create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/RemoveRedundantLocalRankRule.scala create mode 100644 flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/RemoveRedundantLocalSortAggRule.scala create mode 100644 flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/RemoveCollationTest.xml create mode 100644 flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/RemoveShuffleTest.xml create mode 100644 flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/rules/logical/WindowGroupReorderRuleTest.xml create mode 100644 flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/rules/physical/batch/RemoveRedundantLocalHashAggRuleTest.xml create mode 100644 flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/rules/physical/batch/RemoveRedundantLocalRankRuleTest.xml create mode 100644 flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/rules/physical/batch/RemoveRedundantLocalSortAggRuleTest.xml create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/RemoveCollationTest.scala create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/RemoveShuffleTest.scala create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/WindowGroupReorderRuleTest.scala create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/physical/batch/RemoveRedundantLocalHashAggRuleTest.scala create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/physical/batch/RemoveRedundantLocalRankRuleTest.scala create mode 100644 flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/physical/batch/RemoveRedundantLocalSortAggRuleTest.scala diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/FlinkPhysicalRel.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/FlinkPhysicalRel.scala index cd85e7ba347..cc49d0ad37a 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/FlinkPhysicalRel.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/FlinkPhysicalRel.scala @@ -35,8 +35,8 @@ trait FlinkPhysicalRel extends FlinkRelNode { * * @param requiredTraitSet required traits * @return A converted node which satisfy required traits by inputs node of current node. - * Returns null if required traits cannot be pushed down into inputs. + * Returns None if required traits cannot be satisfied. */ - def satisfyTraitsByInput(requiredTraitSet: RelTraitSet): RelNode = null + def satisfyTraits(requiredTraitSet: RelTraitSet): Option[RelNode] = None } diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecCalc.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecCalc.scala index ace866d2839..1e184dc217f 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecCalc.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecCalc.scala @@ -24,6 +24,7 @@ import org.apache.flink.table.api.{BatchTableEnvironment, TableConfigOptions} import org.apache.flink.table.calcite.FlinkTypeFactory import org.apache.flink.table.codegen.{CalcCodeGenerator, CodeGeneratorContext} import org.apache.flink.table.dataformat.BaseRow +import org.apache.flink.table.plan.`trait`.{FlinkRelDistribution, FlinkRelDistributionTraitDef, TraitUtil} import org.apache.flink.table.plan.nodes.common.CommonCalc import org.apache.flink.table.plan.nodes.exec.{BatchExecNode, ExecNode} import org.apache.flink.table.plan.util.RelExplainUtil @@ -32,7 +33,9 @@ import org.apache.calcite.plan._ import org.apache.calcite.rel._ import org.apache.calcite.rel.`type`.RelDataType import org.apache.calcite.rel.core.Calc -import org.apache.calcite.rex.RexProgram +import org.apache.calcite.rex.{RexCall, RexInputRef, RexProgram} +import org.apache.calcite.sql.SqlKind +import org.apache.calcite.util.mapping.{Mapping, MappingType, Mappings} import java.util @@ -57,6 +60,64 @@ class BatchExecCalc( new BatchExecCalc(cluster, traitSet, child, program, outputRowType) } + override def satisfyTraits(requiredTraitSet: RelTraitSet): Option[RelNode] = { + val requiredDistribution = requiredTraitSet.getTrait(FlinkRelDistributionTraitDef.INSTANCE) + // Does not push broadcast distribution trait down into Calc. + if (requiredDistribution.getType == RelDistribution.Type.BROADCAST_DISTRIBUTED) { + return None + } + val projects = calcProgram.getProjectList.map(calcProgram.expandLocalRef) + + def getProjectMapping: Mapping = { + val mapping = Mappings.create(MappingType.INVERSE_FUNCTION, + getInput.getRowType.getFieldCount, projects.size) + projects.zipWithIndex.foreach { + case (project, index) => + project match { + case inputRef: RexInputRef => mapping.set(inputRef.getIndex, index) + case call: RexCall if call.getKind == SqlKind.AS => + call.getOperands.head match { + case inputRef: RexInputRef => mapping.set(inputRef.getIndex, index) + case _ => // ignore + } + case _ => // ignore + } + } + mapping.inverse() + } + + val mapping = getProjectMapping + val appliedDistribution = requiredDistribution.apply(mapping) + // If both distribution and collation can be satisfied, satisfy both. If only distribution + // can be satisfied, only satisfy distribution. There is no possibility to only satisfy + // collation here except for there is no distribution requirement. + if ((!requiredDistribution.isTop) && (appliedDistribution eq FlinkRelDistribution.ANY)) { + return None + } + + val requiredCollation = requiredTraitSet.getTrait(RelCollationTraitDef.INSTANCE) + val appliedCollation = TraitUtil.apply(requiredCollation, mapping) + val canCollationPushedDown = !appliedCollation.getFieldCollations.isEmpty + // If required traits only contains collation requirements, but collation keys are not columns + // from input, then no need to satisfy required traits. + if ((appliedDistribution eq FlinkRelDistribution.ANY) && !canCollationPushedDown) { + return None + } + + var inputRequiredTraits = getInput.getTraitSet + var providedTraits = getTraitSet + if (!appliedDistribution.isTop) { + inputRequiredTraits = inputRequiredTraits.replace(appliedDistribution) + providedTraits = providedTraits.replace(requiredDistribution) + } + if (canCollationPushedDown) { + inputRequiredTraits = inputRequiredTraits.replace(appliedCollation) + providedTraits = providedTraits.replace(requiredCollation) + } + val newInput = RelOptRule.convert(getInput, inputRequiredTraits) + Some(copy(providedTraits, Seq(newInput))) + } + //~ ExecNode methods ----------------------------------------------------------- override def getDamBehavior = DamBehavior.PIPELINED diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecCorrelate.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecCorrelate.scala index 7097e82c1a4..16f0cb52e7d 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecCorrelate.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecCorrelate.scala @@ -23,16 +23,18 @@ import org.apache.flink.table.api.{BatchTableEnvironment, TableConfigOptions} import org.apache.flink.table.codegen.{CodeGeneratorContext, CorrelateCodeGenerator} import org.apache.flink.table.dataformat.BaseRow import org.apache.flink.table.functions.utils.TableSqlFunction +import org.apache.flink.table.plan.`trait`.{FlinkRelDistribution, FlinkRelDistributionTraitDef, TraitUtil} import org.apache.flink.table.plan.nodes.exec.{BatchExecNode, ExecNode} import org.apache.flink.table.plan.nodes.logical.FlinkLogicalTableFunctionScan import org.apache.flink.table.plan.util.RelExplainUtil -import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} +import org.apache.calcite.plan.{RelOptCluster, RelOptRule, RelTraitSet} import org.apache.calcite.rel.`type`.RelDataType import org.apache.calcite.rel.core.Correlate -import org.apache.calcite.rel.{RelNode, RelWriter, SingleRel} -import org.apache.calcite.rex.{RexCall, RexNode, RexProgram} -import org.apache.calcite.sql.SemiJoinType +import org.apache.calcite.rel.{RelCollationTraitDef, RelDistribution, RelFieldCollation, RelNode, RelWriter, SingleRel} +import org.apache.calcite.rex.{RexCall, RexInputRef, RexNode, RexProgram} +import org.apache.calcite.sql.{SemiJoinType, SqlKind} +import org.apache.calcite.util.mapping.{Mapping, MappingType, Mappings} import java.util @@ -94,6 +96,79 @@ class BatchExecCorrelate( .itemIf("condition", condition.orNull, condition.isDefined) } + override def satisfyTraits(requiredTraitSet: RelTraitSet): Option[RelNode] = { + val requiredDistribution = requiredTraitSet.getTrait(FlinkRelDistributionTraitDef.INSTANCE) + // Correlate could not provide broadcast distribution + if (requiredDistribution.getType == RelDistribution.Type.BROADCAST_DISTRIBUTED) { + return None + } + + def getOutputInputMapping: Mapping = { + val inputFieldCnt = getInput.getRowType.getFieldCount + projectProgram match { + case Some(program) => + val projects = program.getProjectList.map(program.expandLocalRef) + val mapping = Mappings.create(MappingType.INVERSE_FUNCTION, inputFieldCnt, projects.size) + projects.zipWithIndex.foreach { + case (project, index) => + project match { + case inputRef: RexInputRef => mapping.set(inputRef.getIndex, index) + case call: RexCall if call.getKind == SqlKind.AS => + call.getOperands.head match { + case inputRef: RexInputRef => mapping.set(inputRef.getIndex, index) + case _ => // ignore + } + case _ => // ignore + } + } + mapping.inverse() + case _ => + val mapping = Mappings.create(MappingType.FUNCTION, inputFieldCnt, inputFieldCnt) + (0 until inputFieldCnt).foreach { + index => mapping.set(index, index) + } + mapping + } + } + + val mapping = getOutputInputMapping + val appliedDistribution = requiredDistribution.apply(mapping) + // If both distribution and collation can be satisfied, satisfy both. If only distribution + // can be satisfied, only satisfy distribution. There is no possibility to only satisfy + // collation here except for there is no distribution requirement. + if ((!requiredDistribution.isTop) && (appliedDistribution eq FlinkRelDistribution.ANY)) { + return None + } + + val requiredCollation = requiredTraitSet.getTrait(RelCollationTraitDef.INSTANCE) + val appliedCollation = TraitUtil.apply(requiredCollation, mapping) + // the required collation can be satisfied if field collations are not empty + // and the direction of each field collation is non-STRICTLY + val canSatisfyCollation = appliedCollation.getFieldCollations.nonEmpty && + !appliedCollation.getFieldCollations.exists { c => + (c.getDirection eq RelFieldCollation.Direction.STRICTLY_ASCENDING) || + (c.getDirection eq RelFieldCollation.Direction.STRICTLY_DESCENDING) + } + // If required traits only contains collation requirements, but collation keys are not columns + // from input, then no need to satisfy required traits. + if ((appliedDistribution eq FlinkRelDistribution.ANY) && !canSatisfyCollation) { + return None + } + + var inputRequiredTraits = getInput.getTraitSet + var providedTraits = getTraitSet + if (!appliedDistribution.isTop) { + inputRequiredTraits = inputRequiredTraits.replace(appliedDistribution) + providedTraits = providedTraits.replace(requiredDistribution) + } + if (canSatisfyCollation) { + inputRequiredTraits = inputRequiredTraits.replace(appliedCollation) + providedTraits = providedTraits.replace(requiredCollation) + } + val newInput = RelOptRule.convert(getInput, inputRequiredTraits) + Some(copy(providedTraits, Seq(newInput))) + } + //~ ExecNode methods ----------------------------------------------------------- override def getDamBehavior: DamBehavior = DamBehavior.PIPELINED diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecGroupAggregateBase.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecGroupAggregateBase.scala index 60481a66edc..fc3456010f7 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecGroupAggregateBase.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecGroupAggregateBase.scala @@ -18,9 +18,9 @@ package org.apache.flink.table.plan.nodes.physical.batch -import org.apache.flink.table.api.TableException +import org.apache.flink.table.api.{AggPhaseEnforcer, PlannerConfigOptions, TableException} import org.apache.flink.table.functions.UserDefinedFunction -import org.apache.flink.table.plan.util.RelExplainUtil +import org.apache.flink.table.plan.util.{FlinkRelOptUtil, RelExplainUtil} import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.`type`.RelDataType @@ -84,4 +84,11 @@ abstract class BatchExecGroupAggregateBase( isFinal) } + protected def isEnforceTwoStageAgg: Boolean = { + val tableConfig = FlinkRelOptUtil.getTableConfigFromContext(this) + val aggConfig = tableConfig.getConf.getString( + PlannerConfigOptions.SQL_OPTIMIZER_AGG_PHASE_ENFORCER) + AggPhaseEnforcer.TWO_PHASE.toString.equalsIgnoreCase(aggConfig) + } + } diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecHashAggregate.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecHashAggregate.scala index 19bc869f47d..2827987790c 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecHashAggregate.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecHashAggregate.scala @@ -19,19 +19,24 @@ package org.apache.flink.table.plan.nodes.physical.batch import org.apache.flink.runtime.operators.DamBehavior import org.apache.flink.streaming.api.transformations.StreamTransformation -import org.apache.flink.table.api.TableConfig +import org.apache.flink.table.api.{PlannerConfigOptions, TableConfig} import org.apache.flink.table.dataformat.BaseRow import org.apache.flink.table.functions.UserDefinedFunction -import org.apache.flink.table.plan.util.RelExplainUtil +import org.apache.flink.table.plan.`trait`.{FlinkRelDistribution, FlinkRelDistributionTraitDef} +import org.apache.flink.table.plan.util.{FlinkRelOptUtil, RelExplainUtil} -import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} +import org.apache.calcite.plan.{RelOptCluster, RelOptRule, RelTraitSet} +import org.apache.calcite.rel.RelDistribution.Type.{HASH_DISTRIBUTED, SINGLETON} import org.apache.calcite.rel.`type`.RelDataType import org.apache.calcite.rel.core.AggregateCall import org.apache.calcite.rel.{RelNode, RelWriter} import org.apache.calcite.tools.RelBuilder +import org.apache.calcite.util.{ImmutableIntList, Util} import java.util +import scala.collection.JavaConversions._ + /** * Batch physical RelNode for (global) hash-based aggregate operator. * @@ -94,6 +99,53 @@ class BatchExecHashAggregate( isGlobal = true)) } + override def satisfyTraits(requiredTraitSet: RelTraitSet): Option[RelNode] = { + val requiredDistribution = requiredTraitSet.getTrait(FlinkRelDistributionTraitDef.INSTANCE) + val canSatisfy = requiredDistribution.getType match { + case SINGLETON => grouping.length == 0 + case HASH_DISTRIBUTED => + val shuffleKeys = requiredDistribution.getKeys + val groupKeysList = ImmutableIntList.of(grouping.indices.toArray: _*) + if (requiredDistribution.requireStrict) { + shuffleKeys == groupKeysList + } else if (Util.startsWith(shuffleKeys, groupKeysList)) { + // If required distribution is not strict, Hash[a] can satisfy Hash[a, b]. + // so return true if shuffleKeys(Hash[a, b]) start with groupKeys(Hash[a]) + true + } else { + // If partialKey is enabled, try to use partial key to satisfy the required distribution + val tableConfig = FlinkRelOptUtil.getTableConfigFromContext(this) + val partialKeyEnabled = tableConfig.getConf.getBoolean( + PlannerConfigOptions.SQL_OPTIMIZER_SHUFFLE_PARTIAL_KEY_ENABLED) + partialKeyEnabled && groupKeysList.containsAll(shuffleKeys) + } + case _ => false + } + if (!canSatisfy) { + return None + } + + val inputRequiredDistribution = requiredDistribution.getType match { + case SINGLETON => requiredDistribution + case HASH_DISTRIBUTED => + val shuffleKeys = requiredDistribution.getKeys + val groupKeysList = ImmutableIntList.of(grouping.indices.toArray: _*) + if (requiredDistribution.requireStrict) { + FlinkRelDistribution.hash(grouping, requireStrict = true) + } else if (Util.startsWith(shuffleKeys, groupKeysList)) { + // Hash[a] can satisfy Hash[a, b] + FlinkRelDistribution.hash(grouping, requireStrict = false) + } else { + // use partial key to satisfy the required distribution + FlinkRelDistribution.hash(shuffleKeys.map(grouping(_)).toArray, requireStrict = false) + } + } + + val newInput = RelOptRule.convert(getInput, inputRequiredDistribution) + val newProvidedTraitSet = getTraitSet.replace(requiredDistribution) + Some(copy(newProvidedTraitSet, Seq(newInput))) + } + //~ ExecNode methods ----------------------------------------------------------- override def getDamBehavior = DamBehavior.FULL_DAM diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecHashJoin.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecHashJoin.scala index 579a7065cb3..2205c0fd764 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecHashJoin.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecHashJoin.scala @@ -21,7 +21,9 @@ import org.apache.flink.runtime.operators.DamBehavior import org.apache.flink.streaming.api.transformations.StreamTransformation import org.apache.flink.table.api.{BatchTableEnvironment, TableException} import org.apache.flink.table.dataformat.BaseRow +import org.apache.flink.table.plan.`trait`.{FlinkRelDistribution, FlinkRelDistributionTraitDef} import org.apache.flink.table.plan.cost.{FlinkCost, FlinkCostFactory} +import org.apache.flink.table.plan.nodes.FlinkConventions import org.apache.flink.table.plan.nodes.exec.ExecNode import org.apache.flink.table.plan.util.{FlinkRelMdUtil, JoinUtil} import org.apache.flink.table.runtime.join.HashJoinType @@ -127,6 +129,37 @@ class BatchExecHashJoin( } } + override def satisfyTraits(requiredTraitSet: RelTraitSet): Option[RelNode] = { + if (!isBroadcast) { + satisfyTraitsOnNonBroadcastHashJoin(requiredTraitSet) + } else { + satisfyTraitsOnBroadcastJoin(requiredTraitSet, leftIsBuild) + } + } + + private def satisfyTraitsOnNonBroadcastHashJoin( + requiredTraitSet: RelTraitSet): Option[RelNode] = { + val requiredDistribution = requiredTraitSet.getTrait(FlinkRelDistributionTraitDef.INSTANCE) + val (canSatisfyDistribution, leftRequiredDistribution, rightRequiredDistribution) = + satisfyHashDistributionOnNonBroadcastJoin(requiredDistribution) + if (!canSatisfyDistribution) { + return None + } + + val toRestrictHashDistributionByKeys = (distribution: FlinkRelDistribution) => + getCluster.getPlanner + .emptyTraitSet + .replace(FlinkConventions.BATCH_PHYSICAL) + .replace(distribution) + val leftRequiredTraits = toRestrictHashDistributionByKeys(leftRequiredDistribution) + val rightRequiredTraits = toRestrictHashDistributionByKeys(rightRequiredDistribution) + val newLeft = RelOptRule.convert(getLeft, leftRequiredTraits) + val newRight = RelOptRule.convert(getRight, rightRequiredTraits) + val providedTraits = getTraitSet.replace(requiredDistribution) + // HashJoin can not satisfy collation. + Some(copy(providedTraits, Seq(newLeft, newRight))) + } + //~ ExecNode methods ----------------------------------------------------------- override def getDamBehavior: DamBehavior = { diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecJoinBase.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecJoinBase.scala index f22a64accb6..ff7a9c3467b 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecJoinBase.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecJoinBase.scala @@ -22,13 +22,19 @@ import org.apache.flink.table.api.TableConfig import org.apache.flink.table.codegen.{CodeGeneratorContext, ExprCodeGenerator, FunctionCodeGenerator} import org.apache.flink.table.dataformat.BaseRow import org.apache.flink.table.generated.GeneratedJoinCondition +import org.apache.flink.table.plan.`trait`.{FlinkRelDistribution, FlinkRelDistributionTraitDef} import org.apache.flink.table.plan.nodes.common.CommonPhysicalJoin import org.apache.flink.table.plan.nodes.exec.BatchExecNode -import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} -import org.apache.calcite.rel.RelNode +import org.apache.calcite.plan.{RelOptCluster, RelOptRule, RelTraitSet} +import org.apache.calcite.rel.RelDistribution.Type.{HASH_DISTRIBUTED, RANGE_DISTRIBUTED} import org.apache.calcite.rel.core.{Join, JoinRelType} +import org.apache.calcite.rel.{RelCollations, RelNode} import org.apache.calcite.rex.RexNode +import org.apache.calcite.util.ImmutableIntList + +import scala.collection.JavaConversions._ +import scala.collection.mutable /** * Batch physical RelNode for [[Join]] @@ -70,4 +76,150 @@ abstract class BatchExecJoinBase( "JoinConditionFunction", body) } + + /** + * Try to satisfy hash distribution on Non-BroadcastJoin (including SortMergeJoin and + * Non-Broadcast HashJoin). + * + * @param requiredDistribution distribution requirement + * @return a Tuple including 3 element. + * The first element is a flag which indicates whether the requirement can be satisfied. + * The second element is the distribution requirement of left child if the requirement + * can be push down into join. + * The third element is the distribution requirement of right child if the requirement + * can be push down into join. + */ + def satisfyHashDistributionOnNonBroadcastJoin( + requiredDistribution: FlinkRelDistribution + ): (Boolean, FlinkRelDistribution, FlinkRelDistribution) = { + // Only Non-broadcast HashJoin could provide HashDistribution + if (requiredDistribution.getType != HASH_DISTRIBUTED) { + return (false, null, null) + } + // Full outer join cannot provide Hash distribute because it will generate null for left/right + // side if there is no match row. + if (joinType == JoinRelType.FULL) { + return (false, null, null) + } + + val leftKeys = joinInfo.leftKeys + val rightKeys = joinInfo.rightKeys + val leftKeysToRightKeys = leftKeys.zip(rightKeys).toMap + val rightKeysToLeftKeys = rightKeys.zip(leftKeys).toMap + val leftFieldCnt = getLeft.getRowType.getFieldCount + val requiredShuffleKeys = requiredDistribution.getKeys + val requiredLeftShuffleKeys = mutable.ArrayBuffer[Int]() + val requiredRightShuffleKeys = mutable.ArrayBuffer[Int]() + requiredShuffleKeys.foreach { key => + if (key < leftFieldCnt && joinType != JoinRelType.RIGHT) { + leftKeysToRightKeys.get(key) match { + case Some(rk) => + requiredLeftShuffleKeys += key + requiredRightShuffleKeys += rk + case None if requiredDistribution.requireStrict => + // Cannot satisfy required hash distribution due to required distribution is restrict + // however the key is not found in right + return (false, null, null) + case _ => // do nothing + } + } else if (key >= leftFieldCnt && + (joinType == JoinRelType.RIGHT || joinType == JoinRelType.INNER)) { + val keysOnRightChild = key - leftFieldCnt + rightKeysToLeftKeys.get(keysOnRightChild) match { + case Some(lk) => + requiredLeftShuffleKeys += lk + requiredRightShuffleKeys += keysOnRightChild + case None if requiredDistribution.requireStrict => + // Cannot satisfy required hash distribution due to required distribution is restrict + // however the key is not found in left + return (false, null, null) + case _ => // do nothing + } + } else { + // cannot satisfy required hash distribution if requirement shuffle keys are not come from + // left side when Join is LOJ or are not come from right side when Join is ROJ. + return (false, null, null) + } + } + if (requiredLeftShuffleKeys.isEmpty) { + // the join can not satisfy the required hash distribution + // due to the required input shuffle keys are empty + return (false, null, null) + } + + val (leftShuffleKeys, rightShuffleKeys) = if (joinType == JoinRelType.INNER && + !requiredDistribution.requireStrict) { + (requiredLeftShuffleKeys.distinct, requiredRightShuffleKeys.distinct) + } else { + (requiredLeftShuffleKeys, requiredRightShuffleKeys) + } + (true, + FlinkRelDistribution.hash(ImmutableIntList.of(leftShuffleKeys: _*), requireStrict = true), + FlinkRelDistribution.hash(ImmutableIntList.of(rightShuffleKeys: _*), requireStrict = true)) + } + + /** + * Try to satisfy the given required traits on BroadcastJoin (including Broadcast-HashJoin and + * NestedLoopJoin). + * + * @param requiredTraitSet requirement traitSets + * @return Equivalent Join which satisfies required traitSet, return null if + * requirement cannot be satisfied. + */ + protected def satisfyTraitsOnBroadcastJoin( + requiredTraitSet: RelTraitSet, + leftIsBroadcast: Boolean): Option[RelNode] = { + val requiredDistribution = requiredTraitSet.getTrait(FlinkRelDistributionTraitDef.INSTANCE) + val keys = requiredDistribution.getKeys + val left = getLeft + val right = getRight + val leftFieldCnt = left.getRowType.getFieldCount + val canSatisfy = requiredDistribution.getType match { + case HASH_DISTRIBUTED | RANGE_DISTRIBUTED => + // required distribution can be satisfied only if distribution keys all from + // non-broadcast side of BroadcastJoin + if (leftIsBroadcast) { + // all distribution keys must come from right child + keys.forall(_ >= leftFieldCnt) + } else { + // all distribution keys must come from left child + keys.forall(_ < leftFieldCnt) + } + // SINGLETON, BROADCAST_DISTRIBUTED, ANY, RANDOM_DISTRIBUTED, ROUND_ROBIN_DISTRIBUTED + // distribution cannot be pushed down. + case _ => false + } + if (!canSatisfy) { + return None + } + + val keysInProbeSide = if (leftIsBroadcast) { + ImmutableIntList.of(keys.map(_ - leftFieldCnt): _ *) + } else { + keys + } + + val inputRequiredDistribution = requiredDistribution.getType match { + case HASH_DISTRIBUTED => + FlinkRelDistribution.hash(keysInProbeSide, requiredDistribution.requireStrict) + case RANGE_DISTRIBUTED => + FlinkRelDistribution.range(keysInProbeSide) + } + // remove collation traits from input traits and provided traits + val (newLeft, newRight) = if (leftIsBroadcast) { + val rightRequiredTraitSet = right.getTraitSet + .replace(inputRequiredDistribution) + .replace(RelCollations.EMPTY) + val newRight = RelOptRule.convert(right, rightRequiredTraitSet) + (left, newRight) + } else { + val leftRequiredTraitSet = left.getTraitSet + .replace(inputRequiredDistribution) + .replace(RelCollations.EMPTY) + val newLeft = RelOptRule.convert(left, leftRequiredTraitSet) + (newLeft, right) + } + val providedTraitSet = requiredTraitSet.replace(RelCollations.EMPTY) + Some(copy(providedTraitSet, Seq(newLeft, newRight))) + } } diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecLocalHashAggregate.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecLocalHashAggregate.scala index 9bf440ce201..b475f4653cb 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecLocalHashAggregate.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecLocalHashAggregate.scala @@ -19,21 +19,24 @@ package org.apache.flink.table.plan.nodes.physical.batch import org.apache.flink.runtime.operators.DamBehavior import org.apache.flink.streaming.api.transformations.StreamTransformation -import org.apache.flink.table.api.{BatchTableEnvironment, TableConfig} +import org.apache.flink.table.api.TableConfig import org.apache.flink.table.dataformat.BaseRow import org.apache.flink.table.functions.UserDefinedFunction -import org.apache.flink.table.plan.nodes.exec.ExecNode +import org.apache.flink.table.plan.`trait`.{FlinkRelDistribution, FlinkRelDistributionTraitDef} import org.apache.flink.table.plan.util.RelExplainUtil -import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} +import org.apache.calcite.plan.{RelOptCluster, RelOptRule, RelTraitSet} +import org.apache.calcite.rel.RelDistribution.Type import org.apache.calcite.rel.`type`.RelDataType import org.apache.calcite.rel.core.AggregateCall import org.apache.calcite.rel.{RelNode, RelWriter} import org.apache.calcite.tools.RelBuilder +import org.apache.calcite.util.ImmutableIntList import java.util import scala.collection.JavaConversions._ +import scala.collection.mutable /** * Batch physical RelNode for local hash-based aggregate operator. @@ -92,6 +95,37 @@ class BatchExecLocalHashAggregate( isGlobal = false)) } + override def satisfyTraits(requiredTraitSet: RelTraitSet): Option[RelNode] = { + // Does not to try to satisfy requirement by localAgg's input if enforce to use two-stage agg. + if (isEnforceTwoStageAgg) { + return None + } + + val requiredDistribution = requiredTraitSet.getTrait(FlinkRelDistributionTraitDef.INSTANCE) + val canSatisfy = requiredDistribution.getType match { + case Type.HASH_DISTRIBUTED | Type.RANGE_DISTRIBUTED => + val groupCount = grouping.length + // Cannot satisfy distribution if keys are not group keys of agg + requiredDistribution.getKeys.forall(_ < groupCount) + case _ => false + } + if (!canSatisfy) { + return None + } + + val keys = requiredDistribution.getKeys.map(grouping(_)) + val inputRequiredDistributionKeys = ImmutableIntList.of(keys: _*) + val inputRequiredDistribution = requiredDistribution.getType match { + case Type.HASH_DISTRIBUTED => + FlinkRelDistribution.hash(inputRequiredDistributionKeys, requiredDistribution.requireStrict) + case Type.RANGE_DISTRIBUTED => FlinkRelDistribution.range(inputRequiredDistributionKeys) + } + val inputRequiredTraits = input.getTraitSet.replace(inputRequiredDistribution) + val newInput = RelOptRule.convert(getInput, inputRequiredTraits) + val providedTraits = getTraitSet.replace(requiredDistribution) + Some(copy(providedTraits, Seq(newInput))) + } + //~ ExecNode methods ----------------------------------------------------------- override def getDamBehavior: DamBehavior = { diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecLocalSortAggregate.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecLocalSortAggregate.scala index bac96fd18ef..e62c850aee9 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecLocalSortAggregate.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecLocalSortAggregate.scala @@ -22,16 +22,22 @@ import org.apache.flink.streaming.api.transformations.StreamTransformation import org.apache.flink.table.api.TableConfig import org.apache.flink.table.dataformat.BaseRow import org.apache.flink.table.functions.UserDefinedFunction -import org.apache.flink.table.plan.util.RelExplainUtil +import org.apache.flink.table.plan.`trait`.{FlinkRelDistribution, FlinkRelDistributionTraitDef} +import org.apache.flink.table.plan.util.{FlinkRelOptUtil, RelExplainUtil} -import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} +import org.apache.calcite.plan.{RelOptCluster, RelOptRule, RelTraitSet} +import org.apache.calcite.rel.RelDistribution.Type import org.apache.calcite.rel._ import org.apache.calcite.rel.`type`.RelDataType import org.apache.calcite.rel.core.AggregateCall import org.apache.calcite.tools.RelBuilder +import org.apache.calcite.util.ImmutableIntList import java.util +import scala.collection.JavaConversions._ +import scala.collection.mutable + /** * Batch physical RelNode for local sort-based aggregate operator. * @@ -90,6 +96,44 @@ class BatchExecLocalSortAggregate( isGlobal = false)) } + override def satisfyTraits(requiredTraitSet: RelTraitSet): Option[RelNode] = { + // Does not to try to satisfy requirement by localAgg's input if enforce to use two-stage agg. + if (isEnforceTwoStageAgg) { + return None + } + val groupCount = grouping.length + val requiredDistribution = requiredTraitSet.getTrait(FlinkRelDistributionTraitDef.INSTANCE) + val canSatisfy = requiredDistribution.getType match { + case Type.HASH_DISTRIBUTED | Type.RANGE_DISTRIBUTED => + // Cannot satisfy distribution if keys are not group keys of agg + requiredDistribution.getKeys.forall(_ < groupCount) + case _ => false + } + if (!canSatisfy) { + return None + } + + val keys = requiredDistribution.getKeys.map(grouping(_)) + val inputRequiredDistributionKeys = ImmutableIntList.of(keys: _*) + val inputRequiredDistribution = requiredDistribution.getType match { + case Type.HASH_DISTRIBUTED => + FlinkRelDistribution.hash(inputRequiredDistributionKeys, requiredDistribution.requireStrict) + case Type.RANGE_DISTRIBUTED => + FlinkRelDistribution.range(inputRequiredDistributionKeys) + } + val requiredCollation = requiredTraitSet.getTrait(RelCollationTraitDef.INSTANCE) + val providedFieldCollations = (0 until groupCount).map(FlinkRelOptUtil.ofRelFieldCollation) + val providedCollation = RelCollations.of(providedFieldCollations) + val newProvidedTraits = if (providedCollation.satisfies(requiredCollation)) { + getTraitSet.replace(requiredDistribution).replace(requiredCollation) + } else { + getTraitSet.replace(requiredDistribution) + } + val inputRequiredTraits = getInput.getTraitSet.replace(inputRequiredDistribution) + val newInput = RelOptRule.convert(getInput, inputRequiredTraits) + Some(copy(newProvidedTraits, Seq(newInput))) + } + //~ ExecNode methods ----------------------------------------------------------- override def getDamBehavior: DamBehavior = { diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecNestedLoopJoin.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecNestedLoopJoin.scala index a9919f8805a..1043c6506cf 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecNestedLoopJoin.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecNestedLoopJoin.scala @@ -82,6 +82,7 @@ class BatchExecNestedLoopJoin( if (leftRowCnt == null || rightRowCnt == null) { return null } + val buildRel = if (leftIsBuild) getLeft else getRight val buildRows = mq.getRowCount(buildRel) val buildRowSize = mq.getAverageRowSize(buildRel) @@ -104,6 +105,11 @@ class BatchExecNestedLoopJoin( } } + override def satisfyTraits(requiredTraitSet: RelTraitSet): Option[RelNode] = { + // Assume NestedLoopJoin always broadcast data from child which smaller. + satisfyTraitsOnBroadcastJoin(requiredTraitSet, leftIsBuild) + } + //~ ExecNode methods ----------------------------------------------------------- override def getDamBehavior: DamBehavior = DamBehavior.PIPELINED diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecOverAggregate.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecOverAggregate.scala index 14f50a8af25..1c07de73425 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecOverAggregate.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecOverAggregate.scala @@ -22,7 +22,7 @@ import org.apache.flink.runtime.operators.DamBehavior import org.apache.flink.streaming.api.transformations.{OneInputTransformation, StreamTransformation} import org.apache.flink.table.CalcitePair import org.apache.flink.table.`type`.InternalTypes -import org.apache.flink.table.api.{BatchTableEnvironment, TableConfig, TableConfigOptions} +import org.apache.flink.table.api.{BatchTableEnvironment, PlannerConfigOptions, TableConfig, TableConfigOptions} import org.apache.flink.table.calcite.FlinkTypeFactory import org.apache.flink.table.codegen.CodeGeneratorContext import org.apache.flink.table.codegen.agg.AggsHandlerCodeGenerator @@ -31,17 +31,19 @@ import org.apache.flink.table.codegen.sort.ComparatorCodeGenerator import org.apache.flink.table.dataformat.{BaseRow, BinaryRow} import org.apache.flink.table.functions.UserDefinedFunction import org.apache.flink.table.generated.GeneratedRecordComparator +import org.apache.flink.table.plan.`trait`.{FlinkRelDistribution, FlinkRelDistributionTraitDef} import org.apache.flink.table.plan.cost.{FlinkCost, FlinkCostFactory} import org.apache.flink.table.plan.nodes.exec.{BatchExecNode, ExecNode} import org.apache.flink.table.plan.nodes.physical.batch.OverWindowMode.OverWindowMode import org.apache.flink.table.plan.util.AggregateUtil.transformToBatchAggregateInfoList import org.apache.flink.table.plan.util.OverAggregateUtil.getLongBoundary -import org.apache.flink.table.plan.util.{OverAggregateUtil, RelExplainUtil} +import org.apache.flink.table.plan.util.{FlinkRelOptUtil, OverAggregateUtil, RelExplainUtil} import org.apache.flink.table.runtime.over.frame.OffsetOverFrame.CalcOffsetFunc import org.apache.flink.table.runtime.over.frame.{InsensitiveOverFrame, OffsetOverFrame, OverWindowFrame, RangeSlidingOverFrame, RangeUnboundedFollowingOverFrame, RangeUnboundedPrecedingOverFrame, RowSlidingOverFrame, RowUnboundedFollowingOverFrame, RowUnboundedPrecedingOverFrame, UnboundedOverWindowFrame} import org.apache.flink.table.runtime.over.{BufferDataOverWindowOperator, NonBufferOverWindowOperator} import org.apache.calcite.plan._ +import org.apache.calcite.rel.RelDistribution.Type._ import org.apache.calcite.rel._ import org.apache.calcite.rel.`type`.RelDataType import org.apache.calcite.rel.core.Window.Group @@ -51,6 +53,7 @@ import org.apache.calcite.rex.RexWindowBound import org.apache.calcite.sql.SqlKind import org.apache.calcite.sql.fun.SqlLeadLagAggFunction import org.apache.calcite.tools.RelBuilder +import org.apache.calcite.util.ImmutableIntList import java.util @@ -179,7 +182,7 @@ class BatchExecOverAggregate( yield new CalcitePair[AggregateCall, String](aggregateCalls.get(i), "windowAgg$" + i) } - private[flink] def splitOutOffsetOrInsensitiveGroup() + private def splitOutOffsetOrInsensitiveGroup() : Seq[(OverWindowMode, Window.Group, Seq[(AggregateCall, UserDefinedFunction)])] = { def compareTo(o1: Window.RexWinAggCall, o2: Window.RexWinAggCall): Boolean = { @@ -242,6 +245,102 @@ class BatchExecOverAggregate( windowGroupInfo } + override def satisfyTraits(requiredTraitSet: RelTraitSet): Option[RelNode] = { + val requiredDistribution = requiredTraitSet.getTrait(FlinkRelDistributionTraitDef.INSTANCE) + val requiredCollation = requiredTraitSet.getTrait(RelCollationTraitDef.INSTANCE) + if (requiredDistribution.getType == ANY && requiredCollation.getFieldCollations.isEmpty) { + return None + } + + val selfProvidedTraitSet = inferProvidedTraitSet() + if (selfProvidedTraitSet.satisfies(requiredTraitSet)) { + // Current node can satisfy the requiredTraitSet,return the current node with ProvidedTraitSet + return Some(copy(selfProvidedTraitSet, Seq(getInput))) + } + + val inputFieldCnt = getInput.getRowType.getFieldCount + val canSatisfy = if (requiredDistribution.getType == ANY) { + true + } else { + if (!grouping.isEmpty) { + if (requiredDistribution.requireStrict) { + requiredDistribution.getKeys == ImmutableIntList.of(grouping: _*) + } else { + val isAllFieldsFromInput = requiredDistribution.getKeys.forall(_ < inputFieldCnt) + if (isAllFieldsFromInput) { + val tableConfig = FlinkRelOptUtil.getTableConfigFromContext(this) + if (tableConfig.getConf.getBoolean( + PlannerConfigOptions.SQL_OPTIMIZER_SHUFFLE_PARTIAL_KEY_ENABLED)) { + ImmutableIntList.of(grouping: _*).containsAll(requiredDistribution.getKeys) + } else { + requiredDistribution.getKeys == ImmutableIntList.of(grouping: _*) + } + } else { + // If requirement distribution keys are not all comes from input directly, + // cannot satisfy requirement distribution and collations. + false + } + } + } else { + requiredDistribution.getType == SINGLETON + } + } + // If OverAggregate can provide distribution, but it's traits cannot satisfy required + // distribution, cannot push down distribution and collation requirement (because later + // shuffle will destroy previous collation. + if (!canSatisfy) { + return None + } + + var inputRequiredTraits = getInput.getTraitSet + var providedTraits = selfProvidedTraitSet + val providedCollation = selfProvidedTraitSet.getTrait(RelCollationTraitDef.INSTANCE) + if (!requiredDistribution.isTop) { + inputRequiredTraits = inputRequiredTraits.replace(requiredDistribution) + providedTraits = providedTraits.replace(requiredDistribution) + } + + if (providedCollation.satisfies(requiredCollation)) { + // the providedCollation can satisfy the requirement, + // so don't push down the sort into it's input. + } else if (providedCollation.getFieldCollations.isEmpty && + requiredCollation.getFieldCollations.nonEmpty) { + // If OverAgg cannot provide collation itself, try to push down collation requirements into + // it's input if collation fields all come from input node. + val canPushDownCollation = requiredCollation.getFieldCollations + .forall(_.getFieldIndex < inputFieldCnt) + if (canPushDownCollation) { + inputRequiredTraits = inputRequiredTraits.replace(requiredCollation) + providedTraits = providedTraits.replace(requiredCollation) + } + } else { + // Don't push down the sort into it's input, + // due to the provided collation will destroy the input's provided collation. + } + val newInput = RelOptRule.convert(getInput, inputRequiredTraits) + Some(copy(providedTraits, Seq(newInput))) + } + + private def inferProvidedTraitSet(): RelTraitSet = { + var selfProvidedTraitSet = getTraitSet + // provided distribution + val providedDistribution = if (grouping.nonEmpty) { + FlinkRelDistribution.hash(grouping.map(Integer.valueOf).toList, requireStrict = false) + } else { + FlinkRelDistribution.SINGLETON + } + selfProvidedTraitSet = selfProvidedTraitSet.replace(providedDistribution) + // provided collation + val firstGroup = windowGroupToAggCallToAggFunction.head._1 + if (OverAggregateUtil.needCollationTrait(logicWindow, firstGroup)) { + val collation = OverAggregateUtil.createCollation(firstGroup) + if (!collation.equals(RelCollations.EMPTY)) { + selfProvidedTraitSet = selfProvidedTraitSet.replace(collation) + } + } + selfProvidedTraitSet + } + //~ ExecNode methods ----------------------------------------------------------- override def getDamBehavior = DamBehavior.PIPELINED diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecRank.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecRank.scala index 1363a4707ee..ce030e70c7c 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecRank.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecRank.scala @@ -20,22 +20,25 @@ package org.apache.flink.table.plan.nodes.physical.batch import org.apache.flink.runtime.operators.DamBehavior import org.apache.flink.streaming.api.transformations.{OneInputTransformation, StreamTransformation} -import org.apache.flink.table.api.{BatchTableEnvironment, TableException} +import org.apache.flink.table.api.{BatchTableEnvironment, PlannerConfigOptions, TableException} import org.apache.flink.table.calcite.FlinkTypeFactory import org.apache.flink.table.codegen.sort.ComparatorCodeGenerator import org.apache.flink.table.dataformat.BaseRow +import org.apache.flink.table.plan.`trait`.{FlinkRelDistribution, FlinkRelDistributionTraitDef} import org.apache.flink.table.plan.cost.{FlinkCost, FlinkCostFactory} import org.apache.flink.table.plan.nodes.calcite.Rank import org.apache.flink.table.plan.nodes.exec.{BatchExecNode, ExecNode} -import org.apache.flink.table.plan.util.RelExplainUtil +import org.apache.flink.table.plan.util.{FlinkRelOptUtil, RelExplainUtil} import org.apache.flink.table.runtime.rank.{ConstantRankRange, RankRange, RankType} import org.apache.flink.table.runtime.sort.RankOperator import org.apache.calcite.plan._ +import org.apache.calcite.rel.RelDistribution.Type +import org.apache.calcite.rel.RelDistribution.Type.{HASH_DISTRIBUTED, SINGLETON} import org.apache.calcite.rel._ import org.apache.calcite.rel.`type`.RelDataTypeField import org.apache.calcite.rel.metadata.RelMetadataQuery -import org.apache.calcite.util.ImmutableBitSet +import org.apache.calcite.util.{ImmutableBitSet, ImmutableIntList, Util} import java.util @@ -112,6 +115,122 @@ class BatchExecRank( costFactory.makeCost(rowCount, cpuCost, 0, 0, memCost) } + override def satisfyTraits(requiredTraitSet: RelTraitSet): Option[RelNode] = { + if (isGlobal) { + satisfyTraitsOnGlobalRank(requiredTraitSet) + } else { + satisfyTraitsOnLocalRank(requiredTraitSet) + } + } + + private def satisfyTraitsOnGlobalRank(requiredTraitSet: RelTraitSet): Option[RelNode] = { + val requiredDistribution = requiredTraitSet.getTrait(FlinkRelDistributionTraitDef.INSTANCE) + val canSatisfy = requiredDistribution.getType match { + case SINGLETON => partitionKey.cardinality() == 0 + case HASH_DISTRIBUTED => + val shuffleKeys = requiredDistribution.getKeys + val partitionKeyList = ImmutableIntList.of(partitionKey.toArray: _*) + if (requiredDistribution.requireStrict) { + shuffleKeys == partitionKeyList + } else if (Util.startsWith(shuffleKeys, partitionKeyList)) { + // If required distribution is not strict, Hash[a] can satisfy Hash[a, b]. + // so return true if shuffleKeys(Hash[a, b]) start with partitionKeyList(Hash[a]) + true + } else { + // If partialKey is enabled, try to use partial key to satisfy the required distribution + val tableConfig = FlinkRelOptUtil.getTableConfigFromContext(this) + val partialKeyEnabled = tableConfig.getConf.getBoolean( + PlannerConfigOptions.SQL_OPTIMIZER_SHUFFLE_PARTIAL_KEY_ENABLED) + partialKeyEnabled && partitionKeyList.containsAll(shuffleKeys) + } + case _ => false + } + if (!canSatisfy) { + return None + } + + val inputRequiredDistribution = requiredDistribution.getType match { + case SINGLETON => requiredDistribution + case HASH_DISTRIBUTED => + val shuffleKeys = requiredDistribution.getKeys + val partitionKeyList = ImmutableIntList.of(partitionKey.toArray: _*) + if (requiredDistribution.requireStrict) { + FlinkRelDistribution.hash(partitionKeyList) + } else if (Util.startsWith(shuffleKeys, partitionKeyList)) { + // Hash[a] can satisfy Hash[a, b] + FlinkRelDistribution.hash(partitionKeyList, requireStrict = false) + } else { + // use partial key to satisfy the required distribution + FlinkRelDistribution.hash(shuffleKeys.map(partitionKeyList(_)), requireStrict = false) + } + } + + // sort by partition keys + orderby keys + val providedFieldCollations = partitionKey.toArray.map { + k => FlinkRelOptUtil.ofRelFieldCollation(k) + }.toList ++ orderKey.getFieldCollations + val providedCollation = RelCollations.of(providedFieldCollations) + val requiredCollation = requiredTraitSet.getTrait(RelCollationTraitDef.INSTANCE) + val newProvidedTraitSet = if (providedCollation.satisfies(requiredCollation)) { + getTraitSet.replace(requiredDistribution).replace(requiredCollation) + } else { + getTraitSet.replace(requiredDistribution) + } + val newInput = RelOptRule.convert(getInput, inputRequiredDistribution) + Some(copy(newProvidedTraitSet, Seq(newInput))) + } + + private def satisfyTraitsOnLocalRank(requiredTraitSet: RelTraitSet): Option[RelNode] = { + val requiredDistribution = requiredTraitSet.getTrait(FlinkRelDistributionTraitDef.INSTANCE) + requiredDistribution.getType match { + case Type.SINGLETON => + val inputRequiredDistribution = requiredDistribution + // sort by orderby keys + val providedCollation = orderKey + val requiredCollation = requiredTraitSet.getTrait(RelCollationTraitDef.INSTANCE) + val newProvidedTraitSet = if (providedCollation.satisfies(requiredCollation)) { + getTraitSet.replace(requiredDistribution).replace(requiredCollation) + } else { + getTraitSet.replace(requiredDistribution) + } + + val inputRequiredTraits = getInput.getTraitSet.replace(inputRequiredDistribution) + val newInput = RelOptRule.convert(getInput, inputRequiredTraits) + Some(copy(newProvidedTraitSet, Seq(newInput))) + case Type.HASH_DISTRIBUTED => + val shuffleKeys = requiredDistribution.getKeys + if (outputRankNumber) { + // rank function column is the last one + val rankColumnIndex = getRowType.getFieldCount - 1 + if (!shuffleKeys.contains(rankColumnIndex)) { + // Cannot satisfy required distribution if some keys are not from input + return None + } + } + + val inputRequiredDistributionKeys = shuffleKeys + val inputRequiredDistribution = FlinkRelDistribution.hash( + inputRequiredDistributionKeys, requiredDistribution.requireStrict) + + // sort by partition keys + orderby keys + val providedFieldCollations = partitionKey.toArray.map { + k => FlinkRelOptUtil.ofRelFieldCollation(k) + }.toList ++ orderKey.getFieldCollations + val providedCollation = RelCollations.of(providedFieldCollations) + val requiredCollation = requiredTraitSet.getTrait(RelCollationTraitDef.INSTANCE) + val newProvidedTraitSet = if (providedCollation.satisfies(requiredCollation)) { + getTraitSet.replace(requiredDistribution).replace(requiredCollation) + } else { + getTraitSet.replace(requiredDistribution) + } + + val inputRequiredTraits = getInput.getTraitSet.replace(inputRequiredDistribution) + val newInput = RelOptRule.convert(getInput, inputRequiredTraits) + Some(copy(newProvidedTraitSet, Seq(newInput))) + case _ => None + } + } + //~ ExecNode methods ----------------------------------------------------------- override def getDamBehavior: DamBehavior = DamBehavior.PIPELINED diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSortAggregate.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSortAggregate.scala index 208ad9ec1e5..1fded572bf7 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSortAggregate.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSortAggregate.scala @@ -19,19 +19,24 @@ package org.apache.flink.table.plan.nodes.physical.batch import org.apache.flink.runtime.operators.DamBehavior import org.apache.flink.streaming.api.transformations.StreamTransformation -import org.apache.flink.table.api.TableConfig +import org.apache.flink.table.api.{PlannerConfigOptions, TableConfig} import org.apache.flink.table.dataformat.BaseRow import org.apache.flink.table.functions.UserDefinedFunction -import org.apache.flink.table.plan.util.RelExplainUtil +import org.apache.flink.table.plan.`trait`.{FlinkRelDistribution, FlinkRelDistributionTraitDef} +import org.apache.flink.table.plan.util.{FlinkRelOptUtil, RelExplainUtil} -import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} +import org.apache.calcite.plan.{RelOptCluster, RelOptRule, RelTraitSet} +import org.apache.calcite.rel.RelDistribution.Type.{HASH_DISTRIBUTED, SINGLETON} import org.apache.calcite.rel._ import org.apache.calcite.rel.`type`.RelDataType import org.apache.calcite.rel.core.AggregateCall import org.apache.calcite.tools.RelBuilder +import org.apache.calcite.util.{ImmutableIntList, Util} import java.util +import scala.collection.JavaConversions._ + /** * Batch physical RelNode for (global) sort-based aggregate operator. * @@ -95,6 +100,64 @@ class BatchExecSortAggregate( isGlobal = true)) } + override def satisfyTraits(requiredTraitSet: RelTraitSet): Option[RelNode] = { + val requiredDistribution = requiredTraitSet.getTrait(FlinkRelDistributionTraitDef.INSTANCE) + val canSatisfy = requiredDistribution.getType match { + case SINGLETON => grouping.length == 0 + case HASH_DISTRIBUTED => + val shuffleKeys = requiredDistribution.getKeys + val groupKeysList = ImmutableIntList.of(grouping.indices.toArray: _*) + if (requiredDistribution.requireStrict) { + shuffleKeys == groupKeysList + } else if (Util.startsWith(shuffleKeys, groupKeysList)) { + // If required distribution is not strict, Hash[a] can satisfy Hash[a, b]. + // so return true if shuffleKeys(Hash[a, b]) start with groupKeys(Hash[a]) + true + } else { + // If partialKey is enabled, try to use partial key to satisfy the required distribution + val tableConfig = FlinkRelOptUtil.getTableConfigFromContext(this) + val partialKeyEnabled = tableConfig.getConf.getBoolean( + PlannerConfigOptions.SQL_OPTIMIZER_SHUFFLE_PARTIAL_KEY_ENABLED) + partialKeyEnabled && groupKeysList.containsAll(shuffleKeys) + } + case _ => false + } + if (!canSatisfy) { + return None + } + + val inputRequiredDistribution = requiredDistribution.getType match { + case SINGLETON => requiredDistribution + case HASH_DISTRIBUTED => + val shuffleKeys = requiredDistribution.getKeys + val groupKeysList = ImmutableIntList.of(grouping.indices.toArray: _*) + if (requiredDistribution.requireStrict) { + FlinkRelDistribution.hash(grouping, requireStrict = true) + } else if (Util.startsWith(shuffleKeys, groupKeysList)) { + // Hash [a] can satisfy Hash[a, b] + FlinkRelDistribution.hash(grouping, requireStrict = false) + } else { + // use partial key to satisfy the required distribution + FlinkRelDistribution.hash(shuffleKeys.map(grouping(_)).toArray, requireStrict = false) + } + } + + val providedCollation = if (grouping.length == 0) { + RelCollations.EMPTY + } else { + val providedFieldCollations = grouping.map(FlinkRelOptUtil.ofRelFieldCollation).toList + RelCollations.of(providedFieldCollations) + } + val requiredCollation = requiredTraitSet.getTrait(RelCollationTraitDef.INSTANCE) + val newProvidedTraitSet = if (providedCollation.satisfies(requiredCollation)) { + getTraitSet.replace(requiredDistribution).replace(requiredCollation) + } else { + getTraitSet.replace(requiredDistribution) + } + val newInput = RelOptRule.convert(getInput, inputRequiredDistribution) + Some(copy(newProvidedTraitSet, Seq(newInput))) + } + //~ ExecNode methods ----------------------------------------------------------- override def getDamBehavior: DamBehavior = { diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSortMergeJoin.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSortMergeJoin.scala index fd24c169156..42d248b0e50 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSortMergeJoin.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSortMergeJoin.scala @@ -26,16 +26,17 @@ import org.apache.flink.table.codegen.CodeGeneratorContext import org.apache.flink.table.codegen.ProjectionCodeGenerator.generateProjection import org.apache.flink.table.codegen.sort.SortCodeGenerator import org.apache.flink.table.dataformat.BaseRow +import org.apache.flink.table.plan.`trait`.FlinkRelDistributionTraitDef import org.apache.flink.table.plan.cost.{FlinkCost, FlinkCostFactory} import org.apache.flink.table.plan.nodes.ExpressionFormat import org.apache.flink.table.plan.nodes.exec.ExecNode -import org.apache.flink.table.plan.util.{FlinkRelMdUtil, JoinUtil, SortUtil} +import org.apache.flink.table.plan.util.{FlinkRelMdUtil, FlinkRelOptUtil, JoinUtil, SortUtil} import org.apache.flink.table.runtime.join.{FlinkJoinType, SortMergeJoinOperator} import org.apache.calcite.plan._ import org.apache.calcite.rel.core._ import org.apache.calcite.rel.metadata.RelMetadataQuery -import org.apache.calcite.rel.{RelNode, RelWriter} +import org.apache.calcite.rel.{RelCollationTraitDef, RelNode, RelWriter} import org.apache.calcite.rex.RexNode import java.util @@ -142,6 +143,55 @@ class BatchExecSortMergeJoin( costFactory.makeCost(rowCount, cpuCost, 0, 0, sortMemCost) } + override def satisfyTraits(requiredTraitSet: RelTraitSet): Option[RelNode] = { + val requiredDistribution = requiredTraitSet.getTrait(FlinkRelDistributionTraitDef.INSTANCE) + val (canSatisfyDistribution, leftRequiredDistribution, rightRequiredDistribution) = + satisfyHashDistributionOnNonBroadcastJoin(requiredDistribution) + if (!canSatisfyDistribution) { + return None + } + + val requiredCollation = requiredTraitSet.getTrait(RelCollationTraitDef.INSTANCE) + val requiredFieldCollations = requiredCollation.getFieldCollations + val shuffleKeysSize = leftRequiredDistribution.getKeys.size + + val newLeft = RelOptRule.convert(getLeft, leftRequiredDistribution) + val newRight = RelOptRule.convert(getRight, rightRequiredDistribution) + + // SortMergeJoin can provide collation trait, check whether provided collation can satisfy + // required collations + val canProvideCollation = if (requiredCollation.getFieldCollations.isEmpty) { + false + } else if (requiredFieldCollations.size > shuffleKeysSize) { + // Sort by [a, b] can satisfy [a], but cannot satisfy [a, b, c] + false + } else { + val leftKeys = leftRequiredDistribution.getKeys + val leftFieldCnt = getLeft.getRowType.getFieldCount + val rightKeys = rightRequiredDistribution.getKeys.map(_ + leftFieldCnt) + requiredFieldCollations.zipWithIndex.forall { case (collation, index) => + val idxOfCollation = collation.getFieldIndex + // Full outer join is handled before, so does not need care about it + if (idxOfCollation < leftFieldCnt && joinType != JoinRelType.RIGHT) { + val fieldCollationOnLeftSortKey = FlinkRelOptUtil.ofRelFieldCollation(leftKeys.get(index)) + collation == fieldCollationOnLeftSortKey + } else if (idxOfCollation >= leftFieldCnt && + (joinType == JoinRelType.RIGHT || joinType == JoinRelType.INNER)) { + val fieldCollationOnRightSortKey = + FlinkRelOptUtil.ofRelFieldCollation(rightKeys.get(index)) + collation == fieldCollationOnRightSortKey + } else { + false + } + } + } + var newProvidedTraitSet = getTraitSet.replace(requiredDistribution) + if (canProvideCollation) { + newProvidedTraitSet = newProvidedTraitSet.replace(requiredCollation) + } + Some(copy(newProvidedTraitSet, Seq(newLeft, newRight))) + } + //~ ExecNode methods ----------------------------------------------------------- /** diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecUnion.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecUnion.scala index 9919d0bb7a0..b0f6aa03fe4 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecUnion.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecUnion.scala @@ -21,9 +21,11 @@ import org.apache.flink.runtime.operators.DamBehavior import org.apache.flink.streaming.api.transformations.{StreamTransformation, UnionTransformation} import org.apache.flink.table.api.BatchTableEnvironment import org.apache.flink.table.dataformat.BaseRow +import org.apache.flink.table.plan.`trait`.{FlinkRelDistribution, FlinkRelDistributionTraitDef} import org.apache.flink.table.plan.nodes.exec.{BatchExecNode, ExecNode} -import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} +import org.apache.calcite.plan.{RelOptCluster, RelOptRule, RelTraitSet} +import org.apache.calcite.rel.RelDistribution.Type.{ANY, BROADCAST_DISTRIBUTED, HASH_DISTRIBUTED, RANDOM_DISTRIBUTED, RANGE_DISTRIBUTED, ROUND_ROBIN_DISTRIBUTED, SINGLETON} import org.apache.calcite.rel.`type`.RelDataType import org.apache.calcite.rel.core.{SetOp, Union} import org.apache.calcite.rel.{RelNode, RelWriter} @@ -58,6 +60,41 @@ class BatchExecUnion( .item("union", getRowType.getFieldNames.mkString(", ")) } + override def satisfyTraits(requiredTraitSet: RelTraitSet): Option[RelNode] = { + // union will destroy collation trait. So does not handle collation requirement. + val requiredDistribution = requiredTraitSet.getTrait(FlinkRelDistributionTraitDef.INSTANCE) + val canSatisfy = requiredDistribution.getType match { + case RANDOM_DISTRIBUTED | + ROUND_ROBIN_DISTRIBUTED | + BROADCAST_DISTRIBUTED | + HASH_DISTRIBUTED => true + // range distribution cannot be satisfied because partition's [lower, upper] of each union + // child may be different. + case RANGE_DISTRIBUTED => false + // singleton cannot cannot be satisfied because singleton exchange limits the parallelism of + // exchange output RelNode to 1. + // Push down Singleton into input of union will destroy the limitation. + case SINGLETON => false + // there is no need to satisfy Any distribution + case ANY => false + } + if (!canSatisfy) { + return None + } + + val inputRequiredDistribution = requiredDistribution.getType match { + case RANDOM_DISTRIBUTED | ROUND_ROBIN_DISTRIBUTED | BROADCAST_DISTRIBUTED => + requiredDistribution + case HASH_DISTRIBUTED => + // apply strict hash distribution of each child + // to avoid inconsistent of shuffle of each child + FlinkRelDistribution.hash(requiredDistribution.getKeys) + } + val newInputs = getInputs.map(RelOptRule.convert(_, inputRequiredDistribution)) + val providedTraitSet = getTraitSet.replace(inputRequiredDistribution) + Some(copy(providedTraitSet, newInputs)) + } + //~ ExecNode methods ----------------------------------------------------------- override def getDamBehavior: DamBehavior = DamBehavior.PIPELINED 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 2474dba8eca..d7e45b30794 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 @@ -192,7 +192,8 @@ object FlinkBatchRuleSets { val WINDOW_RULES: RuleSet = RuleSets.ofList( // slices a project into sections which contain window agg functions and sections which do not. ProjectToWindowRule.PROJECT, - // TODO add ExchangeWindowGroupRule + //adjust the sequence of window's groups. + WindowGroupReorderRule.INSTANCE, // Transform window to LogicalWindowAggregate WindowPropertiesRules.WINDOW_PROPERTIES_RULE, WindowPropertiesRules.WINDOW_PROPERTIES_HAVING_RULE @@ -311,28 +312,44 @@ object FlinkBatchRuleSets { */ val PHYSICAL_OPT_RULES: RuleSet = RuleSets.ofList( FlinkExpandConversionRule.BATCH_INSTANCE, + // source BatchExecBoundedStreamScanRule.INSTANCE, BatchExecScanTableSourceRule.INSTANCE, BatchExecIntermediateTableScanRule.INSTANCE, BatchExecValuesRule.INSTANCE, + // calc BatchExecCalcRule.INSTANCE, + // union BatchExecUnionRule.INSTANCE, + // sort BatchExecSortRule.INSTANCE, BatchExecLimitRule.INSTANCE, BatchExecSortLimitRule.INSTANCE, + // rank BatchExecRankRule.INSTANCE, + RemoveRedundantLocalRankRule.INSTANCE, + // expand BatchExecExpandRule.INSTANCE, + // group agg BatchExecHashAggRule.INSTANCE, BatchExecSortAggRule.INSTANCE, + RemoveRedundantLocalSortAggRule.WITHOUT_SORT, + RemoveRedundantLocalSortAggRule.WITH_SORT, + RemoveRedundantLocalHashAggRule.INSTANCE, + // over agg + BatchExecOverAggregateRule.INSTANCE, + // window agg + BatchExecWindowAggregateRule.INSTANCE, + // join BatchExecHashJoinRule.INSTANCE, BatchExecSortMergeJoinRule.INSTANCE, BatchExecNestedLoopJoinRule.INSTANCE, BatchExecSingleRowJoinRule.INSTANCE, - BatchExecCorrelateRule.INSTANCE, - BatchExecOverAggregateRule.INSTANCE, - BatchExecWindowAggregateRule.INSTANCE, BatchExecLookupJoinRule.SNAPSHOT_ON_TABLESCAN, BatchExecLookupJoinRule.SNAPSHOT_ON_CALC_TABLESCAN, + // correlate + BatchExecCorrelateRule.INSTANCE, + // sink BatchExecSinkRule.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 c030929f069..ce99a988d42 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 @@ -293,27 +293,39 @@ object FlinkStreamRuleSets { */ val PHYSICAL_OPT_RULES: RuleSet = RuleSets.ofList( FlinkExpandConversionRule.STREAM_INSTANCE, + // source StreamExecDataStreamScanRule.INSTANCE, StreamExecTableSourceScanRule.INSTANCE, StreamExecIntermediateTableScanRule.INSTANCE, StreamExecValuesRule.INSTANCE, + // calc StreamExecCalcRule.INSTANCE, + // union StreamExecUnionRule.INSTANCE, + // sort StreamExecSortRule.INSTANCE, StreamExecLimitRule.INSTANCE, StreamExecSortLimitRule.INSTANCE, - StreamExecRankRule.INSTANCE, StreamExecTemporalSortRule.INSTANCE, + // rank + StreamExecRankRule.INSTANCE, StreamExecDeduplicateRule.RANK_INSTANCE, + // expand + StreamExecExpandRule.INSTANCE, + // group agg StreamExecGroupAggregateRule.INSTANCE, + // over agg StreamExecOverAggregateRule.INSTANCE, + // window agg StreamExecGroupWindowAggregateRule.INSTANCE, - StreamExecExpandRule.INSTANCE, + // join StreamExecJoinRule.INSTANCE, StreamExecWindowJoinRule.INSTANCE, - StreamExecCorrelateRule.INSTANCE, StreamExecLookupJoinRule.SNAPSHOT_ON_TABLESCAN, StreamExecLookupJoinRule.SNAPSHOT_ON_CALC_TABLESCAN, + // correlate + StreamExecCorrelateRule.INSTANCE, + // sink StreamExecSinkRule.INSTANCE ) diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/WindowGroupReorderRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/WindowGroupReorderRule.scala new file mode 100644 index 00000000000..c263371e27c --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/WindowGroupReorderRule.scala @@ -0,0 +1,137 @@ +/* + * 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.logical + +import org.apache.calcite.plan.RelOptRule.{any, operand} +import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall} +import org.apache.calcite.rel.`type`.RelDataType +import org.apache.calcite.rel.core.Window.Group +import org.apache.calcite.rel.logical.{LogicalProject, LogicalWindow} +import org.apache.calcite.rel.{RelCollation, RelNode} +import org.apache.calcite.rex.RexInputRef + +import java.util +import java.util.Comparator + +import scala.collection.JavaConversions._ + +/** + * Planner rule that makes the over window groups which have the same shuffle keys and order keys + * together. + */ +class WindowGroupReorderRule extends RelOptRule( + operand(classOf[LogicalWindow], + operand(classOf[RelNode], any)), + "ExchangeWindowGroupRule") { + + override def matches(call: RelOptRuleCall): Boolean = { + val window: LogicalWindow = call.rel(0) + window.groups.size() > 1 + } + + override def onMatch(call: RelOptRuleCall): Unit = { + val window: LogicalWindow = call.rel(0) + val input: RelNode = call.rel(1) + val oldGroups: util.List[Group] = new util.ArrayList(window.groups) + val sequenceGroups: util.List[Group] = new util.ArrayList(window.groups) + + sequenceGroups.sort(new Comparator[Group] { + override def compare(o1: Group, o2: Group): Int = { + val keyComp = o1.keys.compareTo(o2.keys) + if (keyComp == 0) { + compareRelCollation(o1.orderKeys, o2.orderKeys) + } else { + keyComp + } + } + }) + + if (!sequenceGroups.equals(oldGroups) && !sequenceGroups.reverse.equals(oldGroups)) { + var offset = input.getRowType.getFieldCount + val aggTypeIndexes = oldGroups.map { group => + val aggCount = group.aggCalls.size() + val typeIndexes = (0 until aggCount).map(_ + offset).toArray + offset += aggCount + typeIndexes + } + + offset = input.getRowType.getFieldCount + val mapToOldTypeIndexes = (0 until offset).toArray ++ + sequenceGroups.flatMap { newGroup => + val aggCount = newGroup.aggCalls.size() + val oldIndex = oldGroups.indexOf(newGroup) + offset += aggCount + (0 until aggCount).map { + aggIndex => aggTypeIndexes(oldIndex)(aggIndex) + } + }.toArray[Int] + + val oldRowTypeFields = window.getRowType.getFieldList + val newFieldList = new util.ArrayList[util.Map.Entry[String, RelDataType]] + mapToOldTypeIndexes.foreach { index => + newFieldList.add(oldRowTypeFields.get(index)) + } + val intermediateRowType = window.getCluster.getTypeFactory.createStructType(newFieldList) + val newLogicalWindow = LogicalWindow.create( + window.getCluster.getPlanner.emptyTraitSet(), + input, + window.constants, + intermediateRowType, + sequenceGroups) + + val mapToNewTypeIndexes = mapToOldTypeIndexes.zipWithIndex.sortBy(_._1) + + val projects = mapToNewTypeIndexes.map { index => + new RexInputRef(index._2, newFieldList.get(index._2).getValue) + } + val project = LogicalProject.create( + newLogicalWindow, + projects.toList, + window.getRowType) + call.transformTo(project) + } + } + + private def compareRelCollation(o1: RelCollation, o2: RelCollation): Int = { + val comp = o1.compareTo(o2) + if (comp == 0) { + val collations1 = o1.getFieldCollations + val collations2 = o2.getFieldCollations + for (index <- 0 until collations1.length) { + val collation1 = collations1(index) + val collation2 = collations2(index) + val direction = collation1.direction.shortString.compareTo(collation2.direction.shortString) + if (direction == 0) { + val nullDirection = collation1.nullDirection.nullComparison.compare( + collation2.nullDirection.nullComparison) + if (nullDirection != 0) { + return nullDirection + } + } else { + return direction + } + } + } + comp + } +} + +object WindowGroupReorderRule { + val INSTANCE = new WindowGroupReorderRule +} diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/FlinkExpandConversionRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/FlinkExpandConversionRule.scala index 9b7d61a89f1..97f3332ecd0 100644 --- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/FlinkExpandConversionRule.scala +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/FlinkExpandConversionRule.scala @@ -81,14 +81,16 @@ class FlinkExpandConversionRule(flinkConvention: Convention) call: RelOptRuleCall): Unit = { node match { case batchRel: BatchPhysicalRel => - var otherChoice = batchRel.satisfyTraitsByInput(requiredTraits) - if (otherChoice != null) { - // It is possible only push down distribution instead of push down both distribution and - // collation. So it is necessary to check whether collation satisfy requirement. - val requiredCollation = requiredTraits.getTrait(RelCollationTraitDef.INSTANCE) - otherChoice = satisfyCollation(flinkConvention, otherChoice, requiredCollation) - checkSatisfyRequiredTrait(otherChoice, requiredTraits) - call.transformTo(otherChoice) + val otherChoice = batchRel.satisfyTraits(requiredTraits) + otherChoice match { + case Some(newRel) => + // It is possible only push down distribution instead of push down both distribution and + // collation. So it is necessary to check whether collation satisfy requirement. + val requiredCollation = requiredTraits.getTrait(RelCollationTraitDef.INSTANCE) + val finalRel = satisfyCollation(flinkConvention, newRel, requiredCollation) + checkSatisfyRequiredTrait(finalRel, requiredTraits) + call.transformTo(finalRel) + case _ => // do nothing } case _ => // ignore } diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/RemoveRedundantLocalHashAggRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/RemoveRedundantLocalHashAggRule.scala new file mode 100644 index 00000000000..379e5b4a302 --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/RemoveRedundantLocalHashAggRule.scala @@ -0,0 +1,60 @@ +/* + * 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.physical.batch.{BatchExecHashAggregate, BatchExecLocalHashAggregate} + +import org.apache.calcite.plan.RelOptRule._ +import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall} +import org.apache.calcite.rel.RelNode + +/** + * There maybe exist a subTree like localHashAggregate -> globalHashAggregate which the middle + * shuffle is removed. The rule could remove redundant localHashAggregate node. + */ +class RemoveRedundantLocalHashAggRule extends RelOptRule( + operand(classOf[BatchExecHashAggregate], + operand(classOf[BatchExecLocalHashAggregate], + operand(classOf[RelNode], FlinkConventions.BATCH_PHYSICAL, any))), + "RemoveRedundantLocalHashAggRule") { + + override def onMatch(call: RelOptRuleCall): Unit = { + val globalAgg = call.rels(0).asInstanceOf[BatchExecHashAggregate] + val localAgg = call.rels(1).asInstanceOf[BatchExecLocalHashAggregate] + val inputOfLocalAgg = localAgg.getInput + val newGlobalAgg = new BatchExecHashAggregate( + globalAgg.getCluster, + call.builder(), + globalAgg.getTraitSet, + inputOfLocalAgg, + globalAgg.getRowType, + inputOfLocalAgg.getRowType, + inputOfLocalAgg.getRowType, + localAgg.getGrouping, + localAgg.getAuxGrouping, + globalAgg.getAggCallToAggFunction, + isMerge = false) + call.transformTo(newGlobalAgg) + } +} + +object RemoveRedundantLocalHashAggRule { + val INSTANCE = new RemoveRedundantLocalHashAggRule +} diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/RemoveRedundantLocalRankRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/RemoveRedundantLocalRankRule.scala new file mode 100644 index 00000000000..304710db9e7 --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/RemoveRedundantLocalRankRule.scala @@ -0,0 +1,60 @@ +/* + * 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.physical.batch.BatchExecRank + +import org.apache.calcite.plan.RelOptRule._ +import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall} +import org.apache.calcite.rel.RelNode + +import scala.collection.JavaConversions._ + +/** + * Planner rule that matches a global [[BatchExecRank]] on a local [[BatchExecRank]], + * and merge them into a global [[BatchExecRank]]. + */ +class RemoveRedundantLocalRankRule extends RelOptRule( + operand(classOf[BatchExecRank], + operand(classOf[BatchExecRank], + operand(classOf[RelNode], FlinkConventions.BATCH_PHYSICAL, any))), + "RemoveRedundantLocalRankRule") { + + override def matches(call: RelOptRuleCall): Boolean = { + val globalRank: BatchExecRank = call.rel(0) + val localRank: BatchExecRank = call.rel(1) + globalRank.isGlobal && !localRank.isGlobal && + globalRank.rankType == localRank.rankType && + globalRank.partitionKey == localRank.partitionKey && + globalRank.orderKey == globalRank.orderKey && + globalRank.rankEnd == localRank.rankEnd + } + + override def onMatch(call: RelOptRuleCall): Unit = { + val globalRank: BatchExecRank = call.rel(0) + val inputOfLocalRank: RelNode = call.rel(2) + val newGlobalRank = globalRank.copy(globalRank.getTraitSet, List(inputOfLocalRank)) + call.transformTo(newGlobalRank) + } +} + +object RemoveRedundantLocalRankRule { + val INSTANCE: RelOptRule = new RemoveRedundantLocalRankRule +} diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/RemoveRedundantLocalSortAggRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/RemoveRedundantLocalSortAggRule.scala new file mode 100644 index 00000000000..4f719ebd889 --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/RemoveRedundantLocalSortAggRule.scala @@ -0,0 +1,110 @@ +/* + * 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.physical.batch.{BatchExecLocalSortAggregate, BatchExecSort, BatchExecSortAggregate} + +import org.apache.calcite.plan.RelOptRule._ +import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall, RelOptRuleOperand} +import org.apache.calcite.rel.RelNode + +/** + * There maybe exist a subTree like localSortAggregate -> globalSortAggregate, or + * localSortAggregate -> sort -> globalSortAggregate which the middle shuffle is removed. + * The rule could remove redundant localSortAggregate node. + */ +abstract class RemoveRedundantLocalSortAggRule( + operand: RelOptRuleOperand, + ruleName: String) extends RelOptRule(operand, ruleName) { + + override def onMatch(call: RelOptRuleCall): Unit = { + val globalAgg = getOriginalGlobalAgg(call) + val localAgg = getOriginalLocalAgg(call) + val inputOfLocalAgg = getOriginalInputOfLocalAgg(call) + val newGlobalAgg = new BatchExecSortAggregate( + globalAgg.getCluster, + call.builder(), + globalAgg.getTraitSet, + inputOfLocalAgg, + globalAgg.getRowType, + inputOfLocalAgg.getRowType, + inputOfLocalAgg.getRowType, + localAgg.getGrouping, + localAgg.getAuxGrouping, + globalAgg.getAggCallToAggFunction, + isMerge = false) + call.transformTo(newGlobalAgg) + } + + private[table] def getOriginalGlobalAgg(call: RelOptRuleCall): BatchExecSortAggregate + + private[table] def getOriginalLocalAgg(call: RelOptRuleCall): BatchExecLocalSortAggregate + + private[table] def getOriginalInputOfLocalAgg(call: RelOptRuleCall): RelNode + +} + +class RemoveRedundantLocalSortAggWithoutSortRule extends RemoveRedundantLocalSortAggRule( + operand(classOf[BatchExecSortAggregate], + operand(classOf[BatchExecLocalSortAggregate], + operand(classOf[RelNode], FlinkConventions.BATCH_PHYSICAL, any))), + "RemoveRedundantLocalSortAggWithoutSortRule") { + + override private[table] def getOriginalGlobalAgg(call: RelOptRuleCall): BatchExecSortAggregate = { + call.rels(0).asInstanceOf[BatchExecSortAggregate] + } + + override private[table] def getOriginalLocalAgg( + call: RelOptRuleCall): BatchExecLocalSortAggregate = { + call.rels(1).asInstanceOf[BatchExecLocalSortAggregate] + } + + override private[table] def getOriginalInputOfLocalAgg(call: RelOptRuleCall): RelNode = { + call.rels(2) + } + +} + +class RemoveRedundantLocalSortAggWithSortRule extends RemoveRedundantLocalSortAggRule( + operand(classOf[BatchExecSortAggregate], + operand(classOf[BatchExecSort], + operand(classOf[BatchExecLocalSortAggregate], + operand(classOf[RelNode], FlinkConventions.BATCH_PHYSICAL, any)))), + "RemoveRedundantLocalSortAggWithSortRule") { + + override private[table] def getOriginalGlobalAgg(call: RelOptRuleCall): BatchExecSortAggregate = { + call.rels(0).asInstanceOf[BatchExecSortAggregate] + } + + override private[table] def getOriginalLocalAgg( + call: RelOptRuleCall): BatchExecLocalSortAggregate = { + call.rels(2).asInstanceOf[BatchExecLocalSortAggregate] + } + + override private[table] def getOriginalInputOfLocalAgg(call: RelOptRuleCall): RelNode = { + call.rels(3) + } + +} + +object RemoveRedundantLocalSortAggRule { + val WITHOUT_SORT = new RemoveRedundantLocalSortAggWithoutSortRule + val WITH_SORT = new RemoveRedundantLocalSortAggWithSortRule +} 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 index a58970bfd39..911bbd2f41c 100644 --- 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 @@ -726,15 +726,14 @@ Sink(fields=[a1, b, c1]) :- 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]) + +- 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]) ]]> @@ -784,21 +783,19 @@ Sink(fields=[a, b, c]) :- 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]) + +- 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]) + +- 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]) ]]> diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/DeadlockBreakupTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/DeadlockBreakupTest.xml index f035f65cbbc..69ec9d6436a 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/DeadlockBreakupTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/DeadlockBreakupTest.xml @@ -89,22 +89,21 @@ LogicalProject(a=[$0], b=[$1], c=[$2], a0=[$3], b0=[$4], c0=[$5], a1=[$6], b1=[$ HashJoin(joinType=[InnerJoin], where=[=(c, c1)], select=[a, b, c, a0, b0, c0, a1, b1, c1, a00, b00, c00], build=[right]) :- Exchange(distribution=[hash[c]], exchange_mode=[BATCH]) : +- HashJoin(joinType=[InnerJoin], where=[=(a, a0)], select=[a, b, c, a0, b0, c0], build=[left]) -: :- Exchange(distribution=[hash[a]]) -: : +- Calc(select=[a, b, c], where=[>(b, 10)]) -: : +- SortAggregate(isMerge=[true], groupBy=[a], select=[a, Final_SUM(sum$0) AS b, Final_MAX(max$1) AS c], reuse_id=[1]) -: : +- Sort(orderBy=[a ASC]) -: : +- Exchange(distribution=[hash[a]]) -: : +- LocalSortAggregate(groupBy=[a], select=[a, Partial_SUM(b) AS sum$0, Partial_MAX(c) AS max$1]) -: : +- Sort(orderBy=[a ASC]) -: : +- TableSourceScan(table=[[x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) -: +- Exchange(distribution=[hash[a]], exchange_mode=[BATCH], reuse_id=[2]) +: :- Calc(select=[a, b, c], where=[>(b, 10)]) +: : +- SortAggregate(isMerge=[true], groupBy=[a], select=[a, Final_SUM(sum$0) AS b, Final_MAX(max$1) AS c], reuse_id=[1]) +: : +- Sort(orderBy=[a ASC]) +: : +- Exchange(distribution=[hash[a]]) +: : +- LocalSortAggregate(groupBy=[a], select=[a, Partial_SUM(b) AS sum$0, Partial_MAX(c) AS max$1]) +: : +- Sort(orderBy=[a ASC]) +: : +- TableSourceScan(table=[[x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- Exchange(distribution=[hash[a]], exchange_mode=[BATCH]) : +- Reused(reference_id=[1]) +- Exchange(distribution=[hash[c]]) +- HashJoin(joinType=[InnerJoin], where=[=(a, a0)], select=[a, b, c, a0, b0, c0], build=[left]) - :- Exchange(distribution=[hash[a]]) - : +- Calc(select=[a, b, c], where=[<(b, 5)]) - : +- Reused(reference_id=[1]) - +- Reused(reference_id=[2]) + :- Calc(select=[a, b, c], where=[<(b, 5)]) + : +- Reused(reference_id=[1]) + +- Exchange(distribution=[hash[a]], exchange_mode=[BATCH]) + +- Reused(reference_id=[1]) ]]> @@ -141,17 +140,16 @@ Calc(select=[a, b]) :- Exchange(distribution=[hash[a]], exchange_mode=[BATCH]) : +- Calc(select=[a], where=[=(b, 5:BIGINT)]) : +- TableSourceScan(table=[[x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c], reuse_id=[1]) - +- Exchange(distribution=[hash[a]]) - +- HashJoin(joinType=[LeftSemiJoin], where=[=(a, a0)], select=[a, b], build=[left]) - :- Exchange(distribution=[hash[a]]) - : +- Calc(select=[a, b]) - : +- Limit(offset=[0], fetch=[10], global=[true]) - : +- Exchange(distribution=[single]) - : +- Limit(offset=[0], fetch=[10], global=[false]) - : +- Reused(reference_id=[1]) - +- Exchange(distribution=[hash[a]], exchange_mode=[BATCH]) - +- Calc(select=[a], where=[>(b, 5)]) - +- Reused(reference_id=[1]) + +- HashJoin(joinType=[LeftSemiJoin], where=[=(a, a0)], select=[a, b], build=[left]) + :- Exchange(distribution=[hash[a]]) + : +- Calc(select=[a, b]) + : +- Limit(offset=[0], fetch=[10], global=[true]) + : +- Exchange(distribution=[single]) + : +- Limit(offset=[0], fetch=[10], global=[false]) + : +- Reused(reference_id=[1]) + +- Exchange(distribution=[hash[a]], exchange_mode=[BATCH]) + +- Calc(select=[a], where=[>(b, 5)]) + +- Reused(reference_id=[1]) ]]> @@ -298,13 +296,11 @@ HashJoin(joinType=[InnerJoin], where=[=(c, c0)], select=[a, b, c, a0, b0, c0], b :- Exchange(distribution=[hash[c]], exchange_mode=[BATCH]) : +- Calc(select=[w0$o0 AS a, b, c]) : +- OverAggregate(partitionBy=[b], orderBy=[b ASC], window#0=[MAX($2) AS w0$o0 RANG BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING], select=[b, c, $2, w0$o0]) -: +- Sort(orderBy=[b ASC], reuse_id=[1]) -: +- Exchange(distribution=[hash[b]]) -: +- Calc(select=[b, c, CASE(>(w0$o0, 0:BIGINT), w0$o1, null:INTEGER) AS $2]) -: +- OverAggregate(partitionBy=[b], orderBy=[b ASC], window#0=[COUNT(a) AS w0$o0, $SUM0(a) AS w0$o1 RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[a, b, c, w0$o0, w0$o1]) -: +- Sort(orderBy=[b ASC]) -: +- Exchange(distribution=[hash[b]]) -: +- TableSourceScan(table=[[x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: +- Calc(select=[b, c, CASE(>(w0$o0, 0:BIGINT), w0$o1, null:INTEGER) AS $2], reuse_id=[1]) +: +- OverAggregate(partitionBy=[b], orderBy=[b ASC], window#0=[COUNT(a) AS w0$o0, $SUM0(a) AS w0$o1 RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[a, b, c, w0$o0, w0$o1]) +: +- Sort(orderBy=[b ASC]) +: +- Exchange(distribution=[hash[b]]) +: +- TableSourceScan(table=[[x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +- Exchange(distribution=[hash[c]]) +- Calc(select=[w0$o0 AS a, b, c]) +- OverAggregate(partitionBy=[b], orderBy=[b ASC], window#0=[MIN($2) AS w0$o0 RANG BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING], select=[b, c, $2, w0$o0]) diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/RemoveCollationTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/RemoveCollationTest.xml new file mode 100644 index 00000000000..283177ea21f --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/RemoveCollationTest.xml @@ -0,0 +1,708 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + (COUNT($1) OVER (PARTITION BY $0 ORDER BY $0 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), 0), $SUM0($1) OVER (PARTITION BY $0 ORDER BY $0 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), null:BIGINT)):DOUBLE, COUNT($1) OVER (PARTITION BY $0 ORDER BY $0 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW))], rn=[RANK() OVER (PARTITION BY $0 ORDER BY $0 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)]) ++- LogicalAggregate(group=[{0}], sum_b=[SUM($1)]) + +- LogicalProject(a=[$0], b=[$1]) + +- LogicalTableScan(table=[[x, source: [TestTableSource(a, b, c)]]]) +]]> + + + (w0$o0, 0:BIGINT), w0$o1, null:BIGINT)), w0$o0) AS avg_b, w0$o2 AS rn]) ++- OverAggregate(partitionBy=[a], orderBy=[a ASC], window#0=[COUNT(sum_b) AS w0$o0, $SUM0(sum_b) AS w0$o1, RANK(*) AS w0$o2 RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], window#1=[ RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[a, sum_b, w0$o0, w0$o1, w0$o2]) + +- SortAggregate(isMerge=[true], groupBy=[a], select=[a, Final_SUM(sum$0) AS sum_b]) + +- Sort(orderBy=[a ASC]) + +- Exchange(distribution=[hash[a]]) + +- LocalSortAggregate(groupBy=[a], select=[a, Partial_SUM(b) AS sum$0]) + +- Sort(orderBy=[a ASC]) + +- Calc(select=[a, b]) + +- TableSourceScan(table=[[x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/RemoveShuffleTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/RemoveShuffleTest.xml new file mode 100644 index 00000000000..e23d8a044f1 --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/RemoveShuffleTest.xml @@ -0,0 +1,1264 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + (COUNT($2) OVER (PARTITION BY $0, $1 RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING), 0), $SUM0($2) OVER (PARTITION BY $0, $1 RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING), null:BIGINT)):DOUBLE, COUNT($2) OVER (PARTITION BY $0, $1 RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING))], rn=[RANK() OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST, $1 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)], c=[$1]) ++- LogicalAggregate(group=[{0, 1}], sum_b=[SUM($2)]) + +- LogicalProject(a=[$0], c=[$2], b=[$1]) + +- LogicalTableScan(table=[[x, source: [TestTableSource(a, b, c)]]]) +]]> + + + (w0$o0, 0:BIGINT), w0$o1, null:BIGINT)), w0$o0) AS avg_b, w1$o0 AS rn, c]) ++- OverAggregate(partitionBy=[c], orderBy=[a ASC, c ASC], window#0=[RANK(*) AS w1$o0 RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[a, c, sum_b, w0$o0, w0$o1, w1$o0]) + +- Sort(orderBy=[c ASC, a ASC]) + +- Exchange(distribution=[hash[c]]) + +- OverAggregate(partitionBy=[a, c], window#0=[COUNT(sum_b) AS w0$o0, $SUM0(sum_b) AS w0$o1 RANG BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING], select=[a, c, sum_b, w0$o0, w0$o1]) + +- Sort(orderBy=[a ASC, c ASC]) + +- HashAggregate(isMerge=[true], groupBy=[a, c], select=[a, c, Final_SUM(sum$0) AS sum_b]) + +- Exchange(distribution=[hash[a, c]]) + +- LocalHashAggregate(groupBy=[a, c], select=[a, c, Partial_SUM(b) AS sum$0]) + +- TableSourceScan(table=[[x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +]]> + + + + + + + + + + + + + + + + + + + (COUNT($1) OVER (PARTITION BY $0 RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING), 0), $SUM0($1) OVER (PARTITION BY $0 RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING), null:BIGINT)):DOUBLE, COUNT($1) OVER (PARTITION BY $0 RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING))], rn=[RANK() OVER (PARTITION BY $0 ORDER BY $0 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)], c=[$0]) ++- LogicalAggregate(group=[{0}], sum_b=[SUM($1)]) + +- LogicalProject(c=[$2], b=[$1]) + +- LogicalTableScan(table=[[x, source: [TestTableSource(a, b, c)]]]) +]]> + + + (w0$o0, 0:BIGINT), w0$o1, null:BIGINT)), w0$o0) AS avg_b, w1$o0 AS rn, c]) ++- OverAggregate(partitionBy=[c], orderBy=[], window#0=[COUNT(sum_b) AS w0$o0, $SUM0(sum_b) AS w0$o1 RANG BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING], window#1=[RANK(*) AS w1$o0 RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[c, sum_b, w0$o0, w0$o1, w1$o0]) + +- Sort(orderBy=[c ASC]) + +- HashAggregate(isMerge=[true], groupBy=[c], select=[c, Final_SUM(sum$0) AS sum_b]) + +- Exchange(distribution=[hash[c]]) + +- LocalHashAggregate(groupBy=[c], select=[c, Partial_SUM(b) AS sum$0]) + +- Calc(select=[c, b]) + +- TableSourceScan(table=[[x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +]]> + + + + + + + + (COUNT($2) OVER (PARTITION BY $1 RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING), 0), $SUM0($2) OVER (PARTITION BY $1 RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING), null:BIGINT)):DOUBLE, COUNT($2) OVER (PARTITION BY $1 RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING))], rn=[RANK() OVER (PARTITION BY $1 ORDER BY $1 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)], c=[$1]) ++- LogicalAggregate(group=[{0, 1}], sum_b=[SUM($2)]) + +- LogicalProject(a=[$0], c=[$2], b=[$1]) + +- LogicalTableScan(table=[[x, source: [TestTableSource(a, b, c)]]]) +]]> + + + (w0$o0, 0:BIGINT), w0$o1, null:BIGINT)), w0$o0) AS avg_b, w1$o0 AS rn, c]) ++- OverAggregate(partitionBy=[c], orderBy=[], window#0=[COUNT(sum_b) AS w0$o0, $SUM0(sum_b) AS w0$o1 RANG BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING], window#1=[RANK(*) AS w1$o0 RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[c, sum_b, w0$o0, w0$o1, w1$o0]) + +- Calc(select=[c, sum_b]) + +- Sort(orderBy=[c ASC]) + +- HashAggregate(isMerge=[true], groupBy=[a, c], select=[a, c, Final_SUM(sum$0) AS sum_b]) + +- Exchange(distribution=[hash[c]]) + +- LocalHashAggregate(groupBy=[a, c], select=[a, c, Partial_SUM(b) AS sum$0]) + +- TableSourceScan(table=[[x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + (SELECT sum(b) * 0.1 FROM x)]]> + + + ($2, $SCALAR_QUERY({ +LogicalProject(EXPR$0=[*($0, 0.1:DECIMAL(2, 1))]) + LogicalAggregate(group=[{}], agg#0=[SUM($0)]) + LogicalProject(b=[$1]) + LogicalTableScan(table=[[x, source: [TestTableSource(a, b, c)]]]) +}))]) + +- LogicalAggregate(group=[{0}], EXPR$0=[AVG($1)], agg#1=[SUM($1)]) + +- LogicalProject(c=[$2], b=[$1]) + +- LogicalTableScan(table=[[x, source: [TestTableSource(a, b, c)]]]) +]]> + + + ($f2, $f0)], select=[EXPR$0, $f2, $f0], build=[right], singleRowJoin=[true]) + :- Calc(select=[EXPR$0, $f2]) + : +- HashAggregate(isMerge=[true], groupBy=[c], select=[c, Final_AVG(sum$0, count$1) AS EXPR$0, Final_SUM(sum$2) AS $f2]) + : +- Exchange(distribution=[hash[c]]) + : +- LocalHashAggregate(groupBy=[c], select=[c, Partial_AVG(b) AS (sum$0, count$1), Partial_SUM(b) AS sum$2]) + : +- Calc(select=[c, b]) + : +- TableSourceScan(table=[[x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- Exchange(distribution=[broadcast]) + +- SortAggregate(isMerge=[false], select=[SINGLE_VALUE(EXPR$0) AS $f0]) + +- Calc(select=[*($f0, 0.1:DECIMAL(2, 1)) AS EXPR$0]) + +- SortAggregate(isMerge=[true], select=[Final_SUM(sum$0) AS $f0]) + +- Exchange(distribution=[single]) + +- LocalSortAggregate(select=[Partial_SUM(b) AS sum$0]) + +- Calc(select=[b]) + +- TableSourceScan(table=[[x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +]]> + + + + + 10 group by c +UNION ALL +SELECT count(d) as cnt, f FROM y WHERE e < 100 group by f) +SELECT r1.c, r1.cnt, r2.c, r2.cnt FROM r r1, r r2 WHERE r1.c = r2.c and r1.cnt < 10 + ]]> + + + ($1, 10)]) + : : +- LogicalTableScan(table=[[x, source: [TestTableSource(a, b, c)]]]) + : +- LogicalProject(cnt=[$1], f=[$0]) + : +- LogicalAggregate(group=[{0}], cnt=[COUNT($1)]) + : +- LogicalProject(f=[$2], d=[$0]) + : +- LogicalFilter(condition=[<($1, 100)]) + : +- LogicalTableScan(table=[[y, source: [TestTableSource(d, e, f)]]]) + +- LogicalUnion(all=[true]) + :- LogicalProject(cnt=[$1], c=[$0]) + : +- LogicalAggregate(group=[{0}], cnt=[COUNT($1)]) + : +- LogicalProject(c=[$2], a=[$0]) + : +- LogicalFilter(condition=[>($1, 10)]) + : +- LogicalTableScan(table=[[x, source: [TestTableSource(a, b, c)]]]) + +- LogicalProject(cnt=[$1], f=[$0]) + +- LogicalAggregate(group=[{0}], cnt=[COUNT($1)]) + +- LogicalProject(f=[$2], d=[$0]) + +- LogicalFilter(condition=[<($1, 100)]) + +- LogicalTableScan(table=[[y, source: [TestTableSource(d, e, f)]]]) +]]> + + + (b, 10)]) + : : +- TableSourceScan(table=[[x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : +- Calc(select=[cnt, f], where=[<(cnt, 10)]) + : +- HashAggregate(isMerge=[true], groupBy=[f], select=[f, Final_COUNT(count$0) AS cnt]) + : +- Exchange(distribution=[hash[f]]) + : +- LocalHashAggregate(groupBy=[f], select=[f, Partial_COUNT(d) AS count$0]) + : +- Calc(select=[f, d], where=[<(e, 100)]) + : +- TableSourceScan(table=[[y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- Union(all=[true], union=[cnt, c]) + :- Calc(select=[cnt, c]) + : +- 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], where=[>(b, 10)]) + : +- TableSourceScan(table=[[x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- Calc(select=[cnt, f]) + +- HashAggregate(isMerge=[true], groupBy=[f], select=[f, Final_COUNT(count$0) AS cnt]) + +- Exchange(distribution=[hash[f]]) + +- LocalHashAggregate(groupBy=[f], select=[f, Partial_COUNT(d) AS count$0]) + +- Calc(select=[f, d], where=[<(e, 100)]) + +- TableSourceScan(table=[[y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) +]]> + + + diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/SubplanReuseTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/SubplanReuseTest.xml index 302bc021ffe..a1cc277f0b2 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/SubplanReuseTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/SubplanReuseTest.xml @@ -240,28 +240,26 @@ LogicalProject(c=[$0], e=[$1], avg_b=[$2], sum_b=[$3], psum=[$4], nsum=[$5], avg (w0$o0, 0:BIGINT), w0$o1, null:BIGINT)), w0$o0) AS avg_b, w1$o0 AS rn, c, e], where=[AND(<>(c, _UTF-16LE'':VARCHAR(65536) CHARACTER SET "UTF-16LE"), >(-(sum_b, /(CAST(CASE(>(w0$o0, 0:BIGINT), w0$o1, null:BIGINT)), w0$o0)), 3))]) - : : +- OverAggregate(partitionBy=[c, e], orderBy=[], window#0=[COUNT(sum_b) AS w0$o0, $SUM0(sum_b) AS w0$o1 RANG BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING], window#1=[RANK(*) AS w1$o0 RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[c, e, sum_b, w0$o0, w0$o1, w1$o0], reuse_id=[1]) - : : +- Sort(orderBy=[c ASC, e ASC], reuse_id=[2]) - : : +- Exchange(distribution=[hash[c, e]]) - : : +- HashAggregate(isMerge=[true], groupBy=[c, e], select=[c, e, Final_SUM(sum$0) AS sum_b]) - : : +- Exchange(distribution=[hash[c, e]]) - : : +- LocalHashAggregate(groupBy=[c, e], select=[c, e, Partial_SUM(b) AS sum$0]) - : : +- Calc(select=[c, e, b]) - : : +- HashJoin(joinType=[InnerJoin], where=[=(a, d)], select=[a, b, c, d, e], build=[right]) - : : :- Exchange(distribution=[hash[a]]) - : : : +- Calc(select=[a, b, c], where=[IS NOT NULL(c)]) - : : : +- TableSourceScan(table=[[x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) - : : +- Exchange(distribution=[hash[d]]) - : : +- Calc(select=[d, e], where=[>(e, 10)]) - : : +- TableSourceScan(table=[[y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) - : +- Exchange(distribution=[hash[c, e, $f5]], exchange_mode=[BATCH]) - : +- Calc(select=[sum_b, /(CAST(CASE(>(w0$o0, 0:BIGINT), w0$o1, null:BIGINT)), w0$o0) AS avg_b, c, e, +(w1$o0, 1) AS $f5]) - : +- Reused(reference_id=[1]) + :- Calc(select=[sum_b, avg_b, rn, c, e, sum_b0, avg_b0]) + : +- HashJoin(joinType=[InnerJoin], where=[AND(=(c, c0), =(e, e0), =(rn, $f5))], select=[sum_b, avg_b, rn, c, e, sum_b0, avg_b0, c0, e0, $f5], build=[left]) + : :- Exchange(distribution=[hash[c, e, rn]]) + : : +- Calc(select=[sum_b, /(CAST(CASE(>(w0$o0, 0:BIGINT), w0$o1, null:BIGINT)), w0$o0) AS avg_b, w1$o0 AS rn, c, e], where=[AND(<>(c, _UTF-16LE'':VARCHAR(65536) CHARACTER SET "UTF-16LE"), >(-(sum_b, /(CAST(CASE(>(w0$o0, 0:BIGINT), w0$o1, null:BIGINT)), w0$o0)), 3))]) + : : +- OverAggregate(partitionBy=[c, e], orderBy=[], window#0=[COUNT(sum_b) AS w0$o0, $SUM0(sum_b) AS w0$o1 RANG BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING], window#1=[RANK(*) AS w1$o0 RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[c, e, sum_b, w0$o0, w0$o1, w1$o0], reuse_id=[1]) + : : +- Sort(orderBy=[c ASC, e ASC], reuse_id=[2]) + : : +- HashAggregate(isMerge=[true], groupBy=[c, e], select=[c, e, Final_SUM(sum$0) AS sum_b]) + : : +- Exchange(distribution=[hash[c, e]]) + : : +- LocalHashAggregate(groupBy=[c, e], select=[c, e, Partial_SUM(b) AS sum$0]) + : : +- Calc(select=[c, e, b]) + : : +- HashJoin(joinType=[InnerJoin], where=[=(a, d)], select=[a, b, c, d, e], build=[right]) + : : :- Exchange(distribution=[hash[a]]) + : : : +- Calc(select=[a, b, c], where=[IS NOT NULL(c)]) + : : : +- TableSourceScan(table=[[x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : : +- Exchange(distribution=[hash[d]]) + : : +- Calc(select=[d, e], where=[>(e, 10)]) + : : +- TableSourceScan(table=[[y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + : +- Exchange(distribution=[hash[c, e, $f5]], exchange_mode=[BATCH]) + : +- Calc(select=[sum_b, /(CAST(CASE(>(w0$o0, 0:BIGINT), w0$o1, null:BIGINT)), w0$o0) AS avg_b, c, e, +(w1$o0, 1) AS $f5]) + : +- Reused(reference_id=[1]) +- Exchange(distribution=[hash[c, e, $f5]], exchange_mode=[BATCH]) +- Calc(select=[sum_b, c, e, -(w0$o0, 1) AS $f5]) +- OverAggregate(partitionBy=[c, e], orderBy=[c ASC, e ASC], window#0=[RANK(*) AS w0$o0 RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[c, e, sum_b, w0$o0]) @@ -651,8 +649,7 @@ HashJoin(joinType=[InnerJoin], where=[=(a, d0)], select=[a, b, c, d, e, f, a0, b : : +- TableSourceScan(table=[[x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) : +- Exchange(distribution=[hash[d]]) : +- TableSourceScan(table=[[y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) -+- Exchange(distribution=[hash[d]]) - +- Reused(reference_id=[1]) ++- Reused(reference_id=[1]) ]]> @@ -1061,16 +1058,14 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5], a0=[$6], b0=[$7], diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/agg/GroupingSetsTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/agg/GroupingSetsTest.xml index 07fc423e4ca..7113b095454 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/agg/GroupingSetsTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/agg/GroupingSetsTest.xml @@ -279,18 +279,16 @@ LogicalProject(deptno=[$0], gender=[$2], min_name=[$3]) ($f5, 2), AND(=(gender, _UTF-16LE'M':VARCHAR(65536) CHARACTER SET "UTF-16LE"), =(deptno, 10)))]) -+- SortAggregate(isMerge=[true], groupBy=[deptno, gender, deptno0, $e], select=[deptno, gender, deptno0, $e, Final_MIN(min$0) AS min_name, Final_COUNT(count1$1) AS $f5]) ++- SortAggregate(isMerge=[false], groupBy=[deptno, gender, deptno0, $e], select=[deptno, gender, deptno0, $e, MIN(ename) AS min_name, COUNT(*) AS $f5]) +- Sort(orderBy=[deptno ASC, gender ASC, deptno0 ASC, $e ASC]) +- Exchange(distribution=[hash[deptno, gender, deptno0, $e]]) - +- LocalSortAggregate(groupBy=[deptno, gender, deptno0, $e], select=[deptno, gender, deptno0, $e, Partial_MIN(ename) AS min$0, Partial_COUNT(*) AS count1$1]) - +- Sort(orderBy=[deptno ASC, gender ASC, deptno0 ASC, $e ASC]) - +- Expand(projects=[{ename=[$0], deptno=[$1], gender=[$2], deptno0=[$3], $e=[0]}, {ename=[$0], deptno=[$1], gender=[$2], deptno0=[null], $e=[1]}, {ename=[$0], deptno=[$1], gender=[null], deptno0=[$3], $e=[2]}, {ename=[$0], deptno=[$1], gender=[null], deptno0=[null], $e=[3]}, {ename=[$0], deptno=[null], gender=[$2], deptno0=[$3], $e=[4]}, {ename=[$0], deptno=[null], gender=[$2], deptno0=[null], $e=[5]}, {ename=[$0], deptno=[null], gender=[null], deptno0=[$3], $e=[6]}, {ename=[$0], deptno=[null], gender=[null], deptno0=[null], $e=[7]}], projects=[{ename, deptno, gender, deptno0, 0 AS $e}, {ename, deptno, gender, null AS deptno0, 1 AS $e}, {ename, deptno, null AS gender, deptno0, 2 AS $e}, {ename, deptno, null AS gender, null AS deptno0, 3 AS $e}, {ename, null AS deptno, gender, deptno0, 4 AS $e}, {ename, null AS deptno, gender, null AS deptno0, 5 AS $e}, {ename, null AS deptno, null AS gender, deptno0, 6 AS $e}, {ename, null AS deptno, null AS gender, null AS deptno0, 7 AS $e}]) - +- HashJoin(joinType=[InnerJoin], where=[=(deptno, deptno0)], select=[ename, deptno, gender, deptno0], build=[right]) - :- Exchange(distribution=[hash[deptno]]) - : +- TableSourceScan(table=[[emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) - +- Exchange(distribution=[hash[deptno]]) - +- Calc(select=[deptno]) - +- TableSourceScan(table=[[dept, source: [TestTableSource(deptno, dname)]]], fields=[deptno, dname]) + +- Expand(projects=[{ename=[$0], deptno=[$1], gender=[$2], deptno0=[$3], $e=[0]}, {ename=[$0], deptno=[$1], gender=[$2], deptno0=[null], $e=[1]}, {ename=[$0], deptno=[$1], gender=[null], deptno0=[$3], $e=[2]}, {ename=[$0], deptno=[$1], gender=[null], deptno0=[null], $e=[3]}, {ename=[$0], deptno=[null], gender=[$2], deptno0=[$3], $e=[4]}, {ename=[$0], deptno=[null], gender=[$2], deptno0=[null], $e=[5]}, {ename=[$0], deptno=[null], gender=[null], deptno0=[$3], $e=[6]}, {ename=[$0], deptno=[null], gender=[null], deptno0=[null], $e=[7]}], projects=[{ename, deptno, gender, deptno0, 0 AS $e}, {ename, deptno, gender, null AS deptno0, 1 AS $e}, {ename, deptno, null AS gender, deptno0, 2 AS $e}, {ename, deptno, null AS gender, null AS deptno0, 3 AS $e}, {ename, null AS deptno, gender, deptno0, 4 AS $e}, {ename, null AS deptno, gender, null AS deptno0, 5 AS $e}, {ename, null AS deptno, null AS gender, deptno0, 6 AS $e}, {ename, null AS deptno, null AS gender, null AS deptno0, 7 AS $e}]) + +- HashJoin(joinType=[InnerJoin], where=[=(deptno, deptno0)], select=[ename, deptno, gender, deptno0], build=[right]) + :- Exchange(distribution=[hash[deptno]]) + : +- TableSourceScan(table=[[emp, source: [TestTableSource(ename, deptno, gender)]]], fields=[ename, deptno, gender]) + +- Exchange(distribution=[hash[deptno]]) + +- Calc(select=[deptno]) + +- TableSourceScan(table=[[dept, source: [TestTableSource(deptno, dname)]]], fields=[deptno, dname]) ]]> diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/agg/OverAggregateTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/agg/OverAggregateTest.xml index 5a6196a8ef2..c10ad212c47 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/agg/OverAggregateTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/agg/OverAggregateTest.xml @@ -16,7 +16,40 @@ See the License for the specific language governing permissions and limitations under the License. --> - + + + + + + (COUNT($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), 0), $SUM0($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), null:INTEGER)], EXPR$1=[MAX($0) OVER (PARTITION BY $1 ORDER BY $2 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)], EXPR$2=[/(CAST(CASE(>(COUNT($0) OVER (PARTITION BY $2 ORDER BY $0 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), 0), $SUM0($0) OVER (PARTITION BY $2 ORDER BY $0 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), null:INTEGER)):DOUBLE, COUNT($0) OVER (PARTITION BY $2 ORDER BY $0 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW))], EXPR$3=[RANK() OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)], EXPR$4=[MIN($0) OVER (PARTITION BY $2 ORDER BY $0 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)]) ++- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]]) +]]> + + + (w0$o0, 0:BIGINT), w0$o1, null:INTEGER) AS EXPR$0, w1$o0 AS EXPR$1, /(CAST(CASE(>(w2$o0, 0:BIGINT), w2$o1, null:INTEGER)), w2$o0) AS EXPR$2, w0$o2 AS EXPR$3, w2$o2 AS EXPR$4]) ++- OverAggregate(partitionBy=[c], orderBy=[a ASC], window#0=[COUNT(a) AS w2$o0, $SUM0(a) AS w2$o1, MIN(a) AS w2$o2 RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[a, b, c, w0$o0, w0$o1, w0$o2, w1$o0, w2$o0, w2$o1, w2$o2]) + +- Sort(orderBy=[c ASC, a ASC]) + +- Exchange(distribution=[hash[c]]) + +- OverAggregate(partitionBy=[b], orderBy=[c ASC], window#0=[MAX(a) AS w1$o0 RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[a, b, c, w0$o0, w0$o1, w0$o2, w1$o0]) + +- Sort(orderBy=[b ASC, c ASC]) + +- OverAggregate(partitionBy=[b], orderBy=[a ASC], window#0=[COUNT(a) AS w0$o0, $SUM0(a) AS w0$o1, RANK(*) AS w0$o2 RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], window#1=[ RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[a, b, c, w0$o0, w0$o1, w0$o2]) + +- Sort(orderBy=[b ASC, a ASC]) + +- Exchange(distribution=[hash[b]]) + +- TableSourceScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +]]> + + + (COUNT($0) OVER (PARTITION BY $1 ORDER BY $2 NULLS (w0$o0, 0:BIGINT), w0$o1, null:INTEGER) AS EXPR$0, w1$o0 AS EXPR$1, w2$o0 AS EXPR$2, w0$o2 AS EXPR$3, /(CAST(CASE(>(w3$o0, 0:BIGINT), w3$o1, null:INTEGER)), w3$o0) AS EXPR$4]) -+- OverAggregate(orderBy=[b ASC], window#0=[COUNT(a) AS w3$o0, $SUM0(a) AS w3$o1 RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[a, b, c, w0$o0, w0$o1, w0$o2, w1$o0, w2$o0, w3$o0, w3$o1]) - +- Sort(orderBy=[b ASC]) - +- Exchange(distribution=[single]) - +- OverAggregate(orderBy=[c ASC, a ASC], window#0=[MIN(a) AS w2$o0 RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[a, b, c, w0$o0, w0$o1, w0$o2, w1$o0, w2$o0]) - +- Sort(orderBy=[c ASC, a ASC]) - +- Exchange(distribution=[single]) - +- OverAggregate(partitionBy=[c], orderBy=[a ASC], window#0=[MAX(a) AS w1$o0 RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[a, b, c, w0$o0, w0$o1, w0$o2, w1$o0]) ++- OverAggregate(partitionBy=[c], orderBy=[a ASC], window#0=[MAX(a) AS w0$o0 RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[a, b, c, w0$o2, w1$o0, w0$o1, w3$o0, w3$o1, w2$o0, w0$o0]) + +- Sort(orderBy=[c ASC, a ASC]) + +- Exchange(distribution=[hash[c]]) + +- OverAggregate(partitionBy=[b], orderBy=[c ASC], window#0=[COUNT(a) AS w3$o0, $SUM0(a) AS w3$o1, RANK(*) AS w2$o0 RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], window#1=[ RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[a, b, c, w0$o2, w1$o0, w0$o1, w3$o0, w3$o1, w2$o0]) + +- Sort(orderBy=[b ASC, c ASC]) + +- Exchange(distribution=[hash[b]]) + +- OverAggregate(orderBy=[c ASC, a ASC], window#0=[MIN(a) AS w0$o1 RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[a, b, c, w0$o2, w1$o0, w0$o1]) +- Sort(orderBy=[c ASC, a ASC]) - +- Exchange(distribution=[hash[c]]) - +- OverAggregate(partitionBy=[b], orderBy=[c ASC], window#0=[COUNT(a) AS w0$o0, $SUM0(a) AS w0$o1, RANK(*) AS w0$o2 RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], window#1=[ RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[a, b, c, w0$o0, w0$o1, w0$o2]) - +- Sort(orderBy=[b ASC, c ASC]) - +- Exchange(distribution=[hash[b]]) - +- TableSourceScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + +- OverAggregate(orderBy=[b ASC], window#0=[COUNT(a) AS w0$o2, $SUM0(a) AS w1$o0 RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[a, b, c, w0$o2, w1$o0]) + +- Sort(orderBy=[b ASC]) + +- Exchange(distribution=[single]) + +- TableSourceScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> @@ -102,14 +134,12 @@ LogicalProject(EXPR$0=[COUNT() OVER (PARTITION BY $2 ORDER BY $0 NULLS FIRST RAN (w1$o0, 0:BIGINT), w1$o1, null:INTEGER) AS EXPR$1, w2$o0 AS EXPR$2, CASE(>(w3$o0, 0:BIGINT), w3$o1, null:INTEGER) AS EXPR$3, w4$o0 AS EXPR$4]) -+- OverAggregate(partitionBy=[c], orderBy=[c ASC], window#0=[COUNT(*) AS w4$o0 ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING], select=[a, c, w0$o0, w1$o0, w1$o1, w2$o0, w3$o0, w3$o1, w4$o0]) - +- Sort(orderBy=[c ASC]) - +- Exchange(distribution=[hash[c]]) - +- OverAggregate(partitionBy=[c], orderBy=[a ASC], window#0=[COUNT(*) AS w0$o0 RANG BETWEEN -1 PRECEDING AND 10 FOLLOWING], window#1=[COUNT(a) AS w1$o0, $SUM0(a) AS w1$o1 RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], window#2=[RANK(*) AS w2$o0 RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], window#3=[COUNT(a) AS w3$o0, $SUM0(a) AS w3$o1 RANG BETWEEN 1 PRECEDING AND 10 FOLLOWING], select=[a, c, w0$o0, w1$o0, w1$o1, w2$o0, w3$o0, w3$o1]) - +- Sort(orderBy=[c ASC, a ASC]) - +- Exchange(distribution=[hash[c]]) - +- Calc(select=[a, c]) - +- TableSourceScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ++- OverAggregate(partitionBy=[c], orderBy=[c ASC], window#0=[COUNT(*) AS w4$o0 ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING], select=[a, c, w0$o0, w1$o0, w1$o1, w3$o1, w2$o0, w3$o0, w4$o0]) + +- OverAggregate(partitionBy=[c], orderBy=[a ASC], window#0=[COUNT(*) AS w0$o0 RANG BETWEEN -1 PRECEDING AND 10 FOLLOWING], window#1=[COUNT(a) AS w1$o0, $SUM0(a) AS w1$o1 RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], window#2=[COUNT(a) AS w3$o1, $SUM0(a) AS w2$o0 RANG BETWEEN 1 PRECEDING AND 10 FOLLOWING], window#3=[RANK(*) AS w3$o0 RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[a, c, w0$o0, w1$o0, w1$o1, w3$o1, w2$o0, w3$o0]) + +- Sort(orderBy=[c ASC, a ASC]) + +- Exchange(distribution=[hash[c]]) + +- Calc(select=[a, c]) + +- TableSourceScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> @@ -185,36 +215,6 @@ Calc(select=[c, w0$o0 AS $1]) +- Exchange(distribution=[hash[c]]) +- Calc(select=[a, c]) +- TableSourceScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) -]]> - - - - - - - - (COUNT($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), 0), $SUM0($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), null:INTEGER)], EXPR$1=[MIN($0) OVER (PARTITION BY $1 RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)], EXPR$2=[MAX($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)]) -+- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]]) -]]> - - - (w0$o0, 0:BIGINT), w0$o1, null:INTEGER) AS EXPR$0, w1$o0 AS EXPR$1, w0$o2 AS EXPR$2]) -+- OverAggregate(partitionBy=[b], window#0=[MIN(a) AS w1$o0 RANG BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING], select=[a, b, w0$o0, w0$o1, w0$o2, w1$o0]) - +- Sort(orderBy=[b ASC]) - +- Exchange(distribution=[hash[b]]) - +- OverAggregate(partitionBy=[b], orderBy=[a ASC], window#0=[COUNT(a) AS w0$o0, $SUM0(a) AS w0$o1, MAX(a) AS w0$o2 RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[a, b, w0$o0, w0$o1, w0$o2]) - +- Sort(orderBy=[b ASC, a ASC]) - +- Exchange(distribution=[hash[b]]) - +- Calc(select=[a, b]) - +- TableSourceScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> @@ -417,33 +417,38 @@ Calc(select=[CASE(>(w0$o0, 0:BIGINT), w0$o1, null:INTEGER) AS EXPR$0, w0$o2 AS E ]]> - + (COUNT($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), 0), $SUM0($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), null:INTEGER)], EXPR$1=[MAX($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)]) +LogicalProject(EXPR$0=[CASE(>(COUNT($0) OVER (PARTITION BY $1 ORDER BY $2 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), 0), $SUM0($0) OVER (PARTITION BY $1 ORDER BY $2 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), null:INTEGER)], EXPR$1=[MAX($0) OVER (PARTITION BY $1 ORDER BY $1 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)], EXPR$2=[/(CAST(CASE(>(COUNT($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), 0), $SUM0($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), null:INTEGER)):DOUBLE, COUNT($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW))], EXPR$3=[RANK() OVER (PARTITION BY $1 ORDER BY $2 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)], EXPR$4=[MIN($0) OVER (PARTITION BY $1 ORDER BY $1 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)]) +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]]) ]]> (w0$o0, 0:BIGINT), w0$o1, null:INTEGER) AS EXPR$0, w0$o2 AS EXPR$1]) -+- OverAggregate(partitionBy=[b], orderBy=[a ASC], window#0=[COUNT(a) AS w0$o0, $SUM0(a) AS w0$o1, MAX(a) AS w0$o2 RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[a, b, w0$o0, w0$o1, w0$o2]) - +- Sort(orderBy=[b ASC, a ASC]) - +- Exchange(distribution=[hash[b]]) - +- Calc(select=[a, b]) - +- TableSourceScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +Calc(select=[CASE(>(w0$o0, 0:BIGINT), w0$o1, null:INTEGER) AS EXPR$0, w1$o0 AS EXPR$1, /(CAST(CASE(>(w2$o0, 0:BIGINT), w2$o1, null:INTEGER)), w2$o0) AS EXPR$2, w0$o2 AS EXPR$3, w1$o1 AS EXPR$4]) ++- OverAggregate(partitionBy=[b], orderBy=[c ASC], window#0=[COUNT(a) AS w2$o0, $SUM0(a) AS w2$o1, RANK(*) AS w1$o0 RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], window#1=[ RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[a, b, c, w0$o1, w0$o2, w1$o1, w0$o0, w2$o0, w2$o1, w1$o0]) + +- Sort(orderBy=[b ASC, c ASC]) + +- OverAggregate(partitionBy=[b], orderBy=[b ASC], window#0=[MAX(a) AS w1$o1, MIN(a) AS w0$o0 RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[a, b, c, w0$o1, w0$o2, w1$o1, w0$o0]) + +- OverAggregate(partitionBy=[b], orderBy=[a ASC], window#0=[COUNT(a) AS w0$o1, $SUM0(a) AS w0$o2 RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[a, b, c, w0$o1, w0$o2]) + +- Sort(orderBy=[b ASC, a ASC]) + +- Exchange(distribution=[hash[b]]) + +- TableSourceScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> - + (COUNT($0) OVER (PARTITION BY $1 ORDER BY $2 NULLS (w0$o0, 0:BIGINT), w0$o1, null:INTEGER) AS EXPR$0, w1$o0 AS EXPR$1, /(CAST(CASE(>(w2$o0, 0:BIGINT), w2$o1, null:INTEGER)), w2$o0) AS EXPR$2, w3$o0 AS EXPR$3, w4$o0 AS EXPR$4]) -+- OverAggregate(partitionBy=[b], orderBy=[b ASC], window#0=[MIN(a) AS w4$o0 RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[a, b, c, w0$o0, w0$o1, w1$o0, w2$o0, w2$o1, w3$o0, w4$o0]) - +- Sort(orderBy=[b ASC]) - +- Exchange(distribution=[hash[b]]) - +- OverAggregate(partitionBy=[b], orderBy=[a ASC, b ASC], window#0=[RANK(*) AS w3$o0 RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[a, b, c, w0$o0, w0$o1, w1$o0, w2$o0, w2$o1, w3$o0]) - +- Sort(orderBy=[b ASC, a ASC]) - +- Exchange(distribution=[hash[b]]) - +- OverAggregate(partitionBy=[b], orderBy=[a ASC, c ASC], window#0=[COUNT(a) AS w2$o0, $SUM0(a) AS w2$o1 RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[a, b, c, w0$o0, w0$o1, w1$o0, w2$o0, w2$o1]) - +- Sort(orderBy=[b ASC, a ASC, c ASC]) - +- Exchange(distribution=[hash[b]]) - +- OverAggregate(partitionBy=[b], orderBy=[a ASC], window#0=[MAX(a) AS w1$o0 RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[a, b, c, w0$o0, w0$o1, w1$o0]) - +- Sort(orderBy=[b ASC, a ASC]) - +- Exchange(distribution=[hash[b]]) - +- OverAggregate(partitionBy=[b], orderBy=[c ASC], window#0=[COUNT(a) AS w0$o0, $SUM0(a) AS w0$o1 RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[a, b, c, w0$o0, w0$o1]) - +- Sort(orderBy=[b ASC, c ASC]) - +- Exchange(distribution=[hash[b]]) - +- TableSourceScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ++- OverAggregate(partitionBy=[b], orderBy=[c ASC], window#0=[COUNT(a) AS w1$o0, $SUM0(a) AS w3$o0 RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[a, b, c, w2$o0, w0$o0, w2$o1, w4$o0, w0$o1, w1$o0, w3$o0]) + +- Sort(orderBy=[b ASC, c ASC]) + +- OverAggregate(partitionBy=[b], orderBy=[b ASC], window#0=[MIN(a) AS w0$o1 RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[a, b, c, w2$o0, w0$o0, w2$o1, w4$o0, w0$o1]) + +- OverAggregate(partitionBy=[b], orderBy=[a ASC, c ASC], window#0=[COUNT(a) AS w2$o1, $SUM0(a) AS w4$o0 RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[a, b, c, w2$o0, w0$o0, w2$o1, w4$o0]) + +- Sort(orderBy=[b ASC, a ASC, c ASC]) + +- OverAggregate(partitionBy=[b], orderBy=[a ASC], window#0=[MAX(a) AS w2$o0 RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], window#1=[RANK(*) AS w0$o0 RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[a, b, c, w2$o0, w0$o0]) + +- Sort(orderBy=[b ASC, a ASC]) + +- Exchange(distribution=[hash[b]]) + +- TableSourceScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> - + (COUNT($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), 0), $SUM0($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), null:INTEGER)], EXPR$1=[MAX($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)]) +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]]) ]]> (w0$o0, 0:BIGINT), w0$o1, null:INTEGER) AS EXPR$0, w0$o2 AS EXPR$1]) ++- OverAggregate(partitionBy=[b], orderBy=[a ASC], window#0=[COUNT(a) AS w0$o0, $SUM0(a) AS w0$o1, MAX(a) AS w0$o2 RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[a, b, w0$o0, w0$o1, w0$o2]) + +- Sort(orderBy=[b ASC, a ASC]) +- Exchange(distribution=[hash[b]]) - +- OverAggregate(partitionBy=[b], orderBy=[c ASC, a DESC], window#0=[RANK(*) AS w0$o0 RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[a, b, c, w0$o0]) - +- Sort(orderBy=[b ASC, c ASC, a DESC]) - +- Exchange(distribution=[hash[b]]) + +- Calc(select=[a, b]) + +- TableSourceScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +]]> + + + + + + + + (COUNT($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), 0), $SUM0($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), null:INTEGER)], EXPR$1=[MAX($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)], EXPR$2=[/(CAST(CASE(>(COUNT($0) OVER (PARTITION BY $1 ORDER BY $0 DESC NULLS LAST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), 0), $SUM0($0) OVER (PARTITION BY $1 ORDER BY $0 DESC NULLS LAST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), null:INTEGER)):DOUBLE, COUNT($0) OVER (PARTITION BY $1 ORDER BY $0 DESC NULLS LAST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW))], EXPR$3=[RANK() OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)], EXPR$4=[MIN($0) OVER (PARTITION BY $1 ORDER BY $0 DESC NULLS LAST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)]) ++- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]]) +]]> + + + (w0$o0, 0:BIGINT), w0$o1, null:INTEGER) AS EXPR$0, w0$o2 AS EXPR$1, /(CAST(CASE(>(w1$o0, 0:BIGINT), w1$o1, null:INTEGER)), w1$o0) AS EXPR$2, w0$o3 AS EXPR$3, w1$o2 AS EXPR$4]) ++- OverAggregate(partitionBy=[b], orderBy=[a DESC], window#0=[COUNT(a) AS w1$o0, $SUM0(a) AS w1$o1, MIN(a) AS w1$o2 RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[a, b, w0$o0, w0$o1, w0$o2, w0$o3, w1$o0, w1$o1, w1$o2]) + +- Sort(orderBy=[b ASC, a DESC]) + +- OverAggregate(partitionBy=[b], orderBy=[a ASC], window#0=[COUNT(a) AS w0$o0, $SUM0(a) AS w0$o1, MAX(a) AS w0$o2, RANK(*) AS w0$o3 RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], window#1=[ RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[a, b, w0$o0, w0$o1, w0$o2, w0$o3]) + +- Sort(orderBy=[b ASC, a ASC]) + +- Exchange(distribution=[hash[b]]) + +- Calc(select=[a, b]) +- TableSourceScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> - + + + + + + + + + (COUNT($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), 0), $SUM0($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), null:INTEGER)], EXPR$1=[MIN($0) OVER (PARTITION BY $1 RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)], EXPR$2=[MAX($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)]) ++- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]]) +]]> + + + (w0$o0, 0:BIGINT), w0$o1, null:INTEGER) AS EXPR$0, w1$o0 AS EXPR$1, w0$o2 AS EXPR$2]) ++- OverAggregate(partitionBy=[b], orderBy=[a ASC], window#0=[COUNT(a) AS w1$o0, $SUM0(a) AS w0$o0, MAX(a) AS w0$o1 RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[a, b, w0$o2, w1$o0, w0$o0, w0$o1]) +- Sort(orderBy=[b ASC, a ASC]) - +- Exchange(distribution=[hash[b]]) - +- OverAggregate(partitionBy=[b], orderBy=[a DESC], window#0=[RANK(*) AS w0$o0 RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[a, b, c, w0$o0]) - +- Sort(orderBy=[b ASC, a DESC]) - +- Exchange(distribution=[hash[b]]) + +- OverAggregate(partitionBy=[b], window#0=[MIN(a) AS w0$o2 RANG BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING], select=[a, b, w0$o2]) + +- Sort(orderBy=[b ASC]) + +- Exchange(distribution=[hash[b]]) + +- Calc(select=[a, b]) +- TableSourceScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +]]> + + + + + + + + + + + diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/join/BroadcastHashSemiAntiJoinTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/join/BroadcastHashSemiAntiJoinTest.xml index a2bb48731b8..39ec889a551 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/join/BroadcastHashSemiAntiJoinTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/join/BroadcastHashSemiAntiJoinTest.xml @@ -1054,13 +1054,12 @@ Calc(select=[b]) : +- TableSourceScan(table=[[r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) +- Exchange(distribution=[broadcast]) +- SortAggregate(isMerge=[false], select=[SINGLE_VALUE(EXPR$0) AS $f0]) - +- Exchange(distribution=[single]) - +- Calc(select=[*(0.5:DECIMAL(2, 1), $f0) AS EXPR$0]) - +- HashAggregate(isMerge=[true], select=[Final_SUM(sum$0) AS $f0]) - +- Exchange(distribution=[single]) - +- LocalHashAggregate(select=[Partial_SUM(j) AS sum$0]) - +- Calc(select=[j], where=[<(i, 100)]) - +- TableSourceScan(table=[[t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + +- Calc(select=[*(0.5:DECIMAL(2, 1), $f0) AS EXPR$0]) + +- SortAggregate(isMerge=[true], select=[Final_SUM(sum$0) AS $f0]) + +- Exchange(distribution=[single]) + +- LocalSortAggregate(select=[Partial_SUM(j) AS sum$0]) + +- Calc(select=[j], where=[<(i, 100)]) + +- TableSourceScan(table=[[t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) ]]> diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/join/NestedLoopJoinTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/join/NestedLoopJoinTest.xml index aefff811e83..441d2f168b2 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/join/NestedLoopJoinTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/join/NestedLoopJoinTest.xml @@ -124,8 +124,8 @@ LogicalProject(d=[$0], e=[$1], f=[$2], g=[$3], h=[$4], a=[$6], b=[$7], c=[$8]) diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/join/SemiAntiJoinTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/join/SemiAntiJoinTest.xml index 117d8804f38..8ad7125a2cf 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/join/SemiAntiJoinTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/join/SemiAntiJoinTest.xml @@ -1000,12 +1000,11 @@ Calc(select=[a]) : :- Exchange(distribution=[hash[e]]) : : +- Calc(select=[d, e]) : : +- TableSourceScan(table=[[r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) - : +- Exchange(distribution=[hash[j]]) - : +- HashAggregate(isMerge=[true], groupBy=[j], select=[j]) - : +- Exchange(distribution=[hash[j]]) - : +- LocalHashAggregate(groupBy=[j], select=[j]) - : +- Calc(select=[j]) - : +- TableSourceScan(table=[[t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[1]) + : +- HashAggregate(isMerge=[true], groupBy=[j], select=[j]) + : +- Exchange(distribution=[hash[j]]) + : +- LocalHashAggregate(groupBy=[j], select=[j]) + : +- Calc(select=[j]) + : +- TableSourceScan(table=[[t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[1]) +- Exchange(distribution=[hash[i, k]]) +- Calc(select=[i, k]) +- Reused(reference_id=[1]) @@ -1330,13 +1329,12 @@ Calc(select=[b]) : +- TableSourceScan(table=[[r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) +- Exchange(distribution=[broadcast]) +- SortAggregate(isMerge=[false], select=[SINGLE_VALUE(EXPR$0) AS $f0]) - +- Exchange(distribution=[single]) - +- Calc(select=[*(0.5:DECIMAL(2, 1), $f0) AS EXPR$0]) - +- HashAggregate(isMerge=[true], select=[Final_SUM(sum$0) AS $f0]) - +- Exchange(distribution=[single]) - +- LocalHashAggregate(select=[Partial_SUM(j) AS sum$0]) - +- Calc(select=[j], where=[<(i, 100)]) - +- TableSourceScan(table=[[t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + +- Calc(select=[*(0.5:DECIMAL(2, 1), $f0) AS EXPR$0]) + +- SortAggregate(isMerge=[true], select=[Final_SUM(sum$0) AS $f0]) + +- Exchange(distribution=[single]) + +- LocalSortAggregate(select=[Partial_SUM(j) AS sum$0]) + +- Calc(select=[j], where=[<(i, 100)]) + +- TableSourceScan(table=[[t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) ]]> @@ -1611,15 +1609,14 @@ Calc(select=[a]) :- Exchange(distribution=[hash[b]]) : +- Calc(select=[a, b]) : +- TableSourceScan(table=[[l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) - +- Exchange(distribution=[hash[e]]) - +- HashAggregate(isMerge=[true], groupBy=[e], select=[e]) - +- Exchange(distribution=[hash[e]]) - +- LocalHashAggregate(groupBy=[e], select=[e]) - +- Union(all=[true], union=[e]) - :- Calc(select=[e], where=[>(d, 10)]) - : +- TableSourceScan(table=[[r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) - +- Calc(select=[CAST(i) AS i], where=[<(i, 100)]) - +- TableSourceScan(table=[[t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + +- HashAggregate(isMerge=[true], groupBy=[e], select=[e]) + +- Exchange(distribution=[hash[e]]) + +- LocalHashAggregate(groupBy=[e], select=[e]) + +- Union(all=[true], union=[e]) + :- Calc(select=[e], where=[>(d, 10)]) + : +- TableSourceScan(table=[[r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- Calc(select=[CAST(i) AS i], where=[<(i, 100)]) + +- TableSourceScan(table=[[t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) ]]> @@ -1933,19 +1930,18 @@ LogicalFilter(condition=[AND(=($cor0.a, $0), <($1, 100))]) =(CAST(c), 1:BIGINT)]) -: : : +- TableSourceScan(table=[[l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) -: : +- Exchange(distribution=[hash[j]]) -: : +- Calc(select=[j], where=[>(CAST(k), 50:BIGINT)]) -: : +- TableSourceScan(table=[[t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[1]) -: +- Exchange(distribution=[hash[i]]) -: +- Calc(select=[i], where=[<(j, 100)]) -: +- Reused(reference_id=[1]) +:- HashJoin(joinType=[LeftAntiJoin], where=[=(a, i)], select=[a, b, c], build=[right]) +: :- Exchange(distribution=[hash[a]]) +: : +- HashJoin(joinType=[LeftAntiJoin], where=[=(b, j)], select=[a, b, c], build=[right]) +: : :- Exchange(distribution=[hash[b]]) +: : : +- Calc(select=[a, b, c], where=[>=(CAST(c), 1:BIGINT)]) +: : : +- TableSourceScan(table=[[l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: : +- Exchange(distribution=[hash[j]]) +: : +- Calc(select=[j], where=[>(CAST(k), 50:BIGINT)]) +: : +- TableSourceScan(table=[[t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[1]) +: +- Exchange(distribution=[hash[i]]) +: +- Calc(select=[i], where=[<(j, 100)]) +: +- Reused(reference_id=[1]) +- Exchange(distribution=[hash[d]]) +- TableSourceScan(table=[[r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) ]]> @@ -2213,12 +2209,11 @@ LogicalFilter(condition=[=($cor0.a, $0)]) HashJoin(joinType=[LeftAntiJoin], where=[=(a, c)], select=[a, b], build=[right]) :- Exchange(distribution=[hash[a]]) : +- TableSourceScan(table=[[leftT, source: [TestTableSource(a, b)]]], fields=[a, b]) -+- Exchange(distribution=[hash[c]]) - +- HashAggregate(isMerge=[true], groupBy=[c], select=[c]) - +- Exchange(distribution=[hash[c]]) - +- LocalHashAggregate(groupBy=[c], select=[c]) - +- Calc(select=[c]) - +- TableSourceScan(table=[[rightT, source: [TestTableSource(c, d)]]], fields=[c, d]) ++- HashAggregate(isMerge=[true], groupBy=[c], select=[c]) + +- Exchange(distribution=[hash[c]]) + +- LocalHashAggregate(groupBy=[c], select=[c]) + +- Calc(select=[c]) + +- TableSourceScan(table=[[rightT, source: [TestTableSource(c, d)]]], fields=[c, d]) ]]> @@ -2312,39 +2307,36 @@ Calc(select=[b]) :- Exchange(distribution=[hash[c]]) : +- Calc(select=[b, c, CASE(OR(=(c0, 0), AND(<>(c0, 0), IS NULL(i0), >=(ck, c0), IS NOT NULL(a))), 1, OR(=(c1, 0), AND(<>(c1, 0), IS NULL(i), >=(ck0, c1), IS NOT NULL(a))), 2, 3) AS $f3]) : +- HashJoin(joinType=[LeftOuterJoin], where=[=(a, EXPR$0)], select=[a, b, c, c0, ck, i0, c1, ck0, EXPR$0, i], build=[right]) - : :- Exchange(distribution=[hash[a]]) - : : +- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c, c0, ck, i0, c1, ck0], build=[right], singleRowJoin=[true]) - : : :- Calc(select=[a, b, c, c0, ck, i0]) - : : : +- HashJoin(joinType=[LeftOuterJoin], where=[=(a, i)], select=[a, b, c, c0, ck, i, i0], build=[right]) - : : : :- Exchange(distribution=[hash[a]]) - : : : : +- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c, c0, ck], build=[right], singleRowJoin=[true]) - : : : : :- TableSourceScan(table=[[l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) - : : : : +- Exchange(distribution=[broadcast]) - : : : : +- SortAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c, Final_COUNT(count$1) AS ck]) - : : : : +- Exchange(distribution=[single]) - : : : : +- LocalSortAggregate(select=[Partial_COUNT(*) AS count1$0, Partial_COUNT(i) AS count$1]) - : : : : +- Calc(select=[i]) - : : : : +- TableSourceScan(table=[[t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[1]) - : : : +- Exchange(distribution=[hash[i]]) - : : : +- Calc(select=[i, true AS i0]) - : : : +- HashAggregate(isMerge=[true], groupBy=[i], select=[i]) - : : : +- Exchange(distribution=[hash[i]]) - : : : +- LocalHashAggregate(groupBy=[i], select=[i]) - : : : +- Calc(select=[i, true AS i0]) - : : : +- Reused(reference_id=[1]) - : : +- Exchange(distribution=[broadcast]) - : : +- SortAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c, Final_COUNT(count$1) AS ck]) - : : +- Exchange(distribution=[single]) - : : +- LocalSortAggregate(select=[Partial_COUNT(*) AS count1$0, Partial_COUNT(EXPR$0) AS count$1]) - : : +- Calc(select=[CAST(j) AS EXPR$0]) - : : +- Reused(reference_id=[1]) - : +- Exchange(distribution=[hash[EXPR$0]]) - : +- Calc(select=[EXPR$0, true AS i]) - : +- HashAggregate(isMerge=[true], groupBy=[EXPR$0], select=[EXPR$0]) - : +- Exchange(distribution=[hash[EXPR$0]]) - : +- LocalHashAggregate(groupBy=[EXPR$0], select=[EXPR$0]) - : +- Calc(select=[CAST(j) AS EXPR$0, true AS i]) - : +- Reused(reference_id=[1]) + : :- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c, c0, ck, i0, c1, ck0], build=[right], singleRowJoin=[true]) + : : :- Calc(select=[a, b, c, c0, ck, i0]) + : : : +- HashJoin(joinType=[LeftOuterJoin], where=[=(a, i)], select=[a, b, c, c0, ck, i, i0], build=[right]) + : : : :- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c, c0, ck], build=[right], singleRowJoin=[true]) + : : : : :- Exchange(distribution=[hash[a]]) + : : : : : +- TableSourceScan(table=[[l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : : : : +- Exchange(distribution=[broadcast]) + : : : : +- SortAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c, Final_COUNT(count$1) AS ck]) + : : : : +- Exchange(distribution=[single]) + : : : : +- LocalSortAggregate(select=[Partial_COUNT(*) AS count1$0, Partial_COUNT(i) AS count$1]) + : : : : +- Calc(select=[i]) + : : : : +- TableSourceScan(table=[[t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[1]) + : : : +- Calc(select=[i, true AS i0]) + : : : +- HashAggregate(isMerge=[true], groupBy=[i], select=[i]) + : : : +- Exchange(distribution=[hash[i]]) + : : : +- LocalHashAggregate(groupBy=[i], select=[i]) + : : : +- Calc(select=[i, true AS i0]) + : : : +- Reused(reference_id=[1]) + : : +- Exchange(distribution=[broadcast]) + : : +- SortAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c, Final_COUNT(count$1) AS ck]) + : : +- Exchange(distribution=[single]) + : : +- LocalSortAggregate(select=[Partial_COUNT(*) AS count1$0, Partial_COUNT(EXPR$0) AS count$1]) + : : +- Calc(select=[CAST(j) AS EXPR$0]) + : : +- Reused(reference_id=[1]) + : +- Calc(select=[EXPR$0, true AS i]) + : +- HashAggregate(isMerge=[true], groupBy=[EXPR$0], select=[EXPR$0]) + : +- Exchange(distribution=[hash[EXPR$0]]) + : +- LocalHashAggregate(groupBy=[EXPR$0], select=[EXPR$0]) + : +- Calc(select=[CAST(j) AS EXPR$0, true AS i]) + : +- Reused(reference_id=[1]) +- Exchange(distribution=[hash[f]]) +- Calc(select=[d, f]) +- TableSourceScan(table=[[r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) @@ -2557,36 +2549,33 @@ Calc(select=[b]) +- NestedLoopJoin(joinType=[LeftAntiJoin], where=[AND(OR(=(b, e), IS NULL(b), IS NULL(e)), OR(=($f3, d), IS NULL(d)))], select=[b, $f3], build=[right]) :- Calc(select=[b, CASE(OR(=(c0, 0), AND(<>(c0, 0), IS NULL(i0), >=(ck, c0), IS NOT NULL(a))), 1, OR(=(c, 0), AND(<>(c, 0), IS NULL(i), >=(ck0, c), IS NOT NULL(a))), 2, 3) AS $f3]) : +- HashJoin(joinType=[LeftOuterJoin], where=[=(a, j)], select=[a, b, c0, ck, i0, c, ck0, j, i], build=[right]) - : :- Exchange(distribution=[hash[a]]) - : : +- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c0, ck, i0, c, ck0], build=[right], singleRowJoin=[true]) - : : :- Calc(select=[a, b, c AS c0, ck, i0]) - : : : +- HashJoin(joinType=[LeftOuterJoin], where=[=(a, i)], select=[a, b, c, ck, i, i0], build=[right]) - : : : :- Exchange(distribution=[hash[a]]) - : : : : +- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c, ck], build=[right], singleRowJoin=[true]) - : : : : :- Calc(select=[a, b]) - : : : : : +- TableSourceScan(table=[[l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) - : : : : +- Exchange(distribution=[broadcast]) - : : : : +- SortAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c, Final_COUNT(count$1) AS ck]) - : : : : +- Exchange(distribution=[single]) - : : : : +- LocalSortAggregate(select=[Partial_COUNT(*) AS count1$0, Partial_COUNT(i) AS count$1]) - : : : : +- TableSourceScan(table=[[t1, source: [TestTableSource(i)]]], fields=[i], reuse_id=[1]) - : : : +- Exchange(distribution=[hash[i]]) - : : : +- Calc(select=[i, true AS i0]) - : : : +- HashAggregate(isMerge=[true], groupBy=[i], select=[i]) - : : : +- Exchange(distribution=[hash[i]]) - : : : +- LocalHashAggregate(groupBy=[i], select=[i]) - : : : +- Reused(reference_id=[1]) - : : +- Exchange(distribution=[broadcast]) - : : +- HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c, Final_COUNT(count$1) AS ck]) - : : +- Exchange(distribution=[single]) - : : +- LocalHashAggregate(select=[Partial_COUNT(*) AS count1$0, Partial_COUNT(j) AS count$1]) - : : +- TableSourceScan(table=[[t2, source: [TestTableSource(j)]]], fields=[j], reuse_id=[2]) - : +- Exchange(distribution=[hash[j]]) - : +- Calc(select=[j, true AS i]) - : +- HashAggregate(isMerge=[true], groupBy=[j], select=[j]) - : +- Exchange(distribution=[hash[j]]) - : +- LocalHashAggregate(groupBy=[j], select=[j]) - : +- Reused(reference_id=[2]) + : :- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c0, ck, i0, c, ck0], build=[right], singleRowJoin=[true]) + : : :- Calc(select=[a, b, c AS c0, ck, i0]) + : : : +- HashJoin(joinType=[LeftOuterJoin], where=[=(a, i)], select=[a, b, c, ck, i, i0], build=[right]) + : : : :- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c, ck], build=[right], singleRowJoin=[true]) + : : : : :- Exchange(distribution=[hash[a]]) + : : : : : +- Calc(select=[a, b]) + : : : : : +- TableSourceScan(table=[[l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : : : : +- Exchange(distribution=[broadcast]) + : : : : +- SortAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c, Final_COUNT(count$1) AS ck]) + : : : : +- Exchange(distribution=[single]) + : : : : +- LocalSortAggregate(select=[Partial_COUNT(*) AS count1$0, Partial_COUNT(i) AS count$1]) + : : : : +- TableSourceScan(table=[[t1, source: [TestTableSource(i)]]], fields=[i], reuse_id=[1]) + : : : +- Calc(select=[i, true AS i0]) + : : : +- HashAggregate(isMerge=[true], groupBy=[i], select=[i]) + : : : +- Exchange(distribution=[hash[i]]) + : : : +- LocalHashAggregate(groupBy=[i], select=[i]) + : : : +- Reused(reference_id=[1]) + : : +- Exchange(distribution=[broadcast]) + : : +- HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c, Final_COUNT(count$1) AS ck]) + : : +- Exchange(distribution=[single]) + : : +- LocalHashAggregate(select=[Partial_COUNT(*) AS count1$0, Partial_COUNT(j) AS count$1]) + : : +- TableSourceScan(table=[[t2, source: [TestTableSource(j)]]], fields=[j], reuse_id=[2]) + : +- Calc(select=[j, true AS i]) + : +- HashAggregate(isMerge=[true], groupBy=[j], select=[j]) + : +- Exchange(distribution=[hash[j]]) + : +- LocalHashAggregate(groupBy=[j], select=[j]) + : +- Reused(reference_id=[2]) +- Exchange(distribution=[broadcast]) +- Calc(select=[e, d]) +- TableSourceScan(table=[[r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/join/ShuffledHashJoinTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/join/ShuffledHashJoinTest.xml index c687cc57777..e84cca4a9b4 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/join/ShuffledHashJoinTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/join/ShuffledHashJoinTest.xml @@ -59,8 +59,8 @@ LogicalProject(d=[$0], e=[$1], f=[$2], g=[$3], h=[$4], a=[$6], b=[$7], c=[$8]) @@ -1324,15 +1322,14 @@ Calc(select=[a]) :- Exchange(distribution=[hash[b]]) : +- Calc(select=[a, b]) : +- TableSourceScan(table=[[l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) - +- Exchange(distribution=[hash[e]]) - +- HashAggregate(isMerge=[true], groupBy=[e], select=[e]) - +- Exchange(distribution=[hash[e]]) - +- LocalHashAggregate(groupBy=[e], select=[e]) - +- Union(all=[true], union=[e]) - :- Calc(select=[e], where=[>(d, 10)]) - : +- TableSourceScan(table=[[r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) - +- Calc(select=[CAST(i) AS i], where=[<(i, 100)]) - +- TableSourceScan(table=[[t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + +- HashAggregate(isMerge=[true], groupBy=[e], select=[e]) + +- Exchange(distribution=[hash[e]]) + +- LocalHashAggregate(groupBy=[e], select=[e]) + +- Union(all=[true], union=[e]) + :- Calc(select=[e], where=[>(d, 10)]) + : +- TableSourceScan(table=[[r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- Calc(select=[CAST(i) AS i], where=[<(i, 100)]) + +- TableSourceScan(table=[[t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) ]]> @@ -1439,19 +1436,18 @@ LogicalFilter(condition=[AND(=($cor0.a, $0), <($1, 100))]) =(CAST(c), 1:BIGINT)]) -: : : +- TableSourceScan(table=[[l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) -: : +- Exchange(distribution=[hash[j]]) -: : +- Calc(select=[j], where=[>(CAST(k), 50:BIGINT)]) -: : +- TableSourceScan(table=[[t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[1]) -: +- Exchange(distribution=[hash[i]]) -: +- Calc(select=[i], where=[<(j, 100)]) -: +- Reused(reference_id=[1]) +:- HashJoin(joinType=[LeftAntiJoin], where=[=(a, i)], select=[a, b, c], build=[right]) +: :- Exchange(distribution=[hash[a]]) +: : +- HashJoin(joinType=[LeftAntiJoin], where=[=(b, j)], select=[a, b, c], build=[right]) +: : :- Exchange(distribution=[hash[b]]) +: : : +- Calc(select=[a, b, c], where=[>=(CAST(c), 1:BIGINT)]) +: : : +- TableSourceScan(table=[[l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: : +- Exchange(distribution=[hash[j]]) +: : +- Calc(select=[j], where=[>(CAST(k), 50:BIGINT)]) +: : +- TableSourceScan(table=[[t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[1]) +: +- Exchange(distribution=[hash[i]]) +: +- Calc(select=[i], where=[<(j, 100)]) +: +- Reused(reference_id=[1]) +- Exchange(distribution=[hash[d]]) +- TableSourceScan(table=[[r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) ]]> @@ -1805,12 +1801,11 @@ LogicalFilter(condition=[=($cor0.a, $0)]) HashJoin(joinType=[LeftAntiJoin], where=[=(a, c)], select=[a, b], build=[right]) :- Exchange(distribution=[hash[a]]) : +- TableSourceScan(table=[[leftT, source: [TestTableSource(a, b)]]], fields=[a, b]) -+- Exchange(distribution=[hash[c]]) - +- HashAggregate(isMerge=[true], groupBy=[c], select=[c]) - +- Exchange(distribution=[hash[c]]) - +- LocalHashAggregate(groupBy=[c], select=[c]) - +- Calc(select=[c]) - +- TableSourceScan(table=[[rightT, source: [TestTableSource(c, d)]]], fields=[c, d]) ++- HashAggregate(isMerge=[true], groupBy=[c], select=[c]) + +- Exchange(distribution=[hash[c]]) + +- LocalHashAggregate(groupBy=[c], select=[c]) + +- Calc(select=[c]) + +- TableSourceScan(table=[[rightT, source: [TestTableSource(c, d)]]], fields=[c, d]) ]]> @@ -1904,39 +1899,36 @@ Calc(select=[b]) :- Exchange(distribution=[hash[c]]) : +- Calc(select=[b, c, CASE(OR(=(c0, 0), AND(<>(c0, 0), IS NULL(i0), >=(ck, c0), IS NOT NULL(a))), 1, OR(=(c1, 0), AND(<>(c1, 0), IS NULL(i), >=(ck0, c1), IS NOT NULL(a))), 2, 3) AS $f3]) : +- HashJoin(joinType=[LeftOuterJoin], where=[=(a, EXPR$0)], select=[a, b, c, c0, ck, i0, c1, ck0, EXPR$0, i], build=[right]) - : :- Exchange(distribution=[hash[a]]) - : : +- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c, c0, ck, i0, c1, ck0], build=[right], singleRowJoin=[true]) - : : :- Calc(select=[a, b, c, c0, ck, i0]) - : : : +- HashJoin(joinType=[LeftOuterJoin], where=[=(a, i)], select=[a, b, c, c0, ck, i, i0], build=[right]) - : : : :- Exchange(distribution=[hash[a]]) - : : : : +- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c, c0, ck], build=[right], singleRowJoin=[true]) - : : : : :- TableSourceScan(table=[[l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) - : : : : +- Exchange(distribution=[broadcast]) - : : : : +- SortAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c, Final_COUNT(count$1) AS ck]) - : : : : +- Exchange(distribution=[single]) - : : : : +- LocalSortAggregate(select=[Partial_COUNT(*) AS count1$0, Partial_COUNT(i) AS count$1]) - : : : : +- Calc(select=[i]) - : : : : +- TableSourceScan(table=[[t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[1]) - : : : +- Exchange(distribution=[hash[i]]) - : : : +- Calc(select=[i, true AS i0]) - : : : +- HashAggregate(isMerge=[true], groupBy=[i], select=[i]) - : : : +- Exchange(distribution=[hash[i]]) - : : : +- LocalHashAggregate(groupBy=[i], select=[i]) - : : : +- Calc(select=[i, true AS i0]) - : : : +- Reused(reference_id=[1]) - : : +- Exchange(distribution=[broadcast]) - : : +- SortAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c, Final_COUNT(count$1) AS ck]) - : : +- Exchange(distribution=[single]) - : : +- LocalSortAggregate(select=[Partial_COUNT(*) AS count1$0, Partial_COUNT(EXPR$0) AS count$1]) - : : +- Calc(select=[CAST(j) AS EXPR$0]) - : : +- Reused(reference_id=[1]) - : +- Exchange(distribution=[hash[EXPR$0]]) - : +- Calc(select=[EXPR$0, true AS i]) - : +- HashAggregate(isMerge=[true], groupBy=[EXPR$0], select=[EXPR$0]) - : +- Exchange(distribution=[hash[EXPR$0]]) - : +- LocalHashAggregate(groupBy=[EXPR$0], select=[EXPR$0]) - : +- Calc(select=[CAST(j) AS EXPR$0, true AS i]) - : +- Reused(reference_id=[1]) + : :- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c, c0, ck, i0, c1, ck0], build=[right], singleRowJoin=[true]) + : : :- Calc(select=[a, b, c, c0, ck, i0]) + : : : +- HashJoin(joinType=[LeftOuterJoin], where=[=(a, i)], select=[a, b, c, c0, ck, i, i0], build=[right]) + : : : :- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c, c0, ck], build=[right], singleRowJoin=[true]) + : : : : :- Exchange(distribution=[hash[a]]) + : : : : : +- TableSourceScan(table=[[l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : : : : +- Exchange(distribution=[broadcast]) + : : : : +- SortAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c, Final_COUNT(count$1) AS ck]) + : : : : +- Exchange(distribution=[single]) + : : : : +- LocalSortAggregate(select=[Partial_COUNT(*) AS count1$0, Partial_COUNT(i) AS count$1]) + : : : : +- Calc(select=[i]) + : : : : +- TableSourceScan(table=[[t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[1]) + : : : +- Calc(select=[i, true AS i0]) + : : : +- HashAggregate(isMerge=[true], groupBy=[i], select=[i]) + : : : +- Exchange(distribution=[hash[i]]) + : : : +- LocalHashAggregate(groupBy=[i], select=[i]) + : : : +- Calc(select=[i, true AS i0]) + : : : +- Reused(reference_id=[1]) + : : +- Exchange(distribution=[broadcast]) + : : +- SortAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c, Final_COUNT(count$1) AS ck]) + : : +- Exchange(distribution=[single]) + : : +- LocalSortAggregate(select=[Partial_COUNT(*) AS count1$0, Partial_COUNT(EXPR$0) AS count$1]) + : : +- Calc(select=[CAST(j) AS EXPR$0]) + : : +- Reused(reference_id=[1]) + : +- Calc(select=[EXPR$0, true AS i]) + : +- HashAggregate(isMerge=[true], groupBy=[EXPR$0], select=[EXPR$0]) + : +- Exchange(distribution=[hash[EXPR$0]]) + : +- LocalHashAggregate(groupBy=[EXPR$0], select=[EXPR$0]) + : +- Calc(select=[CAST(j) AS EXPR$0, true AS i]) + : +- Reused(reference_id=[1]) +- Exchange(distribution=[hash[f]]) +- Calc(select=[d, f]) +- TableSourceScan(table=[[r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/join/SingleRowJoinTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/join/SingleRowJoinTest.xml index 14e1cdcc310..d1b60a4f380 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/join/SingleRowJoinTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/join/SingleRowJoinTest.xml @@ -184,36 +184,6 @@ Calc(select=[a2]) +- LocalHashAggregate(select=[Partial_COUNT(*) AS count1$0]) +- Calc(select=[0 AS $f0]) +- TableSourceScan(table=[[B, source: [TestTableSource(b1, b2)]]], fields=[b1, b2]) -]]> - - - - - - - - - - - @@ -274,13 +244,12 @@ Calc(select=[a2, EXPR$1]) : +- TableSourceScan(table=[[A, source: [TestTableSource(a1, a2)]]], fields=[a1, a2], reuse_id=[1]) +- Exchange(distribution=[broadcast]) +- SortAggregate(isMerge=[false], select=[SINGLE_VALUE(EXPR$0) AS $f0]) - +- Exchange(distribution=[single]) - +- Calc(select=[*($f0, 0.1:DECIMAL(2, 1)) AS EXPR$0]) - +- HashAggregate(isMerge=[true], select=[Final_SUM(sum$0) AS $f0]) - +- Exchange(distribution=[single]) - +- LocalHashAggregate(select=[Partial_SUM(a1) AS sum$0]) - +- Calc(select=[a1]) - +- Reused(reference_id=[1]) + +- Calc(select=[*($f0, 0.1:DECIMAL(2, 1)) AS EXPR$0]) + +- SortAggregate(isMerge=[true], select=[Final_SUM(sum$0) AS $f0]) + +- Exchange(distribution=[single]) + +- LocalSortAggregate(select=[Partial_SUM(a1) AS sum$0]) + +- Calc(select=[a1]) + +- Reused(reference_id=[1]) ]]> diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/join/SortMergeJoinTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/join/SortMergeJoinTest.xml index 51724cbb2f2..dadd6be1741 100644 --- a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/join/SortMergeJoinTest.xml +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/batch/sql/join/SortMergeJoinTest.xml @@ -59,8 +59,8 @@ LogicalProject(d=[$0], e=[$1], f=[$2], g=[$3], h=[$4], a=[$6], b=[$7], c=[$8]) @@ -1409,15 +1407,14 @@ Calc(select=[a]) :- Exchange(distribution=[hash[b]]) : +- Calc(select=[a, b]) : +- TableSourceScan(table=[[l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) - +- Exchange(distribution=[hash[e]]) - +- HashAggregate(isMerge=[true], groupBy=[e], select=[e]) - +- Exchange(distribution=[hash[e]]) - +- LocalHashAggregate(groupBy=[e], select=[e]) - +- Union(all=[true], union=[e]) - :- Calc(select=[e], where=[>(d, 10)]) - : +- TableSourceScan(table=[[r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) - +- Calc(select=[CAST(i) AS i], where=[<(i, 100)]) - +- TableSourceScan(table=[[t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) + +- HashAggregate(isMerge=[true], groupBy=[e], select=[e]) + +- Exchange(distribution=[hash[e]]) + +- LocalHashAggregate(groupBy=[e], select=[e]) + +- Union(all=[true], union=[e]) + :- Calc(select=[e], where=[>(d, 10)]) + : +- TableSourceScan(table=[[r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) + +- Calc(select=[CAST(i) AS i], where=[<(i, 100)]) + +- TableSourceScan(table=[[t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k]) ]]> @@ -1524,19 +1521,18 @@ LogicalFilter(condition=[AND(=($cor0.a, $0), <($1, 100))]) =(CAST(c), 1:BIGINT)]) -: : : +- TableSourceScan(table=[[l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) -: : +- Exchange(distribution=[hash[j]]) -: : +- Calc(select=[j], where=[>(CAST(k), 50:BIGINT)]) -: : +- TableSourceScan(table=[[t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[1]) -: +- Exchange(distribution=[hash[i]]) -: +- Calc(select=[i], where=[<(j, 100)]) -: +- Reused(reference_id=[1]) +:- SortMergeJoin(joinType=[LeftAntiJoin], where=[=(a, i)], select=[a, b, c]) +: :- Exchange(distribution=[hash[a]]) +: : +- SortMergeJoin(joinType=[LeftAntiJoin], where=[=(b, j)], select=[a, b, c]) +: : :- Exchange(distribution=[hash[b]]) +: : : +- Calc(select=[a, b, c], where=[>=(CAST(c), 1:BIGINT)]) +: : : +- TableSourceScan(table=[[l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +: : +- Exchange(distribution=[hash[j]]) +: : +- Calc(select=[j], where=[>(CAST(k), 50:BIGINT)]) +: : +- TableSourceScan(table=[[t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[1]) +: +- Exchange(distribution=[hash[i]]) +: +- Calc(select=[i], where=[<(j, 100)]) +: +- Reused(reference_id=[1]) +- Exchange(distribution=[hash[d]]) +- TableSourceScan(table=[[r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) ]]> @@ -1890,12 +1886,11 @@ LogicalFilter(condition=[=($cor0.a, $0)]) SortMergeJoin(joinType=[LeftAntiJoin], where=[=(a, c)], select=[a, b]) :- Exchange(distribution=[hash[a]]) : +- TableSourceScan(table=[[leftT, source: [TestTableSource(a, b)]]], fields=[a, b]) -+- Exchange(distribution=[hash[c]]) - +- HashAggregate(isMerge=[true], groupBy=[c], select=[c]) - +- Exchange(distribution=[hash[c]]) - +- LocalHashAggregate(groupBy=[c], select=[c]) - +- Calc(select=[c]) - +- TableSourceScan(table=[[rightT, source: [TestTableSource(c, d)]]], fields=[c, d]) ++- HashAggregate(isMerge=[true], groupBy=[c], select=[c]) + +- Exchange(distribution=[hash[c]]) + +- LocalHashAggregate(groupBy=[c], select=[c]) + +- Calc(select=[c]) + +- TableSourceScan(table=[[rightT, source: [TestTableSource(c, d)]]], fields=[c, d]) ]]> @@ -1989,39 +1984,36 @@ Calc(select=[b]) :- Exchange(distribution=[hash[c]]) : +- Calc(select=[b, c, CASE(OR(=(c0, 0), AND(<>(c0, 0), IS NULL(i0), >=(ck, c0), IS NOT NULL(a))), 1, OR(=(c1, 0), AND(<>(c1, 0), IS NULL(i), >=(ck0, c1), IS NOT NULL(a))), 2, 3) AS $f3]) : +- SortMergeJoin(joinType=[LeftOuterJoin], where=[=(a, EXPR$0)], select=[a, b, c, c0, ck, i0, c1, ck0, EXPR$0, i]) - : :- Exchange(distribution=[hash[a]]) - : : +- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c, c0, ck, i0, c1, ck0], build=[right], singleRowJoin=[true]) - : : :- Calc(select=[a, b, c, c0, ck, i0]) - : : : +- SortMergeJoin(joinType=[LeftOuterJoin], where=[=(a, i)], select=[a, b, c, c0, ck, i, i0]) - : : : :- Exchange(distribution=[hash[a]]) - : : : : +- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c, c0, ck], build=[right], singleRowJoin=[true]) - : : : : :- TableSourceScan(table=[[l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) - : : : : +- Exchange(distribution=[broadcast]) - : : : : +- SortAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c, Final_COUNT(count$1) AS ck]) - : : : : +- Exchange(distribution=[single]) - : : : : +- LocalSortAggregate(select=[Partial_COUNT(*) AS count1$0, Partial_COUNT(i) AS count$1]) - : : : : +- Calc(select=[i]) - : : : : +- TableSourceScan(table=[[t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[1]) - : : : +- Exchange(distribution=[hash[i]]) - : : : +- Calc(select=[i, true AS i0]) - : : : +- HashAggregate(isMerge=[true], groupBy=[i], select=[i]) - : : : +- Exchange(distribution=[hash[i]]) - : : : +- LocalHashAggregate(groupBy=[i], select=[i]) - : : : +- Calc(select=[i, true AS i0]) - : : : +- Reused(reference_id=[1]) - : : +- Exchange(distribution=[broadcast]) - : : +- SortAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c, Final_COUNT(count$1) AS ck]) - : : +- Exchange(distribution=[single]) - : : +- LocalSortAggregate(select=[Partial_COUNT(*) AS count1$0, Partial_COUNT(EXPR$0) AS count$1]) - : : +- Calc(select=[CAST(j) AS EXPR$0]) - : : +- Reused(reference_id=[1]) - : +- Exchange(distribution=[hash[EXPR$0]]) - : +- Calc(select=[EXPR$0, true AS i]) - : +- HashAggregate(isMerge=[true], groupBy=[EXPR$0], select=[EXPR$0]) - : +- Exchange(distribution=[hash[EXPR$0]]) - : +- LocalHashAggregate(groupBy=[EXPR$0], select=[EXPR$0]) - : +- Calc(select=[CAST(j) AS EXPR$0, true AS i]) - : +- Reused(reference_id=[1]) + : :- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c, c0, ck, i0, c1, ck0], build=[right], singleRowJoin=[true]) + : : :- Calc(select=[a, b, c, c0, ck, i0]) + : : : +- SortMergeJoin(joinType=[LeftOuterJoin], where=[=(a, i)], select=[a, b, c, c0, ck, i, i0]) + : : : :- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, b, c, c0, ck], build=[right], singleRowJoin=[true]) + : : : : :- Exchange(distribution=[hash[a]]) + : : : : : +- TableSourceScan(table=[[l, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) + : : : : +- Exchange(distribution=[broadcast]) + : : : : +- SortAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c, Final_COUNT(count$1) AS ck]) + : : : : +- Exchange(distribution=[single]) + : : : : +- LocalSortAggregate(select=[Partial_COUNT(*) AS count1$0, Partial_COUNT(i) AS count$1]) + : : : : +- Calc(select=[i]) + : : : : +- TableSourceScan(table=[[t, source: [TestTableSource(i, j, k)]]], fields=[i, j, k], reuse_id=[1]) + : : : +- Calc(select=[i, true AS i0]) + : : : +- HashAggregate(isMerge=[true], groupBy=[i], select=[i]) + : : : +- Exchange(distribution=[hash[i]]) + : : : +- LocalHashAggregate(groupBy=[i], select=[i]) + : : : +- Calc(select=[i, true AS i0]) + : : : +- Reused(reference_id=[1]) + : : +- Exchange(distribution=[broadcast]) + : : +- SortAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS c, Final_COUNT(count$1) AS ck]) + : : +- Exchange(distribution=[single]) + : : +- LocalSortAggregate(select=[Partial_COUNT(*) AS count1$0, Partial_COUNT(EXPR$0) AS count$1]) + : : +- Calc(select=[CAST(j) AS EXPR$0]) + : : +- Reused(reference_id=[1]) + : +- Calc(select=[EXPR$0, true AS i]) + : +- HashAggregate(isMerge=[true], groupBy=[EXPR$0], select=[EXPR$0]) + : +- Exchange(distribution=[hash[EXPR$0]]) + : +- LocalHashAggregate(groupBy=[EXPR$0], select=[EXPR$0]) + : +- Calc(select=[CAST(j) AS EXPR$0, true AS i]) + : +- Reused(reference_id=[1]) +- Exchange(distribution=[hash[f]]) +- Calc(select=[d, f]) +- TableSourceScan(table=[[r, source: [TestTableSource(d, e, f)]]], fields=[d, e, f]) diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/rules/logical/WindowGroupReorderRuleTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/rules/logical/WindowGroupReorderRuleTest.xml new file mode 100644 index 00000000000..652470593ff --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/rules/logical/WindowGroupReorderRuleTest.xml @@ -0,0 +1,280 @@ + + + + + + + + + (COUNT($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), 0), $SUM0($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), null:INTEGER)], EXPR$1=[MAX($0) OVER (PARTITION BY $1 ORDER BY $2 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)], EXPR$2=[/(CAST(CASE(>(COUNT($0) OVER (PARTITION BY $2 ORDER BY $0 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), 0), $SUM0($0) OVER (PARTITION BY $2 ORDER BY $0 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), null:INTEGER)):DOUBLE, COUNT($0) OVER (PARTITION BY $2 ORDER BY $0 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW))], EXPR$3=[RANK() OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)], EXPR$4=[MIN($0) OVER (PARTITION BY $2 ORDER BY $0 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)]) ++- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]]) +]]> + + + ($3, 0), $4, null:INTEGER)], EXPR$1=[$6], EXPR$2=[/(CAST(CASE(>($7, 0), $8, null:INTEGER)):DOUBLE, $7)], EXPR$3=[$5], EXPR$4=[$9]) ++- LogicalWindow(window#0=[window(partition {1} order by [0 ASC-nulls-first] range between UNBOUNDED PRECEDING and CURRENT ROW aggs [COUNT($0), $SUM0($0), RANK()])], window#1=[window(partition {1} order by [2 ASC-nulls-first] range between UNBOUNDED PRECEDING and CURRENT ROW aggs [MAX($0)])], window#2=[window(partition {2} order by [0 ASC-nulls-first] range between UNBOUNDED PRECEDING and CURRENT ROW aggs [COUNT($0), $SUM0($0), MIN($0)])]) + +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]]) +]]> + + + + + + + + (COUNT($0) OVER (PARTITION BY $1 ORDER BY $2 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), 0), $SUM0($0) OVER (PARTITION BY $1 ORDER BY $2 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), null:INTEGER)], EXPR$1=[MAX($0) OVER (PARTITION BY $2 ORDER BY $0 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)], EXPR$2=[MIN($0) OVER (ORDER BY $2 NULLS FIRST, $0 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)], EXPR$3=[RANK() OVER (PARTITION BY $1 ORDER BY $2 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)], EXPR$4=[/(CAST(CASE(>(COUNT($0) OVER (ORDER BY $1 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), 0), $SUM0($0) OVER (ORDER BY $1 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), null:INTEGER)):DOUBLE, COUNT($0) OVER (ORDER BY $1 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW))]) ++- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]]) +]]> + + + ($3, 0), $4, null:INTEGER)], EXPR$1=[$6], EXPR$2=[$7], EXPR$3=[$5], EXPR$4=[/(CAST(CASE(>($8, 0), $9, null:INTEGER)):DOUBLE, $8)]) ++- LogicalProject(a=[$0], b=[$1], c=[$2], w0$o0=[$6], w0$o1=[$7], w0$o2=[$8], w1$o0=[$9], w2$o0=[$5], w3$o0=[$3], w3$o1=[$4]) + +- LogicalWindow(window#0=[window(partition {} order by [1 ASC-nulls-first] range between UNBOUNDED PRECEDING and CURRENT ROW aggs [COUNT($0), $SUM0($0)])], window#1=[window(partition {} order by [2 ASC-nulls-first, 0 ASC-nulls-first] range between UNBOUNDED PRECEDING and CURRENT ROW aggs [MIN($0)])], window#2=[window(partition {1} order by [2 ASC-nulls-first] range between UNBOUNDED PRECEDING and CURRENT ROW aggs [COUNT($0), $SUM0($0), RANK()])], window#3=[window(partition {2} order by [0 ASC-nulls-first] range between UNBOUNDED PRECEDING and CURRENT ROW aggs [MAX($0)])]) + +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]]) +]]> + + + + + + + + (COUNT($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), 0), $SUM0($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), null:INTEGER)], EXPR$1=[MAX($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)], EXPR$2=[/(CAST(CASE(>(COUNT($0) OVER (PARTITION BY $2 ORDER BY $0 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), 0), $SUM0($0) OVER (PARTITION BY $2 ORDER BY $0 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), null:INTEGER)):DOUBLE, COUNT($0) OVER (PARTITION BY $2 ORDER BY $0 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW))], EXPR$3=[RANK() OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)], EXPR$4=[MIN($0) OVER (PARTITION BY $2 ORDER BY $0 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)]) ++- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]]) +]]> + + + ($3, 0), $4, null:INTEGER)], EXPR$1=[$5], EXPR$2=[/(CAST(CASE(>($7, 0), $8, null:INTEGER)):DOUBLE, $7)], EXPR$3=[$6], EXPR$4=[$9]) ++- LogicalWindow(window#0=[window(partition {1} order by [0 ASC-nulls-first] range between UNBOUNDED PRECEDING and CURRENT ROW aggs [COUNT($0), $SUM0($0), MAX($0), RANK()])], window#1=[window(partition {2} order by [0 ASC-nulls-first] range between UNBOUNDED PRECEDING and CURRENT ROW aggs [COUNT($0), $SUM0($0), MIN($0)])]) + +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]]) +]]> + + + + + + + + (COUNT($0) OVER (PARTITION BY $2 ORDER BY $0 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), 0), $SUM0($0) OVER (PARTITION BY $2 ORDER BY $0 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), null:INTEGER)], EXPR$2=[RANK() OVER (PARTITION BY $2 ORDER BY $0 NULLS FIRST, $2 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)], EXPR$3=[CASE(>(COUNT($0) OVER (PARTITION BY $2 ORDER BY $0 NULLS FIRST RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING), 0), $SUM0($0) OVER (PARTITION BY $2 ORDER BY $0 NULLS FIRST RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING), null:INTEGER)], EXPR$4=[COUNT() OVER (PARTITION BY $2 ORDER BY $2 NULLS FIRST ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING)]) ++- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]]) +]]> + + + ($3, 0), $4, null:INTEGER)], EXPR$2=[$5], EXPR$3=[CASE(>($6, 0), $7, null:INTEGER)], EXPR$4=[$8]) ++- LogicalProject(a=[$0], c=[$1], w0$o0=[$2], w1$o0=[$3], w1$o1=[$4], w2$o0=[$7], w3$o0=[$5], w3$o1=[$6], w4$o0=[$8]) + +- LogicalWindow(window#0=[window(partition {1} order by [0 ASC-nulls-first] range between $2 PRECEDING and $3 FOLLOWING aggs [COUNT()])], window#1=[window(partition {1} order by [0 ASC-nulls-first] range between UNBOUNDED PRECEDING and CURRENT ROW aggs [COUNT($0), $SUM0($0)])], window#2=[window(partition {1} order by [0 ASC-nulls-first] range between $4 PRECEDING and $3 FOLLOWING aggs [COUNT($0), $SUM0($0)])], window#3=[window(partition {1} order by [0 ASC-nulls-first, 1 ASC-nulls-first] range between UNBOUNDED PRECEDING and CURRENT ROW aggs [RANK()])], window#4=[window(partition {1} order by [1 ASC-nulls-first] rows between $4 PRECEDING and $3 FOLLOWING aggs [COUNT()])]) + +- LogicalProject(a=[$0], c=[$2]) + +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]]) +]]> + + + + + + + + (COUNT($0) OVER (PARTITION BY $1 ORDER BY $2 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), 0), $SUM0($0) OVER (PARTITION BY $1 ORDER BY $2 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), null:INTEGER)], EXPR$1=[MAX($0) OVER (PARTITION BY $1 ORDER BY $1 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)], EXPR$2=[/(CAST(CASE(>(COUNT($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), 0), $SUM0($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), null:INTEGER)):DOUBLE, COUNT($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW))], EXPR$3=[RANK() OVER (PARTITION BY $1 ORDER BY $2 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)], EXPR$4=[MIN($0) OVER (PARTITION BY $1 ORDER BY $1 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)]) ++- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]]) +]]> + + + ($3, 0), $4, null:INTEGER)], EXPR$1=[$6], EXPR$2=[/(CAST(CASE(>($8, 0), $9, null:INTEGER)):DOUBLE, $8)], EXPR$3=[$5], EXPR$4=[$7]) ++- LogicalProject(a=[$0], b=[$1], c=[$2], w0$o0=[$7], w0$o1=[$8], w0$o2=[$9], w1$o0=[$5], w1$o1=[$6], w2$o0=[$3], w2$o1=[$4]) + +- LogicalWindow(window#0=[window(partition {1} order by [0 ASC-nulls-first] range between UNBOUNDED PRECEDING and CURRENT ROW aggs [COUNT($0), $SUM0($0)])], window#1=[window(partition {1} order by [1 ASC-nulls-first] range between UNBOUNDED PRECEDING and CURRENT ROW aggs [MAX($0), MIN($0)])], window#2=[window(partition {1} order by [2 ASC-nulls-first] range between UNBOUNDED PRECEDING and CURRENT ROW aggs [COUNT($0), $SUM0($0), RANK()])]) + +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]]) +]]> + + + + + + + + (COUNT($0) OVER (PARTITION BY $1 ORDER BY $2 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), 0), $SUM0($0) OVER (PARTITION BY $1 ORDER BY $2 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), null:INTEGER)], EXPR$1=[MAX($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)], EXPR$2=[/(CAST(CASE(>(COUNT($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST, $2 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), 0), $SUM0($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST, $2 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), null:INTEGER)):DOUBLE, COUNT($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST, $2 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW))], EXPR$3=[RANK() OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST, $1 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)], EXPR$4=[MIN($0) OVER (PARTITION BY $1 ORDER BY $1 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)]) ++- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]]) +]]> + + + ($3, 0), $4, null:INTEGER)], EXPR$1=[$5], EXPR$2=[/(CAST(CASE(>($6, 0), $7, null:INTEGER)):DOUBLE, $6)], EXPR$3=[$8], EXPR$4=[$9]) ++- LogicalProject(a=[$0], b=[$1], c=[$2], w0$o0=[$8], w0$o1=[$9], w1$o0=[$3], w2$o0=[$5], w2$o1=[$6], w3$o0=[$4], w4$o0=[$7]) + +- LogicalWindow(window#0=[window(partition {1} order by [0 ASC-nulls-first] range between UNBOUNDED PRECEDING and CURRENT ROW aggs [MAX($0)])], window#1=[window(partition {1} order by [0 ASC-nulls-first, 1 ASC-nulls-first] range between UNBOUNDED PRECEDING and CURRENT ROW aggs [RANK()])], window#2=[window(partition {1} order by [0 ASC-nulls-first, 2 ASC-nulls-first] range between UNBOUNDED PRECEDING and CURRENT ROW aggs [COUNT($0), $SUM0($0)])], window#3=[window(partition {1} order by [1 ASC-nulls-first] range between UNBOUNDED PRECEDING and CURRENT ROW aggs [MIN($0)])], window#4=[window(partition {1} order by [2 ASC-nulls-first] range between UNBOUNDED PRECEDING and CURRENT ROW aggs [COUNT($0), $SUM0($0)])]) + +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]]) +]]> + + + + + + + + (COUNT($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), 0), $SUM0($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), null:INTEGER)], EXPR$1=[MAX($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)], EXPR$2=[/(CAST(CASE(>(COUNT($0) OVER (PARTITION BY $1 ORDER BY $0 DESC NULLS LAST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), 0), $SUM0($0) OVER (PARTITION BY $1 ORDER BY $0 DESC NULLS LAST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), null:INTEGER)):DOUBLE, COUNT($0) OVER (PARTITION BY $1 ORDER BY $0 DESC NULLS LAST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW))], EXPR$3=[RANK() OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)], EXPR$4=[MIN($0) OVER (PARTITION BY $1 ORDER BY $0 DESC NULLS LAST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)]) ++- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]]) +]]> + + + ($2, 0), $3, null:INTEGER)], EXPR$1=[$4], EXPR$2=[/(CAST(CASE(>($6, 0), $7, null:INTEGER)):DOUBLE, $6)], EXPR$3=[$5], EXPR$4=[$8]) ++- LogicalWindow(window#0=[window(partition {1} order by [0 ASC-nulls-first] range between UNBOUNDED PRECEDING and CURRENT ROW aggs [COUNT($0), $SUM0($0), MAX($0), RANK()])], window#1=[window(partition {1} order by [0 DESC-nulls-last] range between UNBOUNDED PRECEDING and CURRENT ROW aggs [COUNT($0), $SUM0($0), MIN($0)])]) + +- LogicalProject(a=[$0], b=[$1]) + +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]]) +]]> + + + + + + + + + + + + + + + + + + + (COUNT($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), 0), $SUM0($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), null:INTEGER)], EXPR$1=[MIN($0) OVER (PARTITION BY $1 RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)], EXPR$2=[MAX($0) OVER (PARTITION BY $1 ORDER BY $0 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)]) ++- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]]) +]]> + + + ($2, 0), $3, null:INTEGER)], EXPR$1=[$5], EXPR$2=[$4]) ++- LogicalProject(a=[$0], b=[$1], w0$o0=[$3], w0$o1=[$4], w0$o2=[$5], w1$o0=[$2]) + +- LogicalWindow(window#0=[window(partition {1} order by [] range between UNBOUNDED PRECEDING and UNBOUNDED FOLLOWING aggs [MIN($0)])], window#1=[window(partition {1} order by [0 ASC-nulls-first] range between UNBOUNDED PRECEDING and CURRENT ROW aggs [COUNT($0), $SUM0($0), MAX($0)])]) + +- LogicalProject(a=[$0], b=[$1]) + +- LogicalTableScan(table=[[MyTable, source: [TestTableSource(a, b, c)]]]) +]]> + + + + + + + + + + + + + + diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/rules/physical/batch/RemoveRedundantLocalHashAggRuleTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/rules/physical/batch/RemoveRedundantLocalHashAggRuleTest.xml new file mode 100644 index 00000000000..2777bf1ae7b --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/rules/physical/batch/RemoveRedundantLocalHashAggRuleTest.xml @@ -0,0 +1,84 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/rules/physical/batch/RemoveRedundantLocalRankRuleTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/rules/physical/batch/RemoveRedundantLocalRankRuleTest.xml new file mode 100644 index 00000000000..68d7f219e82 --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/rules/physical/batch/RemoveRedundantLocalRankRuleTest.xml @@ -0,0 +1,114 @@ + + + + + + + + + + + + + + + + + = 2 + ]]> + + + =($1, 2))]) + +- LogicalProject(a=[$0], rk=[RANK() OVER (PARTITION BY $0 ORDER BY $1 NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)]) + +- LogicalAggregate(group=[{0}], agg#0=[SUM($1)]) + +- LogicalProject(a=[$0], b=[$1]) + +- LogicalTableScan(table=[[x, source: [TestTableSource(a, b, c)]]]) +]]> + + + + + + + + + + + + + + + + + diff --git a/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/rules/physical/batch/RemoveRedundantLocalSortAggRuleTest.xml b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/rules/physical/batch/RemoveRedundantLocalSortAggRuleTest.xml new file mode 100644 index 00000000000..6e791c8a685 --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/test/resources/org/apache/flink/table/plan/rules/physical/batch/RemoveRedundantLocalSortAggRuleTest.xml @@ -0,0 +1,86 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/RemoveCollationTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/RemoveCollationTest.scala new file mode 100644 index 00000000000..60902273fe6 --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/RemoveCollationTest.scala @@ -0,0 +1,384 @@ +/* + * 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.common.typeinfo.TypeInformation +import org.apache.flink.api.scala._ +import org.apache.flink.table.api.{PlannerConfigOptions, TableConfigOptions, Types} +import org.apache.flink.table.plan.stats.TableStats +import org.apache.flink.table.runtime.utils.JavaUserDefinedTableFunctions.StringSplit +import org.apache.flink.table.util.{TableFunc1, TableTestBase} + +import com.google.common.collect.ImmutableSet +import org.junit.{Before, Test} + +class RemoveCollationTest extends TableTestBase { + + private val util = batchTestUtil() + + @Before + def setup(): Unit = { + util.addTableSource("x", + Array[TypeInformation[_]](Types.INT, Types.LONG, Types.STRING), + Array("a", "b", "c"), + tableStats = Some(new TableStats(100L)) + ) + util.addTableSource("y", + Array[TypeInformation[_]](Types.INT, Types.LONG, Types.STRING), + Array("d", "e", "f"), + tableStats = Some(new TableStats(100L)) + ) + util.addTableSource("t1", + Array[TypeInformation[_]](Types.INT, Types.LONG, Types.STRING), + Array("a1", "b1", "c1"), + tableStats = Some(new TableStats(100L)) + ) + util.addTableSource("t2", + Array[TypeInformation[_]](Types.INT, Types.LONG, Types.STRING), + Array("d1", "e1", "f1"), + tableStats = Some(new TableStats(100L)) + ) + + util.tableEnv.getConfig.getConf.setBoolean( + PlannerConfigOptions.SQL_OPTIMIZER_SMJ_REMOVE_SORT_ENABLED, true) + } + + @Test + def testRemoveCollation_OverWindowAgg(): Unit = { + util.tableEnv.getConfig.getConf.setString( + TableConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin,HashAgg") + val sqlQuery = + """ + | SELECT + | SUM(b) sum_b, + | AVG(SUM(b)) OVER (PARTITION BY a order by a) avg_b, + | RANK() OVER (PARTITION BY a ORDER BY a) rn + | FROM x + | GROUP BY a + """.stripMargin + util.verifyPlan(sqlQuery) + } + + @Test + def testRemoveCollation_Aggregate(): Unit = { + util.tableEnv.getConfig.getConf.setString( + TableConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin") + val sqlQuery = + """ + |WITH r AS (SELECT * FROM x, y WHERE a = d AND c LIKE 'He%') + |SELECT sum(b) FROM r group by a + """.stripMargin + util.verifyPlan(sqlQuery) + } + + @Test + def testRemoveCollation_Aggregate_1(): Unit = { + util.tableEnv.getConfig.getConf.setString( + TableConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin") + val sqlQuery = + """ + |WITH r AS (SELECT * FROM x, y WHERE a = d AND c LIKE 'He%') + |SELECT sum(b) FROM r group by d + """.stripMargin + util.verifyPlan(sqlQuery) + } + + @Test + def testRemoveCollation_Sort(): Unit = { + util.tableEnv.getConfig.getConf.setBoolean(TableConfigOptions.SQL_EXEC_SORT_RANGE_ENABLED, true) + val sqlQuery = + """ + |WITH r AS (SELECT a, b, COUNT(c) AS cnt FROM x GROUP BY a, b) + |SELECT * FROM r ORDER BY a + """.stripMargin + util.verifyPlan(sqlQuery) + } + + @Test + def testRemoveCollation_Aggregate_3(): Unit = { + util.tableEnv.getConfig.getConf.setString( + TableConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashAgg") + util.tableEnv.getConfig.getConf.setBoolean(TableConfigOptions.SQL_EXEC_SORT_RANGE_ENABLED, true) + val sqlQuery = + """ + |WITH r AS (SELECT * FROM x ORDER BY a, b) + |SELECT a, b, COUNT(c) AS cnt FROM r GROUP BY a, b + """.stripMargin + util.verifyPlan(sqlQuery) + } + + @Test + def testRemoveCollation_Rank_1(): Unit = { + util.tableEnv.getConfig.getConf.setString( + TableConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashAgg") + val sqlQuery = + """ + |SELECT a, SUM(b) FROM ( + | SELECT * FROM ( + | SELECT a, b, RANK() OVER(PARTITION BY a ORDER BY b) rk FROM x) + | WHERE rk <= 10 + |) GROUP BY a + """.stripMargin + util.verifyPlan(sqlQuery) + } + + @Test + def testRemoveCollation_Rank_2(): Unit = { + util.tableEnv.getConfig.getConf.setString( + TableConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashAgg") + val sqlQuery = + """ + |SELECT a, b, MAX(c) FROM ( + | SELECT * FROM ( + | SELECT a, b, c, RANK() OVER(PARTITION BY a ORDER BY b) rk FROM x) + | WHERE rk <= 10 + |) GROUP BY a, b + """.stripMargin + util.verifyPlan(sqlQuery) + } + + @Test + def testRemoveCollation_Rank_3(): Unit = { + // TODO remove local rank for single distribution input + val sqlQuery = + """ + |SELECT * FROM ( + | SELECT a, b, c, RANK() OVER(PARTITION BY a ORDER BY b) rk FROM ( + | SELECT a, b, c FROM x ORDER BY a, b + | ) + |) WHERE rk <= 10 + """.stripMargin + util.verifyPlan(sqlQuery) + } + + @Test + def testRemoveCollation_Rank_4(): Unit = { + util.tableEnv.getConfig.getConf.setString( + TableConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashAgg") + val sqlQuery = + """ + |SELECT * FROM ( + | SELECT a, c, RANK() OVER(PARTITION BY a ORDER BY a) rk FROM ( + | SELECT a, COUNT(c) AS c FROM x GROUP BY a + | ) + |) WHERE rk <= 10 + """.stripMargin + util.verifyPlan(sqlQuery) + } + + @Test + def testRemoveCollation_Rank_Singleton(): Unit = { + util.tableEnv.getConfig.getConf.setString( + TableConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashAgg") + val sqlQuery = + """ + |SELECT COUNT(a), SUM(b) FROM ( + | SELECT * FROM ( + | SELECT a, b, RANK() OVER(ORDER BY b) rk FROM x) + | WHERE rk <= 10 + |) + """.stripMargin + util.verifyPlan(sqlQuery) + } + + @Test + def testRemoveCollation_MultipleSortMergeJoins1(): Unit = { + util.tableEnv.getConfig.getConf.setString( + TableConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin") + + val sql = + """ + |select * from + | x join y on a = d + | join t1 on a = a1 + | left outer join t2 on a = d1 + """.stripMargin + + util.verifyPlan(sql) + } + + @Test + def testRemoveCollation_MultipleSortMergeJoins_MultiJoinKeys1(): Unit = { + util.tableEnv.getConfig.getConf.setString( + TableConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin") + + val sql = + """ + |select * from + | x join y on a = d and b = e + | join t1 on a = a1 and b = b1 + | left outer join t2 on a = d1 and b = e1 + """.stripMargin + + util.verifyPlan(sql) + } + + @Test + def testRemoveCollation_MultipleSortMergeJoins2(): Unit = { + util.tableEnv.getConfig.getConf.setString( + TableConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin") + + val sql = + """ + |select * from + | x join y on a = d + | join t1 on d = a1 + | left outer join t2 on a1 = d1 + """.stripMargin + + util.verifyPlan(sql) + } + + @Test + def testRemoveCollation_MultipleSortMergeJoins_MultiJoinKeys2(): Unit = { + util.tableEnv.getConfig.getConf.setString( + TableConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin") + + val sql = + """ + |select * from + | x join y on a = d and b = e + | join t1 on d = a1 and e = b1 + | left outer join t2 on a1 = d1 and b1 = e1 + """.stripMargin + + util.verifyPlan(sql) + } + + @Test + def testRemoveCollation_MultipleSortMergeJoins3(): Unit = { + util.tableEnv.getConfig.getConf.setString( + TableConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin") + util.addTableSource("tb1", + Array[TypeInformation[_]]( + Types.STRING, Types.STRING, Types.STRING, Types.STRING, Types.STRING), + Array("id", "key", "tb2_ids", "tb3_ids", "name"), + uniqueKeys = Some(ImmutableSet.of(ImmutableSet.of("id"))) + ) + util.addTableSource("tb2", + Array[TypeInformation[_]](Types.STRING, Types.STRING), + Array("id", "name"), + uniqueKeys = Some(ImmutableSet.of(ImmutableSet.of("id"))) + ) + util.addTableSource("tb3", + Array[TypeInformation[_]](Types.STRING, Types.STRING), + Array("id", "name"), + uniqueKeys = Some(ImmutableSet.of(ImmutableSet.of("id"))) + ) + util.addTableSource("tb4", + Array[TypeInformation[_]](Types.STRING, Types.STRING), + Array("id", "name"), + uniqueKeys = Some(ImmutableSet.of(ImmutableSet.of("id"))) + ) + util.addTableSource("tb5", + Array[TypeInformation[_]](Types.STRING, Types.STRING), + Array("id", "name"), + uniqueKeys = Some(ImmutableSet.of(ImmutableSet.of("id"))) + ) + util.tableEnv.registerFunction("split", new StringSplit()) + + val sql = + """ + |with v1 as ( + | select id, tb2_id from tb1, LATERAL TABLE(split(tb2_ids)) AS T(tb2_id) + |), + |v2 as ( + | select id, tb3_id from tb1, LATERAL TABLE(split(tb3_ids)) AS T(tb3_id) + |), + | + |join_tb2 as ( + | select tb1_id, concat_agg(tb2_name, ',') as tb2_names + | from ( + | select v1.id as tb1_id, tb2.name as tb2_name + | from v1 left outer join tb2 on tb2_id = tb2.id + | ) group by tb1_id + |), + | + |join_tb3 as ( + | select tb1_id, concat_agg(tb3_name, ',') as tb3_names + | from ( + | select v2.id as tb1_id, tb3.name as tb3_name + | from v2 left outer join tb3 on tb3_id = tb3.id + | ) group by tb1_id + |) + | + |select + | tb1.id, + | tb1.tb2_ids, + | tb1.tb3_ids, + | tb1.name, + | tb2_names, + | tb3_names, + | tb4.name, + | tb5.name + | from tb1 + | left outer join join_tb2 on tb1.id = join_tb2.tb1_id + | left outer join join_tb3 on tb1.id = join_tb3.tb1_id + | left outer join tb4 on tb1.key = tb4.id + | left outer join tb5 on tb1.key = tb5.id + """.stripMargin + + util.verifyPlan(sql) + } + + @Test + def testRemoveCollation_Correlate1(): Unit = { + util.tableEnv.getConfig.getConf.setString( + TableConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin,HashAgg") + util.tableEnv.registerFunction("split", new TableFunc1) + val sqlQuery = + """ + |WITH r AS (SELECT f, count(f) as cnt FROM y GROUP BY f), + | v as (SELECT f1, f, cnt FROM r, LATERAL TABLE(split(f)) AS T(f1)) + |SELECT * FROM x, v WHERE c = f + """.stripMargin + util.verifyPlan(sqlQuery) + } + + @Test + def testRemoveCollation_Correlate2(): Unit = { + util.tableEnv.getConfig.getConf.setString( + TableConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin,HashAgg") + util.tableEnv.registerFunction("split", new TableFunc1) + val sqlQuery = + """ + |WITH r AS (SELECT f, count(f) as cnt FROM y GROUP BY f), + | v as (SELECT f, f1 FROM r, LATERAL TABLE(split(f)) AS T(f1)) + |SELECT * FROM x, v WHERE c = f AND f LIKE '%llo%' + """.stripMargin + util.verifyPlan(sqlQuery) + } + + @Test + def testRemoveCollation_Correlate3(): Unit = { + // do not remove shuffle + util.tableEnv.getConfig.getConf.setString( + TableConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin,HashAgg") + util.tableEnv.registerFunction("split", new TableFunc1) + val sqlQuery = + """ + |WITH r AS (SELECT f, count(f) as cnt FROM y GROUP BY f), + | v as (SELECT f1 FROM r, LATERAL TABLE(split(f)) AS T(f1)) + |SELECT * FROM x, v WHERE c = f1 + """.stripMargin + util.verifyPlan(sqlQuery) + } + +} + diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/RemoveShuffleTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/RemoveShuffleTest.scala new file mode 100644 index 00000000000..29b14b9813c --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/RemoveShuffleTest.scala @@ -0,0 +1,547 @@ +/* + * 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.common.typeinfo.TypeInformation +import org.apache.flink.api.scala._ +import org.apache.flink.table.api.{PlannerConfigOptions, TableConfigOptions, Types} +import org.apache.flink.table.plan.stats.TableStats +import org.apache.flink.table.util.{TableFunc1, TableTestBase} + +import org.junit.{Before, Test} + +class RemoveShuffleTest extends TableTestBase { + + private val util = batchTestUtil() + + @Before + def setup(): Unit = { + util.addTableSource("x", + Array[TypeInformation[_]](Types.INT, Types.LONG, Types.STRING), + Array("a", "b", "c"), + tableStats = Some(new TableStats(100L)) + ) + util.addTableSource("y", + Array[TypeInformation[_]](Types.INT, Types.LONG, Types.STRING), + Array("d", "e", "f"), + tableStats = Some(new TableStats(100L)) + ) + util.tableEnv.getConfig.getConf.setBoolean( + PlannerConfigOptions.SQL_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, false) + } + + @Test + def testRemoveHashShuffle_OverWindowAgg(): Unit = { + util.tableEnv.getConfig.getConf.setString( + TableConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin,SortAgg") + val sqlQuery = + """ + | SELECT + | SUM(b) sum_b, + | AVG(SUM(b)) OVER (PARTITION BY c) avg_b, + | RANK() OVER (PARTITION BY c ORDER BY c) rn, + | c + | FROM x + | GROUP BY c + """.stripMargin + util.verifyPlan(sqlQuery) + } + + @Test + def testRemoveHashShuffle_MultiOverWindowAgg(): Unit = { + util.tableEnv.getConfig.getConf.setString( + TableConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin,SortAgg") + val sqlQuery = + """ + | SELECT + | SUM(b) sum_b, + | AVG(SUM(b)) OVER (PARTITION BY a, c) avg_b, + | RANK() OVER (PARTITION BY c ORDER BY a, c) rn, + | c + | FROM x + | GROUP BY a, c + """.stripMargin + util.verifyPlan(sqlQuery) + } + + @Test + def testRemoveHashShuffle_OverWindowAgg_PartialKey(): Unit = { + util.tableEnv.getConfig.getConf.setString( + TableConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin,SortAgg") + util.tableEnv.getConfig.getConf.setBoolean( + PlannerConfigOptions.SQL_OPTIMIZER_SHUFFLE_PARTIAL_KEY_ENABLED, true) + // push down HashExchange[c] into HashAgg + val sqlQuery = + """ + | SELECT + | SUM(b) sum_b, + | AVG(SUM(b)) OVER (PARTITION BY c) avg_b, + | RANK() OVER (PARTITION BY c ORDER BY c) rn, + | c + | FROM x + | GROUP BY a, c + """.stripMargin + util.verifyPlan(sqlQuery) + } + + @Test + def testRemoveHashShuffle_Agg_PartialKey(): Unit = { + util.tableEnv.getConfig.getConf.setString( + TableConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin,SortAgg") + util.tableEnv.getConfig.getConf.setBoolean( + PlannerConfigOptions.SQL_OPTIMIZER_SHUFFLE_PARTIAL_KEY_ENABLED, true) + // push down HashExchange[c] into HashAgg + val sqlQuery = + """ + | WITH r AS (SELECT a, c, count(b) as cnt FROM x GROUP BY a, c) + | SELECT count(cnt) FROM r group by c + """.stripMargin + util.verifyPlan(sqlQuery) + } + + @Test + def testRemoveHashShuffle_HashAggregate(): Unit = { + util.tableEnv.getConfig.getConf.setString( + TableConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "SortMergeJoin,NestedLoopJoin,SortAgg") + // disable BroadcastHashJoin + util.tableEnv.getConfig.getConf.setLong( + PlannerConfigOptions.SQL_OPTIMIZER_HASH_JOIN_BROADCAST_THRESHOLD, -1) + val sqlQuery = + """ + |WITH r AS (SELECT * FROM x, y WHERE a = d AND c LIKE 'He%') + |SELECT sum(b) FROM r group by a + """.stripMargin + util.verifyPlan(sqlQuery) + } + + @Test + def testRemoveHashShuffle_HashAggregate_1(): Unit = { + util.tableEnv.getConfig.getConf.setString( + TableConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "SortMergeJoin,NestedLoopJoin,SortAgg") + // disable BroadcastHashJoin + util.tableEnv.getConfig.getConf.setLong( + PlannerConfigOptions.SQL_OPTIMIZER_HASH_JOIN_BROADCAST_THRESHOLD, -1) + val sqlQuery = + """ + |WITH r AS (SELECT * FROM x, y WHERE a = d AND c LIKE 'He%') + |SELECT sum(b) FROM r group by a, d + """.stripMargin + util.verifyPlan(sqlQuery) + } + + @Test + def testRemoveHashShuffle_HashAggregate_2(): Unit = { + util.tableEnv.getConfig.getConf.setString( + TableConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "SortMergeJoin,NestedLoopJoin,SortAgg") + // disable BroadcastHashJoin + util.tableEnv.getConfig.getConf.setLong( + PlannerConfigOptions.SQL_OPTIMIZER_HASH_JOIN_BROADCAST_THRESHOLD, -1) + val sqlQuery = + """ + |WITH r AS (SELECT * FROM x, y WHERE a = d AND c LIKE 'He%') + |SELECT sum(b) FROM r group by d + """.stripMargin + util.verifyPlan(sqlQuery) + } + + @Test + def testRemoveHashShuffle_SortAggregate(): Unit = { + util.tableEnv.getConfig.getConf.setString( + TableConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "SortMergeJoin,NestedLoopJoin,HashAgg") + // disable BroadcastHashJoin + util.tableEnv.getConfig.getConf.setLong( + PlannerConfigOptions.SQL_OPTIMIZER_HASH_JOIN_BROADCAST_THRESHOLD, -1) + val sqlQuery = + """ + |WITH r AS (SELECT * FROM x, y WHERE a = d AND c LIKE 'He%') + |SELECT sum(b) FROM r group by a + """.stripMargin + util.verifyPlan(sqlQuery) + } + + @Test + def testRemoveHashShuffle_SortAggregate_1(): Unit = { + util.tableEnv.getConfig.getConf.setString( + TableConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "SortMergeJoin,NestedLoopJoin,HashAgg") + // disable BroadcastHashJoin + util.tableEnv.getConfig.getConf.setLong( + PlannerConfigOptions.SQL_OPTIMIZER_HASH_JOIN_BROADCAST_THRESHOLD, -1) + val sqlQuery = + """ + |WITH r AS (SELECT * FROM x, y WHERE a = d AND c LIKE 'He%') + |SELECT sum(b) FROM r group by a, d + """.stripMargin + util.verifyPlan(sqlQuery) + } + + @Test + def testRemoveHashShuffle_SortAggregate_2(): Unit = { + util.tableEnv.getConfig.getConf.setString( + TableConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "SortMergeJoin,NestedLoopJoin,HashAgg") + // disable BroadcastHashJoin + util.tableEnv.getConfig.getConf.setLong( + PlannerConfigOptions.SQL_OPTIMIZER_HASH_JOIN_BROADCAST_THRESHOLD, -1) + val sqlQuery = + """ + |WITH r AS (SELECT * FROM x, y WHERE a = d AND c LIKE 'He%') + |SELECT sum(b) FROM r group by d + """.stripMargin + util.verifyPlan(sqlQuery) + } + + @Test + def testRemoveHashShuffle_SortMergeJoin(): Unit = { + util.tableEnv.getConfig.getConf.setString( + TableConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin") + util.tableEnv.getConfig.getConf.setBoolean( + PlannerConfigOptions.SQL_OPTIMIZER_SMJ_REMOVE_SORT_ENABLED, true) + val sqlQuery = + """ + |WITH r AS (SELECT * FROM x, y WHERE a = d AND c LIKE 'He%') + |SELECT * FROM r r1, r r2 WHERE r1.a = r2.d + """.stripMargin + util.verifyPlan(sqlQuery) + } + + @Test + def testRemoveHashShuffle_SortMergeJoin_LOJ(): Unit = { + util.tableEnv.getConfig.getConf.setString( + TableConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin") + util.tableEnv.getConfig.getConf.setBoolean( + PlannerConfigOptions.SQL_OPTIMIZER_SMJ_REMOVE_SORT_ENABLED, true) + val sqlQuery = + """ + |WITH r AS (SELECT * FROM x left join (SELECT * FROM y WHERE e = 2) r on a = d) + |SELECT * FROM r r1, r r2 WHERE r1.a = r2.d + """.stripMargin + util.verifyPlan(sqlQuery) + } + + @Test + def testRemoveHashShuffle_SortMergeJoin_ROJ(): Unit = { + util.tableEnv.getConfig.getConf.setString( + TableConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin") + util.tableEnv.getConfig.getConf.setBoolean( + PlannerConfigOptions.SQL_OPTIMIZER_SMJ_REMOVE_SORT_ENABLED, true) + val sqlQuery = + """ + |WITH r AS (SELECT * FROM x right join (SELECT * FROM y WHERE e = 2) r on a = d) + |SELECT * FROM r r1, r r2 WHERE r1.a = r2.d + """.stripMargin + util.verifyPlan(sqlQuery) + } + + @Test + def testRemoveHashShuffle_SortMergeJoin_FOJ(): Unit = { + util.tableEnv.getConfig.getConf.setString( + TableConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin") + val sqlQuery = + """ + |WITH r AS (SELECT * FROM x full join (SELECT * FROM y WHERE e = 2) r on a = d) + |SELECT * FROM r r1, r r2 WHERE r1.a = r2.d + """.stripMargin + util.verifyPlan(sqlQuery) + } + + @Test + def testRemoveHashShuffle_HashJoin(): Unit = { + util.tableEnv.getConfig.getConf.setString( + TableConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin") + // disable BroadcastHashJoin + util.tableEnv.getConfig.getConf.setLong( + PlannerConfigOptions.SQL_OPTIMIZER_HASH_JOIN_BROADCAST_THRESHOLD, -1) + val sqlQuery = + """ + |WITH r AS (SELECT * FROM x, y WHERE a = d AND c LIKE 'He%') + |SELECT * FROM r r1, r r2 WHERE r1.a = r2.d + """.stripMargin + util.verifyPlan(sqlQuery) + } + + @Test + def testRemoveHashShuffle_BroadcastHashJoin(): Unit = { + util.tableEnv.getConfig.getConf.setString( + TableConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin") + val sqlQuery = + """ + |WITH r AS (SELECT * FROM x, y WHERE a = d AND c LIKE 'He%') + |SELECT * FROM r r1, r r2 WHERE r1.a = r2.d + """.stripMargin + util.verifyPlan(sqlQuery) + } + + @Test + def testRemoveHashShuffle_HashJoin_LOJ(): Unit = { + util.tableEnv.getConfig.getConf.setString( + TableConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin") + // disable BroadcastHashJoin + util.tableEnv.getConfig.getConf.setLong( + PlannerConfigOptions.SQL_OPTIMIZER_HASH_JOIN_BROADCAST_THRESHOLD, -1) + val sqlQuery = + """ + |WITH r AS (SELECT * FROM x left join (SELECT * FROM y WHERE e = 2) r on a = d) + |SELECT * FROM r r1, r r2 WHERE r1.a = r2.d + """.stripMargin + util.verifyPlan(sqlQuery) + } + + @Test + def testRemoveHashShuffle_HashJoin_ROJ(): Unit = { + util.tableEnv.getConfig.getConf.setString( + TableConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin") + // disable BroadcastHashJoin + util.tableEnv.getConfig.getConf.setLong( + PlannerConfigOptions.SQL_OPTIMIZER_HASH_JOIN_BROADCAST_THRESHOLD, -1) + val sqlQuery = + """ + |WITH r AS (SELECT * FROM x right join (SELECT * FROM y WHERE e = 2) r on a = d) + |SELECT * FROM r r1, r r2 WHERE r1.a = r2.d + """.stripMargin + util.verifyPlan(sqlQuery) + } + + @Test + def testRemoveHashShuffle_HashJoin_FOJ(): Unit = { + util.tableEnv.getConfig.getConf.setString( + TableConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin") + // disable BroadcastHashJoin + util.tableEnv.getConfig.getConf.setLong( + PlannerConfigOptions.SQL_OPTIMIZER_HASH_JOIN_BROADCAST_THRESHOLD, -1) + val sqlQuery = + """ + |WITH r AS (SELECT * FROM x full join (SELECT * FROM y WHERE e = 2) r on a = d) + |SELECT * FROM r r1, r r2 WHERE r1.a = r2.d + """.stripMargin + util.verifyPlan(sqlQuery) + } + + @Test + def testRemoveHashShuffle_HashJoin_1(): Unit = { + util.tableEnv.getConfig.getConf.setString( + TableConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin") + // disable BroadcastHashJoin + util.tableEnv.getConfig.getConf.setLong( + PlannerConfigOptions.SQL_OPTIMIZER_HASH_JOIN_BROADCAST_THRESHOLD, -1) + val sqlQuery = + """ + |WITH r1 AS (SELECT a, c, sum(b) FROM x group by a, c), + |r2 AS (SELECT a, c, sum(b) FROM x group by a, c) + |SELECT * FROM r1, r2 WHERE r1.a = r2.a and r1.c = r2.c + """.stripMargin + util.verifyPlan(sqlQuery) + } + + @Test + def testRemoveHashShuffle_NestedLoopJoin(): Unit = { + util.tableEnv.getConfig.getConf.setString( + TableConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin,SortMergeJoin") + val sqlQuery = + """ + |WITH r AS (SELECT * FROM x, y WHERE a = d AND c LIKE 'He%') + |SELECT * FROM r r1, r r2 WHERE r1.a = r2.d + """.stripMargin + util.verifyPlan(sqlQuery) + } + + @Test + def testRemoveHashShuffle_Join_PartialKey(): Unit = { + util.tableEnv.getConfig.getConf.setString( + TableConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "SortMergeJoin,NestedLoopJoin,SortAgg") + // disable BroadcastHashJoin + util.tableEnv.getConfig.getConf.setLong( + PlannerConfigOptions.SQL_OPTIMIZER_HASH_JOIN_BROADCAST_THRESHOLD, -1) + util.tableEnv.getConfig.getConf.setBoolean( + PlannerConfigOptions.SQL_OPTIMIZER_SHUFFLE_PARTIAL_KEY_ENABLED, true) + val sqlQuery = + """ + |WITH r AS (SELECT d, count(f) as cnt FROM y GROUP BY d) + |SELECT * FROM x, r WHERE x.a = r.d AND x.b = r.cnt + """.stripMargin + util.verifyPlan(sqlQuery) + } + + @Test + def testRemoveSingleExchange_Agg(): Unit = { + val sqlQuery = "SELECT avg(b) FROM x GROUP BY c HAVING sum(b) > (SELECT sum(b) * 0.1 FROM x)" + util.verifyPlan(sqlQuery) + } + + @Test + def testRemoveHashShuffle_Union(): Unit = { + util.tableEnv.getConfig.getConf.setString( + TableConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin,SortAgg") + val sqlQuery = + """ + |WITH r AS ( + |SELECT count(a) as cnt, c FROM x WHERE b > 10 group by c + |UNION ALL + |SELECT count(d) as cnt, f FROM y WHERE e < 100 group by f) + |SELECT r1.c, r1.cnt, r2.c, r2.cnt FROM r r1, r r2 WHERE r1.c = r2.c and r1.cnt < 10 + """.stripMargin + util.verifyPlan(sqlQuery) + } + + @Test + def testRemoveHashShuffle_Rank(): Unit = { + util.tableEnv.getConfig.getConf.setString( + TableConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "SortAgg") + val sqlQuery = + """ + |SELECT * FROM ( + | SELECT a, b, RANK() OVER(PARTITION BY a ORDER BY b) rk FROM ( + | SELECT a, SUM(b) AS b FROM x GROUP BY a + | ) + |) WHERE rk <= 10 + """.stripMargin + util.verifyPlan(sqlQuery) + } + + @Test + def testRemoveHashShuffle_Rank_PartialKey1(): Unit = { + util.tableEnv.getConfig.getConf.setString( + TableConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "SortAgg") + util.tableEnv.getConfig.getConf.setBoolean( + PlannerConfigOptions.SQL_OPTIMIZER_SHUFFLE_PARTIAL_KEY_ENABLED, true) + val sqlQuery = + """ + |SELECT a, SUM(b) FROM ( + | SELECT * FROM ( + | SELECT a, b, c, RANK() OVER(PARTITION BY a, c ORDER BY b) rk FROM x) + | WHERE rk <= 10 + |) GROUP BY a + """.stripMargin + util.verifyPlan(sqlQuery) + } + + @Test + def testRemoveHashShuffle_Rank_PartialKey2(): Unit = { + util.tableEnv.getConfig.getConf.setString( + TableConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "SortAgg") + util.tableEnv.getConfig.getConf.setBoolean( + PlannerConfigOptions.SQL_OPTIMIZER_SHUFFLE_PARTIAL_KEY_ENABLED, false) + val sqlQuery = + """ + |SELECT * FROM ( + | SELECT a, b, c, RANK() OVER(PARTITION BY a, c ORDER BY b) rk FROM ( + | SELECT a, SUM(b) AS b, COUNT(c) AS c FROM x GROUP BY a + | ) + |) WHERE rk <= 10 + """.stripMargin + util.verifyPlan(sqlQuery) + } + + @Test + def testRemoveHashShuffle_Rank_PartialKey3(): Unit = { + util.tableEnv.getConfig.getConf.setString( + TableConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "SortAgg") + util.tableEnv.getConfig.getConf.setBoolean( + PlannerConfigOptions.SQL_OPTIMIZER_SHUFFLE_PARTIAL_KEY_ENABLED, true) + val sqlQuery = + """ + |SELECT * FROM ( + | SELECT a, b, c, RANK() OVER(PARTITION BY a, c ORDER BY b) rk FROM ( + | SELECT a, SUM(b) AS b, COUNT(c) AS c FROM x GROUP BY a + | ) + |) WHERE rk <= 10 + """.stripMargin + util.verifyPlan(sqlQuery) + } + + @Test + def testRemoveHashShuffle_Rank_Singleton1(): Unit = { + util.tableEnv.getConfig.getConf.setString( + TableConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "SortAgg") + val sqlQuery = + """ + |SELECT * FROM ( + | SELECT a, b, RANK() OVER(ORDER BY b) rk FROM ( + | SELECT COUNT(a) AS a, SUM(b) AS b FROM x + | ) + |) WHERE rk <= 10 + """.stripMargin + util.verifyPlan(sqlQuery) + } + + @Test + def testRemoveHashShuffle_Rank_Singleton2(): Unit = { + util.tableEnv.getConfig.getConf.setString( + TableConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "SortAgg") + val sqlQuery = + """ + |SELECT * FROM ( + | SELECT a, b, RANK() OVER(PARTITION BY a ORDER BY b) rk FROM ( + | SELECT COUNT(a) AS a, SUM(b) AS b FROM x + | ) + |) WHERE rk <= 10 + """.stripMargin + util.verifyPlan(sqlQuery) + } + + @Test + def testRemoveHashShuffle_Correlate1(): Unit = { + util.tableEnv.getConfig.getConf.setString( + TableConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "SortMergeJoin,NestedLoopJoin,SortAgg") + // disable BroadcastHashJoin + util.tableEnv.getConfig.getConf.setLong( + PlannerConfigOptions.SQL_OPTIMIZER_HASH_JOIN_BROADCAST_THRESHOLD, -1) + util.tableEnv.registerFunction("split", new TableFunc1) + val sqlQuery = + """ + |WITH r AS (SELECT f, count(f) as cnt FROM y GROUP BY f), + | v as (SELECT f1, f, cnt FROM r, LATERAL TABLE(split(f)) AS T(f1)) + |SELECT * FROM x, v WHERE c = f + """.stripMargin + util.verifyPlan(sqlQuery) + } + + @Test + def testRemoveHashShuffle_Correlate2(): Unit = { + util.tableEnv.getConfig.getConf.setString( + TableConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "SortMergeJoin,NestedLoopJoin,SortAgg") + // disable BroadcastHashJoin + util.tableEnv.getConfig.getConf.setLong( + PlannerConfigOptions.SQL_OPTIMIZER_HASH_JOIN_BROADCAST_THRESHOLD, -1) + util.tableEnv.registerFunction("split", new TableFunc1) + val sqlQuery = + """ + |WITH r AS (SELECT f, count(f) as cnt FROM y GROUP BY f), + | v as (SELECT f, f1 FROM r, LATERAL TABLE(split(f)) AS T(f1)) + |SELECT * FROM x, v WHERE c = f AND f LIKE '%llo%' + """.stripMargin + util.verifyPlan(sqlQuery) + } + + @Test + def testRemoveHashShuffle_Correlate3(): Unit = { + // do not remove shuffle + util.tableEnv.getConfig.getConf.setString( + TableConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "SortMergeJoin,NestedLoopJoin,SortAgg") + // disable BroadcastHashJoin + util.tableEnv.getConfig.getConf.setLong( + PlannerConfigOptions.SQL_OPTIMIZER_HASH_JOIN_BROADCAST_THRESHOLD, -1) + util.tableEnv.registerFunction("split", new TableFunc1) + val sqlQuery = + """ + |WITH r AS (SELECT f, count(f) as cnt FROM y GROUP BY f), + | v as (SELECT f1 FROM r, LATERAL TABLE(split(f)) AS T(f1)) + |SELECT * FROM x, v WHERE c = f1 + """.stripMargin + util.verifyPlan(sqlQuery) + } +} diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/agg/OverAggregateTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/agg/OverAggregateTest.scala index d7d45596eec..0a8dbbeea58 100644 --- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/agg/OverAggregateTest.scala +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/agg/OverAggregateTest.scala @@ -61,7 +61,22 @@ class OverAggregateTest extends TableTestBase { } @Test - def testDiffPartitionKeysWithDiffOrderKeys(): Unit = { + def testDiffPartitionKeysWithDiffOrderKeys1(): Unit = { + val sqlQuery = + """ + |SELECT + | SUM(a) OVER (PARTITION BY b ORDER BY a), + | MAX(a) OVER (PARTITION BY b ORDER BY c), + | AVG(a) OVER (PARTITION BY c ORDER BY a), + | RANK() OVER (PARTITION BY b ORDER BY a), + | MIN(a) OVER (PARTITION BY c ORDER BY a) + |FROM MyTable + """.stripMargin + util.verifyPlan(sqlQuery) + } + + @Test + def testDiffPartitionKeysWithDiffOrderKeys2(): Unit = { val sqlQuery = """ |SELECT @@ -76,7 +91,22 @@ class OverAggregateTest extends TableTestBase { } @Test - def testSamePartitionKeysWithDiffOrderKeys(): Unit = { + def testSamePartitionKeysWithDiffOrderKeys1(): Unit = { + val sqlQuery = + """ + |SELECT + | SUM(a) OVER (PARTITION BY b ORDER BY c), + | MAX(a) OVER (PARTITION BY b ORDER BY b), + | AVG(a) OVER (PARTITION BY b ORDER BY a), + | RANK() OVER (PARTITION BY b ORDER BY c), + | MIN(a) OVER (PARTITION BY b ORDER BY b) + |FROM MyTable + """.stripMargin + util.verifyPlan(sqlQuery) + } + + @Test + def testSamePartitionKeysWithDiffOrderKeys2(): Unit = { val sqlQuery = """ |SELECT @@ -116,7 +146,22 @@ class OverAggregateTest extends TableTestBase { } @Test - def testSamePartitionKeysWithSameOrderKeysDiffDirection(): Unit = { + def testSamePartitionKeysWithSameOrderKeysDiffDirection1(): Unit = { + val sqlQuery = + """ + |SELECT + | SUM(a) OVER (PARTITION BY b ORDER BY a ASC), + | MAX(a) OVER (PARTITION BY b ORDER BY a ASC), + | AVG(a) OVER (PARTITION BY b ORDER BY a DESC), + | RANK() OVER (PARTITION BY b ORDER BY a ASC), + | MIN(a) OVER (PARTITION BY b ORDER BY a DESC) + |FROM MyTable + """.stripMargin + util.verifyPlan(sqlQuery) + } + + @Test + def testSamePartitionKeysWithSameOrderKeysDiffDirection2(): Unit = { val sqlQuery = """ |SELECT diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/WindowGroupReorderRuleTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/WindowGroupReorderRuleTest.scala new file mode 100644 index 00000000000..723b0720756 --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/WindowGroupReorderRuleTest.scala @@ -0,0 +1,179 @@ +/* + * 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.logical + +import org.apache.flink.api.scala._ +import org.apache.flink.table.plan.optimize.program._ +import org.apache.flink.table.util.TableTestBase + +import org.junit.{Before, Test} + +/** + * Test for [[WindowGroupReorderRule]]. + */ +class WindowGroupReorderRuleTest extends TableTestBase { + private val util = batchTestUtil() + + @Before + def setup(): Unit = { + util.buildBatchProgram(FlinkBatchProgram.LOGICAL) + util.addTableSource[(Int, Int, String)]("MyTable", 'a, 'b, 'c) + } + + @Test + def testSamePartitionKeysWithSameOrderKeysPrefix(): Unit = { + val sqlQuery = + """ + |SELECT a, + | RANK() OVER (PARTITION BY b ORDER BY c, a DESC), + | RANK() OVER (PARTITION BY b ORDER BY c, b) + |FROM MyTable + """.stripMargin + util.verifyPlan(sqlQuery) + } + + @Test + def testSamePartitionKeysWithDiffOrderKeys1(): Unit = { + val sqlQuery = + """ + |SELECT + | SUM(a) OVER (PARTITION BY b ORDER BY c), + | MAX(a) OVER (PARTITION BY b ORDER BY b), + | AVG(a) OVER (PARTITION BY b ORDER BY a), + | RANK() OVER (PARTITION BY b ORDER BY c), + | MIN(a) OVER (PARTITION BY b ORDER BY b) + |FROM MyTable + """.stripMargin + util.verifyPlan(sqlQuery) + } + + @Test + def testSamePartitionKeysWithDiffOrderKeys2(): Unit = { + val sqlQuery = + """ + |SELECT + | SUM(a) OVER (PARTITION BY b ORDER BY c), + | MAX(a) OVER (PARTITION BY b ORDER BY a), + | AVG(a) OVER (PARTITION BY b ORDER BY a, c), + | RANK() OVER (PARTITION BY b ORDER BY a, b), + | MIN(a) OVER (PARTITION BY b ORDER BY b) + |FROM MyTable + """.stripMargin + util.verifyPlan(sqlQuery) + } + + @Test + def testSamePartitionKeysWithSameOrderKeysDiffDirection1(): Unit = { + val sqlQuery = + """ + |SELECT + | SUM(a) OVER (PARTITION BY b ORDER BY a ASC), + | MAX(a) OVER (PARTITION BY b ORDER BY a ASC), + | AVG(a) OVER (PARTITION BY b ORDER BY a DESC), + | RANK() OVER (PARTITION BY b ORDER BY a ASC), + | MIN(a) OVER (PARTITION BY b ORDER BY a DESC) + |FROM MyTable + """.stripMargin + util.verifyPlan(sqlQuery) + } + + @Test + def testSamePartitionKeysWithSameOrderKeysDiffDirection2(): Unit = { + val sqlQuery = + """ + |SELECT + | RANK() OVER (PARTITION BY b ORDER BY a DESC), + | RANK() OVER (PARTITION BY b ORDER BY a ASC) + |FROM MyTable + """.stripMargin + util.verifyPlan(sqlQuery) + } + + @Test + def testSamePartitionKeysWithSameOrderKeysWithEmptyOrder(): Unit = { + val sqlQuery = + """ + |SELECT + | SUM(a) OVER (PARTITION BY b ORDER BY a), + | MIN(a) OVER (PARTITION BY b), + | MAX(a) OVER (PARTITION BY b ORDER BY a) + |FROM MyTable + """.stripMargin + util.verifyPlan(sqlQuery) + } + + @Test + def testDiffPartitionKeysWithSameOrderKeys(): Unit = { + val sqlQuery = + """ + |SELECT + | SUM(a) OVER (PARTITION BY b ORDER BY a), + | MAX(a) OVER (PARTITION BY b ORDER BY a), + | AVG(a) OVER (PARTITION BY c ORDER BY a), + | RANK() OVER (PARTITION BY b ORDER BY a), + | MIN(a) OVER (PARTITION BY c ORDER BY a) + |FROM MyTable + """.stripMargin + util.verifyPlan(sqlQuery) + } + + @Test + def testDiffPartitionKeysWithDiffOrderKeys1(): Unit = { + val sqlQuery = + """ + |SELECT + | SUM(a) OVER (PARTITION BY b ORDER BY a), + | MAX(a) OVER (PARTITION BY b ORDER BY c), + | AVG(a) OVER (PARTITION BY c ORDER BY a), + | RANK() OVER (PARTITION BY b ORDER BY a), + | MIN(a) OVER (PARTITION BY c ORDER BY a) + |FROM MyTable + """.stripMargin + util.verifyPlan(sqlQuery) + } + + @Test + def testDiffPartitionKeysWithDiffOrderKeys2(): Unit = { + val sqlQuery = + """ + |SELECT + | SUM(a) OVER (PARTITION BY b ORDER BY c), + | MAX(a) OVER (PARTITION BY c ORDER BY a), + | MIN(a) OVER (ORDER BY c, a), + | RANK() OVER (PARTITION BY b ORDER BY c), + | AVG(a) OVER (ORDER BY b) + |FROM MyTable + """.stripMargin + util.verifyPlan(sqlQuery) + } + + @Test + def testMultiOverWindowRangeType(): Unit = { + val sqlQuery = + """ + |SELECT + | COUNT(*) OVER (PARTITION BY c ORDER BY a RANGE BETWEEN -1 PRECEDING AND 10 FOLLOWING), + | SUM(a) OVER (PARTITION BY c ORDER BY a), + | RANK() OVER (PARTITION BY c ORDER BY a, c), + | SUM(a) OVER (PARTITION BY c ORDER BY a RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING), + | COUNT(*) OVER (PARTITION BY c ORDER BY c ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING) + | FROM MyTable + """.stripMargin + util.verifyPlan(sqlQuery) + } +} diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/physical/batch/RemoveRedundantLocalHashAggRuleTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/physical/batch/RemoveRedundantLocalHashAggRuleTest.scala new file mode 100644 index 00000000000..acf9972e538 --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/physical/batch/RemoveRedundantLocalHashAggRuleTest.scala @@ -0,0 +1,71 @@ +/* + * 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.api.scala._ +import org.apache.flink.table.api.{PlannerConfigOptions, TableConfigOptions} +import org.apache.flink.table.util.TableTestBase + +import org.junit.{Before, Test} + +/** + * Test for [[RemoveRedundantLocalHashAggRule]]. + */ +class RemoveRedundantLocalHashAggRuleTest extends TableTestBase { + + private val util = batchTestUtil() + + @Before + def setup(): Unit = { + util.addTableSource[(Int, Long, String)]("x", 'a, 'b, 'c) + util.addTableSource[(Int, Long, String)]("y", 'd, 'e, 'f) + } + + @Test + def testRemoveRedundantLocalHashAgg_ShuffleKeyFromJoin(): Unit = { + util.tableEnv.getConfig.getConf.setString( + TableConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "SortMergeJoin,NestedLoopJoin,SortAgg") + // disable BroadcastHashJoin + util.tableEnv.getConfig.getConf.setLong( + PlannerConfigOptions.SQL_OPTIMIZER_HASH_JOIN_BROADCAST_THRESHOLD, -1) + val sqlQuery = + """ + |WITH r AS (SELECT * FROM x, y WHERE a = d AND c LIKE 'He%') + |SELECT SUM(b) FROM r GROUP BY a + """.stripMargin + util.verifyPlan(sqlQuery) + } + + @Test + def testRemoveRedundantLocalHashAgg_ShuffleKeyFromRank(): Unit = { + util.tableEnv.getConfig.getConf.setString( + TableConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "SortAgg") + util.tableEnv.getConfig.getConf.setBoolean( + PlannerConfigOptions.SQL_OPTIMIZER_SHUFFLE_PARTIAL_KEY_ENABLED, true) + val sqlQuery = + """ + |SELECT a, SUM(b) FROM ( + | SELECT * FROM ( + | SELECT a, b, c, RANK() OVER (PARTITION BY a, c ORDER BY b) rk FROM x) + | WHERE rk <= 10 + |) GROUP BY a + """.stripMargin + util.verifyPlan(sqlQuery) + } + +} diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/physical/batch/RemoveRedundantLocalRankRuleTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/physical/batch/RemoveRedundantLocalRankRuleTest.scala new file mode 100644 index 00000000000..35e50ad3774 --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/physical/batch/RemoveRedundantLocalRankRuleTest.scala @@ -0,0 +1,72 @@ +/* + * 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.api.scala._ +import org.apache.flink.table.util.TableTestBase + +import org.junit.{Before, Test} + +/** + * Tests for [[RemoveRedundantLocalRankRule]]. + */ +class RemoveRedundantLocalRankRuleTest extends TableTestBase { + + private val util = batchTestUtil() + + @Before + def setup(): Unit = { + util.addTableSource[(Int, Long, String)]("x", 'a, 'b, 'c) + } + + @Test + def testSameRankRange(): Unit = { + val sqlQuery = + """ + |SELECT a FROM ( + | SELECT a, RANK() OVER(PARTITION BY a ORDER BY SUM(b)) rk FROM x GROUP BY a + |) WHERE rk <= 5 + """.stripMargin + util.verifyPlan(sqlQuery) + } + + @Test + def testDiffRankRange(): Unit = { + val sqlQuery = + """ + |SELECT a FROM ( + | SELECT a, RANK() OVER(PARTITION BY a ORDER BY SUM(b)) rk FROM x GROUP BY a + |) WHERE rk <= 5 and rk >= 2 + """.stripMargin + util.verifyPlan(sqlQuery) + } + + @Test + def testMultiRanks(): Unit = { + val sqlQuery = + """ + |SELECT * FROM ( + | SELECT a, b, rk, RANK() OVER(PARTITION BY a ORDER BY b) rk1 FROM ( + | SELECT a, b, RANK() OVER(PARTITION BY a ORDER BY b) rk FROM x + | ) WHERE rk <= 5 + |) WHERE rk1 <= 5 + """.stripMargin + util.verifyPlan(sqlQuery) + } + +} diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/physical/batch/RemoveRedundantLocalSortAggRuleTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/physical/batch/RemoveRedundantLocalSortAggRuleTest.scala new file mode 100644 index 00000000000..36c05abe8d4 --- /dev/null +++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/physical/batch/RemoveRedundantLocalSortAggRuleTest.scala @@ -0,0 +1,66 @@ +/* + * 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.api.scala._ +import org.apache.flink.table.api.{PlannerConfigOptions, TableConfigOptions} +import org.apache.flink.table.util.TableTestBase + +import org.junit.{Before, Test} + +/** + * Test for [[RemoveRedundantLocalSortAggRule]]. + */ +class RemoveRedundantLocalSortAggRuleTest extends TableTestBase { + + private val util = batchTestUtil() + + @Before + def setup(): Unit = { + util.addTableSource[(Int, Long, String)]("x", 'a, 'b, 'c) + util.addTableSource[(Int, Long, String)]("y", 'd, 'e, 'f) + } + + @Test + def testRemoveRedundantLocalSortAggWithSort(): Unit = { + util.tableEnv.getConfig.getConf.setString( + TableConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "SortMergeJoin,NestedLoopJoin,HashAgg") + // disable BroadcastHashJoin + util.tableEnv.getConfig.getConf.setLong( + PlannerConfigOptions.SQL_OPTIMIZER_HASH_JOIN_BROADCAST_THRESHOLD, -1) + val sqlQuery = + """ + |WITH r AS (SELECT * FROM x, y WHERE a = d AND c LIKE 'He%') + |SELECT sum(b) FROM r group by a + """.stripMargin + util.verifyPlan(sqlQuery) + } + + @Test + def testRemoveRedundantLocalSortAggWithoutSort(): Unit = { + util.tableEnv.getConfig.getConf.setString( + TableConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin,HashAgg") + val sqlQuery = + """ + |WITH r AS (SELECT * FROM x, y WHERE a = d AND c LIKE 'He%') + |SELECT sum(b) FROM r group by a + """.stripMargin + util.verifyPlan(sqlQuery) + } + +} -- GitLab