Re: Gelly Library. Need an example

2016-09-20 Thread Till Rohrmann
+1 for the either type solution :-)

On Sat, Sep 17, 2016 at 10:49 AM, Martin Junghanns 
wrote:

> Hi all,
>
> thanks for reporting the issue! I just looked into it. The
> VertexGroupReduce outputs two semantically different tuples: one for each
> vertex without the value (null) and one tuple representing the whole group
> including the value. As Till pointed out, this crashes if the value has no
> serializer for null values.
>
> Since we cannot have two different output types for the same GroupReduce
> function, I propose using a Either here. If there are no
> objections I will take the issue and fix it.
>
> Again, thx for pointing it out.
>
> Best,
>
> Martin
>
>
> On 15.09.2016 20:10, Vasiliki Kalavri wrote:
>
>> Hi,
>>
>> thanks for looking into this Till! I'm not quite sure what the algorithm
>> behavior should be when the vertex value is null (probably skip the
>> record?). Let's wait for Martin's input.
>>
>> Cheers,
>> -V.
>>
>> On 15 September 2016 at 19:19, Olga Golovneva 
>> wrote:
>>
>> Hi Till,
>>>
>>> Thanks a lot for your help! I'll try to use another variable type in the
>>> meantime.
>>>
>>> Best regards,
>>> Olga
>>>
>>>
>>> Best regards,
>>> Olga Golovneva
>>>
>>> On Thu, Sep 15, 2016 at 1:03 PM, Till Rohrmann 
>>> wrote:
>>>
>>> Hi Olga,

 it’s indeed an error in Flink’s Summarization algorithm. The problem is

>>> the
>>>
 following: The vertex group value of the VertexGroupItem is null in the
 VertexGroupReducer. This works in the SummarizationIT case because the
 vertex value is of type String and the StringSerializer can deal with

>>> null
>>>
 values.

 However, in your case where you use longs, it fails, because the
 LongSerializer cannot handle null values. You can verify this behaviour

>>> by
>>>
 changing the vertex value type to String. Then everything should work
 without a problem.

 I’ve cc’ed Martin who can tell you probably more about the Summarization
 algorithm. I’ve also opened a JIRA ticket [1] to fix this problem.

 Thanks for reporting this bug.

 [1] https://issues.apache.org/jira/browse/FLINK-4624

 Cheers,
 Till
 ​

 On Thu, Sep 15, 2016 at 5:05 PM, Olga Golovneva 
 wrote:

 Hi Till,
>
> I've created a simple (Java) example to show you what's going on. The
>
 code

> is in attachment and shown below. This example creates simple graph
>
 with
>>>
 Double EV and Long VV. Then it runs Summarization, that should compute
>
 a
>>>
 condensed version of the input graph by grouping vertices and edges
>
 based
>>>
 on their values. I run this code with IntelliJ IDEA. The code executes
>
 fine

> until you want to see what is written in resulted edges (just uncomment
> line 46, edgesOut.print();). Then it throws the following Exception:
>
> _EXCEPTION START_
> Exception in thread "main" org.apache.flink.runtime.
>
 client.JobExecutionException:

> Job execution failed.
> at org.apache.flink.runtime.jobmanager.JobManager$$
> anonfun$handleMessage$1$$anonfun$applyOrElse$8.apply$
> mcV$sp(JobManager.scala:830)
> at org.apache.flink.runtime.jobmanager.JobManager$$
> anonfun$handleMessage$1$$anonfun$applyOrElse$8.apply(
>
 JobManager.scala:773)

> at org.apache.flink.runtime.jobmanager.JobManager$$
> anonfun$handleMessage$1$$anonfun$applyOrElse$8.apply(
>
 JobManager.scala:773)

> at scala.concurrent.impl.Future$PromiseCompletingRunnable.
> liftedTree1$1(Future.scala:24)
> at scala.concurrent.impl.Future$PromiseCompletingRunnable.run(
> Future.scala:24)
> at akka.dispatch.TaskInvocation.run(AbstractDispatcher.scala:41)
> at akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(
> AbstractDispatcher.scala:401)
> at scala.concurrent.forkjoin.ForkJoinTask.doExec(
>
 ForkJoinTask.java:260)
