[
https://issues.apache.org/jira/browse/FLINK-5654?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15947514#comment-15947514
]
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_r108722922
--- Diff:
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala
---
@@ -1208,5 +1208,40 @@ object AggregateUtil {
private def gcd(a: Long, b: Long): Long = {
if (b == 0) a else gcd(b, a % b)
}
-}
+
+ /**
+ * Create an
[[org.apache.flink.streaming.api.functions.ProcessFunction]] to evaluate final
+ * aggregate value over a window with processing time boundaries.
+ *
+ * @param namedAggregates List of calls to aggregate functions and their
output field names
+ * @param inputType Input row type
+ * @param timeBoundary time limit of the window boundary expressed in
milliseconds
+ * @param isPartitioned Flag to indicate whether the input is
partitioned or not
+ * @return [[org.apache.flink.streaming.api.functions.ProcessFunction]]
+ */
+ private[flink] def createTimeBoundedProcessingOverProcessFunction(
+ namedAggregates: Seq[CalcitePair[AggregateCall, String]],
+ inputType: RelDataType,
+ timeBoundary: Long,
+ isPartitioned: Boolean = true): ProcessFunction[Row, Row] = {
+
+ val (aggFields, aggregates) =
+ transformToAggregateFunctions(
+ namedAggregates.map(_.getKey),
+ inputType,
+ needRetraction = false)
+ val aggregationStateType: RowTypeInfo =
+ createDataSetAggregateBufferDataType(Array(), aggregates, inputType)
--- End diff --
use `createAccumulatorRowType(aggregates)`
> 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)