Re: Union of multiple datasets vs Join

2015-03-18 Thread Flavio Pompermaier
I don't know if that could be useful, do you?

On Tue, Mar 17, 2015 at 10:29 PM, Stephan Ewen  wrote:

> Yes, that is the way to do it.
>
> This makes me think that it would be nice to have a method that builds the
> union of a list of data sets.
>
> DataSet union(DataSet... sets)
>
> It would be implemented like in your loop. Would that be helpful?
>
> Stephan
>
>
> On Tue, Mar 17, 2015 at 6:03 PM, Flavio Pompermaier 
> wrote:
>
>> As always one minute after I sent the email I found the problem!
>> It was that I should reassign the initial dataset:
>> ret = ret.union(sourceDs);
>>
>> Bye,
>> Flavio
>>
>> On Tue, Mar 17, 2015 at 5:58 PM, Flavio Pompermaier > > wrote:
>>
>>> Hi Fabian,
>>> I was trying to use the strategy you suggested with flink 0.8.1 but it
>>> seems that the union of the datasets cannot be created programmatically
>>> because the union operator gives a name to the generated dataset that is
>>> the name of the calling function so that  only the first dataset is read.
>>> My code looks like:
>>>
>>> private static DataSet getSourceDs(ExecutionEnvironment env,
>>>  final String outputGraph, List tableNames) {
>>> DataSet> ret = null;
>>> for (String tableName : tableNames) {
>>> DataSet> sourceDs = env.createInput(new
>>> MyTableInputFormat(tableName))
>>> 
>>>
>>> if(ret==null)
>>> ret = sourceDs;
>>> else
>>> ret.union(sourceDs);
>>>}
>>>   return ret;
>>>}
>>>
>>> Is this a bug or am I'm doing something wrong?
>>> Thanks in advance,
>>> Flavio
>>>
>>> On Mon, Dec 22, 2014 at 2:42 PM,  wrote:
>>>
  Union is just combining data from multiple sources into a single
 dataset.
 That’s it. No memory, no disk involved.

 In you case you have

 input1.union(input2).groupBy(1).reduce(…)

 This will translate into:

 input1 -> repartition ->
 read-both-inputs ->  sort ->
 reduce
 input2 -> repartition ->

 So, in your case not even additional network transfer is involved,
 because both data sets would need to be partitioned for the reduce anyway.

 Note, union in Flink has SQL union-all semantics, i.e., there is
 not removal of duplicates.

 Cheers, Fabian

 *From:* Flavio Pompermaier 
 *Sent:* ‎Monday‎, ‎22‎. ‎December‎, ‎2014 ‎14‎:‎32
 *To:* u...@flink.incubator.apache.org

 Ok thanks Fabian. I'd like just to know the internals of the union of
 multiple datasets (partitioning, distribution among server, memory/disk,
 etc..). Do you have any ref to this?

 Thanks in advance,
 Flavio

 On Mon, Dec 22, 2014 at 12:46 PM, Fabian Hueske 
 wrote:

> Follow the first approach.
> Joins are expensive, union comes for free.
>
> Best, Fabian
>
> 2014-12-22 11:47 GMT+01:00 Flavio Pompermaier :
>
>> Hi guys,
>>
>> In my use case I have multiple Datasets with the same structure (e.g.
>> Tuple3) and I want to produce an output Dataset containing all Tuple3
>> grouped by the first field (0).
>> I can obtain the same results performing a union of all datasets and
>> then a group by (simplest implementation) or join all of them pairwise
>> (((A->B)->C)->D)..) or I don't know if there is any other solution. When
>> should I use the first or the second approach? Could you help me in
>> figuring out the internals of the two approaches? I always have some fear
>> when using multiple joins when I don't know exactly their size..
>>
>> Best,
>> Flavio
>>
>
>

>>
>


Re: RuntimeException Gelly API: Memory ran out. Compaction failed.

2015-03-18 Thread Robert Waury
Hi,

I managed to reproduce the behavior and as far as I can tell it seems to be
a problem with the memory allocation.

I have filed a bug report in JIRA to get the attention of somebody who
knows the runtime better than I do.

https://issues.apache.org/jira/browse/FLINK-1734

Cheers,
Robert

On Tue, Mar 17, 2015 at 3:52 PM, Mihail Vieru  wrote:

>  Hi Robert,
>
> thank you for your reply.
>
> I'm starting the job from the Scala IDE. So only one JobManager and one
> TaskManager in the same JVM.
> I've doubled the memory in the eclipse.ini settings but I still get the
> Exception.
>
> -vmargs
> -Xmx2048m
> -Xms100m
> -XX:MaxPermSize=512m
>
> Best,
> Mihail
>
>
> On 17.03.2015 10:11, Robert Waury wrote:
>
>   Hi,
>
>  can you tell me how much memory your job has and how many workers you are
> running?
>
>  From the trace it seems the internal hash table allocated only 7 MB for
> the graph data and therefore runs out of memory pretty quickly.
>
>  Skewed data could also be an issue but with a minimum of 5 pages and a
> maximum of 8 it seems to be distributed fairly even to the different
> partitions.
>
>  Cheers,
>  Robert
>
> On Tue, Mar 17, 2015 at 1:25 AM, Mihail Vieru <
> vi...@informatik.hu-berlin.de> wrote:
>
>> And the correct SSSPUnweighted attached.
>>
>>
>> On 17.03.2015 01:23, Mihail Vieru wrote:
>>
>>> Hi,
>>>
>>> I'm getting the following RuntimeException for an adaptation of the
>>> SingleSourceShortestPaths example using the Gelly API (see attachment).
>>> It's been adapted for unweighted graphs having vertices with Long values.
>>>
>>> As an input graph I'm using the social network graph (~200MB unpacked)
>>> from here: https://snap.stanford.edu/data/higgs-twitter.html
>>>
>>> For the small SSSPDataUnweighted graph (also attached) it terminates and
>>> computes the distances correctly.
>>>
>>>
>>> 03/16/2015 17:18:23IterationHead(WorksetIteration (Vertex-centric
>>> iteration
>>> (org.apache.flink.graph.library.SingleSourceShortestPathsUnweighted$VertexDistanceUpdater@dca6fe4
>>> |
>>> org.apache.flink.graph.library.SingleSourceShortestPathsUnweighted$MinDistanceMessenger@6577e8ce)))(2/4)
>>> switched to FAILED
>>> java.lang.RuntimeException: Memory ran out. Compaction failed.
>>> numPartitions: 32 minPartition: 5 maxPartition: 8 number of overflow
>>> segments: 176 bucketSize: 217 Overall memory: 20316160 Partition memory:
>>> 7208960 Message: Index: 8, Size: 7
>>> at
>>> org.apache.flink.runtime.operators.hash.CompactingHashTable.insert(CompactingHashTable.java:390)
>>> at
>>> org.apache.flink.runtime.operators.hash.CompactingHashTable.buildTable(CompactingHashTable.java:337)
>>> at
>>> org.apache.flink.runtime.iterative.task.IterationHeadPactTask.readInitialSolutionSet(IterationHeadPactTask.java:216)
>>> at
>>> org.apache.flink.runtime.iterative.task.IterationHeadPactTask.run(IterationHeadPactTask.java:278)
>>> at
>>> org.apache.flink.runtime.operators.RegularPactTask.invoke(RegularPactTask.java:362)
>>> at
>>> org.apache.flink.runtime.execution.RuntimeEnvironment.run(RuntimeEnvironment.java:205)
>>> at java.lang.Thread.run(Thread.java:745)
>>>
>>>
>>> Best,
>>> Mihail
>>>
>>
>>
>
>


Re: RuntimeException Gelly API: Memory ran out. Compaction failed.

2015-03-18 Thread Mihail Vieru

Hi,

great! Thanks!

I really need this bug fixed because I'm laying the groundwork for my 
Diplom thesis and I need to be sure that the Gelly API is reliable and 
can handle large datasets as intended.


Cheers,
Mihail

On 18.03.2015 15:40, Robert Waury wrote:

Hi,

I managed to reproduce the behavior and as far as I can tell it seems 
to be a problem with the memory allocation.


I have filed a bug report in JIRA to get the attention of somebody who 
knows the runtime better than I do.


https://issues.apache.org/jira/browse/FLINK-1734 



Cheers,
Robert

On Tue, Mar 17, 2015 at 3:52 PM, Mihail Vieru 
mailto:vi...@informatik.hu-berlin.de>> 
wrote:


Hi Robert,

thank you for your reply.

I'm starting the job from the Scala IDE. So only one JobManager
and one TaskManager in the same JVM.
I've doubled the memory in the eclipse.ini settings but I still
get the Exception.

-vmargs
-Xmx2048m
-Xms100m
-XX:MaxPermSize=512m

Best,
Mihail


On 17.03.2015 10:11, Robert Waury wrote:

Hi,

can you tell me how much memory your job has and how many workers
you are running?

From the trace it seems the internal hash table allocated only 7
MB for the graph data and therefore runs out of memory pretty
quickly.

Skewed data could also be an issue but with a minimum of 5 pages
and a maximum of 8 it seems to be distributed fairly even to the
different partitions.

Cheers,
Robert

On Tue, Mar 17, 2015 at 1:25 AM, Mihail Vieru
mailto:vi...@informatik.hu-berlin.de>> wrote:

And the correct SSSPUnweighted attached.


On 17.03.2015 01:23, Mihail Vieru wrote:

Hi,

I'm getting the following RuntimeException for an
adaptation of the SingleSourceShortestPaths example using
the Gelly API (see attachment). It's been adapted for
unweighted graphs having vertices with Long values.

As an input graph I'm using the social network graph
(~200MB unpacked) from here:
https://snap.stanford.edu/data/higgs-twitter.html

For the small SSSPDataUnweighted graph (also attached) it
terminates and computes the distances correctly.


03/16/2015 17:18:23 IterationHead(WorksetIteration
(Vertex-centric iteration

(org.apache.flink.graph.library.SingleSourceShortestPathsUnweighted$VertexDistanceUpdater@dca6fe4
|

org.apache.flink.graph.library.SingleSourceShortestPathsUnweighted$MinDistanceMessenger@6577e8ce)))(2/4)
switched to FAILED
java.lang.RuntimeException: Memory ran out. Compaction
failed. numPartitions: 32 minPartition: 5 maxPartition: 8
number of overflow segments: 176 bucketSize: 217 Overall
memory: 20316160 Partition memory: 7208960 Message:
Index: 8, Size: 7
at

org.apache.flink.runtime.operators.hash.CompactingHashTable.insert(CompactingHashTable.java:390)
at

org.apache.flink.runtime.operators.hash.CompactingHashTable.buildTable(CompactingHashTable.java:337)
at

org.apache.flink.runtime.iterative.task.IterationHeadPactTask.readInitialSolutionSet(IterationHeadPactTask.java:216)
at

org.apache.flink.runtime.iterative.task.IterationHeadPactTask.run(IterationHeadPactTask.java:278)
at

org.apache.flink.runtime.operators.RegularPactTask.invoke(RegularPactTask.java:362)
at

org.apache.flink.runtime.execution.RuntimeEnvironment.run(RuntimeEnvironment.java:205)
at java.lang.Thread.run(Thread.java:745)


Best,
Mihail










Re: RuntimeException Gelly API: Memory ran out. Compaction failed.

2015-03-18 Thread Stephan Ewen
This job probably suffers from overly conservative memory assignment,
giving the solution set too little memory.

Can you try to make the solution set "unmanaged", excluding it from Flink's
memory management? That may help with the problem.

See here:
https://github.com/apache/flink/blob/master/flink-java/src/main/java/org/apache/flink/api/java/operators/DeltaIteration.java#L224



On Wed, Mar 18, 2015 at 3:54 PM, Mihail Vieru  wrote:

>  Hi,
>
> great! Thanks!
>
> I really need this bug fixed because I'm laying the groundwork for my
> Diplom thesis and I need to be sure that the Gelly API is reliable and can
> handle large datasets as intended.
>
> Cheers,
> Mihail
>
>
> On 18.03.2015 15:40, Robert Waury wrote:
>
>   Hi,
>
>  I managed to reproduce the behavior and as far as I can tell it seems to
> be a problem with the memory allocation.
>
>  I have filed a bug report in JIRA to get the attention of somebody who
> knows the runtime better than I do.
>
> https://issues.apache.org/jira/browse/FLINK-1734
>
>  Cheers,
>  Robert
>
> On Tue, Mar 17, 2015 at 3:52 PM, Mihail Vieru <
> vi...@informatik.hu-berlin.de> wrote:
>
>>  Hi Robert,
>>
>> thank you for your reply.
>>
>> I'm starting the job from the Scala IDE. So only one JobManager and one
>> TaskManager in the same JVM.
>> I've doubled the memory in the eclipse.ini settings but I still get the
>> Exception.
>>
>> -vmargs
>> -Xmx2048m
>> -Xms100m
>> -XX:MaxPermSize=512m
>>
>> Best,
>> Mihail
>>
>>
>> On 17.03.2015 10:11, Robert Waury wrote:
>>
>>   Hi,
>>
>>  can you tell me how much memory your job has and how many workers you
>> are running?
>>
>>  From the trace it seems the internal hash table allocated only 7 MB for
>> the graph data and therefore runs out of memory pretty quickly.
>>
>>  Skewed data could also be an issue but with a minimum of 5 pages and a
>> maximum of 8 it seems to be distributed fairly even to the different
>> partitions.
>>
>>  Cheers,
>>  Robert
>>
>> On Tue, Mar 17, 2015 at 1:25 AM, Mihail Vieru <
>> vi...@informatik.hu-berlin.de> wrote:
>>
>>> And the correct SSSPUnweighted attached.
>>>
>>>
>>> On 17.03.2015 01:23, Mihail Vieru wrote:
>>>
 Hi,

 I'm getting the following RuntimeException for an adaptation of the
 SingleSourceShortestPaths example using the Gelly API (see attachment).
 It's been adapted for unweighted graphs having vertices with Long values.

 As an input graph I'm using the social network graph (~200MB unpacked)
 from here: https://snap.stanford.edu/data/higgs-twitter.html

 For the small SSSPDataUnweighted graph (also attached) it terminates
 and computes the distances correctly.


 03/16/2015 17:18:23IterationHead(WorksetIteration (Vertex-centric
 iteration
 (org.apache.flink.graph.library.SingleSourceShortestPathsUnweighted$VertexDistanceUpdater@dca6fe4
 |
 org.apache.flink.graph.library.SingleSourceShortestPathsUnweighted$MinDistanceMessenger@6577e8ce)))(2/4)
 switched to FAILED
 java.lang.RuntimeException: Memory ran out. Compaction failed.
 numPartitions: 32 minPartition: 5 maxPartition: 8 number of overflow
 segments: 176 bucketSize: 217 Overall memory: 20316160 Partition memory:
 7208960 Message: Index: 8, Size: 7
 at
 org.apache.flink.runtime.operators.hash.CompactingHashTable.insert(CompactingHashTable.java:390)
 at
 org.apache.flink.runtime.operators.hash.CompactingHashTable.buildTable(CompactingHashTable.java:337)
 at
 org.apache.flink.runtime.iterative.task.IterationHeadPactTask.readInitialSolutionSet(IterationHeadPactTask.java:216)
 at
 org.apache.flink.runtime.iterative.task.IterationHeadPactTask.run(IterationHeadPactTask.java:278)
 at
 org.apache.flink.runtime.operators.RegularPactTask.invoke(RegularPactTask.java:362)
 at
 org.apache.flink.runtime.execution.RuntimeEnvironment.run(RuntimeEnvironment.java:205)
 at java.lang.Thread.run(Thread.java:745)


 Best,
 Mihail

>>>
>>>
>>
>>
>
>


Re: RuntimeException Gelly API: Memory ran out. Compaction failed.

2015-03-18 Thread Vasiliki Kalavri
Hi Mihail, Robert,

I've tried reproducing this, but I couldn't.
I'm using the same twitter input graph from SNAP that you link to and also
Scala IDE.
The job finishes without a problem (both the SSSP example from Gelly and
the unweighted version).

The only thing I changed to run your version was creating the graph from
the edge set only, i.e. like this:

Graph graph = Graph.fromDataSet(edges,
new MapFunction() {
public Long map(Long value) {
return Long.MAX_VALUE;
}
}, env);

Since the twitter input is an edge list, how do you generate the vertex
dataset in your case?

Thanks,
-Vasia.

On 18 March 2015 at 16:54, Mihail Vieru 
wrote:

