SlideShare a Scribd company logo
Batch & Stream Graph Processing
with Apache Flink
Vasia Kalavri
vasia@apache.org
@vkalavri
Apache Flink Meetup London
October 5th, 2016
2
Graphs capture relationships
between data items
connections, interactions, purchases,
dependencies, friendships, etc.
Recommenders
Social networks
Bioinformatics
Web search
Outline
• Distributed Graph Processing 101
• Gelly: Batch Graph Processing with Apache Flink
• BREAK!
• Gelly-Stream: Continuous Graph Processing with
Apache Flink
Apache Flink
• An open-source, distributed data analysis framework
• True streaming at its core
• Streaming & Batch API
4
Historic data
Kafka, RabbitMQ, ...
HDFS, JDBC, ...
Event logs
ETL, Graphs,

Machine Learning

Relational, …
Low latency,

windowing,
aggregations, ...
WHEN DO YOU NEED
DISTRIBUTED GRAPH
PROCESSING?
MY GRAPH IS SO BIG,
IT DOESN’T FIT IN A
SINGLE MACHINE
Big Data Ninja
MISCONCEPTION #1
A SOCIAL NETWORK
NAIVE WHO(M)-T0-FOLLOW
▸ Naive Who(m) to Follow:
▸ compute a friends-of-friends
list per user
▸ exclude existing friends
▸ rank by common
connections
DON’T JUST
CONSIDER YOUR
INPUT GRAPH SIZE.
INTERMEDIATE DATA
MATTERS TOO!
DISTRIBUTED PROCESSING
IS ALWAYS FASTER THAN
SINGLE-NODE
Data Science Rockstar
MISCONCEPTION #2
GRAPHS DON’T APPEAR OUT OF THIN AIR
Expectation…
GRAPHS DON’T APPEAR OUT OF THIN AIR
Reality!
WHEN DO YOU NEED DISTRIBUTED GRAPH PROCESSING?
▸ When you do have really big graphs
▸ When the intermediate data is big
▸ When your data is already distributed
▸ When you want to build end-to-end graph pipelines
HOW DO WE EXPRESS A
DISTRIBUTED GRAPH
ANALYSIS TASK?
RECENT DISTRIBUTED GRAPH PROCESSING HISTORY
2004
MapReduce
Pegasus
2009
Pregel
2010
Signal-Collect
PowerGraph
2012
Iterative value propagation
Giraph++
2013
Graph Traversals
NScale
2014
Ego-network analysis
Arabesque
2015
Pattern Matching
Tinkerpop
PREGEL: THINK LIKE A VERTEX
1
5
4
3
2
1 3, 4
2 1, 4
5 3
...
PREGEL: SUPERSTEPS
(Vi+1, outbox) <— compute(Vi, inbox)
1 3, 4
2 1, 4
5 3
..
1 3, 4
2 1, 4
5 3
..
Superstep i Superstep i+1
PAGERANK: THE WORD COUNT OF GRAPH PROCESSING
VertexID Out-degree
Transition
Probability
1 2 1/2
2 2 1/2
3 0 -
4 3 1/3
5 1 1
1
5
4
3
2
PAGERANK: THE WORD COUNT OF GRAPH PROCESSING
VertexID Out-degree
Transition
Probability
1 2 1/2
2 2 1/2
3 0 -
4 3 1/3
5 1 1
PR(3) = 0.5*PR(1) + 0.33*PR(4) + PR(5)
1
5
4
3
2
1
5
4
3
2
PAGERANK: THE WORD COUNT OF GRAPH PROCESSING
VertexID Out-degree
Transition
Probability
1 2 1/2
2 2 1/2
3 0 -
4 3 1/3
5 1 1
PR(3) = 0.5*PR(1) + 0.33*PR(4) + PR(5)
PAGERANK: THE WORD COUNT OF GRAPH PROCESSING
VertexID Out-degree
Transition
Probability
1 2 1/2
2 2 1/2
3 0 -
4 3 1/3
5 1 1
PR(3) = 0.5*PR(1) + 0.33*PR(4) + PR(5)
1
5
4
3
2
PAGERANK: THE WORD COUNT OF GRAPH PROCESSING
VertexID Out-degree
Transition
Probability
1 2 1/2
2 2 1/2
3 0 -
4 3 1/3
5 1 1
PR(3) = 0.5*PR(1) + 0.33*PR(4) + PR(5)
1
5
4
3
2
PREGEL EXAMPLE: PAGERANK
void compute(messages):
sum = 0.0
for (m <- messages) do
sum = sum + m
end for
setValue(0.15/numVertices() + 0.85*sum)
for (edge <- getOutEdges()) do
sendMessageTo(
edge.target(), getValue()/numEdges)
end for
sum up received
messages
update vertex rank
distribute rank
to neighbors
SIGNAL-COLLECT
outbox <— signal(Vi)
1 3, 4
2 1, 4
5 3
..
1 3, 4
2 1, 4
5 3
..
Superstep i
Vi+1 <— collect(inbox)
1 3, 4
2 1, 4
5 3
..
Signal Collect
Superstep i+1
SIGNAL-COLLECT EXAMPLE: PAGERANK
void signal():
for (edge <- getOutEdges()) do
sendMessageTo(
edge.target(), getValue()/numEdges)
end for
void collect(messages):
sum = 0.0
for (m <- messages) do
sum = sum + m
end for
setValue(0.15/numVertices() + 0.85*sum)
distribute rank
to neighbors
sum up
messages
update vertex
rank
GATHER-SUM-APPLY (POWERGRAPH)
1
...
...
Gather Sum
1
2
5
...
Apply
3
1 5
5 3
1
...
Gather
3
1 5
5 3
Superstep i Superstep i+1
GSA EXAMPLE: PAGERANK
double gather(source, edge, target):
return target.value() / target.numEdges()
double sum(rank1, rank2):
return rank1 + rank2
double apply(sum, currentRank):
return 0.15 + 0.85*sum
compute
partial rank
combine
partial ranks
update rank
PREGEL VS. SIGNAL-COLLECT VS. GSA
Update Function
Properties
Update Function
Logic
Communication
Scope
Communication
Logic
Pregel arbitrary arbitrary any vertex arbitrary
Signal-Collect arbitrary
based on
received
messages
any vertex
based on vertex
state
GSA
associative &
commutative
based on
neighbors’
values
neighborhood
based on vertex
state
CAN WE HAVE IT ALL?
▸ Data pipeline integration: built on top of an
efficient distributed processing engine
▸ Graph ETL: high-level API with abstractions and
methods to transform graphs
▸ Familiar programming model: support popular
programming abstractions
Gelly
the Apache Flink Graph API
Apache Flink Stack
Gelly
Table/SQL
ML
SAMOA
DataSet (Java/Scala) DataStream (Java/Scala)
HadoopM/R
Local Remote Yarn Embedded
Dataflow
Dataflow
Table/SQL
Cascading
Streaming dataflow runtime
CEP
Meet Gelly
• Java & Scala Graph APIs on top of Flink’s DataSet API
Flink Core
Scala API
(batch and streaming)
Java API
(batch and streaming)
FlinkML GellyTable API ...
Transformations
and Utilities
Iterative Graph
Processing
Graph Library
34
Gelly is NOT
• a graph database
• a specialized graph processor
35
Hello, Gelly!
ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
DataSet<Edge<Long, NullValue>> edges = getEdgesDataSet(env);
Graph<Long, Long, NullValue> graph = Graph.fromDataSet(edges, env);
DataSet<Vertex<Long, Long>> verticesWithMinIds = graph.run(
new ConnectedComponents(maxIterations));
val env = ExecutionEnvironment.getExecutionEnvironment
val edges: DataSet[Edge[Long, NullValue]] = getEdgesDataSet(env)
val graph = Graph.fromDataSet(edges, env)
val components = graph.run(new ConnectedComponents(maxIterations))
Java
Scala
Graph Methods
Graph Properties
getVertexIds
getEdgeIds
numberOfVertices
numberOfEdges
getDegrees
Mutations
add vertex/edge
remove vertex/edge
Transformations
map, filter, join
subgraph, union,
difference
reverse, undirected
getTriplets
Generators
R-Mat (power-law)
Grid
Star
Complete
…
Example: mapVertices
// increment each vertex value by one

val graph = Graph.fromDataSet(...)



// increment each vertex value by one

val updatedGraph = graph.mapVertices(v => v.getValue + 1)
4
2
8
5
5
3
1
7
4
5
Example: subGraph
val graph: Graph[Long, Long, Long] = ...



// keep only vertices with positive values

// and only edges with negative values

