[
https://issues.apache.org/jira/browse/FLINK-5654?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15927882#comment-15927882
]
ASF GitHub Bot commented on FLINK-5654:
---------------------------------------
Github user hongyuhong commented on a diff in the pull request:
https://github.com/apache/flink/pull/3550#discussion_r106395452
--- Diff:
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala
---
@@ -119,6 +154,64 @@ class DataStreamOverAggregate(
}
+ 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
+
+ // final long time_boundary =
+ //
Long.parseLong(windowReference.getConstants().get(1).getValue().toString());
+ val index =
overWindow.lowerBound.getOffset.asInstanceOf[RexInputRef].getIndex
+ val count = input.getRowType().getFieldCount()
+ val lowerboundIndex = index - count
+ val time_boundary = logicWindow.constants.get(lowerboundIndex)
+ .getValue2.asInstanceOf[java.math.BigDecimal].longValue()
+
+
+ val (aggFields, aggregates) =
AggregateUtil.transformToAggregateFunctions(
+ namedAggregates.map(_.getKey),inputType, needRetraction = false)
+
+
+ // As we it is not possible to operate neither on sliding count neither
+ // on sliding time we need to manage the eviction of the events that
+ // expire ourselves based on the proctime (system time). Therefore the
+ // current system time is assign as the timestamp of the event to be
+ // recognize by the evictor
+
+ val inputDataStreamTimed = inputDS
+ .assignTimestampsAndWatermarks(new ProcTimeTimestampExtractor())
+
+ // get the output types
+ val rowTypeInfo =
FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf[RowTypeInfo]
+
+ val result: DataStream[Row] =
+ if (partitionKeys.nonEmpty) {
+ inputDataStreamTimed.keyBy(0)
--- End diff --
It seems should be inputDataStreamTimed.keyBy(partitionKeys:_*)
> 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)