Re: Graph Analytics on HBase With HGraphDB and Apache Flink Gelly

2017-07-28 Thread Vasiliki Kalavri
Thank you for sharing! On 28 July 2017 at 05:01, Robert Yokota wrote: > Also Google Cloud Bigtable has such a page at https://cloud.google.com/ > bigtable/docs/integrations > > On Thu, Jul 27, 2017 at 6:57 PM, Robert Yokota wrote: > >> >> One thing I

Re: Gelly - bipartite graph runs vertex-centric

2017-06-26 Thread Vasiliki Kalavri
Hi Marc, the BipartiteGraph type doesn't support vertex-centric iterations yet. You can either represent your bipartite graph using the Graph type and e.g. having an extra attribute in the vertex value to distinguish between top and bottom vertices or define your own custom delta iteration on top

Re: Graph iteration with triplets or access to edges

2017-04-28 Thread Vasiliki Kalavri
Hi Marc, you can access the edge values inside the ScatterFunction using the getEdges() method. For an example look at SingleSourceShortestPaths [1] which sums up edge values to compute distances. I hope that helps! -Vasia. [1]:

Re: Questions about the V-C Iteration in Gelly

2017-02-13 Thread Vasiliki Kalavri
ating vertices values in each super step or sending >>>> heartbeat messages to vertices themselves (which will bring a lot of extra >>>> work to the MessageCombiner). >>>> >>>> c) I know it's not elegant or even an awful idea to store the edge info >>

Re: Questions about the V-C Iteration in Gelly

2017-02-10 Thread Vasiliki Kalavri
). Considering the > complexity, the example is not > provided.) > > Really appreciate for all your help. > > Best, > Xingcan > > On Thu, Feb 9, 2017 at 5:36 PM, Vasiliki Kalavri < > vasilikikala...@gmail.com> wrote: > >> Hi Xingcan, >> >> On

Re: Questions about the V-C Iteration in Gelly