val subGraph = graph.subgraph(
vertex => vertex.getValue > 0,
edge => edge.getValue < 0
)
Neighborhood Methods
Apply a reduce function to the 1st-hop neighborhood
of each vertex in parallel
graph.reduceOnNeighbors(
new MinValue, EdgeDirection.OUT)
What makes Gelly unique?
• Batch graph processing on top of a streaming
dataflow engine
• Built for end-to-end analytics
• Support for multiple iteration abstractions
• Graph algorithm building blocks
• A large open-source library of graph algorithms
Why streaming dataflow?
• Batch engines materialize data… even if they don’t
have to
• the graph is always loaded and materialized in memory,
even if not needed, e.g. mapping, filtering, transformation
• Communication and computation overlap
• We can do continuous graph processing (more
after the break!)
End-to-end analytics
• Graphs don’t appear out of thin air…
• We need to support pre- and post-processing
• Gelly can be easily mixed with the DataSet API:
pre-processing, graph analysis, and post-
processing in the same Flink program
Iterative Graph Processing
• Gelly offers iterative graph processing abstractions
on top of Flink’s Delta iterations
• vertex-centric
• scatter-gather
• gather-sum-apply
• partition-centric*
Flink Iteration Operators
Input
Iterative
Update Function
Result
Replace
Workset
Iterative
Update Function
Result
Solution Set
State
Optimization
• the runtime is aware of the iterative execution
• no scheduling overhead between iterations
• caching and state maintenance are handled automatically
Push work

