dianfu commented on a change in pull request #7177: [FLINK-7599] [table] 
Support for aggregates in MATCH_RECOGNIZE
URL: https://github.com/apache/flink/pull/7177#discussion_r237731718
 
 

 ##########
 File path: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/MatchCodeGenerator.scala
 ##########
 @@ -442,4 +582,245 @@ class MatchCodeGenerator(
 
     generateFieldAccess(patternVariableRef.copy(code = NO_CODE), 
fieldRef.getIndex)
   }
+
+  class AggBuilder(variable: String) {
+
+    private val aggregates = new mutable.ListBuffer[RexCall]()
+
+    private val variableUID = newName("variable")
+
+    private val resultRowTerm = newName(s"aggRow_$variableUID")
+
+    private val rowTypeTerm = "org.apache.flink.types.Row"
+
+    def generateAggAccess(aggCall: RexCall): GeneratedExpression = {
+      reusableInputUnboxingExprs.get((aggCall.toString, 0)) match  {
+        case Some(expr) =>
+          expr
+
+        case None =>
+          val exp: GeneratedExpression = doGenerateAggAccess(aggCall)
+          aggregates += aggCall
+          reusableInputUnboxingExprs((aggCall.toString, 0)) = exp
+          exp.copy(code = NO_CODE)
+      }
+    }
+
+    private def doGenerateAggAccess(call: RexCall) = {
+      val singleResultTerm = newName("result")
+      val singleResultNullTerm = newName("nullTerm")
+      val singleResultType = FlinkTypeFactory.toTypeInfo(call.`type`)
+      val primitiveSingleResultTypeTerm = 
primitiveTypeTermForTypeInfo(singleResultType)
+      val boxedSingleResultTypeTerm = 
boxedTypeTermForTypeInfo(singleResultType)
+
+      val patternName = findEventsByPatternName(variable)
+
+      val codeForAgg =
+        j"""
+           |$rowTypeTerm $resultRowTerm = 
calculateAgg_$variableUID(${patternName.resultTerm});
+           |""".stripMargin
+
+      reusablePerRecordStatements += codeForAgg
+
+      val defaultValue = primitiveDefaultValue(singleResultType)
+      val codeForSingleAgg = if (nullCheck) {
+        j"""
+           |boolean $singleResultNullTerm;
+           |$primitiveSingleResultTypeTerm $singleResultTerm;
+           |if ($resultRowTerm.getField(${aggregates.size}) != null) {
+           |  $singleResultTerm = ($boxedSingleResultTypeTerm) $resultRowTerm
+           |    .getField(${aggregates.size});
+           |  $singleResultNullTerm = false;
+           |} else {
+           |  $singleResultNullTerm = true;
+           |  $singleResultTerm = $defaultValue;
+           |}
+           |""".stripMargin
+      } else {
+        j"""
+           |boolean $singleResultNullTerm = false;
+           |$primitiveSingleResultTypeTerm $singleResultTerm =
+           |    ($boxedSingleResultTypeTerm) 
$resultRowTerm.getField(${aggregates.size});
+           |""".stripMargin
+      }
+
+      reusablePerRecordStatements += codeForSingleAgg
+
+      GeneratedExpression(singleResultTerm, singleResultNullTerm, NO_CODE, 
singleResultType)
+    }
+
+    def generateAggFunction() : Unit = {
+      val matchAgg = extractAggregatesAndExpressions
+
+      val aggGenerator = new AggregationCodeGenerator(config, false, input, 
None)
+
+      val aggFunc = aggGenerator.generateAggregations(
+        s"AggFunction_$variableUID",
+        matchAgg.inputExprs.map(r => FlinkTypeFactory.toTypeInfo(r.getType)),
+        matchAgg.aggregations.map(_.aggFunction).toArray,
+        matchAgg.aggregations.map(_.inputIndices).toArray,
+        matchAgg.aggregations.indices.toArray,
+        Array.fill(matchAgg.aggregations.size)(false),
+        isStateBackedDataViews = false,
+        partialResults = false,
+        Array.emptyIntArray,
+        None,
+        matchAgg.aggregations.size,
+        needRetract = false,
+        needMerge = false,
+        needReset = false,
+        None
+      )
+
+      reusableMemberStatements.add(aggFunc.code)
+
+      val transformFuncName = s"transformRowForAgg_$variableUID"
+      val inputTransform: String = generateAggInputExprEvaluation(
+        matchAgg.inputExprs,
+        transformFuncName)
+
+      generateAggCalculation(aggFunc, transformFuncName, inputTransform)
+    }
+
+    private case class LogicalMatchAggCall(
+      function: SqlAggFunction,
+      inputTypes: Seq[RelDataType],
+      exprIndices: Seq[Int]
+    )
+
+    private case class MatchAggCall(
+      aggFunction: TableAggregateFunction[_, _],
+      inputIndices: Array[Int],
+      dataViews: Seq[DataViewSpec[_]]
+    )
+
+    private case class MatchAgg(
+      aggregations: Seq[MatchAggCall],
+      inputExprs: Seq[RexNode]
+    )
+
+    private def extractAggregatesAndExpressions: MatchAgg = {
+      val inputRows = new mutable.LinkedHashMap[String, (RexNode, Int)]
+
+      val logicalAggregates = aggregates.map(rexAggCall => {
+        val callsWithIndices = rexAggCall.operands.asScala.map(innerCall => {
+          inputRows.get(innerCall.toString) match {
+            case Some(x) =>
+              x
+
+            case None =>
+              val callWithIndex = (innerCall, inputRows.size)
+              inputRows(innerCall.toString) = callWithIndex
+              callWithIndex
+          }
+        }).toList
+
+        val agg = rexAggCall.getOperator.asInstanceOf[SqlAggFunction]
+        LogicalMatchAggCall(agg,
+          callsWithIndices.map(_._1.getType),
+          callsWithIndices.map(_._2).toArray)
+      })
+
+      val aggs = logicalAggregates.zipWithIndex.map {
+        case (agg, index) =>
+          val result = AggregateUtil.extractAggregateCallMetadata(
+            agg.function,
+            isDistinct = false,
+            agg.inputTypes,
+            needRetraction = false,
+            config,
+            isStateBackedDataViews = false,
+            index)
+
+          MatchAggCall(result.aggregateFunction, agg.exprIndices.toArray, 
result.accumulatorSpecs)
+      }
+
+      MatchAgg(aggs, inputRows.values.map(_._1).toSeq)
+    }
+
+    private def generateAggCalculation(
+        aggFunc: GeneratedAggregationsFunction,
+        transformFuncName: String,
+        inputTransform: String)
+      : Unit = {
+      val aggregatorTerm = s"aggregator_$variableUID"
+      val code =
+        j"""
+           |private final ${aggFunc.name} $aggregatorTerm;
+           |
+           |$inputTransform
+           |
+           |private $rowTypeTerm calculateAgg_$variableUID(java.util.List 
input)
+           |    throws Exception {
+           |  $rowTypeTerm accumulator = $aggregatorTerm.createAccumulators();
+           |  for ($rowTypeTerm row : input) {
+           |    $aggregatorTerm.accumulate(accumulator, 
$transformFuncName(row));
+           |  }
+           |  $rowTypeTerm result = $aggregatorTerm.createOutputRow();
+           |  $aggregatorTerm.setAggregationResults(accumulator, result);
+           |  return result;
+           |}
+         """.stripMargin
+
+      reusableInitStatements.add(s"$aggregatorTerm = new ${aggFunc.name}();")
+      reusableOpenStatements.add(s"$aggregatorTerm.open(getRuntimeContext());")
+      reusableCloseStatements.add(s"$aggregatorTerm.close();")
+      reusableMemberStatements.add(code)
+    }
+
+    private def generateAggInputExprEvaluation(
+        inputExprs: Seq[RexNode],
+        funcName: String)
+      : String = {
+      isWithinAggExprState = true
+      val resultTerm = newName("result")
+      val exprs = inputExprs.zipWithIndex.map(row => {
 
 Review comment:
   Use `inputExprs.zipWithIndex.map { case (inputExpr, index)` instead? It will 
make the code more readable.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

Reply via email to