2017-02-09 Thread Vasiliki Kalavri
Hi Xingcan, On 7 February 2017 at 10:10, Xingcan Cui wrote: > Hi all, > > I got some question about the vertex-centric iteration in Gelly. > > a) It seems the postSuperstep method is called before the superstep > barrier (I got different aggregate values of the same

Re: Apache Flink 1.1.4 - Gelly - LocalClusteringCoefficient - Returning values above 1?

2017-01-23 Thread Vasiliki Kalavri
.e.coimbra > > On 20 January 2017 at 19:31, Greg Hogan <c...@greghogan.com> wrote: > >> Hi Miguel, >> >> The '--output print' option describes the values and also displays the >> local clustering coefficient value. >> >> You're running the undirect

Re: Apache Flink 1.1.4 - Gelly - LocalClusteringCoefficient - Returning values above 1?

2017-01-20 Thread Vasiliki Kalavri
Hi Miguel, the LocalClusteringCoefficient algorithm returns a DataSet of type Result, which basically wraps a vertex id, its degree, and the number of triangles containing this vertex. The number 11 you see is indeed the degree of vertex 5113. The Result type contains the method

Re: Apache Flink 1.1.4 - Java 8 - CommunityDetection.java:158 - java.lang.NullPointerException

2017-01-18 Thread Vasiliki Kalavri
; -- Forwarded message -- > From: Vasiliki Kalavri <vasilikikala...@gmail.com> > To: user@flink.apache.org > Cc: > Date: Sun, 15 Jan 2017 18:01:41 +0100 > Subject: Re: Apache Flink 1.1.4 - Java 8 - CommunityDetection.java:158 - > java.lang.NullPointer

Re: Apache Flink 1.1.4 - Java 8 - CommunityDetection.java:158 - java.lang.NullPointerException

2017-01-16 Thread Vasiliki Kalavri
Hi Miguel, thank you for opening the issue! Changes/improvements to the documentation are also typically handled with JIRAs and pull requests [1]. Would you like to give it a try and improve the community detection docs? Cheers, -Vasia. [1]:

Re: Executing graph algorithms on Gelly that are larger then memmory

2016-11-30 Thread Vasiliki Kalavri
Hi, can you give us some more details about the algorithm you are testing and your configuration? Flink DataSet operators like join, coGroup, reduce, etc. spill to disk if there is not enough memory. If you are using a delta iteration operator though, the state that is kept across iterations

Re: Type of TypeVariable 'K' in 'class <> could not be determined

2016-11-18 Thread Vasiliki Kalavri
on't know if it solve this problem but in general if the input > type is known it should be passed for input type inference. > > Am 18/11/16 um 11:28 schrieb Vasiliki Kalavri: > > Hi Timo, > > thanks for looking into this! Are you referring to the 4th argument in [1]? > &

Re: Type of TypeVariable 'K' in 'class <> could not be determined

2016-11-18 Thread Vasiliki Kalavri
Hi Timo, thanks for looking into this! Are you referring to the 4th argument in [1]? Thanks, -Vasia. [1]: https://github.com/apache/flink/blob/master/ flink-libraries/flink-gelly/src/main/java/org/apache/ flink/graph/Graph.java#L506 On 18 November 2016 at 10:25, Timo Walther

Re: Type of TypeVariable 'K' in 'class <> could not be determined

2016-11-17 Thread Vasiliki Kalavri
Hi Wouter, with InitVerticesMapper() are you trying to map the vertex value to a Tuple2 or to a Double? Your mapper is turning the vertex values into a Tuple2<> but your scatter-gather UDFs are defining Double vertex values. -Vasia. On 17 November 2016 at 14:03, otherwise777

Re: 33 segments problem with configuration set

2016-11-16 Thread Vasiliki Kalavri
Dear Wouter, first of all, as I noted in another thread already, betweenness centrality is an extremely demanding algorithm and a distributed data engine such as Flink is probably not the best system to implement it into. On top of that, the message-passing model for graph computations would

Re: Too few memory segments provided. Hash Table needs at least 33 memory segments.

2016-11-15 Thread Vasiliki Kalavri
Hi Miguel, I'm sorry for the late reply; this e-mail got stuck in my spam folder. I'm glad that you've found a solution :) I've never used flink with docker, so I'm probably not the best person to advise you on this. However, if I understand correctly, you're changing the configuration before

Re: Retrieving a single element from a DataSet

2016-11-05 Thread Vasiliki Kalavri
Hi all, @Wouter: I'm not sure I completely understand what you want to do, but would broadcast variables [1] help? @all: All-pairs-shortest-paths and betweenness centrality are very challenging algorithms to implement efficiently in a distributed way. APSP requires each vertex to store distances

Re: Flink error: Too few memory segments provided

2016-10-21 Thread Vasiliki Kalavri
Hi, On 21 October 2016 at 11:17, otherwise777 wrote: > I tried increasing the taskmanager.network.numberOfBuffers to 4k and > later to > 8k, i'm not sure if my configuration file is even read, it's stored inside > my IDE as follows: http://prntscr.com/cx0vrx

Re: [ANNOUNCE] Flink 1.1.0 Released

2016-08-08 Thread Vasiliki Kalavri
yoo-hoo finally announced  Thanks for managing the release Ufuk! On 8 August 2016 at 18:36, Ufuk Celebi wrote: > The Flink PMC is pleased to announce the availability of Flink 1.1.0. > > On behalf of the PMC, I would like to thank everybody who contributed > to the release. >

Re: sampling function

2016-07-11 Thread Vasiliki Kalavri
Hi Do, Paris and Martha worked on sampling techniques for data streams on Flink last year. If you want to implement your own samplers, you might find Martha's master thesis helpful [1]. -Vasia. [1]: http://kth.diva-portal.org/smash/get/diva2:910695/FULLTEXT01.pdf On 11 July 2016 at 11:31,

Re: Flink stops deploying jobs on normal iteration

2016-07-05 Thread Vasiliki Kalavri
Hi Truong, I'm afraid what you're experiencing is to be expected. Currently, for loops do not perform well in Flink since there is no support for caching intermediate results yet. This has been a quite often requested feature lately, so maybe it will be added soon :) Until then, I suggest you try