>  Hi,
>
> great! Thanks!
>
> I really need this bug fixed because I'm laying the groundwork for my
> Diplom thesis and I need to be sure that the Gelly API is reliable and can
> handle large datasets as intended.
>
> Cheers,
> Mihail
>
>
> On 18.03.2015 15:40, Robert Waury wrote:
>
>   Hi,
>
>  I managed to reproduce the behavior and as far as I can tell it seems to
> be a problem with the memory allocation.
>
>  I have filed a bug report in JIRA to get the attention of somebody who
> knows the runtime better than I do.
>
> https://issues.apache.org/jira/browse/FLINK-1734
>
>  Cheers,
>  Robert
>
> On Tue, Mar 17, 2015 at 3:52 PM, Mihail Vieru <
> vi...@informatik.hu-berlin.de> wrote:
>
>>  Hi Robert,
>>
>> thank you for your reply.
>>
>> I'm starting the job from the Scala IDE. So only one JobManager and one
>> TaskManager in the same JVM.
>> I've doubled the memory in the eclipse.ini settings but I still get the
>> Exception.
>>
>> -vmargs
>> -Xmx2048m
>> -Xms100m
>> -XX:MaxPermSize=512m
>>
>> Best,
>> Mihail
>>
>>
>> On 17.03.2015 10:11, Robert Waury wrote:
>>
>>   Hi,
>>
>>  can you tell me how much memory your job has and how many workers you
>> are running?
>>
>>  From the trace it seems the internal hash table allocated only 7 MB for
>> the graph data and therefore runs out of memory pretty quickly.
>>
>>  Skewed data could also be an issue but with a minimum of 5 pages and a
>> maximum of 8 it seems to be distributed fairly even to the different
>> partitions.
>>
>>  Cheers,
>>  Robert
>>
>> On Tue, Mar 17, 2015 at 1:25 AM, Mihail Vieru <
>> vi...@informatik.hu-berlin.de> wrote:
>>
>>> And the correct SSSPUnweighted attached.
>>>
>>>
>>> On 17.03.2015 01:23, Mihail Vieru wrote:
>>>
 Hi,

 I'm getting the following RuntimeException for an adaptation of the
 SingleSourceShortestPaths example using the Gelly API (see attachment).
 It's been adapted for unweighted graphs having vertices with Long values.

 As an input graph I'm using the social network graph (~200MB unpacked)
 from here: https://snap.stanford.edu/data/higgs-twitter.html

 For the small SSSPDataUnweighted graph (also attached) it terminates
 and computes the distances correctly.


 03/16/2015 17:18:23IterationHead(WorksetIteration (Vertex-centric
 iteration
 (org.apache.flink.graph.library.SingleSourceShortestPathsUnweighted$VertexDistanceUpdater@dca6fe4
 |
 org.apache.flink.graph.library.SingleSourceShortestPathsUnweighted$MinDistanceMessenger@6577e8ce)))(2/4)
 switched to FAILED
 java.lang.RuntimeException: Memory ran out. Compaction failed.
 numPartitions: 32 minPartition: 5 maxPartition: 8 number of overflow
 segments: 176 bucketSize: 217 Overall memory: 20316160 Partition memory:
 7208960 Message: Index: 8, Size: 7
 at
 org.apache.flink.runtime.operators.hash.CompactingHashTable.insert(CompactingHashTable.java:390)
 at
 org.apache.flink.runtime.operators.hash.CompactingHashTable.buildTable(CompactingHashTable.java:337)
 at
 org.apache.flink.runtime.iterative.task.IterationHeadPactTask.readInitialSolutionSet(IterationHeadPactTask.java:216)
 at
 org.apache.flink.runtime.iterative.task.IterationHeadPactTask.run(IterationHeadPactTask.java:278)
 at
 org.apache.flink.runtime.operators.RegularPactTask.invoke(RegularPactTask.java:362)
 at
 org.apache.flink.runtime.execution.RuntimeEnvironment.run(RuntimeEnvironment.java:205)
 at java.lang.Thread.run(Thread.java:745)


 Best,
 Mihail

>>>
>>>
>>
>>
>
>


Re: RuntimeException Gelly API: Memory ran out. Compaction failed.

2015-03-18 Thread Robert Waury
Hi Vasia,

How much memory does your job use?

I think the problem is as Stephan says a too conservative allocation but
that it will work if you throw enough memory at it.

Or did your setup succeed with an amount of memory comparable to Mihail's
and mine?

My main point is that it shouldn't take 10x more memory than the input size
for such a job.

Cheers,
Robert
On Mar 18, 2015 5:06 PM, "Vasiliki Kalavri" 
wrote:

> Hi Mihail, Robert,
>
> I've tried reproducing this, but I couldn't.
> I'm using the same twitter input graph from SNAP that you link to and also
> Scala IDE.
> The job finishes without a problem (both the SSSP example from Gelly and
> the unweighted version).
>
> The only thing I changed to run your version was creating the graph from
> the edge set only, i.e. like this:
>
> Graph graph = Graph.fromDataSet(edges,
> new MapFunction() {
> public Long map(Long value) {
> return Long.MAX_VALUE;
> }
> }, env);
>
> Since the twitter input is an edge list, how do you generate the vertex
> dataset in your case?
>
> Thanks,
> -Vasia.
>
> On 18 March 2015 at 16:54, Mihail Vieru 
> wrote:
>
>>  Hi,
>>
>> great! Thanks!
>>
>> I really need this bug fixed because I'm laying the groundwork for my
>> Diplom thesis and I need to be sure that the Gelly API is reliable and can
>> handle large datasets as intended.
>>
>> Cheers,
>> Mihail
>>
>>
>> On 18.03.2015 15:40, Robert Waury wrote:
>>
>>   Hi,
>>
>>  I managed to reproduce the behavior and as far as I can tell it seems to
>> be a problem with the memory allocation.
>>
>>  I have filed a bug report in JIRA to get the attention of somebody who
>> knows the runtime better than I do.
>>
>> https://issues.apache.org/jira/browse/FLINK-1734
>>
>>  Cheers,
>>  Robert
>>
>> On Tue, Mar 17, 2015 at 3:52 PM, Mihail Vieru <
>> vi...@informatik.hu-berlin.de> wrote:
>>
>>>  Hi Robert,
>>>
>>> thank you for your reply.
>>>
>>> I'm starting the job from the Scala IDE. So only one JobManager and one
>>> TaskManager in the same JVM.
>>> I've doubled the memory in the eclipse.ini settings but I still get the
>>> Exception.
>>>
>>> -vmargs
>>> -Xmx2048m
>>> -Xms100m
>>> -XX:MaxPermSize=512m
>>>
>>> Best,
>>> Mihail
>>>
>>>
>>> On 17.03.2015 10:11, Robert Waury wrote:
>>>
>>>   Hi,
>>>
>>>  can you tell me how much memory your job has and how many workers you
>>> are running?
>>>
>>>  From the trace it seems the internal hash table allocated only 7 MB for
>>> the graph data and therefore runs out of memory pretty quickly.
>>>
>>>  Skewed data could also be an issue but with a minimum of 5 pages and a
>>> maximum of 8 it seems to be distributed fairly even to the different
>>> partitions.
>>>
>>>  Cheers,
>>>  Robert
>>>
>>> On Tue, Mar 17, 2015 at 1:25 AM, Mihail Vieru <
>>> vi...@informatik.hu-berlin.de> wrote:
>>>
 And the correct SSSPUnweighted attached.


 On 17.03.2015 01:23, Mihail Vieru wrote:

> Hi,
>
> I'm getting the following RuntimeException for an adaptation of the
> SingleSourceShortestPaths example using the Gelly API (see attachment).
> It's been adapted for unweighted graphs having vertices with Long values.
>
> As an input graph I'm using the social network graph (~200MB unpacked)
> from here: https://snap.stanford.edu/data/higgs-twitter.html
>
> For the small SSSPDataUnweighted graph (also attached) it terminates
> and computes the distances correctly.
>
>
> 03/16/2015 17:18:23IterationHead(WorksetIteration (Vertex-centric
> iteration
> (org.apache.flink.graph.library.SingleSourceShortestPathsUnweighted$VertexDistanceUpdater@dca6fe4
> |
> org.apache.flink.graph.library.SingleSourceShortestPathsUnweighted$MinDistanceMessenger@6577e8ce)))(2/4)
> switched to FAILED
> java.lang.RuntimeException: Memory ran out. Compaction failed.
> numPartitions: 32 minPartition: 5 maxPartition: 8 number of overflow
> segments: 176 bucketSize: 217 Overall memory: 20316160 Partition memory:
> 7208960 Message: Index: 8, Size: 7
> at
> org.apache.flink.runtime.operators.hash.CompactingHashTable.insert(CompactingHashTable.java:390)
> at
> org.apache.flink.runtime.operators.hash.CompactingHashTable.buildTable(CompactingHashTable.java:337)
> at
> org.apache.flink.runtime.iterative.task.IterationHeadPactTask.readInitialSolutionSet(IterationHeadPactTask.java:216)
> at
> org.apache.flink.runtime.iterative.task.IterationHeadPactTask.run(IterationHeadPactTask.java:278)
> at
> org.apache.flink.runtime.operators.RegularPactTask.invoke(RegularPactTask.java:362)
> at
> org.apache.flink.runtime.execution.RuntimeEnvironment.run(RuntimeEnvironment.java:205)
> at java.lang.Thread.run(Thread.java:745)
>
>
> Best,
> Mihail
>


>>>
>>>
>>
>>
>


Re: RuntimeException Gelly API: Memory ran out. Compaction failed.

2015-03-18 Thread Vasiliki Kalavri
Hi Robert,

my setup has even less memory than your setup, ~900MB in total.

