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

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

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

 ##########
 File path: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/MatchCodeGenerator.scala
 ##########
 @@ -445,4 +498,223 @@ 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 getOrAddAggregation(call: RexCall): GeneratedExpression = {
+      reusableInputUnboxingExprs.get((call.toString, 0)) match  {
+        case Some(expr) =>
+          expr
+
+        case None =>
+          val exp: GeneratedExpression = generateAggAccess(call)
+          aggregates += call
+          reusableInputUnboxingExprs((call.toString, 0)) = exp
+          exp.copy(code = NO_CODE)
+      }
+    }
+
+    private def generateAggAccess(call: RexCall) = {
+      val singleResultTerm = newName("result")
+      val singleResultNullTerm = newName("nullTerm")
+      val singleResultType = FlinkTypeFactory.toTypeInfo(call.`type`)
+      val singleResultTypeTerm = boxedTypeTermForTypeInfo(singleResultType)
+
+      val patternName = findEventsByPatternName(variable)
+
+      val codeForAgg =
+        j"""
+           |$rowTypeTerm $resultRowTerm = 
calculateAgg_$variableUID(${patternName.resultTerm});
+           |""".stripMargin
+
+      reusablePerRecordStatements += codeForAgg
+
+      val defaultValue = primitiveDefaultValue(singleResultType)
+      val codeForSingleAgg =
+        j"""
+           |boolean $singleResultNullTerm;
+           |$singleResultTypeTerm $singleResultTerm = ($singleResultTypeTerm) 
$resultRowTerm
+           |  .getField(${aggregates.size});
+           |if ($singleResultTerm != null) {
+           |  $singleResultNullTerm = false;
+           |} else {
+           |  $singleResultNullTerm = true;
+           |  $singleResultTerm = $defaultValue;
+           |}
+           |""".stripMargin
+
+      reusablePerRecordStatements += codeForSingleAgg
+
+      val exp = GeneratedExpression(singleResultTerm,
+        singleResultNullTerm,
+        NO_CODE,
+        singleResultType)
+      exp
+    }
+
+    def generateAggFunction() : Unit = {
+      val (aggs, exprs) = extractAggregatesAndExpressions
+
+      val aggGenerator = new AggregationCodeGenerator(config, false, input, 
None)
+
+      val aggFunc = aggGenerator.generateAggregations(
+        s"AggFunction_$variableUID",
+        exprs.map(r => FlinkTypeFactory.toTypeInfo(r.getType)),
+        aggs.map(_.aggFunction).toArray,
+        aggs.map(_.inputIndices).toArray,
+        aggs.indices.toArray,
+        Array.fill(aggs.size)(false),
+        isStateBackedDataViews = false,
+        partialResults = false,
+        Array.emptyIntArray,
+        None,
+        aggs.size,
+        needRetract = false,
+        needMerge = false,
+        needReset = false,
+        None
+      )
+
+      reusableMemberStatements.add(aggFunc.code)
+
+      val transformFuncName = s"transformRowForAgg_$variableUID"
+      val inputTransform: String = generateAggInputExprEvaluation(
+        exprs,
+        transformFuncName)
+
+      generateAggCalculation(aggFunc, transformFuncName, inputTransform)
+    }
+
+    private case class MatchAggCall(
+      aggFunction: TableAggregateFunction[_, _],
+      inputIndices: Array[Int],
+      dataViews: Seq[DataViewSpec[_]]
+    )
+
+    private def extractAggregatesAndExpressions: (Seq[MatchAggCall], 
Seq[RexNode]) = {
+      val inputRows = new mutable.LinkedHashMap[String, (RexNode, Int)]
+
+      val aggs = 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]
+        (agg, callsWithIndices.map(_._1), callsWithIndices.map(_._2).toArray)
+      }).zipWithIndex.map {
+        case (agg, index) =>
+          val result = AggregateUtil
+            .transformToAggregateFunction(agg._1,
+              isDistinct = false,
+              agg._2.map(_.getType),
+              needRetraction = false,
+              config,
+              isStateBackedDataViews = false,
+              index)
+
+          MatchAggCall(result._1, agg._3, result._3)
+      }
+
+      (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 = {
+      innerAggExpr = true
+      val resultTerm = newName("result")
+      val exprs = inputExprs.zipWithIndex.map(row => {
+        val expr = generateExpression(row._1)
+        s"""
+           |${expr.code}
+           |if (${expr.nullTerm}) {
+           |  $resultTerm.setField(${row._2}, null);
+           |} else {
+           |  $resultTerm.setField(${row._2}, ${expr.resultTerm});
+           |}
+         """.stripMargin
+      }).mkString("\n")
+      innerAggExpr = false
+
+      j"""
+         |private $rowTypeTerm $funcName($rowTypeTerm $inputAggRowTerm) {
+         |  $rowTypeTerm $resultTerm = new $rowTypeTerm(${inputExprs.size});
+         |  $exprs
+         |  return $resultTerm;
+         |}
+       """.stripMargin
+    }
+  }
+}
+
+class PatternVariableFinder extends RexDefaultVisitor[Option[String]] {
 
 Review comment:
   This class has one additional function other than just validation. It also 
extracts the variable from aggregate `RexCall`, which I need for grouping 
aggregates by it. I could add the validation at an earlier stage, but 
effectively I would need to duplicate/reuse the same code. If you feel it's 
worth doing, I will update it accordingly.

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


> Support aggregation functions in the define and measures clause of 
> MatchRecognize
> ---------------------------------------------------------------------------------
>
>                 Key: FLINK-7599
>                 URL: https://issues.apache.org/jira/browse/FLINK-7599
>             Project: Flink
>          Issue Type: Sub-task
>          Components: CEP, Table API & SQL
>            Reporter: Dian Fu
>            Assignee: Dawid Wysakowicz
>            Priority: Major
>              Labels: pull-request-available
>             Fix For: 1.8.0
>
>




--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to