Re: Send to all in gelly scatter

2016-06-15 Thread Vasiliki Kalavri
I forgot the reference [1] :S Here it is: [1] https://ci.apache.org/projects/flink/flink-docs-master/apis/batch/libs/gelly.html#iteration-abstractions-comparison On 15 June 2016 at 20:59, Vasiliki Kalavri <vasilikikala...@gmail.com> wrote: > Hi Alieh, > > you can send a message

Re: Send to all in gelly scatter

2016-06-15 Thread Vasiliki Kalavri
Hi Alieh, you can send a message from any vertex to any other vertex if you know the vertex ID. In [1] you will find a table that compares the update logic and communication scope for all gelly iteration models. Bear in mind though, that sending a message from all vertices to all other vertices

Re: Gelly Scatter/Gather - Vertex update

2016-06-15 Thread Vasiliki Kalavri
Hi Alieh, the scatter-gather model is built on top of Flink delta iterations exactly for the reason to allow de-activating vertices that do not need to participate in the computation of a certain superstep. If you want all vertices to participate in all iterations of scatter-gather, you can send

Re: Gelly scatter/gather

2016-06-13 Thread Vasiliki Kalavri
Hi Alieh, the VertexUpdateFunction and the MessagingFunction both have a method "getSuperstepNumber()" which will give you the current iteration number. -Vasia. On 13 June 2016 at 18:06, Alieh Saeedi wrote: > Hi > Is it possible to access iteration number in gelly

Re: "Memory ran out" error when running connected components

2016-05-14 Thread Vasiliki Kalavri
Hey Rob, On 13 May 2016 at 15:45, Arkay wrote: > Thanks for the link, I had experimented with those options, apart from > taskmanager.memory.off-heap: true. Turns out that allows it to run through > happily! I don't know if that is a peculiarity of a windows JVM, as I >

Re: "Memory ran out" error when running connected components

2016-05-13 Thread Vasiliki Kalavri
On 13 May 2016 at 14:28, Arkay wrote: > Hi Vasia, > > It seems to work OK up to about 50MB of input, and dies after that point. > If i disable just this connected components step the rest of my program is > happy with the full 1.5GB test dataset. It seems to be specifically

Re: "Memory ran out" error when running connected components

2016-05-13 Thread Vasiliki Kalavri
Thanks for checking Rob! I don't see any reason for the job to fail with this configuration and input size. I have no experience running Flink on windows though, so I might be missing something. Do you get a similar error with smaller inputs? -Vasia. On 13 May 2016 at 13:27, Arkay

Re: "Memory ran out" error when running connected components

2016-05-13 Thread Vasiliki Kalavri
Hi Rob, On 13 May 2016 at 11:22, Arkay wrote: > Hi to all, > > I’m aware there are a few threads on this, but I haven’t been able to solve > an issue I am seeing and hoped someone can help. I’m trying to run the > following: > > val connectedNetwork = new

Re: Scatter-Gather Iteration aggregators

2016-05-13 Thread Vasiliki Kalavri
.getValue({ > > aggregator.reset(); > aggregator.aggregate(sum); > > } > setNewVertexValue(sum); > } > } > > > > > Am 13.05.2016 um 09:25 schrieb Vasiliki Kalavri <vasilikikala...@gmail.com > >: >

Re: Scatter-Gather Iteration aggregators

