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

    https://github.com/apache/flink/pull/3472#discussion_r104282353
  
    --- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/DataSetAggregate.scala
 ---
    @@ -87,47 +89,67 @@ class DataSetAggregate(
     
       override def translateToPlan(tableEnv: BatchTableEnvironment): 
DataSet[Row] = {
     
    -    val groupingKeys = grouping.indices.toArray
    -
    -    val mapFunction = AggregateUtil.createPrepareMapFunction(
    -      namedAggregates,
    -      grouping,
    -      inputType)
    -
    -    val groupReduceFunction = 
AggregateUtil.createAggregateGroupReduceFunction(
    -      namedAggregates,
    -      inputType,
    -      rowRelDataType,
    -      grouping,
    -      inGroupingSet)
    +    val (preAgg: Option[DataSetPreAggFunction],
    +          preAggType: Option[TypeInformation[Row]],
    +          finalAgg: GroupReduceFunction[Row, Row]) =
    +      AggregateUtil.createDataSetAggregateFunctions(
    +        namedAggregates,
    +        inputType,
    +        rowRelDataType,
    +        grouping,
    +        inGroupingSet)
     
         val inputDS = 
getInput.asInstanceOf[DataSetRel].translateToPlan(tableEnv)
     
         val aggString = aggregationToString(inputType, grouping, getRowType, 
namedAggregates, Nil)
    -    val prepareOpName = s"prepare select: ($aggString)"
    -    val mappedInput = inputDS.map(mapFunction).name(prepareOpName)
     
         val rowTypeInfo = 
FlinkTypeFactory.toInternalRowTypeInfo(getRowType).asInstanceOf[RowTypeInfo]
     
    -    if (groupingKeys.length > 0) {
    +    if (grouping.length > 0) {
           // grouped aggregation
           val aggOpName = s"groupBy: (${groupingToString(inputType, 
grouping)}), " +
             s"select: ($aggString)"
     
    -      mappedInput.asInstanceOf[DataSet[Row]]
    -        .groupBy(groupingKeys: _*)
    -        .reduceGroup(groupReduceFunction)
    -        .returns(rowTypeInfo)
    -        .name(aggOpName)
    +      if (preAgg.isDefined) {
    +        inputDS
    +          // pre-aggregation
    +          .groupBy(grouping: _*)
    +          .combineGroup(preAgg.get)
    +          .returns(preAggType.get)
    +          .name(aggOpName)
    +          // final aggregation
    +          .groupBy(grouping.indices: _*)
    --- End diff --
    
    The pre-aggregation function modifies the schema of rows. It puts all 
grouping keys first, followed by all accumulators. Therefore, the following 
final aggregation needs to group on the first `n` fields.
    Before, the prepare mapper changed the layout.


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