wuchong commented on a change in pull request #10098: 
[FLINK-14326][FLINK-14324][table] Support to apply watermark assigner in planner
URL: https://github.com/apache/flink/pull/10098#discussion_r343516296
 
 

 ##########
 File path: 
flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/codegen/WatermarkGeneratorCodeGenTest.scala
 ##########
 @@ -0,0 +1,123 @@
+/*
+ * 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.planner.codegen
+
+import org.apache.flink.configuration.Configuration
+import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
+import org.apache.flink.streaming.util.MockStreamingRuntimeContext
+import org.apache.flink.table.api.scala.internal.StreamTableEnvironmentImpl
+import org.apache.flink.table.api.{EnvironmentSettings, TableConfig}
+import org.apache.flink.table.dataformat.GenericRow
+import org.apache.flink.table.planner.calcite.FlinkTypeFactory
+import org.apache.flink.table.planner.delegation.StreamPlanner
+import 
org.apache.flink.table.planner.runtime.utils.JavaUserDefinedScalarFunctions.JavaFunc5
+import org.apache.flink.table.runtime.operators.wmassigners.WatermarkGenerator
+import org.apache.flink.table.types.logical.{IntType, TimestampType}
+import org.junit.Assert.assertEquals
+import org.junit.Assert.assertTrue
+import org.junit.Test
+
+import java.lang.{Integer => JInt, Long => JLong}
+
+/**
+  * Tests the generated [[WatermarkGenerator]] from 
[[WatermarkGeneratorCodeGenerator]].
+  */
+class WatermarkGeneratorCodeGenTest {
+
+  val tEnv: StreamTableEnvironmentImpl = StreamTableEnvironmentImpl.create(
+    StreamExecutionEnvironment.getExecutionEnvironment,
+    EnvironmentSettings.newInstance().useBlinkPlanner().build(),
+    new TableConfig)
+  val data = List(
+    GenericRow.of(JLong.valueOf(1000L), JInt.valueOf(5)),
+    GenericRow.of(null, JInt.valueOf(4)),
+    GenericRow.of(JLong.valueOf(3000L), null),
+    GenericRow.of(JLong.valueOf(5000L), JInt.valueOf(3)),
+    GenericRow.of(JLong.valueOf(4000L), JInt.valueOf(10)),
+    GenericRow.of(JLong.valueOf(6000L), JInt.valueOf(8))
+  )
+
+  @Test
+  def testAscendingWatermark(): Unit = {
+    val generator = generateWatermarkGenerator("ts - INTERVAL '0.001' SECOND")
+    val results = data.map(d => generator.currentWatermark(d))
+    val expected = List(
+      JLong.valueOf(999L),
+      null,
+      JLong.valueOf(2999),
+      JLong.valueOf(4999),
+      JLong.valueOf(3999),
+      JLong.valueOf(5999))
+    assertEquals(expected, results)
+  }
+
+  @Test
+  def testBoundedOutOfOrderWatermark(): Unit = {
+    val generator = generateWatermarkGenerator("ts - INTERVAL '5' SECOND")
+    val results = data.map(d => generator.currentWatermark(d))
+    val expected = List(
+      JLong.valueOf(-4000L),
+      null,
+      JLong.valueOf(-2000L),
+      JLong.valueOf(0L),
+      JLong.valueOf(-1000L),
+      JLong.valueOf(1000L))
+    assertEquals(expected, results)
+  }
+
+  @Test
+  def testCustomizedWatermark(): Unit = {
+    JavaFunc5.openCalled = false
+    JavaFunc5.closeCalled = false
+    tEnv.registerFunction("myFunc", new JavaFunc5)
+    val generator = generateWatermarkGenerator("myFunc(ts, `offset`)")
+    // mock open and close invoking
+    generator.setRuntimeContext(new MockStreamingRuntimeContext(false, 1, 1))
+    generator.open(new Configuration())
+    val results = data.map(d => generator.currentWatermark(d))
+    generator.close()
+    val expected = List(
+      JLong.valueOf(995L),
+      null,
+      null,
+      JLong.valueOf(4997L),
+      JLong.valueOf(3990L),
+      JLong.valueOf(5992L))
+    assertEquals(expected, results)
+    assertTrue(JavaFunc5.openCalled)
+    assertTrue(JavaFunc5.closeCalled)
+  }
+
+  private def generateWatermarkGenerator(expr: String): WatermarkGenerator = {
+    val planner = tEnv.getPlanner.asInstanceOf[StreamPlanner]
+    val tableRowType = planner.getTypeFactory.buildRelNodeRowType(
 
 Review comment:
   However, I need to register UDFs in the tests. And mocking 
`FlinkPlannerImpl` is not easy. 
   I think creating a `TableEnvironment` is not much overhead, it will not 
spawn cluster. 

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