[
https://issues.apache.org/jira/browse/FLINK-7599?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16703170#comment-16703170
]
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_r237464842
##########
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]) = {
Review comment:
Can you add comments to this method or break the logic into more variables
in the method. It is very difficult to understand the Scala magic here. You
call the method `extractAggregatesAndExpressions` but what do the expression
that come out of this method describe? Also you use too generic `aggs` and
`agg` variables name with different data types and meaning.
----------------------------------------------------------------
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:
[email protected]
> 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)