[ 
https://issues.apache.org/jira/browse/FLINK-7669?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16214356#comment-16214356
 ] 

Aljoscha Krettek edited comment on FLINK-7669 at 10/22/17 4:17 PM:
-------------------------------------------------------------------

[~raymondtay] I did reproduce the problem by also building a fat-jar that 
contains the {{ExecutionConfig}} class. However, I was able to run the program 
by setting {{classloader.resolve-order: parent-first}} in the Flink conf and 
restarting the cluster.

The problem can be fixed by also using the configured classloader resolution 
order in the client, I'll be working on that. (I actually had that code in my 
original PR for introducing the ClassLoader resolution-order so that should be 
easy to do. 🙂)


was (Author: aljoscha):
[~raymondtay] I did reproduce the problem by also building a fat-jar that 
contains the {{ExecutionConfig}} class. However, I was able to run the program 
by setting {{classloader.resolve-order: parent-first}} in the Flink conf and 
restarting the cluster.

The problem can be fixed by also using the configured classloader resolution 
order in the client, I'll be working on that.

> ClassLoader resolution-order setting is not used on client
> ----------------------------------------------------------
>
>                 Key: FLINK-7669
>                 URL: https://issues.apache.org/jira/browse/FLINK-7669
>             Project: Flink
>          Issue Type: Bug
>          Components: Local Runtime
>    Affects Versions: 1.4.0
>         Environment: - OS: macOS Sierra 
> - Oracle JDK 1.8
> - Scala 2.11.11
> - sbt 0.13.16
> - Build from trunk code at commit hash 
> {{42cc3a2a9c41dda7cf338db36b45131db9150674}}
> -- started a local flink node 
>            Reporter: Raymond Tay
>            Assignee: Aljoscha Krettek
>            Priority: Blocker
>             Fix For: 1.4.0
>
>
> Latest code pulled from trunk threw errors at runtime when i ran a job 
> against it; but when i ran the JAR against the stable version {{1.3.2}} it 
> was OK. Here is the stacktrace. 
> An exception is being thrown :
> {noformat}
> Cluster configuration: Standalone cluster with JobManager at 
> localhost/127.0.0.1:6123
> Using address localhost:6123 to connect to JobManager.
> JobManager web interface address http://localhost:8081
> Starting execution of 
> programhttps://issues.apache.org/jira/issues/?jql=text%20~%20%22org.apache.flink.api.common.ExecutionConfig%20cannot%20be%20cast%20to%22#
> Submitting job with JobID: 05dd8e60c6fda3b96fc22ef6cf389a23. Waiting for job 
> completion.
> Connected to JobManager at 
> Actor[akka.tcp://flink@localhost:6123/user/jobmanager#-234825544] with leader 
> session id 00000000-0000-0000-0000-000000000000.
> ------------------------------------------------------------
>  The program finished with the following exception:
> org.apache.flink.client.program.ProgramInvocationException: The program 
> execution failed: Failed to submit job 05dd8e60c6fda3b96fc22ef6cf389a23 
> (Flink Streaming Job)
>       at 
> org.apache.flink.client.program.ClusterClient.run(ClusterClient.java:479)
>       at 
> org.apache.flink.client.program.StandaloneClusterClient.submitJob(StandaloneClusterClient.java:105)
>       at 
> org.apache.flink.client.program.ClusterClient.run(ClusterClient.java:443)
>       at 
> org.apache.flink.streaming.api.environment.StreamContextEnvironment.execute(StreamContextEnvironment.java:66)
>       at 
> org.apache.flink.streaming.api.environment.StreamExecutionEnvironment.execute(StreamExecutionEnvironment.java:1501)
>       at 
> org.apache.flink.streaming.api.scala.StreamExecutionEnvironment.execute(StreamExecutionEnvironment.scala:629)
>       at 
> org.example.streams.split.SimpleSplitStreams$.main(04splitstreams.scala:53)
>       at 
> org.example.streams.split.SimpleSplitStreams.main(04splitstreams.scala)
>       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:525)
>       at 
> org.apache.flink.client.program.PackagedProgram.invokeInteractiveModeForExecution(PackagedProgram.java:417)
>       at 
> org.apache.flink.client.program.ClusterClient.run(ClusterClient.java:383)
>       at 
> org.apache.flink.client.CliFrontend.executeProgram(CliFrontend.java:840)
>       at org.apache.flink.client.CliFrontend.run(CliFrontend.java:285)
>       at 
> org.apache.flink.client.CliFrontend.parseParameters(CliFrontend.java:1088)
>       at org.apache.flink.client.CliFrontend$2.call(CliFrontend.java:1135)
>       at org.apache.flink.client.CliFrontend$2.call(CliFrontend.java:1132)
>       at 
> org.apache.flink.runtime.security.HadoopSecurityContext$1.run(HadoopSecurityContext.java:44)
>       at java.security.AccessController.doPrivileged(Native Method)
>       at javax.security.auth.Subject.doAs(Subject.java:422)
>       at 
> org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1556)
>       at 
> org.apache.flink.runtime.security.HadoopSecurityContext.runSecured(HadoopSecurityContext.java:41)
>       at org.apache.flink.client.CliFrontend.main(CliFrontend.java:1132)
> Caused by: org.apache.flink.runtime.client.JobExecutionException: Failed to 
> submit job 05dd8e60c6fda3b96fc22ef6cf389a23 (Flink Streaming Job)
>       at 
> org.apache.flink.runtime.jobmanager.JobManager.org$apache$flink$runtime$jobmanager$JobManager$$submitJob(JobManager.scala:1358)
>       at 
> org.apache.flink.runtime.jobmanager.JobManager$$anonfun$handleMessage$1.applyOrElse(JobManager.scala:484)
>       at 
> scala.runtime.AbstractPartialFunction.apply(AbstractPartialFunction.scala:36)
>       at 
> org.apache.flink.runtime.LeaderSessionMessageFilter$$anonfun$receive$1.applyOrElse(LeaderSessionMessageFilter.scala:38)
>       at 
> scala.runtime.AbstractPartialFunction.apply(AbstractPartialFunction.scala:36)
>       at 
> org.apache.flink.runtime.LogMessages$$anon$1.apply(LogMessages.scala:33)
>       at 
> org.apache.flink.runtime.LogMessages$$anon$1.apply(LogMessages.scala:28)
>       at scala.PartialFunction$class.applyOrElse(PartialFunction.scala:123)
>       at 
> org.apache.flink.runtime.LogMessages$$anon$1.applyOrElse(LogMessages.scala:28)
>       at akka.actor.Actor$class.aroundReceive(Actor.scala:467)
>       at 
> org.apache.flink.runtime.jobmanager.JobManager.aroundReceive(JobManager.scala:125)
>       at akka.actor.ActorCell.receiveMessage(ActorCell.scala:516)
>       at akka.actor.ActorCell.invoke(ActorCell.scala:487)
>       at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:238)
>       at akka.dispatch.Mailbox.run(Mailbox.scala:220)
>       at 
> akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(AbstractDispatcher.scala:397)
>       at scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
>       at 
> scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)
>       at 
> scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)
>       at 
> scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)
> Caused by: java.lang.ClassCastException: 
> org.apache.flink.api.common.ExecutionConfig cannot be cast to 
> org.apache.flink.api.common.ExecutionConfig
>       at 
> org.apache.flink.runtime.jobmanager.JobManager.org$apache$flink$runtime$jobmanager$JobManager$$submitJob(JobManager.scala:1279)
>       ... 19 more
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Reply via email to