>>>
 at scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.
> pollAndExecAll(ForkJoinPool.java:1253)
> at scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.
> runTask(ForkJoinPool.java:1346)
> at scala.concurrent.forkjoin.ForkJoinPool.runWorker(
> ForkJoinPool.java:1979)
> at scala.concurrent.forkjoin.ForkJoinWorkerThread.run(
> ForkJoinWorkerThread.java:107)
> Caused by: org.apache.flink.types.NullFieldException: Field 2 is null,
> but expected to hold a value.
> at org.apache.flink.api.java.typeutils.runtime.
>
 TupleSerializer.serialize(

> TupleSerializer.java:126)
> at org.apache.flink.api.java.typeutils.runtime.
>
 TupleSerializer.serialize(

> TupleSerializer.java:30)
> at org.apache.flink.runtime.plugable.SerializationDelegate.write(
> SerializationDelegate.java:56)
> at org.apache.flink.runtime.io.network.api.serialization.
> SpanningRecordSerializer

Re: Gelly Library. Need an example

2016-09-17 Thread Martin Junghanns

Hi all,

thanks for reporting the issue! I just looked into it. The 
VertexGroupReduce outputs two semantically different tuples: one for 
each vertex without the value (null) and one tuple representing the 
whole group including the value. As Till pointed out, this crashes if 
the value has no serializer for null values.


Since we cannot have two different output types for the same GroupReduce 
function, I propose using a Either here. If there are no 
objections I will take the issue and fix it.


Again, thx for pointing it out.

Best,

Martin

On 15.09.2016 20:10, Vasiliki Kalavri wrote:

Hi,

thanks for looking into this Till! I'm not quite sure what the algorithm
behavior should be when the vertex value is null (probably skip the
record?). Let's wait for Martin's input.

Cheers,
-V.

On 15 September 2016 at 19:19, Olga Golovneva  wrote:


Hi Till,

Thanks a lot for your help! I'll try to use another variable type in the
meantime.

Best regards,
Olga


Best regards,
Olga Golovneva

On Thu, Sep 15, 2016 at 1:03 PM, Till Rohrmann 
wrote:


Hi Olga,

it’s indeed an error in Flink’s Summarization algorithm. The problem is

the

following: The vertex group value of the VertexGroupItem is null in the
VertexGroupReducer. This works in the SummarizationIT case because the
vertex value is of type String and the StringSerializer can deal with

null

values.

However, in your case where you use longs, it fails, because the
LongSerializer cannot handle null values. You can verify this behaviour

by

changing the vertex value type to String. Then everything should work
without a problem.

I’ve cc’ed Martin who can tell you probably more about the Summarization
algorithm. I’ve also opened a JIRA ticket [1] to fix this problem.

Thanks for reporting this bug.

[1] https://issues.apache.org/jira/browse/FLINK-4624

Cheers,
Till
​

On Thu, Sep 15, 2016 at 5:05 PM, Olga Golovneva 
wrote:


Hi Till,

I've created a simple (Java) example to show you what's going on. The

code

is in attachment and shown below. This example creates simple graph

with

Double EV and Long VV. Then it runs Summarization, that should compute

a

condensed version of the input graph by grouping vertices and edges

based

on their values. I run this code with IntelliJ IDEA. The code executes

fine

until you want to see what is written in resulted edges (just uncomment
line 46, edgesOut.print();). Then it throws the following Exception:

_EXCEPTION START_
Exception in thread "main" org.apache.flink.runtime.

client.JobExecutionException:

Job execution failed.
at org.apache.flink.runtime.jobmanager.JobManager$$
anonfun$handleMessage$1$$anonfun$applyOrElse$8.apply$
mcV$sp(JobManager.scala:830)
at org.apache.flink.runtime.jobmanager.JobManager$$
anonfun$handleMessage$1$$anonfun$applyOrElse$8.apply(

JobManager.scala:773)

at org.apache.flink.runtime.jobmanager.JobManager$$
anonfun$handleMessage$1$$anonfun$applyOrElse$8.apply(

JobManager.scala:773)

