Details
-
Bug
-
Status: Resolved
-
Major
-
Resolution: Fixed
-
2.0.0, 3.0.0
-
None
Description
SPARK-746 added KryoSerializer for GenericRecord and GenericData.Record Avro objects. However, Kryo serialization of other GenericData types like array, enum and fixed fails. Note that if such objects are within a GenericRecord, then current code works. However if these types are top level objects we want to distribute, then Kryo fails.
We should register KryoSerializer(s) for these GenericData types.
Code to reproduce:
import org.apache.avro.{Schema, SchemaBuilder} import org.apache.avro.generic.GenericData.Array val arraySchema = SchemaBuilder.array().items().intType() val array = new Array[Integer](1, arraySchema) array.add(1) sc.parallelize((0 until 10).map((_, array)), 2).collect
Similar code can be written for enums and fixed types
Errors:
GenericData.Array
java.io.IOException: java.lang.NullPointerException at org.apache.spark.util.Utils$.tryOrIOException(Utils.scala:1410) at org.apache.spark.rdd.ParallelCollectionPartition.readObject(ParallelCollectionRDD.scala:69) 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:1158) at java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:2176) at java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:2067) at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1571) at java.io.ObjectInputStream.defaultReadFields(ObjectInputStream.java:2285) at java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:2209) at java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:2067) at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1571) at java.io.ObjectInputStream.readObject(ObjectInputStream.java:431) at org.apache.spark.serializer.JavaDeserializationStream.readObject(JavaSerializer.scala:76) at org.apache.spark.serializer.JavaSerializerInstance.deserialize(JavaSerializer.scala:115) at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:458) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) at java.lang.Thread.run(Thread.java:748) Caused by: java.lang.NullPointerException at org.apache.avro.generic.GenericData$Array.add(GenericData.java:383) at java.util.AbstractList.add(AbstractList.java:108) at com.esotericsoftware.kryo.serializers.CollectionSerializer.read(CollectionSerializer.java:134) at com.esotericsoftware.kryo.serializers.CollectionSerializer.read(CollectionSerializer.java:40) at com.esotericsoftware.kryo.Kryo.readObject(Kryo.java:731) at com.esotericsoftware.kryo.serializers.DefaultArraySerializers$ObjectArraySerializer.read(DefaultArraySerializers.java:391) at com.esotericsoftware.kryo.serializers.DefaultArraySerializers$ObjectArraySerializer.read(DefaultArraySerializers.java:302) at com.esotericsoftware.kryo.Kryo.readClassAndObject(Kryo.java:813) at com.twitter.chill.WrappedArraySerializer.read(WrappedArraySerializer.scala:35) at com.twitter.chill.WrappedArraySerializer.read(WrappedArraySerializer.scala:23) at com.esotericsoftware.kryo.Kryo.readClassAndObject(Kryo.java:813) at org.apache.spark.serializer.KryoDeserializationStream.readObject(KryoSerializer.scala:303) at org.apache.spark.rdd.ParallelCollectionPartition.$anonfun$readObject$2(ParallelCollectionRDD.scala:79) at org.apache.spark.rdd.ParallelCollectionPartition.$anonfun$readObject$2$adapted(ParallelCollectionRDD.scala:79) at org.apache.spark.util.Utils$.deserializeViaNestedStream(Utils.scala:171) at org.apache.spark.rdd.ParallelCollectionPartition.$anonfun$readObject$1(ParallelCollectionRDD.scala:79) at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23) at org.apache.spark.util.Utils$.tryOrIOException(Utils.scala:1403) ... 20 more
GenericData.EnumSymbol
com.esotericsoftware.kryo.KryoException: java.lang.NullPointerException Serialization trace: props (org.apache.avro.Schema$EnumSchema) schema (org.apache.avro.generic.GenericData$EnumSymbol) at com.esotericsoftware.kryo.serializers.ObjectField.write(ObjectField.java:101) at com.esotericsoftware.kryo.serializers.FieldSerializer.write(FieldSerializer.java:508) at com.esotericsoftware.kryo.Kryo.writeObject(Kryo.java:575) at com.esotericsoftware.kryo.serializers.ObjectField.write(ObjectField.java:79) at com.esotericsoftware.kryo.serializers.FieldSerializer.write(FieldSerializer.java:508) at com.esotericsoftware.kryo.Kryo.writeClassAndObject(Kryo.java:651) at com.esotericsoftware.kryo.serializers.DefaultArraySerializers$ObjectArraySerializer.write(DefaultArraySerializers.java:361) at com.esotericsoftware.kryo.serializers.DefaultArraySerializers$ObjectArraySerializer.write(DefaultArraySerializers.java:302) at com.esotericsoftware.kryo.Kryo.writeClassAndObject(Kryo.java:651) at org.apache.spark.serializer.KryoSerializerInstance.serialize(KryoSerializer.scala:384) at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:543) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) at java.lang.Thread.run(Thread.java:748) Caused by: java.lang.NullPointerException at org.apache.avro.JsonProperties$2$1$1.<init>(JsonProperties.java:175) at org.apache.avro.JsonProperties$2$1.iterator(JsonProperties.java:174) at com.esotericsoftware.kryo.serializers.MapSerializer.write(MapSerializer.java:98) at com.esotericsoftware.kryo.serializers.MapSerializer.write(MapSerializer.java:39) at com.esotericsoftware.kryo.Kryo.writeObject(Kryo.java:575) at com.esotericsoftware.kryo.serializers.ObjectField.write(ObjectField.java:79) ... 13 more
GenericData.Fixed
com.esotericsoftware.kryo.KryoException: java.lang.NullPointerException Serialization trace: props (org.apache.avro.Schema$FixedSchema) schema (org.apache.avro.generic.GenericData$Fixed) at com.esotericsoftware.kryo.serializers.ObjectField.write(ObjectField.java:101) at com.esotericsoftware.kryo.serializers.FieldSerializer.write(FieldSerializer.java:508) at com.esotericsoftware.kryo.Kryo.writeObject(Kryo.java:575) at com.esotericsoftware.kryo.serializers.ObjectField.write(ObjectField.java:79) at com.esotericsoftware.kryo.serializers.FieldSerializer.write(FieldSerializer.java:508) at com.esotericsoftware.kryo.Kryo.writeClassAndObject(Kryo.java:651) at com.esotericsoftware.kryo.serializers.DefaultArraySerializers$ObjectArraySerializer.write(DefaultArraySerializers.java:361) at com.esotericsoftware.kryo.serializers.DefaultArraySerializers$ObjectArraySerializer.write(DefaultArraySerializers.java:302) at com.esotericsoftware.kryo.Kryo.writeClassAndObject(Kryo.java:651) at org.apache.spark.serializer.KryoSerializerInstance.serialize(KryoSerializer.scala:384) at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:543) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) at java.lang.Thread.run(Thread.java:748) Caused by: java.lang.NullPointerException at org.apache.avro.JsonProperties$2$1$1.<init>(JsonProperties.java:175) at org.apache.avro.JsonProperties$2$1.iterator(JsonProperties.java:174) at com.esotericsoftware.kryo.serializers.MapSerializer.write(MapSerializer.java:98) at com.esotericsoftware.kryo.serializers.MapSerializer.write(MapSerializer.java:39) at com.esotericsoftware.kryo.Kryo.writeObject(Kryo.java:575) at com.esotericsoftware.kryo.serializers.ObjectField.write(ObjectField.java:79) ... 13 more
Attachments
Issue Links
- is related to
-
SPARK-746 Automatically Use Avro Serialization for Avro Objects
- Resolved
- links to