Skip to content

Commit

Permalink
[FLINK-12173][table] Optimize SELECT DISTINCT
Browse files Browse the repository at this point in the history
  • Loading branch information
yiyutian1 committed Dec 9, 2024
1 parent 8e17931 commit 1610130
Show file tree
Hide file tree
Showing 14 changed files with 280 additions and 80 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -416,7 +416,9 @@ object FlinkStreamRuleSets {
// Avoid async calls which call async calls.
AsyncCalcSplitRule.NESTED_SPLIT,
// Avoid having async calls in multiple projections in a single calc.
AsyncCalcSplitRule.ONE_PER_CALC_SPLIT
AsyncCalcSplitRule.ONE_PER_CALC_SPLIT,
// Optimize SELECT DISTINCT to use FlinkLogicalRank
StreamLogicalOptimizeSelectDistinctRule.INSTANCE
)

/** RuleSet to do physical optimize for stream */
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,163 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.flink.table.planner.plan.rules.physical.stream
import org.apache.flink.table.planner.JList
import org.apache.flink.table.planner.calcite.FlinkTypeFactory
import org.apache.flink.table.planner.calcite.FlinkTypeFactory.{isRowtimeIndicatorType, isTimeIndicatorType}
import org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery
import org.apache.flink.table.planner.plan.nodes.FlinkConventions
import org.apache.flink.table.planner.plan.nodes.logical.{FlinkLogicalAggregate, FlinkLogicalCalc, FlinkLogicalJoin, FlinkLogicalRank}
import org.apache.flink.table.planner.plan.nodes.physical.stream.{StreamPhysicalIntervalJoin, StreamPhysicalRank, StreamPhysicalTemporalSort}
import org.apache.flink.table.planner.plan.utils.{RankProcessStrategy, RankUtil, WindowJoinUtil}
import org.apache.flink.table.planner.plan.utils.WindowUtil.groupingContainsWindowStartEnd
import org.apache.flink.table.runtime.operators.rank.{ConstantRankRange, RankType}
import org.apache.flink.table.types.logical.IntType

import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall}
import org.apache.calcite.plan.RelOptRule.{any, operand}
import org.apache.calcite.rel.`type`.{RelDataType, RelDataTypeField, RelDataTypeFieldImpl, RelDataTypeSystem}
import org.apache.calcite.rel.{RelCollation, RelCollations, RelFieldCollation}
import org.apache.calcite.rel.RelNode
import org.apache.calcite.rel.convert.ConverterRule.Config
import org.apache.calcite.rel.core.JoinRelType
import org.apache.calcite.rel.hint.RelHint
import org.apache.calcite.rel.logical.LogicalProject
import org.apache.calcite.rex.{RexInputRef, RexNode, RexProgram}
import org.apache.calcite.util.ImmutableBitSet

import java.util
import java.util.Collections

import scala.collection.convert.ImplicitConversions.`iterable AsScalaIterable`

