[ https://issues.apache.org/jira/browse/FLINK-5654?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15947506#comment-15947506 ]
ASF GitHub Bot commented on FLINK-5654: --------------------------------------- Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3641#discussion_r108722044 --- 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 -- The `isPartitioned` parameter is not used in the `createTimeBoundedProcessingOverProcessFunction()`. So both calls are identical and we can move it outside of the condition and save a few LOCs. > Add processing time OVER RANGE BETWEEN x PRECEDING aggregation to SQL > --------------------------------------------------------------------- > > Key: FLINK-5654 > URL: https://issues.apache.org/jira/browse/FLINK-5654 > Project: Flink > Issue Type: Sub-task > Components: Table API & SQL > Reporter: Fabian Hueske > Assignee: radu > > The goal of this issue is to add support for OVER RANGE aggregations on > processing time streams to the SQL interface. > Queries similar to the following should be supported: > {code} > SELECT > a, > SUM(b) OVER (PARTITION BY c ORDER BY procTime() RANGE BETWEEN INTERVAL '1' > HOUR PRECEDING AND CURRENT ROW) AS sumB, > MIN(b) OVER (PARTITION BY c ORDER BY procTime() RANGE BETWEEN INTERVAL '1' > HOUR PRECEDING AND CURRENT ROW) AS minB > FROM myStream > {code} > The following restrictions should initially apply: > - 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 clause may only have procTime() as parameter. procTime() is a > parameterless scalar function that just indicates processing time mode. > - UNBOUNDED PRECEDING is not supported (see FLINK-5657) > - FOLLOWING is not supported. > The restrictions will be resolved in follow up issues. If we find that some > of the restrictions are trivial to address, we can add the functionality in > this issue as well. > This issue includes: > - Design of the DataStream operator to compute OVER ROW aggregates > - Translation from Calcite's RelNode representation (LogicalProject with > RexOver expression). -- This message was sent by Atlassian JIRA (v6.3.15#6346)