Nico Kruber created FLINK-14575:
-----------------------------------

             Summary: Wrong (parent-first) class loader during serialization 
while submitting jobs
                 Key: FLINK-14575
                 URL: https://issues.apache.org/jira/browse/FLINK-14575
             Project: Flink
          Issue Type: Bug
          Components: Client / Job Submission
    Affects Versions: 1.9.1, 1.8.2
            Reporter: Nico Kruber


When building the user code classloader for job submission, Flink uses a parent 
first class loader for serializing the ExecutionConfig which may lead to 
problems in the following case:

# have hadoop in the system class loader from lib/ (this also provides avro 
1.8.3)
# have a user jar with a newer avro, e.g. 1.9.1
# register an Avro class with the execution config, e.g. through 
{{registerPojoType}} (please ignore for a second that this is not needed)

During submission, a parent-first classloader will be used and thus, avro 1.8.3 
will be used which does not map the version in the user classloader that will 
be used for deserialization.

Exception during submission:

{code}
Caused by: java.io.InvalidClassException: 
org.apache.avro.specific.SpecificRecordBase; local class incompatible: stream 
classdesc serialVersionUID = 189988654766568477, local class serialVersionUID = 
-1463700717714793795
        at java.io.ObjectStreamClass.initNonProxy(ObjectStreamClass.java:699)
        at 
java.io.ObjectInputStream.readNonProxyDesc(ObjectInputStream.java:1885)
        at java.io.ObjectInputStream.readClassDesc(ObjectInputStream.java:1751)
        at 
java.io.ObjectInputStream.readNonProxyDesc(ObjectInputStream.java:1885)
        at java.io.ObjectInputStream.readClassDesc(ObjectInputStream.java:1751)
        at java.io.ObjectInputStream.readClass(ObjectInputStream.java:1716)
        at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1556)
        at java.io.ObjectInputStream.readObject(ObjectInputStream.java:431)
        at java.util.HashSet.readObject(HashSet.java:341)
        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 
java.io.ObjectStreamClass.invokeReadObject(ObjectStreamClass.java:1170)
        at java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:2178)
        at 
java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:2069)
        at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1573)
        at 
java.io.ObjectInputStream.defaultReadFields(ObjectInputStream.java:2287)
        at java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:2211)
        at 
java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:2069)
        at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1573)
        at java.io.ObjectInputStream.readObject(ObjectInputStream.java:431)
        at 
org.apache.flink.util.InstantiationUtil.deserializeObject(InstantiationUtil.java:566)
        at 
org.apache.flink.util.InstantiationUtil.deserializeObject(InstantiationUtil.java:552)
        at 
org.apache.flink.util.InstantiationUtil.deserializeObject(InstantiationUtil.java:540)
        at 
org.apache.flink.util.SerializedValue.deserializeValue(SerializedValue.java:58)
        at 
org.apache.flink.runtime.jobmaster.JobMaster.<init>(JobMaster.java:278)
        at 
org.apache.flink.runtime.jobmaster.factories.DefaultJobMasterServiceFactory.createJobMasterService(DefaultJobMasterServiceFactory.java:83)
        at 
org.apache.flink.runtime.jobmaster.factories.DefaultJobMasterServiceFactory.createJobMasterService(DefaultJobMasterServiceFactory.java:37)
        at 
org.apache.flink.runtime.jobmaster.JobManagerRunner.<init>(JobManagerRunner.java:146)
        ... 10 more
{code}

The incriminating code is in
* Flink 1.8.0: 
{{org.apache.flink.client.program.JobWithJars#buildUserCodeClassLoader}}
* Flink master: {{org.apache.flink.client.ClientUtils#buildUserCodeClassLoader}}

Thanks [~chesnay] for looking into this with me. [~aljoscha] Do you know why we 
use parent-first there?



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to