[ https://issues.apache.org/jira/browse/FLINK-6584?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16217042#comment-16217042 ]
ASF GitHub Bot commented on FLINK-6584: --------------------------------------- Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/4199#discussion_r146575894 --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/rules/common/WindowPropertiesRule.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.plan.rules.common + +import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall, RelOptRuleOperand} +import org.apache.calcite.rel.RelNode +import org.apache.calcite.rel.logical.{LogicalFilter, LogicalProject} +import org.apache.calcite.rex.{RexCall, RexNode} +import org.apache.calcite.tools.RelBuilder +import org.apache.flink.table.api.{TableException, ValidationException} +import org.apache.flink.table.calcite.FlinkRelBuilder.NamedWindowProperty +import org.apache.flink.table.expressions._ +import org.apache.flink.table.plan.logical.rel.LogicalWindowAggregate +import org.apache.flink.table.validate.BasicOperatorTable + +import scala.collection.JavaConversions._ + +abstract class WindowPropertiesBaseRule(rulePredicate: RelOptRuleOperand, ruleName: String) + extends RelOptRule(rulePredicate, ruleName) { + + override def matches(call: RelOptRuleCall): Boolean = { + val project = call.rel(0).asInstanceOf[LogicalProject] + // project includes at least on group auxiliary function + + def hasGroupAuxiliaries(node: RexNode): Boolean = { + node match { + case c: RexCall if c.getOperator.isGroupAuxiliary => true + case c: RexCall => + c.operands.exists(hasGroupAuxiliaries) + case _ => false + } + } + + project.getProjects.exists(hasGroupAuxiliaries) + } + + def convertWindowNodes( + builder: RelBuilder, + project: LogicalProject, + filter: Option[LogicalFilter], + innerProject: LogicalProject, + agg: LogicalWindowAggregate) + : RelNode = { + + val rexBuilder = builder.getRexBuilder + + val window = agg.getWindow + + val isRowtime = ExpressionUtils.isRowtimeAttribute(window.timeAttribute) + val isProctime = ExpressionUtils.isProctimeAttribute(window.timeAttribute) + + def propertyName(name: String): String = + window.aliasAttribute.asInstanceOf[WindowReference].name + name + + val startEndProperties = Seq( + NamedWindowProperty(propertyName("start"), WindowStart(window.aliasAttribute)), + NamedWindowProperty(propertyName("end"), WindowEnd(window.aliasAttribute))) + + // allow rowtime/proctime for rowtime windows and proctime for proctime windows + val timeProperties = if (isRowtime) { + Seq( + NamedWindowProperty(propertyName("rowtime"), RowtimeAttribute(window.aliasAttribute)), + NamedWindowProperty(propertyName("proctime"), ProctimeAttribute(window.aliasAttribute))) + } else if (isProctime) { + Seq(NamedWindowProperty(propertyName("proctime"), ProctimeAttribute(window.aliasAttribute))) + } else { + Seq() + } + + val properties = startEndProperties ++ timeProperties + + // retrieve window start and end properties + builder.push(agg.copy(properties)) + + // forward window start and end properties --- End diff -- `forward window properties`? > Support multiple consecutive windows in SQL > ------------------------------------------- > > Key: FLINK-6584 > URL: https://issues.apache.org/jira/browse/FLINK-6584 > Project: Flink > Issue Type: New Feature > Components: Table API & SQL > Reporter: Timo Walther > Assignee: Timo Walther > Priority: Blocker > Fix For: 1.4.0 > > > Right now, the Table API supports multiple consecutive windows as follows: > {code} > val table = stream.toTable(tEnv, 'rowtime.rowtime, 'int, 'double, 'float, > 'bigdec, 'string) > val t = table > .window(Tumble over 2.millis on 'rowtime as 'w) > .groupBy('w) > .select('w.rowtime as 'rowtime, 'int.count as 'int) > .window(Tumble over 4.millis on 'rowtime as 'w2) > .groupBy('w2) > .select('w2.rowtime, 'w2.end, 'int.count) > {code} > Similar behavior should be supported by the SQL API as well. We need to > introduce a new auxiliary group function, but this should happen in sync with > Apache Calcite. -- This message was sent by Atlassian JIRA (v6.4.14#64029)