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

    https://github.com/apache/flink/pull/3364#discussion_r104925507
  
    --- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/aggregate/AggregateUtil.scala
 ---
    @@ -186,6 +200,130 @@ object AggregateUtil {
       }
     
       /**
    +    * Create a 
[[org.apache.flink.api.common.functions.GroupReduceFunction]] that prepares for
    +    * partial 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 with partial 
aggregates 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,
    +      needRetraction = false)._2
    +
    +    val returnType: RowTypeInfo = createDataSetAggregateBufferDataType(
    +      groupings,
    +      aggregates,
    +      inputType,
    +      Some(Array(BasicTypeInfo.LONG_TYPE_INFO)))
    +
    +    window match {
    +      case EventTimeSlidingGroupWindow(_, _, size, slide) if 
isTimeInterval(size.resultType) =>
    +        // sliding time-window
    +        // for partial aggregations
    +        new DataSetSlideTimeWindowAggReduceCombineFunction(
    +          aggregates,
    +          groupings.length,
    +          returnType.getArity - 1,
    +          asLong(size),
    +          asLong(slide),
    +          returnType)
    +
    +      case _ =>
    +        throw new UnsupportedOperationException(s"$window is currently not 
supported on batch.")
    +    }
    +  }
    +
    +  /**
    +    * Create a [[org.apache.flink.api.common.functions.FlatMapFunction]] 
that prepares for
    +    * non-incremental aggregates of sliding windows (time-windows).
    +    *
    +    * It requires a prepared input (with intermediate aggregate fields), 
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.
    +    *
    +    * 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 time-based sliding windows on 
batch tables.
    +    */
    +  def createDataSetSlideWindowPrepareFlatMapFunction(
    +      window: LogicalWindow,
    +      namedAggregates: Seq[CalcitePair[AggregateCall, String]],
    +      groupings: Array[Int],
    +      inputType: RelDataType,
    +      isParserCaseSensitive: Boolean)
    +    : FlatMapFunction[Row, Row] = {
    +
    +    val aggregates = transformToAggregateFunctions(
    +      namedAggregates.map(_.getKey),
    +      inputType,
    +      needRetraction = false)._2
    +
    +    val mapReturnType: RowTypeInfo = createDataSetAggregateBufferDataType(
    --- End diff --
    
    output type should be equal to input type.


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

Reply via email to