Github user rtudoran commented on a diff in the pull request:
https://github.com/apache/flink/pull/3641#discussion_r108715233
--- Diff:
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala
---
@@ -274,6 +286,57 @@ class DataStreamOverAggregate(
}
result
}
+
+ def createTimeBoundedProcessingTimeOverWindow(inputDS: DataStream[Row]):
DataStream[Row] = {
+
+ val overWindow: Group = logicWindow.groups.get(0)
+ val partitionKeys: Array[Int] = overWindow.keys.toArray
+ val namedAggregates: Seq[CalcitePair[AggregateCall, String]] =
generateNamedAggregates
+
+ val index =
overWindow.lowerBound.getOffset.asInstanceOf[RexInputRef].getIndex
+ val count = input.getRowType.getFieldCount
+ val lowerBoundIndex = index - count
+
+
+ val timeBoundary =
logicWindow.constants.get(lowerBoundIndex).getValue2 match {
+ case bd: java.math.BigDecimal => bd.longValue()
+ case _ => throw new TableException("OVER Window boundaries must be
numeric")
+ }
+
+ // get the output types
+ val rowTypeInfo =
FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf[RowTypeInfo]
+
+ val result: DataStream[Row] =
+ // partitioned aggregation
+ if (partitionKeys.nonEmpty) {
+
+ val processFunction =
AggregateUtil.createTimeBoundedProcessingOverProcessFunction(
+ namedAggregates,
+ inputType,
+ timeBoundary)
+
+ inputDS
+ .keyBy(partitionKeys: _*)
+ .process(processFunction)
+ .returns(rowTypeInfo)
+ .name(aggOpName)
+ .asInstanceOf[DataStream[Row]]
+ } else { // non-partitioned aggregation
+ val processFunction =
AggregateUtil.createTimeBoundedProcessingOverProcessFunction(
--- End diff --
@sunjincheng121 I disagree with this. Even if we move the creation of the
process function at the top of if (which is fine), we still need to make the
differentiation on what to use in the KeyBy - the actual field or a
NullSelector. Hence, as the if is needed i prefer to keep it as it is
---
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.
---