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

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

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

    https://github.com/apache/flink/pull/3547#discussion_r106519205
  
    --- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala
 ---
    @@ -130,32 +142,76 @@ class DataStreamOverAggregate(
         val rowTypeInfo = 
FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf[RowTypeInfo]
     
         val result: DataStream[Row] =
    -        // partitioned aggregation
    -        if (partitionKeys.nonEmpty) {
    -          val processFunction = 
AggregateUtil.CreateUnboundedProcessingOverProcessFunction(
    -            namedAggregates,
    -            inputType)
    +      // partitioned aggregation
    +      if (partitionKeys.nonEmpty) {
    +        val processFunction = 
AggregateUtil.CreateUnboundedProcessingOverProcessFunction(
    +          namedAggregates,
    +          inputType)
     
    -          inputDS
    +        inputDS
               .keyBy(partitionKeys: _*)
               .process(processFunction)
               .returns(rowTypeInfo)
               .name(aggOpName)
               .asInstanceOf[DataStream[Row]]
    -        }
    -        // non-partitioned aggregation
    -        else {
    -          val processFunction = 
AggregateUtil.CreateUnboundedProcessingOverProcessFunction(
    -            namedAggregates,
    -            inputType,
    -            false)
    -
    -          inputDS
    -            
.process(processFunction).setParallelism(1).setMaxParallelism(1)
    -            .returns(rowTypeInfo)
    -            .name(aggOpName)
    -            .asInstanceOf[DataStream[Row]]
    -        }
    +      } // non-partitioned aggregation
    +      else {
    +        val processFunction = 
AggregateUtil.CreateUnboundedProcessingOverProcessFunction(
    +          namedAggregates,
    +          inputType,
    +          false)
    +
    +        inputDS
    +          .process(processFunction).setParallelism(1).setMaxParallelism(1)
    +          .returns(rowTypeInfo)
    +          .name(aggOpName)
    +          .asInstanceOf[DataStream[Row]]
    +      }
    +    result
    +  }
    +
    +  def createBoundedAndCurrentRowProcessingTimeOverWindow(
    +    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
    +
    +    // get the output types
    +    val rowTypeInfo = 
FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf[RowTypeInfo]
    +
    +    val lowerbound: Int = AggregateUtil.getLowerBoundary(
    +      logicWindow.constants,
    +      overWindow.lowerBound,
    +      getInput())
    +
    +    val result: DataStream[Row] =
    +      // partitioned aggregation
    +      if (partitionKeys.nonEmpty) {
    +        val windowFunction = 
AggregateUtil.CreateBoundedProcessingOverWindowFunction(
    +          namedAggregates,
    +          inputType)
    +        inputDS
    +          .keyBy(partitionKeys: _*)
    +          .countWindow(lowerbound,1)
    +          .apply(windowFunction)
    +          .returns(rowTypeInfo)
    +          .name(aggOpName)
    +          .asInstanceOf[DataStream[Row]]
    +      } // global non-partitioned aggregation
    +      else {
    +        val windowFunction = 
AggregateUtil.CreateBoundedProcessingOverGlobalWindowFunction(
    +          namedAggregates,
    +          inputType)
    +
    +        inputDS
    +          .countWindowAll(lowerbound,1)
    --- End diff --
    
    From the [MS SQL Server 
documentation](https://msdn.microsoft.com/en-us/library/ms189461.aspx): 
    > For example, ROWS BETWEEN 2 PRECEDING AND CURRENT ROW means that the 
window of rows that the function operates on is three rows in size, starting 
with 2 rows preceding until and including the current row.
    
    So, since `lowerBound` is already `AggregateUtil.getLowerBoundary(...) + 
1`, we should be good.


> 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