flink-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "ASF GitHub Bot (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (FLINK-5654) Add processing time OVER RANGE BETWEEN x PRECEDING aggregation to SQL
Date Wed, 22 Mar 2017 12:33:41 GMT

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

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

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

    https://github.com/apache/flink/pull/3590#discussion_r107401146
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamOverAggregate.scala
---
    @@ -119,6 +150,57 @@ 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
    +
    +    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 match {
    +      case _: java.math.BigDecimal => logicWindow.constants.get(lowerboundIndex)
    +         .getValue2.asInstanceOf[java.math.BigDecimal].longValue()
    +      case _ => throw new TableException("OVER Window boundaries must be numeric")
    +    }
    +
    +     // get the output types
    +    val rowTypeInfo = FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf[RowTypeInfo]
    +         
    +    val result: DataStream[Row] =
    +        // partitioned aggregation
    +        if (partitionKeys.nonEmpty) {
    +          
    +          val processFunction = AggregateUtil.CreateTimeBoundedProcessingOverProcessFunction(
    +            namedAggregates,
    +            inputType,
    +            time_boundary)
    +          
    +          inputDS
    +          .keyBy(partitionKeys: _*)
    +          .process(processFunction)
    +          .returns(rowTypeInfo)
    +          .name(aggOpName)
    +          .asInstanceOf[DataStream[Row]]
    +        } else { // non-partitioned aggregation
    +          val processFunction = AggregateUtil.CreateTimeBoundedProcessingOverProcessFunction(
    --- End diff --
    
    @fhueske - i would be fine with me to do that. However when we discussed this issue on
the initial design and i proposed this solution for the JIRA,  you said that is not worth
consuming extra resources  - and i agree with this ( it is work paying the price of having
2 function at compiling time to get less resource usage...IMHO)
    Also - if you look on what is in the code base for unbound window it is the same - shouldn't
we have the same?
    @sunjincheng121 


> 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)

Mime
View raw message