SlideShare a Scribd company logo
1 of 41
Download to read offline
Vasia Kalavri
Flink committer & PhD student @KTH
vasia@apache.org
@vkalavri
Large-Scale Graph Processing
with Apache Flink
GraphDevroom
FOSDEM ‘15
Overview
● What is Apache Flink?
● Why Graph Processing with Flink:
○ user perspective
○ system perspective
● Gelly: the upcoming Flink Graph API
● Example: Music Profiles
Apache Flink
quick intro
What is Apache Flink?
● Large-scale data processing engine
● Java and Scala APIs
● Batch and Streaming Analytics
● Runs locally, on your cluster, on YARN
● Performs well even when memory runs out
4
The growing Flink stack
5
Flink Optimizer Flink Stream Builder
Common API
Scala API
(batch and streaming)
Java API
(batch and streaming)
Python API
(upcoming)
Graph API
Apache
MRQL
Flink Local Runtime
Embedded
environment
(Java collections)
Local
Environment
(for debugging)
Remote environment
(Regular cluster execution)
Apache Tez
Data
storage
HDFSFiles S3 JDBC Redis
Rabbit
MQ
Kafka
Azure
tables …
Single node execution Standalone or YARN cluster
● map, flatMap
● filter
● reduce,
reduceGroup
● join
● coGroup
● aggregate
Available Transformations
● cross
● project
● distinct
● union
● iterate
● iterateDelta
● ...
6
DataSet<String> text = env.readTextFile(input);
DataSet<Tuple2<String, Integer>> result = text
.flatMap((str, out) -> {
for (String token : value.split("W")) {
out.collect(new Tuple2<>(token, 1));
})
.groupBy(0)
.aggregate(SUM, 1);
Word Count
val input = env.readTextFile(input);
val words = input flatMap { line => line.split("W+")}
map { word => (word, 1)}
val counts = words groupBy(0) sum(1)
Java
Scala
7
Why Graph Processing
with Flink?
user perspective
Typical graph data analysis pipeline
load clean
create
graph
analyze
graph
clean transformload result
clean transformload
9
A more realistic pipeline
load clean
create
graph
analyze
graph
clean transformload result
clean transformload
often, it’s not easy to get the
graph properties and the
analysis algorithm right the
first time!
10
A more realistic pipeline
load clean
create
graph
analyze
graph
clean transformload result
clean transformload
11
A more user-friendly pipeline
load
load result
load
12
General-purpose or specialized?
- fast application
development and
deployment
- easier maintenance
- non-intuitive APIs
- time-consuming
- use, configure and integrate
different systems
- hard to maintain
- rich APIs and features
general-purpose specialized
what about performance?
13
Why Graph Processing
with Flink?
system perspective
Efficient Iterations
● Fink supports
iterations natively
○ the runtime is aware of
the iterative execution
○ no scheduling overhead
between iterations
○ caching and state
maintenance are handled
automatically
15
Flink Iteration Operators
Iterate IterateDelta
16
Input
Iterative
Update Function
Result
Replace
Workset
Iterative
Update Function
Result
Solution Set
State
Flink Optimizer
● The optimizer selects
an execution plan for
a program
● Think of an AI system
manipulating your
program for you
17
Optimization of Iterative algorithms
18
Caching Loop-invariant
Data
Pushing work
“out of the loop”
Maintain state as
index
Performance
● in-memory data
streaming
● memory
management
● serialization
framework
19
Scalability
20
http://data-artisans.com/computing-recommendations-with-flink.html
Gelly
the upcoming Flink Graph API
● Java Graph API on top of Flink
● Initial version coming with Flink 0.9
● Can be seamlessly mixed with the standard
Flink API
● Easily implement applications that use both
record-based and graph-based analysis
Meet Gelly
22
In Gelly, a Graph is simply represented by a DataSet of
Vertices and a DataSet of Edges:
Hello, Gelly!
Graph<String, Long, Double> graph = Graph.fromDataSet(vertices, edges, env);
Graph<String, Long, NullValue> graph = Graph.fromCollection(edges,
new MapFunction<String, Long>() {
public Long map(String value) {
return 1l;
}
}, env);
23
● Graph Properties
○ getVertexIds
○ getEdgeIds
○ numberOfVertices
○ numberOfEdges
○ getDegrees
○ isWeaklyConnected
○ ...
Available Methods
● Transformations
○ map, filter, join
○ subgraph, union
○ reverse, undirected
○ ...
● Mutations
○ add vertex/edge
○ remove vertex/edge
24
- Apply a reduce function to the 1st-hop
neighborhood of each vertex in parallel
Neighborhood Methods
3
4
7
4
4
graph.reduceOnNeighbors(new MinValue(), EdgeDirection.OUT);
3
9
7
4
5
25
● Validate a Graph according to given criteria
○ do the edge ids correspond to vertex ids?
○ are there duplicates?
○ is the graph bipartite?
Graph Validation
edges = { (1, 2), (3, 4), (1, 5), (2, 3), (6, 5) }
vertices = { 1, 2, 3, 4, 5 }
graph = Graph.fromCollection(vertices, edges);
graph.validate(new InvalidVertexIdsValidator()); // false
26
● Wraps the Flink Spargel
(Pregel-like) API
● The user only
implements two
functions
○ VertexUpdateFunction
○ MessagingFunction
● Internally creates a
delta iteration
Vertex-centric Iterations
27
updateVertex(K key, Double value,
MessageIterator msgs) {
Double minDist = Double.MAX_VALUE;
for (double msg : msgs) {
if (msg < minDist)
minDist = msg;
}
if (value > minDist)
setNewVertexValue(minDist);
}
Vertex-centric SSSP
sendMessages(K key, Double newDist) {
for (Edge edge : getOutgoingEdges()) {
sendMessageTo(edge.getTarget(),
newDist + edge.getValue());
}
shortestPaths = graph.runVertexCentricIteration(
new DistanceUpdater(), new DistanceMessenger()).getVertices();
DistanceUpdater: VertexUpdateFunction DistanceMessenger: MessagingFunction
28
● PageRank
● Single Source Shortest Paths
● Label Propagation
● Weakly Connected Components
Library of Algorithms
29
Example
User Music Profiles
Music Profiles
31
Problem Description
Input:
● <userId, songId, playCount> triplets
● a set of bad records (not to be trusted)
Tasks:
1. filter out bad records
2. compute the top song per user (most listened to)
3. create a user-user similarity graph based on common songs
4. detect communities on the similarity graph
32
1. Filter out bad records
/** Read <userID>t<songID>t<playcount> triplets */
DataSet<Tuple3> triplets = getTriplets();
/** Read the bad records songIDs */
DataSet<Tuple1> mismatches = getMismatches();
/** Filter out the mismatches from the triplets dataset */
DataSet<Tuple3> validTriplets = triplets.coGroup(mismatches).where(1).equalTo(0)
.with(new CoGroupFunction {
void coGroup(Iterable triplets, Iterable invalidSongs, Collector out) {
if (!invalidSongs.iterator().hasNext())
for (Tuple3 triplet : triplets) // this is a valid triplet
out.collect(triplet);
}
33
2a. Compute top song per user
/** Create a user -> song weighted bipartite graph where the edge weights correspond
to play counts */
Graph userSongGraph = Graph.fromTupleDataSet(validTriplets, env);
/** Get the top track (most listened) for each user */
DataSet<Tuple2> usersWithTopTrack = userSongGraph
.reduceOnEdges(new GetTopSongPerUser(), EdgeDirection.OUT);
34
Tom “I like birds”
“elephant woman”
“red morning”
323 plays
18 plays
42plays
2b. Compute top song per user
35
class GetTopSongPerUser implements EdgesFunctionWithVertexValue {
void iterateEdges(Vertex vertex, Iterable<Edge> edges) {
int maxPlaycount = 0;
String topSong = "";
for (Edge edge : edges) {
if (edge.getValue() > maxPlaycount) {
maxPlaycount = edge.getValue();
topSong = edge.getTarget();
}
}
return new Tuple2(vertex.getId(), topSong);
}
}
user-song to user-user graph
36
“red morning”
“I like birds”
“get lucky”
“disorder”
Tom
Steve
Wendy
“elephant woman”
Emily
Tom Steve
Wendy
Emily
Emily
3. Create a user-user similarity graph
/**Create a user-user similarity graph:
two users that listen to the same song are connected */
DataSet<Edge> similarUsers = userSongGraph.getEdges().groupBy(1)
.reduceGroup(new GroupReduceFunction() {
void reduce(Iterable<Edge> edges, Collector<Edge> out) {
List users = new ArrayList();
for (Edge edge : edges)
users.add(edge.getSource());
for (int i = 0; i < users.size() - 1; i++)
for (int j = i+1; j < users.size() - 1; j++)
out.collect(new Edge(users.get(i), users.get(j)));
}
}).distinct();
Graph similarUsersGraph = Graph.fromDataSet(similarUsers).getUndirected();
37
4. Cluster similar users
/** Detect user communities using label propagation */
// Initialize each vertex with a unique numeric label
DataSet<Tuple2> idsWithLabels = similarUsersGraph
.getVertices().reduceGroup(new AssignInitialLabel());
// update the vertex values and run the label propagation algorithm
DataSet<Vertex> verticesWithCommunity = similarUsersGraph
.joinWithVertices(idsWithlLabels, new MapFunction() {
public Long map(Tuple2 idWithLabel) {
return idWithLabel.f1;
}
}).run(new LabelPropagation(numIterations)).getVertices();
38
Music Profiles Recap
39
● Filter out bad records : record API
● Create user-song graph : record API
● Top song per user : Gelly
● Create user-user graph : record API
● Cluster users : Gelly
What’s next, Gelly?
● Gather-Sum-Apply
● Scala API
● More library methods
○ Clustering Coefficient
○ Minimum Spanning Tree
● Integration with the Flink Streaming API
● Specialized Operators for Skewed Graphs
40
Keep in touch!
● Gelly development repository
http://github.com/project-flink/flink-graph
● Apache Flink mailing lists
http://flink.apache.org/community.html#mailing-lists
● Follow @ApacheFlink
41

More Related Content

What's hot

Apache Flink@ Strata & Hadoop World London
Apache Flink@ Strata & Hadoop World LondonApache Flink@ Strata & Hadoop World London
Apache Flink@ Strata & Hadoop World London
Stephan Ewen
 

What's hot (20)

Apache Flink Internals: Stream & Batch Processing in One System – Apache Flin...
Apache Flink Internals: Stream & Batch Processing in One System – Apache Flin...Apache Flink Internals: Stream & Batch Processing in One System – Apache Flin...
Apache Flink Internals: Stream & Batch Processing in One System – Apache Flin...
 
Apache Flink Training: System Overview
Apache Flink Training: System OverviewApache Flink Training: System Overview
Apache Flink Training: System Overview
 
Architecture of Flink's Streaming Runtime @ ApacheCon EU 2015
Architecture of Flink's Streaming Runtime @ ApacheCon EU 2015Architecture of Flink's Streaming Runtime @ ApacheCon EU 2015
Architecture of Flink's Streaming Runtime @ ApacheCon EU 2015
 
Flink 0.10 @ Bay Area Meetup (October 2015)
Flink 0.10 @ Bay Area Meetup (October 2015)Flink 0.10 @ Bay Area Meetup (October 2015)
Flink 0.10 @ Bay Area Meetup (October 2015)
 
Gelly-Stream: Single-Pass Graph Streaming Analytics with Apache Flink
Gelly-Stream: Single-Pass Graph Streaming Analytics with Apache FlinkGelly-Stream: Single-Pass Graph Streaming Analytics with Apache Flink
Gelly-Stream: Single-Pass Graph Streaming Analytics with Apache Flink
 
Apache Flink & Graph Processing
Apache Flink & Graph ProcessingApache Flink & Graph Processing
Apache Flink & Graph Processing
 
Apache Flink internals
Apache Flink internalsApache Flink internals
Apache Flink internals
 
Real-time Stream Processing with Apache Flink
Real-time Stream Processing with Apache FlinkReal-time Stream Processing with Apache Flink
Real-time Stream Processing with Apache Flink
 
ApacheCon: Apache Flink - Fast and Reliable Large-Scale Data Processing
ApacheCon: Apache Flink - Fast and Reliable Large-Scale Data ProcessingApacheCon: Apache Flink - Fast and Reliable Large-Scale Data Processing
ApacheCon: Apache Flink - Fast and Reliable Large-Scale Data Processing
 
Streaming Data Flow with Apache Flink @ Paris Flink Meetup 2015
Streaming Data Flow with Apache Flink @ Paris Flink Meetup 2015Streaming Data Flow with Apache Flink @ Paris Flink Meetup 2015
Streaming Data Flow with Apache Flink @ Paris Flink Meetup 2015
 
Unified Stream and Batch Processing with Apache Flink
Unified Stream and Batch Processing with Apache FlinkUnified Stream and Batch Processing with Apache Flink
Unified Stream and Batch Processing with Apache Flink
 
Apache Flink Deep Dive
Apache Flink Deep DiveApache Flink Deep Dive
Apache Flink Deep Dive
 
Apache Flink@ Strata & Hadoop World London
Apache Flink@ Strata & Hadoop World LondonApache Flink@ Strata & Hadoop World London
Apache Flink@ Strata & Hadoop World London
 
Data Stream Processing with Apache Flink
Data Stream Processing with Apache FlinkData Stream Processing with Apache Flink
Data Stream Processing with Apache Flink
 
Apache Flink - Overview and Use cases of a Distributed Dataflow System (at pr...
Apache Flink - Overview and Use cases of a Distributed Dataflow System (at pr...Apache Flink - Overview and Use cases of a Distributed Dataflow System (at pr...
Apache Flink - Overview and Use cases of a Distributed Dataflow System (at pr...
 
Taking a look under the hood of Apache Flink's relational APIs.
Taking a look under the hood of Apache Flink's relational APIs.Taking a look under the hood of Apache Flink's relational APIs.
Taking a look under the hood of Apache Flink's relational APIs.
 
Unified Stream & Batch Processing with Apache Flink (Hadoop Summit Dublin 2016)
Unified Stream & Batch Processing with Apache Flink (Hadoop Summit Dublin 2016)Unified Stream & Batch Processing with Apache Flink (Hadoop Summit Dublin 2016)
Unified Stream & Batch Processing with Apache Flink (Hadoop Summit Dublin 2016)
 
Don't Cross The Streams - Data Streaming And Apache Flink
Don't Cross The Streams  - Data Streaming And Apache FlinkDon't Cross The Streams  - Data Streaming And Apache Flink
Don't Cross The Streams - Data Streaming And Apache Flink
 
Gelly in Apache Flink Bay Area Meetup
Gelly in Apache Flink Bay Area MeetupGelly in Apache Flink Bay Area Meetup
Gelly in Apache Flink Bay Area Meetup
 
Marton Balassi – Stateful Stream Processing
Marton Balassi – Stateful Stream ProcessingMarton Balassi – Stateful Stream Processing
Marton Balassi – Stateful Stream Processing
 

Viewers also liked

Anwar Rizal – Streaming & Parallel Decision Tree in Flink
Anwar Rizal – Streaming & Parallel Decision Tree in FlinkAnwar Rizal – Streaming & Parallel Decision Tree in Flink
Anwar Rizal – Streaming & Parallel Decision Tree in Flink
Flink Forward
 
Apache Flink Training: DataStream API Part 1 Basic
 Apache Flink Training: DataStream API Part 1 Basic Apache Flink Training: DataStream API Part 1 Basic
Apache Flink Training: DataStream API Part 1 Basic
Flink Forward
 
William Vambenepe – Google Cloud Dataflow and Flink , Stream Processing by De...
William Vambenepe – Google Cloud Dataflow and Flink , Stream Processing by De...William Vambenepe – Google Cloud Dataflow and Flink , Stream Processing by De...
William Vambenepe – Google Cloud Dataflow and Flink , Stream Processing by De...
Flink Forward
 
Albert Bifet – Apache Samoa: Mining Big Data Streams with Apache Flink
Albert Bifet – Apache Samoa: Mining Big Data Streams with Apache FlinkAlbert Bifet – Apache Samoa: Mining Big Data Streams with Apache Flink
Albert Bifet – Apache Samoa: Mining Big Data Streams with Apache Flink
Flink Forward
 
Marc Schwering – Using Flink with MongoDB to enhance relevancy in personaliza...
Marc Schwering – Using Flink with MongoDB to enhance relevancy in personaliza...Marc Schwering – Using Flink with MongoDB to enhance relevancy in personaliza...
Marc Schwering – Using Flink with MongoDB to enhance relevancy in personaliza...
Flink Forward
 
Moon soo Lee – Data Science Lifecycle with Apache Flink and Apache Zeppelin
Moon soo Lee – Data Science Lifecycle with Apache Flink and Apache ZeppelinMoon soo Lee – Data Science Lifecycle with Apache Flink and Apache Zeppelin
Moon soo Lee – Data Science Lifecycle with Apache Flink and Apache Zeppelin
Flink Forward
 
Maximilian Michels – Google Cloud Dataflow on Top of Apache Flink
Maximilian Michels – Google Cloud Dataflow on Top of Apache FlinkMaximilian Michels – Google Cloud Dataflow on Top of Apache Flink
Maximilian Michels – Google Cloud Dataflow on Top of Apache Flink
Flink Forward
 
Chris Hillman – Beyond Mapreduce Scientific Data Processing in Real-time
Chris Hillman – Beyond Mapreduce Scientific Data Processing in Real-timeChris Hillman – Beyond Mapreduce Scientific Data Processing in Real-time
Chris Hillman – Beyond Mapreduce Scientific Data Processing in Real-time
Flink Forward
 
Mohamed Amine Abdessemed – Real-time Data Integration with Apache Flink & Kafka
Mohamed Amine Abdessemed – Real-time Data Integration with Apache Flink & KafkaMohamed Amine Abdessemed – Real-time Data Integration with Apache Flink & Kafka
Mohamed Amine Abdessemed – Real-time Data Integration with Apache Flink & Kafka
Flink Forward
 
Tran Nam-Luc – Stale Synchronous Parallel Iterations on Flink
Tran Nam-Luc – Stale Synchronous Parallel Iterations on FlinkTran Nam-Luc – Stale Synchronous Parallel Iterations on Flink
Tran Nam-Luc – Stale Synchronous Parallel Iterations on Flink
Flink Forward
 

Viewers also liked (20)

K. Tzoumas & S. Ewen – Flink Forward Keynote
K. Tzoumas & S. Ewen – Flink Forward KeynoteK. Tzoumas & S. Ewen – Flink Forward Keynote
K. Tzoumas & S. Ewen – Flink Forward Keynote
 
Matthias J. Sax – A Tale of Squirrels and Storms
Matthias J. Sax – A Tale of Squirrels and StormsMatthias J. Sax – A Tale of Squirrels and Storms
Matthias J. Sax – A Tale of Squirrels and Storms
 
Anwar Rizal – Streaming & Parallel Decision Tree in Flink
Anwar Rizal – Streaming & Parallel Decision Tree in FlinkAnwar Rizal – Streaming & Parallel Decision Tree in Flink
Anwar Rizal – Streaming & Parallel Decision Tree in Flink
 
Apache Flink - Hadoop MapReduce Compatibility
Apache Flink - Hadoop MapReduce CompatibilityApache Flink - Hadoop MapReduce Compatibility
Apache Flink - Hadoop MapReduce Compatibility
 
Fabian Hueske – Cascading on Flink
Fabian Hueske – Cascading on FlinkFabian Hueske – Cascading on Flink
Fabian Hueske – Cascading on Flink
 
Assaf Araki – Real Time Analytics at Scale
Assaf Araki – Real Time Analytics at ScaleAssaf Araki – Real Time Analytics at Scale
Assaf Araki – Real Time Analytics at Scale
 
Apache Flink Training: DataStream API Part 1 Basic
 Apache Flink Training: DataStream API Part 1 Basic Apache Flink Training: DataStream API Part 1 Basic
Apache Flink Training: DataStream API Part 1 Basic
 
William Vambenepe – Google Cloud Dataflow and Flink , Stream Processing by De...
William Vambenepe – Google Cloud Dataflow and Flink , Stream Processing by De...William Vambenepe – Google Cloud Dataflow and Flink , Stream Processing by De...
William Vambenepe – Google Cloud Dataflow and Flink , Stream Processing by De...
 
Introduction to Apache Flink - Fast and reliable big data processing
Introduction to Apache Flink - Fast and reliable big data processingIntroduction to Apache Flink - Fast and reliable big data processing
Introduction to Apache Flink - Fast and reliable big data processing
 
Albert Bifet – Apache Samoa: Mining Big Data Streams with Apache Flink
Albert Bifet – Apache Samoa: Mining Big Data Streams with Apache FlinkAlbert Bifet – Apache Samoa: Mining Big Data Streams with Apache Flink
Albert Bifet – Apache Samoa: Mining Big Data Streams with Apache Flink
 
Marc Schwering – Using Flink with MongoDB to enhance relevancy in personaliza...
Marc Schwering – Using Flink with MongoDB to enhance relevancy in personaliza...Marc Schwering – Using Flink with MongoDB to enhance relevancy in personaliza...
Marc Schwering – Using Flink with MongoDB to enhance relevancy in personaliza...
 
Mikio Braun – Data flow vs. procedural programming
Mikio Braun – Data flow vs. procedural programming Mikio Braun – Data flow vs. procedural programming
Mikio Braun – Data flow vs. procedural programming
 
Moon soo Lee – Data Science Lifecycle with Apache Flink and Apache Zeppelin
Moon soo Lee – Data Science Lifecycle with Apache Flink and Apache ZeppelinMoon soo Lee – Data Science Lifecycle with Apache Flink and Apache Zeppelin
Moon soo Lee – Data Science Lifecycle with Apache Flink and Apache Zeppelin
 
Flink Case Study: Bouygues Telecom
Flink Case Study: Bouygues TelecomFlink Case Study: Bouygues Telecom
Flink Case Study: Bouygues Telecom
 
Slim Baltagi – Flink vs. Spark
Slim Baltagi – Flink vs. SparkSlim Baltagi – Flink vs. Spark
Slim Baltagi – Flink vs. Spark
 
Maximilian Michels – Google Cloud Dataflow on Top of Apache Flink
Maximilian Michels – Google Cloud Dataflow on Top of Apache FlinkMaximilian Michels – Google Cloud Dataflow on Top of Apache Flink
Maximilian Michels – Google Cloud Dataflow on Top of Apache Flink
 
Chris Hillman – Beyond Mapreduce Scientific Data Processing in Real-time
Chris Hillman – Beyond Mapreduce Scientific Data Processing in Real-timeChris Hillman – Beyond Mapreduce Scientific Data Processing in Real-time
Chris Hillman – Beyond Mapreduce Scientific Data Processing in Real-time
 
Mohamed Amine Abdessemed – Real-time Data Integration with Apache Flink & Kafka
Mohamed Amine Abdessemed – Real-time Data Integration with Apache Flink & KafkaMohamed Amine Abdessemed – Real-time Data Integration with Apache Flink & Kafka
Mohamed Amine Abdessemed – Real-time Data Integration with Apache Flink & Kafka
 
Aljoscha Krettek – Notions of Time
Aljoscha Krettek – Notions of TimeAljoscha Krettek – Notions of Time
Aljoscha Krettek – Notions of Time
 
Tran Nam-Luc – Stale Synchronous Parallel Iterations on Flink
Tran Nam-Luc – Stale Synchronous Parallel Iterations on FlinkTran Nam-Luc – Stale Synchronous Parallel Iterations on Flink
Tran Nam-Luc – Stale Synchronous Parallel Iterations on Flink
 

Similar to Large-scale graph processing with Apache Flink @GraphDevroom FOSDEM'15

Similar to Large-scale graph processing with Apache Flink @GraphDevroom FOSDEM'15 (20)

Flink internals web
Flink internals web Flink internals web
Flink internals web
 
Strata Singapore: Gearpump Real time DAG-Processing with Akka at Scale
Strata Singapore: GearpumpReal time DAG-Processing with Akka at ScaleStrata Singapore: GearpumpReal time DAG-Processing with Akka at Scale
Strata Singapore: Gearpump Real time DAG-Processing with Akka at Scale
 
Processing large-scale graphs with Google(TM) Pregel
Processing large-scale graphs with Google(TM) PregelProcessing large-scale graphs with Google(TM) Pregel
Processing large-scale graphs with Google(TM) Pregel
 
Frank Celler – Processing large-scale graphs with Google(TM) Pregel - NoSQL m...
Frank Celler – Processing large-scale graphs with Google(TM) Pregel - NoSQL m...Frank Celler – Processing large-scale graphs with Google(TM) Pregel - NoSQL m...
Frank Celler – Processing large-scale graphs with Google(TM) Pregel - NoSQL m...
 
A Future for R: Parallel and Distributed Processing in R for Everyone
A Future for R: Parallel and Distributed Processing in R for EveryoneA Future for R: Parallel and Distributed Processing in R for Everyone
A Future for R: Parallel and Distributed Processing in R for Everyone
 
Processing large-scale graphs with Google(TM) Pregel by MICHAEL HACKSTEIN at...
 Processing large-scale graphs with Google(TM) Pregel by MICHAEL HACKSTEIN at... Processing large-scale graphs with Google(TM) Pregel by MICHAEL HACKSTEIN at...
Processing large-scale graphs with Google(TM) Pregel by MICHAEL HACKSTEIN at...
 
Compiler Construction for DLX Processor
Compiler Construction for DLX Processor Compiler Construction for DLX Processor
Compiler Construction for DLX Processor
 
The Effect of Hierarchical Memory on the Design of Parallel Algorithms and th...
The Effect of Hierarchical Memory on the Design of Parallel Algorithms and th...The Effect of Hierarchical Memory on the Design of Parallel Algorithms and th...
The Effect of Hierarchical Memory on the Design of Parallel Algorithms and th...
 
Async fun
Async funAsync fun
Async fun
 
Hierarchical free monads and software design in fp
Hierarchical free monads and software design in fpHierarchical free monads and software design in fp
Hierarchical free monads and software design in fp
 
H2O Design and Infrastructure with Matt Dowle
H2O Design and Infrastructure with Matt DowleH2O Design and Infrastructure with Matt Dowle
H2O Design and Infrastructure with Matt Dowle
 
Beyond Breakpoints: A Tour of Dynamic Analysis
Beyond Breakpoints: A Tour of Dynamic AnalysisBeyond Breakpoints: A Tour of Dynamic Analysis
Beyond Breakpoints: A Tour of Dynamic Analysis
 
Kerberizing Spark: Spark Summit East talk by Abel Rincon and Jorge Lopez-Malla
Kerberizing Spark: Spark Summit East talk by Abel Rincon and Jorge Lopez-MallaKerberizing Spark: Spark Summit East talk by Abel Rincon and Jorge Lopez-Malla
Kerberizing Spark: Spark Summit East talk by Abel Rincon and Jorge Lopez-Malla
 
NVIDIA HPC ソフトウエア斜め読み
NVIDIA HPC ソフトウエア斜め読みNVIDIA HPC ソフトウエア斜め読み
NVIDIA HPC ソフトウエア斜め読み
 
Custom Pregel Algorithms in ArangoDB
Custom Pregel Algorithms in ArangoDBCustom Pregel Algorithms in ArangoDB
Custom Pregel Algorithms in ArangoDB
 
Apache Spark in your likeness - low and high level customization
Apache Spark in your likeness - low and high level customizationApache Spark in your likeness - low and high level customization
Apache Spark in your likeness - low and high level customization
 
Data Analysis With Apache Flink
Data Analysis With Apache FlinkData Analysis With Apache Flink
Data Analysis With Apache Flink
 
Data Analysis with Apache Flink (Hadoop Summit, 2015)
Data Analysis with Apache Flink (Hadoop Summit, 2015)Data Analysis with Apache Flink (Hadoop Summit, 2015)
Data Analysis with Apache Flink (Hadoop Summit, 2015)
 
ClojureScript loves React, DomCode May 26 2015
ClojureScript loves React, DomCode May 26 2015ClojureScript loves React, DomCode May 26 2015
ClojureScript loves React, DomCode May 26 2015
 
2011.10.14 Apache Giraph - Hortonworks
2011.10.14 Apache Giraph - Hortonworks2011.10.14 Apache Giraph - Hortonworks
2011.10.14 Apache Giraph - Hortonworks
 

More from Vasia Kalavri

More from Vasia Kalavri (14)

From data stream management to distributed dataflows and beyond
From data stream management to distributed dataflows and beyondFrom data stream management to distributed dataflows and beyond
From data stream management to distributed dataflows and beyond
 
Self-managed and automatically reconfigurable stream processing
Self-managed and automatically reconfigurable stream processingSelf-managed and automatically reconfigurable stream processing
Self-managed and automatically reconfigurable stream processing
 
Predictive Datacenter Analytics with Strymon
Predictive Datacenter Analytics with StrymonPredictive Datacenter Analytics with Strymon
Predictive Datacenter Analytics with Strymon
 
Online performance analysis of distributed dataflow systems (O'Reilly Velocit...
Online performance analysis of distributed dataflow systems (O'Reilly Velocit...Online performance analysis of distributed dataflow systems (O'Reilly Velocit...
Online performance analysis of distributed dataflow systems (O'Reilly Velocit...
 
The shortest path is not always a straight line
The shortest path is not always a straight lineThe shortest path is not always a straight line
The shortest path is not always a straight line
 
Graphs as Streams: Rethinking Graph Processing in the Streaming Era
Graphs as Streams: Rethinking Graph Processing in the Streaming EraGraphs as Streams: Rethinking Graph Processing in the Streaming Era
Graphs as Streams: Rethinking Graph Processing in the Streaming Era
 
Demystifying Distributed Graph Processing
Demystifying Distributed Graph ProcessingDemystifying Distributed Graph Processing
Demystifying Distributed Graph Processing
 
Like a Pack of Wolves: Community Structure of Web Trackers
Like a Pack of Wolves: Community Structure of Web TrackersLike a Pack of Wolves: Community Structure of Web Trackers
Like a Pack of Wolves: Community Structure of Web Trackers
 
Big data processing systems research
Big data processing systems researchBig data processing systems research
Big data processing systems research
 
Asymmetry in Large-Scale Graph Analysis, Explained
Asymmetry in Large-Scale Graph Analysis, ExplainedAsymmetry in Large-Scale Graph Analysis, Explained
Asymmetry in Large-Scale Graph Analysis, Explained
 
Block Sampling: Efficient Accurate Online Aggregation in MapReduce
Block Sampling: Efficient Accurate Online Aggregation in MapReduceBlock Sampling: Efficient Accurate Online Aggregation in MapReduce
Block Sampling: Efficient Accurate Online Aggregation in MapReduce
 
m2r2: A Framework for Results Materialization and Reuse
m2r2: A Framework for Results Materialization and Reusem2r2: A Framework for Results Materialization and Reuse
m2r2: A Framework for Results Materialization and Reuse
 
MapReduce: Optimizations, Limitations, and Open Issues
MapReduce: Optimizations, Limitations, and Open IssuesMapReduce: Optimizations, Limitations, and Open Issues
MapReduce: Optimizations, Limitations, and Open Issues
 
A Skype case study (2011)
A Skype case study (2011)A Skype case study (2011)
A Skype case study (2011)
 

Recently uploaded

Mg Road Call Girls Service: 🍓 7737669865 🍓 High Profile Model Escorts | Banga...
Mg Road Call Girls Service: 🍓 7737669865 🍓 High Profile Model Escorts | Banga...Mg Road Call Girls Service: 🍓 7737669865 🍓 High Profile Model Escorts | Banga...
Mg Road Call Girls Service: 🍓 7737669865 🍓 High Profile Model Escorts | Banga...
amitlee9823
 
Call Girls Indiranagar Just Call 👗 9155563397 👗 Top Class Call Girl Service B...
Call Girls Indiranagar Just Call 👗 9155563397 👗 Top Class Call Girl Service B...Call Girls Indiranagar Just Call 👗 9155563397 👗 Top Class Call Girl Service B...
Call Girls Indiranagar Just Call 👗 9155563397 👗 Top Class Call Girl Service B...
only4webmaster01
 
Call Girls In Hsr Layout ☎ 7737669865 🥵 Book Your One night Stand
Call Girls In Hsr Layout ☎ 7737669865 🥵 Book Your One night StandCall Girls In Hsr Layout ☎ 7737669865 🥵 Book Your One night Stand
Call Girls In Hsr Layout ☎ 7737669865 🥵 Book Your One night Stand
amitlee9823
 
CHEAP Call Girls in Saket (-DELHI )🔝 9953056974🔝(=)/CALL GIRLS SERVICE
CHEAP Call Girls in Saket (-DELHI )🔝 9953056974🔝(=)/CALL GIRLS SERVICECHEAP Call Girls in Saket (-DELHI )🔝 9953056974🔝(=)/CALL GIRLS SERVICE
CHEAP Call Girls in Saket (-DELHI )🔝 9953056974🔝(=)/CALL GIRLS SERVICE
9953056974 Low Rate Call Girls In Saket, Delhi NCR
 
Call Girls Begur Just Call 👗 7737669865 👗 Top Class Call Girl Service Bangalore
Call Girls Begur Just Call 👗 7737669865 👗 Top Class Call Girl Service BangaloreCall Girls Begur Just Call 👗 7737669865 👗 Top Class Call Girl Service Bangalore
Call Girls Begur Just Call 👗 7737669865 👗 Top Class Call Girl Service Bangalore
amitlee9823
 
Call Girls Bommasandra Just Call 👗 7737669865 👗 Top Class Call Girl Service B...
Call Girls Bommasandra Just Call 👗 7737669865 👗 Top Class Call Girl Service B...Call Girls Bommasandra Just Call 👗 7737669865 👗 Top Class Call Girl Service B...
Call Girls Bommasandra Just Call 👗 7737669865 👗 Top Class Call Girl Service B...
amitlee9823
 
Abortion pills in Doha Qatar (+966572737505 ! Get Cytotec
Abortion pills in Doha Qatar (+966572737505 ! Get CytotecAbortion pills in Doha Qatar (+966572737505 ! Get Cytotec
Abortion pills in Doha Qatar (+966572737505 ! Get Cytotec
Abortion pills in Riyadh +966572737505 get cytotec
 
Call Girls In Bellandur ☎ 7737669865 🥵 Book Your One night Stand
Call Girls In Bellandur ☎ 7737669865 🥵 Book Your One night StandCall Girls In Bellandur ☎ 7737669865 🥵 Book Your One night Stand
Call Girls In Bellandur ☎ 7737669865 🥵 Book Your One night Stand
amitlee9823
 
Call Girls Bannerghatta Road Just Call 👗 7737669865 👗 Top Class Call Girl Ser...
Call Girls Bannerghatta Road Just Call 👗 7737669865 👗 Top Class Call Girl Ser...Call Girls Bannerghatta Road Just Call 👗 7737669865 👗 Top Class Call Girl Ser...
Call Girls Bannerghatta Road Just Call 👗 7737669865 👗 Top Class Call Girl Ser...
amitlee9823
 
Chintamani Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore ...
Chintamani Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore ...Chintamani Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore ...
Chintamani Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore ...
amitlee9823
 

Recently uploaded (20)

April 2024 - Crypto Market Report's Analysis
April 2024 - Crypto Market Report's AnalysisApril 2024 - Crypto Market Report's Analysis
April 2024 - Crypto Market Report's Analysis
 
Mg Road Call Girls Service: 🍓 7737669865 🍓 High Profile Model Escorts | Banga...
Mg Road Call Girls Service: 🍓 7737669865 🍓 High Profile Model Escorts | Banga...Mg Road Call Girls Service: 🍓 7737669865 🍓 High Profile Model Escorts | Banga...
Mg Road Call Girls Service: 🍓 7737669865 🍓 High Profile Model Escorts | Banga...
 
Call Girls Indiranagar Just Call 👗 9155563397 👗 Top Class Call Girl Service B...
Call Girls Indiranagar Just Call 👗 9155563397 👗 Top Class Call Girl Service B...Call Girls Indiranagar Just Call 👗 9155563397 👗 Top Class Call Girl Service B...
Call Girls Indiranagar Just Call 👗 9155563397 👗 Top Class Call Girl Service B...
 
Discover Why Less is More in B2B Research
Discover Why Less is More in B2B ResearchDiscover Why Less is More in B2B Research
Discover Why Less is More in B2B Research
 
Call Girls In Hsr Layout ☎ 7737669865 🥵 Book Your One night Stand
Call Girls In Hsr Layout ☎ 7737669865 🥵 Book Your One night StandCall Girls In Hsr Layout ☎ 7737669865 🥵 Book Your One night Stand
Call Girls In Hsr Layout ☎ 7737669865 🥵 Book Your One night Stand
 
CHEAP Call Girls in Saket (-DELHI )🔝 9953056974🔝(=)/CALL GIRLS SERVICE
CHEAP Call Girls in Saket (-DELHI )🔝 9953056974🔝(=)/CALL GIRLS SERVICECHEAP Call Girls in Saket (-DELHI )🔝 9953056974🔝(=)/CALL GIRLS SERVICE
CHEAP Call Girls in Saket (-DELHI )🔝 9953056974🔝(=)/CALL GIRLS SERVICE
 
Invezz.com - Grow your wealth with trading signals
Invezz.com - Grow your wealth with trading signalsInvezz.com - Grow your wealth with trading signals
Invezz.com - Grow your wealth with trading signals
 
Call Girls Begur Just Call 👗 7737669865 👗 Top Class Call Girl Service Bangalore
Call Girls Begur Just Call 👗 7737669865 👗 Top Class Call Girl Service BangaloreCall Girls Begur Just Call 👗 7737669865 👗 Top Class Call Girl Service Bangalore
Call Girls Begur Just Call 👗 7737669865 👗 Top Class Call Girl Service Bangalore
 
Call Girls Bommasandra Just Call 👗 7737669865 👗 Top Class Call Girl Service B...
Call Girls Bommasandra Just Call 👗 7737669865 👗 Top Class Call Girl Service B...Call Girls Bommasandra Just Call 👗 7737669865 👗 Top Class Call Girl Service B...
Call Girls Bommasandra Just Call 👗 7737669865 👗 Top Class Call Girl Service B...
 
Sampling (random) method and Non random.ppt
Sampling (random) method and Non random.pptSampling (random) method and Non random.ppt
Sampling (random) method and Non random.ppt
 
Abortion pills in Doha Qatar (+966572737505 ! Get Cytotec
Abortion pills in Doha Qatar (+966572737505 ! Get CytotecAbortion pills in Doha Qatar (+966572737505 ! Get Cytotec
Abortion pills in Doha Qatar (+966572737505 ! Get Cytotec
 
Call Girls In Bellandur ☎ 7737669865 🥵 Book Your One night Stand
Call Girls In Bellandur ☎ 7737669865 🥵 Book Your One night StandCall Girls In Bellandur ☎ 7737669865 🥵 Book Your One night Stand
Call Girls In Bellandur ☎ 7737669865 🥵 Book Your One night Stand
 
Mature dropshipping via API with DroFx.pptx
Mature dropshipping via API with DroFx.pptxMature dropshipping via API with DroFx.pptx
Mature dropshipping via API with DroFx.pptx
 
5CL-ADBA,5cladba, Chinese supplier, safety is guaranteed
5CL-ADBA,5cladba, Chinese supplier, safety is guaranteed5CL-ADBA,5cladba, Chinese supplier, safety is guaranteed
5CL-ADBA,5cladba, Chinese supplier, safety is guaranteed
 
Anomaly detection and data imputation within time series
Anomaly detection and data imputation within time seriesAnomaly detection and data imputation within time series
Anomaly detection and data imputation within time series
 
Midocean dropshipping via API with DroFx
Midocean dropshipping via API with DroFxMidocean dropshipping via API with DroFx
Midocean dropshipping via API with DroFx
 
BigBuy dropshipping via API with DroFx.pptx
BigBuy dropshipping via API with DroFx.pptxBigBuy dropshipping via API with DroFx.pptx
BigBuy dropshipping via API with DroFx.pptx
 
Halmar dropshipping via API with DroFx
Halmar  dropshipping  via API with DroFxHalmar  dropshipping  via API with DroFx
Halmar dropshipping via API with DroFx
 
Call Girls Bannerghatta Road Just Call 👗 7737669865 👗 Top Class Call Girl Ser...
Call Girls Bannerghatta Road Just Call 👗 7737669865 👗 Top Class Call Girl Ser...Call Girls Bannerghatta Road Just Call 👗 7737669865 👗 Top Class Call Girl Ser...
Call Girls Bannerghatta Road Just Call 👗 7737669865 👗 Top Class Call Girl Ser...
 
Chintamani Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore ...
Chintamani Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore ...Chintamani Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore ...
Chintamani Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore ...
 

Large-scale graph processing with Apache Flink @GraphDevroom FOSDEM'15

  • 1. Vasia Kalavri Flink committer & PhD student @KTH vasia@apache.org @vkalavri Large-Scale Graph Processing with Apache Flink GraphDevroom FOSDEM ‘15
  • 2. Overview ● What is Apache Flink? ● Why Graph Processing with Flink: ○ user perspective ○ system perspective ● Gelly: the upcoming Flink Graph API ● Example: Music Profiles
  • 4. What is Apache Flink? ● Large-scale data processing engine ● Java and Scala APIs ● Batch and Streaming Analytics ● Runs locally, on your cluster, on YARN ● Performs well even when memory runs out 4
  • 5. The growing Flink stack 5 Flink Optimizer Flink Stream Builder Common API Scala API (batch and streaming) Java API (batch and streaming) Python API (upcoming) Graph API Apache MRQL Flink Local Runtime Embedded environment (Java collections) Local Environment (for debugging) Remote environment (Regular cluster execution) Apache Tez Data storage HDFSFiles S3 JDBC Redis Rabbit MQ Kafka Azure tables … Single node execution Standalone or YARN cluster
  • 6. ● map, flatMap ● filter ● reduce, reduceGroup ● join ● coGroup ● aggregate Available Transformations ● cross ● project ● distinct ● union ● iterate ● iterateDelta ● ... 6
  • 7. DataSet<String> text = env.readTextFile(input); DataSet<Tuple2<String, Integer>> result = text .flatMap((str, out) -> { for (String token : value.split("W")) { out.collect(new Tuple2<>(token, 1)); }) .groupBy(0) .aggregate(SUM, 1); Word Count val input = env.readTextFile(input); val words = input flatMap { line => line.split("W+")} map { word => (word, 1)} val counts = words groupBy(0) sum(1) Java Scala 7
  • 8. Why Graph Processing with Flink? user perspective
  • 9. Typical graph data analysis pipeline load clean create graph analyze graph clean transformload result clean transformload 9
  • 10. A more realistic pipeline load clean create graph analyze graph clean transformload result clean transformload often, it’s not easy to get the graph properties and the analysis algorithm right the first time! 10
  • 11. A more realistic pipeline load clean create graph analyze graph clean transformload result clean transformload 11
  • 12. A more user-friendly pipeline load load result load 12
  • 13. General-purpose or specialized? - fast application development and deployment - easier maintenance - non-intuitive APIs - time-consuming - use, configure and integrate different systems - hard to maintain - rich APIs and features general-purpose specialized what about performance? 13
  • 14. Why Graph Processing with Flink? system perspective
  • 15. Efficient Iterations ● Fink supports iterations natively ○ the runtime is aware of the iterative execution ○ no scheduling overhead between iterations ○ caching and state maintenance are handled automatically 15
  • 16. Flink Iteration Operators Iterate IterateDelta 16 Input Iterative Update Function Result Replace Workset Iterative Update Function Result Solution Set State
  • 17. Flink Optimizer ● The optimizer selects an execution plan for a program ● Think of an AI system manipulating your program for you 17
  • 18. Optimization of Iterative algorithms 18 Caching Loop-invariant Data Pushing work “out of the loop” Maintain state as index
  • 19. Performance ● in-memory data streaming ● memory management ● serialization framework 19
  • 22. ● Java Graph API on top of Flink ● Initial version coming with Flink 0.9 ● Can be seamlessly mixed with the standard Flink API ● Easily implement applications that use both record-based and graph-based analysis Meet Gelly 22
  • 23. In Gelly, a Graph is simply represented by a DataSet of Vertices and a DataSet of Edges: Hello, Gelly! Graph<String, Long, Double> graph = Graph.fromDataSet(vertices, edges, env); Graph<String, Long, NullValue> graph = Graph.fromCollection(edges, new MapFunction<String, Long>() { public Long map(String value) { return 1l; } }, env); 23
  • 24. ● Graph Properties ○ getVertexIds ○ getEdgeIds ○ numberOfVertices ○ numberOfEdges ○ getDegrees ○ isWeaklyConnected ○ ... Available Methods ● Transformations ○ map, filter, join ○ subgraph, union ○ reverse, undirected ○ ... ● Mutations ○ add vertex/edge ○ remove vertex/edge 24
  • 25. - Apply a reduce function to the 1st-hop neighborhood of each vertex in parallel Neighborhood Methods 3 4 7 4 4 graph.reduceOnNeighbors(new MinValue(), EdgeDirection.OUT); 3 9 7 4 5 25
  • 26. ● Validate a Graph according to given criteria ○ do the edge ids correspond to vertex ids? ○ are there duplicates? ○ is the graph bipartite? Graph Validation edges = { (1, 2), (3, 4), (1, 5), (2, 3), (6, 5) } vertices = { 1, 2, 3, 4, 5 } graph = Graph.fromCollection(vertices, edges); graph.validate(new InvalidVertexIdsValidator()); // false 26
  • 27. ● Wraps the Flink Spargel (Pregel-like) API ● The user only implements two functions ○ VertexUpdateFunction ○ MessagingFunction ● Internally creates a delta iteration Vertex-centric Iterations 27
  • 28. updateVertex(K key, Double value, MessageIterator msgs) { Double minDist = Double.MAX_VALUE; for (double msg : msgs) { if (msg < minDist) minDist = msg; } if (value > minDist) setNewVertexValue(minDist); } Vertex-centric SSSP sendMessages(K key, Double newDist) { for (Edge edge : getOutgoingEdges()) { sendMessageTo(edge.getTarget(), newDist + edge.getValue()); } shortestPaths = graph.runVertexCentricIteration( new DistanceUpdater(), new DistanceMessenger()).getVertices(); DistanceUpdater: VertexUpdateFunction DistanceMessenger: MessagingFunction 28
  • 29. ● PageRank ● Single Source Shortest Paths ● Label Propagation ● Weakly Connected Components Library of Algorithms 29
  • 32. Problem Description Input: ● <userId, songId, playCount> triplets ● a set of bad records (not to be trusted) Tasks: 1. filter out bad records 2. compute the top song per user (most listened to) 3. create a user-user similarity graph based on common songs 4. detect communities on the similarity graph 32
  • 33. 1. Filter out bad records /** Read <userID>t<songID>t<playcount> triplets */ DataSet<Tuple3> triplets = getTriplets(); /** Read the bad records songIDs */ DataSet<Tuple1> mismatches = getMismatches(); /** Filter out the mismatches from the triplets dataset */ DataSet<Tuple3> validTriplets = triplets.coGroup(mismatches).where(1).equalTo(0) .with(new CoGroupFunction { void coGroup(Iterable triplets, Iterable invalidSongs, Collector out) { if (!invalidSongs.iterator().hasNext()) for (Tuple3 triplet : triplets) // this is a valid triplet out.collect(triplet); } 33
  • 34. 2a. Compute top song per user /** Create a user -> song weighted bipartite graph where the edge weights correspond to play counts */ Graph userSongGraph = Graph.fromTupleDataSet(validTriplets, env); /** Get the top track (most listened) for each user */ DataSet<Tuple2> usersWithTopTrack = userSongGraph .reduceOnEdges(new GetTopSongPerUser(), EdgeDirection.OUT); 34 Tom “I like birds” “elephant woman” “red morning” 323 plays 18 plays 42plays
  • 35. 2b. Compute top song per user 35 class GetTopSongPerUser implements EdgesFunctionWithVertexValue { void iterateEdges(Vertex vertex, Iterable<Edge> edges) { int maxPlaycount = 0; String topSong = ""; for (Edge edge : edges) { if (edge.getValue() > maxPlaycount) { maxPlaycount = edge.getValue(); topSong = edge.getTarget(); } } return new Tuple2(vertex.getId(), topSong); } }
  • 36. user-song to user-user graph 36 “red morning” “I like birds” “get lucky” “disorder” Tom Steve Wendy “elephant woman” Emily Tom Steve Wendy Emily Emily
  • 37. 3. Create a user-user similarity graph /**Create a user-user similarity graph: two users that listen to the same song are connected */ DataSet<Edge> similarUsers = userSongGraph.getEdges().groupBy(1) .reduceGroup(new GroupReduceFunction() { void reduce(Iterable<Edge> edges, Collector<Edge> out) { List users = new ArrayList(); for (Edge edge : edges) users.add(edge.getSource()); for (int i = 0; i < users.size() - 1; i++) for (int j = i+1; j < users.size() - 1; j++) out.collect(new Edge(users.get(i), users.get(j))); } }).distinct(); Graph similarUsersGraph = Graph.fromDataSet(similarUsers).getUndirected(); 37
  • 38. 4. Cluster similar users /** Detect user communities using label propagation */ // Initialize each vertex with a unique numeric label DataSet<Tuple2> idsWithLabels = similarUsersGraph .getVertices().reduceGroup(new AssignInitialLabel()); // update the vertex values and run the label propagation algorithm DataSet<Vertex> verticesWithCommunity = similarUsersGraph .joinWithVertices(idsWithlLabels, new MapFunction() { public Long map(Tuple2 idWithLabel) { return idWithLabel.f1; } }).run(new LabelPropagation(numIterations)).getVertices(); 38
  • 39. Music Profiles Recap 39 ● Filter out bad records : record API ● Create user-song graph : record API ● Top song per user : Gelly ● Create user-user graph : record API ● Cluster users : Gelly
  • 40. What’s next, Gelly? ● Gather-Sum-Apply ● Scala API ● More library methods ○ Clustering Coefficient ○ Minimum Spanning Tree ● Integration with the Flink Streaming API ● Specialized Operators for Skewed Graphs 40
  • 41. Keep in touch! ● Gelly development repository http://github.com/project-flink/flink-graph ● Apache Flink mailing lists http://flink.apache.org/community.html#mailing-lists ● Follow @ApacheFlink 41