at scala.concurrent.impl.Future$PromiseCompletingRunnable.
liftedTree1$1(Future.scala:24)
at scala.concurrent.impl.Future$PromiseCompletingRunnable.run(
Future.scala:24)
at akka.dispatch.TaskInvocation.run(AbstractDispatcher.scala:41)
at akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(
AbstractDispatcher.scala:401)
at scala.concurrent.forkjoin.ForkJoinTask.doExec(

ForkJoinTask.java:260)

at scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.
pollAndExecAll(ForkJoinPool.java:1253)
at scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.
runTask(ForkJoinPool.java:1346)
at scala.concurrent.forkjoin.ForkJoinPool.runWorker(
ForkJoinPool.java:1979)
at scala.concurrent.forkjoin.ForkJoinWorkerThread.run(
ForkJoinWorkerThread.java:107)
Caused by: org.apache.flink.types.NullFieldException: Field 2 is null,
but expected to hold a value.
at org.apache.flink.api.java.typeutils.runtime.

TupleSerializer.serialize(

TupleSerializer.java:126)
at org.apache.flink.api.java.typeutils.runtime.

TupleSerializer.serialize(

TupleSerializer.java:30)
at org.apache.flink.runtime.plugable.SerializationDelegate.write(
SerializationDelegate.java:56)
at org.apache.flink.runtime.io.network.api.serialization.
SpanningRecordSerializer.addRecord(SpanningRecordSerializer.java:83)
at org.apache.flink.runtime.io.network.api.writer.RecordWriter.emit(
RecordWriter.java:85)
at org.apache.flink.runtime.operators.shipping.

OutputCollector.collect(

OutputCollector.java:65)
at org.apache.flink.runtime.operators.util.metrics.
CountingCollector.collect(CountingCollector.java:35)
at org.apache.flink.api.java.operators.translation.PlanFilterOperator$
FlatMapFilter.flatMap(PlanFilterOperator.java:51)
at org.apache.flink.runtime.operators.chaining.
ChainedFlatMapDriver.collect(ChainedFlatMapDriver.java:80)
at org.apache.flink.runtime.operators.util.metrics.
CountingCollector.collect(CountingCollector.java:35)
at org.apache.flink.graph.library.Summarization$

VertexGroupReducer.reduce(

Summarization.java:323)
at org.

Re: Gelly Library. Need an example

2016-09-15 Thread Vasiliki Kalavri
Hi,

thanks for looking into this Till! I'm not quite sure what the algorithm
behavior should be when the vertex value is null (probably skip the
record?). Let's wait for Martin's input.

Cheers,
-V.

On 15 September 2016 at 19:19, Olga Golovneva  wrote:

