[ https://issues.apache.org/jira/browse/FLINK-7373?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Rong Rong updated FLINK-7373: ----------------------------- Description: Using URLClassLoader to load, say from Artifactory, and instantiate UDF instances will cause some Rule failed during runHepPlanner or runVolcanoPlanner. One example could add an ITCase in: {code:title=flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/table/CalcITCase.scala} @Test def testUserDefinedFunctionDynamicClassloader() { val env = ExecutionEnvironment.getExecutionEnvironment val tableEnv = TableEnvironment.getTableEnvironment(env, config) val jarFileURI = "file://org/apache/flink/table/udf/HelloWorld.jar" val udfClassLoader: ClassLoader = new URLClassLoader(List(new URI(jarFileURI).toURL).toArray) val clazz = udfClassLoader.loadClass("org.apache.flink.table.udf.HelloWorld") val helloWorldUDF: ScalarFunction = clazz.newInstance().asInstanceOf[ScalarFunction] tableEnv.registerFunction("helloWorld", helloWorldUDF) val table = env.fromElements("a", "b", "c").toTable(tableEnv, 'text) val result = table.select("text.helloWorld()") val results = result.toDataSet[Row].collect() val expected = "Hello World!" TestBaseUtils.compareResultAsText(results.asJava, expected) } {code} where {code:title=HelloWorld.java} package org.apache.flink.table.udf; import org.apache.flink.table.functions.ScalarFunction; public class HelloWorld extends ScalarFunction { public String eval() { return "Hello World"; } } {code} This triggers the following Exception: {panel:title=Exception} org.apache.flink.api.common.InvalidProgramException: Table program cannot be compiled. This is a bug. Please file an issue. at org.apache.flink.table.codegen.Compiler$class.compile(Compiler.scala:36) at org.apache.flink.table.runtime.FlatMapRunner.compile(FlatMapRunner.scala:31) at org.apache.flink.table.runtime.FlatMapRunner.open(FlatMapRunner.scala:45) at org.apache.flink.api.common.functions.util.FunctionUtils.openFunction(FunctionUtils.java:36) at org.apache.flink.api.common.operators.base.FlatMapOperatorBase.executeOnCollections(FlatMapOperatorBase.java:62) ...... Caused by: org.codehaus.commons.compiler.CompileException: Line 6, Column 22: Cannot determine simple type name "org" at org.codehaus.janino.UnitCompiler.compileError(UnitCompiler.java:11672) at org.codehaus.janino.UnitCompiler.getReferenceType(UnitCompiler.java:6416) at org.codehaus.janino.UnitCompiler.getReferenceType(UnitCompiler.java:6177) ...... {panel} was: Using URLClassLoader to load, say from Artifactory, and instantiate UDF instances will cause some Rule failed during runHepPlanner or runVolcanoPlanner. One example could add an ITCase in: {code:title=flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/table/CalcITCase.scala} @Test def testUserDefinedFunctionDynamicClassloader() { val env = ExecutionEnvironment.getExecutionEnvironment val tableEnv = TableEnvironment.getTableEnvironment(env, config) val jarFileURI = "file://org/apache/flink/table/udf/HelloWorld.jar" val udfClassLoader: ClassLoader = new URLClassLoader(List(new URI(jarFileURI).toURL).toArray) val clazz = udfClassLoader.loadClass("org.apache.flink.table.udf.HelloWorld") val helloWorldUDF: ScalarFunction = clazz.newInstance().asInstanceOf[ScalarFunction] tableEnv.registerFunction("helloWorld", helloWorldUDF) val table = env.fromElements("a", "b", "c").toTable(tableEnv, 'text) val result = table.select("text.helloWorld()") val results = result.toDataSet[Row].collect() val expected = "Hello World!" TestBaseUtils.compareResultAsText(results.asJava, expected) } {code} where {code:title=HelloWorld.java} package org.apache.flink.table.udf; import org.apache.flink.table.functions.ScalarFunction; public class HelloWorld extends ScalarFunction { public String eval(String o) { if (o == null) { return "Hello World"; } else { return "Hellow World " + o.toString(); } } } {code} This triggers the following Exception: {panel:title=Exception} org.apache.flink.api.common.InvalidProgramException: Table program cannot be compiled. This is a bug. Please file an issue. at org.apache.flink.table.codegen.Compiler$class.compile(Compiler.scala:36) at org.apache.flink.table.runtime.FlatMapRunner.compile(FlatMapRunner.scala:31) at org.apache.flink.table.runtime.FlatMapRunner.open(FlatMapRunner.scala:45) at org.apache.flink.api.common.functions.util.FunctionUtils.openFunction(FunctionUtils.java:36) at org.apache.flink.api.common.operators.base.FlatMapOperatorBase.executeOnCollections(FlatMapOperatorBase.java:62) ...... Caused by: org.codehaus.commons.compiler.CompileException: Line 6, Column 22: Cannot determine simple type name "org" at org.codehaus.janino.UnitCompiler.compileError(UnitCompiler.java:11672) at org.codehaus.janino.UnitCompiler.getReferenceType(UnitCompiler.java:6416) at org.codehaus.janino.UnitCompiler.getReferenceType(UnitCompiler.java:6177) ...... {panel} > Using URLClassLoader to load UDF triggers HepPlanner unexpected > ClassNotFoundException > -------------------------------------------------------------------------------------- > > Key: FLINK-7373 > URL: https://issues.apache.org/jira/browse/FLINK-7373 > Project: Flink > Issue Type: Bug > Components: Table API & SQL > Affects Versions: 1.3.1 > Reporter: Rong Rong > > Using URLClassLoader to load, say from Artifactory, and instantiate UDF > instances will cause some Rule failed during runHepPlanner or > runVolcanoPlanner. > One example could add an ITCase in: > {code:title=flink-libraries/flink-table/src/test/scala/org/apache/flink/table/runtime/batch/table/CalcITCase.scala} > @Test > def testUserDefinedFunctionDynamicClassloader() { > val env = ExecutionEnvironment.getExecutionEnvironment > val tableEnv = TableEnvironment.getTableEnvironment(env, config) > val jarFileURI = "file://org/apache/flink/table/udf/HelloWorld.jar" > val udfClassLoader: ClassLoader = new URLClassLoader(List(new > URI(jarFileURI).toURL).toArray) > val clazz = > udfClassLoader.loadClass("org.apache.flink.table.udf.HelloWorld") > val helloWorldUDF: ScalarFunction = > clazz.newInstance().asInstanceOf[ScalarFunction] > tableEnv.registerFunction("helloWorld", helloWorldUDF) > val table = env.fromElements("a", "b", "c").toTable(tableEnv, 'text) > val result = table.select("text.helloWorld()") > val results = result.toDataSet[Row].collect() > val expected = "Hello World!" > TestBaseUtils.compareResultAsText(results.asJava, expected) > } > {code} > where > {code:title=HelloWorld.java} > package org.apache.flink.table.udf; > import org.apache.flink.table.functions.ScalarFunction; > public class HelloWorld extends ScalarFunction { > public String eval() { > return "Hello World"; > } > } > {code} > This triggers the following Exception: > {panel:title=Exception} > org.apache.flink.api.common.InvalidProgramException: Table program cannot be > compiled. This is a bug. Please file an issue. > at > org.apache.flink.table.codegen.Compiler$class.compile(Compiler.scala:36) > at > org.apache.flink.table.runtime.FlatMapRunner.compile(FlatMapRunner.scala:31) > at > org.apache.flink.table.runtime.FlatMapRunner.open(FlatMapRunner.scala:45) > at > org.apache.flink.api.common.functions.util.FunctionUtils.openFunction(FunctionUtils.java:36) > at > org.apache.flink.api.common.operators.base.FlatMapOperatorBase.executeOnCollections(FlatMapOperatorBase.java:62) > ...... > Caused by: org.codehaus.commons.compiler.CompileException: Line 6, Column 22: > Cannot determine simple type name "org" > at > org.codehaus.janino.UnitCompiler.compileError(UnitCompiler.java:11672) > at > org.codehaus.janino.UnitCompiler.getReferenceType(UnitCompiler.java:6416) > at > org.codehaus.janino.UnitCompiler.getReferenceType(UnitCompiler.java:6177) > ...... > {panel} -- This message was sent by Atlassian JIRA (v6.4.14#64029)