“out of the loop”
Maintain state as indexCache Loop-invariant Data
Vertex-Centric SSSP
final class SSSPComputeFunction extends ComputeFunction {
override def compute(vertex: Vertex, messages: MessageIterator) = {
var minDistance = if (vertex.getId == srcId) 0 else Double.MaxValue
while (messages.hasNext) {
val msg = messages.next
if (msg < minDistance)
minDistance = msg
}
if (vertex.getValue > minDistance) {
setNewVertexValue(minDistance)
for (edge: Edge <- getEdges)
sendMessageTo(edge.getTarget, vertex.getValue + edge.getValue)
}
Algorithms building blocks
• Allow operator re-use across graph algorithms
when processing the same input with a similar
configuration
Library of Algorithms
• PageRank
• Single Source Shortest Paths
• Label Propagation
• Weakly Connected Components
• Community Detection
• Triangle Count & Enumeration
• Local and Global Clustering Coefficient
• HITS
• Jaccard & Adamic-Adar Similarity
• Graph Summarization
• val ranks = inputGraph.run(new PageRank(0.85, 20))
Tracker
Tracker
Ad Server
display
relevant ads
cookie
exchange
profiling
Web Tracking
Can’t we block them?
proxy
Tracker
Tracker
Ad Server
Legitimate site
• not frequently updated
• not sure who or based on what criteria URLs are
blacklisted
• miss “hidden” trackers or dual-role nodes
• blocking requires manual matching against the list
• can you buy your way into the whitelist?
Available Solutions
Crowd-sourced “black lists” of tracker URLs:
- AdBlock, DoNotTrack, EasyPrivacy
DataSet
• 6 months (Nov 2014 - April 2015) of augmented
Apache logs from a web proxy
• 80m requests, 2m distinct URLs, 3k users
h2
h3 h4
h5 h6
h8
h7
h1
h3
h4
h5
h6
h1
h2
h7
h8
r1
r2
r3
r5
r6
r7
NT
NT
T
T
?
T
NT
NT
r4
r1
r2
r3
r3
r3
r4
r5r6
r7
hosts-projection graph
: referer
: non-tracker host
: tracker host
: unlabeled host
The Hosts-Projection Graph
U: Referers
referer-hosts graph
V: hosts
Classification via Label Propagation
non-tracker tracker unlabeled
55
Data Pipeline
raw logs
cleaned
logs
1: logs pre-
processing
2: bipartite graph
creation
3: largest
connected
component
extraction
4: hosts-
projection
graph creation
5: community
detection
google-analytics.com: T
bscored-research.com: T
facebook.com: NT
github.com: NT
cdn.cxense.com: NT
...
6: results
DataSet API
Gelly
DataSet API
Feeling Gelly?
• Gelly Guide
https://ci.apache.org/projects/flink/flink-docs-master/libs/
gelly_guide.html
• To Petascale and Beyond @Flink Forward ‘16
http://flink-forward.org/kb_sessions/to-petascale-and-beyond-apache-
flink-in-the-clouds/
• Web Tracker Detection @Flink Forward ’15
https://www.youtube.com/watch?v=ZBCXXiDr3TU
paper: Kalavri, Vasiliki, et al. "Like a pack of wolves: Community
structure of web trackers." International Conference on Passive and
Active Network Measurement, 2016.
Gelly-Stream
single-pass stream graph
processing with Flink
Real Graphs are dynamic
Graphs are created from events happening in real-time
How we’ve done graph processing so far
1. Load: read the graph
from disk and partition it in
memory
2. Compute: read and
mutate the graph state
How we’ve done graph processing so far
1. Load: read the graph
from disk and partition it in
memory
3. Store: write the final
graph state back to disk
How we’ve done graph processing so far
2. Compute: read and
mutate the graph state
1. Load: read the graph
from disk and partition it in
memory
What’s wrong with this model?
• It is slow
• wait until the computation is over before you see
any result
• pre-processing and partitioning
• It is expensive
• lots of memory and CPU required in order to
scale
• It requires re-computation for graph changes
• no efficient way to deal with updates
Can we do graph processing
on streams?
• Maintain the
dynamic graph
structure
• Provide up-to-date
results with low
latency
• Compute on fresh
state only
Single-pass graph streaming
• Each event is an edge addition
• Maintain only a graph summary
• Recent events are grouped in graph
windows
Graph Summaries
• spanners for distance estimation
• sparsifiers for cut estimation
• sketches for homomorphic properties
graph summary
algorithm algorithm~R1 R2
1
43
2
5
i=0
Batch Connected
Components
6
7
8
1
43
2
5
6
7
8
i=0
Batch Connected
Components
1
4
3
4
5
2
3
5
2
4
7
8
6
7
6
8
1
21
2
2
i=1
Batch Connected
Components
6
6
6
1
21
1
2
6
6
6
i=1
Batch Connected
Components
2
1
2
2
1
1
2
1
2
7
6
6
6
1
11
1
1
i=2
Batch Connected
Components
6
6
6
54
76
86
42
31
52
Stream Connected
Components
Graph Summary: Disjoint
Set (Union-Find)
• Only store component IDs
and vertex IDs
54
76
86
42
43
31
52
1
3
Cid = 1
54
76
86
42
43
87
31
52
1
3
Cid = 1
2
5
Cid = 2
54
76
86
42
43
87
41
31
52
1
3
Cid = 1
2
5
Cid = 2
4
54
76
86
42
43
87
41
31
52
1
3
Cid = 1
2
5
Cid = 2
4
6
7
Cid = 6
54
76
86
42
43
87
41
31
52
1
3
Cid = 1
2
5
Cid = 2
4
6
7
Cid = 6
8
54
76
86
42
43
87
41
52
1
3
Cid = 1
2
5
Cid = 2
4
6
7
Cid = 6
8
54
76
86
42
43
87
41
6
7
Cid = 6
8
1
3
Cid = 1
2
5
Cid = 2
4
54
76
86
42
43
87
41
1
3
Cid = 1
2
5
4
6
7
Cid = 6
8
Distributed Stream Connected
Components
Stream Connected
Components with Flink
DataStream<DisjointSet> cc =
edgeStream

.keyBy(0)

.timeWindow(Time.of(100, TimeUnit.MILLISECONDS))

.fold(new DisjointSet(), new UpdateCC())

.flatMap(new Merger())

.setParallelism(1);
Stream Connected
Components with Flink
DataStream<DisjointSet> cc =
edgeStream

.keyBy(0)

.timeWindow(Time.of(100, TimeUnit.MILLISECONDS))

.fold(new DisjointSet(), new UpdateCC())

.flatMap(new Merger())

.setParallelism(1);
Partition the edge
stream
Stream Connected
Components with Flink
DataStream<DisjointSet> cc =
edgeStream

.keyBy(0)

.timeWindow(Time.of(100, TimeUnit.MILLISECONDS))

.fold(new DisjointSet(), new UpdateCC())

.flatMap(new Merger())

.setParallelism(1);
Define the merging
frequency
Stream Connected
Components with Flink
DataStream<DisjointSet> cc =
edgeStream

.keyBy(0)

.timeWindow(Time.of(100, TimeUnit.MILLISECONDS))

.fold(new DisjointSet(), new UpdateCC())

.flatMap(new Merger())

.setParallelism(1);
merge locally
Stream Connected
Components with Flink
DataStream<DisjointSet> cc =
edgeStream

.keyBy(0)

.timeWindow(Time.of(100, TimeUnit.MILLISECONDS))

.fold(new DisjointSet(), new UpdateCC())

.flatMap(new Merger())

.setParallelism(1); merge globally
Gelly on Streams
DataStreamDataSet
Distributed Dataflow
Deployment
Gelly Gelly-Stream
• Static Graphs
• Multi-Pass Algorithms
• Full Computations
• Dynamic Graphs
• Single-Pass Algorithms
• Approximate Computations
DataStream
Introducing Gelly-Stream
Gelly-Stream enriches the DataStream API with two new additional ADTs:
• GraphStream:
• A representation of a data stream of edges.
• Edges can have state (e.g. weights).
• Supports property streams, transformations and aggregations.
• GraphWindow:
• A “time-slice” of a graph stream.
• It enables neighborhood aggregations
GraphStream Operations
.getEdges()
.getVertices()
.numberOfVertices()
.numberOfEdges()
.getDegrees()
.inDegrees()
.outDegrees()
GraphStream -> DataStream
.mapEdges();
.distinct();
.filterVertices();
.filterEdges();
.reverse();
.undirected();
.union();
GraphStream -> GraphStream
Property Streams Transformations
Graph Stream Aggregations
result
aggregate
property streamgraph
stream
(window) fold
combine
fold
reduce
local
summaries
global
summary
edges
agg
global aggregates
can be persistent or transient
graphStream.aggregate(
new MyGraphAggregation(window, fold, combine, transform))
Slicing Graph Streams
graphStream.slice(Time.of(1, MINUTE));
11:40 11:41 11:42 11:43
Aggregating Slices
graphStream.slice(Time.of(1, MINUTE), direction)
.reduceOnEdges();
.foldNeighbors();
.applyOnNeighbors();
• Slicing collocates edges by vertex
information
• Neighborhood aggregations on sliced
graphs
source
target
Aggregations
Finding Matches Nearby
graphStream.filterVertices(GraphGeeks())
.slice(Time.of(15, MINUTE), EdgeDirection.IN)
.applyOnNeighbors(FindPairs())
slice
GraphStream :: graph geek check-ins
wendy checked_in soap_bar
steve checked_in soap_bar
tom checked_in joe’s_grill
sandra checked_in soap_bar
rafa checked_in joe’s_grill
wendy
steve
sandra
soap
bar
tom
rafa
joe’s
grill
FindPairs
{wendy, steve}
{steve, sandra}
{wendy, sandra}
{tom, rafa}
GraphWindow :: user-place
Feeling Gelly?
• Gelly Guide
https://ci.apache.org/projects/flink/flink-docs-master/libs/
gelly_guide.html
• Gelly-Stream Repository
https://github.com/vasia/gelly-streaming
• Gelly-Stream talk @FOSDEM16
https://fosdem.org/2016/schedule/event/graph_processing_apache_flink/
• Related Papers
http://www.citeulike.org/user/vasiakalavri/tag/graph-streaming
Batch & Stream Graph Processing
with Apache Flink
Vasia Kalavri
vasia@apache.org
@vkalavri
Apache Flink Meetup London
October 5th, 2016

More Related Content

What's hot

Morel, a Functional Query Language
Morel, a Functional Query LanguageMorel, a Functional Query Language
Morel, a Functional Query Language
Julian Hyde
 
Reactive Streams: Handling Data-Flow the Reactive Way
Reactive Streams: Handling Data-Flow the Reactive WayReactive Streams: Handling Data-Flow the Reactive Way
Reactive Streams: Handling Data-Flow the Reactive Way
Roland Kuhn
 
Apache Beam and Google Cloud Dataflow - IDG - final
Apache Beam and Google Cloud Dataflow - IDG - finalApache Beam and Google Cloud Dataflow - IDG - final
Apache Beam and Google Cloud Dataflow - IDG - finalSub Szabolcs Feczak
 
Ray: Enterprise-Grade, Distributed Python
Ray: Enterprise-Grade, Distributed PythonRay: Enterprise-Grade, Distributed Python
Ray: Enterprise-Grade, Distributed Python
Databricks
 
Real-Time Spark: From Interactive Queries to Streaming
Real-Time Spark: From Interactive Queries to StreamingReal-Time Spark: From Interactive Queries to Streaming
Real-Time Spark: From Interactive Queries to Streaming
Databricks
 
What Is Apache Spark? | Introduction To Apache Spark | Apache Spark Tutorial ...
What Is Apache Spark? | Introduction To Apache Spark | Apache Spark Tutorial ...What Is Apache Spark? | Introduction To Apache Spark | Apache Spark Tutorial ...
What Is Apache Spark? | Introduction To Apache Spark | Apache Spark Tutorial ...
Simplilearn
 
Apache Spark vs Apache Flink
Apache Spark vs Apache FlinkApache Spark vs Apache Flink
Apache Spark vs Apache Flink
AKASH SIHAG
 
Apache Calcite overview
Apache Calcite overviewApache Calcite overview
Apache Calcite overview
Julian Hyde
 
Introduction To Flink
Introduction To FlinkIntroduction To Flink
Introduction To Flink
Knoldus Inc.
 
Dask: Scaling Python
Dask: Scaling PythonDask: Scaling Python
Dask: Scaling Python
Matthew Rocklin
 
Flink Streaming
Flink StreamingFlink Streaming
Flink Streaming
Gyula Fóra
 
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
Till Rohrmann
 
Apache Spark GraphX & GraphFrame Synthetic ID Fraud Use Case
Apache Spark GraphX & GraphFrame Synthetic ID Fraud Use CaseApache Spark GraphX & GraphFrame Synthetic ID Fraud Use Case
Apache Spark GraphX & GraphFrame Synthetic ID Fraud Use Case
Mo Patel
 
Reactive Stream Processing with Akka Streams
Reactive Stream Processing with Akka StreamsReactive Stream Processing with Akka Streams
Reactive Stream Processing with Akka StreamsKonrad Malawski
 
Substrait Overview.pdf
Substrait Overview.pdfSubstrait Overview.pdf
Substrait Overview.pdf
Rinat Abdullin
 
Strata sf - Amundsen presentation
Strata sf - Amundsen presentationStrata sf - Amundsen presentation
Strata sf - Amundsen presentation
Tao Feng
 
Apache Spark overview
Apache Spark overviewApache Spark overview
Apache Spark overview
DataArt
 
Spark
SparkSpark
Apache Flink Training: System Overview
Apache Flink Training: System OverviewApache Flink Training: System Overview
Apache Flink Training: System Overview
Flink Forward
 

What's hot (20)

Morel, a Functional Query Language
Morel, a Functional Query LanguageMorel, a Functional Query Language
Morel, a Functional Query Language
 
Reactive Streams: Handling Data-Flow the Reactive Way
Reactive Streams: Handling Data-Flow the Reactive WayReactive Streams: Handling Data-Flow the Reactive Way
Reactive Streams: Handling Data-Flow the Reactive Way
 
Apache Beam and Google Cloud Dataflow - IDG - final
Apache Beam and Google Cloud Dataflow - IDG - finalApache Beam and Google Cloud Dataflow - IDG - final
Apache Beam and Google Cloud Dataflow - IDG - final
 
Ray: Enterprise-Grade, Distributed Python
Ray: Enterprise-Grade, Distributed PythonRay: Enterprise-Grade, Distributed Python
Ray: Enterprise-Grade, Distributed Python
 
Real-Time Spark: From Interactive Queries to Streaming
Real-Time Spark: From Interactive Queries to StreamingReal-Time Spark: From Interactive Queries to Streaming
Real-Time Spark: From Interactive Queries to Streaming
 
What Is Apache Spark? | Introduction To Apache Spark | Apache Spark Tutorial ...
What Is Apache Spark? | Introduction To Apache Spark | Apache Spark Tutorial ...What Is Apache Spark? | Introduction To Apache Spark | Apache Spark Tutorial ...
What Is Apache Spark? | Introduction To Apache Spark | Apache Spark Tutorial ...
 
Apache Spark vs Apache Flink
Apache Spark vs Apache FlinkApache Spark vs Apache Flink
Apache Spark vs Apache Flink
 
Apache Calcite overview
Apache Calcite overviewApache Calcite overview
Apache Calcite overview
 
Introduction To Flink
Introduction To FlinkIntroduction To Flink
Introduction To Flink
 
Dask: Scaling Python
Dask: Scaling PythonDask: Scaling Python
Dask: Scaling Python
 
Flink Streaming
Flink StreamingFlink Streaming
Flink Streaming
 
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
 
Apache Spark GraphX & GraphFrame Synthetic ID Fraud Use Case
Apache Spark GraphX & GraphFrame Synthetic ID Fraud Use CaseApache Spark GraphX & GraphFrame Synthetic ID Fraud Use Case
Apache Spark GraphX & GraphFrame Synthetic ID Fraud Use Case
 
Reactive Stream Processing with Akka Streams
Reactive Stream Processing with Akka StreamsReactive Stream Processing with Akka Streams
Reactive Stream Processing with Akka Streams
 
Substrait Overview.pdf
Substrait Overview.pdfSubstrait Overview.pdf
Substrait Overview.pdf
 
Strata sf - Amundsen presentation
Strata sf - Amundsen presentationStrata sf - Amundsen presentation
Strata sf - Amundsen presentation
 
Apache Spark overview
Apache Spark overviewApache Spark overview
Apache Spark overview
 
Spark
SparkSpark
Spark
 
Apache Flink Training: System Overview
Apache Flink Training: System OverviewApache Flink Training: System Overview
Apache Flink Training: System Overview
 
Builder pattern
Builder patternBuilder pattern
Builder pattern
 

Viewers also liked

Gradoop: Scalable Graph Analytics with Apache Flink @ Flink & Neo4j Meetup Be...
Gradoop: Scalable Graph Analytics with Apache Flink @ Flink & Neo4j Meetup Be...Gradoop: Scalable Graph Analytics with Apache Flink @ Flink & Neo4j Meetup Be...
Gradoop: Scalable Graph Analytics with Apache Flink @ Flink & Neo4j Meetup Be...
Martin Junghanns
 
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
Vasia Kalavri
 
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
Vasia Kalavri
 
Large-scale graph processing with Apache Flink @GraphDevroom FOSDEM'15
Large-scale graph processing with Apache Flink @GraphDevroom FOSDEM'15Large-scale graph processing with Apache Flink @GraphDevroom FOSDEM'15
Large-scale graph processing with Apache Flink @GraphDevroom FOSDEM'15
Vasia Kalavri
 
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
Vasia Kalavri
 
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
Vasia Kalavri
 
Overview of Apache Flink: Next-Gen Big Data Analytics Framework
Overview of Apache Flink: Next-Gen Big Data Analytics FrameworkOverview of Apache Flink: Next-Gen Big Data Analytics Framework
Overview of Apache Flink: Next-Gen Big Data Analytics Framework
Slim Baltagi
 
Undirected graphs
Undirected graphsUndirected graphs
Undirected graphs
Nick Lee
 
Trade-offs in Processing Large Graphs: Representations, Storage, Systems and ...
Trade-offs in Processing Large Graphs: Representations, Storage, Systems and ...Trade-offs in Processing Large Graphs: Representations, Storage, Systems and ...
Trade-offs in Processing Large Graphs: Representations, Storage, Systems and ...
Deepak Ajwani
 
Stream Processing with Apache Flink
Stream Processing with Apache FlinkStream Processing with Apache Flink
Stream Processing with Apache Flink
C4Media
 
Distributed Graph Analytics with Gradoop
Distributed Graph Analytics with GradoopDistributed Graph Analytics with Gradoop
Distributed Graph Analytics with Gradoop
Martin Junghanns
 
HadoopCon'16, Taipei @myui
HadoopCon'16, Taipei @myuiHadoopCon'16, Taipei @myui
HadoopCon'16, Taipei @myui
Makoto Yui
 
Computing recommendations at extreme scale with Apache Flink @Buzzwords 2015
Computing recommendations at extreme scale with Apache Flink @Buzzwords 2015Computing recommendations at extreme scale with Apache Flink @Buzzwords 2015
Computing recommendations at extreme scale with Apache Flink @Buzzwords 2015
Till Rohrmann
 
Data Analysis With Apache Flink
Data Analysis With Apache FlinkData Analysis With Apache Flink
Data Analysis With Apache Flink
DataWorks Summit
 
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
Vasia Kalavri
 
Big data processing systems research
Big data processing systems researchBig data processing systems research
Big data processing systems research
Vasia Kalavri
 
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
Vasia Kalavri
 
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
Vasia Kalavri
 
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
Vasia Kalavri
 
Graphs in data structure
Graphs in data structureGraphs in data structure
Graphs in data structure
hamza javed
 

Viewers also liked (20)

Gradoop: Scalable Graph Analytics with Apache Flink @ Flink & Neo4j Meetup Be...
Gradoop: Scalable Graph Analytics with Apache Flink @ Flink & Neo4j Meetup Be...Gradoop: Scalable Graph Analytics with Apache Flink @ Flink & Neo4j Meetup Be...
Gradoop: Scalable Graph Analytics with Apache Flink @ Flink & Neo4j Meetup Be...
 
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
 
Large-scale graph processing with Apache Flink @GraphDevroom FOSDEM'15
Large-scale graph processing with Apache Flink @GraphDevroom FOSDEM'15Large-scale graph processing with Apache Flink @GraphDevroom FOSDEM'15
Large-scale graph processing with Apache Flink @GraphDevroom FOSDEM'15
 
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
 
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
 
Overview of Apache Flink: Next-Gen Big Data Analytics Framework
Overview of Apache Flink: Next-Gen Big Data Analytics FrameworkOverview of Apache Flink: Next-Gen Big Data Analytics Framework
Overview of Apache Flink: Next-Gen Big Data Analytics Framework
 
Undirected graphs
Undirected graphsUndirected graphs
Undirected graphs
 
Trade-offs in Processing Large Graphs: Representations, Storage, Systems and ...
Trade-offs in Processing Large Graphs: Representations, Storage, Systems and ...Trade-offs in Processing Large Graphs: Representations, Storage, Systems and ...
Trade-offs in Processing Large Graphs: Representations, Storage, Systems and ...
 
Stream Processing with Apache Flink
Stream Processing with Apache FlinkStream Processing with Apache Flink
Stream Processing with Apache Flink
 
Distributed Graph Analytics with Gradoop
Distributed Graph Analytics with GradoopDistributed Graph Analytics with Gradoop
Distributed Graph Analytics with Gradoop
 
HadoopCon'16, Taipei @myui
HadoopCon'16, Taipei @myuiHadoopCon'16, Taipei @myui
HadoopCon'16, Taipei @myui
 
Computing recommendations at extreme scale with Apache Flink @Buzzwords 2015
Computing recommendations at extreme scale with Apache Flink @Buzzwords 2015Computing recommendations at extreme scale with Apache Flink @Buzzwords 2015
Computing recommendations at extreme scale with Apache Flink @Buzzwords 2015
 
Data Analysis With Apache Flink
Data Analysis With Apache FlinkData Analysis With Apache Flink
Data Analysis With Apache Flink
 
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
 
Big data processing systems research
Big data processing systems researchBig data processing systems research
Big data processing systems research
 
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
 
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
 
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
 
Graphs in data structure
Graphs in data structureGraphs in data structure
Graphs in data structure
 

Similar to Apache Flink & Graph Processing

Apache Flink Deep Dive
Apache Flink Deep DiveApache Flink Deep Dive
Apache Flink Deep Dive
DataWorks Summit
 
Introduction into scalable graph analysis with Apache Giraph and Spark GraphX
Introduction into scalable graph analysis with Apache Giraph and Spark GraphXIntroduction into scalable graph analysis with Apache Giraph and Spark GraphX
Introduction into scalable graph analysis with Apache Giraph and Spark GraphX
rhatr
 
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
ArangoDB Database
 
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...
NoSQLmatters
 
Productionizing your Streaming Jobs
Productionizing your Streaming JobsProductionizing your Streaming Jobs
Productionizing your Streaming Jobs
Databricks
 
Greg Hogan – To Petascale and Beyond- Apache Flink in the Clouds
Greg Hogan – To Petascale and Beyond- Apache Flink in the CloudsGreg Hogan – To Petascale and Beyond- Apache Flink in the Clouds
Greg Hogan – To Petascale and Beyond- Apache Flink in the Clouds
Flink Forward
 
Hadoop trainingin bangalore
Hadoop trainingin bangaloreHadoop trainingin bangalore
Hadoop trainingin bangalore
appaji intelhunt
 
Big Data Day LA 2016/ Hadoop/ Spark/ Kafka track - Iterative Spark Developmen...
Big Data Day LA 2016/ Hadoop/ Spark/ Kafka track - Iterative Spark Developmen...Big Data Day LA 2016/ Hadoop/ Spark/ Kafka track - Iterative Spark Developmen...
Big Data Day LA 2016/ Hadoop/ Spark/ Kafka track - Iterative Spark Developmen...
Data Con LA
 
Big Data Scala by the Bay: Interactive Spark in your Browser
Big Data Scala by the Bay: Interactive Spark in your BrowserBig Data Scala by the Bay: Interactive Spark in your Browser
Big Data Scala by the Bay: Interactive Spark in your Browser
gethue
 
Three Functional Programming Technologies for Big Data
Three Functional Programming Technologies for Big DataThree Functional Programming Technologies for Big Data
Three Functional Programming Technologies for Big Data
Dynamical Software, Inc.
 
Apache Flink internals
Apache Flink internalsApache Flink internals
Apache Flink internals
Kostas Tzoumas
 
So you think you can stream.pptx
So you think you can stream.pptxSo you think you can stream.pptx
So you think you can stream.pptx
Prakash Chockalingam
 
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...
Big Data Spain
 
Apache Flink Deep-Dive @ Hadoop Summit 2015 in San Jose, CA
Apache Flink Deep-Dive @ Hadoop Summit 2015 in San Jose, CAApache Flink Deep-Dive @ Hadoop Summit 2015 in San Jose, CA
Apache Flink Deep-Dive @ Hadoop Summit 2015 in San Jose, CA
Robert Metzger
 
Meetup tensorframes
Meetup tensorframesMeetup tensorframes
Meetup tensorframes
Paolo Platter
 
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)
Stephan Ewen
 
Natural Language Processing with CNTK and Apache Spark with Ali Zaidi
Natural Language Processing with CNTK and Apache Spark with Ali ZaidiNatural Language Processing with CNTK and Apache Spark with Ali Zaidi
Natural Language Processing with CNTK and Apache Spark with Ali Zaidi
Databricks
 
Vasia Kalavri – Training: Gelly School
Vasia Kalavri – Training: Gelly School Vasia Kalavri – Training: Gelly School
Vasia Kalavri – Training: Gelly School
Flink Forward
 
Large volume data analysis on the Typesafe Reactive Platform
Large volume data analysis on the Typesafe Reactive PlatformLarge volume data analysis on the Typesafe Reactive Platform
Large volume data analysis on the Typesafe Reactive Platform
Martin Zapletal
 

Similar to Apache Flink & Graph Processing (20)

Apache Flink Deep Dive
Apache Flink Deep DiveApache Flink Deep Dive
Apache Flink Deep Dive
 
Introduction into scalable graph analysis with Apache Giraph and Spark GraphX
Introduction into scalable graph analysis with Apache Giraph and Spark GraphXIntroduction into scalable graph analysis with Apache Giraph and Spark GraphX
Introduction into scalable graph analysis with Apache Giraph and Spark GraphX
 
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...
 
Productionizing your Streaming Jobs
Productionizing your Streaming JobsProductionizing your Streaming Jobs
Productionizing your Streaming Jobs
 
Flink internals web
Flink internals web Flink internals web
Flink internals web
 
Greg Hogan – To Petascale and Beyond- Apache Flink in the Clouds
Greg Hogan – To Petascale and Beyond- Apache Flink in the CloudsGreg Hogan – To Petascale and Beyond- Apache Flink in the Clouds
Greg Hogan – To Petascale and Beyond- Apache Flink in the Clouds
 
Hadoop trainingin bangalore
Hadoop trainingin bangaloreHadoop trainingin bangalore
Hadoop trainingin bangalore
 
Big Data Day LA 2016/ Hadoop/ Spark/ Kafka track - Iterative Spark Developmen...
Big Data Day LA 2016/ Hadoop/ Spark/ Kafka track - Iterative Spark Developmen...Big Data Day LA 2016/ Hadoop/ Spark/ Kafka track - Iterative Spark Developmen...
Big Data Day LA 2016/ Hadoop/ Spark/ Kafka track - Iterative Spark Developmen...
 
Big Data Scala by the Bay: Interactive Spark in your Browser
Big Data Scala by the Bay: Interactive Spark in your BrowserBig Data Scala by the Bay: Interactive Spark in your Browser
Big Data Scala by the Bay: Interactive Spark in your Browser
 
Three Functional Programming Technologies for Big Data
Three Functional Programming Technologies for Big DataThree Functional Programming Technologies for Big Data
Three Functional Programming Technologies for Big Data
 
Apache Flink internals
Apache Flink internalsApache Flink internals
Apache Flink internals
 
So you think you can stream.pptx
So you think you can stream.pptxSo you think you can stream.pptx
So you think you can stream.pptx
 
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...
 
Apache Flink Deep-Dive @ Hadoop Summit 2015 in San Jose, CA
Apache Flink Deep-Dive @ Hadoop Summit 2015 in San Jose, CAApache Flink Deep-Dive @ Hadoop Summit 2015 in San Jose, CA
Apache Flink Deep-Dive @ Hadoop Summit 2015 in San Jose, CA
 
Meetup tensorframes
Meetup tensorframesMeetup tensorframes
Meetup tensorframes
 
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)
 
