[GitHub] dawidwys commented on a change in pull request #7177: [FLINK-7599] [table] Support for aggregates in MATCH_RECOGNIZE
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_r241977805 ## File path: docs/dev/table/streaming/match_recognize.md ## @@ -211,6 +211,65 @@ If a condition is not defined for a pattern variable, a default condition will b For a more detailed explanation about expressions that can be used in those clauses, please have a look at the [event stream navigation](#pattern-navigation) section. +### Aggregations + +Aggregations can be used in `DEFINE` and `MEASURES` clauses. Both [built-in]({{ site.baseurl }}/dev/table/sql.html#built-in-functions) and custom [user defined]({{ site.baseurl }}/dev/table/udfs.html) functions are supported. + +Aggregate functions are applied to each subset of rows mapped to a match. In order to understand how those subsets are evaluated have a look at the [event stream navigation](#pattern-navigation) section. + +The task of the following example is to find the longest period of time for which the average price of a ticker did not go below certain threshold. It shows how expressible `MATCH_RECOGNIZE` can become with aggregations. +This task can be performed with the following query: + +{% highlight sql %} +SELECT * +FROM Ticker +MATCH_RECOGNIZE ( +PARTITION BY symbol +ORDER BY rowtime +MEASURES +FIRST(A.rowtime) AS start_tstamp, +LAST(A.rowtime) AS end_tstamp, +AVG(A.price) AS avgPrice +ONE ROW PER MATCH +AFTER MATCH SKIP TO FIRST B +PATTERN (A+ B) +DEFINE +A AS AVG(A.price) < 15 +) MR; Review comment: It is optional, but it is needed if we want to e.g. join the result of MR with other table. For example in a query: SELECT MR.aid as mId, T.id as tId FROM MyTable MATCH_RECOGNIZE ( ORDER BY proctime MEASURES A.id AS aid, A.name as aName PATTERN (A) DEFINE A as A.id > 0 ) MR, MyTable T WHERE MR.aName = T.name 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
[GitHub] dawidwys commented on a change in pull request #7177: [FLINK-7599] [table] Support for aggregates in MATCH_RECOGNIZE
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_r238226741 ## File path: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/MatchCodeGenerator.scala ## @@ -442,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) = { Review comment: It actually generates only the access. It does not generate computing the aggregate. That name might not be perfect, but actually I would like to differentiate it somehow. 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
[GitHub] dawidwys commented on a change in pull request #7177: [FLINK-7599] [table] Support for aggregates in MATCH_RECOGNIZE
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_r238226363 ## 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") Review comment: I think it makes sense here, as it is a term for a single aggregation from a set of all calculated for a pattern variable. It makes even more sense as I moved a variable that holds a term name for result containing all aggregates into this function. 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
[GitHub] dawidwys commented on a change in pull request #7177: [FLINK-7599] [table] Support for aggregates in MATCH_RECOGNIZE
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_r237569734 ## File path: docs/dev/table/streaming/match_recognize.md ## @@ -211,6 +211,66 @@ If a condition is not defined for a pattern variable, a default condition will b For a more detailed explanation about expressions that can be used in those clauses, please have a look at the [event stream navigation](#pattern-navigation) section. +### Scalar & Aggregate functions + +One can use scalar and aggregate functions in those clauses, both [built-in]({{ site.baseurl }}/dev/table/sql.html#built-in-functions) as well as provide [user defined]({{ site.baseurl }}/dev/table/udfs.html) functions. + +Aggregate functions are applied to subset of rows mapped to a match. To understand how those subsets are evaluated have a look at the [event stream navigation](#pattern-navigation) section. + +With a task to find the longest period of time for which the average price of a ticker did not go below certain threshold, one can see how expressible `MATCH_RECOGNIZE` can become with aggregations. +This task can be performed with the following query: + +{% highlight sql %} +SELECT * +FROM Ticker +MATCH_RECOGNIZE ( +PARTITION BY symbol +ORDER BY rowtime +MEASURES +FIRST(A.rowtime) AS start_tstamp, +LAST(A.rowtime) AS end_tstamp, +AVG(A.price) AS avgPrice +ONE ROW PER MATCH +AFTER MATCH SKIP TO FIRST B +PATTERN (A+ B) +DEFINE +A AS AVG(A.price) < 15 +) MR; +{% endhighlight %} + +Given this query and following input values: + +{% highlight text %} +symbol rowtime pricetax +== === === +'ACME' '01-Apr-11 10:00:00' 12 1 +'ACME' '01-Apr-11 10:00:01' 17 2 +'ACME' '01-Apr-11 10:00:02' 13 1 +'ACME' '01-Apr-11 10:00:03' 16 3 +'ACME' '01-Apr-11 10:00:04' 25 2 +'ACME' '01-Apr-11 10:00:05' 2 1 +'ACME' '01-Apr-11 10:00:06' 4 1 +'ACME' '01-Apr-11 10:00:07' 10 2 +'ACME' '01-Apr-11 10:00:08' 15 2 +'ACME' '01-Apr-11 10:00:09' 25 2 +'ACME' '01-Apr-11 10:00:10' 30 1 +{% endhighlight %} + +The query will accumulate events as part of `A` pattern variable as long as the average price of them does not exceed 15. Which will happen at `01-Apr-11 10:00:04`. The next such period that starts then will +exceed average price of 15 at `01-Apr-11 10:00:10`. Thus the results for said query will be: + +{% highlight text %} + symbol start_tstamp end_tstamp avgPrice += == == +ACME 01-APR-11 10:00:00 01-APR-11 10:00:03 14.5 +ACME 01-APR-11 10:00:04 01-APR-11 10:00:09 13.5 +{% endhighlight %} + +An important thing to have in mind is how aggregates behave in situation when no rows where mapped to certain pattern variable. Every aggregate, beside `COUNT` will produce `null` in those cases. `COUNT` on the other hand will Review comment: You are right, didn't think of UDAGs. Do you think it is worth adding such disclaimer at all. I've added it in the first place, because there was something similar in the SQL standard, but there are no UDAGs there. 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
[GitHub] dawidwys commented on a change in pull request #7177: [FLINK-7599] [table] Support for aggregates in MATCH_RECOGNIZE
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) +} + +
[GitHub] dawidwys commented on a change in pull request #7177: [FLINK-7599] [table] Support for aggregates in MATCH_RECOGNIZE
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_r237529902 ## 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) +} + +
[GitHub] dawidwys commented on a change in pull request #7177: [FLINK-7599] [table] Support for aggregates in MATCH_RECOGNIZE
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_r237529378 ## 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) +} + +
[GitHub] dawidwys commented on a change in pull request #7177: [FLINK-7599] [table] Support for aggregates in MATCH_RECOGNIZE
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_r237529378 ## 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) +} + +
[GitHub] dawidwys commented on a change in pull request #7177: [FLINK-7599] [table] Support for aggregates in MATCH_RECOGNIZE
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_r237526604 ## 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: Hmmm, no I am not. Have you maybe missed the `.getField()` call? 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
[GitHub] dawidwys commented on a change in pull request #7177: [FLINK-7599] [table] Support for aggregates in MATCH_RECOGNIZE
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_r237526604 ## 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: Hmmm, no I am not. Have you maybe missed the `.getField()`? 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