[jira] [Commented] (FLINK-10263) User-defined function with LITERAL paramters yields CompileException
[ https://issues.apache.org/jira/browse/FLINK-10263?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16627711#comment-16627711 ] ASF GitHub Bot commented on FLINK-10263: asfgit closed pull request #6725: [FLINK-10263] [sql-client] Fix classloader issues in SQL Client URL: https://github.com/apache/flink/pull/6725 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-end-to-end-tests/test-scripts/test_sql_client.sh b/flink-end-to-end-tests/test-scripts/test_sql_client.sh index b5830725db1..ca0251365e6 100755 --- a/flink-end-to-end-tests/test-scripts/test_sql_client.sh +++ b/flink-end-to-end-tests/test-scripts/test_sql_client.sh @@ -212,6 +212,8 @@ tables: type: VARCHAR - name: duplicate_count type: BIGINT + - name: constant +type: VARCHAR connector: type: filesystem path: $RESULT @@ -226,6 +228,8 @@ tables: type: VARCHAR - name: duplicate_count type: BIGINT +- name: constant + type: VARCHAR functions: - name: RegReplace @@ -261,7 +265,7 @@ $FLINK_DIR/bin/sql-client.sh embedded \ read -r -d '' SQL_STATEMENT_2 << EOF INSERT INTO CsvSinkTable - SELECT * + SELECT AvroBothTable.*, RegReplace('Test constant folding.', 'Test', 'Success') AS constant FROM AvroBothTable EOF @@ -285,4 +289,4 @@ for i in {1..10}; do sleep 5 done -check_result_hash "SQLClient" $RESULT "dca08a82cc09f6b19950291dbbef16bb" +check_result_hash "SQLClient" $RESULT "0a1bf8bf716069b7269f575f87a802c0" diff --git a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ExecutionContext.java b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ExecutionContext.java index 85b3e9265a8..552d0b37dca 100644 --- a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ExecutionContext.java +++ b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ExecutionContext.java @@ -75,6 +75,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.function.Supplier; /** * Context for executing table programs. This class caches everything that can be cached across @@ -183,6 +184,19 @@ public EnvironmentInstance createEnvironmentInstance() { return tableSinks; } + /** +* Executes the given supplier using the execution context's classloader as thread classloader. +*/ + public R wrapClassLoader(Supplier supplier) { + final ClassLoader previousClassloader = Thread.currentThread().getContextClassLoader(); + Thread.currentThread().setContextClassLoader(classLoader); + try { + return supplier.get(); + } finally { + Thread.currentThread().setContextClassLoader(previousClassloader); + } + } + // private static CommandLine createCommandLine(Deployment deployment, Options commandLineOptions) { diff --git a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/LocalExecutor.java b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/LocalExecutor.java index 3b9e8e99b82..1318043faf1 100644 --- a/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/LocalExecutor.java +++ b/flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/LocalExecutor.java @@ -219,14 +219,16 @@ public TableSchema getTableSchema(SessionContext session, String name) throws Sq @Override public String explainStatement(SessionContext session, String statement) throws SqlExecutionException { - final TableEnvironment tableEnv = getOrCreateExecutionContext(session) + final ExecutionContext context = getOrCreateExecutionContext(session); + final TableEnvironment tableEnv = context .createEnvironmentInstance() .getTableEnvironment(); // translate try { final Table table = createTable(tableEnv, statement); - return tableEnv.explain(table); + // explanation requires an optimization step that might reference UDFs during code compilation + return context.wrapClassLoader(() -> tableEnv.explain(table));
[jira] [Commented] (FLINK-10263) User-defined function with LITERAL paramters yields CompileException
[ https://issues.apache.org/jira/browse/FLINK-10263?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16623381#comment-16623381 ] ASF GitHub Bot commented on FLINK-10263: twalthr commented on issue #6725: [FLINK-10263] [sql-client] Fix classloader issues in SQL Client URL: https://github.com/apache/flink/pull/6725#issuecomment-423486904 Thank you @dawidwys. I simplified the code there. 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 > User-defined function with LITERAL paramters yields CompileException > > > Key: FLINK-10263 > URL: https://issues.apache.org/jira/browse/FLINK-10263 > Project: Flink > Issue Type: Bug > Components: Table API SQL >Affects Versions: 1.7.0 >Reporter: Fabian Hueske >Assignee: Timo Walther >Priority: Major > Labels: pull-request-available > > When using a user-defined scalar function only with literal parameters, a > {{CompileException}} is thrown. For example > {code} > SELECT myFunc(CAST(40.750444 AS FLOAT), CAST(-73.993475 AS FLOAT)) > public class MyFunc extends ScalarFunction { > public int eval(float lon, float lat) { > // do something > } > } > {code} > results in > {code} > [ERROR] Could not execute SQL statement. Reason: > org.codehaus.commons.compiler.CompileException: Line 5, Column 10: Cannot > determine simple type name "com" > {code} > The problem is probably caused by the expression reducer because it > disappears if a regular attribute is added to a parameter expression. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Commented] (FLINK-10263) User-defined function with LITERAL paramters yields CompileException
[ https://issues.apache.org/jira/browse/FLINK-10263?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16623191#comment-16623191 ] ASF GitHub Bot commented on FLINK-10263: dawidwys commented on a change in pull request #6725: [FLINK-10263] [sql-client] Fix classloader issues in SQL Client URL: https://github.com/apache/flink/pull/6725#discussion_r219405018 ## File path: flink-libraries/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ExecutionContext.java ## @@ -183,6 +184,21 @@ public EnvironmentInstance createEnvironmentInstance() { return tableSinks; } + /** +* Executes the given supplier using the execution context's classloader as thread classloader. +*/ + public R wrapClassLoader(Supplier supplier) { + final ClassLoader previousClassloader = Thread.currentThread().getContextClassLoader(); + Thread.currentThread().setContextClassLoader(classLoader); + R returnValue; + try { + returnValue = supplier.get(); Review comment: Can't we just return it here? 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 > User-defined function with LITERAL paramters yields CompileException > > > Key: FLINK-10263 > URL: https://issues.apache.org/jira/browse/FLINK-10263 > Project: Flink > Issue Type: Bug > Components: Table API SQL >Affects Versions: 1.7.0 >Reporter: Fabian Hueske >Assignee: Timo Walther >Priority: Major > Labels: pull-request-available > > When using a user-defined scalar function only with literal parameters, a > {{CompileException}} is thrown. For example > {code} > SELECT myFunc(CAST(40.750444 AS FLOAT), CAST(-73.993475 AS FLOAT)) > public class MyFunc extends ScalarFunction { > public int eval(float lon, float lat) { > // do something > } > } > {code} > results in > {code} > [ERROR] Could not execute SQL statement. Reason: > org.codehaus.commons.compiler.CompileException: Line 5, Column 10: Cannot > determine simple type name "com" > {code} > The problem is probably caused by the expression reducer because it > disappears if a regular attribute is added to a parameter expression. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Commented] (FLINK-10263) User-defined function with LITERAL paramters yields CompileException
[ https://issues.apache.org/jira/browse/FLINK-10263?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16622274#comment-16622274 ] ASF GitHub Bot commented on FLINK-10263: twalthr opened a new pull request #6725: [FLINK-10263] [sql-client] Fix classloader issues in SQL Client URL: https://github.com/apache/flink/pull/6725 ## What is the purpose of the change Fixes classloading issues when using a UDF with constant parameters. Every optimization might need to compile code (i.e. for constant folding), thus, needs access to the user-code classloader. ## Brief change log Set SQL Client's execution context class loader as thread classloader and use it for constant folding. ## Verifying this change End-to-end test has been extended. ## Does this pull request potentially affect one of the following parts: - Dependencies (does it add or upgrade a dependency): no - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: no - The serializers: no - The runtime per-record code paths (performance sensitive): no - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Yarn/Mesos, ZooKeeper: no - The S3 file system connector: no ## Documentation - Does this pull request introduce a new feature? no - If yes, how is the feature documented? not applicable 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 > User-defined function with LITERAL paramters yields CompileException > > > Key: FLINK-10263 > URL: https://issues.apache.org/jira/browse/FLINK-10263 > Project: Flink > Issue Type: Bug > Components: Table API SQL >Affects Versions: 1.7.0 >Reporter: Fabian Hueske >Assignee: Timo Walther >Priority: Major > Labels: pull-request-available > > When using a user-defined scalar function only with literal parameters, a > {{CompileException}} is thrown. For example > {code} > SELECT myFunc(CAST(40.750444 AS FLOAT), CAST(-73.993475 AS FLOAT)) > public class MyFunc extends ScalarFunction { > public int eval(float lon, float lat) { > // do something > } > } > {code} > results in > {code} > [ERROR] Could not execute SQL statement. Reason: > org.codehaus.commons.compiler.CompileException: Line 5, Column 10: Cannot > determine simple type name "com" > {code} > The problem is probably caused by the expression reducer because it > disappears if a regular attribute is added to a parameter expression. -- This message was sent by Atlassian JIRA (v7.6.3#76005)