Re: java serialization errors with spark.files.userClassPathFirst=true
after removing all class paramater of class Path from my code, i tried again. different but related eror when i set spark.files.userClassPathFirst=true now i dont even use FileInputFormat directly. HadoopRDD does... 14/05/16 12:17:17 ERROR Executor: Exception in task ID 45 java.lang.NoClassDefFoundError: org/apache/hadoop/mapred/FileInputFormat at java.lang.ClassLoader.defineClass1(Native Method) at java.lang.ClassLoader.defineClass(ClassLoader.java:792) at java.security.SecureClassLoader.defineClass(SecureClassLoader.java:142) at java.net.URLClassLoader.defineClass(URLClassLoader.java:449) at java.net.URLClassLoader.access$100(URLClassLoader.java:71) at java.net.URLClassLoader$1.run(URLClassLoader.java:361) at java.net.URLClassLoader$1.run(URLClassLoader.java:355) at java.security.AccessController.doPrivileged(Native Method) at java.net.URLClassLoader.findClass(URLClassLoader.java:354) at org.apache.spark.executor.ChildExecutorURLClassLoader$userClassLoader$.findClass(ExecutorURLClassLoader.scala:42) at java.lang.ClassLoader.loadClass(ClassLoader.java:424) at java.lang.ClassLoader.loadClass(ClassLoader.java:357) at org.apache.spark.executor.ChildExecutorURLClassLoader.findClass(ExecutorURLClassLoader.scala:51) at java.lang.ClassLoader.loadClass(ClassLoader.java:424) at java.lang.ClassLoader.loadClass(ClassLoader.java:357) at java.lang.Class.forName0(Native Method) at java.lang.Class.forName(Class.java:270) at org.apache.spark.serializer.JavaDeserializationStream$$anon$1.resolveClass(JavaSerializer.scala:57) at java.io.ObjectInputStream.readNonProxyDesc(ObjectInputStream.java:1610) at java.io.ObjectInputStream.readClassDesc(ObjectInputStream.java:1515) at java.io.ObjectInputStream.readClass(ObjectInputStream.java:1481) at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1331) at java.io.ObjectInputStream.defaultReadFields(ObjectInputStream.java:1989) at java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:1913) at java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:1796) at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1348) at java.io.ObjectInputStream.defaultReadFields(ObjectInputStream.java:1989) at java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:1913) at java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:1796) at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1348) at java.io.ObjectInputStream.readObject(ObjectInputStream.java:370) at scala.collection.immutable.$colon$colon.readObject(List.scala:362) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:606) at java.io.ObjectStreamClass.invokeReadObject(ObjectStreamClass.java:1017) at java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:1891) at java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:1796) at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1348) On Thu, May 15, 2014 at 3:03 PM, Koert Kuipers ko...@tresata.com wrote: when i set spark.files.userClassPathFirst=true, i get java serialization errors in my tasks, see below. when i set userClassPathFirst back to its default of false, the serialization errors are gone. my spark.serializer is KryoSerializer. the class org.apache.hadoop.fs.Path is in the spark assembly jar, but not in my task jars (the ones i added to the SparkConf). so looks like the ClosureSerializer is having trouble with this class once the ChildExecutorURLClassLoader is used? thats me just guessing. Exception in thread main org.apache.spark.SparkException: Job aborted due to stage failure: Task 1.0:5 failed 4 times, most recent failure: Exception failure in TID 31 on host node05.tresata.com: java.lang.NoClassDefFoundError: org/apache/hadoop/fs/Path java.lang.Class.getDeclaredConstructors0(Native Method) java.lang.Class.privateGetDeclaredConstructors(Class.java:2398) java.lang.Class.getDeclaredConstructors(Class.java:1838) java.io.ObjectStreamClass.computeDefaultSUID(ObjectStreamClass.java:1697) java.io.ObjectStreamClass.access$100(ObjectStreamClass.java:50) java.io.ObjectStreamClass$1.run(ObjectStreamClass.java:203) java.security.AccessController.doPrivileged(Native Method) java.io.ObjectStreamClass.getSerialVersionUID(ObjectStreamClass.java:200) java.io.ObjectStreamClass.initNonProxy(ObjectStreamClass.java:556)
Re: java serialization errors with spark.files.userClassPathFirst=true
well, i modified ChildExecutorURLClassLoader to also delegate to parentClassloader if NoClassDefFoundError is thrown... now i get yet another error. i am clearly missing something with these classloaders. such nasty stuff... giving up for now. just going to have to not use spark.files.userClassPathFirst=true for now, until i have more time to look at this. 14/05/16 13:58:59 ERROR Executor: Exception in task ID 3 java.lang.ClassCastException: cannot assign instance of scala.None$ to field org.apache.spark.rdd.RDD.checkpointData of type scala.Option in instance of MyRDD at java.io.ObjectStreamClass$FieldReflector.setObjFieldValues(ObjectStreamClass.java:2083) at java.io.ObjectStreamClass.setObjFieldValues(ObjectStreamClass.java:1261) at java.io.ObjectInputStream.defaultReadFields(ObjectInputStream.java:1995) at java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:1913) at java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:1796) at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1348) at java.io.ObjectInputStream.defaultReadFields(ObjectInputStream.java:1989) at java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:1913) at java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:1796) at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1348) at java.io.ObjectInputStream.readObject(ObjectInputStream.java:370) at scala.collection.immutable.$colon$colon.readObject(List.scala:362) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:606) at java.io.ObjectStreamClass.invokeReadObject(ObjectStreamClass.java:1017) at java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:1891) at java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:1796) at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1348) at java.io.ObjectInputStream.defaultReadFields(ObjectInputStream.java:1989) at java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:1913) at java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:1796) at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1348) at java.io.ObjectInputStream.readObject(ObjectInputStream.java:370) at org.apache.spark.serializer.JavaDeserializationStream.readObject(JavaSerializer.scala:60) On Fri, May 16, 2014 at 1:46 PM, Koert Kuipers ko...@tresata.com wrote: after removing all class paramater of class Path from my code, i tried again. different but related eror when i set spark.files.userClassPathFirst=true now i dont even use FileInputFormat directly. HadoopRDD does... 14/05/16 12:17:17 ERROR Executor: Exception in task ID 45 java.lang.NoClassDefFoundError: org/apache/hadoop/mapred/FileInputFormat at java.lang.ClassLoader.defineClass1(Native Method) at java.lang.ClassLoader.defineClass(ClassLoader.java:792) at java.security.SecureClassLoader.defineClass(SecureClassLoader.java:142) at java.net.URLClassLoader.defineClass(URLClassLoader.java:449) at java.net.URLClassLoader.access$100(URLClassLoader.java:71) at java.net.URLClassLoader$1.run(URLClassLoader.java:361) at java.net.URLClassLoader$1.run(URLClassLoader.java:355) at java.security.AccessController.doPrivileged(Native Method) at java.net.URLClassLoader.findClass(URLClassLoader.java:354) at org.apache.spark.executor.ChildExecutorURLClassLoader$userClassLoader$.findClass(ExecutorURLClassLoader.scala:42) at java.lang.ClassLoader.loadClass(ClassLoader.java:424) at java.lang.ClassLoader.loadClass(ClassLoader.java:357) at org.apache.spark.executor.ChildExecutorURLClassLoader.findClass(ExecutorURLClassLoader.scala:51) at java.lang.ClassLoader.loadClass(ClassLoader.java:424) at java.lang.ClassLoader.loadClass(ClassLoader.java:357) at java.lang.Class.forName0(Native Method) at java.lang.Class.forName(Class.java:270) at org.apache.spark.serializer.JavaDeserializationStream$$anon$1.resolveClass(JavaSerializer.scala:57) at java.io.ObjectInputStream.readNonProxyDesc(ObjectInputStream.java:1610) at java.io.ObjectInputStream.readClassDesc(ObjectInputStream.java:1515) at java.io.ObjectInputStream.readClass(ObjectInputStream.java:1481) at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1331) at java.io.ObjectInputStream.defaultReadFields(ObjectInputStream.java:1989) at java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:1913)
Re: java serialization errors with spark.files.userClassPathFirst=true
i do not think the current solution will work. i tried writing a version of ChildExecutorURLClassLoader that does have a proper parent and has a modified loadClass to reverse the order of parent and child in finding classes, and that seems to work, but now classes like SparkEnv are loaded by the child and somehow this means the companion objects are reset or something like that because i get NPEs. On Fri, May 16, 2014 at 3:54 PM, Koert Kuipers ko...@tresata.com wrote: ok i think the issue is visibility: a classloader can see all classes loaded by its parent classloader. but userClassLoader does not have a parent classloader, so its not able to see any classes that parentLoader is responsible for. in my case userClassLoader is trying to get AvroInputFormat which probably somewhere statically references FileInputFormat, which is invisible to userClassLoader. On Fri, May 16, 2014 at 3:32 PM, Koert Kuipers ko...@tresata.com wrote: ok i put lots of logging statements in the ChildExecutorURLClassLoader. this is what i see: * the urls for userClassLoader are correct and includes only my one jar. * for one class that only exists in my jar i see it gets loaded correctly using userClassLoader * for a class that exists in both my jar and spark kernel it tries to use userClassLoader and ends up with a NoClassDefFoundError. the class is org.apache.avro.mapred.AvroInputFormat and the NoClassDefFoundError is for org.apache.hadoop.mapred.FileInputFormat (which the parentClassLoader is responsible for since it is not in my jar). i currently catch this NoClassDefFoundError and call parentClassLoader.loadClass but thats clearly not a solution since it loads the wrong version. On Fri, May 16, 2014 at 2:25 PM, Koert Kuipers ko...@tresata.com wrote: well, i modified ChildExecutorURLClassLoader to also delegate to parentClassloader if NoClassDefFoundError is thrown... now i get yet another error. i am clearly missing something with these classloaders. such nasty stuff... giving up for now. just going to have to not use spark.files.userClassPathFirst=true for now, until i have more time to look at this. 14/05/16 13:58:59 ERROR Executor: Exception in task ID 3 java.lang.ClassCastException: cannot assign instance of scala.None$ to field org.apache.spark.rdd.RDD.checkpointData of type scala.Option in instance of MyRDD at java.io.ObjectStreamClass$FieldReflector.setObjFieldValues(ObjectStreamClass.java:2083) at java.io.ObjectStreamClass.setObjFieldValues(ObjectStreamClass.java:1261) at java.io.ObjectInputStream.defaultReadFields(ObjectInputStream.java:1995) at java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:1913) at java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:1796) at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1348) at java.io.ObjectInputStream.defaultReadFields(ObjectInputStream.java:1989) at java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:1913) at java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:1796) at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1348) at java.io.ObjectInputStream.readObject(ObjectInputStream.java:370) at scala.collection.immutable.$colon$colon.readObject(List.scala:362) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:606) at java.io.ObjectStreamClass.invokeReadObject(ObjectStreamClass.java:1017) at java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:1891) at java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:1796) at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1348) at java.io.ObjectInputStream.defaultReadFields(ObjectInputStream.java:1989) at java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:1913) at java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:1796) at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1348) at java.io.ObjectInputStream.readObject(ObjectInputStream.java:370) at org.apache.spark.serializer.JavaDeserializationStream.readObject(JavaSerializer.scala:60) On Fri, May 16, 2014 at 1:46 PM, Koert Kuipers ko...@tresata.comwrote: after removing all class paramater of class Path from my code, i tried again. different but related eror when i set spark.files.userClassPathFirst=true now i dont even use FileInputFormat directly. HadoopRDD does... 14/05/16 12:17:17 ERROR Executor: Exception in task ID 45 java.lang.NoClassDefFoundError: