Nico Kruber created FLINK-17171:
-----------------------------------
Summary: Blink planner fails to compile Table program with POJO source
Key: FLINK-17171
URL:
https://issues.apache.org/jira/browse/FLINK-17171 Project: Flink
Issue Type: Bug
Components: Table SQL / Planner, Table SQL / Runtime
Affects Versions: 1.10.0
Reporter: Nico Kruber
Attachments: error.log
It seems as if FLINK-13993 made the Table API (Blink planner) unusable for POJO sources where the POJO class is in user code.
For
https://github.com/ververica/lab-sql-vs-datastream/blob/master/src/main/java/com/ververica/LateralTableJoin.java I get the following Exception when I run it on a Flink 1.10.0 cluster (full version attached):
{code}
2020-04-15 17:19:15,561 ERROR org.apache.flink.runtime.webmonitor.handlers.JarRunHandler - Unhandled exception.
org.apache.flink.util.FlinkRuntimeException: org.apache.flink.api.common.InvalidProgramException: Table program cannot be compiled. This is a bug. Please file an issue.
...
Caused by: org.codehaus.commons.compiler.CompileException: Line 28, Column 175: Cannot determine simple type name "com"
...
at org.apache.flink.table.runtime.generated.CompileUtils.doCompile(CompileUtils.java:78)
{code}
I enabled debug logs and this is what it is trying to compile:
{code}
@Override
public void processElement(org.apache.flink.streaming.runtime.streamrecord.StreamRecord element) throws Exception {
org.apache.flink.table.dataformat.BaseRow in1 = (org.apache.flink.table.dataformat.BaseRow) (org.apache.flink.table.dataformat.BaseRow) converter$15.toInternal((com.ververica.tables.FactTable.Fact) element.getValue());
...
{code}
I use a standalone cluster and submit via web UI and also verified that my jar file does not contain anything else but its compiled classes.
This code is working fine inside the IDE and was also working with Flink 1.10 and VVP 2.0 which did not use a dedicated class loader for user code.
My guess is that the (generated) code does not have access to {{FactTable.Fact}} and the Janino compiler does not produce the right error message seeing "com" as a primitive type instead.
FLINK-7490 and FLINK-9220 seem related but too old (legacy planner).
--
This message was sent by Atlassian Jira
(v8.3.4#803005)