2016-05-13 Thread Vasiliki Kalavri
gt; > Am 13.05.2016 um 08:04 schrieb Vasiliki Kalavri <vasilikikala...@gmail.com > >: > > Hi Lydia, > > registered aggregators through the ScatterGatherConfiguration are > accessible both in the VertexUpdateFunction and in the MessageFunction. > > Cheers, > -V

Re: normalize vertex values

2016-05-12 Thread Vasiliki Kalavri
Hi Lydia, there is no dedicated Gelly API method that performs normalization. If you know the max value, then a mapVertices() would suffice. Otherwise, you can get the Dataset of vertices with getVertices() and apply any kind of operation supported by the Dataset API on it. Best, -Vasia. On May

Re: Bug while using Table API

2016-05-12 Thread Vasiliki Kalavri
Good to know :) On 12 May 2016 at 11:16, Simone Robutti <simone.robu...@radicalbit.io> wrote: > Ok, I tested it and it works on the same example. :) > > 2016-05-11 12:25 GMT+02:00 Vasiliki Kalavri <vasilikikala...@gmail.com>: > >> Hi Simone, >> >>

Re: Bug while using Table API

2016-05-11 Thread Vasiliki Kalavri
/flink/commit/7ed07933d2dd3cf41948287dc8fd79dbef902311 On 4 May 2016 at 17:33, Vasiliki Kalavri <vasilikikala...@gmail.com> wrote: > Thanks Simone! I've managed to reproduce the error. I'll try to figure out > what's wrong and I'll keep you updated. > > -Vasia. > On May 4,

Re: Bug while using Table API

2016-05-04 Thread Vasiliki Kalavri
Thanks Simone! I've managed to reproduce the error. I'll try to figure out what's wrong and I'll keep you updated. -Vasia. On May 4, 2016 3:25 PM, "Simone Robutti" wrote: > Here is the code: > > package org.example > > import org.apache.flink.api.scala._ > import

Re: Bug while using Table API

2016-05-04 Thread Vasiliki Kalavri
Hi Simone, I tried reproducing your problem with no luck. I ran the WordCountTable example using sbt quickstart with Flink 1.1-SNAPSHOT and Scala 2.10 and it worked fine. Can you maybe post the code you tried? Thanks, -Vasia. On 4 May 2016 at 11:20, Simone Robutti

Re: aggregation problem

2016-04-28 Thread Vasiliki Kalavri
Hi Riccardo, can you please be a bit more specific? What do you mean by "it didn't work"? Did it crash? Did it give you a wrong value? Something else? -Vasia. On 28 April 2016 at 16:52, Riccardo Diomedi wrote: > Hi everybody > > In a DeltaIteration I have a

Re: Gelly CommunityDetection in scala example

2016-04-27 Thread Vasiliki Kalavri
Hi Trevor, note that the community detection algorithm returns a new graph where the vertex values correspond to the computed communities. Also, note that the current implementation expects a graph with java.lang.Long vertex values and java.lang.Double edge values. The following should work:

Re: Job hangs

2016-04-27 Thread Vasiliki Kalavri
Hi Timur, I've previously seen large batch jobs hang because of join deadlocks. We should have fixed those problems, but we might have missed some corner case. Did you check whether there was any cpu activity when the job hangs? Can you try running htop on the taskmanager machines and see if

Re: Intermediate solution set of delta iteration

2016-03-23 Thread Vasiliki Kalavri
Hi Mengqi, if what you are trying to do is output the solution set of every iteration, before the iteration has finished, then that is not possible. i.e. you can not output the solution set to a sink or another operator during the iteration. However, you can add elements to the solution set and

Re: Memory ran out PageRank

2016-03-14 Thread Vasiliki Kalavri
Hi Ovidiu, this option won't fix the problem if your system doesn't have enough memory :) It only defines whether the solution set is kept in managed memory or not. For more iteration configuration options, take a look at the Gelly documentation [1]. -Vasia. [1]:

Re: time spent for iteration