/**
* Rule that matches [[FlinkLogicalAggregate]], and converts it to [[FlinkLogicalRank]] in the case
* of SELECT DISTINCT queries.
*
* e.g. {SELECT DISTINCT a, b, c;} will be converted to [[FlinkLogicalRank]] instead of
* [[FlinkLogicalAggregate]] in rowtime.
*/
class StreamLogicalOptimizeSelectDistinctRule
extends RelOptRule(operand(classOf[FlinkLogicalAggregate], any)) {
private val classLoader = Thread.currentThread().getContextClassLoader
private val typeSystem = RelDataTypeSystem.DEFAULT
private val typeFactory = new FlinkTypeFactory(classLoader, typeSystem)
private val intType: RelDataType =
typeFactory.createFieldTypeFromLogicalType(new IntType(false))

override def matches(call: RelOptRuleCall): Boolean = {
val rel: FlinkLogicalAggregate = call.rel(0)
// check if it's a SELECT DISTINCT query
val ret =
rel.getGroupSet.cardinality() == rel.getRowType.getFieldCount && rel.getAggCallList.isEmpty

val mq = call.getMetadataQuery
val fmq = FlinkRelMetadataQuery.reuseOrCreate(mq)
val windowProperties = fmq.getRelWindowProperties(rel.getInput)
val grouping = rel.getGroupSet
if (groupingContainsWindowStartEnd(grouping, windowProperties)) {
return false // do not match if the grouping set contains window start and end
}

if (ret) {
rel.getGroupSet.toList.foreach(
i => {
val field: RelDataTypeField = rel.getInput.getRowType.getFieldList.get(i)
if (isTimeIndicatorType(field.getType)) {
return false
}
})
}
ret
}

override def onMatch(call: RelOptRuleCall): Unit = {
val agg: FlinkLogicalAggregate = call.rel(0)

val input = agg.getInput
val traitSet = agg.getTraitSet
val cluster = agg.getCluster

// Create a list of all group keys
val groupKeys = agg.getGroupSet

// Create a projection to select only the fields in the DISTINCT clause
val projectList: JList[RexNode] = {
val list = new util.ArrayList[RexNode](groupKeys.cardinality())
groupKeys.toList.foreach(i => list.add(RexInputRef.of(i, input.getRowType)))
list
}

// Create the projected row type
val projectedFields: JList[RelDataTypeField] = {
val fields = new util.ArrayList[RelDataTypeField](projectList.size())
projectList.forEach {
rexNode =>
val rexInputRef = rexNode.asInstanceOf[RexInputRef]
fields.add(input.getRowType.getFieldList.get(rexInputRef.getIndex))
}
fields
}

// Create the projected row type
val projectedRowType: RelDataType = typeFactory.createStructType(projectedFields)

// Create a RelCollation based on all group keys
val fieldCollations: JList[RelFieldCollation] = {
val collations = new util.ArrayList[RelFieldCollation](groupKeys.cardinality())
groupKeys.foreach {
key => collations.add(new RelFieldCollation(key, RelFieldCollation.Direction.ASCENDING))
}
collations
}
val collation: RelCollation = RelCollations.of(fieldCollations)

// Create a projection to select only the fields in the DISTINCT clause
val projection: RelNode = FlinkLogicalCalc.create(
input,
RexProgram.create(
input.getRowType,
projectList,
null,
projectedRowType,
cluster.getRexBuilder
))

val rank = new FlinkLogicalRank(
cluster,
traitSet,
projection,
groupKeys,
collation,
RankType.ROW_NUMBER,
new ConstantRankRange(1, 1), // We only want the first row for each group
new RelDataTypeFieldImpl("rk", projectedRowType.getFieldCount - 1, intType),
false
)
try RankUtil.canConvertToDeduplicate(rank)
catch {
case _: Exception =>
return
}
call.transformTo(rank)
}
}

