https://issues.apache.org/jira/browse/FLINK-25525?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17469151#comment-17469151
Francesco Guardiani commented on FLINK-25525:
---------------------------------------------
I tried to change both table-runtime and table-planner-loader at {{compile}}
scope, and I get this different exception:
{code:java}
Exception in thread "main" java.lang.NoClassDefFoundError:
org/apache/flink/calcite/shaded/com/jayway/jsonpath/spi/json/JsonProvider
at java.lang.Class.getDeclaredMethods0(Native Method)
at java.lang.Class.privateGetDeclaredMethods(Class.java:2701)
at java.lang.Class.privateGetMethodRecursive(Class.java:3048)
at java.lang.Class.getMethod0(Class.java:3018)
at java.lang.Class.getMethod(Class.java:1784)
at org.apache.calcite.linq4j.tree.Types.lookupMethod(Types.java:314)
at org.apache.calcite.util.BuiltInMethod.<init>(BuiltInMethod.java:646)
org.apache.calcite.util.BuiltInMethod.<clinit>(BuiltInMethod.java:345)
org.apache.calcite.rel.metadata.RelMdPercentageOriginalRows.<clinit>(RelMdPercentageOriginalRows.java:41)
org.apache.calcite.rel.metadata.DefaultRelMetadataProvider.<init>(DefaultRelMetadataProvider.java:42)
org.apache.calcite.rel.metadata.DefaultRelMetadataProvider.<clinit>(DefaultRelMetadataProvider.java:28)
at org.apache.calcite.plan.RelOptCluster.<init>(RelOptCluster.java:91)
at org.apache.calcite.plan.RelOptCluster.create(RelOptCluster.java:100)
org.apache.flink.table.planner.calcite.FlinkRelOptClusterFactory$.create(FlinkRelOptClusterFactory.scala:36)
org.apache.flink.table.planner.calcite.FlinkRelOptClusterFactory.create(FlinkRelOptClusterFactory.scala)
org.apache.flink.table.planner.delegation.PlannerContext.<init>(PlannerContext.java:137)
org.apache.flink.table.planner.delegation.PlannerBase.<init>(PlannerBase.scala:113)
org.apache.flink.table.planner.delegation.StreamPlanner.<init>(StreamPlanner.scala:55)
org.apache.flink.table.planner.delegation.DefaultPlannerFactory.create(DefaultPlannerFactory.java:62)
org.apache.flink.table.planner.loader.DelegatePlannerFactory.create(DelegatePlannerFactory.java:36)
org.apache.flink.table.factories.PlannerFactoryUtil.createPlanner(PlannerFactoryUtil.java:53)
org.apache.flink.table.api.bridge.java.internal.StreamTableEnvironmentImpl.create(StreamTableEnvironmentImpl.java:123)
org.apache.flink.table.api.bridge.java.StreamTableEnvironment.create(StreamTableEnvironment.java:128)
org.apache.flink.table.api.bridge.java.StreamTableEnvironment.create(StreamTableEnvironment.java:96)
org.apache.flink.table.examples.java.basics.StreamSQLExample.main(StreamSQLExample.java:55)
Caused by: java.lang.ClassNotFoundException:
org.apache.flink.calcite.shaded.com.jayway.jsonpath.spi.json.JsonProvider
at java.net.URLClassLoader.findClass(URLClassLoader.java:387)
at java.lang.ClassLoader.loadClass(ClassLoader.java:418)
at sun.misc.Launcher$AppClassLoader.loadClass(Launcher.java:352)
at java.lang.ClassLoader.loadClass(ClassLoader.java:351)
org.apache.flink.core.classloading.ComponentClassLoader.loadClassFromOwnerOnly(ComponentClassLoader.java:139)
org.apache.flink.core.classloading.ComponentClassLoader.loadClassFromComponentFirst(ComponentClassLoader.java:133)
org.apache.flink.core.classloading.ComponentClassLoader.loadClass(ComponentClassLoader.java:95)
at java.lang.ClassLoader.loadClass(ClassLoader.java:351)
... 25 more
{code}
I investigated this one and I have some findings:
* Looking in the built jar of flink-table-runtime, the class is there. So
there's no issue with the packaging itself.
* Executing {{mvn exec:java
-Dexec.mainClass="org.apache.flink.table.examples.java.basics.StreamSQLExample"
-pl flink-examples/flink-examples-table}} works fine, so both code and maven
configuration are correct
* Debugging the code, the correct classloader (the AppClassLoader built from
IDEA) is used for resolving this class.
So my feeling is that there's something wrong with IDEA here.
> flink-examples-table is not runnable in the IDE
> -----------------------------------------------
> Key: FLINK-25525
> URL:
https://issues.apache.org/jira/browse/FLINK-25525
> Project: Flink
> Issue Type: Sub-task
> Components: Table SQL / Ecosystem
> Reporter: Timo Walther
> Assignee: Francesco Guardiani
> Priority: Major
> The following exception is thrown:
> {code}
> Exception in thread "main" org.apache.flink.table.api.ValidationException:
> Could not find any factories that implement
> 'org.apache.flink.table.delegation.ExecutorFactory' in the classpath.
> org.apache.flink.table.factories.FactoryUtil.discoverFactory(FactoryUtil.java:453)
> org.apache.flink.table.api.internal.TableEnvironmentImpl.create(TableEnvironmentImpl.java:295)
> org.apache.flink.table.api.internal.TableEnvironmentImpl.create(TableEnvironmentImpl.java:266)
> org.apache.flink.table.api.TableEnvironment.create(TableEnvironment.java:95)
> org.apache.flink.table.examples.scala.basics.GettingStartedExample$.main(GettingStartedExample.scala:55)
> org.apache.flink.table.examples.scala.basics.GettingStartedExample.main(GettingStartedExample.scala)
> {code}
This message was sent by Atlassian Jira
(v8.20.1#820001)