> Hi Till,
>
> Thanks a lot for your help! I'll try to use another variable type in the
> meantime.
>
> Best regards,
> Olga
>
>
> Best regards,
> Olga Golovneva
>
> On Thu, Sep 15, 2016 at 1:03 PM, Till Rohrmann 
> wrote:
>
> > Hi Olga,
> >
> > it’s indeed an error in Flink’s Summarization algorithm. The problem is
> the
> > following: The vertex group value of the VertexGroupItem is null in the
> > VertexGroupReducer. This works in the SummarizationIT case because the
> > vertex value is of type String and the StringSerializer can deal with
> null
> > values.
> >
> > However, in your case where you use longs, it fails, because the
> > LongSerializer cannot handle null values. You can verify this behaviour
> by
> > changing the vertex value type to String. Then everything should work
> > without a problem.
> >
> > I’ve cc’ed Martin who can tell you probably more about the Summarization
> > algorithm. I’ve also opened a JIRA ticket [1] to fix this problem.
> >
> > Thanks for reporting this bug.
> >
> > [1] https://issues.apache.org/jira/browse/FLINK-4624
> >
> > Cheers,
> > Till
> > ​
> >
> > On Thu, Sep 15, 2016 at 5:05 PM, Olga Golovneva 
> > wrote:
> >
> > > Hi Till,
> > >
> > > I've created a simple (Java) example to show you what's going on. The
> > code
> > > is in attachment and shown below. This example creates simple graph
> with
> > > Double EV and Long VV. Then it runs Summarization, that should compute
> a
> > > condensed version of the input graph by grouping vertices and edges
> based
> > > on their values. I run this code with IntelliJ IDEA. The code executes
> > fine
> > > until you want to see what is written in resulted edges (just uncomment
> > > line 46, edgesOut.print();). Then it throws the following Exception:
> > >
> > > _EXCEPTION START_
> > > Exception in thread "main" org.apache.flink.runtime.
> > client.JobExecutionException:
> > > Job execution failed.
> > > at org.apache.flink.runtime.jobmanager.JobManager$$
> > > anonfun$handleMessage$1$$anonfun$applyOrElse$8.apply$
> > > mcV$sp(JobManager.scala:830)
> > > at org.apache.flink.runtime.jobmanager.JobManager$$
> > > anonfun$handleMessage$1$$anonfun$applyOrElse$8.apply(
> > JobManager.scala:773)
> > > at org.apache.flink.runtime.jobmanager.JobManager$$
> > > anonfun$handleMessage$1$$anonfun$applyOrElse$8.apply(
> > JobManager.scala:773)
> > > at scala.concurrent.impl.Future$PromiseCompletingRunnable.
> > > liftedTree1$1(Future.scala:24)
> > > at scala.concurrent.impl.Future$PromiseCompletingRunnable.run(
> > > Future.scala:24)
> > > at akka.dispatch.TaskInvocation.run(AbstractDispatcher.scala:41)
> > > at akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(
> > > AbstractDispatcher.scala:401)
> > > at scala.concurrent.forkjoin.ForkJoinTask.doExec(
> ForkJoinTask.java:260)
> > > at scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.
> > > pollAndExecAll(ForkJoinPool.java:1253)
> > > at scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.
> > > runTask(ForkJoinPool.java:1346)
> > > at scala.concurrent.forkjoin.ForkJoinPool.runWorker(
> > > ForkJoinPool.java:1979)
> > > at scala.concurrent.forkjoin.ForkJoinWorkerThread.run(
> > > ForkJoinWorkerThread.java:107)
> > > Caused by: org.apache.flink.types.NullFieldException: Field 2 is null,
> > > but expected to hold a value.
> > > at org.apache.flink.api.java.typeutils.runtime.
> > TupleSerializer.serialize(
> > > TupleSerializer.java:126)
> > > at org.apache.flink.api.java.typeutils.runtime.
> > TupleSerializer.serialize(
> > > TupleSerializer.java:30)
> > > at org.apache.flink.runtime.plugable.SerializationDelegate.write(
> > > SerializationDelegate.java:56)
> > > at org.apache.flink.runtime.io.network.api.serialization.
> > > SpanningRecordSerializer.addRecord(SpanningRecordSerializer.java:83)
> > > at org.apache.flink.runtime.io.network.api.writer.RecordWriter.emit(
> > > RecordWriter.java:85)
> > > at org.apache.flink.runtime.operators.shipping.
> OutputCollector.collect(
> > > OutputCollector.java:65)
> > > at org.apache.flink.runtime.operators.util.metrics.
> > > CountingCollector.collect(CountingCollector.java:35)
> > > at org.apache.flink.api.java.operators.translation.PlanFilterOperator$
> > > FlatMapFilter.flatMap(PlanFilterOperator.java:51)
> > > at org.apache.flink.runtime.operators.chaining.
> > > ChainedFlatMapDriver.collect(ChainedFlatMapDriver.java:80)
> > > at org.apache.flink.runtime.operators.util.metrics.
> > > CountingCollector.collect(CountingCollector.java:35)
> > > at org.apache.flink.graph.library.Summarization$
> > VertexGroupReducer.reduce(
> > > Summarization.java:323)
> > > at org.apache.flink.runtime.operators.GroupReduceDriver.
> > > run(GroupReduceDriver.java:131)
> > > at or

Re: Gelly Library. Need an example

2016-09-15 Thread Olga Golovneva
Hi Till,

Thanks a lot for your help! I'll try to use another variable type in the
meantime.

Best regards,
Olga


Best regards,
Olga Golovneva

On Thu, Sep 15, 2016 at 1:03 PM, Till Rohrmann  wrote:

> Hi Olga,
>
> it’s indeed an error in Flink’s Summarization algorithm. The problem is the
> following: The vertex group value of the VertexGroupItem is null in the
> VertexGroupReducer. This works in the SummarizationIT case because the
> vertex value is of type String and the StringSerializer can deal with null
> values.
>
> However, in your case where you use longs, it fails, because the
> LongSerializer cannot handle null values. You can verify this behaviour by
> changing the vertex value type to String. Then everything should work
> without a problem.
>
> I’ve cc’ed Martin who can tell you probably more about the Summarization
> algorithm. I’ve also opened a JIRA ticket [1] to fix this problem.
>
> Thanks for reporting this bug.
>
> [1] https://issues.apache.org/jira/browse/FLINK-4624
>
> Cheers,
> Till
> ​
>
> On Thu, Sep 15, 2016 at 5:05 PM, Olga Golovneva 
> wrote:
>
> > Hi Till,
> >
> > I've created a simple (Java) example to show you what's going on. The
> code
> > is in attachment and shown below. This example creates simple graph with
> > Double EV and Long VV. Then it runs Summarization, that should compute a
> > condensed version of the input graph by grouping vertices and edges based
> > on their values. I run this code with IntelliJ IDEA. The code executes
> fine
> > until you want to see what is written in resulted edges (just uncomment
> > line 46, edgesOut.print();). Then it throws the following Exception:
> >
> > _EXCEPTION START_
> > Exception in thread "main" org.apache.flink.runtime.
> client.JobExecutionException:
> > Job execution failed.
> > at org.apache.flink.runtime.jobmanager.JobManager$$
> > anonfun$handleMessage$1$$anonfun$applyOrElse$8.apply$
> > mcV$sp(JobManager.scala:830)
> > at org.apache.flink.runtime.jobmanager.JobManager$$
> > anonfun$handleMessage$1$$anonfun$applyOrElse$8.apply(
> JobManager.scala:773)
> > at org.apache.flink.runtime.jobmanager.JobManager$$
> > anonfun$handleMessage$1$$anonfun$applyOrElse$8.apply(
> JobManager.scala:773)
> > at scala.concurrent.impl.Future$PromiseCompletingRunnable.
> > liftedTree1$1(Future.scala:24)
> > at scala.concurrent.impl.Future$PromiseCompletingRunnable.run(
> > Future.scala:24)
> > at akka.dispatch.TaskInvocation.run(AbstractDispatcher.scala:41)
> > at akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(
> > AbstractDispatcher.scala:401)
> > at scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
> > at scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.
> > pollAndExecAll(ForkJoinPool.java:1253)
> > at scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.
> > runTask(ForkJoinPool.java:1346)
> > at scala.concurrent.forkjoin.ForkJoinPool.runWorker(
> > ForkJoinPool.java:1979)
> > at scala.concurrent.forkjoin.ForkJoinWorkerThread.run(
> > ForkJoinWorkerThread.java:107)
> > Caused by: org.apache.flink.types.NullFieldException: Field 2 is null,
> > but expected to hold a value.
> > at org.apache.flink.api.java.typeutils.runtime.
> TupleSerializer.serialize(
> > TupleSerializer.java:126)
> > at org.apache.flink.api.java.typeutils.runtime.
> TupleSerializer.serialize(
> > TupleSerializer.java:30)
> > at org.apache.flink.runtime.plugable.SerializationDelegate.write(
> > SerializationDelegate.java:56)
> > at org.apache.flink.runtime.io.network.api.serialization.
> > SpanningRecordSerializer.addRecord(SpanningRecordSerializer.java:83)
> > at org.apache.flink.runtime.io.network.api.writer.RecordWriter.emit(
> > RecordWriter.java:85)
> > at org.apache.flink.runtime.operators.shipping.OutputCollector.collect(
> > OutputCollector.java:65)
> > at org.apache.flink.runtime.operators.util.metrics.
> > CountingCollector.collect(CountingCollector.java:35)
> > at org.apache.flink.api.java.operators.translation.PlanFilterOperator$
> > FlatMapFilter.flatMap(PlanFilterOperator.java:51)
> > at org.apache.flink.runtime.operators.chaining.
> > ChainedFlatMapDriver.collect(ChainedFlatMapDriver.java:80)
> > at org.apache.flink.runtime.operators.util.metrics.
> > CountingCollector.collect(CountingCollector.java:35)
> > at org.apache.flink.graph.library.Summarization$
> VertexGroupReducer.reduce(
> > Summarization.java:323)
> > at org.apache.flink.runtime.operators.GroupReduceDriver.
> > run(GroupReduceDriver.java:131)
> > at org.apache.flink.runtime.operators.BatchTask.run(BatchTask.java:486)
> > at org.apache.flink.runtime.operators.BatchTask.invoke(
> BatchTask.java:351)
> > at org.apache.flink.runtime.taskmanager.Task.run(Task.java:590)
> > at java.lang.Thread.run(Thread.java:745)
> > Caused by: java.lang.NullPointerException
> > at org.apache.flink.api.common.typeutils.base.LongSerializer.
> > serialize(LongSerializer.java:64)
> > at org.apache.flink.api.common.typeutils.base.LongSerializer.
> > serialize(Lo

Re: Gelly Library. Need an example

2016-09-15 Thread Till Rohrmann
Hi Olga,

it’s indeed an error in Flink’s Summarization algorithm. The problem is the
following: The vertex group value of the VertexGroupItem is null in the
VertexGroupReducer. This works in the SummarizationIT case because the
vertex value is of type String and the StringSerializer can deal with null
values.

However, in your case where you use longs, it fails, because the
LongSerializer cannot handle null values. You can verify this behaviour by
changing the vertex value type to String. Then everything should work
without a problem.

I’ve cc’ed Martin who can tell you probably more about the Summarization
algorithm. I’ve also opened a JIRA ticket [1] to fix this problem.

Thanks for reporting this bug.

[1] https://issues.apache.org/jira/browse/FLINK-4624

Cheers,
Till
​

On Thu, Sep 15, 2016 at 5:05 PM, Olga Golovneva  wrote:

> Hi Till,
>
> I've created a simple (Java) example to show you what's going on. The code
> is in attachment and shown below. This example creates simple graph with
> Double EV and Long VV. Then it runs Summarization, that should compute a
> condensed version of the input graph by grouping vertices and edges based
> on their values. I run this code with IntelliJ IDEA. The code executes fine
> until you want to see what is written in resulted edges (just uncomment
> line 46, edgesOut.print();). Then it throws the following Exception:
>
> _EXCEPTION START_
> Exception in thread "main" 
> org.apache.flink.runtime.client.JobExecutionException:
> Job execution failed.
> at org.apache.flink.runtime.jobmanager.JobManager$$
> anonfun$handleMessage$1$$anonfun$applyOrElse$8.apply$
> mcV$sp(JobManager.scala:830)
> at org.apache.flink.runtime.jobmanager.JobManager$$
> anonfun$handleMessage$1$$anonfun$applyOrElse$8.apply(JobManager.scala:773)
> at org.apache.flink.runtime.jobmanager.JobManager$$
> anonfun$handleMessage$1$$anonfun$applyOrElse$8.apply(JobManager.scala:773)
> at scala.concurrent.impl.Future$PromiseCompletingRunnable.
> liftedTree1$1(Future.scala:24)
> at scala.concurrent.impl.Future$PromiseCompletingRunnable.run(
> Future.scala:24)
> at akka.dispatch.TaskInvocation.run(AbstractDispatcher.scala:41)
> at akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(
> AbstractDispatcher.scala:401)
> at scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
> at scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.
> pollAndExecAll(ForkJoinPool.java:1253)
> at scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.
> runTask(ForkJoinPool.java:1346)
> at scala.concurrent.forkjoin.ForkJoinPool.runWorker(
> ForkJoinPool.java:1979)
> at scala.concurrent.forkjoin.ForkJoinWorkerThread.run(
> ForkJoinWorkerThread.java:107)
> Caused by: org.apache.flink.types.NullFieldException: Field 2 is null,
> but expected to hold a value.
> at org.apache.flink.api.java.typeutils.runtime.TupleSerializer.serialize(
> TupleSerializer.java:126)
> at org.apache.flink.api.java.typeutils.runtime.TupleSerializer.serialize(
> TupleSerializer.java:30)
> at org.apache.flink.runtime.plugable.SerializationDelegate.write(
> SerializationDelegate.java:56)
> at org.apache.flink.runtime.io.network.api.serialization.
> SpanningRecordSerializer.addRecord(SpanningRecordSerializer.java:83)
> at org.apache.flink.runtime.io.network.api.writer.RecordWriter.emit(
> RecordWriter.java:85)
> at org.apache.flink.runtime.operators.shipping.OutputCollector.collect(
> OutputCollector.java:65)
> at org.apache.flink.runtime.operators.util.metrics.
> CountingCollector.collect(CountingCollector.java:35)
> at org.apache.flink.api.java.operators.translation.PlanFilterOperator$
> FlatMapFilter.flatMap(PlanFilterOperator.java:51)
> at org.apache.flink.runtime.operators.chaining.
> ChainedFlatMapDriver.collect(ChainedFlatMapDriver.java:80)
> at org.apache.flink.runtime.operators.util.metrics.
> CountingCollector.collect(CountingCollector.java:35)
> at org.apache.flink.graph.library.Summarization$VertexGroupReducer.reduce(
> Summarization.java:323)
> at org.apache.flink.runtime.operators.GroupReduceDriver.
> run(GroupReduceDriver.java:131)
> at org.apache.flink.runtime.operators.BatchTask.run(BatchTask.java:486)
> at org.apache.flink.runtime.operators.BatchTask.invoke(BatchTask.java:351)
> at org.apache.flink.runtime.taskmanager.Task.run(Task.java:590)
> at java.lang.Thread.run(Thread.java:745)
> Caused by: java.lang.NullPointerException
> at org.apache.flink.api.common.typeutils.base.LongSerializer.
> serialize(LongSerializer.java:64)
> at org.apache.flink.api.common.typeutils.base.LongSerializer.
> serialize(LongSerializer.java:27)
> at org.apache.flink.api.java.typeutils.runtime.TupleSerializer.serialize(
> TupleSerializer.java:124)
> ... 15 more
>
> _EXCEPTION END__
>
> It looks like the problem is in the following lines in Summarization:
>
> DataSet> edgesForGrouping = input.getEdges()
>   .join(vertexToRepresentativeMap)
>   .where(0)  // source vertex id
>   .equalTo(0)

Re: Gelly Library. Need an example

2016-09-15 Thread Olga Golovneva
Hi Till,

I've created a simple (Java) example to show you what's going on. The code
is in attachment and shown below. This example creates simple graph with
Double EV and Long VV. Then it runs Summarization, that should compute a
condensed version of the input graph by grouping vertices and edges based
on their values. I run this code with IntelliJ IDEA. The code executes fine
until you want to see what is written in resulted edges (just uncomment
line 46, edgesOut.print();). Then it throws the following Exception:

_EXCEPTION START_
Exception in thread "main"
org.apache.flink.runtime.client.JobExecutionException: Job execution failed.
at
org.apache.flink.runtime.jobmanager.JobManager$$anonfun$handleMessage$1$$anonfun$applyOrElse$8.apply$mcV$sp(JobManager.scala:830)
at
org.apache.flink.runtime.jobmanager.JobManager$$anonfun$handleMessage$1$$anonfun$applyOrElse$8.apply(JobManager.scala:773)
at
org.apache.flink.runtime.jobmanager.JobManager$$anonfun$handleMessage$1$$anonfun$applyOrElse$8.apply(JobManager.scala:773)
at
scala.concurrent.impl.Future$PromiseCompletingRunnable.liftedTree1$1(Future.scala:24)
at
scala.concurrent.impl.Future$PromiseCompletingRunnable.run(Future.scala:24)
at akka.dispatch.TaskInvocation.run(AbstractDispatcher.scala:41)
at
akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(AbstractDispatcher.scala:401)
at scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
at
scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.pollAndExecAll(ForkJoinPool.java:1253)
at
scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1346)
at scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)
at
scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)
Caused by: org.apache.flink.types.NullFieldException: Field 2 is null, but
expected to hold a value.
at
org.apache.flink.api.java.typeutils.runtime.TupleSerializer.serialize(TupleSerializer.java:126)
at
org.apache.flink.api.java.typeutils.runtime.TupleSerializer.serialize(TupleSerializer.java:30)
at
org.apache.flink.runtime.plugable.SerializationDelegate.write(SerializationDelegate.java:56)
at
org.apache.flink.runtime.io.network.api.serialization.SpanningRecordSerializer.addRecord(SpanningRecordSerializer.java:83)
at
org.apache.flink.runtime.io.network.api.writer.RecordWriter.emit(RecordWriter.java:85)
at
org.apache.flink.runtime.operators.shipping.OutputCollector.collect(OutputCollector.java:65)
at
org.apache.flink.runtime.operators.util.metrics.CountingCollector.collect(CountingCollector.java:35)
at
org.apache.flink.api.java.operators.translation.PlanFilterOperator$FlatMapFilter.flatMap(PlanFilterOperator.java:51)
at
org.apache.flink.runtime.operators.chaining.ChainedFlatMapDriver.collect(ChainedFlatMapDriver.java:80)
at
org.apache.flink.runtime.operators.util.metrics.CountingCollector.collect(CountingCollector.java:35)
at
org.apache.flink.graph.library.Summarization$VertexGroupReducer.reduce(Summarization.java:323)
at
org.apache.flink.runtime.operators.GroupReduceDriver.run(GroupReduceDriver.java:131)
at org.apache.flink.runtime.operators.BatchTask.run(BatchTask.java:486)
at org.apache.flink.runtime.operators.BatchTask.invoke(BatchTask.java:351)
at org.apache.flink.runtime.taskmanager.Task.run(Task.java:590)
at java.lang.Thread.run(Thread.java:745)
Caused by: java.lang.NullPointerException
at
org.apache.flink.api.common.typeutils.base.LongSerializer.serialize(LongSerializer.java:64)
at
org.apache.flink.api.common.typeutils.base.LongSerializer.serialize(LongSerializer.java:27)
at
org.apache.flink.api.java.typeutils.runtime.TupleSerializer.serialize(TupleSerializer.java:124)
... 15 more

