[
https://issues.apache.org/jira/browse/FLINK-5803?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15897610#comment-15897610
]
ASF GitHub Bot commented on FLINK-5803:
---------------------------------------
Github user fhueske commented on a diff in the pull request:
https://github.com/apache/flink/pull/3397#discussion_r104431561
--- Diff:
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala
---
@@ -93,6 +94,41 @@ object AggregateUtil {
}
/**
+ * Create an [[RichProcessFunction]] to evaluate final aggregate value.
+ *
+ * @param namedAggregates List of calls to aggregate functions and
their output field names
+ * @param inputType Input row type
+ * @param outputType Output row type
+ * @param forwardedFields All the forwarded fields
+ * @return [[UnboundedProcessingOverProcessFunction]]
+ */
+ private[flink] def CreateUnboundedProcessingOverProcessFunction(
+ namedAggregates: Seq[CalcitePair[AggregateCall, String]],
+ inputType: RelDataType,
+ outputType: RelDataType,
+ forwardedFields: Array[Int]): UnboundedProcessingOverProcessFunction =
{
+
+ val (aggFields, aggregates) =
+ transformToAggregateFunctions(
+ namedAggregates.map(_.getKey),
+ inputType,
+ forwardedFields.length)
--- End diff --
the `groupKeysCount` (here forwardedFields.length) parameter can be removed
from `AggregateUtil.transformToAggregateFunctions()`.
> Add [partitioned] processing time OVER RANGE BETWEEN UNBOUNDED PRECEDING
> aggregation to SQL
> -------------------------------------------------------------------------------------------
>
> Key: FLINK-5803
> URL: https://issues.apache.org/jira/browse/FLINK-5803
> Project: Flink
> Issue Type: Sub-task
> Components: Table API & SQL
> Reporter: sunjincheng
> Assignee: sunjincheng
>
> 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 UNBOUNDED
> PRECEDING AND CURRENT ROW) AS sumB,
> MIN(b) OVER (PARTITION BY c ORDER BY procTime() RANGE BETWEEN UNBOUNDED
> 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 ORDER BY clause may only have procTime() as parameter. procTime() is a
> parameterless scalar function that just indicates processing time mode.
> - bounded PRECEDING is not supported (see FLINK-5654)
> - 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)