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

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

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

 ##########
 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
 
 Review comment:
   Something is wrong here. You are casting a `Row` into the result type of the 
aggregation?

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