2016-03-09 Thread Vasiliki Kalavri
I think it would be useful to allow for easier retrieval of this information. Wouldn't it make sense to expose this to the web UI for example? We actually had a discussion about this some time ago [1]. -Vasia. [1]: https://issues.apache.org/jira/browse/FLINK-1759 On 9 March 2016 at 14:37, Gábor

Re: Graph with stream of updates

2016-02-26 Thread Vasiliki Kalavri
Hi Ankur, you can have custom state in your Flink operators, including a graph. There is no graph state abstraction provided at the moment, but it shouldn't be too hard for you to implement your own. If your use-case only requires processing edge additions only, then you might want to take a look

Re: 2015: A Year in Review for Apache Flink

2015-12-31 Thread Vasiliki Kalavri
Happy new year everyone! Looking forward to all the great things the Apache Flink community will accomplish in 2016 :)) Greetings from snowy Greece! -Vasia. On 31 December 2015 at 04:22, Henry Saputra wrote: > Dear All, > > It is almost end of 2015 and it has been busy

Re: store and retrieve Graph object

2015-11-25 Thread Vasiliki Kalavri
Hi Stefane, let me know if I understand the problem correctly. The vertex values are POJOs that you're somehow inferring from the edge list and this value creation is what takes a lot of time? Since a graph is just a set of 2 datasets (vertices and edges), you could store the values to disk and

Re: store and retrieve Graph object

2015-11-25 Thread Vasiliki Kalavri
Good to know :) On 25 November 2015 at 21:44, Stefanos Antaris <antaris.stefa...@gmail.com> wrote: > Hi, > > It works fine using this approach. > > Thanks, > Stefanos > > On 25 Nov 2015, at 20:32, Vasiliki Kalavri <vasilikikala...@gmail.com> > wrote: &g

Re: LDBC Graph Data into Flink

2015-11-24 Thread Vasiliki Kalavri
nd-apache-flink > > Best, > Martin > > On 06.10.2015 11:00, Martin Junghanns wrote: > > Hi Vasia, > > > > No problem. Sure, Gelly is just a map() call away :) > > > > Best, > > Martin > > > > On 06.10.2015 10:53, Vasiliki Kalavri wrote:

Re: Creating a representative streaming workload

2015-11-16 Thread Vasiliki Kalavri
e: > http://www.sparkbigdata.com/102-spark-blog-slim-baltagi/14-results-of-a-benchmark-between-apache-flink-and-apache-spark > > Best regards, > Ovidiu > > On 16 Nov 2015, at 15:21, Vasiliki Kalavri <vasilikikala...@gmail.com> > wrote: > > Hello squirrels, > > with some collea

Re: Zeppelin Integration

2015-11-04 Thread Vasiliki Kalavri
Great tutorial! Thanks a lot ^^ On 4 November 2015 at 17:12, Leonard Wolters wrote: > Indeed very nice! Thanks > On Nov 4, 2015 5:04 PM, "Till Rohrmann" wrote: > >> Really cool tutorial Trevor :-) >> >> On Wed, Nov 4, 2015 at 3:26 PM, Robert Metzger

Re: Creating Graphs from DataStream in Flink Gelly

2015-11-02 Thread Vasiliki Kalavri
Hi Ufuoma, Gelly doesn't support dynamic streaming graphs yet. The project Andra has linked to is a prototype for *one-pass* streaming graph analytics, i.e. no graph state is maintained. If you would like to keep and maintain the graph state in your streaming program, you would have to implement

Re: compile flink-gelly-scala using sbt

2015-10-28 Thread Vasiliki Kalavri
-dresden.de >>> >>> On Wed, Oct 28, 2015 at 3:50 PM, Theodore Vasiloudis < >>> theodoros.vasilou...@gmail.com> wrote: >>> >>>> Your build.sbt seems correct. >>>> It might be that you are missing some basic imports. >>>> &g