_EXCEPTION END__

It looks like the problem is in the following lines in Summarization:

DataSet> edgesForGrouping = input.getEdges()
  .join(vertexToRepresentativeMap)
  .where(0)  // source vertex id
  .equalTo(0) // vertex id
  .with(new SourceVertexJoinFunction())
  .join(vertexToRepresentativeMap)
  .where(1)  // target vertex id
  .equalTo(0) // vertex id
  .with(new TargetVertexJoinFunction());


If you try to print edges before this step, it works fine. But after this
step my IDE gives the same exception.

I would really appreciate any help.

Thank you,
Olga

_EXAMPLE START_

package org.apache.flink.graph.examples;

import org.apache.flink.api.common.ProgramDescription;
import org.apache.flink.api.java.DataSet;
import org.apache.flink.api.java.ExecutionEnvironment;
import org.apache.flink.graph.Edge;
import org.apache.flink.graph.Graph;
import org.apache.flink.graph.Vertex;
import org.apache.flink.graph.library.Summarization;
import java.util.LinkedList;
import java.util.List;

public class MySummarizationExample implements ProgramDescription {

@SuppressWarnings("serial")
public static void main(String [] args) throws Exception {

ExecutionEnvironment env 

Re: Gelly Library. Need an example

2016-09-15 Thread Till Rohrmann
Hi Olga,

can you provide us with a little bit more details about the problem. The
full stack trace of the exception and the program you're trying to run
would be helpful.

Cheers,
Till

On Wed, Sep 14, 2016 at 9:49 PM, Olga Golovneva  wrote:

> Hi devs,
>
> Do you know if there is an example (besides ITCase) of usage of
> Summarization Library in Gelly? I'm having some problems trying to use it
> in my code. Particularly, I cannot print output edges ( it throws the
> following exception: Exception in thread "main"
> org.apache.flink.runtime.client.JobExecutionException: Job execution
> failed.), while vertices are printed correctly.
>
> Best regards,
> Olga
>