Natural Language Processing with CNTK and Apache Spark with Ali Zaidi
Natural Language Processing with CNTK and Apache Spark with Ali ZaidiNatural Language Processing with CNTK and Apache Spark with Ali Zaidi
Natural Language Processing with CNTK and Apache Spark with Ali Zaidi
 
Vasia Kalavri – Training: Gelly School
Vasia Kalavri – Training: Gelly School Vasia Kalavri – Training: Gelly School
Vasia Kalavri – Training: Gelly School
 
Large volume data analysis on the Typesafe Reactive Platform
Large volume data analysis on the Typesafe Reactive PlatformLarge volume data analysis on the Typesafe Reactive Platform
Large volume data analysis on the Typesafe Reactive Platform
 

More from Vasia Kalavri

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
Vasia Kalavri
 
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
Vasia Kalavri
 
Predictive Datacenter Analytics with Strymon
Predictive Datacenter Analytics with StrymonPredictive Datacenter Analytics with Strymon
Predictive Datacenter Analytics with Strymon
Vasia Kalavri
 
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...
Vasia Kalavri
 
Demystifying Distributed Graph Processing
Demystifying Distributed Graph ProcessingDemystifying Distributed Graph Processing
Demystifying Distributed Graph Processing
Vasia Kalavri
 
