asfgit closed pull request #7014: [FLINK-10770] [Table & SQL] Some generated 
functions are not opened properly.
URL: https://github.com/apache/flink/pull/7014
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git 
a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowMapRunner.scala
 
b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowMapRunner.scala
index 54bac601bcc..cb451b92cd7 100644
--- 
a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowMapRunner.scala
+++ 
b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowMapRunner.scala
@@ -18,6 +18,7 @@
 
 package org.apache.flink.table.runtime
 
+import org.apache.flink.api.common.functions.util.FunctionUtils
 import org.apache.flink.api.common.functions.{MapFunction, RichMapFunction}
 import org.apache.flink.api.common.typeinfo.TypeInformation
 import org.apache.flink.api.java.typeutils.ResultTypeQueryable
@@ -46,6 +47,8 @@ class CRowMapRunner[OUT](
     val clazz = compile(getRuntimeContext.getUserCodeClassLoader, name, code)
     LOG.debug("Instantiating MapFunction.")
     function = clazz.newInstance()
+    FunctionUtils.setFunctionRuntimeContext(function, getRuntimeContext)
+    FunctionUtils.openFunction(function, parameters)
   }
 
   override def map(in: CRow): OUT = {
@@ -53,4 +56,8 @@ class CRowMapRunner[OUT](
   }
 
   override def getProducedType: TypeInformation[OUT] = returnType
+
+  override def close(): Unit = {
+    FunctionUtils.closeFunction(function)
+  }
 }
diff --git 
a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowOutputProcessRunner.scala
 
b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowOutputProcessRunner.scala
index 600b8987a28..ebef577ff83 100644
--- 
a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowOutputProcessRunner.scala
+++ 
b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/CRowOutputProcessRunner.scala
@@ -18,6 +18,7 @@
 
 package org.apache.flink.table.runtime
 
+import org.apache.flink.api.common.functions.util.FunctionUtils
 import org.apache.flink.api.common.typeinfo.TypeInformation
 import org.apache.flink.api.java.typeutils.ResultTypeQueryable
 import org.apache.flink.configuration.Configuration
@@ -49,6 +50,8 @@ class CRowOutputProcessRunner(
     val clazz = compile(getRuntimeContext.getUserCodeClassLoader, name, code)
     LOG.debug("Instantiating ProcessFunction.")
     function = clazz.newInstance()
+    FunctionUtils.setFunctionRuntimeContext(function, getRuntimeContext)
+    FunctionUtils.openFunction(function, parameters)
 
     this.cRowWrapper = new CRowWrappingCollector()
     this.cRowWrapper.setChange(true)
@@ -68,4 +71,8 @@ class CRowOutputProcessRunner(
   }
 
   override def getProducedType: TypeInformation[CRow] = returnType
+
+  override def close(): Unit = {
+    FunctionUtils.closeFunction(function)
+  }
 }
diff --git 
a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/FlatJoinRunner.scala
 
b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/FlatJoinRunner.scala
index 0bf65694367..26ed962e01a 100644
--- 
a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/FlatJoinRunner.scala
+++ 
b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/FlatJoinRunner.scala
@@ -18,6 +18,7 @@
 
 package org.apache.flink.table.runtime
 
+import org.apache.flink.api.common.functions.util.FunctionUtils
 import org.apache.flink.api.common.functions.{FlatJoinFunction, 
RichFlatJoinFunction}
 import org.apache.flink.api.common.typeinfo.TypeInformation
 import org.apache.flink.api.java.typeutils.ResultTypeQueryable
@@ -42,10 +43,16 @@ class FlatJoinRunner[IN1, IN2, OUT](
     val clazz = compile(getRuntimeContext.getUserCodeClassLoader, name, code)
     LOG.debug("Instantiating FlatJoinFunction.")
     function = clazz.newInstance()
+    FunctionUtils.setFunctionRuntimeContext(function, getRuntimeContext)
+    FunctionUtils.openFunction(function, parameters)
   }
 
   override def join(first: IN1, second: IN2, out: Collector[OUT]): Unit =
     function.join(first, second, out)
 
   override def getProducedType: TypeInformation[OUT] = returnType
+
+  override def close(): Unit = {
+    FunctionUtils.closeFunction(function)
+  }
 }
diff --git 
a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapRunner.scala
 
b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapRunner.scala
index 00d18ecc007..7f56ffe72cc 100644
--- 
a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapRunner.scala
+++ 
b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapRunner.scala
@@ -18,6 +18,7 @@
 
 package org.apache.flink.table.runtime
 
