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

    https://github.com/apache/flink/pull/3364#discussion_r104507420
  
    --- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala
 ---
    @@ -153,6 +170,156 @@ object AggregateUtil {
       }
     
       /**
    +    * Create a 
[[org.apache.flink.api.common.functions.GroupReduceFunction]] that prepares for
    +    * incremental aggregates of sliding windows (time and count-windows).
    +    * It requires a prepared input (with intermediate aggregate fields and 
aligned rowtime for
    +    * pre-tumbling in case of time-windows), pre-aggregates (pre-tumbles) 
rows, aligns the
    +    * window-start, and replicates or omits records for different panes of 
a sliding window.
    +    *
    +    * The output of the function contains the grouping keys, the 
intermediate aggregate values of
    +    * all aggregate function and the aligned window start. Window start 
must not be a timestamp,
    +    * but can also be a count value for count-windows.
    +    *
    +    * The output is stored in Row by the following format:
    +    *
    +    * {{{
    +    *                      avg(x) aggOffsetInRow = 2      count(z) 
aggOffsetInRow = 5
    +    *                            |                          |
    +    *                            v                          v
    +    *        
+---------+---------+--------+--------+--------+--------+-------------+
    +    *        |groupKey1|groupKey2|  sum1  | count1 |  sum2  | count2 | 
windowStart |
    +    *        
+---------+---------+--------+--------+--------+--------+-------------+
    +    *                                              ^                 ^
    +    *                                              |                 |
    +    *                                 sum(y) aggOffsetInRow = 4    window 
start for pane mapping
    +    * }}}
    +    *
    +    * NOTE: this function is only used for sliding windows on batch tables.
    +    */
    +  def createDataSetSlideWindowPrepareGroupReduceFunction(
    +      window: LogicalWindow,
    +      namedAggregates: Seq[CalcitePair[AggregateCall, String]],
    +      groupings: Array[Int],
    +      inputType: RelDataType,
    +      isParserCaseSensitive: Boolean)
    +    : RichGroupReduceFunction[Row, Row] = {
    +
    +    val aggregates = transformToAggregateFunctions(
    +      namedAggregates.map(_.getKey),
    +      inputType,
    +      groupings.length)._2
    +
    +    val returnType: RowTypeInfo = createAggregateBufferDataType(
    +      groupings,
    +      aggregates,
    +      inputType,
    +      Some(Array(BasicTypeInfo.LONG_TYPE_INFO)))
    +
    +    window match {
    +      case EventTimeSlidingGroupWindow(_, _, size, slide) if 
isTimeInterval(size.resultType) =>
    +        // sliding time-window
    +        if (aggregates.forall(_.supportPartial)) {
    +          // for incremental aggregations
    +          new DataSetSlideTimeWindowAggReduceCombineFunction(
    +            aggregates,
    +            groupings.length,
    +            returnType.getArity - 1,
    +            asLong(size),
    +            asLong(slide),
    +            returnType)
    +        } else {
    +          // for non-incremental aggregations
    --- End diff --
    
    Do we need this case? Without partial aggregation, we cannot use tumbling 
windows to compute sliding windows. So this method is only called if all 
aggregates support partial aggregation, right? 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---

Reply via email to