[ 
https://issues.apache.org/jira/browse/FLINK-5653?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15946598#comment-15946598
 ] 

ASF GitHub Bot commented on FLINK-5653:
---------------------------------------

Github user sunjincheng121 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3574#discussion_r108598268
  
    --- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala
 ---
    @@ -729,6 +747,39 @@ object AggregateUtil {
     
         (aggFunction, accumulatorRowType, aggResultRowType)
       }
    +  
    +  
    +     /*
    +   * Function for building the processing logic for aggregating data in 
row bounded windows
    +   *  @param namedAggregates List of calls to aggregate functions and 
their output field names
    +    * @param inputType Input row type
    +    * @param rowType Type info of row
    +    * @param lowerBound the window boundary
    +    * @return [[org.apache.flink.streaming.api.functions.ProcessFunction]]
    +   */
    +  private[flink] def createBoundedProcessingOverProcessFunction(
    +    namedAggregates: Seq[CalcitePair[AggregateCall, String]],
    +    inputType: RelDataType,
    +    rowType: RowTypeInfo,
    +    lowerBound: Int): ProcessFunction[Row, Row] = {
    +
    +    val (aggFields, aggregates) =
    +      transformToAggregateFunctions(
    +        namedAggregates.map(_.getKey),
    +        inputType,
    +        needRetraction = true)
    +
    +    val aggregationStateType: RowTypeInfo =
    +      createDataSetAggregateBufferDataType(Array(), aggregates, inputType)
    +
    +    new BoundedProcessingOverRowProcessFunction(
    +      aggregates,
    +      aggFields, lowerBound,
    --- End diff --
    
    `aggFields, lowerBound,` ->
    ` aggFields,`
     `lowerBound,`


> Add processing time OVER ROWS BETWEEN x PRECEDING aggregation to SQL
> --------------------------------------------------------------------
>
>                 Key: FLINK-5653
>                 URL: https://issues.apache.org/jira/browse/FLINK-5653
>             Project: Flink
>          Issue Type: Sub-task
>          Components: Table API & SQL
>            Reporter: Fabian Hueske
>            Assignee: Stefano Bortoli
>
> The goal of this issue is to add support for OVER ROWS 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() ROWS BETWEEN 2 PRECEDING 
> AND CURRENT ROW) AS sumB,
>   MIN(b) OVER (PARTITION BY c ORDER BY procTime() ROWS BETWEEN 2 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-5656)
> - 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)

Reply via email to