+import org.apache.flink.api.common.functions.util.FunctionUtils
 import org.apache.flink.api.common.functions.{MapFunction, RichMapFunction}
 import org.apache.flink.api.common.typeinfo.TypeInformation
 import org.apache.flink.api.java.typeutils.ResultTypeQueryable
@@ -41,10 +42,16 @@ class MapRunner[IN, OUT](
     val clazz = compile(getRuntimeContext.getUserCodeClassLoader, name, code)
     LOG.debug("Instantiating MapFunction.")
     function = clazz.newInstance()
+    FunctionUtils.setFunctionRuntimeContext(function, getRuntimeContext)
+    FunctionUtils.openFunction(function, parameters)
   }
 
   override def map(in: IN): OUT =
     function.map(in)
 
   override def getProducedType: TypeInformation[OUT] = returnType
+
+  override def close(): Unit = {
+    FunctionUtils.closeFunction(function)
+  }
 }
diff --git 
a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapSideJoinRunner.scala
 
b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapSideJoinRunner.scala
index 5f5a2cc4a19..f15aaa6432e 100644
--- 
a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapSideJoinRunner.scala
+++ 
b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/MapSideJoinRunner.scala
@@ -18,6 +18,7 @@
 
 package org.apache.flink.table.runtime
 
+import org.apache.flink.api.common.functions.util.FunctionUtils
 import org.apache.flink.api.common.functions.{FlatJoinFunction, 
RichFlatMapFunction}
 import org.apache.flink.api.common.typeinfo.TypeInformation
 import org.apache.flink.api.java.typeutils.ResultTypeQueryable
@@ -43,6 +44,8 @@ abstract class MapSideJoinRunner[IN1, IN2, SINGLE_IN, 
MULTI_IN, OUT](
     val clazz = compile(getRuntimeContext.getUserCodeClassLoader, name, code)
     LOG.debug("Instantiating FlatJoinFunction.")
     function = clazz.newInstance()
+    FunctionUtils.setFunctionRuntimeContext(function, getRuntimeContext)
+    FunctionUtils.openFunction(function, parameters)
     broadcastSet = retrieveBroadcastSet
   }
 
@@ -56,4 +59,8 @@ abstract class MapSideJoinRunner[IN1, IN2, SINGLE_IN, 
MULTI_IN, OUT](
   }
 
   override def getProducedType: TypeInformation[OUT] = returnType
+
+  override def close(): Unit = {
+    FunctionUtils.closeFunction(function)
+  }
 }
diff --git 
a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowJoin.scala
 
b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowJoin.scala
index 0fe2e39c3a7..1463e7022c7 100644
--- 
a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowJoin.scala
+++ 
b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/NonWindowJoin.scala
@@ -18,6 +18,7 @@
 package org.apache.flink.table.runtime.join
 
 import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.functions.util.FunctionUtils
 import org.apache.flink.api.common.state.{MapState, MapStateDescriptor, 
ValueState, ValueStateDescriptor}
 import org.apache.flink.api.common.typeinfo.TypeInformation
 import org.apache.flink.api.java.tuple.{Tuple2 => JTuple2}
@@ -85,6 +86,8 @@ abstract class NonWindowJoin(
       genJoinFuncCode)
 
     joinFunction = clazz.newInstance()
+    FunctionUtils.setFunctionRuntimeContext(joinFunction, getRuntimeContext)
+    FunctionUtils.openFunction(joinFunction, parameters)
 
     // initialize left and right state, the first element of tuple2 indicates 
how many rows of
     // this row, while the second element represents the expired time of this 
row.
@@ -291,4 +294,8 @@ abstract class NonWindowJoin(
       joinFunction.join(otherSideRow, inputRow, cRowWrapper)
     }
   }
+
+  override def close(): Unit = {
+    FunctionUtils.closeFunction(joinFunction)
+  }
 }
diff --git 
a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamJoin.scala
 
b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamJoin.scala
index a44bdef2e09..1bdfcb5be0f 100644
--- 
a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamJoin.scala
+++ 
b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/TimeBoundedStreamJoin.scala
@@ -22,6 +22,7 @@ import java.util
 import java.util.{List => JList}
 
 import org.apache.flink.api.common.functions.FlatJoinFunction
+import org.apache.flink.api.common.functions.util.FunctionUtils
 import org.apache.flink.api.common.state._
 import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation}
 import org.apache.flink.api.java.operators.join.JoinType
