LDBC Graph Data into Flink

2015-10-06 Thread Martin Junghanns
Hi all, For our benchmarks with Flink, we are using a data generator provided by the LDBC project (Linked Data Benchmark Council) [1][2]. The generator uses MapReduce to create directed, labeled, attributed graphs that mimic properties of real online social networks (e.g, degree distribution,

Re: LDBC Graph Data into Flink

2015-10-06 Thread Martin Junghanns
should also be Gelly-compatible :) > > Cheers, > Vasia. > > On 6 October 2015 at 10:03, Martin Junghanns > wrote: > >> Hi all, >> >> For our benchmarks with Flink, we are using a data generator provided by >> the LDBC project (Linked Data Benchmar

Re: Fwd: Problem applying a groupReduce function to a grouped data set

2015-11-01 Thread Martin Junghanns
Hi, just an idea: In the source code documentation, it states that projectFirst and projectSecond lose type information, which could explain why your group reduce expects . I found an example [1] that calls .types() to define the returned types, but this method is deprecated. What I would tr

Re: Long ids from String

2015-11-03 Thread Martin Junghanns
Hi Flavio, If you just want to assign a unique Long identifier to each element in your dataset, you can use the DataSetUtils.zipWithUniqueId() method [1]. Best, Martin [1] https://github.com/apache/flink/blob/master/flink-java/src/main/java/org/apache/flink/api/java/utils/DataSetUtils.java#L131

Re: Long ids from String

2015-11-03 Thread Martin Junghanns
e 1st tuple with the 2nd (so I should know > that c=>2). > Which strategy do you think it could be the best option to achieve that? > At the moment I was thinking to persist the ids and use a temporary table > with an autoincrement long id but maybe there's a simpler solution..

Re: LDBC Graph Data into Flink

2015-11-24 Thread Martin Junghanns
Hi, I wrote a short blog post about the ldbc-flink tool including a short overview of Flink and a Gelly example. http://ldbcouncil.org/blog/ldbc-and-apache-flink Best, Martin On 06.10.2015 11:00, Martin Junghanns wrote: > Hi Vasia, > > No problem. Sure, Gelly is just a map()

Re: Memory ran out PageRank

2016-03-14 Thread Martin Junghanns
Hi I think this is the same issue we had before on the list [1]. Stephan recommended the following workaround: A possible workaround is to use the option "setSolutionSetUnmanaged(true)" on the iteration. That will eliminate the fragmentation issue, at least. Unfortunately, you cannot set th

Re: Memory ran out PageRank

2016-03-14 Thread Martin Junghanns
java.lang.Thread.run(Thread.java:745) Best, Ovidiu On 14 Mar 2016, at 17:36, Martin Junghanns wrote: Hi I think this is the same issue we had before on the list [1]. Stephan recommended the following workaround: A possible workaround is to use the option "setSolutionSetUnmanaged(true

Context-specific step function in Iteration

2016-05-29 Thread Martin Junghanns
Hi everyone, In a step-function (bulk) I'd like to join the working set W with another data set T. The join field of T depends on the current super step. Unfortunately, W has no access to the iteration runtime context. I tried to extract the current superstep at the beginning of the step functio

Re: Context-specific step function in Iteration

2016-05-30 Thread Martin Junghanns
also work with cross instead of broadcasting, I did not test which way works faster, yet. Apologies if anyone thought about this when it was my error in the end :) Cheers, Martin On 29.05.2016 14:05, Martin Junghanns wrote: Hi everyone, In a step-function (bulk) I'd like to join the wor

Re: Support/connector for Neo4j?

2017-03-24 Thread Martin Junghanns
Please let me know, if you need help with the connector or if you want to extend it. Cheers, Martin On 24.03.2017 16:07, alex.decastro wrote: Thanks Tim! I missed that one on Jira. :-) -- View this message in context: http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/Su

Re: VertexUpdateFunction

2017-05-29 Thread Martin Junghanns
Hi Ali :) You could compute the degrees beforehand (e.g. using the Graph.[in|out|get]degrees()) methods and use the resulting dataset as a new vertex dataset. You can now run your vertex-centric computation and access the degrees as vertex value. Cheers, Martin On 29.05.2017 09:28, rost..

Re: VertexUpdateFunction

2017-05-31 Thread Martin Junghanns
a map (vertex_id -> degree) to the constructor. Regards, Ali Zitat von Martin Junghanns : Hi Ali :) You could compute the degrees beforehand (e.g. using the Graph.[in|out|get]degrees()) methods and use the resulting dataset as a new vertex dataset. You can now run your vertex-centric com

Containment Join Support

2015-07-16 Thread Martin Junghanns
Hi everyone, at first, thanks for building this great framework! We are using Flink and especially Gelly for building a graph analytics stack (gradoop.com). I was wondering if there is a [planned] support for a containment join operator. Consider the following example: DataSet> left := {[0, 1],

Re: Containment Join Support

2015-07-17 Thread Martin Junghanns
2015-07-16 9:39 GMT+02:00 Martin Junghanns mailto:martin.jungha...@gmx.net>>: Hi everyone, at first, thanks for building this great framework! We are using Flink and especially Gelly for building a graph analytics stack (gradoop.com &l

Re: Inheritance and FlatMap with custom POJO

2015-09-16 Thread Martin Junghanns
Hi Giacomo, I ran into the same issue. Seems to be coupled to the serialization mechanism of UDFs. I solved it by letting the base class implement the UDF interface (e.g. FlatMapFunction) and in addition make it generic (which should be necessary in your example). public [abstract] class Cul