object StreamLogicalOptimizeSelectDistinctRule {
val INSTANCE = new StreamLogicalOptimizeSelectDistinctRule()
}
Original file line number Diff line number Diff line change
Expand Up @@ -27,14 +27,14 @@ LogicalSink(table=[default_catalog.default_database.partitioned_sink], targetCol
== Optimized Physical Plan ==
Sink(table=[default_catalog.default_database.partitioned_sink], targetColumns=[[4],[0],[3]], fields=[a, EXPR$1, d, e, EXPR$4, EXPR$5])
+- Calc(select=[a, null:VARCHAR(2147483647) CHARACTER SET "UTF-16LE" AS EXPR$1, d, e, null:BIGINT AS EXPR$4, null:INTEGER AS EXPR$5])
+- GroupAggregate(groupBy=[a, b, c, d, e], select=[a, b, c, d, e])
+- Rank(strategy=[AppendFastStrategy], rankType=[ROW_NUMBER], rankRange=[rankStart=1, rankEnd=1], partitionBy=[a, b, c, d, e], orderBy=[a ASC, b ASC, c ASC, d ASC, e ASC], select=[a, b, c, d, e])
+- Exchange(distribution=[hash[a, b, c, d, e]])
+- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e])
== Optimized Execution Plan ==
Sink(table=[default_catalog.default_database.partitioned_sink], targetColumns=[[4],[0],[3]], fields=[a, EXPR$1, d, e, EXPR$4, EXPR$5])
+- Calc(select=[a, null:VARCHAR(2147483647) AS EXPR$1, d, e, null:BIGINT AS EXPR$4, null:INTEGER AS EXPR$5])
+- GroupAggregate(groupBy=[a, b, c, d, e], select=[a, b, c, d, e])
+- Rank(strategy=[AppendFastStrategy], rankType=[ROW_NUMBER], rankRange=[rankStart=1, rankEnd=1], partitionBy=[a, b, c, d, e], orderBy=[a ASC, b ASC, c ASC, d ASC, e ASC], select=[a, b, c, d, e])
+- Exchange(distribution=[hash[a, b, c, d, e]])
+- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e])
]]>
Expand Down Expand Up @@ -119,7 +119,7 @@ LogicalSink(table=[default_catalog.default_database.sink], targetColumns=[[1],[4
<![CDATA[
Sink(table=[default_catalog.default_database.sink], targetColumns=[[1],[4],[0],[6],[5],[2],[3]], fields=[a, b, c, d, e, f, g])
+- Calc(select=[a, b, c, d, e, 123 AS f, 456 AS g])
+- GroupAggregate(groupBy=[a, b, c, d, e], select=[a, b, c, d, e])
+- Rank(strategy=[AppendFastStrategy], rankType=[ROW_NUMBER], rankRange=[rankStart=1, rankEnd=1], partitionBy=[a, b, c, d, e], orderBy=[a ASC, b ASC, c ASC, d ASC, e ASC], select=[a, b, c, d, e])
+- Exchange(distribution=[hash[a, b, c, d, e]])
+- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e])
]]>
Expand Down Expand Up @@ -164,7 +164,7 @@ LogicalSink(table=[default_catalog.default_database.partitioned_sink], targetCol
<![CDATA[
Sink(table=[default_catalog.default_database.partitioned_sink], targetColumns=[[4],[0],[6],[5],[2],[3]], fields=[a, c, d, e, f, g])
+- Calc(select=[a, c, d, e, 123 AS f, 456 AS g])
+- GroupAggregate(groupBy=[a, b, c, d, e], select=[a, b, c, d, e])
+- Rank(strategy=[AppendFastStrategy], rankType=[ROW_NUMBER], rankRange=[rankStart=1, rankEnd=1], partitionBy=[a, b, c, d, e], orderBy=[a ASC, b ASC, c ASC, d ASC, e ASC], select=[a, b, c, d, e])
+- Exchange(distribution=[hash[a, b, c, d, e]])
+- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e])
]]>
Expand Down Expand Up @@ -221,11 +221,11 @@ Sink(table=[default_catalog.default_database.partitioned_sink], targetColumns=[[
+- Exchange(distribution=[hash[a, c, d, e, f, g]])
+- Union(all=[true], union=[a, c, d, e, f, g, vcol_marker])
:- Calc(select=[a, c, d, e, CAST(123 AS BIGINT) AS f, CAST(456 AS INTEGER) AS g, 1 AS vcol_marker])
: +- GroupAggregate(groupBy=[a, b, c, d, e], select=[a, b, c, d, e])
: +- Rank(strategy=[AppendFastStrategy], rankType=[ROW_NUMBER], rankRange=[rankStart=1, rankEnd=1], partitionBy=[a, b, c, d, e], orderBy=[a ASC, b ASC, c ASC, d ASC, e ASC], select=[a, b, c, d, e])
: +- Exchange(distribution=[hash[a, b, c, d, e]])
: +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e])
+- Calc(select=[a, c, d, e, CAST(123 AS BIGINT) AS f, CAST(456 AS INTEGER) AS g, -1 AS vcol_marker])
+- GroupAggregate(groupBy=[a, b, c, d, e], select=[a, b, c, d, e])
+- Rank(strategy=[AppendFastStrategy], rankType=[ROW_NUMBER], rankRange=[rankStart=1, rankEnd=1], partitionBy=[a, b, c, d, e], orderBy=[a ASC, b ASC, c ASC, d ASC, e ASC], select=[a, b, c, d, e])
+- Exchange(distribution=[hash[a, b, c, d, e]])
+- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e])
]]>
Expand Down Expand Up @@ -299,11 +299,11 @@ Sink(table=[default_catalog.default_database.partitioned_sink], targetColumns=[[
+- Exchange(distribution=[hash[a, c, d, e, f, g]])
+- Union(all=[true], union=[a, c, d, e, f, g, vcol_left_marker, vcol_right_marker])
:- Calc(select=[a, c, d, e, CAST(123 AS BIGINT) AS f, CAST(456 AS INTEGER) AS g, true AS vcol_left_marker, null:BOOLEAN AS vcol_right_marker])
: +- GroupAggregate(groupBy=[a, b, c, d, e], select=[a, b, c, d, e])
: +- Rank(strategy=[AppendFastStrategy], rankType=[ROW_NUMBER], rankRange=[rankStart=1, rankEnd=1], partitionBy=[a, b, c, d, e], orderBy=[a ASC, b ASC, c ASC, d ASC, e ASC], select=[a, b, c, d, e])
: +- Exchange(distribution=[hash[a, b, c, d, e]])
: +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e])
+- Calc(select=[a, c, d, e, CAST(456 AS BIGINT) AS f, CAST(789 AS INTEGER) AS g, null:BOOLEAN AS vcol_left_marker, true AS vcol_right_marker])
+- GroupAggregate(groupBy=[a, b, c, d, e], select=[a, b, c, d, e])
+- Rank(strategy=[AppendFastStrategy], rankType=[ROW_NUMBER], rankRange=[rankStart=1, rankEnd=1], partitionBy=[a, b, c, d, e], orderBy=[a ASC, b ASC, c ASC, d ASC, e ASC], select=[a, b, c, d, e])
+- Exchange(distribution=[hash[a, b, c, d, e]])
+- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e])
]]>
Expand Down Expand Up @@ -455,15 +455,15 @@ LogicalSink(table=[default_catalog.default_database.partitioned_sink], targetCol
<![CDATA[
Sink(table=[default_catalog.default_database.partitioned_sink], targetColumns=[[4],[0],[6],[5],[2],[3]], fields=[a, c, d, e, f, g])
+- Calc(select=[a, c, d, e, CAST(EXPR$4 AS BIGINT) AS f, CAST(EXPR$5 AS INTEGER) AS g])
+- GroupAggregate(groupBy=[a, c, d, e, EXPR$4, EXPR$5], select=[a, c, d, e, EXPR$4, EXPR$5])
+- Rank(strategy=[RetractStrategy], rankType=[ROW_NUMBER], rankRange=[rankStart=1, rankEnd=1], partitionBy=[a, c, d, e, EXPR$4, EXPR$5], orderBy=[a ASC, c ASC, d ASC, e ASC, EXPR$4 ASC, EXPR$5 ASC], select=[a, c, d, e, EXPR$4, EXPR$5])
+- Exchange(distribution=[hash[a, c, d, e, EXPR$4, EXPR$5]])
+- Union(all=[true], union=[a, c, d, e, EXPR$4, EXPR$5])
:- Calc(select=[a, c, d, e, 123 AS EXPR$4, 456 AS EXPR$5])
: +- GroupAggregate(groupBy=[a, b, c, d, e], select=[a, b, c, d, e])
: +- Rank(strategy=[AppendFastStrategy], rankType=[ROW_NUMBER], rankRange=[rankStart=1, rankEnd=1], partitionBy=[a, b, c, d, e], orderBy=[a ASC, b ASC, c ASC, d ASC, e ASC], select=[a, b, c, d, e])
: +- Exchange(distribution=[hash[a, b, c, d, e]])
: +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e])
+- Calc(select=[a, c, d, e, 456 AS EXPR$4, 789 AS EXPR$5])
+- GroupAggregate(groupBy=[a, b, c, d, e], select=[a, b, c, d, e])
+- Rank(strategy=[AppendFastStrategy], rankType=[ROW_NUMBER], rankRange=[rankStart=1, rankEnd=1], partitionBy=[a, b, c, d, e], orderBy=[a ASC, b ASC, c ASC, d ASC, e ASC], select=[a, b, c, d, e])
+- Exchange(distribution=[hash[a, b, c, d, e]])
+- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e])
]]>
Expand Down Expand Up @@ -491,11 +491,11 @@ LogicalSink(table=[default_catalog.default_database.partitioned_sink], targetCol
Sink(table=[default_catalog.default_database.partitioned_sink], targetColumns=[[4],[0],[6],[5],[2],[3]], fields=[a, c, d, e, f, g])
+- Union(all=[true], union=[a, c, d, e, f, g])
:- Calc(select=[a, c, d, e, CAST(123 AS BIGINT) AS f, CAST(456 AS INTEGER) AS g])
: +- GroupAggregate(groupBy=[a, b, c, d, e], select=[a, b, c, d, e])
: +- Rank(strategy=[AppendFastStrategy], rankType=[ROW_NUMBER], rankRange=[rankStart=1, rankEnd=1], partitionBy=[a, b, c, d, e], orderBy=[a ASC, b ASC, c ASC, d ASC, e ASC], select=[a, b, c, d, e])
: +- Exchange(distribution=[hash[a, b, c, d, e]])
: +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e])
+- Calc(select=[a, c, d, e, CAST(456 AS BIGINT) AS f, CAST(789 AS INTEGER) AS g])
+- GroupAggregate(groupBy=[a, b, c, d, e], select=[a, b, c, d, e])
+- Rank(strategy=[AppendFastStrategy], rankType=[ROW_NUMBER], rankRange=[rankStart=1, rankEnd=1], partitionBy=[a, b, c, d, e], orderBy=[a ASC, b ASC, c ASC, d ASC, e ASC], select=[a, b, c, d, e])
+- Exchange(distribution=[hash[a, b, c, d, e]])
+- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e])
]]>
Expand Down Expand Up @@ -612,15 +612,15 @@ Sink(table=[default_catalog.default_database.partitioned_sink], targetColumns=[[
+- Union(all=[true], union=[a, c, d, e, f, g])
:- Union(all=[true], union=[a, c, d, e, f, g])
: :- Calc(select=[a, c, d, e, CAST(123 AS BIGINT) AS f, CAST(456 AS INTEGER) AS g])
: : +- GroupAggregate(groupBy=[a, b, c, d, e], select=[a, b, c, d, e])
: : +- Rank(strategy=[AppendFastStrategy], rankType=[ROW_NUMBER], rankRange=[rankStart=1, rankEnd=1], partitionBy=[a, b, c, d, e], orderBy=[a ASC, b ASC, c ASC, d ASC, e ASC], select=[a, b, c, d, e])
: : +- Exchange(distribution=[hash[a, b, c, d, e]])
: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e])
: +- Calc(select=[a, c, d, e, CAST(456 AS BIGINT) AS f, CAST(789 AS INTEGER) AS g])
: +- GroupAggregate(groupBy=[a, b, c, d, e], select=[a, b, c, d, e])
: +- Rank(strategy=[AppendFastStrategy], rankType=[ROW_NUMBER], rankRange=[rankStart=1, rankEnd=1], partitionBy=[a, b, c, d, e], orderBy=[a ASC, b ASC, c ASC, d ASC, e ASC], select=[a, b, c, d, e])
: +- Exchange(distribution=[hash[a, b, c, d, e]])
: +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e])
+- Calc(select=[a, c, d, e, CAST(456 AS BIGINT) AS f, CAST(123 AS INTEGER) AS g])
+- GroupAggregate(groupBy=[a, b, c, d, e], select=[a, b, c, d, e])
+- Rank(strategy=[AppendFastStrategy], rankType=[ROW_NUMBER], rankRange=[rankStart=1, rankEnd=1], partitionBy=[a, b, c, d, e], orderBy=[a ASC, b ASC, c ASC, d ASC, e ASC], select=[a, b, c, d, e])
+- Exchange(distribution=[hash[a, b, c, d, e]])
+- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c, d, e)]]], fields=[a, b, c, d, e])
]]>
Expand Down
Loading

0 comments on commit 1610130

Please sign in to comment.