Forwarding these here to keep dev@ in the loop :)

---------- Forwarded message ----------
From: Martin Junghanns <m.jungha...@mailbox.org>
Date: 29 October 2015 at 18:37
Subject: Re: neo4j - Flink connector
To: Martin Liesenberg <martin.liesenb...@gmail.com>, Vasia Kalavri <
vasilikikala...@gmail.com>
Cc: Alexander Keller <a...@graphaware.com>, Martin Neumann <mneum...@sics.se
>


My idea was to start with a (non-parallel) Neo4jInputFormat for Flink and
see how this REST endpoint works with streaming. As Cypher query results
are rows in the end, this should work well with Flink Tuples (similar to
our JDBCInputFormat).

I'll keep you updated!

Best,
Martin


On 29.10.2015 17:00, Martin Liesenberg wrote:

Also, if you need any help with the implementation, I'd be glad to chime
in.

Best regards
Martin

Martin Liesenberg <martin.liesenb...@gmail.com> schrieb am Do., 29. Okt.
2015 15:07:

> While using neo4j for a small project at work I came across the feature of
> streaming the results from neo4j with the REST API [1]. we didnt end up
> using it, so I can't comment on performance etc., but intuitively it seems
> like a better chunking.
>
> Nice to see another connector for Flink. :)
>
> Best regards,
>
> Martin
>
> [1] http://neo4j.com/docs/stable/rest-api-streaming.html
>
> Vasiliki Kalavri < <vasilikikala...@gmail.com>vasilikikala...@gmail.com>
> schrieb am Do., 29. Okt. 2015 um 14:51 Uhr:
>
>> Hello everyone,
>>
>> Martin, Martin, Alex (cc'ed) and myself have started discussing about
>> implementing a neo4j-Flink connector. I've opened a corresponding JIRA
>> (FLINK-2941) containing an initial document [1], but we'd also like to
>> share our ideas here to engage the community and get your feedback.
>>
>> We've had a skype call today and I will try to summarize some of the key
>> points here. The main use-cases we see are the following:
>>
>> - Use Flink for ETL / creating the graph and then insert it to a graph
>> database, like neo4j, for querying and search.
>> - Query neo4j on some topic or search the graph for patterns and extract a
>> subgraph, on which we'd then like to run some iterative graph analysis
>> task. This is where Flink/Gelly can help, by complementing the querying
>> (neo4j) with efficient iterative computation.
>>
>> We all agreed that the main challenge is efficiently getting the data out
>> of neo4j and into Flink. There have been some attempts to do similar
>> things
>> with neo4j and Spark, but the performance results are not very promising:
>>
>> - Mazerunner [2] is using HDFS for communication. We think that's it's not
>> worth it going towards this direction, as dumping the neo4j database to
>> HDFS and then reading it back to Flink would probably be terribly slow.
>> - In [3], you can see Michael Hunger's findings on using neo's HTTP
>> interface to import data into Spark, run PageRank and then put data back
>> into neo4j. It seems that this took > 2h for a 125m edge graph. The main
>> bottlenecks appear to be (1) reading the data as an RDD => this had to be
>> performed into small batches to avoid OOM errors and (2) PageRank
>> computation itself, which seems weird to me.
>>
>> We decided to experiment with neo4j HTTP and Flink and we'll report back
>> when we have some results.
>>
>> In the meantime, if you have any ideas on how we could speed up reading
>> from neo4j or any suggestion on approaches that I haven't mentioned,
>> please
>> feel free to reply to this e-mail or add your comment in the shared
>> document.
>>
>> Cheers,
>> -Vasia.
>>
>> [1]:
>>
>> https://docs.google.com/document/d/13qT_e-y8aTNWQnD43jRBq1074Y1LggPNDsic_Obwc28/edit?usp=sharing
>> [2]: https://github.com/kbastani/neo4j-mazerunner
>> [3]: https://gist.github.com/jexp/0dfad34d49a16000e804
>>
>

Reply via email to