You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "ASF GitHub Bot (JIRA)" <ji...@apache.org> on 2015/01/23 08:41:34 UTC

[jira] [Commented] (FLINK-1391) Kryo fails to properly serialize avro collection types

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

ASF GitHub Bot commented on FLINK-1391:
---------------------------------------

Github user StephanEwen commented on a diff in the pull request:

    https://github.com/apache/flink/pull/323#discussion_r23435527
  
    --- Diff: flink-java/src/main/java/org/apache/flink/api/java/typeutils/runtime/KryoSerializer.java ---
    @@ -237,6 +244,25 @@ private void checkKryoInitialized() {
     			// Throwable and all subclasses should be serialized via java serialization
     			kryo.addDefaultSerializer(Throwable.class, new JavaSerializer());
     
    +			// If the type we have to serialize as a GenricType is implementing SpecificRecordBase,
    +			// we have to register the avro serializer
    +			// This rule only applies if users explicitly use the GenericTypeInformation for the avro types
    +			// usually, we are able to handle Avro POJOs with the POJO serializer.
    +			if(SpecificRecordBase.class.isAssignableFrom(type)) {
    +				ClassTag<SpecificRecordBase> tag = scala.reflect.ClassTag$.MODULE$.apply(type);
    +				this.kryo.register(type, com.twitter.chill.avro.AvroSerializer.SpecificRecordSerializer(tag));
    +
    +			}
    +			// Avro POJOs contain java.util.List which have GenericData.Array as their runtime type
    +			// because Kryo is not able to serialize them properly, we use this serializer for them
    +			this.kryo.register(GenericData.Array.class, new SpecificInstanceCollectionSerializer(ArrayList.class));
    --- End diff --
    
    Should we make this registration conditional so that it only happens when we have encountered an Avro type?


> Kryo fails to properly serialize avro collection types
> ------------------------------------------------------
>
>                 Key: FLINK-1391
>                 URL: https://issues.apache.org/jira/browse/FLINK-1391
>             Project: Flink
>          Issue Type: Improvement
>    Affects Versions: 0.8, 0.9
>            Reporter: Robert Metzger
>            Assignee: Robert Metzger
>
> Before FLINK-610, Avro was the default generic serializer.
> Now, special types coming from Avro are handled by Kryo .. which seems to cause errors like:
> {code}
> Exception in thread "main" org.apache.flink.runtime.client.JobExecutionException: java.lang.NullPointerException
> 	at org.apache.avro.generic.GenericData$Array.add(GenericData.java:200)
> 	at com.esotericsoftware.kryo.serializers.CollectionSerializer.read(CollectionSerializer.java:116)
> 	at com.esotericsoftware.kryo.serializers.CollectionSerializer.read(CollectionSerializer.java:22)
> 	at com.esotericsoftware.kryo.Kryo.readClassAndObject(Kryo.java:761)
> 	at org.apache.flink.api.java.typeutils.runtime.KryoSerializer.deserialize(KryoSerializer.java:143)
> 	at org.apache.flink.api.java.typeutils.runtime.KryoSerializer.deserialize(KryoSerializer.java:148)
> 	at org.apache.flink.api.java.typeutils.runtime.PojoSerializer.deserialize(PojoSerializer.java:244)
> 	at org.apache.flink.runtime.plugable.DeserializationDelegate.read(DeserializationDelegate.java:56)
> 	at org.apache.flink.runtime.io.network.serialization.AdaptiveSpanningRecordDeserializer.getNextRecord(AdaptiveSpanningRecordDeserializer.java:71)
> 	at org.apache.flink.runtime.io.network.channels.InputChannel.readRecord(InputChannel.java:189)
> 	at org.apache.flink.runtime.io.network.gates.InputGate.readRecord(InputGate.java:176)
> 	at org.apache.flink.runtime.io.network.api.MutableRecordReader.next(MutableRecordReader.java:51)
> 	at org.apache.flink.runtime.operators.util.ReaderIterator.next(ReaderIterator.java:53)
> 	at org.apache.flink.runtime.operators.DataSinkTask.invoke(DataSinkTask.java:170)
> 	at org.apache.flink.runtime.execution.RuntimeEnvironment.run(RuntimeEnvironment.java:257)
> 	at java.lang.Thread.run(Thread.java:744)
> {code}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)