MapReduce: Optimizations, Limitations, and Open Issues
MapReduce: Optimizations, Limitations, and Open IssuesMapReduce: Optimizations, Limitations, and Open Issues
MapReduce: Optimizations, Limitations, and Open Issues
Vasia Kalavri
 
A Skype case study (2011)
A Skype case study (2011)A Skype case study (2011)
A Skype case study (2011)
Vasia Kalavri
 
Apache Flink Deep Dive
Apache Flink Deep DiveApache Flink Deep Dive
Apache Flink Deep Dive
Vasia Kalavri
 

More from Vasia Kalavri (8)

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...
 
Demystifying Distributed Graph Processing
Demystifying Distributed Graph ProcessingDemystifying Distributed Graph Processing
Demystifying Distributed Graph Processing
 
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)
 
Apache Flink Deep Dive
Apache Flink Deep DiveApache Flink Deep Dive
Apache Flink Deep Dive
 

Recently uploaded

standardisation of garbhpala offhgfffghh
standardisation of garbhpala offhgfffghhstandardisation of garbhpala offhgfffghh
standardisation of garbhpala offhgfffghh
ArpitMalhotra16
 
一比一原版(Bradford毕业证书)布拉德福德大学毕业证如何办理
一比一原版(Bradford毕业证书)布拉德福德大学毕业证如何办理一比一原版(Bradford毕业证书)布拉德福德大学毕业证如何办理
一比一原版(Bradford毕业证书)布拉德福德大学毕业证如何办理
mbawufebxi
 
一比一原版(CBU毕业证)不列颠海角大学毕业证成绩单
一比一原版(CBU毕业证)不列颠海角大学毕业证成绩单一比一原版(CBU毕业证)不列颠海角大学毕业证成绩单
一比一原版(CBU毕业证)不列颠海角大学毕业证成绩单
nscud
 
原版制作(Deakin毕业证书)迪肯大学毕业证学位证一模一样
原版制作(Deakin毕业证书)迪肯大学毕业证学位证一模一样原版制作(Deakin毕业证书)迪肯大学毕业证学位证一模一样
原版制作(Deakin毕业证书)迪肯大学毕业证学位证一模一样
u86oixdj
 
Predicting Product Ad Campaign Performance: A Data Analysis Project Presentation
Predicting Product Ad Campaign Performance: A Data Analysis Project PresentationPredicting Product Ad Campaign Performance: A Data Analysis Project Presentation
Predicting Product Ad Campaign Performance: A Data Analysis Project Presentation
Boston Institute of Analytics
 
Empowering Data Analytics Ecosystem.pptx
Empowering Data Analytics Ecosystem.pptxEmpowering Data Analytics Ecosystem.pptx
Empowering Data Analytics Ecosystem.pptx
benishzehra469
 
The affect of service quality and online reviews on customer loyalty in the E...
The affect of service quality and online reviews on customer loyalty in the E...The affect of service quality and online reviews on customer loyalty in the E...
The affect of service quality and online reviews on customer loyalty in the E...
jerlynmaetalle
 
Malana- Gimlet Market Analysis (Portfolio 2)
Malana- Gimlet Market Analysis (Portfolio 2)Malana- Gimlet Market Analysis (Portfolio 2)
Malana- Gimlet Market Analysis (Portfolio 2)
TravisMalana
 
一比一原版(CBU毕业证)卡普顿大学毕业证成绩单
一比一原版(CBU毕业证)卡普顿大学毕业证成绩单一比一原版(CBU毕业证)卡普顿大学毕业证成绩单
一比一原版(CBU毕业证)卡普顿大学毕业证成绩单
nscud
 
FP Growth Algorithm and its Applications
FP Growth Algorithm and its ApplicationsFP Growth Algorithm and its Applications
FP Growth Algorithm and its Applications
MaleehaSheikh2
 
做(mqu毕业证书)麦考瑞大学毕业证硕士文凭证书学费发票原版一模一样
做(mqu毕业证书)麦考瑞大学毕业证硕士文凭证书学费发票原版一模一样做(mqu毕业证书)麦考瑞大学毕业证硕士文凭证书学费发票原版一模一样
做(mqu毕业证书)麦考瑞大学毕业证硕士文凭证书学费发票原版一模一样
axoqas
 
一比一原版(RUG毕业证)格罗宁根大学毕业证成绩单
一比一原版(RUG毕业证)格罗宁根大学毕业证成绩单一比一原版(RUG毕业证)格罗宁根大学毕业证成绩单
一比一原版(RUG毕业证)格罗宁根大学毕业证成绩单
vcaxypu
 
