[
https://issues.apache.org/jira/browse/FLINK-6228?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15978486#comment-15978486
]
ASF GitHub Bot commented on FLINK-6228:
---------------------------------------
Github user fhueske commented on a diff in the pull request:
https://github.com/apache/flink/pull/3743#discussion_r112652207
--- Diff:
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/call.scala
---
@@ -49,6 +59,160 @@ case class Call(functionName: String, args:
Seq[Expression]) extends Expression
}
/**
+ * Over expression for calcite over transform.
+ *
+ * @param agg over-agg expression
+ * @param overWindowAlias over window alias
+ * @param overWindow over window
+ */
+case class OverCall(
+ agg: Aggregation,
+ overWindowAlias: Expression,
+ var overWindow: OverWindow = null) extends Expression {
+
+ override private[flink] def toRexNode(implicit relBuilder: RelBuilder):
RexNode = {
+
+ val rexBuilder = relBuilder.getRexBuilder
+
+ val operator: SqlAggFunction = agg.getSqlAggFunction()
+
+ val relDataType = relBuilder
+ .getTypeFactory.asInstanceOf[FlinkTypeFactory]
+ .createTypeFromTypeInfo(agg.resultType)
+
+ val aggExprs: util.ArrayList[RexNode] = new util.ArrayList[RexNode]()
+ val aggChildName = agg.child.asInstanceOf[ResolvedFieldReference].name
+
+ aggExprs.add(relBuilder.field(aggChildName))
+
+ val orderKeys: ImmutableList.Builder[RexFieldCollation] =
+ new ImmutableList.Builder[RexFieldCollation]()
+
+ val sets: util.HashSet[SqlKind] = new util.HashSet[SqlKind]()
+ val orderName =
overWindow.orderBy.asInstanceOf[UnresolvedFieldReference].name
+
+ val rexNode =
+ if (orderName.equalsIgnoreCase("rowtime")) {
--- End diff --
We cannot properly distinguish batch and streaming here.
If a batch table has a column `rowtime`, this will fail.
We could replace the `UnresolvedFieldReference` for `rowtime` / `proctime`
by `RowTime` / `ProcTime` `TimeIndicator` expression in
`ProjectionTranslator.translateOverWindow()`.
> Integrating the OVER windows in the Table API
> ---------------------------------------------
>
> Key: FLINK-6228
> URL: https://issues.apache.org/jira/browse/FLINK-6228
> Project: Flink
> Issue Type: Sub-task
> Components: Table API & SQL
> Reporter: sunjincheng
> Assignee: sunjincheng
>
> Syntax:
> {code}
> table
> .overWindows(
> (Rows|Range [ partitionBy value_expression , ... [ n ]] [ orderBy
> order_by_expression]
> (preceding
> UNBOUNDED|value_specification.(rows|milli|second|minute|hour|day|month|year)|CURRENTROW)
> [following
> UNBOUNDED|value_specification.(rows|milli|second|minute|hour|day|month|year)|CURRENTROW]
> as alias,...[n])
> )
> .select( [col1,...[n]], (agg(col1) OVER overWindowAlias, … [n])
> {code}
> Implement restrictions:
> * All OVER clauses in the same SELECT clause must be exactly the same.
> * The PARTITION BY clause is optional (no partitioning results in single
> threaded execution).
> * The ORDER BY Before the
> [FLINK-5884|https://issues.apache.org/jira/browse/FLINK-5884] implementation
> orderBy may only have ‘rowtime/’proctime(for stream)/‘specific-time-field(for
> batch).
> * FOLLOWING is not supported.
> User interface design document [See |
> https://docs.google.com/document/d/13Z-Ovx3jwtmzkSweJyGkMy751BouhuJ29Y1CTNZt2DI/edit#]
--
This message was sent by Atlassian JIRA
(v6.3.15#6346)