When using the local environment (running the job through your IDE), the
available of memory is split equally between the JobManager and
TaskManager. Then, the default memory kept for network buffers is
subtracted from the TaskManager's part.
Finally, the TaskManager is assigned 70% (by default) of what is left.
In my case, this was 255MB.

So, I'm guessing that either the options you're passing to eclipse are not
properly read (I haven't tried it myself) or that there's something wrong
in the way you're generating the graph. That's why I asked how you produce
the vertex dataset.

Cheers,
V.



On 18 March 2015 at 18:27, Robert Waury  wrote:

> Hi Vasia,
>
> How much memory does your job use?
>
> I think the problem is as Stephan says a too conservative allocation but
> that it will work if you throw enough memory at it.
>
> Or did your setup succeed with an amount of memory comparable to Mihail's
> and mine?
>
> My main point is that it shouldn't take 10x more memory than the input
> size for such a job.
>
> Cheers,
> Robert
> On Mar 18, 2015 5:06 PM, "Vasiliki Kalavri" 
> wrote:
>
>> Hi Mihail, Robert,
>>
>> I've tried reproducing this, but I couldn't.
>> I'm using the same twitter input graph from SNAP that you link to and
>> also Scala IDE.
>> The job finishes without a problem (both the SSSP example from Gelly and
>> the unweighted version).
>>
>> The only thing I changed to run your version was creating the graph from
>> the edge set only, i.e. like this:
>>
>> Graph graph = Graph.fromDataSet(edges,
>> new MapFunction() {
>> public Long map(Long value) {
>> return Long.MAX_VALUE;
>> }
>> }, env);
>>
>> Since the twitter input is an edge list, how do you generate the vertex
>> dataset in your case?
>>
>> Thanks,
>> -Vasia.
>>
>> On 18 March 2015 at 16:54, Mihail Vieru 
>> wrote:
>>
>>>  Hi,
>>>
>>> great! Thanks!
>>>
>>> I really need this bug fixed because I'm laying the groundwork for my
>>> Diplom thesis and I need to be sure that the Gelly API is reliable and can
>>> handle large datasets as intended.
>>>
>>> Cheers,
>>> Mihail
>>>
>>>
>>> On 18.03.2015 15:40, Robert Waury wrote:
>>>
>>>   Hi,
>>>
>>>  I managed to reproduce the behavior and as far as I can tell it seems
>>> to be a problem with the memory allocation.
>>>
>>>  I have filed a bug report in JIRA to get the attention of somebody who
>>> knows the runtime better than I do.
>>>
>>> https://issues.apache.org/jira/browse/FLINK-1734
>>>
>>>  Cheers,
>>>  Robert
>>>
>>> On Tue, Mar 17, 2015 at 3:52 PM, Mihail Vieru <
>>> vi...@informatik.hu-berlin.de> wrote:
>>>
  Hi Robert,

 thank you for your reply.

 I'm starting the job from the Scala IDE. So only one JobManager and one
 TaskManager in the same JVM.
 I've doubled the memory in the eclipse.ini settings but I still get the
 Exception.

 -vmargs
 -Xmx2048m
 -Xms100m
 -XX:MaxPermSize=512m

 Best,
 Mihail


 On 17.03.2015 10:11, Robert Waury wrote:

   Hi,

  can you tell me how much memory your job has and how many workers you
 are running?

  From the trace it seems the internal hash table allocated only 7 MB
 for the graph data and therefore runs out of memory pretty quickly.

  Skewed data could also be an issue but with a minimum of 5 pages and
 a maximum of 8 it seems to be distributed fairly even to the different
 partitions.

  Cheers,
  Robert

 On Tue, Mar 17, 2015 at 1:25 AM, Mihail Vieru <
 vi...@informatik.hu-berlin.de> wrote:

> And the correct SSSPUnweighted attached.
>
>
> On 17.03.2015 01:23, Mihail Vieru wrote:
>
>> Hi,
>>
>> I'm getting the following RuntimeException for an adaptation of the
>> SingleSourceShortestPaths example using the Gelly API (see attachment).
>> It's been adapted for unweighted graphs having vertices with Long values.
>>
>> As an input graph I'm using the social network graph (~200MB
>> unpacked) from here:
>> https://snap.stanford.edu/data/higgs-twitter.html
>>
>> For the small SSSPDataUnweighted graph (also attached) it terminates
>> and computes the distances correctly.
>>
>>
>> 03/16/2015 17:18:23IterationHead(WorksetIteration (Vertex-centric
>> iteration
>> (org.apache.flink.graph.library.SingleSourceShortestPathsUnweighted$VertexDistanceUpdater@dca6fe4
>> |
>> org.apache.flink.graph.library.SingleSourceShortestPathsUnweighted$MinDistanceMessenger@6577e8ce)))(2/4)
>> switched to FAILED
>> java.lang.RuntimeException: Memory ran out. Compaction failed.
>> numPartitions: 32 minPartition: 5 maxPartition: 8 number of overflow
>> segments: 176 bucketSize: 217 Overall memory: 20316160 Partition memory:
>> 7208960 Message: Index: 8, Size: 7
>> at
>> org.apache.flink.runtime.operators.hash.Compacti

Re: RuntimeException Gelly API: Memory ran out. Compaction failed.

2015-03-18 Thread Mihail Vieru

Hi Vasia,

I have used a simple job (attached) to generate a file which looks like 
this:


0 0
1 1
2 2
...
456629 456629
456630 456630

I need the vertices to be generated from a file for my future work.

Cheers,
Mihail


On 18.03.2015 17:04, Vasiliki Kalavri wrote:

Hi Mihail, Robert,

I've tried reproducing this, but I couldn't.
I'm using the same twitter input graph from SNAP that you link to and 
also Scala IDE.
The job finishes without a problem (both the SSSP example from Gelly 
and the unweighted version).


The only thing I changed to run your version was creating the graph 
from the edge set only, i.e. like this:


Graph graph = Graph.fromDataSet(edges,
new MapFunction() {
public Long map(Long value) {
return Long.MAX_VALUE;
}
}, env);
Since the twitter input is an edge list, how do you generate the 
vertex dataset in your case?


Thanks,
-Vasia.

On 18 March 2015 at 16:54, Mihail Vieru > wrote:


Hi,

great! Thanks!

I really need this bug fixed because I'm laying the groundwork for
my Diplom thesis and I need to be sure that the Gelly API is
reliable and can handle large datasets as intended.

Cheers,
Mihail


On 18.03.2015 15:40, Robert Waury wrote:

Hi,

I managed to reproduce the behavior and as far as I can tell it
seems to be a problem with the memory allocation.

I have filed a bug report in JIRA to get the attention of
somebody who knows the runtime better than I do.

https://issues.apache.org/jira/browse/FLINK-1734

Cheers,
Robert

On Tue, Mar 17, 2015 at 3:52 PM, Mihail Vieru
mailto:vi...@informatik.hu-berlin.de>> wrote:

Hi Robert,

thank you for your reply.

I'm starting the job from the Scala IDE. So only one
JobManager and one TaskManager in the same JVM.
I've doubled the memory in the eclipse.ini settings but I
still get the Exception.

-vmargs
-Xmx2048m
-Xms100m
-XX:MaxPermSize=512m

Best,
Mihail


On 17.03.2015 10:11, Robert Waury wrote:

Hi,

can you tell me how much memory your job has and how many
workers you are running?

From the trace it seems the internal hash table allocated
only 7 MB for the graph data and therefore runs out of
memory pretty quickly.

Skewed data could also be an issue but with a minimum of 5
pages and a maximum of 8 it seems to be distributed fairly
even to the different partitions.

Cheers,
Robert

On Tue, Mar 17, 2015 at 1:25 AM, Mihail Vieru
mailto:vi...@informatik.hu-berlin.de>> wrote:

And the correct SSSPUnweighted attached.


On 17.03.2015 01:23, Mihail Vieru wrote:

Hi,

I'm getting the following RuntimeException for an
adaptation of the SingleSourceShortestPaths example
using the Gelly API (see attachment). It's been
adapted for unweighted graphs having vertices with
Long values.

As an input graph I'm using the social network graph
(~200MB unpacked) from here:
https://snap.stanford.edu/data/higgs-twitter.html

For the small SSSPDataUnweighted graph (also
attached) it terminates and computes the distances
correctly.


03/16/2015 17:18:23 IterationHead(WorksetIteration
(Vertex-centric iteration

(org.apache.flink.graph.library.SingleSourceShortestPathsUnweighted$VertexDistanceUpdater@dca6fe4
|

org.apache.flink.graph.library.SingleSourceShortestPathsUnweighted$MinDistanceMessenger@6577e8ce)))(2/4)
switched to FAILED
java.lang.RuntimeException: Memory ran out.
Compaction failed. numPartitions: 32 minPartition: 5
maxPartition: 8 number of overflow segments: 176
bucketSize: 217 Overall memory: 20316160 Partition
memory: 7208960 Message: Index: 8, Size: 7
at

