beyond1920 commented on a change in pull request #14905: URL: https://github.com/apache/flink/pull/14905#discussion_r580009166
########## File path: flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/sql/SqlCumulateTableFunction.java ########## @@ -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.planner.functions.sql; + +import org.apache.flink.shaded.guava18.com.google.common.collect.ImmutableList; + +import org.apache.calcite.sql.SqlCallBinding; +import org.apache.calcite.sql.SqlOperator; + +/** + * SqlCumulateTableFunction implements an operator for cumulative. + * + * <p>It allows four parameters: + * + * <ol> + * <li>a table Review comment: missing `</li>`? ########## File path: flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/SplitAggregateRule.scala ########## @@ -36,6 +36,8 @@ import org.apache.calcite.sql.fun.SqlStdOperatorTable._ import org.apache.calcite.sql.fun.{SqlMinMaxAggFunction, SqlStdOperatorTable} import org.apache.calcite.sql.{SqlAggFunction, SqlKind} import org.apache.calcite.util.{ImmutableBitSet, ImmutableIntList} +import org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery Review comment: move this import to Line 29? ########## File path: flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamPhysicalWindowAggregateRule.scala ########## @@ -0,0 +1,256 @@ +/* + * 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.calcite.plan.{RelOptRule, RelOptRuleCall, RelTraitSet} +import org.apache.calcite.rel.RelNode +import org.apache.calcite.rel.convert.ConverterRule +import org.apache.calcite.rel.core.Aggregate.Group +import org.apache.calcite.rex.{RexInputRef, RexProgram} +import org.apache.flink.table.api.TableException +import org.apache.flink.table.planner.calcite.FlinkRelBuilder.PlannerNamedWindowProperty +import org.apache.flink.table.planner.expressions._ +import org.apache.flink.table.planner.plan.`trait`.{FlinkRelDistribution, RelWindowProperties} +import org.apache.flink.table.planner.plan.logical.{WindowAttachedWindowingStrategy, WindowingStrategy} +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 +import org.apache.flink.table.planner.plan.nodes.physical.stream.{StreamPhysicalCalc, StreamPhysicalWindowAggregate} +import org.apache.flink.table.planner.plan.rules.physical.stream.StreamPhysicalWindowAggregateRule.{WINDOW_END, WINDOW_START, WINDOW_TIME} +import org.apache.flink.table.planner.plan.utils.WindowUtil + +import scala.collection.JavaConverters._ +import scala.collection.mutable.ArrayBuffer + +/** + * Rule to convert a [[FlinkLogicalAggregate]] into a [[StreamPhysicalWindowAggregate]]. + */ +class StreamPhysicalWindowAggregateRule + extends ConverterRule( + classOf[FlinkLogicalAggregate], + FlinkConventions.LOGICAL, + FlinkConventions.STREAM_PHYSICAL, + "StreamPhysicalWindowAggregateRule") { + + override def matches(call: RelOptRuleCall): Boolean = { + val agg: FlinkLogicalAggregate = call.rel(0) + + // check if we have grouping sets + if (agg.getGroupType != Group.SIMPLE || agg.indicator) { + throw new TableException("GROUPING SETS are currently not supported.") + } + + val fmq = FlinkRelMetadataQuery.reuseOrCreate(call.getMetadataQuery) + val windowProperties = fmq.getRelWindowProperties(agg.getInput) + val grouping = agg.getGroupSet + WindowUtil.groupingContainsWindowStartEnd(grouping, windowProperties) Review comment: `matches` method has many deduplicate code with `StreamPhysicalGroupAggregateRule', could we find a way to remove the deduplicate code? ########## File path: flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/FlinkRexUtil.scala ########## @@ -83,6 +83,16 @@ object FlinkRexUtil { new CnfHelper(rexBuilder, maxCnfNodeCnt).toCnf(rex) } + /** + * Returns whether a given tree contains expected [[RexInputRef]] nodes. + */ Review comment: It's better to clarify return true if input RexNode contains any node in the given expected [[RexInputRef]] nodes. ########## File path: flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/stats/FlinkStatistic.scala ########## @@ -36,7 +35,8 @@ import scala.collection.JavaConversions._ class FlinkStatistic( tableStats: TableStats, uniqueKeys: util.Set[_ <: util.Set[String]] = null, - relModifiedMonotonicity: RelModifiedMonotonicity = null) + relModifiedMonotonicity: RelModifiedMonotonicity = null, + relWindowProperties: RelWindowProperties = null) Review comment: take `relWindowProperties ` into consideration in `toString` method ########## File path: flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/WindowUtil.scala ########## @@ -0,0 +1,235 @@ +/* + * 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.utils + +import org.apache.calcite.rel.`type`.RelDataType +import org.apache.calcite.rel.core.Calc +import org.apache.calcite.rex.{RexBuilder, RexCall, RexInputRef, RexLiteral, RexNode, RexProgram, RexProgramBuilder} +import org.apache.calcite.sql.SqlKind +import org.apache.calcite.sql.`type`.SqlTypeFamily +import org.apache.calcite.util.ImmutableBitSet +import org.apache.flink.table.api.{TableException, ValidationException} +import org.apache.flink.table.planner.JBigDecimal +import org.apache.flink.table.planner.calcite.FlinkTypeFactory +import org.apache.flink.table.planner.functions.sql.{FlinkSqlOperatorTable, SqlWindowTableFunction} +import org.apache.flink.table.planner.plan.`trait`.RelWindowProperties +import org.apache.flink.table.planner.plan.logical.{CumulativeWindowSpec, HoppingWindowSpec, TimeAttributeWindowingStrategy, TumblingWindowSpec, WindowingStrategy} Review comment: remove unused import `WindowingStrategy`. ########## File path: flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/functions/sql/SqlHopTableFunction.java ########## @@ -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.planner.functions.sql; + +import org.apache.flink.shaded.guava18.com.google.common.collect.ImmutableList; + +import org.apache.calcite.sql.SqlCallBinding; +import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.SqlOperator; + +/** + * SqlHopTableFunction implements an operator for hopping. + * + * <p>It allows four parameters: + * + * <ol> + * <li>a table Review comment: missing `</li>`? ########## File path: flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/PullUpWindowTableFunctionIntoWindowAggregateRule.scala ########## @@ -0,0 +1,149 @@ +/* + * 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.plan.`trait`.FlinkRelDistribution +import org.apache.flink.table.planner.plan.logical.TimeAttributeWindowingStrategy +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.physical.stream.{StreamPhysicalCalc, StreamPhysicalExchange, StreamPhysicalWindowAggregate, StreamPhysicalWindowTableFunction} +import org.apache.flink.table.planner.plan.utils.WindowUtil +import org.apache.flink.table.planner.plan.utils.WindowUtil.buildNewProgramWithoutWindowColumns + +import org.apache.calcite.plan.RelOptRule.{any, operand} +import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall} +import org.apache.calcite.rel.{RelCollations, RelNode} +import org.apache.calcite.util.ImmutableBitSet + +import scala.collection.JavaConversions._ + +/** + * Planner rule that tries to pull up [[StreamPhysicalWindowTableFunction]] into a + * [[StreamPhysicalWindowAggregate]]. + */ +class PullUpWindowTableFunctionIntoWindowAggregateRule + extends RelOptRule( + operand(classOf[StreamPhysicalWindowAggregate], + operand(classOf[StreamPhysicalExchange], + operand(classOf[StreamPhysicalCalc], + operand(classOf[StreamPhysicalWindowTableFunction], any())))), + "PullUpWindowTableFunctionIntoWindowAggregateRule"){ + + override def matches(call: RelOptRuleCall): Boolean = { + val windowAgg: StreamPhysicalWindowAggregate = call.rel(0) + val calc: StreamPhysicalCalc = call.rel(2) + val fmq = FlinkRelMetadataQuery.reuseOrCreate(windowAgg.getCluster.getMetadataQuery) + + // condition and projection of Calc shouldn't contain calls on window columns, + // otherwise, we can't transpose WindowTVF and Calc + if (WindowUtil.calcContainsCallsOnWindowColumns(calc, fmq)) { + return false + } + + val aggInputWindowProps = fmq.getRelWindowProperties(calc).getWindowColumns + // aggregate call shouldn't be on window columns + // TODO: this can be supported in the future by referencing them as a RexFieldVariable + windowAgg.aggCalls.forall { call => + aggInputWindowProps.intersect(ImmutableBitSet.of(call.getArgList)).isEmpty + } + } + + override def onMatch(call: RelOptRuleCall): Unit = { + val windowAgg: StreamPhysicalWindowAggregate = call.rel(0) + val calc: StreamPhysicalCalc = call.rel(2) + val windowTVF: StreamPhysicalWindowTableFunction = call.rel(3) + val fmq = FlinkRelMetadataQuery.reuseOrCreate(windowAgg.getCluster.getMetadataQuery) + val cluster = windowAgg.getCluster + val input = windowTVF.getInput Review comment: Line 73 ~ Line 95 is deduplicate with `ExpandWindowTableFunctionTransposeRule`, could we find a way to remove the deduplicate code? ########## File path: flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/WindowUtil.scala ########## @@ -0,0 +1,235 @@ +/* + * 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.utils + +import org.apache.calcite.rel.`type`.RelDataType +import org.apache.calcite.rel.core.Calc +import org.apache.calcite.rex.{RexBuilder, RexCall, RexInputRef, RexLiteral, RexNode, RexProgram, RexProgramBuilder} +import org.apache.calcite.sql.SqlKind +import org.apache.calcite.sql.`type`.SqlTypeFamily +import org.apache.calcite.util.ImmutableBitSet +import org.apache.flink.table.api.{TableException, ValidationException} +import org.apache.flink.table.planner.JBigDecimal +import org.apache.flink.table.planner.calcite.FlinkTypeFactory +import org.apache.flink.table.planner.functions.sql.{FlinkSqlOperatorTable, SqlWindowTableFunction} +import org.apache.flink.table.planner.plan.`trait`.RelWindowProperties +import org.apache.flink.table.planner.plan.logical.{CumulativeWindowSpec, HoppingWindowSpec, TimeAttributeWindowingStrategy, TumblingWindowSpec, WindowingStrategy} +import org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery +import org.apache.flink.table.types.logical.TimestampType + +import java.time.Duration + +import scala.collection.JavaConversions._ +import scala.collection.mutable.ArrayBuffer + +/** + * Utilities for window table-valued functions. + */ +object WindowUtil { + + /** + * Returns true if the grouping keys contain window_start and window_end properties. + */ + def groupingContainsWindowStartEnd( + grouping: ImmutableBitSet, + windowProperties: RelWindowProperties): Boolean = { + if (windowProperties != null) { + val windowStarts = windowProperties.getWindowStartColumns + val windowEnds = windowProperties.getWindowEndColumns + val hasWindowStart = !windowStarts.intersect(grouping).isEmpty + val hasWindowEnd = !windowEnds.intersect(grouping).isEmpty + hasWindowStart && hasWindowEnd + } else { + false + } + } + + /** + * Returns true if the [[RexNode]] is a window table-valued function call. + */ + def isWindowTableFunctionCall(node: RexNode): Boolean = node match { + case call: RexCall => call.getOperator.isInstanceOf[SqlWindowTableFunction] + case _ => false + } + + /** + * Returns true if expressions in [[Calc]] contain calls on window columns. + */ + def calcContainsCallsOnWindowColumns(calc: Calc, fmq: FlinkRelMetadataQuery): Boolean = { + val calcInput = calc.getInput + val calcInputWindowColumns = fmq.getRelWindowProperties(calcInput).getWindowColumns + val calcProgram = calc.getProgram + val condition = calcProgram.getCondition + if (condition != null) { + val predicate = calcProgram.expandLocalRef(condition) + // condition shouldn't contain window columns + if (FlinkRexUtil.containsExpectedInputRef(predicate, calcInputWindowColumns)) { + return true + } + } + // the expressions shouldn't contain calls on window columns + val callsContainProps = calcProgram.getProjectList.map(calcProgram.expandLocalRef).exists { + case rex: RexCall => FlinkRexUtil.containsExpectedInputRef(rex, calcInputWindowColumns) + case _ => false + } + callsContainProps + } + + /** + * Builds a new RexProgram on the input of window-tvf to exclude window columns, + * but include time attribute. + * + * The return tuple consists of 3 elements: Review comment: 3 elements: => 4 elements? ########## File path: flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/StreamPhysicalWindowAggregateRule.scala ########## @@ -0,0 +1,256 @@ +/* + * 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.calcite.plan.{RelOptRule, RelOptRuleCall, RelTraitSet} +import org.apache.calcite.rel.RelNode +import org.apache.calcite.rel.convert.ConverterRule +import org.apache.calcite.rel.core.Aggregate.Group +import org.apache.calcite.rex.{RexInputRef, RexProgram} +import org.apache.flink.table.api.TableException +import org.apache.flink.table.planner.calcite.FlinkRelBuilder.PlannerNamedWindowProperty +import org.apache.flink.table.planner.expressions._ +import org.apache.flink.table.planner.plan.`trait`.{FlinkRelDistribution, RelWindowProperties} +import org.apache.flink.table.planner.plan.logical.{WindowAttachedWindowingStrategy, WindowingStrategy} +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 +import org.apache.flink.table.planner.plan.nodes.physical.stream.{StreamPhysicalCalc, StreamPhysicalWindowAggregate} +import org.apache.flink.table.planner.plan.rules.physical.stream.StreamPhysicalWindowAggregateRule.{WINDOW_END, WINDOW_START, WINDOW_TIME} +import org.apache.flink.table.planner.plan.utils.WindowUtil + +import scala.collection.JavaConverters._ +import scala.collection.mutable.ArrayBuffer + +/** + * Rule to convert a [[FlinkLogicalAggregate]] into a [[StreamPhysicalWindowAggregate]]. + */ +class StreamPhysicalWindowAggregateRule + extends ConverterRule( + classOf[FlinkLogicalAggregate], + FlinkConventions.LOGICAL, + FlinkConventions.STREAM_PHYSICAL, + "StreamPhysicalWindowAggregateRule") { + + override def matches(call: RelOptRuleCall): Boolean = { + val agg: FlinkLogicalAggregate = call.rel(0) + + // check if we have grouping sets + if (agg.getGroupType != Group.SIMPLE || agg.indicator) { + throw new TableException("GROUPING SETS are currently not supported.") + } + + val fmq = FlinkRelMetadataQuery.reuseOrCreate(call.getMetadataQuery) Review comment: Whether `StreamPhysicalWindowAggregateRule ` need to exclude python aggregate like `StreamPhysicalGroupAggregateRule`? ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org