dianfu commented on a change in pull request #8974: 
[FLINK-13082][table-planner-blink] Support MatchRecognize in blink planner
URL: https://github.com/apache/flink/pull/8974#discussion_r300523549
 
 

 ##########
 File path: 
flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/MatchRecognizeITCase.scala
 ##########
 @@ -0,0 +1,778 @@
+/*
+ * 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.runtime.stream.sql
+
+import org.apache.flink.api.common.time.Time
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo
+import org.apache.flink.api.scala._
+import org.apache.flink.streaming.api.TimeCharacteristic
+import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.api.{TableConfig, Types}
+import org.apache.flink.table.functions.{AggregateFunction, FunctionContext, 
ScalarFunction}
+import org.apache.flink.table.plan.util.JavaUserDefinedAggFunctions.WeightedAvg
+import 
org.apache.flink.table.runtime.utils.StreamingWithStateTestBase.StateBackendMode
+import 
org.apache.flink.table.runtime.utils.TimeTestUtil.EventTimeSourceFunction
+import org.apache.flink.table.runtime.utils.{StreamingWithStateTestBase, 
TestingAppendSink, UserDefinedFunctionTestUtils}
+import org.apache.flink.types.Row
+import org.junit.Assert.assertEquals
+import org.junit.Test
+import org.junit.runner.RunWith
+import org.junit.runners.Parameterized
+
+import java.sql.Timestamp
+import java.util.TimeZone
+
+import scala.collection.mutable
+
+@RunWith(classOf[Parameterized])
+class MatchRecognizeITCase(backend: StateBackendMode) extends 
StreamingWithStateTestBase(backend) {
+
+  @Test
+  def testSimplePattern(): Unit = {
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    val tEnv = StreamTableEnvironment.create(env)
+
+    val data = new mutable.MutableList[(Int, String)]
+    data.+=((1, "a"))
+    data.+=((2, "z"))
+    data.+=((3, "b"))
+    data.+=((4, "c"))
+    data.+=((5, "d"))
+    data.+=((6, "a"))
+    data.+=((7, "b"))
+    data.+=((8, "c"))
+    data.+=((9, "h"))
+
+    val t = env.fromCollection(data).toTable(tEnv,'id, 'name, 'proctime)
+    tEnv.registerTable("MyTable", t)
+
+    val sqlQuery =
+      s"""
+         |SELECT T.aid, T.bid, T.cid
+         |FROM MyTable
+         |MATCH_RECOGNIZE (
+         |  ORDER BY proctime
+         |  MEASURES
+         |    `A"`.id AS aid,
+         |    \u006C.id AS bid,
+         |    C.id AS cid
+         |  PATTERN (`A"` \u006C C)
+         |  DEFINE
+         |    `A"` AS name = 'a',
+         |    \u006C AS name = 'b',
+         |    C AS name = 'c'
+         |) AS T
+         |""".stripMargin
+
+    val sink = new TestingAppendSink()
+    val result = tEnv.sqlQuery(sqlQuery).toAppendStream[Row]
+    result.addSink(sink)
+    env.execute()
+
+    val expected = mutable.MutableList("6,7,8")
+    assertEquals(expected.sorted, sink.getAppendResults.sorted)
+  }
+
+  @Test
+  def testSimplePatternWithNulls(): Unit = {
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    val tEnv = StreamTableEnvironment.create(env)
+
+    val data = new mutable.MutableList[(Int, String, String)]
+    data.+=((1, "a", null))
+    data.+=((2, "b", null))
+    data.+=((3, "c", null))
+    data.+=((4, "d", null))
+    data.+=((5, null, null))
+    data.+=((6, "a", null))
+    data.+=((7, "b", null))
+    data.+=((8, "c", null))
+    data.+=((9, null, null))
+
+    val t = env.fromCollection(data).toTable(tEnv,'id, 'name, 'nullField, 
'proctime)
+    tEnv.registerTable("MyTable", t)
+
+    val sqlQuery =
+      s"""
+         |SELECT T.aid, T.bNull, T.cid, T.aNull
+         |FROM MyTable
+         |MATCH_RECOGNIZE (
+         |  ORDER BY proctime
+         |  MEASURES
+         |    A.id AS aid,
+         |    A.nullField AS aNull,
+         |    LAST(B.nullField) AS bNull,
+         |    C.id AS cid
+         |  PATTERN (A B C)
+         |  DEFINE
+         |    A AS name = 'a' AND nullField IS NULL,
+         |    B AS name = 'b' AND LAST(A.nullField) IS NULL,
+         |    C AS name = 'c'
+         |) AS T
+         |""".stripMargin
+
+    val sink = new TestingAppendSink()
+    val result = tEnv.sqlQuery(sqlQuery).toAppendStream[Row]
+    result.addSink(sink)
+    env.execute()
+
+    val expected = mutable.MutableList("1,null,3,null", "6,null,8,null")
+    assertEquals(expected.sorted, sink.getAppendResults.sorted)
+  }
+
+  @Test
+  def testCodeSplitsAreProperlyGenerated(): Unit = {
+    val env = StreamExecutionEnvironment.getExecutionEnvironment
+    val tableConfig = new TableConfig
+    tableConfig.setMaxGeneratedCodeLength(1)
 
 Review comment:
   The code split config is not considered in Blink yet. What about adding a 
TODO here?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
[email protected]


With regards,
Apache Git Services

Reply via email to