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

Till Rohrmann commented on FLINK-1531:
--------------------------------------

Thanks for the great bug description with the additional UnitTest [~jkirsch]. 

The reason for the failure was that the KryoSerializer did not properly 
forwarded the EOFException of the underlying stream. The EOFException is used 
by the system to know when all data of an input channel has been consumed. 

I created a PR for this and once all tests pass on Travis, I'll commit it. 

> Custom Kryo Serializer fails in itertation scenario
> ---------------------------------------------------
>
>                 Key: FLINK-1531
>                 URL: https://issues.apache.org/jira/browse/FLINK-1531
>             Project: Flink
>          Issue Type: Bug
>          Components: Iterations
>    Affects Versions: 0.9
>            Reporter: Johannes
>            Priority: Minor
>         Attachments: TestKryoIterationSerializer.java
>
>
> When using iterations with a custom serializer for a domain object, the 
> iteration will fail.
> {code:java}
> org.apache.flink.runtime.client.JobExecutionException: 
> com.esotericsoftware.kryo.KryoException: Buffer underflow
>       at 
> org.apache.flink.api.java.typeutils.runtime.NoFetchingInput.require(NoFetchingInput.java:76)
>       at com.esotericsoftware.kryo.io.Input.readVarInt(Input.java:355)
>       at 
> com.esotericsoftware.kryo.util.DefaultClassResolver.readClass(DefaultClassResolver.java:109)
>       at com.esotericsoftware.kryo.Kryo.readClass(Kryo.java:641)
>       at com.esotericsoftware.kryo.Kryo.readClassAndObject(Kryo.java:752)
>       at 
> org.apache.flink.api.java.typeutils.runtime.KryoSerializer.deserialize(KryoSerializer.java:198)
>       at 
> org.apache.flink.api.java.typeutils.runtime.KryoSerializer.deserialize(KryoSerializer.java:203)
>       at 
> org.apache.flink.runtime.io.disk.InputViewIterator.next(InputViewIterator.java:43)
>       at 
> org.apache.flink.runtime.iterative.task.IterationHeadPactTask.streamOutFinalOutputBulk(IterationHeadPactTask.java:404)
>       at 
> org.apache.flink.runtime.iterative.task.IterationHeadPactTask.run(IterationHeadPactTask.java:377)
>       at 
> org.apache.flink.runtime.operators.RegularPactTask.invoke(RegularPactTask.java:360)
>       at 
> org.apache.flink.runtime.execution.RuntimeEnvironment.run(RuntimeEnvironment.java:204)
>       at java.lang.Thread.run(Thread.java:745)
> {code}



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

Reply via email to