Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 3B49D200CDB for ; Sat, 5 Aug 2017 20:06:07 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 39A37164F03; Sat, 5 Aug 2017 18:06:07 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 7F6BE1642A4 for ; Sat, 5 Aug 2017 20:06:06 +0200 (CEST) Received: (qmail 2831 invoked by uid 500); 5 Aug 2017 18:06:05 -0000 Mailing-List: contact issues-help@flink.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@flink.apache.org Delivered-To: mailing list issues@flink.apache.org Received: (qmail 2822 invoked by uid 99); 5 Aug 2017 18:06:05 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd2-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Sat, 05 Aug 2017 18:06:05 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd2-us-west.apache.org (ASF Mail Server at spamd2-us-west.apache.org) with ESMTP id 266201A1A17 for ; Sat, 5 Aug 2017 18:06:05 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd2-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -100.002 X-Spam-Level: X-Spam-Status: No, score=-100.002 tagged_above=-999 required=6.31 tests=[RP_MATCHES_RCVD=-0.001, SPF_PASS=-0.001, USER_IN_WHITELIST=-100] autolearn=disabled Received: from mx1-lw-us.apache.org ([10.40.0.8]) by localhost (spamd2-us-west.apache.org [10.40.0.9]) (amavisd-new, port 10024) with ESMTP id uvreoCnHwA7j for ; Sat, 5 Aug 2017 18:06:04 +0000 (UTC) Received: from mailrelay1-us-west.apache.org (mailrelay1-us-west.apache.org [209.188.14.139]) by mx1-lw-us.apache.org (ASF Mail Server at mx1-lw-us.apache.org) with ESMTP id E5AC25F6C2 for ; Sat, 5 Aug 2017 18:06:03 +0000 (UTC) Received: from jira-lw-us.apache.org (unknown [207.244.88.139]) by mailrelay1-us-west.apache.org (ASF Mail Server at mailrelay1-us-west.apache.org) with ESMTP id B6C6BE0237 for ; Sat, 5 Aug 2017 18:06:02 +0000 (UTC) Received: from jira-lw-us.apache.org (localhost [127.0.0.1]) by jira-lw-us.apache.org (ASF Mail Server at jira-lw-us.apache.org) with ESMTP id 6D9A621E14 for ; Sat, 5 Aug 2017 18:06:00 +0000 (UTC) Date: Sat, 5 Aug 2017 18:06:00 +0000 (UTC) From: "Rong Rong (JIRA)" To: issues@flink.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Created] (FLINK-7373) Using URLClassLoader to load UDF triggers HepPlanner unexpected ClassNotFoundException MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 archived-at: Sat, 05 Aug 2017 18:06:07 -0000 Rong Rong created FLINK-7373: -------------------------------- Summary: 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(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} -- This message was sent by Atlassian JIRA (v6.4.14#64029)