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

Elias Levy commented on FLINK-8752:
-----------------------------------

Some other information:  Cluster is configured in HA mode with S3 as storage 
for checkpoints and using the Presto S3 jar.  I've confirmed that the artifact 
downloaded by the blob cache in the JM matches the job's jar and that is 
contains the class in question.  Using Java 8 update 152.

> ClassNotFoundException when using the user code class loader
> ------------------------------------------------------------
>
>                 Key: FLINK-8752
>                 URL: https://issues.apache.org/jira/browse/FLINK-8752
>             Project: Flink
>          Issue Type: Bug
>          Components: JobManager
>    Affects Versions: 1.4.1
>            Reporter: Elias Levy
>            Priority: Major
>
> Attempting to submit a job results in the job failing while it is being 
> started in the JMs with a ClassNotFoundException error: 
> {code:java}
> java.lang.ClassNotFoundException: com.foo.flink.common.util.TimeAssigner
>       at java.net.URLClassLoader.findClass(Unknown Source)
>       at java.lang.ClassLoader.loadClass(Unknown Source)
>       at sun.misc.Launcher$AppClassLoader.loadClass(Unknown Source)
>       at java.lang.ClassLoader.loadClass(Unknown Source)
>       at java.lang.Class.forName0(Native Method)
>       at java.lang.Class.forName(Unknown Source)
>       at 
> org.apache.flink.util.InstantiationUtil$ClassLoaderObjectInputStream.resolveClass(InstantiationUtil.java:73)
>       at java.io.ObjectInputStream.readNonProxyDesc(Unknown Source)
>       at java.io.ObjectInputStream.readClassDesc(Unknown Source)
>       at java.io.ObjectInputStream.readOrdinaryObject(Unknown Source)
>       at java.io.ObjectInputStream.readObject0(Unknown Source)
>       at java.io.ObjectInputStream.readObject(Unknown Source)
>       at 
> org.apache.flink.util.InstantiationUtil.deserializeObject(InstantiationUtil.java:393)
>       at 
> org.apache.flink.util.InstantiationUtil.deserializeObject(InstantiationUtil.java:380)
>       at 
> org.apache.flink.util.InstantiationUtil.deserializeObject(InstantiationUtil.java:368)
>       at 
> org.apache.flink.util.SerializedValue.deserializeValue(SerializedValue.java:58)
>       at 
> org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher.createPartitionStateHolders(AbstractFetcher.java:542)
>       at 
> org.apache.flink.streaming.connectors.kafka.internals.AbstractFetcher.<init>(AbstractFetcher.java:167)
>       at 
> org.apache.flink.streaming.connectors.kafka.internal.Kafka09Fetcher.<init>(Kafka09Fetcher.java:89)
>       at 
> org.apache.flink.streaming.connectors.kafka.internal.Kafka010Fetcher.<init>(Kafka010Fetcher.java:62)
>       at 
> org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer010.createFetcher(FlinkKafkaConsumer010.java:203)
>       at 
> org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumerBase.run(FlinkKafkaConsumerBase.java:564)
>       at 
> org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:86)
>       at 
> org.apache.flink.streaming.api.operators.StreamSource.run(StreamSource.java:55)
>       at 
> org.apache.flink.streaming.runtime.tasks.SourceStreamTask.run(SourceStreamTask.java:94)
>       at 
> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:264)
>       at org.apache.flink.runtime.taskmanager.Task.run(Task.java:718)
>       at java.lang.Thread.run(Unknown Source)
> {code}
> If I drop the job's jar into the lib folder in the JM and configure the JM to 
>  classloader.resolve-order to parent-first the job starts successfully.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to