org.apache.flink.runtime.operators.hash.CompactingHashTable.insert(CompactingHashTable.java:390)
at

org.apache.flink.runtime.operators.hash.CompactingHashTable.buildTable(CompactingHashTable.java:337)
at

org.apache.flink.runtime.iterative.task.IterationHeadPactTask.readInitialSolutionSet(IterationHeadPactTask.java:216)
at

org.apache.flink.runtime.iterative.task.IterationHeadPactTask.run(IterationHeadPactTask.java:278)
at

org.apache.flink.runtime.operators.RegularPactTask.invoke(RegularPactTask.java:362)
at

org.apache.flink.runtime.execution.RuntimeEnvironment.run(RuntimeEnvironme

Re: RuntimeException Gelly API: Memory ran out. Compaction failed.

2015-03-18 Thread Vasiliki Kalavri
Hi Mihail,

I used your code to generate the vertex file, then gave this and the edge
list as input to your SSSP implementation and still couldn't reproduce the
exception. I'm using the same local setup as I describe above.
I'm not aware of any recent changes that might be relevant, but, just in
case, are you using the latest master?

Cheers,
V.

On 18 March 2015 at 19:21, Mihail Vieru 
wrote:

>  Hi Vasia,
>
> I have used a simple job (attached) to generate a file which looks like
> this:
>
> 0 0
> 1 1
> 2 2
> ...
> 456629 456629
> 456630 456630
>
> I need the vertices to be generated from a file for my future work.
>
> Cheers,
> Mihail
>
>
>
> On 18.03.2015 17:04, Vasiliki Kalavri wrote:
>
>  Hi Mihail, Robert,
>
>  I've tried reproducing this, but I couldn't.
> I'm using the same twitter input graph from SNAP that you link to and also
> Scala IDE.
> The job finishes without a problem (both the SSSP example from Gelly and
> the unweighted version).
>
>  The only thing I changed to run your version was creating the graph from
> the edge set only, i.e. like this:
>
>  Graph graph = Graph.fromDataSet(edges,
>  new MapFunction() {
>  public Long map(Long value) {
>  return Long.MAX_VALUE;
>  }
>  }, env);
>
> Since the twitter input is an edge list, how do you generate the vertex
> dataset in your case?
>
>  Thanks,
> -Vasia.
>
> On 18 March 2015 at 16:54, Mihail Vieru 
> wrote:
>
>>  Hi,
>>
>> great! Thanks!
>>
>> I really need this bug fixed because I'm laying the groundwork for my
>> Diplom thesis and I need to be sure that the Gelly API is reliable and can
>> handle large datasets as intended.
>>
>> Cheers,
>> Mihail
>>
>>
>> On 18.03.2015 15:40, Robert Waury wrote:
>>
>>   Hi,
>>
>>  I managed to reproduce the behavior and as far as I can tell it seems to
>> be a problem with the memory allocation.
>>
>>  I have filed a bug report in JIRA to get the attention of somebody who
>> knows the runtime better than I do.
>>
>> https://issues.apache.org/jira/browse/FLINK-1734
>>
>>  Cheers,
>>  Robert
>>
>> On Tue, Mar 17, 2015 at 3:52 PM, Mihail Vieru <
>> vi...@informatik.hu-berlin.de> wrote:
>>
>>>  Hi Robert,
>>>
>>> thank you for your reply.
>>>
>>> I'm starting the job from the Scala IDE. So only one JobManager and one
>>> TaskManager in the same JVM.
>>> I've doubled the memory in the eclipse.ini settings but I still get the
>>> Exception.
>>>
>>> -vmargs
>>> -Xmx2048m
>>> -Xms100m
>>> -XX:MaxPermSize=512m
>>>
>>> Best,
>>> Mihail
>>>
>>>
>>> On 17.03.2015 10:11, Robert Waury wrote:
>>>
>>>   Hi,
>>>
>>>  can you tell me how much memory your job has and how many workers you
>>> are running?
>>>
>>>  From the trace it seems the internal hash table allocated only 7 MB for
>>> the graph data and therefore runs out of memory pretty quickly.
>>>
>>>  Skewed data could also be an issue but with a minimum of 5 pages and a
>>> maximum of 8 it seems to be distributed fairly even to the different
>>> partitions.
>>>
>>>  Cheers,
>>>  Robert
>>>
>>> On Tue, Mar 17, 2015 at 1:25 AM, Mihail Vieru <
>>> vi...@informatik.hu-berlin.de> wrote:
>>>
 And the correct SSSPUnweighted attached.


 On 17.03.2015 01:23, Mihail Vieru wrote:

> Hi,
>
> I'm getting the following RuntimeException for an adaptation of the
> SingleSourceShortestPaths example using the Gelly API (see attachment).
> It's been adapted for unweighted graphs having vertices with Long values.
>
> As an input graph I'm using the social network graph (~200MB unpacked)
> from here: https://snap.stanford.edu/data/higgs-twitter.html
>
> For the small SSSPDataUnweighted graph (also attached) it terminates
> and computes the distances correctly.
>
>
> 03/16/2015 17:18:23IterationHead(WorksetIteration (Vertex-centric
> iteration
> (org.apache.flink.graph.library.SingleSourceShortestPathsUnweighted$VertexDistanceUpdater@dca6fe4
> |
> org.apache.flink.graph.library.SingleSourceShortestPathsUnweighted$MinDistanceMessenger@6577e8ce)))(2/4)
> switched to FAILED
> java.lang.RuntimeException: Memory ran out. Compaction failed.
> numPartitions: 32 minPartition: 5 maxPartition: 8 number of overflow
> segments: 176 bucketSize: 217 Overall memory: 20316160 Partition memory:
> 7208960 Message: Index: 8, Size: 7
> at
> org.apache.flink.runtime.operators.hash.CompactingHashTable.insert(CompactingHashTable.java:390)
> at
> org.apache.flink.runtime.operators.hash.CompactingHashTable.buildTable(CompactingHashTable.java:337)
> at
> org.apache.flink.runtime.iterative.task.IterationHeadPactTask.readInitialSolutionSet(IterationHeadPactTask.java:216)
> at
> org.apache.flink.runtime.iterative.task.IterationHeadPactTask.run(IterationHeadPactTask.java:278)
> at
> org.apache.flink.runtime.operators.RegularPactTask.invoke(RegularPactTask.java:362)
> at
> org.apache.flink.runtime.execution.RuntimeEnvironment.

Re: RuntimeException Gelly API: Memory ran out. Compaction failed.

2015-03-18 Thread Mihail Vieru

Hi Vasia,

yes, I am using the latest master. I just did a pull again and the 
problem persists. Perhaps Robert could confirm as well.


I've set the solution set to unmanaged in SSSPUnweighted as Stephan 
proposed and the job finishes. So I am able to proceed using this 
workaround.


An odd thing occurs now though. The distances aren't computed correctly 
for the SNAP graph and remain the one set in InitVerticesMapper(). For 
the small graph in SSSPDataUnweighted they are OK. I'm currently 
investigating this behavior.


Cheers,
Mihail

On 18.03.2015 20:55, Vasiliki Kalavri wrote:

Hi Mihail,

I used your code to generate the vertex file, then gave this and the 
edge list as input to your SSSP implementation and still couldn't 
reproduce the exception. I'm using the same local setup as I describe 
above.
I'm not aware of any recent changes that might be relevant, but, just 
in case, are you using the latest master?


Cheers,
V.

On 18 March 2015 at 19:21, Mihail Vieru > wrote:


Hi Vasia,

I have used a simple job (attached) to generate a file which looks
like this:

0 0
1 1
2 2
...
456629 456629
456630 456630

I need the vertices to be generated from a file for my future work.

Cheers,
Mihail



On 18.03.2015 17:04, Vasiliki Kalavri wrote:

Hi Mihail, Robert,

I've tried reproducing this, but I couldn't.
I'm using the same twitter input graph from SNAP that you link to
and also Scala IDE.
The job finishes without a problem (both the SSSP example from
Gelly and the unweighted version).

The only thing I changed to run your version was creating the
graph from the edge set only, i.e. like this:

Graph graph = Graph.fromDataSet(edges,
new MapFunction() {
public Long map(Long value) {
return Long.MAX_VALUE;
}
}, env);
Since the twitter input is an edge list, how do you generate the
vertex dataset in your case?

Thanks,
-Vasia.

On 18 March 2015 at 16:54, Mihail Vieru
mailto:vi...@informatik.hu-berlin.de>> wrote:

Hi,

great! Thanks!

I really need this bug fixed because I'm laying the
groundwork for my Diplom thesis and I need to be sure that
the Gelly API is reliable and can handle large datasets as
intended.

Cheers,
Mihail


On 18.03.2015 15:40, Robert Waury wrote:

Hi,

I managed to reproduce the behavior and as far as I can tell
it seems to be a problem with the memory allocation.

I have filed a bug report in JIRA to get the attention of
somebody who knows the runtime better than I do.

https://issues.apache.org/jira/browse/FLINK-1734

Cheers,
Robert

On Tue, Mar 17, 2015 at 3:52 PM, Mihail Vieru
mailto:vi...@informatik.hu-berlin.de>> wrote:

Hi Robert,

thank you for your reply.

I'm starting the job from the Scala IDE. So only one
JobManager and one TaskManager in the same JVM.
I've doubled the memory in the eclipse.ini settings but
I still get the Exception.

-vmargs
-Xmx2048m
-Xms100m
-XX:MaxPermSize=512m

Best,
Mihail


On 17.03.2015 10:11, Robert Waury wrote:

Hi,

can you tell me how much memory your job has and how
many workers you are running?

From the trace it seems the internal hash table
allocated only 7 MB for the graph data and therefore
runs out of memory pretty quickly.

Skewed data could also be an issue but with a minimum
of 5 pages and a maximum of 8 it seems to be
distributed fairly even to the different partitions.

Cheers,
Robert

On Tue, Mar 17, 2015 at 1:25 AM, Mihail Vieru
mailto:vi...@informatik.hu-berlin.de>> wrote:

And the correct SSSPUnweighted attached.


On 17.03.2015 01:23, Mihail Vieru wrote:

Hi,

I'm getting the following RuntimeException for
an adaptation of the SingleSourceShortestPaths
example using the Gelly API (see attachment).
It's been adapted for unweighted graphs having
vertices with Long values.

As an input graph I'm using the social network
graph (~200MB unpacked) from here:
https://snap.stanford.edu/data/higgs-twitter.html

For the small SSSPDataUnweighted graph (also
attached) it terminates and computes the
distances correctly.


03/16/2015 17:18:23
IterationHead(WorksetIteration (Vertex-centric
iterati

Re: RuntimeException Gelly API: Memory ran out. Compaction failed.

2015-03-18 Thread Vasiliki Kalavri
hmm, I'm starting to run out of ideas...
What's your source ID parameter? I ran mine with 0.
About the result, you call both createVertexCentricIteration() and
runVertexCentricIteration() on the initialized graph, right?

On 18 March 2015 at 22:33, Mihail Vieru 
wrote:

>  Hi Vasia,
>
> yes, I am using the latest master. I just did a pull again and the problem
> persists. Perhaps Robert could confirm as well.
>
> I've set the solution set to unmanaged in SSSPUnweighted as Stephan
> proposed and the job finishes. So I am able to proceed using this
> workaround.
>
> An odd thing occurs now though. The distances aren't computed correctly
> for the SNAP graph and remain the one set in InitVerticesMapper(). For the
> small graph in SSSPDataUnweighted they are OK. I'm currently investigating
> this behavior.
>
> Cheers,
> Mihail
>
>
> On 18.03.2015 20:55, Vasiliki Kalavri wrote:
>
>  Hi Mihail,
>
>  I used your code to generate the vertex file, then gave this and the
> edge list as input to your SSSP implementation and still couldn't reproduce
> the exception. I'm using the same local setup as I describe above.
> I'm not aware of any recent changes that might be relevant, but, just in
> case, are you using the latest master?
>
>  Cheers,
> V.
>
> On 18 March 2015 at 19:21, Mihail Vieru 
> wrote:
>
>>  Hi Vasia,
>>
>> I have used a simple job (attached) to generate a file which looks like
>> this:
>>
>> 0 0
>> 1 1
>> 2 2
>> ...
>> 456629 456629
>> 456630 456630
>>
>> I need the vertices to be generated from a file for my future work.
>>
>> Cheers,
>> Mihail
>>
>>
>>
>> On 18.03.2015 17:04, Vasiliki Kalavri wrote:
>>
>>  Hi Mihail, Robert,
>>
>>  I've tried reproducing this, but I couldn't.
>> I'm using the same twitter input graph from SNAP that you link to and
>> also Scala IDE.
>> The job finishes without a problem (both the SSSP example from Gelly and
>> the unweighted version).
>>
>>  The only thing I changed to run your version was creating the graph
>> from the edge set only, i.e. like this:
>>
>>  Graph graph = Graph.fromDataSet(edges,
>>  new MapFunction() {
>>  public Long map(Long value) {
>>  return Long.MAX_VALUE;
>>  }
>>  }, env);
>>
>> Since the twitter input is an edge list, how do you generate the vertex
>> dataset in your case?
>>
>>  Thanks,
>> -Vasia.
>>
>> On 18 March 2015 at 16:54, Mihail Vieru 
>> wrote:
>>
>>>  Hi,
>>>
>>> great! Thanks!
>>>
>>> I really need this bug fixed because I'm laying the groundwork for my
>>> Diplom thesis and I need to be sure that the Gelly API is reliable and can
>>> handle large datasets as intended.
>>>
>>> Cheers,
>>> Mihail
>>>
>>>
>>> On 18.03.2015 15:40, Robert Waury wrote:
>>>
>>>   Hi,
>>>
>>>  I managed to reproduce the behavior and as far as I can tell it seems
>>> to be a problem with the memory allocation.
>>>
>>>  I have filed a bug report in JIRA to get the attention of somebody who
>>> knows the runtime better than I do.
>>>
>>> https://issues.apache.org/jira/browse/FLINK-1734
>>>
>>>  Cheers,
>>>  Robert
>>>
>>> On Tue, Mar 17, 2015 at 3:52 PM, Mihail Vieru <
>>> vi...@informatik.hu-berlin.de> wrote:
>>>
  Hi Robert,

 thank you for your reply.

 I'm starting the job from the Scala IDE. So only one JobManager and one
 TaskManager in the same JVM.
 I've doubled the memory in the eclipse.ini settings but I still get the
 Exception.

 -vmargs
 -Xmx2048m
 -Xms100m
 -XX:MaxPermSize=512m

 Best,
 Mihail


 On 17.03.2015 10:11, Robert Waury wrote:

   Hi,

  can you tell me how much memory your job has and how many workers you
 are running?

  From the trace it seems the internal hash table allocated only 7 MB
 for the graph data and therefore runs out of memory pretty quickly.

  Skewed data could also be an issue but with a minimum of 5 pages and
 a maximum of 8 it seems to be distributed fairly even to the different
 partitions.

  Cheers,
  Robert

 On Tue, Mar 17, 2015 at 1:25 AM, Mihail Vieru <
 vi...@informatik.hu-berlin.de> wrote:

> And the correct SSSPUnweighted attached.
>
>
> On 17.03.2015 01:23, Mihail Vieru wrote:
>
>> Hi,
>>
>> I'm getting the following RuntimeException for an adaptation of the
>> SingleSourceShortestPaths example using the Gelly API (see attachment).
>> It's been adapted for unweighted graphs having vertices with Long values.
>>
>> As an input graph I'm using the social network graph (~200MB
>> unpacked) from here:
>> https://snap.stanford.edu/data/higgs-twitter.html
>>
>> For the small SSSPDataUnweighted graph (also attached) it terminates
>> and computes the distances correctly.
>>
>>
>> 03/16/2015 17:18:23IterationHead(WorksetIteration (Vertex-centric
>> iteration
>> (org.apache.flink.graph.library.SingleSourceShortestPathsUnweighted$VertexDistanceUpdater@dca6fe4
>> |
>

Re: RuntimeException Gelly API: Memory ran out. Compaction failed.

2015-03-18 Thread Mihail Vieru

I'm also using 0 as sourceID. The exact program arguments:

0 /home/vieru/dev/flink-experiments/data/social_network.edgelist 
/home/vieru/dev/flink-experiments/data/social_network.verticeslist 
/home/vieru/dev/flink-experiments/sssp-output-higgstwitter 10


And yes, I call both methods on the initialized Graph *mappedInput*. I 
don't understand why the distances are computed correctly for the small 
graph (also read from files) but not for the larger one.

The messages appear to be wrong in the latter case.

On 18.03.2015 21:55, Vasiliki Kalavri wrote:

hmm, I'm starting to run out of ideas...
What's your source ID parameter? I ran mine with 0.
About the result, you call both createVertexCentricIteration() and 
runVertexCentricIteration() on the initialized graph, right?


On 18 March 2015 at 22:33, Mihail Vieru > wrote:


Hi Vasia,

yes, I am using the latest master. I just did a pull again and the
problem persists. Perhaps Robert could confirm as well.

I've set the solution set to unmanaged in SSSPUnweighted as
Stephan proposed and the job finishes. So I am able to proceed
using this workaround.

An odd thing occurs now though. The distances aren't computed
correctly for the SNAP graph and remain the one set in
InitVerticesMapper(). For the small graph in SSSPDataUnweighted
they are OK. I'm currently investigating this behavior.

Cheers,
Mihail


On 18.03.2015 20:55, Vasiliki Kalavri wrote:

Hi Mihail,

I used your code to generate the vertex file, then gave this and
the edge list as input to your SSSP implementation and still
couldn't reproduce the exception. I'm using the same local setup
as I describe above.
I'm not aware of any recent changes that might be relevant, but,
just in case, are you using the latest master?

Cheers,
V.

On 18 March 2015 at 19:21, Mihail Vieru
mailto:vi...@informatik.hu-berlin.de>> wrote:

Hi Vasia,

I have used a simple job (attached) to generate a file which
looks like this:

0 0
1 1
2 2
...
456629 456629
456630 456630

I need the vertices to be generated from a file for my future
work.

Cheers,
Mihail



On 18.03.2015 17:04, Vasiliki Kalavri wrote:

Hi Mihail, Robert,

I've tried reproducing this, but I couldn't.
I'm using the same twitter input graph from SNAP that you
link to and also Scala IDE.
The job finishes without a problem (both the SSSP example
from Gelly and the unweighted version).

The only thing I changed to run your version was creating
the graph from the edge set only, i.e. like this:

Graph graph = Graph.fromDataSet(edges,
new MapFunction() {
public Long map(Long value) {
return Long.MAX_VALUE;
}
}, env);
Since the twitter input is an edge list, how do you generate
the vertex dataset in your case?

Thanks,
-Vasia.

On 18 March 2015 at 16:54, Mihail Vieru
mailto:vi...@informatik.hu-berlin.de>> wrote:

Hi,

great! Thanks!

I really need this bug fixed because I'm laying the
groundwork for my Diplom thesis and I need to be sure
that the Gelly API is reliable and can handle large
datasets as intended.

Cheers,
Mihail


On 18.03.2015 15:40, Robert Waury wrote:

Hi,

I managed to reproduce the behavior and as far as I can
tell it seems to be a problem with the memory allocation.

I have filed a bug report in JIRA to get the attention
of somebody who knows the runtime better than I do.

https://issues.apache.org/jira/browse/FLINK-1734

Cheers,
Robert

On Tue, Mar 17, 2015 at 3:52 PM, Mihail Vieru
mailto:vi...@informatik.hu-berlin.de>> wrote:

Hi Robert,

thank you for your reply.

I'm starting the job from the Scala IDE. So only
one JobManager and one TaskManager in the same JVM.
I've doubled the memory in the eclipse.ini settings
but I still get the Exception.

-vmargs
-Xmx2048m
-Xms100m
-XX:MaxPermSize=512m

Best,
Mihail


On 17.03.2015 10:11, Robert Waury wrote:

Hi,

can you tell me how much memory your job has and
how many workers you are running?

From the trace it seems the internal hash table
allocated only 7 MB for the graph data and
therefore runs out of memory pretty quickly.

Skewed data could also be an issue bu

Re: RuntimeException Gelly API: Memory ran out. Compaction failed.

2015-03-18 Thread Vasiliki Kalavri
Well, one thing I notice is that your vertices and edges args are flipped.
Might be the source of error :-)

On 18 March 2015 at 23:04, Mihail Vieru 
wrote:

>  I'm also using 0 as sourceID. The exact program arguments:
>
> 0 /home/vieru/dev/flink-experiments/data/social_network.edgelist
> /home/vieru/dev/flink-experiments/data/social_network.verticeslist
> /home/vieru/dev/flink-experiments/sssp-output-higgstwitter 10
>
> And yes, I call both methods on the initialized Graph *mappedInput*. I
> don't understand why the distances are computed correctly for the small
> graph (also read from files) but not for the larger one.
> The messages appear to be wrong in the latter case.
>
>
> On 18.03.2015 21:55, Vasiliki Kalavri wrote:
>
>  hmm, I'm starting to run out of ideas...
> What's your source ID parameter? I ran mine with 0.
> About the result, you call both createVertexCentricIteration() and
> runVertexCentricIteration() on the initialized graph, right?
>
> On 18 March 2015 at 22:33, Mihail Vieru 
> wrote:
>
>>  Hi Vasia,
>>
>> yes, I am using the latest master. I just did a pull again and the
>> problem persists. Perhaps Robert could confirm as well.
>>
>> I've set the solution set to unmanaged in SSSPUnweighted as Stephan
>> proposed and the job finishes. So I am able to proceed using this
>> workaround.
>>
>> An odd thing occurs now though. The distances aren't computed correctly
>> for the SNAP graph and remain the one set in InitVerticesMapper(). For the
>> small graph in SSSPDataUnweighted they are OK. I'm currently investigating
>> this behavior.
>>
>> Cheers,
>> Mihail
>>
>>
>> On 18.03.2015 20:55, Vasiliki Kalavri wrote:
>>
>>  Hi Mihail,
>>
>>  I used your code to generate the vertex file, then gave this and the
>> edge list as input to your SSSP implementation and still couldn't reproduce
>> the exception. I'm using the same local setup as I describe above.
>> I'm not aware of any recent changes that might be relevant, but, just in
>> case, are you using the latest master?
>>
>>  Cheers,
>> V.
>>
>> On 18 March 2015 at 19:21, Mihail Vieru 
>> wrote:
>>
>>>  Hi Vasia,
>>>
>>> I have used a simple job (attached) to generate a file which looks like
>>> this:
>>>
>>> 0 0
>>> 1 1
>>> 2 2
>>> ...
>>> 456629 456629
>>> 456630 456630
>>>
>>> I need the vertices to be generated from a file for my future work.
>>>
>>> Cheers,
>>> Mihail
>>>
>>>
>>>
>>> On 18.03.2015 17:04, Vasiliki Kalavri wrote:
>>>
>>>  Hi Mihail, Robert,
>>>
>>>  I've tried reproducing this, but I couldn't.
>>> I'm using the same twitter input graph from SNAP that you link to and
>>> also Scala IDE.
>>> The job finishes without a problem (both the SSSP example from Gelly and
>>> the unweighted version).
>>>
>>>  The only thing I changed to run your version was creating the graph
>>> from the edge set only, i.e. like this:
>>>
>>>  Graph graph = Graph.fromDataSet(edges,
>>>  new MapFunction() {
>>>  public Long map(Long value) {
>>>  return Long.MAX_VALUE;
>>>  }
>>>  }, env);
>>>
>>> Since the twitter input is an edge list, how do you generate the vertex
>>> dataset in your case?
>>>
>>>  Thanks,
>>> -Vasia.
>>>
>>> On 18 March 2015 at 16:54, Mihail Vieru 
>>> wrote:
>>>
  Hi,

 great! Thanks!

 I really need this bug fixed because I'm laying the groundwork for my
 Diplom thesis and I need to be sure that the Gelly API is reliable and can
 handle large datasets as intended.

 Cheers,
 Mihail


 On 18.03.2015 15:40, Robert Waury wrote:

   Hi,

  I managed to reproduce the behavior and as far as I can tell it seems
 to be a problem with the memory allocation.

  I have filed a bug report in JIRA to get the attention of somebody who
 knows the runtime better than I do.

 https://issues.apache.org/jira/browse/FLINK-1734

  Cheers,
  Robert

 On Tue, Mar 17, 2015 at 3:52 PM, Mihail Vieru <
 vi...@informatik.hu-berlin.de> wrote:

>  Hi Robert,
>
> thank you for your reply.
>
> I'm starting the job from the Scala IDE. So only one JobManager and
> one TaskManager in the same JVM.
> I've doubled the memory in the eclipse.ini settings but I still get
> the Exception.
>
> -vmargs
> -Xmx2048m
> -Xms100m
> -XX:MaxPermSize=512m
>
> Best,
> Mihail
>
>
> On 17.03.2015 10:11, Robert Waury wrote:
>
>   Hi,
>
>  can you tell me how much memory your job has and how many workers you
> are running?
>
>  From the trace it seems the internal hash table allocated only 7 MB
> for the graph data and therefore runs out of memory pretty quickly.
>
>  Skewed data could also be an issue but with a minimum of 5 pages and
> a maximum of 8 it seems to be distributed fairly even to the different
> partitions.
>
>  Cheers,
>  Robert
>
> On Tue, Mar 17, 2015 at 1:25 AM, Mihail Vieru <
> vi...@informatik.hu-be

Re: RuntimeException Gelly API: Memory ran out. Compaction failed.

2015-03-18 Thread Vasiliki Kalavri
haha, yes, actually I just confirmed!
If I flip my args, I get the error you mention in the first e-mail. you're
trying to generate a graph giving the edge list as a vertex list and this
is a way too big dataset for your memory settings (cmp. ~15m edges vs. the
actual 400k).

I hope that clear everything out :-)

Cheers,
V.

On 18 March 2015 at 23:44, Vasiliki Kalavri 
wrote:

> Well, one thing I notice is that your vertices and edges args are flipped.
> Might be the source of error :-)
>
> On 18 March 2015 at 23:04, Mihail Vieru 
> wrote:
>
>>  I'm also using 0 as sourceID. The exact program arguments:
>>
>> 0 /home/vieru/dev/flink-experiments/data/social_network.edgelist
>> /home/vieru/dev/flink-experiments/data/social_network.verticeslist
>> /home/vieru/dev/flink-experiments/sssp-output-higgstwitter 10
>>
>> And yes, I call both methods on the initialized Graph *mappedInput*. I
>> don't understand why the distances are computed correctly for the small
>> graph (also read from files) but not for the larger one.
>> The messages appear to be wrong in the latter case.
>>
>>
>> On 18.03.2015 21:55, Vasiliki Kalavri wrote:
>>
>>  hmm, I'm starting to run out of ideas...
>> What's your source ID parameter? I ran mine with 0.
>> About the result, you call both createVertexCentricIteration() and
>> runVertexCentricIteration() on the initialized graph, right?
>>
>> On 18 March 2015 at 22:33, Mihail Vieru 
>> wrote:
>>
>>>  Hi Vasia,
>>>
>>> yes, I am using the latest master. I just did a pull again and the
>>> problem persists. Perhaps Robert could confirm as well.
>>>
>>> I've set the solution set to unmanaged in SSSPUnweighted as Stephan
>>> proposed and the job finishes. So I am able to proceed using this
>>> workaround.
>>>
>>> An odd thing occurs now though. The distances aren't computed correctly
>>> for the SNAP graph and remain the one set in InitVerticesMapper(). For the
>>> small graph in SSSPDataUnweighted they are OK. I'm currently investigating
>>> this behavior.
>>>
>>> Cheers,
>>> Mihail
>>>
>>>
>>> On 18.03.2015 20:55, Vasiliki Kalavri wrote:
>>>
>>>  Hi Mihail,
>>>
>>>  I used your code to generate the vertex file, then gave this and the
>>> edge list as input to your SSSP implementation and still couldn't reproduce
>>> the exception. I'm using the same local setup as I describe above.
>>> I'm not aware of any recent changes that might be relevant, but, just in
>>> case, are you using the latest master?
>>>
>>>  Cheers,
>>> V.
>>>
>>> On 18 March 2015 at 19:21, Mihail Vieru 
>>> wrote:
>>>
  Hi Vasia,

 I have used a simple job (attached) to generate a file which looks like
 this:

 0 0
 1 1
 2 2
 ...
 456629 456629
 456630 456630

 I need the vertices to be generated from a file for my future work.

 Cheers,
 Mihail



 On 18.03.2015 17:04, Vasiliki Kalavri wrote:

  Hi Mihail, Robert,

  I've tried reproducing this, but I couldn't.
 I'm using the same twitter input graph from SNAP that you link to and
 also Scala IDE.
 The job finishes without a problem (both the SSSP example from Gelly
 and the unweighted version).

  The only thing I changed to run your version was creating the graph
 from the edge set only, i.e. like this:

  Graph graph = Graph.fromDataSet(edges,
  new MapFunction() {
  public Long map(Long value) {
  return Long.MAX_VALUE;
  }
  }, env);

 Since the twitter input is an edge list, how do you generate the vertex
 dataset in your case?

  Thanks,
 -Vasia.

 On 18 March 2015 at 16:54, Mihail Vieru 
 wrote:

>  Hi,
>
> great! Thanks!
>
> I really need this bug fixed because I'm laying the groundwork for my
> Diplom thesis and I need to be sure that the Gelly API is reliable and can
> handle large datasets as intended.
>
> Cheers,
> Mihail
>
>
> On 18.03.2015 15:40, Robert Waury wrote:
>
>   Hi,
>
>  I managed to reproduce the behavior and as far as I can tell it seems
> to be a problem with the memory allocation.
>
>  I have filed a bug report in JIRA to get the attention of somebody
> who knows the runtime better than I do.
>
> https://issues.apache.org/jira/browse/FLINK-1734
>
>  Cheers,
>  Robert
>
> On Tue, Mar 17, 2015 at 3:52 PM, Mihail Vieru <
> vi...@informatik.hu-berlin.de> wrote:
>
>>  Hi Robert,
>>
>> thank you for your reply.
>>
>> I'm starting the job from the Scala IDE. So only one JobManager and
>> one TaskManager in the same JVM.
>> I've doubled the memory in the eclipse.ini settings but I still get
>> the Exception.
>>
>> -vmargs
>> -Xmx2048m
>> -Xms100m
>> -XX:MaxPermSize=512m
>>
>> Best,
>> Mihail
>>
>>
>> On 17.03.2015 10:11, Robert Waury wrote:
>>
>>   Hi,
>>
>

Re: RuntimeException Gelly API: Memory ran out. Compaction failed.

2015-03-18 Thread Mihail Vieru

n way... that was it!? :))) Big thanks! :)
The result is also correct now.

