forked from apache/flink
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[FLINK-6261] [table] Support TUMBLE, HOP, SESSION group window functi…
…ons for SQL queries on batch tables. - Drop support for group window translation of "GROUP BY FLOOR/CEIL". This closes apache#3675.
- Loading branch information
Showing
9 changed files
with
762 additions
and
327 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
144 changes: 144 additions & 0 deletions
144
.../src/main/scala/org/apache/flink/table/plan/rules/common/LogicalWindowAggregateRule.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,144 @@ | ||
/* | ||
* 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.common | ||
|
||
import com.google.common.collect.ImmutableList | ||
import org.apache.calcite.plan._ | ||
import org.apache.calcite.plan.hep.HepRelVertex | ||
import org.apache.calcite.rel.`type`.RelDataType | ||
import org.apache.calcite.rel.logical.{LogicalAggregate, LogicalProject} | ||
import org.apache.calcite.rex._ | ||
import org.apache.calcite.sql.fun.SqlStdOperatorTable | ||
import org.apache.calcite.util.ImmutableBitSet | ||
import org.apache.flink.table.api._ | ||
import org.apache.flink.table.calcite.FlinkRelBuilder.NamedWindowProperty | ||
import org.apache.flink.table.plan.logical.rel.LogicalWindowAggregate | ||
|
||
import _root_.scala.collection.JavaConversions._ | ||
|
||
abstract class LogicalWindowAggregateRule(ruleName: String) | ||
extends RelOptRule( | ||
RelOptRule.operand(classOf[LogicalAggregate], | ||
RelOptRule.operand(classOf[LogicalProject], RelOptRule.none())), | ||
ruleName) { | ||
|
||
override def matches(call: RelOptRuleCall): Boolean = { | ||
val agg = call.rel(0).asInstanceOf[LogicalAggregate] | ||
|
||
val distinctAggs = agg.getAggCallList.exists(_.isDistinct) | ||
val groupSets = agg.getGroupSets.size() != 1 || agg.getGroupSets.get(0) != agg.getGroupSet | ||
|
||
val windowExpressions = getWindowExpressions(agg) | ||
if (windowExpressions.length > 1) { | ||
throw new TableException("Only a single window group function may be used in GROUP BY") | ||
} | ||
|
||
!distinctAggs && !groupSets && !agg.indicator && windowExpressions.nonEmpty | ||
} | ||
|
||
/** | ||
* Transform LogicalAggregate with windowing expression to LogicalProject | ||
* + LogicalWindowAggregate + LogicalProject. | ||
* | ||
* The transformation adds an additional LogicalProject at the top to ensure | ||
* that the types are equivalent. | ||
*/ | ||
override def onMatch(call: RelOptRuleCall): Unit = { | ||
val agg = call.rel[LogicalAggregate](0) | ||
val project = agg.getInput.asInstanceOf[HepRelVertex].getCurrentRel.asInstanceOf[LogicalProject] | ||
|
||
val (windowExpr, windowExprIdx) = getWindowExpressions(agg).head | ||
val window = translateWindowExpression(windowExpr, project.getInput.getRowType) | ||
|
||
val builder = call.builder() | ||
val rexBuilder = builder.getRexBuilder | ||
|
||
val inAggGroupExpression = getInAggregateGroupExpression(rexBuilder, windowExpr) | ||
val newGroupSet = agg.getGroupSet.except(ImmutableBitSet.of(windowExprIdx)) | ||
val newAgg = builder | ||
.push(project.getInput) | ||
.project(project.getChildExps.updated(windowExprIdx, inAggGroupExpression)) | ||
.aggregate(builder.groupKey( | ||
newGroupSet, | ||
agg.indicator, ImmutableList.of(newGroupSet)), agg.getAggCallList) | ||
.build().asInstanceOf[LogicalAggregate] | ||
|
||
// Create an additional project to conform with types | ||
val outAggGroupExpression = getOutAggregateGroupExpression(rexBuilder, windowExpr) | ||
val transformed = call.builder() | ||
transformed.push(LogicalWindowAggregate.create( | ||
window.toLogicalWindow, | ||
Seq[NamedWindowProperty](), | ||
newAgg)) | ||
.project(transformed.fields().patch(windowExprIdx, Seq(outAggGroupExpression), 0)) | ||
|
||
call.transformTo(transformed.build()) | ||
} | ||
|
||
private[table] def getWindowExpressions(agg: LogicalAggregate): Seq[(RexCall, Int)] = { | ||
|
||
val project = agg.getInput.asInstanceOf[HepRelVertex].getCurrentRel.asInstanceOf[LogicalProject] | ||
val groupKeys = agg.getGroupSet | ||
|
||
// get grouping expressions | ||
val groupExpr = project.getProjects.zipWithIndex.filter(p => groupKeys.get(p._2)) | ||
|
||
// filter grouping expressions for window expressions | ||
groupExpr.filter { g => | ||
g._1 match { | ||
case call: RexCall => | ||
call.getOperator match { | ||
case SqlStdOperatorTable.TUMBLE => | ||
if (call.getOperands.size() == 2) { | ||
true | ||
} else { | ||
throw TableException("TUMBLE window with alignment is not supported yet.") | ||
} | ||
case SqlStdOperatorTable.HOP => | ||
if (call.getOperands.size() == 3) { | ||
true | ||
} else { | ||
throw TableException("HOP window with alignment is not supported yet.") | ||
} | ||
case SqlStdOperatorTable.SESSION => | ||
if (call.getOperands.size() == 2) { | ||
true | ||
} else { | ||
throw TableException("SESSION window with alignment is not supported yet.") | ||
} | ||
case _ => false | ||
} | ||
case _ => false | ||
} | ||
}.map(w => (w._1.asInstanceOf[RexCall], w._2)) | ||
} | ||
|
||
/** Returns the expression that replaces the window expression before the aggregation. */ | ||
private[table] def getInAggregateGroupExpression( | ||
rexBuilder: RexBuilder, | ||
windowExpression: RexCall): RexNode | ||
|
||
/** Returns the expression that replaces the window expression after the aggregation. */ | ||
private[table] def getOutAggregateGroupExpression( | ||
rexBuilder: RexBuilder, | ||
windowExpression: RexCall): RexNode | ||
|
||
/** translate the group window expression in to a Flink Table window. */ | ||
private[table] def translateWindowExpression(windowExpr: RexCall, rowType: RelDataType): Window | ||
|
||
} |
92 changes: 92 additions & 0 deletions
92
...n/scala/org/apache/flink/table/plan/rules/dataSet/DataSetLogicalWindowAggregateRule.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,92 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one | ||
* or more contributor license agreements. See the NOTICE file | ||
* distributed with this work for additional information | ||
* regarding copyright ownership. The ASF licenses this file | ||
* to you under the Apache License, Version 2.0 (the | ||
* "License"); you may not use this file except in compliance | ||
* with the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
|
||
package org.apache.flink.table.plan.rules.dataSet | ||
|
||
import java.math.BigDecimal | ||
import org.apache.calcite.rel.`type`.RelDataType | ||
import org.apache.calcite.rex._ | ||
import org.apache.calcite.sql.fun.SqlStdOperatorTable | ||
import org.apache.flink.table.api.scala.{Session, Slide, Tumble} | ||
import org.apache.flink.table.api.{TableException, Window} | ||
import org.apache.flink.table.calcite.FlinkTypeFactory | ||
import org.apache.flink.table.expressions.{Expression, Literal, ResolvedFieldReference} | ||
import org.apache.flink.table.plan.rules.common.LogicalWindowAggregateRule | ||
import org.apache.flink.table.typeutils.TimeIntervalTypeInfo | ||
|
||
class DataSetLogicalWindowAggregateRule | ||
extends LogicalWindowAggregateRule("DataSetLogicalWindowAggregateRule") { | ||
|
||
/** Returns the operand of the group window function. */ | ||
override private[table] def getInAggregateGroupExpression( | ||
rexBuilder: RexBuilder, | ||
windowExpression: RexCall): RexNode = windowExpression.getOperands.get(0) | ||
|
||
/** Returns a zero literal of the correct type. */ | ||
override private[table] def getOutAggregateGroupExpression( | ||
rexBuilder: RexBuilder, | ||
windowExpression: RexCall): RexNode = { | ||
|
||
val literalType = windowExpression.getOperands.get(0).getType | ||
rexBuilder.makeZeroLiteral(literalType) | ||
} | ||
|
||
override private[table] def translateWindowExpression( | ||
windowExpr: RexCall, | ||
rowType: RelDataType): Window = { | ||
|
||
def getOperandAsLong(call: RexCall, idx: Int): Long = | ||
call.getOperands.get(idx) match { | ||
case v: RexLiteral => v.getValue.asInstanceOf[BigDecimal].longValue() | ||
case _ => throw new TableException("Only constant window descriptors are supported") | ||
} | ||
|
||
def getFieldReference(operand: RexNode): Expression = { | ||
operand match { | ||
case ref: RexInputRef => | ||
// resolve field name of window attribute | ||
val fieldName = rowType.getFieldList.get(ref.getIndex).getName | ||
val fieldType = rowType.getFieldList.get(ref.getIndex).getType | ||
ResolvedFieldReference(fieldName, FlinkTypeFactory.toTypeInfo(fieldType)) | ||
} | ||
} | ||
|
||
windowExpr.getOperator match { | ||
case SqlStdOperatorTable.TUMBLE => | ||
val interval = getOperandAsLong(windowExpr, 1) | ||
val w = Tumble.over(Literal(interval, TimeIntervalTypeInfo.INTERVAL_MILLIS)) | ||
w.on(getFieldReference(windowExpr.getOperands.get(0))).as("w$") | ||
|
||
case SqlStdOperatorTable.HOP => | ||
val (slide, size) = (getOperandAsLong(windowExpr, 1), getOperandAsLong(windowExpr, 2)) | ||
val w = Slide | ||
.over(Literal(size, TimeIntervalTypeInfo.INTERVAL_MILLIS)) | ||
.every(Literal(slide, TimeIntervalTypeInfo.INTERVAL_MILLIS)) | ||
w.on(getFieldReference(windowExpr.getOperands.get(0))).as("w$") | ||
|
||
case SqlStdOperatorTable.SESSION => | ||
val gap = getOperandAsLong(windowExpr, 1) | ||
val w = Session.withGap(Literal(gap, TimeIntervalTypeInfo.INTERVAL_MILLIS)) | ||
w.on(getFieldReference(windowExpr.getOperands.get(0))).as("w$") | ||
} | ||
} | ||
} | ||
|
||
object DataSetLogicalWindowAggregateRule { | ||
val INSTANCE = new DataSetLogicalWindowAggregateRule | ||
} |
Oops, something went wrong.