Adjusting primitives for graph : SHORT REPORT / NOTES
Adjusting primitives for graph : SHORT REPORT / NOTESAdjusting primitives for graph : SHORT REPORT / NOTES
Adjusting primitives for graph : SHORT REPORT / NOTES
Subhajit Sahu
 
一比一原版(Adelaide毕业证书)阿德莱德大学毕业证如何办理
一比一原版(Adelaide毕业证书)阿德莱德大学毕业证如何办理一比一原版(Adelaide毕业证书)阿德莱德大学毕业证如何办理
一比一原版(Adelaide毕业证书)阿德莱德大学毕业证如何办理
slg6lamcq
 
一比一原版(UPenn毕业证)宾夕法尼亚大学毕业证成绩单
一比一原版(UPenn毕业证)宾夕法尼亚大学毕业证成绩单一比一原版(UPenn毕业证)宾夕法尼亚大学毕业证成绩单
一比一原版(UPenn毕业证)宾夕法尼亚大学毕业证成绩单
ewymefz
 
一比一原版(UVic毕业证)维多利亚大学毕业证成绩单
一比一原版(UVic毕业证)维多利亚大学毕业证成绩单一比一原版(UVic毕业证)维多利亚大学毕业证成绩单
一比一原版(UVic毕业证)维多利亚大学毕业证成绩单
ukgaet
 
一比一原版(BU毕业证)波士顿大学毕业证成绩单
一比一原版(BU毕业证)波士顿大学毕业证成绩单一比一原版(BU毕业证)波士顿大学毕业证成绩单
一比一原版(BU毕业证)波士顿大学毕业证成绩单
ewymefz
 
一比一原版(UofM毕业证)明尼苏达大学毕业证成绩单
一比一原版(UofM毕业证)明尼苏达大学毕业证成绩单一比一原版(UofM毕业证)明尼苏达大学毕业证成绩单
一比一原版(UofM毕业证)明尼苏达大学毕业证成绩单
ewymefz
 
1.Seydhcuxhxyxhccuuxuxyxyxmisolids 2019.pptx
1.Seydhcuxhxyxhccuuxuxyxyxmisolids 2019.pptx1.Seydhcuxhxyxhccuuxuxyxyxmisolids 2019.pptx
1.Seydhcuxhxyxhccuuxuxyxyxmisolids 2019.pptx
Tiktokethiodaily
 
Opendatabay - Open Data Marketplace.pptx
Opendatabay - Open Data Marketplace.pptxOpendatabay - Open Data Marketplace.pptx
Opendatabay - Open Data Marketplace.pptx
Opendatabay
 

Recently uploaded (20)

standardisation of garbhpala offhgfffghh
standardisation of garbhpala offhgfffghhstandardisation of garbhpala offhgfffghh
standardisation of garbhpala offhgfffghh
 
一比一原版(Bradford毕业证书)布拉德福德大学毕业证如何办理
一比一原版(Bradford毕业证书)布拉德福德大学毕业证如何办理一比一原版(Bradford毕业证书)布拉德福德大学毕业证如何办理
一比一原版(Bradford毕业证书)布拉德福德大学毕业证如何办理
 
一比一原版(CBU毕业证)不列颠海角大学毕业证成绩单
一比一原版(CBU毕业证)不列颠海角大学毕业证成绩单一比一原版(CBU毕业证)不列颠海角大学毕业证成绩单
一比一原版(CBU毕业证)不列颠海角大学毕业证成绩单
 
原版制作(Deakin毕业证书)迪肯大学毕业证学位证一模一样
原版制作(Deakin毕业证书)迪肯大学毕业证学位证一模一样原版制作(Deakin毕业证书)迪肯大学毕业证学位证一模一样
原版制作(Deakin毕业证书)迪肯大学毕业证学位证一模一样
 
Predicting Product Ad Campaign Performance: A Data Analysis Project Presentation
Predicting Product Ad Campaign Performance: A Data Analysis Project PresentationPredicting Product Ad Campaign Performance: A Data Analysis Project Presentation
Predicting Product Ad Campaign Performance: A Data Analysis Project Presentation
 
Empowering Data Analytics Ecosystem.pptx
Empowering Data Analytics Ecosystem.pptxEmpowering Data Analytics Ecosystem.pptx
Empowering Data Analytics Ecosystem.pptx
 
The affect of service quality and online reviews on customer loyalty in the E...
The affect of service quality and online reviews on customer loyalty in the E...The affect of service quality and online reviews on customer loyalty in the E...
The affect of service quality and online reviews on customer loyalty in the E...
 
Malana- Gimlet Market Analysis (Portfolio 2)
Malana- Gimlet Market Analysis (Portfolio 2)Malana- Gimlet Market Analysis (Portfolio 2)
Malana- Gimlet Market Analysis (Portfolio 2)
 
一比一原版(CBU毕业证)卡普顿大学毕业证成绩单
一比一原版(CBU毕业证)卡普顿大学毕业证成绩单一比一原版(CBU毕业证)卡普顿大学毕业证成绩单
一比一原版(CBU毕业证)卡普顿大学毕业证成绩单
 
FP Growth Algorithm and its Applications
FP Growth Algorithm and its ApplicationsFP Growth Algorithm and its Applications
FP Growth Algorithm and its Applications
 
做(mqu毕业证书)麦考瑞大学毕业证硕士文凭证书学费发票原版一模一样
做(mqu毕业证书)麦考瑞大学毕业证硕士文凭证书学费发票原版一模一样做(mqu毕业证书)麦考瑞大学毕业证硕士文凭证书学费发票原版一模一样
做(mqu毕业证书)麦考瑞大学毕业证硕士文凭证书学费发票原版一模一样
 
一比一原版(RUG毕业证)格罗宁根大学毕业证成绩单
一比一原版(RUG毕业证)格罗宁根大学毕业证成绩单一比一原版(RUG毕业证)格罗宁根大学毕业证成绩单
一比一原版(RUG毕业证)格罗宁根大学毕业证成绩单
 
Adjusting primitives for graph : SHORT REPORT / NOTES
Adjusting primitives for graph : SHORT REPORT / NOTESAdjusting primitives for graph : SHORT REPORT / NOTES
Adjusting primitives for graph : SHORT REPORT / NOTES
 
一比一原版(Adelaide毕业证书)阿德莱德大学毕业证如何办理
一比一原版(Adelaide毕业证书)阿德莱德大学毕业证如何办理一比一原版(Adelaide毕业证书)阿德莱德大学毕业证如何办理
一比一原版(Adelaide毕业证书)阿德莱德大学毕业证如何办理
 
一比一原版(UPenn毕业证)宾夕法尼亚大学毕业证成绩单
一比一原版(UPenn毕业证)宾夕法尼亚大学毕业证成绩单一比一原版(UPenn毕业证)宾夕法尼亚大学毕业证成绩单
一比一原版(UPenn毕业证)宾夕法尼亚大学毕业证成绩单
 
一比一原版(UVic毕业证)维多利亚大学毕业证成绩单
一比一原版(UVic毕业证)维多利亚大学毕业证成绩单一比一原版(UVic毕业证)维多利亚大学毕业证成绩单
一比一原版(UVic毕业证)维多利亚大学毕业证成绩单
 
一比一原版(BU毕业证)波士顿大学毕业证成绩单
一比一原版(BU毕业证)波士顿大学毕业证成绩单一比一原版(BU毕业证)波士顿大学毕业证成绩单
一比一原版(BU毕业证)波士顿大学毕业证成绩单
 
一比一原版(UofM毕业证)明尼苏达大学毕业证成绩单
一比一原版(UofM毕业证)明尼苏达大学毕业证成绩单一比一原版(UofM毕业证)明尼苏达大学毕业证成绩单
一比一原版(UofM毕业证)明尼苏达大学毕业证成绩单
 
1.Seydhcuxhxyxhccuuxuxyxyxmisolids 2019.pptx
1.Seydhcuxhxyxhccuuxuxyxyxmisolids 2019.pptx1.Seydhcuxhxyxhccuuxuxyxyxmisolids 2019.pptx
1.Seydhcuxhxyxhccuuxuxyxyxmisolids 2019.pptx
 
Opendatabay - Open Data Marketplace.pptx
Opendatabay - Open Data Marketplace.pptxOpendatabay - Open Data Marketplace.pptx
Opendatabay - Open Data Marketplace.pptx
 

