Download - Apache Flink & Graph Processing

Transcript
Page 1: Apache Flink & Graph Processing

Batch & Stream Graph Processing with Apache Flink

Vasia Kalavri [email protected]

@vkalavri

Apache Flink Meetup London October 5th, 2016

Page 2: Apache Flink & Graph Processing

2

Graphs capture relationships between data items

connections, interactions, purchases, dependencies, friendships, etc.

Recommenders

Social networks

Bioinformatics

Web search

Page 3: Apache Flink & Graph Processing

Outline

• Distributed Graph Processing 101

• Gelly: Batch Graph Processing with Apache Flink

• BREAK!

• Gelly-Stream: Continuous Graph Processing with Apache Flink

Page 4: Apache Flink & Graph Processing

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 logsETL, Graphs,Machine LearningRelational, …

Low latency,windowing, aggregations, ...

Page 5: Apache Flink & Graph Processing

WHEN DO YOU NEED DISTRIBUTED GRAPH PROCESSING?

Page 6: Apache Flink & Graph Processing

MY GRAPH IS SO BIG, IT DOESN’T FIT IN A SINGLE MACHINE

Big Data Ninja

MISCONCEPTION #1

Page 7: Apache Flink & Graph Processing

A SOCIAL NETWORK

Page 8: Apache Flink & Graph Processing

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

Page 9: Apache Flink & Graph Processing

DON’T JUST CONSIDER YOUR INPUT GRAPH SIZE. INTERMEDIATE DATA MATTERS TOO!

Page 10: Apache Flink & Graph Processing

DISTRIBUTED PROCESSING IS ALWAYS FASTER THAN SINGLE-NODE

Data Science Rockstar

MISCONCEPTION #2

Page 11: Apache Flink & Graph Processing
Page 12: Apache Flink & Graph Processing
Page 13: Apache Flink & Graph Processing

GRAPHS DON’T APPEAR OUT OF THIN AIR

Expectation…

Page 14: Apache Flink & Graph Processing

GRAPHS DON’T APPEAR OUT OF THIN AIR

Reality!

Page 15: Apache Flink & Graph Processing

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

Page 16: Apache Flink & Graph Processing

HOW DO WE EXPRESS A DISTRIBUTED GRAPH ANALYSIS TASK?

Page 17: Apache Flink & Graph Processing

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

Page 18: Apache Flink & Graph Processing

PREGEL: THINK LIKE A VERTEX

1

5

4

3

2 1 3, 4

2 1, 4

5 3

. . .

Page 19: Apache Flink & Graph Processing

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

Page 20: Apache Flink & Graph Processing

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

Page 21: Apache Flink & Graph Processing

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

Page 22: Apache Flink & Graph Processing

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)

Page 23: Apache Flink & Graph Processing

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

Page 24: Apache Flink & Graph Processing

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

Page 25: Apache Flink & Graph Processing

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

Page 26: Apache Flink & Graph Processing

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

Page 27: Apache Flink & Graph Processing

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

Page 28: Apache Flink & Graph Processing

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

Page 29: Apache Flink & Graph Processing

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

Page 30: Apache Flink & Graph Processing

PREGEL VS. SIGNAL-COLLECT VS. GSA

Update Function Properties

Update Function Logic

Communication Scope

Communication Logic

Pregel arbitrary arbitrary any vertex arbitrary

Signal-Collect arbitrarybased on received

messagesany vertex based on vertex

state

GSA associative & commutative

based on neighbors’

valuesneighborhood based on vertex

state

Page 31: Apache Flink & Graph Processing

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

Page 32: Apache Flink & Graph Processing

Gelly the Apache Flink Graph API

Page 33: Apache Flink & Graph Processing

Apache Flink Stack

Gel

ly

Tabl

e/SQ

L

ML

SAM

OA

DataSet (Java/Scala) DataStream (Java/Scala)

Had

oop

M/R

Local Remote Yarn Embedded

Dat

aflo

w

Dat

aflo

w

Tabl

e/SQ

L

Cas

cadi

ngStreaming dataflow runtime

CEP

Page 34: Apache Flink & Graph Processing

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