Cheers,
M.

On 18.03.2015 22:49, Vasiliki Kalavri wrote:

haha, yes, actually I just confirmed!
If I flip my args, I get the error you mention in the first e-mail. 
you're trying to generate a graph giving the edge list as a vertex 
list and this is a way too big dataset for your memory settings (cmp. 
~15m edges vs. the actual 400k).


I hope that clear everything out :-)

Cheers,
V.

On 18 March 2015 at 23:44, Vasiliki Kalavri > wrote:


Well, one thing I notice is that your vertices and edges args are
flipped. Might be the source of error :-)

On 18 March 2015 at 23:04, Mihail Vieru
mailto:vi...@informatik.hu-berlin.de>> wrote:

I'm also using 0 as sourceID. The exact program arguments:

0
/home/vieru/dev/flink-experiments/data/social_network.edgelist
/home/vieru/dev/flink-experiments/data/social_network.verticeslist
/home/vieru/dev/flink-experiments/sssp-output-higgstwitter 10

And yes, I call both methods on the initialized Graph
*mappedInput*. I don't understand why the distances are
computed correctly for the small graph (also read from files)
but not for the larger one.
The messages appear to be wrong in the latter case.


On 18.03.2015 21:55, Vasiliki Kalavri wrote:

hmm, I'm starting to run out of ideas...
What's your source ID parameter? I ran mine with 0.
About the result, you call both
createVertexCentricIteration() and
runVertexCentricIteration() on the initialized graph, right?