Apache Flink & Graph Processing

  • 1. Batch & Stream Graph Processing with Apache Flink Vasia Kalavri vasia@apache.org @vkalavri Apache Flink Meetup London October 5th, 2016
  • 2. 2 Graphs capture relationships between data items connections, interactions, purchases, dependencies, friendships, etc. Recommenders Social networks Bioinformatics Web search
  • 3. Outline • Distributed Graph Processing 101 • Gelly: Batch Graph Processing with Apache Flink • BREAK! • Gelly-Stream: Continuous Graph Processing with Apache Flink
  • 4. Apache Flink • An open-source, distributed data analysis framework • True streaming at its core • Streaming & Batch API 4 Historic data Kafka, RabbitMQ, ... HDFS, JDBC, ... Event logs ETL, Graphs,
 Machine Learning
 Relational, … Low latency,
 windowing, aggregations, ...
  • 5. WHEN DO YOU NEED DISTRIBUTED GRAPH PROCESSING?
  • 6. MY GRAPH IS SO BIG, IT DOESN’T FIT IN A SINGLE MACHINE Big Data Ninja MISCONCEPTION #1
  • 8. NAIVE WHO(M)-T0-FOLLOW ▸ Naive Who(m) to Follow: ▸ compute a friends-of-friends list per user ▸ exclude existing friends ▸ rank by common connections
  • 9. DON’T JUST CONSIDER YOUR INPUT GRAPH SIZE. INTERMEDIATE DATA MATTERS TOO!
  • 10. DISTRIBUTED PROCESSING IS ALWAYS FASTER THAN SINGLE-NODE Data Science Rockstar MISCONCEPTION #2
  • 11.
  • 12.
  • 13. GRAPHS DON’T APPEAR OUT OF THIN AIR Expectation…
  • 14. GRAPHS DON’T APPEAR OUT OF THIN AIR Reality!
  • 15. WHEN DO YOU NEED DISTRIBUTED GRAPH PROCESSING? ▸ When you do have really big graphs ▸ When the intermediate data is big ▸ When your data is already distributed ▸ When you want to build end-to-end graph pipelines
  • 16. HOW DO WE EXPRESS A DISTRIBUTED GRAPH ANALYSIS TASK?
  • 17. RECENT DISTRIBUTED GRAPH PROCESSING HISTORY 2004 MapReduce Pegasus 2009 Pregel 2010 Signal-Collect PowerGraph 2012 Iterative value propagation Giraph++ 2013 Graph Traversals NScale 2014 Ego-network analysis Arabesque 2015 Pattern Matching Tinkerpop
  • 18. PREGEL: THINK LIKE A VERTEX 1 5 4 3 2 1 3, 4 2 1, 4 5 3 ...
  • 19. PREGEL: SUPERSTEPS (Vi+1, outbox) <— compute(Vi, inbox) 1 3, 4 2 1, 4 5 3 .. 1 3, 4 2 1, 4 5 3 .. Superstep i Superstep i+1
  • 20. PAGERANK: THE WORD COUNT OF GRAPH PROCESSING VertexID Out-degree Transition Probability 1 2 1/2 2 2 1/2 3 0 - 4 3 1/3 5 1 1 1 5 4 3 2
  • 21. PAGERANK: THE WORD COUNT OF GRAPH PROCESSING VertexID Out-degree Transition Probability 1 2 1/2 2 2 1/2 3 0 - 4 3 1/3 5 1 1 PR(3) = 0.5*PR(1) + 0.33*PR(4) + PR(5) 1 5 4 3 2
  • 22. 1 5 4 3 2 PAGERANK: THE WORD COUNT OF GRAPH PROCESSING VertexID Out-degree Transition Probability 1 2 1/2 2 2 1/2 3 0 - 4 3 1/3 5 1 1 PR(3) = 0.5*PR(1) + 0.33*PR(4) + PR(5)
  • 23. PAGERANK: THE WORD COUNT OF GRAPH PROCESSING VertexID Out-degree Transition Probability 1 2 1/2 2 2 1/2 3 0 - 4 3 1/3 5 1 1 PR(3) = 0.5*PR(1) + 0.33*PR(4) + PR(5) 1 5 4 3 2
  • 24. PAGERANK: THE WORD COUNT OF GRAPH PROCESSING VertexID Out-degree Transition Probability 1 2 1/2 2 2 1/2 3 0 - 4 3 1/3 5 1 1 PR(3) = 0.5*PR(1) + 0.33*PR(4) + PR(5) 1 5 4 3 2
  • 25. PREGEL EXAMPLE: PAGERANK void compute(messages): sum = 0.0 for (m <- messages) do sum = sum + m end for setValue(0.15/numVertices() + 0.85*sum) for (edge <- getOutEdges()) do sendMessageTo( edge.target(), getValue()/numEdges) end for sum up received messages update vertex rank distribute rank to neighbors
  • 26. SIGNAL-COLLECT outbox <— signal(Vi) 1 3, 4 2 1, 4 5 3 .. 1 3, 4 2 1, 4 5 3 .. Superstep i Vi+1 <— collect(inbox) 1 3, 4 2 1, 4 5 3 .. Signal Collect Superstep i+1
  • 27. SIGNAL-COLLECT EXAMPLE: PAGERANK void signal(): for (edge <- getOutEdges()) do sendMessageTo( edge.target(), getValue()/numEdges) end for void collect(messages): sum = 0.0 for (m <- messages) do sum = sum + m end for setValue(0.15/numVertices() + 0.85*sum) distribute rank to neighbors sum up messages update vertex rank
  • 28. GATHER-SUM-APPLY (POWERGRAPH) 1 ... ... Gather Sum 1 2 5 ... Apply 3 1 5 5 3 1 ... Gather 3 1 5 5 3 Superstep i Superstep i+1
  • 29. GSA EXAMPLE: PAGERANK double gather(source, edge, target): return target.value() / target.numEdges() double sum(rank1, rank2): return rank1 + rank2 double apply(sum, currentRank): return 0.15 + 0.85*sum compute partial rank combine partial ranks update rank
  • 30. PREGEL VS. SIGNAL-COLLECT VS. GSA Update Function Properties Update Function Logic Communication Scope Communication Logic Pregel arbitrary arbitrary any vertex arbitrary Signal-Collect arbitrary based on received messages any vertex based on vertex state GSA associative & commutative based on neighbors’ values neighborhood based on vertex state
  • 31. CAN WE HAVE IT ALL? ▸ Data pipeline integration: built on top of an efficient distributed processing engine ▸ Graph ETL: high-level API with abstractions and methods to transform graphs ▸ Familiar programming model: support popular programming abstractions
  • 33. Apache Flink Stack Gelly Table/SQL ML SAMOA DataSet (Java/Scala) DataStream (Java/Scala) HadoopM/R Local Remote Yarn Embedded Dataflow Dataflow Table/SQL Cascading Streaming dataflow runtime CEP
  • 34. Meet Gelly • Java & Scala Graph APIs on top of Flink’s DataSet API Flink Core Scala API (batch and streaming) Java API (batch and streaming) FlinkML GellyTable API ... Transformations and Utilities Iterative Graph Processing Graph Library 34
  • 35. Gelly is NOT • a graph database • a specialized graph processor 35
  • 36. Hello, Gelly! ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); DataSet<Edge<Long, NullValue>> edges = getEdgesDataSet(env); Graph<Long, Long, NullValue> graph = Graph.fromDataSet(edges, env); DataSet<Vertex<Long, Long>> verticesWithMinIds = graph.run( new ConnectedComponents(maxIterations)); val env = ExecutionEnvironment.getExecutionEnvironment val edges: DataSet[Edge[Long, NullValue]] = getEdgesDataSet(env) val graph = Graph.fromDataSet(edges, env) val components = graph.run(new ConnectedComponents(maxIterations)) Java Scala
  • 37. Graph Methods Graph Properties getVertexIds getEdgeIds numberOfVertices numberOfEdges getDegrees Mutations add vertex/edge remove vertex/edge Transformations map, filter, join subgraph, union, difference reverse, undirected getTriplets Generators R-Mat (power-law) Grid Star Complete …
  • 38. Example: mapVertices // increment each vertex value by one
 val graph = Graph.fromDataSet(...)
 
 // increment each vertex value by one
 val updatedGraph = graph.mapVertices(v => v.getValue + 1) 4 2 8 5 5 3 1 7 4 5
  • 39. Example: subGraph val graph: Graph[Long, Long, Long] = ...
 
 // keep only vertices with positive values
 // and only edges with negative values
 val subGraph = graph.subgraph( vertex => vertex.getValue > 0, edge => edge.getValue < 0 )
  • 40. Neighborhood Methods Apply a reduce function to the 1st-hop neighborhood of each vertex in parallel graph.reduceOnNeighbors( new MinValue, EdgeDirection.OUT)
  • 41. What makes Gelly unique? • Batch graph processing on top of a streaming dataflow engine • Built for end-to-end analytics • Support for multiple iteration abstractions • Graph algorithm building blocks • A large open-source library of graph algorithms
  • 42. Why streaming dataflow? • Batch engines materialize data… even if they don’t have to • the graph is always loaded and materialized in memory, even if not needed, e.g. mapping, filtering, transformation • Communication and computation overlap • We can do continuous graph processing (more after the break!)
  • 43. End-to-end analytics • Graphs don’t appear out of thin air… • We need to support pre- and post-processing • Gelly can be easily mixed with the DataSet API: pre-processing, graph analysis, and post- processing in the same Flink program
  • 44. Iterative Graph Processing • Gelly offers iterative graph processing abstractions on top of Flink’s Delta iterations • vertex-centric • scatter-gather • gather-sum-apply • partition-centric*
  • 45. Flink Iteration Operators Input Iterative Update Function Result Replace Workset Iterative Update Function Result Solution Set State
  • 46. Optimization • the runtime is aware of the iterative execution • no scheduling overhead between iterations • caching and state maintenance are handled automatically Push work
 “out of the loop” Maintain state as indexCache Loop-invariant Data
  • 47. Vertex-Centric SSSP final class SSSPComputeFunction extends ComputeFunction { override def compute(vertex: Vertex, messages: MessageIterator) = { var minDistance = if (vertex.getId == srcId) 0 else Double.MaxValue while (messages.hasNext) { val msg = messages.next if (msg < minDistance) minDistance = msg } if (vertex.getValue > minDistance) { setNewVertexValue(minDistance) for (edge: Edge <- getEdges) sendMessageTo(edge.getTarget, vertex.getValue + edge.getValue) }
  • 48. Algorithms building blocks • Allow operator re-use across graph algorithms when processing the same input with a similar configuration
  • 49. Library of Algorithms • PageRank • Single Source Shortest Paths • Label Propagation • Weakly Connected Components • Community Detection • Triangle Count & Enumeration • Local and Global Clustering Coefficient • HITS • Jaccard & Adamic-Adar Similarity • Graph Summarization • val ranks = inputGraph.run(new PageRank(0.85, 20))
  • 51. Can’t we block them? proxy Tracker Tracker Ad Server Legitimate site
  • 52. • not frequently updated • not sure who or based on what criteria URLs are blacklisted • miss “hidden” trackers or dual-role nodes • blocking requires manual matching against the list • can you buy your way into the whitelist? Available Solutions Crowd-sourced “black lists” of tracker URLs: - AdBlock, DoNotTrack, EasyPrivacy
  • 53. DataSet • 6 months (Nov 2014 - April 2015) of augmented Apache logs from a web proxy • 80m requests, 2m distinct URLs, 3k users
  • 54. h2 h3 h4 h5 h6 h8 h7 h1 h3 h4 h5 h6 h1 h2 h7 h8 r1 r2 r3 r5 r6 r7 NT NT T T ? T NT NT r4 r1 r2 r3 r3 r3 r4 r5r6 r7 hosts-projection graph : referer : non-tracker host : tracker host : unlabeled host The Hosts-Projection Graph U: Referers referer-hosts graph V: hosts
  • 55. Classification via Label Propagation non-tracker tracker unlabeled 55
  • 56. Data Pipeline raw logs cleaned logs 1: logs pre- processing 2: bipartite graph creation 3: largest connected component extraction 4: hosts- projection graph creation 5: community detection google-analytics.com: T bscored-research.com: T facebook.com: NT github.com: NT cdn.cxense.com: NT ... 6: results DataSet API Gelly DataSet API
  • 57. Feeling Gelly? • Gelly Guide https://ci.apache.org/projects/flink/flink-docs-master/libs/ gelly_guide.html • To Petascale and Beyond @Flink Forward ‘16 http://flink-forward.org/kb_sessions/to-petascale-and-beyond-apache- flink-in-the-clouds/ • Web Tracker Detection @Flink Forward ’15 https://www.youtube.com/watch?v=ZBCXXiDr3TU paper: Kalavri, Vasiliki, et al. "Like a pack of wolves: Community structure of web trackers." International Conference on Passive and Active Network Measurement, 2016.
  • 59. Real Graphs are dynamic Graphs are created from events happening in real-time
  • 60.
  • 61. How we’ve done graph processing so far 1. Load: read the graph from disk and partition it in memory
  • 62. 2. Compute: read and mutate the graph state How we’ve done graph processing so far 1. Load: read the graph from disk and partition it in memory
  • 63. 3. Store: write the final graph state back to disk How we’ve done graph processing so far 2. Compute: read and mutate the graph state 1. Load: read the graph from disk and partition it in memory
  • 64. What’s wrong with this model? • It is slow • wait until the computation is over before you see any result • pre-processing and partitioning • It is expensive • lots of memory and CPU required in order to scale • It requires re-computation for graph changes • no efficient way to deal with updates
  • 65. Can we do graph processing on streams? • Maintain the dynamic graph structure • Provide up-to-date results with low latency • Compute on fresh state only
  • 66. Single-pass graph streaming • Each event is an edge addition • Maintain only a graph summary • Recent events are grouped in graph windows
  • 67.
  • 68. Graph Summaries • spanners for distance estimation • sparsifiers for cut estimation • sketches for homomorphic properties graph summary algorithm algorithm~R1 R2
  • 74. 54 76 86 42 31 52 Stream Connected Components Graph Summary: Disjoint Set (Union-Find) • Only store component IDs and vertex IDs
  • 84. Stream Connected Components with Flink DataStream<DisjointSet> cc = edgeStream
 .keyBy(0)
 .timeWindow(Time.of(100, TimeUnit.MILLISECONDS))
 .fold(new DisjointSet(), new UpdateCC())
 .flatMap(new Merger())
 .setParallelism(1);
  • 85. Stream Connected Components with Flink DataStream<DisjointSet> cc = edgeStream
 .keyBy(0)
 .timeWindow(Time.of(100, TimeUnit.MILLISECONDS))
 .fold(new DisjointSet(), new UpdateCC())
 .flatMap(new Merger())
 .setParallelism(1); Partition the edge stream
  • 86. Stream Connected Components with Flink DataStream<DisjointSet> cc = edgeStream
 .keyBy(0)
 .timeWindow(Time.of(100, TimeUnit.MILLISECONDS))
 .fold(new DisjointSet(), new UpdateCC())
 .flatMap(new Merger())
 .setParallelism(1); Define the merging frequency
  • 87. Stream Connected Components with Flink DataStream<DisjointSet> cc = edgeStream
 .keyBy(0)
 .timeWindow(Time.of(100, TimeUnit.MILLISECONDS))
 .fold(new DisjointSet(), new UpdateCC())
 .flatMap(new Merger())
 .setParallelism(1); merge locally
  • 88. Stream Connected Components with Flink DataStream<DisjointSet> cc = edgeStream
 .keyBy(0)
 .timeWindow(Time.of(100, TimeUnit.MILLISECONDS))
 .fold(new DisjointSet(), new UpdateCC())
 .flatMap(new Merger())
 .setParallelism(1); merge globally
  • 89. Gelly on Streams DataStreamDataSet Distributed Dataflow Deployment Gelly Gelly-Stream • Static Graphs • Multi-Pass Algorithms • Full Computations • Dynamic Graphs • Single-Pass Algorithms • Approximate Computations DataStream
  • 90. Introducing Gelly-Stream Gelly-Stream enriches the DataStream API with two new additional ADTs: • GraphStream: • A representation of a data stream of edges. • Edges can have state (e.g. weights). • Supports property streams, transformations and aggregations. • GraphWindow: • A “time-slice” of a graph stream. • It enables neighborhood aggregations
  • 91. GraphStream Operations .getEdges() .getVertices() .numberOfVertices() .numberOfEdges() .getDegrees() .inDegrees() .outDegrees() GraphStream -> DataStream .mapEdges(); .distinct(); .filterVertices(); .filterEdges(); .reverse(); .undirected(); .union(); GraphStream -> GraphStream Property Streams Transformations
  • 92. Graph Stream Aggregations result aggregate property streamgraph stream (window) fold combine fold reduce local summaries global summary edges agg global aggregates can be persistent or transient graphStream.aggregate( new MyGraphAggregation(window, fold, combine, transform))
  • 93. Slicing Graph Streams graphStream.slice(Time.of(1, MINUTE)); 11:40 11:41 11:42 11:43
  • 94. Aggregating Slices graphStream.slice(Time.of(1, MINUTE), direction) .reduceOnEdges(); .foldNeighbors(); .applyOnNeighbors(); • Slicing collocates edges by vertex information • Neighborhood aggregations on sliced graphs source target Aggregations
  • 95. Finding Matches Nearby graphStream.filterVertices(GraphGeeks()) .slice(Time.of(15, MINUTE), EdgeDirection.IN) .applyOnNeighbors(FindPairs()) slice GraphStream :: graph geek check-ins wendy checked_in soap_bar steve checked_in soap_bar tom checked_in joe’s_grill sandra checked_in soap_bar rafa checked_in joe’s_grill wendy steve sandra soap bar tom rafa joe’s grill FindPairs {wendy, steve} {steve, sandra} {wendy, sandra} {tom, rafa} GraphWindow :: user-place
  • 96. Feeling Gelly? • Gelly Guide https://ci.apache.org/projects/flink/flink-docs-master/libs/ gelly_guide.html • Gelly-Stream Repository https://github.com/vasia/gelly-streaming • Gelly-Stream talk @FOSDEM16 https://fosdem.org/2016/schedule/event/graph_processing_apache_flink/ • Related Papers http://www.citeulike.org/user/vasiakalavri/tag/graph-streaming
  • 97. Batch & Stream Graph Processing with Apache Flink Vasia Kalavri vasia@apache.org @vkalavri Apache Flink Meetup London October 5th, 2016