Page 35: Apache Flink & Graph Processing

Gelly is NOT

• a graph database

• a specialized graph processor

35

Page 36: Apache Flink & Graph Processing

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

Page 37: Apache Flink & Graph Processing

Graph MethodsGraph Properties

getVertexIds getEdgeIds numberOfVertices numberOfEdges getDegrees

Mutationsadd vertex/edge remove vertex/edge

Transformationsmap, filter, join subgraph, union, difference reverse, undirected getTriplets

GeneratorsR-Mat (power-law) Grid Star Complete …

Page 38: Apache Flink & Graph Processing

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

28

5

53

17

4

5

Page 39: Apache Flink & Graph Processing

Example: subGraphval 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

)

Page 40: Apache Flink & Graph Processing

Neighborhood MethodsApply a reduce function to the 1st-hop neighborhood of each vertex in parallelgraph.reduceOnNeighbors(

new MinValue, EdgeDirection.OUT)

Page 41: Apache Flink & Graph Processing

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

Page 42: Apache Flink & Graph Processing

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!)

Page 43: Apache Flink & Graph Processing

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

Page 44: Apache Flink & Graph Processing

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*

Page 45: Apache Flink & Graph Processing

Flink Iteration Operators

Input

Iterative Update Function

Result ReplaceWorkset

Iterative Update Function

Result

Solution Set

State

Page 46: Apache Flink & Graph Processing

Optimization• the runtime is aware of the iterative execution • no scheduling overhead between iterations • caching and state maintenance are handled automaticallyPush work

“out of the loop”Maintain state as indexCache Loop-invariant Data

Page 47: Apache Flink & Graph Processing

Vertex-Centric SSSPfinal 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) }

Page 48: Apache Flink & Graph Processing

Algorithms building blocks• Allow operator re-use across graph algorithms

when processing the same input with a similar configuration

Page 49: Apache Flink & Graph Processing

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))

Page 50: Apache Flink & Graph Processing

Tracker

Tracker

Ad Server

display relevant ads

cookie exchange

profiling

Web Tracking

Page 51: Apache Flink & Graph Processing

Can’t we block them?

proxy

Tracker

Tracker

Ad Server

Legitimate site

Page 52: Apache Flink & Graph Processing

• 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 SolutionsCrowd-sourced “black lists” of tracker URLs: - AdBlock, DoNotTrack, EasyPrivacy

Page 53: Apache Flink & Graph Processing

DataSet

• 6 months (Nov 2014 - April 2015) of augmented Apache logs from a web proxy

• 80m requests, 2m distinct URLs, 3k users

Page 54: Apache Flink & Graph Processing

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

r2r3

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

Page 55: Apache Flink & Graph Processing

Classification via Label Propagation

non-tracker tracker unlabeled

55

Page 56: Apache Flink & Graph Processing

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: Tbscored-research.com: Tfacebook.com: NTgithub.com: NTcdn.cxense.com: NT...

6: results

DataSet API

GellyDataSet API

Page 57: Apache Flink & Graph Processing

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.

Page 58: Apache Flink & Graph Processing

Gelly-Stream single-pass stream graph

processing with Flink

Page 59: Apache Flink & Graph Processing

Real Graphs are dynamic

Graphs are created from events happening in real-time

Page 60: Apache Flink & Graph Processing
Page 61: Apache Flink & Graph Processing

How we’ve done graph processing so far

1. Load: read the graph from disk and partition it in memory

Page 62: Apache Flink & Graph Processing

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

Page 63: Apache Flink & Graph Processing

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

Page 64: Apache Flink & Graph Processing

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

Page 65: Apache Flink & Graph Processing

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

Page 66: Apache Flink & Graph Processing

Single-pass graph streaming

• Each event is an edge addition

• Maintain only a graph summary

• Recent events are grouped in graph windows

Page 67: Apache Flink & Graph Processing
Page 68: Apache Flink & Graph Processing

Graph Summaries

• spanners for distance estimation • sparsifiers for cut estimation • sketches for homomorphic properties

graph summary

algorithm algorithm~R1 R2

Page 69: Apache Flink & Graph Processing

