Hello,

Sorry for the delay. The bug is not in Gelly, but is, as hinted in the
exception and as can be seen in the logs, in Flink's Runtime. Mihail may
actually be on to something. The bug is actually very similar to the one
described in FLINK-1916.

However, as can be seen in the discussion thread there, it's a bit
difficult to fix it without some steps to reproduce. I unfortunately
managed to reproduce it and have opened a Jira... FLINK-2360
<https://issues.apache.org/jira/browse/FLINK-2360>. It's a similar delta
iteration setting. Hope we can get some help with this.

Thanks!
Andra

On Tue, Jul 14, 2015 at 2:12 PM, Mihail Vieru <vi...@informatik.hu-berlin.de
> wrote:

>  Hi,
>
> looks very similar to this bug:
> https://issues.apache.org/jira/browse/FLINK-1916
>
> Best,
> Mihail
>
>
> On 14.07.2015 14:09, Andra Lungu wrote:
>
> Hi Flavio,
>
> Could you also show us a code snippet?
>
> On Tue, Jul 14, 2015 at 2:06 PM, Flavio Pompermaier <pomperma...@okkam.it>
> wrote:
>
>> Hi to all,
>> in my vertex centric iteration I get the following exception, am I doing
>> something wrong or is it a bug of Gelly?
>>
>>  starting iteration [1]:  CoGroup (Messaging) (6/8)
>> IterationHead(WorksetIteration (Vertex-centric iteration
>> (test.gelly.functions.VUpdateFunction@1814786f
>> | test.gelly.functions.VMessagingFunction@67eecbc2))) (4/8) switched to
>> FAILED with exception.
>> java.io.EOFException
>>  at
>> org.apache.flink.runtime.operators.hash.InMemoryPartition$WriteView.nextSegment(InMemoryPartition.java:333)
>>  at
>> org.apache.flink.runtime.memorymanager.AbstractPagedOutputView.advance(AbstractPagedOutputView.java:140)
>>  at
>> org.apache.flink.runtime.memorymanager.AbstractPagedOutputView.write(AbstractPagedOutputView.java:201)
>>  at
>> org.apache.flink.api.java.typeutils.runtime.DataOutputViewStream.write(DataOutputViewStream.java:39)
>>  at com.esotericsoftware.kryo.io.Output.flush(Output.java:163)
>>  at
>> org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer.serialize(KryoSerializer.java:187)
>>  at
>> org.apache.flink.api.java.typeutils.runtime.PojoSerializer.serialize(PojoSerializer.java:372)
>>  at
>> org.apache.flink.api.java.typeutils.runtime.TupleSerializer.serialize(TupleSerializer.java:116)
>>  at
>> org.apache.flink.api.java.typeutils.runtime.TupleSerializer.serialize(TupleSerializer.java:30)
>>  at
>> org.apache.flink.runtime.operators.hash.InMemoryPartition.appendRecord(InMemoryPartition.java:219)
>>  at
>> org.apache.flink.runtime.operators.hash.CompactingHashTable.insertOrReplaceRecord(CompactingHashTable.java:536)
>>  at
>> org.apache.flink.runtime.operators.hash.CompactingHashTable.buildTableWithUniqueKey(CompactingHashTable.java:347)
>>  at
>> org.apache.flink.runtime.iterative.task.IterationHeadPactTask.readInitialSolutionSet(IterationHeadPactTask.java:209)
>>  at
>> org.apache.flink.runtime.iterative.task.IterationHeadPactTask.run(IterationHeadPactTask.java:270)
>>  at
>> org.apache.flink.runtime.operators.RegularPactTask.invoke(RegularPactTask.java:362)
>>  at org.apache.flink.runtime.taskmanager.Task.run(Task.java:559)
>>  at java.lang.Thread.run(Thread.java:745)
>>
>>
>>  Best,
>> Flavio
>>
>
>
>

Reply via email to