@@ -110,6 +111,8 @@ abstract class TimeBoundedStreamJoin(
       genJoinFuncCode)
     LOG.debug("Instantiating JoinFunction.")
     joinFunction = clazz.newInstance()
+    FunctionUtils.setFunctionRuntimeContext(joinFunction, getRuntimeContext)
+    FunctionUtils.openFunction(joinFunction, config)
 
     joinCollector = new EmitAwareCollector()
     joinCollector.setCRowChange(true)
@@ -492,6 +495,11 @@ abstract class TimeBoundedStreamJoin(
     }
   }
 
+
+  override def close(): Unit = {
+    FunctionUtils.closeFunction(joinFunction)
+  }
+
   /**
     * Update the operator time of the two streams.
     * Must be the first call in all processing methods (i.e., 
processElement(), onTimer()).
diff --git 
a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/outerJoinGroupReduceRunners.scala
 
b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/outerJoinGroupReduceRunners.scala
index 9b0f08ea4a1..23ad51a0eff 100644
--- 
a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/outerJoinGroupReduceRunners.scala
+++ 
b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/outerJoinGroupReduceRunners.scala
@@ -19,6 +19,7 @@ package org.apache.flink.table.runtime
 
 import java.lang.Iterable
 
+import org.apache.flink.api.common.functions.util.FunctionUtils
 import org.apache.flink.api.common.functions.{JoinFunction, 
RichGroupReduceFunction}
 import org.apache.flink.api.common.typeinfo.TypeInformation
 import org.apache.flink.configuration.Configuration
@@ -41,6 +42,12 @@ abstract class OuterJoinGroupReduceRunner(
     val clazz = compile(getRuntimeContext.getUserCodeClassLoader, name, code)
     LOG.debug("Instantiating JoinFunction.")
     function = clazz.newInstance()
+    FunctionUtils.setFunctionRuntimeContext(function, getRuntimeContext)
+    FunctionUtils.openFunction(function, config)
+  }
+
+  override def close(): Unit = {
+    FunctionUtils.closeFunction(function)
   }
 }
 
diff --git 
a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/outerJoinRunners.scala
 
b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/outerJoinRunners.scala
index a9e0211c40f..fd3b46fbf63 100644
--- 
a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/outerJoinRunners.scala
+++ 
b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/outerJoinRunners.scala
@@ -20,6 +20,7 @@ package org.apache.flink.table.runtime
 
 import java.lang.{Boolean => JBool}
 
+import org.apache.flink.api.common.functions.util.FunctionUtils
 import org.apache.flink.api.common.functions.{JoinFunction, 
RichFlatJoinFunction}
 import org.apache.flink.api.common.typeinfo.TypeInformation
 import org.apache.flink.api.java.typeutils.ResultTypeQueryable
@@ -45,9 +46,15 @@ abstract class OuterJoinRunner(
     val clazz = compile(getRuntimeContext.getUserCodeClassLoader, name, code)
     LOG.debug("Instantiating FlatJoinFunction.")
     function = clazz.newInstance()
+    FunctionUtils.setFunctionRuntimeContext(function, getRuntimeContext)
+    FunctionUtils.openFunction(function, parameters)
   }
 
   override def getProducedType: TypeInformation[Row] = returnType
+
+  override def close(): Unit = {
+    FunctionUtils.closeFunction(function)
+  }
 }
 
 /**
diff --git 
a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/table/UdfITCase.scala
 
b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/table/UdfITCase.scala
new file mode 100644
index 00000000000..0b9506b7a96
--- /dev/null
+++ 
b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/table/UdfITCase.scala
@@ -0,0 +1,72 @@
+/*
+ * 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.batch.table
+
+import scala.collection.mutable
+
+import org.apache.flink.api.scala._
+import org.apache.flink.api.scala.ExecutionEnvironment
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.api.TableEnvironment
+import org.apache.flink.table.api.scala.concat
+import org.apache.flink.table.runtime.utils.{TableProgramsClusterTestBase}
+import 
org.apache.flink.table.runtime.utils.TableProgramsTestBase.TableConfigMode
+import org.apache.flink.test.util.MultipleProgramsTestBase.TestExecutionMode
+import org.apache.flink.test.util.TestBaseUtils
+import org.apache.flink.types.Row
+import org.junit.Test
+import org.junit.runner.RunWith
+import org.junit.runners.Parameterized
+import scala.collection.JavaConverters._
+
+import org.apache.flink.table.utils.ScalarFunction0
+
+@RunWith(classOf[Parameterized])
+class UdfITCase(
+    mode: TestExecutionMode,
+    configMode: TableConfigMode)
+  extends TableProgramsClusterTestBase(mode, configMode) {
+
+  @Test
+  def testUdfOpen(): Unit = {
+    val env: ExecutionEnvironment = 
ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = TableEnvironment.getTableEnvironment(env, config)
+
+    val data1 = new mutable.MutableList[(Int, String)]
+    data1.+=((1, "Hi1"))
+    data1.+=((2, "Hi2"))
+    data1.+=((3, "Hi3"))
+
+    val data2 = new mutable.MutableList[(Int, String)]
+    data2.+=((1, "Hello1"))
+    data2.+=((2, "Hello2"))
+    data2.+=((3, "Hello3"))
+
+    val t1 = env.fromCollection(data1).toTable(tEnv, 'a, 'b)
+    val t2 = env.fromCollection(data2).toTable(tEnv, 'c, 'd)
+
+    val fun0 = new ScalarFunction0
+    val t = t1.join(t2, 'a === 'c ).select('a, 'b, 
'd).where(fun0(concat('b,'d)))
+
+    val results = t.toDataSet[Row].collect()
+    val expected = Seq("1,Hi1,Hello1", "2,Hi2,Hello2", 
"3,Hi3,Hello3").mkString("\n")
+    TestBaseUtils.compareResultAsText(results.asJava, expected)
+  }
+
+}
diff --git 
a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/table/UdfITCase.scala
 
b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/table/UdfITCase.scala
new file mode 100644
index 00000000000..b1e03e27554
--- /dev/null
+++ 
b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/stream/table/UdfITCase.scala
@@ -0,0 +1,69 @@
+/*
+ * 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.table
+
+import scala.collection.mutable
+
+import org.apache.flink.api.scala._
+import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
+import org.apache.flink.table.api.scala._
+import org.apache.flink.table.api.TableEnvironment
+import org.apache.flink.table.api.scala.StreamTableEnvironment
+import org.apache.flink.table.runtime.utils.StreamITCase
+import org.apache.flink.table.utils.ScalarFunction0
+import org.apache.flink.test.util.AbstractTestBase
+import org.apache.flink.types.Row
+import org.junit.Before
+import org.junit.Assert._
+import org.junit.Test
+
+class UdfITCase extends AbstractTestBase {
+  val env: StreamExecutionEnvironment = 
StreamExecutionEnvironment.getExecutionEnvironment
+  val tEnv: StreamTableEnvironment = TableEnvironment.getTableEnvironment(env)
+  @Before
+  def clear(): Unit = {
+    StreamITCase.clear
+  }
+
+  @Test
+  def testUdfOpen(): Unit = {
+    val data1 = new mutable.MutableList[(Int, String)]
+    data1.+=((1, "Hi1"))
+    data1.+=((2, "Hi2"))
+    data1.+=((3, "Hi3"))
+
+    val data2 = new mutable.MutableList[(Int, String)]
+    data2.+=((1, "Hello1"))
+    data2.+=((2, "Hello2"))
+    data2.+=((3, "Hello3"))
+
+    val t1 = env.fromCollection(data1).toTable(tEnv, 'a, 'b)
+    val t2 = env.fromCollection(data2).toTable(tEnv, 'c, 'd)
+
+    val fun0 = new ScalarFunction0
+    val t = t1.join(t2, 'a === 'c ).select('a, 'b, 
'd).where(fun0(concat('b,'d)))
+
+    StreamITCase.clear
+    val results = t.toAppendStream[Row]
+    results.addSink(new StreamITCase.StringSink[Row])
+    env.execute()
+    val expected = Seq("1,Hi1,Hello1", "2,Hi2,Hello2", "3,Hi3,Hello3")
+    assertEquals(expected.sorted.mkString(","), 
StreamITCase.testResults.sorted.mkString(","))
+  }
+}
diff --git 
a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/UserDefineScalarFunctions.scala
 
b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/UserDefineScalarFunctions.scala
new file mode 100644
index 00000000000..f86bb70431e
--- /dev/null
+++ 
b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/utils/UserDefineScalarFunctions.scala
@@ -0,0 +1,34 @@
+/*
+ * 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.utils
+
+import org.apache.flink.table.functions.{FunctionContext, ScalarFunction}
+
+/**A Scalar function always return TRUE if opened correctly.*/
+class ScalarFunction0 extends ScalarFunction {
+  private var flag: Boolean = false
+
+  override def open(context: FunctionContext): Unit = {
+    flag = true
+  }
+
+  def eval(str: String): Boolean = {
+    flag
+  }
+}


 

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

Reply via email to