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

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

dawidwys commented on a change in pull request #6815:  [FLINK-7062][cep][table] 
Added basic support for MATCH_RECOGNIZE
URL: https://github.com/apache/flink/pull/6815#discussion_r225459604
 
 

 ##########
 File path: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/MatchCodeGenerator.scala
 ##########
 @@ -0,0 +1,320 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.codegen
+
+import java.util
+
+import org.apache.calcite.rex._
+import org.apache.calcite.sql.fun.SqlStdOperatorTable._
+import org.apache.flink.api.common.functions._
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.java.typeutils.ListTypeInfo
+import org.apache.flink.cep.pattern.conditions.IterativeCondition
+import org.apache.flink.cep.{PatternFlatSelectFunction, PatternSelectFunction}
+import org.apache.flink.table.api.{TableConfig, TableException}
+import org.apache.flink.table.codegen.CodeGenUtils.{boxedTypeTermForTypeInfo, 
newName, primitiveDefaultValue}
+import org.apache.flink.table.codegen.Indenter.toISC
+import org.apache.flink.table.plan.schema.RowSchema
+import org.apache.flink.util.Collector
+import org.apache.flink.util.MathUtils.checkedDownCast
+import java.lang.{Long => JLong}
+
+import scala.collection.JavaConverters._
+
+/**
+  * A code generator for generating CEP related functions.
+  *
+  * @param config configuration that determines runtime behavior
+  * @param nullableInput input(s) can be null.
+  * @param input type information about the first input of the Function
+  * @param currentPattern if generating condition the name of pattern, which 
the condition will
+  *                       be applied to
+  */
+class MatchCodeGenerator(
+    config: TableConfig,
+    nullableInput: Boolean,
+    input: TypeInformation[_ <: Any],
+    currentPattern: Option[String] = None)
+  extends CodeGenerator(config, nullableInput, input){
+
+  def generateMatchFunction[F <: Function, T <: Any](
+    name: String,
+    clazz: Class[F],
+    bodyCode: String,
+    returnType: TypeInformation[T])
+  : GeneratedFunction[F, T] = {
+    // This is a separate method from FunctionCodeGenerator#generateFunction 
because as of now
+    // functions in CEP library do not support rich interfaces
+    val funcName = newName(name)
+    val collectorTypeTerm = classOf[Collector[Any]].getCanonicalName
+    val (functionClass, signature, inputStatements, isInterface) =
+      if (clazz == classOf[IterativeCondition[_]]) {
+        val baseClass = classOf[IterativeCondition[_]]
+        val inputTypeTerm = boxedTypeTermForTypeInfo(input)
+        val contextType = 
classOf[IterativeCondition.Context[_]].getCanonicalName
+
+        (baseClass,
+          s"boolean filter( Object _in1, $contextType $contextTerm)",
+          List(s"$inputTypeTerm $input1Term = ($inputTypeTerm) _in1;"),
+          false)
+      } else if (clazz == classOf[PatternSelectFunction[_, _]]) {
+        val baseClass = classOf[PatternSelectFunction[_, _]]
+        val inputTypeTerm =
+          s"java.util.Map<String, 
java.util.List<${boxedTypeTermForTypeInfo(input)}>>"
+
+        (baseClass,
+          s"Object select($inputTypeTerm $input1Term)",
+          List(),
+          true)
+      } else if (clazz == classOf[PatternFlatSelectFunction[_, _]]) {
+        val baseClass = classOf[PatternFlatSelectFunction[_, _]]
+        val inputTypeTerm =
+          s"java.util.Map<String, 
java.util.List<${boxedTypeTermForTypeInfo(input)}>>"
+
+        (baseClass,
+          s"void flatSelect($inputTypeTerm $input1Term, $collectorTypeTerm 
$collectorTerm)",
+          List(),
+          true)
+      } else {
+        throw new CodeGenException("Unsupported Function.")
+      }
+
+    val extendsKeyword = if (isInterface) "implements" else "extends"
+    val funcCode = j"""
+      |public class $funcName $extendsKeyword 
${functionClass.getCanonicalName} {
+      |
+      |  ${reuseMemberCode()}
+      |
+      |  public $funcName() throws Exception {
+      |    ${reuseInitCode()}
+      |  }
+      |
+      |  @Override
+      |  public $signature throws Exception {
+      |    ${inputStatements.mkString("\n")}
+      |    ${reusePerRecordCode()}
+      |    ${reuseInputUnboxingCode()}
+      |    $bodyCode
+      |  }
+      |}
+    """.stripMargin
+
+    GeneratedFunction(funcName, returnType, funcCode)
+  }
+
+  def generatePartitionKeyAccess(
+    partitionKey: RexInputRef
+  ) : GeneratedExpression = {
+    // extracts partition keys from any element of the match
+    val eventNameTerm = newName("event")
+    val eventTypeTerm = boxedTypeTermForTypeInfo(input)
+
+    val keyAccess = generateFieldAccess(input, eventNameTerm, 
partitionKey.getIndex)
+
+    val keyCode = s"""
+       |$eventTypeTerm $eventNameTerm = null;
+       |for (java.util.Map.Entry entry : $input1Term.entrySet()) {
+       |  java.util.List<$eventTypeTerm> value = 
(java.util.List<$eventTypeTerm>) entry.getValue();
+       |  if (value != null && value.size() > 0) {
+       |    $eventNameTerm = ($eventTypeTerm) value.get(0);
+       |    break;
+       |  }
+       |}
+       |
+       |${keyAccess.code}
+       """.stripMargin
+
+    keyAccess.copy(code = keyCode)
+  }
+
+  def generateOneRowPerMatchExpression(
+    partitionKeys: util.List[RexNode],
+    measures: util.Map[String, RexNode],
+    returnType: RowSchema
+  ): GeneratedExpression = {
+    // For "ONE ROW PER MATCH", the output columns include:
+    // 1) the partition columns;
+    // 2) the columns defined in the measures clause.
+    val resultExprs =
+      partitionKeys.asScala.map { case inputRef: RexInputRef =>
+        generatePartitionKeyAccess(inputRef)
+      } ++ returnType.fieldNames.filter(measures.containsKey(_)).map { 
fieldName =>
+        generateExpression(measures.get(fieldName))
+      }
+
+    generateResultExpression(
+      resultExprs,
+      returnType.typeInfo,
+      returnType.fieldNames)
+  }
+
+  override def visitCall(call: RexCall): GeneratedExpression = {
+    call.getOperator match {
+      case PREV | NEXT =>
+        val countLiteral = call.operands.get(1).asInstanceOf[RexLiteral]
+        val count = countLiteral.getValueAs(classOf[JLong])
+        if (count != 0) {
+          throw new TableException("Flink does not support physical offsets 
within partition.")
+        } else {
+          val patternFieldVisitor = new RexPatternFieldRefVisitor(config,
+            nullableInput,
+            input,
+            0,
+            false)
+          call.getOperands.get(0).accept(patternFieldVisitor)
+        }
+
+      case MATCH_NUMBER =>
+        throw new TableException(s"Unsupported call: $call")
+
+      case FIRST | LAST =>
+        val countLiteral = call.operands.get(1).asInstanceOf[RexLiteral]
+        val count = checkedDownCast(countLiteral.getValueAs(classOf[JLong]))
+        val patternRef = call.operands.get(0)
+
+        val patternFieldVisitor = new RexPatternFieldRefVisitor(config,
+          nullableInput,
+          input,
+          count,
+          call.getOperator == FIRST)
+        patternRef.accept(patternFieldVisitor)
+
+      case CLASSIFIER | RUNNING =>
+        throw new TableException(s"${call.getOperator} is not supported yet.")
+      case FINAL =>
+        call.getOperands.get(0).accept(this)
+
+      case _ => super.visitCall(call)
+    }
+  }
+
+  override def visitPatternFieldRef(fieldRef: RexPatternFieldRef): 
GeneratedExpression = {
+    val patternFieldRefVisitor = new RexPatternFieldRefVisitor(config,
+      nullableInput,
+      input,
+      0,
+      false)
+    fieldRef.accept(patternFieldRefVisitor)
+  }
+
+  class RexPatternFieldRefVisitor(
+    config: TableConfig,
+    nullableInput: Boolean,
+    input: TypeInformation[_ <: Any],
+    offset: Int,
+    first: Boolean) extends CodeGenerator(config, nullableInput, input) {
+
+    override def visitPatternFieldRef(fieldRef: RexPatternFieldRef): 
GeneratedExpression = {
+      if (fieldRef.getAlpha.equals("*")) {
+        generateFieldAccess(input, input1Term, fieldRef.getIndex)
+      } else {
+        generatePatternFieldRef(fieldRef)
+      }
+    }
+
+    private def generateDefinePatternVariableExp(
+      patternName: String,
+      currentPattern: String)
+    : GeneratedExpression = {
+      val listTypeTerm = classOf[java.util.List[_]].getCanonicalName
+      val listName = newName("patternEvents")
+      val eventTypeTerm = boxedTypeTermForTypeInfo(input)
+      val eventNameTerm = newName("event")
+
+      val addCurrent = if (currentPattern == patternName) {
+        s"""
+           |$listName.add($input1Term);
+         """.stripMargin
+      } else {
+        ""
+      }
+
+      val code = s"""
+         |$listTypeTerm $listName = new java.util.ArrayList();
+         |for ($eventTypeTerm $eventNameTerm : 
$contextTerm.getEventsForPattern("$patternName")) {
+         |  $listName.add($eventNameTerm);
+         |}
+         |$addCurrent
+         |""".stripMargin
+
+      GeneratedExpression(listName, "false", code, new ListTypeInfo(input))
+    }
+
+    private def generateMeasurePatternVariableExp(patternName: String): 
GeneratedExpression = {
+      val listTypeTerm = classOf[java.util.List[_]].getCanonicalName
+      val listName = newName("patternEvents")
+
+      val code = s"""
+         |$listTypeTerm $listName = ($listTypeTerm) 
$input1Term.get("$patternName");
+         |if ($listName == null) {
+         |  $listName = java.util.Collections.emptyList();
+         |}
+         |""".stripMargin
+
+      GeneratedExpression(listName, "false", code, new ListTypeInfo(input))
+    }
+
+    private def findEventByLogicalPosition(
+      patternName: String
+    ): GeneratedExpression = {
+      val rowNameTerm = newName("row")
+      val rowDefaultValue = primitiveDefaultValue(input)
+      val eventTypeTerm = boxedTypeTermForTypeInfo(input)
+
+      val findEventsByPatternName = currentPattern match {
+        case Some(p) => generateDefinePatternVariableExp(patternName, p)
+        case None => generateMeasurePatternVariableExp(patternName)
+      }
+
+      val listName = findEventsByPatternName.resultTerm
+      val resultIndex = if (first) {
+        s"""$offset"""
+      } else {
+        s"""$listName.size() - $offset - 1"""
+      }
+
+      val getResult =
+        s"""
+           |$eventTypeTerm $rowNameTerm = $rowDefaultValue;
+           |if ($listName.size() > $offset) {
+           |  $rowNameTerm = (($eventTypeTerm) $listName.get($resultIndex));
+           |}
+           |""".stripMargin
+
+      val funcCode = s"""
+         |${findEventsByPatternName.code}
+         |$getResult
+         |""".stripMargin
+
+      GeneratedExpression(rowNameTerm, s"$rowNameTerm == null", funcCode, 
input)
 
 Review comment:
   Yes it is, e.g. in `testLogicalOffsets` in condition `DOWN AS price < 
LAST(DOWN.price, 1) OR LAST(DOWN.price, 1) IS NULL`
   
   I've also added a test where it is checked for `MEASURES` clause.

----------------------------------------------------------------
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 the basic functionality of MATCH_RECOGNIZE
> --------------------------------------------------
>
>                 Key: FLINK-7062
>                 URL: https://issues.apache.org/jira/browse/FLINK-7062
>             Project: Flink
>          Issue Type: Sub-task
>          Components: CEP, Table API &amp; SQL
>            Reporter: Dian Fu
>            Assignee: Dian Fu
>            Priority: Major
>              Labels: pull-request-available
>
> In this JIRA, we will support the basic functionality of {{MATCH_RECOGNIZE}} 
> in Flink SQL API which includes the support of syntax {{MEASURES}}, 
> {{PATTERN}} and {{DEFINE}}. This would allow users write basic cep use cases 
> with SQL like the following example:
> {code}
> SELECT T.aid, T.bid, T.cid
> FROM MyTable
> MATCH_RECOGNIZE (
>   MEASURES
>     A.id AS aid,
>     B.id AS bid,
>     C.id AS cid
>   PATTERN (A B C)
>   DEFINE
>     A AS A.name = 'a',
>     B AS B.name = 'b',
>     C AS C.name = 'c'
> ) AS T
> {code}



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

Reply via email to