Github user fhueske commented on a diff in the pull request:
https://github.com/apache/flink/pull/3397#discussion_r104448367
--- Diff:
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala
---
@@ -0,0 +1,210 @@
+/*
+ * 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.nodes.datastream
+
+import org.apache.calcite.plan.{RelOptCluster, RelTraitSet}
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rel.core.AggregateCall
+import org.apache.calcite.rel.{RelNode, RelWriter, SingleRel}
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.typeutils.RowTypeInfo
+import org.apache.flink.streaming.api.datastream.DataStream
+import org.apache.flink.table.api.{StreamTableEnvironment, TableException}
+import org.apache.flink.table.calcite.FlinkTypeFactory
+import
org.apache.flink.table.runtime.aggregate.AggregateUtil.{CalcitePair, _}
+import org.apache.flink.table.runtime.aggregate._
+import org.apache.flink.table.plan.nodes.CommonAggregate
+import org.apache.flink.types.Row
+import org.apache.calcite.rel.core.Window
+import org.apache.calcite.rel.core.Window.Group
+import java.util.{List => JList}
+
+import org.apache.flink.table.functions.{ProcTimeType, RowTimeType}
+
+import scala.collection.JavaConverters._
+import scala.collection.immutable.IndexedSeq
+
+class DataStreamOverAggregate(
+ logicWindow: Window,
+ cluster: RelOptCluster,
+ traitSet: RelTraitSet,
+ inputNode: RelNode,
+ rowRelDataType: RelDataType,
+ inputType: RelDataType)
+ extends SingleRel(cluster, traitSet, inputNode)
+ with CommonAggregate
+ with DataStreamRel {
+
+ override def deriveRowType(): RelDataType = rowRelDataType
+
+ override def copy(traitSet: RelTraitSet, inputs: JList[RelNode]):
RelNode = {
+ new DataStreamOverAggregate(
+ logicWindow,
+ cluster,
+ traitSet,
+ inputs.get(0),
+ getRowType,
+ inputType)
+ }
+
+ override def toString: String = {
+ val (
+ overWindow: Group,
+ partitionKeys: Array[Int],
+ namedAggregates: IndexedSeq[CalcitePair[AggregateCall, String]]
+ ) = genPartitionKeysAndNamedAggregates
+
+ s"Aggregate(${
+ if (!partitionKeys.isEmpty) {
+ s"partitionBy: (${groupingToString(inputType, partitionKeys)}), "
+ } else {
+ ""
+ }
+ }window: ($overWindow), " +
+ s"select: (${
+ aggregationToString(
+ inputType,
+ partitionKeys,
+ getRowType,
+ namedAggregates,
+ Seq())
+ }))"
+ }
+
+ override def explainTerms(pw: RelWriter): RelWriter = {
+ val (
+ overWindow: Group,
+ partitionKeys: Array[Int],
+ namedAggregates: IndexedSeq[CalcitePair[AggregateCall, String]]
+ ) = genPartitionKeysAndNamedAggregates
+
+ super.explainTerms(pw)
+ .itemIf("partitionBy", groupingToString(inputType, partitionKeys),
!partitionKeys.isEmpty)
+ .item("overWindow", overWindow)
+ .item(
+ "select", aggregationToString(
+ inputType,
+ partitionKeys,
+ getRowType,
+ namedAggregates,
+ Seq()))
+ }
+
+ override def translateToPlan(tableEnv: StreamTableEnvironment):
DataStream[Row] = {
+
+ if (logicWindow.groups.size > 1) {
+ throw new TableException(
+ "Unsupported use of OVER windows. All aggregates must be computed
on the same window.")
+ }
+
+ val overWindow: org.apache.calcite.rel.core.Window.Group =
logicWindow.groups.get(0)
+
+ val inputDS =
input.asInstanceOf[DataStreamRel].translateToPlan(tableEnv)
+
+ if (overWindow.orderKeys.getFieldCollations.size() != 1) {
+ throw new TableException(
+ "Unsupported use of OVER windows. All aggregates must be ordered
on a time mode column.")
+ }
+
+ val timeType = inputType
+ .getFieldList
+ .get(overWindow.orderKeys.getFieldCollations.get(0).getFieldIndex)
+ .getValue
+
+ timeType match {
+ case _: ProcTimeType =>
+ // both ROWS and RANGE clause with UNBOUNDED PRECEDING and CURRENT
ROW condition.
+ if (overWindow.lowerBound.isUnbounded &&
+ overWindow.upperBound.isCurrentRow) {
+ createUnboundedAndCurrentRowProcessingTimeOverWindow(inputDS)
+ } else {
+ throw new TableException(
+ "OVER window only support ProcessingTime UNBOUNDED PRECEDING
and CURRENT ROW " +
+ "condition.")
+ }
+ case _: RowTimeType =>
+ throw new TableException("OVER Window of the EventTime type is not
currently supported.")
+ case _ =>
+ throw new TableException(s"Unsupported time type {$timeType}")
+ }
+
+ }
+
+ def createUnboundedAndCurrentRowProcessingTimeOverWindow(
+ inputDS: DataStream[Row]): DataStream[Row] = {
+
+ val (
+ overWindow: Group,
+ partitionKeys: Array[Int],
+ namedAggregates: IndexedSeq[CalcitePair[AggregateCall, String]]
+ ) = genPartitionKeysAndNamedAggregates
+
+ val inputIndices = (0 until inputType.getFieldCount).toArray
+
+ // get the output types
+ val fieldTypes: Array[TypeInformation[_]] = getRowType
+ .getFieldList.asScala
+ .map(field => FlinkTypeFactory.toTypeInfo(field.getType)).toArray
+
+ val rowTypeInfo = new RowTypeInfo(fieldTypes: _*)
+
+ val aggString = aggregationToString(
+ inputType,
+ inputIndices,
+ getRowType,
+ namedAggregates,
+ Seq())
+
+ val keyedAggOpName = s"partitionBy: (${groupingToString(inputType,
partitionKeys)}), " +
--- End diff --
This string contains duplicate information (PartitionBy and the Aggregates).
How about a naming pattern like this:
`"over: (PARTITION BY: x,y,z, ORDER BY: t, RANGE BETWEEN UNBOUNDED
PRECEDING AND CURRENT ROW), select: (a, b, c, COUNT(a) AS w0$o0))"`?
---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---