1

43

2

5

i=0

Batch Connected Components

6

7

8

Page 70: Apache Flink & Graph Processing

1

43

2

5

6

7

8

i=0

Batch Connected Components

14

3 45

235

24

78

67

68

Page 71: Apache Flink & Graph Processing

1

21

2

2

i=1

Batch Connected Components

6

6

6

Page 72: Apache Flink & Graph Processing

1

21

1

2

6

6

6

i=1

Batch Connected Components

2

1 22

112

12

76

6

6

Page 73: Apache Flink & Graph Processing

1

11

1

1

i=2

Batch Connected Components

6

6

6

Page 74: Apache Flink & Graph Processing

54

76

86

42

31

52

Stream Connected Components

Graph Summary: Disjoint Set (Union-Find)

• Only store component IDs and vertex IDs

Page 75: Apache Flink & Graph Processing

54

76

86

42

43

31

52

1

3

Cid = 1

Page 76: Apache Flink & Graph Processing

54

76

86

42

43

87

31

52

1

3

Cid = 1

2

5

Cid = 2

Page 77: Apache Flink & Graph Processing

54

76

86

42

43

87

41

31

52

1

3

Cid = 1

2

5

Cid = 2

4

Page 78: Apache Flink & Graph Processing

54

76

86

42

43

87

41

31

52

1

3

Cid = 1

2

5

Cid = 2

4

6

7Cid = 6

Page 79: Apache Flink & Graph Processing

54

76

86

42

43

87

41

31

52

1

3

Cid = 1

2

5

Cid = 2

4

6

7Cid = 6

8

Page 80: Apache Flink & Graph Processing

54

76

86

42

43

87

41

31

52

1

3

Cid = 1

2

5

Cid = 2

4

6

7Cid = 6

8

Page 81: Apache Flink & Graph Processing

54

76

86

42

43

87

41

52

6

7Cid = 6

8

1

3

Cid = 1

2

5

Cid = 2

4

Page 82: Apache Flink & Graph Processing

54

76

86

42

43

87

41

52

1

3

Cid = 1

2

5

4

6

7Cid = 6

8

Page 83: Apache Flink & Graph Processing

Distributed Stream Connected Components

Page 84: Apache Flink & Graph Processing

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);

Page 85: Apache Flink & Graph Processing

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

Page 86: Apache Flink & Graph Processing

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

Page 87: Apache Flink & Graph Processing

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

Page 88: Apache Flink & Graph Processing

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

Page 89: Apache Flink & Graph Processing

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

Page 90: Apache Flink & Graph Processing

Introducing Gelly-StreamGelly-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

Page 91: Apache Flink & Graph Processing

GraphStream Operations

.getEdges()

.getVertices()

.numberOfVertices()

.numberOfEdges()

.getDegrees()

.inDegrees()

.outDegrees()

GraphStream -> DataStream

.mapEdges();

.distinct();

.filterVertices();

.filterEdges();

.reverse();

.undirected();

.union();

GraphStream -> GraphStream

Property Streams Transformations

Page 92: Apache Flink & Graph Processing

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))

Page 93: Apache Flink & Graph Processing

Slicing Graph StreamsgraphStream.slice(Time.of(1, MINUTE));

11:40 11:41 11:42 11:43

Page 94: Apache Flink & Graph Processing

Aggregating SlicesgraphStream.slice(Time.of(1, MINUTE), direction)

.reduceOnEdges();

.foldNeighbors();

.applyOnNeighbors();

• Slicing collocates edges by vertex information

• Neighborhood aggregations on sliced graphs

source

target

Aggregations

Page 95: Apache Flink & Graph Processing

Finding Matches NearbygraphStream.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

soapbar

tom

rafa

joe’sgrill

FindPairs

{wendy, steve} {steve, sandra} {wendy, sandra} {tom, rafa}

GraphWindow :: user-place

Page 96: Apache Flink & Graph Processing

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

Page 97: Apache Flink & Graph Processing

Batch & Stream Graph Processing with Apache Flink

Vasia Kalavri [email protected]

@vkalavri

Apache Flink Meetup London October 5th, 2016