On 18 March 2015 at 22:33, Mihail Vieru
mailto:vi...@informatik.hu-berlin.de>> wrote:

Hi Vasia,

yes, I am using the latest master. I just did a pull
again and the problem persists. Perhaps Robert could
confirm as well.

I've set the solution set to unmanaged in SSSPUnweighted
as Stephan proposed and the job finishes. So I am able to
proceed using this workaround.

An odd thing occurs now though. The distances aren't
computed correctly for the SNAP graph and remain the one
set in InitVerticesMapper(). For the small graph in
SSSPDataUnweighted they are OK. I'm currently
investigating this behavior.

Cheers,
Mihail


On 18.03.2015 20:55, Vasiliki Kalavri wrote:

Hi Mihail,

I used your code to generate the vertex file, then gave
this and the edge list as input to your SSSP
implementation and still couldn't reproduce the
exception. I'm using the same local setup as I describe
above.
I'm not aware of any recent changes that might be
relevant, but, just in case, are you using the latest
master?

Cheers,
V.

On 18 March 2015 at 19:21, Mihail Vieru
mailto:vi...@informatik.hu-berlin.de>> wrote:

Hi Vasia,

I have used a simple job (attached) to generate a
file which looks like this:

0 0
1 1
2 2
...
456629 456629
456630 456630

I need the vertices to be generated from a file for
my future work.

Cheers,
Mihail



On 18.03.2015 17:04, Vasiliki Kalavri wrote:

Hi Mihail, Robert,

I've tried reproducing this, but I couldn't.
I'm using the same twitter input graph from SNAP
that you link to and also Scala IDE.
The job finishes without a problem (both the SSSP
example from Gelly and the unweighted version).

The only thing I changed to run your version was
creating the graph from the edge set only, i.e.
like this:

Graph graph =
Graph.fromDataSet(edges,
new MapFunction() {
public Long map(Long value) {
return Long.MAX_VALUE;
}
}, env);
Since the twitter input is an edge list, how do you
generate the vertex dataset in your case?

Thanks,
-Vasia.

On 18 March 2015 at 16:54, Mihail Vieru
mailto:vi...@informatik.hu-berlin.de>> wrote:

Hi,

great! Thanks!

I really need this bug fixed because I'm laying
the groundwork for my Diplom thesis and I need
to be sure that the Gelly API is reliable and
can handle large datasets as intended.