Hi kant,

As Jark said,
Your user jar should not contains "
org.codehaus.commons.compiler.ICompilerFactory" dependencies. This will
make calcite can not work.

In 1.10, have made Flink client respect classloading policy that default
policy is child first [1]. More details can find in [2].

[1]
https://ci.apache.org/projects/flink/flink-docs-release-1.10/release-notes/flink-1.10.html#flink-client-respects-classloading-policy-flink-13749
[2] https://issues.apache.org/jira/browse/FLINK-13749

Best,
Jingsong Lee

On Fri, Feb 28, 2020 at 11:19 AM Jark Wu <imj...@gmail.com> wrote:

> Hi Kant,
>
> Are you depending a custom janino or something like hive-exec in your
> pom.xml?
>
> Best,
> Jark
>
> On Fri, 28 Feb 2020 at 10:13, kant kodali <kanth...@gmail.com> wrote:
>
>> It works within IDE but not when I submit using command using flink run
>> myApp.jar
>>
>>
>>
>> On Thu, Feb 27, 2020 at 3:32 PM kant kodali <kanth...@gmail.com> wrote:
>>
>>> Below is the sample code using Flink 1.10
>>>
>>> public class Test {
>>>
>>>     public static void main(String... args) throws Exception {
>>>
>>>         EnvironmentSettings bsSettings = 
>>> EnvironmentSettings.newInstance().useBlinkPlanner().inStreamingMode().build();
>>>         final StreamExecutionEnvironment env = 
>>> StreamExecutionEnvironment.getExecutionEnvironment();
>>>         env.setStateBackend((StateBackend) new 
>>> RocksDBStateBackend("file:///tmp"));
>>>
>>>
>>>         StreamTableEnvironment bsTableEnv = 
>>> StreamTableEnvironment.create(env, bsSettings);
>>>
>>>         Properties properties = new Properties();
>>>         properties.setProperty("bootstrap.servers", "localhost:9092");
>>>         properties.setProperty("group.id", "test");
>>>
>>>         FlinkKafkaConsumer<String> consumer1 = new FlinkKafkaConsumer<>(
>>>                 java.util.regex.Pattern.compile("test-topic1"),
>>>                 new SimpleStringSchema(),
>>>                 properties);
>>>
>>>         FlinkKafkaConsumer<String> consumer2 = new FlinkKafkaConsumer<>(
>>>                 java.util.regex.Pattern.compile("test-topic2"),
>>>                 new SimpleStringSchema(),
>>>                 properties);
>>>
>>>         DataStream<String> stream1 = env.addSource(consumer1);
>>>         DataStream<String> stream2 = env.addSource(consumer2);
>>>
>>>         bsTableEnv.createTemporaryView("sample1", stream1);
>>>         bsTableEnv.createTemporaryView("sample2", stream2);
>>>
>>>         Table result = bsTableEnv.sqlQuery("SELECT * FROM sample1 INNER 
>>> JOIN sample2 on sample1.f0=sample2.f0");
>>>         result.printSchema();
>>>
>>>
>>>         bsTableEnv.toRetractStream(result, Row.class).print();
>>>         bsTableEnv.execute("sample job");
>>>     }
>>> }
>>>
>>>
>>> On Thu, Feb 27, 2020 at 3:22 PM kant kodali <kanth...@gmail.com> wrote:
>>>
>>>> Fixed the typo.
>>>>
>>>>
>>>> Hi All,
>>>>
>>>>
>>>> My sample program works in Flink 1.9 but in 1.10 I get the following
>>>> error when I am submitting the job. otherwords it fails to submit a job.
>>>> any idea?
>>>>
>>>>
>>>> Thanks!
>>>>
>>>> On Thu, Feb 27, 2020 at 2:19 PM kant kodali <kanth...@gmail.com> wrote:
>>>>
>>>>> Hi All,
>>>>>
>>>>>
>>>>> My sample program works in Flink 1.9 but in 1.0 I get the following
>>>>> error when I am submitting the job. otherwords it fails to submit a job.
>>>>> any idea?
>>>>>
>>>>>
>>>>> Thanks!
>>>>>
>>>>>
>>>>>
>>>>>
>>>>>
>>>>>
>>>>>
>>>>> org.apache.flink.client.program.ProgramInvocationException: The main
>>>>> method caused an error: Unable to instantiate java compiler
>>>>>
>>>>> at
>>>>> org.apache.flink.client.program.PackagedProgram.callMainMethod(PackagedProgram.java:335)
>>>>>
>>>>> at
>>>>> org.apache.flink.client.program.PackagedProgram.invokeInteractiveModeForExecution(PackagedProgram.java:205)
>>>>>
>>>>> at
>>>>> org.apache.flink.client.ClientUtils.executeProgram(ClientUtils.java:138)
>>>>>
>>>>> at
>>>>> org.apache.flink.client.cli.CliFrontend.executeProgram(CliFrontend.java:664)
>>>>>
>>>>> at org.apache.flink.client.cli.CliFrontend.run(CliFrontend.java:213)
>>>>>
>>>>> at
>>>>> org.apache.flink.client.cli.CliFrontend.parseParameters(CliFrontend.java:895)
>>>>>
>>>>> at
>>>>> org.apache.flink.client.cli.CliFrontend.lambda$main$10(CliFrontend.java:968)
>>>>>
>>>>> at
>>>>> org.apache.flink.runtime.security.NoOpSecurityContext.runSecured(NoOpSecurityContext.java:30)
>>>>>
>>>>> at org.apache.flink.client.cli.CliFrontend.main(CliFrontend.java:968)
>>>>>
>>>>> Caused by: java.lang.IllegalStateException: Unable to instantiate java
>>>>> compiler
>>>>>
>>>>> at
>>>>> org.apache.calcite.rel.metadata.JaninoRelMetadataProvider.compile(JaninoRelMetadataProvider.java:434)
>>>>>
>>>>> at
>>>>> org.apache.calcite.rel.metadata.JaninoRelMetadataProvider.load3(JaninoRelMetadataProvider.java:375)
>>>>>
>>>>> at
>>>>> org.apache.calcite.rel.metadata.JaninoRelMetadataProvider.lambda$static$0(JaninoRelMetadataProvider.java:109)
>>>>>
>>>>> at
>>>>> org.apache.flink.calcite.shaded.com.google.common.cache.CacheLoader$FunctionToCacheLoader.load(CacheLoader.java:149)
>>>>>
>>>>> at
>>>>> org.apache.flink.calcite.shaded.com.google.common.cache.LocalCache$LoadingValueReference.loadFuture(LocalCache.java:3542)
>>>>>
>>>>> at
>>>>> org.apache.flink.calcite.shaded.com.google.common.cache.LocalCache$Segment.loadSync(LocalCache.java:2323)
>>>>>
>>>>> at
>>>>> org.apache.flink.calcite.shaded.com.google.common.cache.LocalCache$Segment.lockedGetOrLoad(LocalCache.java:2286)
>>>>>
>>>>> at
>>>>> org.apache.flink.calcite.shaded.com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2201)
>>>>>
>>>>> at
>>>>> org.apache.flink.calcite.shaded.com.google.common.cache.LocalCache.get(LocalCache.java:3953)
>>>>>
>>>>> at
>>>>> org.apache.flink.calcite.shaded.com.google.common.cache.LocalCache.getOrLoad(LocalCache.java:3957)
>>>>>
>>>>> at
>>>>> org.apache.flink.calcite.shaded.com.google.common.cache.LocalCache$LocalLoadingCache.get(LocalCache.java:4875)
>>>>>
>>>>> at
>>>>> org.apache.calcite.rel.metadata.JaninoRelMetadataProvider.create(JaninoRelMetadataProvider.java:475)
>>>>>
>>>>> at
>>>>> org.apache.calcite.rel.metadata.JaninoRelMetadataProvider.revise(JaninoRelMetadataProvider.java:488)
>>>>>
>>>>> at
>>>>> org.apache.calcite.rel.metadata.RelMetadataQuery.revise(RelMetadataQuery.java:193)
>>>>>
>>>>> at
>>>>> org.apache.calcite.rel.metadata.RelMetadataQuery.getPulledUpPredicates(RelMetadataQuery.java:797)
>>>>>
>>>>> at
>>>>> org.apache.calcite.rel.rules.ReduceExpressionsRule$ProjectReduceExpressionsRule.onMatch(ReduceExpressionsRule.java:298)
>>>>>
>>>>> at
>>>>> org.apache.calcite.plan.AbstractRelOptPlanner.fireRule(AbstractRelOptPlanner.java:319)
>>>>>
>>>>> at
>>>>> org.apache.calcite.plan.hep.HepPlanner.applyRule(HepPlanner.java:560)
>>>>>
>>>>> at
>>>>> org.apache.calcite.plan.hep.HepPlanner.applyRules(HepPlanner.java:419)
>>>>>
>>>>> at
>>>>> org.apache.calcite.plan.hep.HepPlanner.executeInstruction(HepPlanner.java:256)
>>>>>
>>>>> at
>>>>> org.apache.calcite.plan.hep.HepInstruction$RuleInstance.execute(HepInstruction.java:127)
>>>>>
>>>>> at
>>>>> org.apache.calcite.plan.hep.HepPlanner.executeProgram(HepPlanner.java:215)
>>>>>
>>>>> at
>>>>> org.apache.calcite.plan.hep.HepPlanner.findBestExp(HepPlanner.java:202)
>>>>>
>>>>> at
>>>>> org.apache.flink.table.planner.plan.optimize.program.FlinkHepProgram.optimize(FlinkHepProgram.scala:69)
>>>>>
>>>>> at
>>>>> org.apache.flink.table.planner.plan.optimize.program.FlinkHepRuleSetProgram.optimize(FlinkHepRuleSetProgram.scala:87)
>>>>>
>>>>> at
>>>>> org.apache.flink.table.planner.plan.optimize.program.FlinkChainedProgram$$anonfun$optimize$1.apply(FlinkChainedProgram.scala:62)
>>>>>
>>>>> at
>>>>> org.apache.flink.table.planner.plan.optimize.program.FlinkChainedProgram$$anonfun$optimize$1.apply(FlinkChainedProgram.scala:58)
>>>>>
>>>>> at
>>>>> scala.collection.TraversableOnce$$anonfun$foldLeft$1.apply(TraversableOnce.scala:157)
>>>>>
>>>>> at
>>>>> scala.collection.TraversableOnce$$anonfun$foldLeft$1.apply(TraversableOnce.scala:157)
>>>>>
>>>>> at scala.collection.Iterator$class.foreach(Iterator.scala:891)
>>>>>
>>>>> at scala.collection.AbstractIterator.foreach(Iterator.scala:1334)
>>>>>
>>>>> at scala.collection.IterableLike$class.foreach(IterableLike.scala:72)
>>>>>
>>>>> at scala.collection.AbstractIterable.foreach(Iterable.scala:54)
>>>>>
>>>>> at
>>>>> scala.collection.TraversableOnce$class.foldLeft(TraversableOnce.scala:157)
>>>>>
>>>>> at scala.collection.AbstractTraversable.foldLeft(Traversable.scala:104)
>>>>>
>>>>> at
>>>>> org.apache.flink.table.planner.plan.optimize.program.FlinkChainedProgram.optimize(FlinkChainedProgram.scala:57)
>>>>>
>>>>> at
>>>>> org.apache.flink.table.planner.plan.optimize.StreamCommonSubGraphBasedOptimizer.optimizeTree(StreamCommonSubGraphBasedOptimizer.scala:170)
>>>>>
>>>>> at
>>>>> org.apache.flink.table.planner.plan.optimize.StreamCommonSubGraphBasedOptimizer.doOptimize(StreamCommonSubGraphBasedOptimizer.scala:90)
>>>>>
>>>>> at
>>>>> org.apache.flink.table.planner.plan.optimize.CommonSubGraphBasedOptimizer.optimize(CommonSubGraphBasedOptimizer.scala:77)
>>>>>
>>>>> at
>>>>> org.apache.flink.table.planner.delegation.PlannerBase.optimize(PlannerBase.scala:248)
>>>>>
>>>>> at
>>>>> org.apache.flink.table.planner.delegation.PlannerBase.translate(PlannerBase.scala:151)
>>>>>
>>>>> at
>>>>> org.apache.flink.table.api.java.internal.StreamTableEnvironmentImpl.toDataStream(StreamTableEnvironmentImpl.java:351)
>>>>>
>>>>> at
>>>>> org.apache.flink.table.api.java.internal.StreamTableEnvironmentImpl.toRetractStream(StreamTableEnvironmentImpl.java:296)
>>>>>
>>>>> at
>>>>> org.apache.flink.table.api.java.internal.StreamTableEnvironmentImpl.toRetractStream(StreamTableEnvironmentImpl.java:287)
>>>>>
>>>>> at Reconciliation.main(Reconciliation.java:52)
>>>>>
>>>>> at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>>>>>
>>>>> at
>>>>> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
>>>>>
>>>>> at
>>>>> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>>>>>
>>>>> at java.lang.reflect.Method.invoke(Method.java:498)
>>>>>
>>>>> at
>>>>> org.apache.flink.client.program.PackagedProgram.callMainMethod(PackagedProgram.java:321)
>>>>>
>>>>> ... 8 more
>>>>>
>>>>> Caused by: java.lang.ClassCastException:
>>>>> org.codehaus.janino.CompilerFactory cannot be cast to
>>>>> org.codehaus.commons.compiler.ICompilerFactory
>>>>>
>>>>> at
>>>>> org.codehaus.commons.compiler.CompilerFactoryFactory.getCompilerFactory(CompilerFactoryFactory.java:129)
>>>>>
>>>>> at
>>>>> org.codehaus.commons.compiler.CompilerFactoryFactory.getDefaultCompilerFactory(CompilerFactoryFactory.java:79)
>>>>>
>>>>> at
>>>>> org.apache.calcite.rel.metadata.JaninoRelMetadataProvider.compile(JaninoRelMetadataProvider.java:432)
>>>>>
>>>>> ... 57 more
>>>>>
>>>>

-- 
Best, Jingsong Lee

Reply via email to