Re: LDBC Graph Data into Flink

2015-10-06 Thread Vasiliki Kalavri
Hi Martin, thanks a lot for sharing! This is a very useful tool. I only had a quick look, but if we merge label and payload inside a Tuple2, then it should also be Gelly-compatible :) Cheers, Vasia. On 6 October 2015 at 10:03, Martin Junghanns wrote: > Hi all, > > For

Re: Too few memory segments provided exception

2015-07-20 Thread Vasiliki Kalavri
Hi Shivani, why are you using a vertex-centric iteration to compute the approximate Adamic-Adar? It's not an iterative computation :) In fact, it should be as complex (in terms of operators) as the exact Adamic-Adar, only more efficient because of the different neighborhood representation. Are

Re: Too few memory segments provided exception

2015-07-20 Thread Vasiliki Kalavri
am curious how we should handle it in the future. On Mon, Jul 20, 2015 at 3:15 PM, Vasiliki Kalavri vasilikikala...@gmail.com wrote: Hi Shivani, why are you using a vertex-centric iteration to compute the approximate Adamic-Adar? It's not an iterative computation :) In fact, it should

Re: Too few memory segments provided exception

2015-07-20 Thread Vasiliki Kalavri
as messages to the vertices? The exact version is passing all the tests. On removing the final GroupReduce the program is working but I need it to add the Partial Adamic Adar edges weights. On Mon, Jul 20, 2015 at 3:15 PM, Vasiliki Kalavri vasilikikala...@gmail.com wrote: Hi Shivani, why

Re: Gelly forward

2015-07-08 Thread Vasiliki Kalavri
? On Wed, Jul 8, 2015 at 10:19 AM, Vasiliki Kalavri vasilikikala...@gmail.com wrote: Is it the same message that you propagate or is it different for each vertex / neighbor? If you have to store a neighborID, msg pair for each neighbor, then you will quickly run out of memory. If it's the same

Re: Gelly forward

2015-07-08 Thread Vasiliki Kalavri
Hi Flavio! Are you talking about vertex-centric iterations in gelly? If yes, you can send messages to a particular vertex with sendMessageTo(vertexId, msg) and to all neighbors with sendMessageToAllNeighbors(msg). These methods are available inside the MessagingFunction. Accessing received

Re: ArrayIndexOutOfBoundsException when running job from JAR

2015-06-29 Thread Vasiliki Kalavri
, there are at least two JVMs involved, and code running in the JM/TM can not access the value from the static variable in the Cli frontend. On Sun, Jun 28, 2015 at 9:43 PM, Vasiliki Kalavri vasilikikala...@gmail.com wrote: Hi everyone, Mihail and I have now solved the issue. The exception

Re: ArrayIndexOutOfBoundsException when running job from JAR

2015-06-26 Thread Vasiliki Kalavri
Hi Mihail, could you share your code or at least the implementations of getVerticesDataSet() and InitVerticesMapper so I can take a look? Where is InitVerticesMapper called above? Cheers, Vasia. On 26 June 2015 at 10:51, Mihail Vieru vi...@informatik.hu-berlin.de wrote: Hi Robert, I'm

Re: Gelly available already?

2015-03-24 Thread Vasiliki Kalavri
Hi all, there is no Scala API for Gelly yet and no corresponding JIRA either. It's definitely in our plans, just not for 0.9 :-) Cheers, -V. On 24 March 2015 at 00:21, Henry Saputra henry.sapu...@gmail.com wrote: Any JIRA filed to add Scala counterparts for Gelly? - Henry On Mon, Mar 23,

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

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

2015-03-18 Thread Vasiliki Kalavri
everything out :-) Cheers, V. On 18 March 2015 at 23:44, Vasiliki Kalavri vasilikikala...@gmail.com 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 vi...@informatik.hu-berlin.de wrote

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

2015-03-18 Thread Vasiliki Kalavri
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