SlideShare a Scribd company logo
1 of 60
Apache Flink™ deep-dive
Unified Batch and Stream Processing
Robert Metzger
@rmetzger_
Flink’s Recent History
April 2014 April 2015Dec 2014
Top Level
Project
Graduation
0.70.60.5 0.90.9-m1
What is Flink
3
Gelly
Table
ML
SAMOA
DataSet (Java/Scala) DataStream
HadoopM/R
Local Remote YARN Tez Embedded
Dataflow
Dataflow(WiP)
MRQL
Table
Cascading(WiP)
Streaming dataflow runtime
Zeppelin
Program compilation
4
case class Path (from: Long, to:
Long)
val tc = edges.iterate(10) {
paths: DataSet[Path] =>
val next = paths
.join(edges)
.where("to")
.equalTo("from") {
(path, edge) =>
Path(path.from, edge.to)
}
.union(paths)
.distinct()
next
}
Optimizer
Type extraction
stack
Task
scheduling
Dataflow
metadata
Pre-flight (Client)
Master
Workers
Data Source
orders.tbl
Filter
Map DataSource
lineitem.tbl
Join
Hybrid Hash
buildHT probe
hash-part [0] hash-part [0]
GroupRed
sort
forward
Program
Dataflow Graph
Independent of
batch or
streaming job
deploy
operators
track
intermediate
results
 Layered Architecture
allows plugging of
components
Native workload support
5
Flink
Streaming
topologies
Long batch
pipelines
Machine Learning at scale
How can an engine natively support all these workloads?
And what does "native" mean?
Graph Analysis
 Low latency
 resource utilization  iterative algorithms
 Mutable state
E.g.: Non-native iterations
6
Step Step Step Step Step
Client
for (int i = 0; i < maxIterations; i++) {
// Execute MapReduce job
}
 Teaching an old elephant new tricks
 Treat system as a black box
E.g.: Non-native streaming
7
stream
discretizer
Job Job Job Job
while (true) {
// get next few records
// issue batch job
}
Data Stream
 Simulate stream processor with batch system
Native workload support
8
Flink
Streaming
topologies
Long batch
pipelines
Machine Learning at scale
How can an engine natively support all these workloads?
And what does "native" mean?
Graph Analysis
 Low latency
 resource utilization  iterative algorithms
 Mutable state
Ingredients for “native” support
1. Execute everything as streams
Pipelined execution, push model
2. Special code paths for batch
Automatic job optimization, fault tolerance
3. Allow some iterative (cyclic) dataflows
4. Allow some mutable state
5. Operate on managed memory
Make data processing on the JVM robust
9
Flink by Use Case
10
Stream data processing
streaming dataflows
11
Full talk tomorrow:
3:10PM, Grand Ballroom 220A
Stream processing with Flink
Pipelined stream processor
12
Streaming
Shuffle!
 Low latency
 Operators push data
forward
Expressive APIs
13
case class Word (word: String, frequency: Int)
val lines: DataStream[String] = env.fromSocketStream(...)
lines.flatMap {line => line.split(" ").map(word => Word(word,1))}
.window(Time.of(5,SECONDS)).every(Time.of(1,SECONDS))
.groupBy("word").sum("frequency")
.print()
val lines: DataSet[String] = env.readTextFile(...)
lines.flatMap {line => line.split(" ").map(word => Word(word,1))}
.groupBy("word").sum("frequency")
.print()
DataSet API (batch):
DataStream API (streaming):
Checkpointing / Recovery
14
Chandy-Lamport Algorithm for consistent asynchronous distributed snapshots
Pushes checkpoint barriers
through the data flow
Data Stream
barrier
Before barrier =
part of the snapshot
After barrier =
Not in snapshot
(backup till next snapshot)
 Guarantees exactly-once
processing
Batch processing
Batch on Streaming
15
Batch on an streaming engine
16
File in HDFS
Filter Map Result 1
Map Result 2
 Batch program, completely pipelined
 Data is never materialized anywhere (in this example)
Batch on an streaming engine
Map
Operator
Map
Operator
Map
Operator
17
Data
Source
(small)
Stream
Data
Sink
Data
Sink
Data
Sink
Join
Operator
in parallel
Data
Source
(large)
Data
Sink
in parallel (once build side finished)
Map
Batch processing requirements
 Get the data processed as fast as possible
• Automatic job optimizer
• Efficient memory management
 Robust processing
• provide fault-tolerance
• again, memory management
18
Optimizer
 Cost-based optimizer
 Select data shipping strategy (forward, partition, broadcast)
 Local execution (sort merge join/hash join)
 Caching of loop invariant data (iterations)
19
case class Path (from: Long, to:
Long)
val tc = edges.iterate(10) {
paths: DataSet[Path] =>
val next = paths
.join(edges)
.where("to")
.equalTo("from") {
(path, edge) =>
Path(path.from, edge.to)
}
.union(paths)
.distinct()
next
}
Optimizer
Type extraction
stack
Pre-flight (Client)
Data
Source
orders.tbl
Filter
Map
DataSourc
e
lineitem.tbl
Join
Hybrid Hash
build
HT
probe
hash-part [0] hash-part [0]
GroupRed
sort
forward
Program
Dataflow
Graph
Two execution plans
20
DataSource
orders.tbl
Filter
Map DataSource
lineitem.tbl
Join
Hybrid Hash
buildHT probe
broadcast forward
Combine
GroupRed
sort
DataSource
orders.tbl
Filter
Map DataSource
lineitem.tbl
Join
Hybrid Hash
buildHT probe
hash-part [0] hash-part [0]
hash-part [0,1]
GroupRed
sort
forward
Best plan
depends on
relative sizes
of input files
Memory Management
21
Operators on managed memory
22
Smooth out-of-core performance
23
More at: http://flink.apache.org/news/2015/03/13/peeking-into-Apache-Flinks-Engine-Room.html
Blue bars are in-memory, orange bars (partially) out-of-core
Machine Learning Algorithms
Iterative data flows
24
Iterate in the Dataflow
26
 API and runtime support
 Automatic caching of loop invariant
data
IterationState state =
getInitialState();
while (!terminationCriterion()) {
state = step(state);
}
setFinalState(state);
Example: Matrix Factorization
27
Factorizing a matrix with
28 billion ratings for
recommendations
More at: http://data-artisans.com/computing-recommendations-with-flink.html
Setups:
• 40 medium instances ("n1-highmem-8" - 8
cores, 52 GB)
• 40 large instances ("n1-highmem-16" - 16
cores, 104 GB)
Flink ML – Machine Learning
 Provide a complete toolchain
• scikit-learn style pipelining
• Data pre-processing
 various algorithms
• Recommendations: ALS
• Supervised learning: Support Vector Machines
• …
 ML on streams: SAMOA. We are planning to add support for
streaming into ML
28
Graph Analysis
Stateful Iterations
29
Graph processing characteristics
0
5000000
10000000
15000000
20000000
25000000
30000000
35000000
40000000
45000000
1 6 11 16 21 26 31 36 41 46 51 56 61
#ofelementsupdated
iteration
Iterate natively with state/deltas
31
 Keep state in an controlled way by having a partitioned hash-
map
 Relax immutability assumption of batch processing
… fast graph analysis
32More at: http://data-artisans.com/data-analysis-with-flink.html
Gelly – Graph Processing API
33
 Transformations: map, filter, subgraph, union, reverse,
undirected
 Mutations: add vertex/edge, remove …
 Pregel style vertex centric iterations
 Library of algorithms
 Utilities: Special data types, loading, graph properties
Gelly and Flink ML:
 Available in Flink 0.9 (so far only beta release)
 Still under heavy development
 Seamlessly integrate with DataSet abstraction
Preprocess data as needed
Use results as needed
 Easy entry point for new contributors
34
Closing
35
Flink Meetup Groups
 SF Spark and Friends
• June 16, San Francisco
 Bay Area Flink Meetup
• June 17, Redwood City
 Chicago Flink Meetup
• June 30
 Stockholm, Sweden
 Berlin, Germany
36
Flink Forward registration & call for
abstracts is open now
flink.apache.org 37
• 12/13 October 2015
• Meet developers and users of Flink!
• With Flink Workshops / Trainings!
flink.apache.org
@ApacheFlink
39
Backup
Flink community
0
20
40
60
80
100
120
Aug-10 Feb-11 Sep-11 Apr-12 Oct-12 May-13 Nov-13 Jun-14 Dec-14 Jul-15
#unique contributor ids by git commits
Flink
Historic data
Kafka, RabbitMQ, ...
HDFS, JDBC, ...
ETL, Graphs,
Machine Learning
Relational, …
Low latency,
windowing,
aggregations, ...
Event logs
Real-time data
streams
What is Apache Flink?
(master)
Cornerpoints of Flink Design
42
Robust Algorithms on
Managed Memory
Pipelined Execution
of Batch Programs
 Better shuffle performance
 No OutOfMemory Errors
 Scales to very large JVMs
 Efficient an robust processing
Flexible Data
Streaming Engine
 Low Latency Steam Proc.
 Highly flexible windows
Native Iterations
 Very fast Graph Processing
 Stateful Iterations for ML
High-level APIs,
beyond key/value pairs
 Java/Scala/Python (upcoming)
 Relational-style optimizer
 Graphs / Machine Learning
 Streaming ML (coming)
 Scales to very large groups
Active Library Development
Defining windows in Flink
 Trigger policy
• When to trigger the computation on current window
 Eviction policy
• When data points should leave the window
• Defines window width/size
 E.g., count-based policy
• evict when #elements > n
• start a new window every n-th element
 Built-in: Count, Time, Delta policies
43
Streaming checkpoints
44
45
46
Program optimization
47
A simple program
48
val orders = …
val lineitems = …
val filteredOrders = orders
.filter(o => dataFormat.parse(l.shipDate).after(date))
.filter(o => o.shipPrio > 2)
val lineitemsOfOrders = filteredOrders
.join(lineitems)
.where(“orderId”).equalTo(“orderId”)
.apply((o,l) => new SelectedItem(o.orderDate, l.extdPrice))
val priceSums = lineitemsOfOrders
.groupBy(“orderDate”).sum(“l.extdPrice”);
Two execution plans
49
DataSource
orders.tbl
Filter
Map DataSource
lineitem.tbl
Join
Hybrid Hash
buildHT probe
broadcast forward
Combine
GroupRed
sort
DataSource
orders.tbl
Filter
Map DataSource
lineitem.tbl
Join
Hybrid Hash
buildHT probe
hash-part [0] hash-part [0]
hash-part [0,1]
GroupRed
sort
forwardBest plan
depends on
relative sizes
of input files
Examples of optimization
 Task chaining
• Coalesce map/filter/etc tasks
 Join optimizations
• Broadcast/partition, build/probe side, hash or sort-merge
 Interesting properties
• Re-use partitioning and sorting for later operations
 Automatic caching
• E.g., for iterations
50
Visualization
51
Visualization tools
52
Visualization tools
53
Visualization tools
54
Batch processing 2
Batch on Streaming
55
Batch Pipelines
56
Operators Execution Overlaps
58
Memory Management
59
Memory Management
60
Smooth out-of-core performance
61
More at: http://flink.apache.org/news/2015/03/13/peeking-into-Apache-Flinks-Engine-Room.html
Blue bars are in-memory, orange bars (partially) out-of-core
optimizer
62

More Related Content

What's hot

Evening out the uneven: dealing with skew in Flink
Evening out the uneven: dealing with skew in FlinkEvening out the uneven: dealing with skew in Flink
Evening out the uneven: dealing with skew in FlinkFlink Forward
 
Apache Flink @ NYC Flink Meetup
Apache Flink @ NYC Flink MeetupApache Flink @ NYC Flink Meetup
Apache Flink @ NYC Flink MeetupStephan Ewen
 
Introduction to Apache Flink
Introduction to Apache FlinkIntroduction to Apache Flink
Introduction to Apache Flinkdatamantra
 
How to understand and analyze Apache Hive query execution plan for performanc...
How to understand and analyze Apache Hive query execution plan for performanc...How to understand and analyze Apache Hive query execution plan for performanc...
How to understand and analyze Apache Hive query execution plan for performanc...DataWorks Summit/Hadoop Summit
 
Parquet performance tuning: the missing guide
Parquet performance tuning: the missing guideParquet performance tuning: the missing guide
Parquet performance tuning: the missing guideRyan Blue
 
Apache Spark Data Source V2 with Wenchen Fan and Gengliang Wang
Apache Spark Data Source V2 with Wenchen Fan and Gengliang WangApache Spark Data Source V2 with Wenchen Fan and Gengliang Wang
Apache Spark Data Source V2 with Wenchen Fan and Gengliang WangDatabricks
 
Producer Performance Tuning for Apache Kafka
Producer Performance Tuning for Apache KafkaProducer Performance Tuning for Apache Kafka
Producer Performance Tuning for Apache KafkaJiangjie Qin
 
Introduction to Kafka Streams
Introduction to Kafka StreamsIntroduction to Kafka Streams
Introduction to Kafka StreamsGuozhang Wang
 
Tuning Apache Spark for Large-Scale Workloads Gaoxiang Liu and Sital Kedia
Tuning Apache Spark for Large-Scale Workloads Gaoxiang Liu and Sital KediaTuning Apache Spark for Large-Scale Workloads Gaoxiang Liu and Sital Kedia
Tuning Apache Spark for Large-Scale Workloads Gaoxiang Liu and Sital KediaDatabricks
 
Efficient Data Storage for Analytics with Apache Parquet 2.0
Efficient Data Storage for Analytics with Apache Parquet 2.0Efficient Data Storage for Analytics with Apache Parquet 2.0
Efficient Data Storage for Analytics with Apache Parquet 2.0Cloudera, Inc.
 
Stream processing using Kafka
Stream processing using KafkaStream processing using Kafka
Stream processing using KafkaKnoldus Inc.
 
Autoscaling Flink with Reactive Mode
Autoscaling Flink with Reactive ModeAutoscaling Flink with Reactive Mode
Autoscaling Flink with Reactive ModeFlink Forward
 
Kafka Streams: What it is, and how to use it?
Kafka Streams: What it is, and how to use it?Kafka Streams: What it is, and how to use it?
Kafka Streams: What it is, and how to use it?confluent
 
Delta Lake: Optimizing Merge
Delta Lake: Optimizing MergeDelta Lake: Optimizing Merge
Delta Lake: Optimizing MergeDatabricks
 
Demystifying DataFrame and Dataset with Kazuaki Ishizaki
Demystifying DataFrame and Dataset with Kazuaki IshizakiDemystifying DataFrame and Dataset with Kazuaki Ishizaki
Demystifying DataFrame and Dataset with Kazuaki IshizakiDatabricks
 
Bringing Kafka Without Zookeeper Into Production with Colin McCabe | Kafka Su...
Bringing Kafka Without Zookeeper Into Production with Colin McCabe | Kafka Su...Bringing Kafka Without Zookeeper Into Production with Colin McCabe | Kafka Su...
Bringing Kafka Without Zookeeper Into Production with Colin McCabe | Kafka Su...HostedbyConfluent
 
Deep Dive into Spark SQL with Advanced Performance Tuning with Xiao Li & Wenc...
Deep Dive into Spark SQL with Advanced Performance Tuning with Xiao Li & Wenc...Deep Dive into Spark SQL with Advanced Performance Tuning with Xiao Li & Wenc...
Deep Dive into Spark SQL with Advanced Performance Tuning with Xiao Li & Wenc...Databricks
 
Apache Flink Training: System Overview
Apache Flink Training: System OverviewApache Flink Training: System Overview
Apache Flink Training: System OverviewFlink Forward
 
The Rise of ZStandard: Apache Spark/Parquet/ORC/Avro
The Rise of ZStandard: Apache Spark/Parquet/ORC/AvroThe Rise of ZStandard: Apache Spark/Parquet/ORC/Avro
The Rise of ZStandard: Apache Spark/Parquet/ORC/AvroDatabricks
 
Introduction to Spark Streaming
Introduction to Spark StreamingIntroduction to Spark Streaming
Introduction to Spark Streamingdatamantra
 

What's hot (20)

Evening out the uneven: dealing with skew in Flink
Evening out the uneven: dealing with skew in FlinkEvening out the uneven: dealing with skew in Flink
Evening out the uneven: dealing with skew in Flink
 
Apache Flink @ NYC Flink Meetup
Apache Flink @ NYC Flink MeetupApache Flink @ NYC Flink Meetup
Apache Flink @ NYC Flink Meetup
 
Introduction to Apache Flink
Introduction to Apache FlinkIntroduction to Apache Flink
Introduction to Apache Flink
 
How to understand and analyze Apache Hive query execution plan for performanc...
How to understand and analyze Apache Hive query execution plan for performanc...How to understand and analyze Apache Hive query execution plan for performanc...
How to understand and analyze Apache Hive query execution plan for performanc...
 
Parquet performance tuning: the missing guide
Parquet performance tuning: the missing guideParquet performance tuning: the missing guide
Parquet performance tuning: the missing guide
 
Apache Spark Data Source V2 with Wenchen Fan and Gengliang Wang
Apache Spark Data Source V2 with Wenchen Fan and Gengliang WangApache Spark Data Source V2 with Wenchen Fan and Gengliang Wang
Apache Spark Data Source V2 with Wenchen Fan and Gengliang Wang
 
Producer Performance Tuning for Apache Kafka
Producer Performance Tuning for Apache KafkaProducer Performance Tuning for Apache Kafka
Producer Performance Tuning for Apache Kafka
 
Introduction to Kafka Streams
Introduction to Kafka StreamsIntroduction to Kafka Streams
Introduction to Kafka Streams
 
Tuning Apache Spark for Large-Scale Workloads Gaoxiang Liu and Sital Kedia
Tuning Apache Spark for Large-Scale Workloads Gaoxiang Liu and Sital KediaTuning Apache Spark for Large-Scale Workloads Gaoxiang Liu and Sital Kedia
Tuning Apache Spark for Large-Scale Workloads Gaoxiang Liu and Sital Kedia
 
Efficient Data Storage for Analytics with Apache Parquet 2.0
Efficient Data Storage for Analytics with Apache Parquet 2.0Efficient Data Storage for Analytics with Apache Parquet 2.0
Efficient Data Storage for Analytics with Apache Parquet 2.0
 
Stream processing using Kafka
Stream processing using KafkaStream processing using Kafka
Stream processing using Kafka
 
Autoscaling Flink with Reactive Mode
Autoscaling Flink with Reactive ModeAutoscaling Flink with Reactive Mode
Autoscaling Flink with Reactive Mode
 
Kafka Streams: What it is, and how to use it?
Kafka Streams: What it is, and how to use it?Kafka Streams: What it is, and how to use it?
Kafka Streams: What it is, and how to use it?
 
Delta Lake: Optimizing Merge
Delta Lake: Optimizing MergeDelta Lake: Optimizing Merge
Delta Lake: Optimizing Merge
 
Demystifying DataFrame and Dataset with Kazuaki Ishizaki
Demystifying DataFrame and Dataset with Kazuaki IshizakiDemystifying DataFrame and Dataset with Kazuaki Ishizaki
Demystifying DataFrame and Dataset with Kazuaki Ishizaki
 
Bringing Kafka Without Zookeeper Into Production with Colin McCabe | Kafka Su...
Bringing Kafka Without Zookeeper Into Production with Colin McCabe | Kafka Su...Bringing Kafka Without Zookeeper Into Production with Colin McCabe | Kafka Su...
Bringing Kafka Without Zookeeper Into Production with Colin McCabe | Kafka Su...
 
Deep Dive into Spark SQL with Advanced Performance Tuning with Xiao Li & Wenc...
Deep Dive into Spark SQL with Advanced Performance Tuning with Xiao Li & Wenc...Deep Dive into Spark SQL with Advanced Performance Tuning with Xiao Li & Wenc...
Deep Dive into Spark SQL with Advanced Performance Tuning with Xiao Li & Wenc...
 
Apache Flink Training: System Overview
Apache Flink Training: System OverviewApache Flink Training: System Overview
Apache Flink Training: System Overview
 
The Rise of ZStandard: Apache Spark/Parquet/ORC/Avro
The Rise of ZStandard: Apache Spark/Parquet/ORC/AvroThe Rise of ZStandard: Apache Spark/Parquet/ORC/Avro
The Rise of ZStandard: Apache Spark/Parquet/ORC/Avro
 
Introduction to Spark Streaming
Introduction to Spark StreamingIntroduction to Spark Streaming
Introduction to Spark Streaming
 

Viewers also liked

Inspiring Travel at Airbnb [WIP]
Inspiring Travel at Airbnb [WIP]Inspiring Travel at Airbnb [WIP]
Inspiring Travel at Airbnb [WIP]DataWorks Summit
 
One Click Hadoop Clusters - Anywhere (Using Docker)
One Click Hadoop Clusters - Anywhere (Using Docker)One Click Hadoop Clusters - Anywhere (Using Docker)
One Click Hadoop Clusters - Anywhere (Using Docker)DataWorks Summit
 
Coexistence and Migration of Vendor HPC based infrastructure to Hadoop Ecosys...
Coexistence and Migration of Vendor HPC based infrastructure to Hadoop Ecosys...Coexistence and Migration of Vendor HPC based infrastructure to Hadoop Ecosys...
Coexistence and Migration of Vendor HPC based infrastructure to Hadoop Ecosys...DataWorks Summit
 
Hadoop in Validated Environment - Data Governance Initiative
Hadoop in Validated Environment - Data Governance InitiativeHadoop in Validated Environment - Data Governance Initiative
Hadoop in Validated Environment - Data Governance InitiativeDataWorks Summit
 
Can you Re-Platform your Teradata, Oracle, Netezza and SQL Server Analytic Wo...
Can you Re-Platform your Teradata, Oracle, Netezza and SQL Server Analytic Wo...Can you Re-Platform your Teradata, Oracle, Netezza and SQL Server Analytic Wo...
Can you Re-Platform your Teradata, Oracle, Netezza and SQL Server Analytic Wo...DataWorks Summit
 
Practical Distributed Machine Learning Pipelines on Hadoop
Practical Distributed Machine Learning Pipelines on HadoopPractical Distributed Machine Learning Pipelines on Hadoop
Practical Distributed Machine Learning Pipelines on HadoopDataWorks Summit
 
Karta an ETL Framework to process high volume datasets
Karta an ETL Framework to process high volume datasets Karta an ETL Framework to process high volume datasets
Karta an ETL Framework to process high volume datasets DataWorks Summit
 
Running Spark and MapReduce together in Production
Running Spark and MapReduce together in ProductionRunning Spark and MapReduce together in Production
Running Spark and MapReduce together in ProductionDataWorks Summit
 
HBase and Drill: How loosley typed SQL is ideal for NoSQL
HBase and Drill: How loosley typed SQL is ideal for NoSQLHBase and Drill: How loosley typed SQL is ideal for NoSQL
HBase and Drill: How loosley typed SQL is ideal for NoSQLDataWorks Summit
 
The Most Valuable Customer on Earth-1298: Comic Book Analysis with Oracel's B...
The Most Valuable Customer on Earth-1298: Comic Book Analysis with Oracel's B...The Most Valuable Customer on Earth-1298: Comic Book Analysis with Oracel's B...
The Most Valuable Customer on Earth-1298: Comic Book Analysis with Oracel's B...DataWorks Summit
 
Carpe Datum: Building Big Data Analytical Applications with HP Haven
Carpe Datum: Building Big Data Analytical Applications with HP HavenCarpe Datum: Building Big Data Analytical Applications with HP Haven
Carpe Datum: Building Big Data Analytical Applications with HP HavenDataWorks Summit
 
Big Data Simplified - Is all about Ab'strakSHeN
Big Data Simplified - Is all about Ab'strakSHeNBig Data Simplified - Is all about Ab'strakSHeN
Big Data Simplified - Is all about Ab'strakSHeNDataWorks Summit
 
Realistic Synthetic Generation Allows Secure Development
Realistic Synthetic Generation Allows Secure DevelopmentRealistic Synthetic Generation Allows Secure Development
Realistic Synthetic Generation Allows Secure DevelopmentDataWorks Summit
 
DeathStar: Easy, Dynamic, Multi-Tenant HBase via YARN
DeathStar: Easy, Dynamic, Multi-Tenant HBase via YARNDeathStar: Easy, Dynamic, Multi-Tenant HBase via YARN
DeathStar: Easy, Dynamic, Multi-Tenant HBase via YARNDataWorks Summit
 
Open Source SQL for Hadoop: Where are we and Where are we Going?
Open Source SQL for Hadoop: Where are we and Where are we Going?Open Source SQL for Hadoop: Where are we and Where are we Going?
Open Source SQL for Hadoop: Where are we and Where are we Going?DataWorks Summit
 
Spark Application Development Made Easy
Spark Application Development Made EasySpark Application Development Made Easy
Spark Application Development Made EasyDataWorks Summit
 
Modus operandi of Spark Streaming - Recipes for Running your Streaming Applic...
Modus operandi of Spark Streaming - Recipes for Running your Streaming Applic...Modus operandi of Spark Streaming - Recipes for Running your Streaming Applic...
Modus operandi of Spark Streaming - Recipes for Running your Streaming Applic...DataWorks Summit
 

Viewers also liked (20)

Inspiring Travel at Airbnb [WIP]
Inspiring Travel at Airbnb [WIP]Inspiring Travel at Airbnb [WIP]
Inspiring Travel at Airbnb [WIP]
 
One Click Hadoop Clusters - Anywhere (Using Docker)
One Click Hadoop Clusters - Anywhere (Using Docker)One Click Hadoop Clusters - Anywhere (Using Docker)
One Click Hadoop Clusters - Anywhere (Using Docker)
 
Coexistence and Migration of Vendor HPC based infrastructure to Hadoop Ecosys...
Coexistence and Migration of Vendor HPC based infrastructure to Hadoop Ecosys...Coexistence and Migration of Vendor HPC based infrastructure to Hadoop Ecosys...
Coexistence and Migration of Vendor HPC based infrastructure to Hadoop Ecosys...
 
Hadoop in Validated Environment - Data Governance Initiative
Hadoop in Validated Environment - Data Governance InitiativeHadoop in Validated Environment - Data Governance Initiative
Hadoop in Validated Environment - Data Governance Initiative
 
Can you Re-Platform your Teradata, Oracle, Netezza and SQL Server Analytic Wo...
Can you Re-Platform your Teradata, Oracle, Netezza and SQL Server Analytic Wo...Can you Re-Platform your Teradata, Oracle, Netezza and SQL Server Analytic Wo...
Can you Re-Platform your Teradata, Oracle, Netezza and SQL Server Analytic Wo...
 
Hadoop for Genomics__HadoopSummit2010
Hadoop for Genomics__HadoopSummit2010Hadoop for Genomics__HadoopSummit2010
Hadoop for Genomics__HadoopSummit2010
 
Practical Distributed Machine Learning Pipelines on Hadoop
Practical Distributed Machine Learning Pipelines on HadoopPractical Distributed Machine Learning Pipelines on Hadoop
Practical Distributed Machine Learning Pipelines on Hadoop
 
Karta an ETL Framework to process high volume datasets
Karta an ETL Framework to process high volume datasets Karta an ETL Framework to process high volume datasets
Karta an ETL Framework to process high volume datasets
 
Running Spark and MapReduce together in Production
Running Spark and MapReduce together in ProductionRunning Spark and MapReduce together in Production
Running Spark and MapReduce together in Production
 
HBase and Drill: How loosley typed SQL is ideal for NoSQL
HBase and Drill: How loosley typed SQL is ideal for NoSQLHBase and Drill: How loosley typed SQL is ideal for NoSQL
HBase and Drill: How loosley typed SQL is ideal for NoSQL
 
The Most Valuable Customer on Earth-1298: Comic Book Analysis with Oracel's B...
The Most Valuable Customer on Earth-1298: Comic Book Analysis with Oracel's B...The Most Valuable Customer on Earth-1298: Comic Book Analysis with Oracel's B...
The Most Valuable Customer on Earth-1298: Comic Book Analysis with Oracel's B...
 
Carpe Datum: Building Big Data Analytical Applications with HP Haven
Carpe Datum: Building Big Data Analytical Applications with HP HavenCarpe Datum: Building Big Data Analytical Applications with HP Haven
Carpe Datum: Building Big Data Analytical Applications with HP Haven
 
50 Shades of SQL
50 Shades of SQL50 Shades of SQL
50 Shades of SQL
 
Big Data Simplified - Is all about Ab'strakSHeN
Big Data Simplified - Is all about Ab'strakSHeNBig Data Simplified - Is all about Ab'strakSHeN
Big Data Simplified - Is all about Ab'strakSHeN
 
Realistic Synthetic Generation Allows Secure Development
Realistic Synthetic Generation Allows Secure DevelopmentRealistic Synthetic Generation Allows Secure Development
Realistic Synthetic Generation Allows Secure Development
 
DeathStar: Easy, Dynamic, Multi-Tenant HBase via YARN
DeathStar: Easy, Dynamic, Multi-Tenant HBase via YARNDeathStar: Easy, Dynamic, Multi-Tenant HBase via YARN
DeathStar: Easy, Dynamic, Multi-Tenant HBase via YARN
 
NoSQL Needs SomeSQL
NoSQL Needs SomeSQLNoSQL Needs SomeSQL
NoSQL Needs SomeSQL
 
Open Source SQL for Hadoop: Where are we and Where are we Going?
Open Source SQL for Hadoop: Where are we and Where are we Going?Open Source SQL for Hadoop: Where are we and Where are we Going?
Open Source SQL for Hadoop: Where are we and Where are we Going?
 
Spark Application Development Made Easy
Spark Application Development Made EasySpark Application Development Made Easy
Spark Application Development Made Easy
 
Modus operandi of Spark Streaming - Recipes for Running your Streaming Applic...
Modus operandi of Spark Streaming - Recipes for Running your Streaming Applic...Modus operandi of Spark Streaming - Recipes for Running your Streaming Applic...
Modus operandi of Spark Streaming - Recipes for Running your Streaming Applic...
 

Similar to Apache Flink Deep Dive

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, CARobert Metzger
 
Apache Flink Overview at SF Spark and Friends
Apache Flink Overview at SF Spark and FriendsApache Flink Overview at SF Spark and Friends
Apache Flink Overview at SF Spark and FriendsStephan Ewen
 
Unified batch and stream processing with Flink @ Big Data Beers Berlin May 2015
Unified batch and stream processing with Flink @ Big Data Beers Berlin May 2015Unified batch and stream processing with Flink @ Big Data Beers Berlin May 2015
Unified batch and stream processing with Flink @ Big Data Beers Berlin May 2015Robert Metzger
 
Apache Flink Meetup Munich (November 2015): Flink Overview, Architecture, Int...
Apache Flink Meetup Munich (November 2015): Flink Overview, Architecture, Int...Apache Flink Meetup Munich (November 2015): Flink Overview, Architecture, Int...
Apache Flink Meetup Munich (November 2015): Flink Overview, Architecture, Int...Robert Metzger
 
First Flink Bay Area meetup
First Flink Bay Area meetupFirst Flink Bay Area meetup
First Flink Bay Area meetupKostas Tzoumas
 
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 ProcessingFabian Hueske
 
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...Stephan Ewen
 
Apache Flink@ Strata & Hadoop World London
Apache Flink@ Strata & Hadoop World LondonApache Flink@ Strata & Hadoop World London
Apache Flink@ Strata & Hadoop World LondonStephan Ewen
 
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 FlinkDataWorks Summit
 
Machine Learning with Apache Flink at Stockholm Machine Learning Group
Machine Learning with Apache Flink at Stockholm Machine Learning GroupMachine Learning with Apache Flink at Stockholm Machine Learning Group
Machine Learning with Apache Flink at Stockholm Machine Learning GroupTill Rohrmann
 
January 2015 HUG: Apache Flink: Fast and reliable large-scale data processing
January 2015 HUG: Apache Flink:  Fast and reliable large-scale data processingJanuary 2015 HUG: Apache Flink:  Fast and reliable large-scale data processing
January 2015 HUG: Apache Flink: Fast and reliable large-scale data processingYahoo Developer Network
 
Flink Streaming Hadoop Summit San Jose
Flink Streaming Hadoop Summit San JoseFlink Streaming Hadoop Summit San Jose
Flink Streaming Hadoop Summit San JoseKostas Tzoumas
 
Simplifying Big Data Analytics with Apache Spark
Simplifying Big Data Analytics with Apache SparkSimplifying Big Data Analytics with Apache Spark
Simplifying Big Data Analytics with Apache SparkDatabricks
 
Advanced Data Science on Spark-(Reza Zadeh, Stanford)
Advanced Data Science on Spark-(Reza Zadeh, Stanford)Advanced Data Science on Spark-(Reza Zadeh, Stanford)
Advanced Data Science on Spark-(Reza Zadeh, Stanford)Spark Summit
 
Productionizing your Streaming Jobs
Productionizing your Streaming JobsProductionizing your Streaming Jobs
Productionizing your Streaming JobsDatabricks
 
Real-time Stream Processing with Apache Flink @ Hadoop Summit
Real-time Stream Processing with Apache Flink @ Hadoop SummitReal-time Stream Processing with Apache Flink @ Hadoop Summit
Real-time Stream Processing with Apache Flink @ Hadoop SummitGyula Fóra
 
Apache Flink & Graph Processing
Apache Flink & Graph ProcessingApache Flink & Graph Processing
Apache Flink & Graph ProcessingVasia Kalavri
 
Apache spark - Architecture , Overview & libraries
Apache spark - Architecture , Overview & librariesApache spark - Architecture , Overview & libraries
Apache spark - Architecture , Overview & librariesWalaa Hamdy Assy
 

Similar to Apache Flink Deep Dive (20)

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
 
Apache Flink Overview at SF Spark and Friends
Apache Flink Overview at SF Spark and FriendsApache Flink Overview at SF Spark and Friends
Apache Flink Overview at SF Spark and Friends
 
Unified batch and stream processing with Flink @ Big Data Beers Berlin May 2015
Unified batch and stream processing with Flink @ Big Data Beers Berlin May 2015Unified batch and stream processing with Flink @ Big Data Beers Berlin May 2015
Unified batch and stream processing with Flink @ Big Data Beers Berlin May 2015
 
Apache Flink Meetup Munich (November 2015): Flink Overview, Architecture, Int...
Apache Flink Meetup Munich (November 2015): Flink Overview, Architecture, Int...Apache Flink Meetup Munich (November 2015): Flink Overview, Architecture, Int...
Apache Flink Meetup Munich (November 2015): Flink Overview, Architecture, Int...
 
First Flink Bay Area meetup
First Flink Bay Area meetupFirst Flink Bay Area meetup
First Flink Bay Area meetup
 
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
 
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...
 
Apache Flink@ Strata & Hadoop World London
Apache Flink@ Strata & Hadoop World LondonApache Flink@ Strata & Hadoop World London
Apache Flink@ Strata & Hadoop World London
 
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
 
Machine Learning with Apache Flink at Stockholm Machine Learning Group
Machine Learning with Apache Flink at Stockholm Machine Learning GroupMachine Learning with Apache Flink at Stockholm Machine Learning Group
Machine Learning with Apache Flink at Stockholm Machine Learning Group
 
January 2015 HUG: Apache Flink: Fast and reliable large-scale data processing
January 2015 HUG: Apache Flink:  Fast and reliable large-scale data processingJanuary 2015 HUG: Apache Flink:  Fast and reliable large-scale data processing
January 2015 HUG: Apache Flink: Fast and reliable large-scale data processing
 
Flink Streaming Hadoop Summit San Jose
Flink Streaming Hadoop Summit San JoseFlink Streaming Hadoop Summit San Jose
Flink Streaming Hadoop Summit San Jose
 
Simplifying Big Data Analytics with Apache Spark
Simplifying Big Data Analytics with Apache SparkSimplifying Big Data Analytics with Apache Spark
Simplifying Big Data Analytics with Apache Spark
 
Advanced Data Science on Spark-(Reza Zadeh, Stanford)
Advanced Data Science on Spark-(Reza Zadeh, Stanford)Advanced Data Science on Spark-(Reza Zadeh, Stanford)
Advanced Data Science on Spark-(Reza Zadeh, Stanford)
 
Flink internals web
Flink internals web Flink internals web
Flink internals web
 
20170126 big data processing
20170126 big data processing20170126 big data processing
20170126 big data processing
 
Productionizing your Streaming Jobs
Productionizing your Streaming JobsProductionizing your Streaming Jobs
Productionizing your Streaming Jobs
 
Real-time Stream Processing with Apache Flink @ Hadoop Summit
Real-time Stream Processing with Apache Flink @ Hadoop SummitReal-time Stream Processing with Apache Flink @ Hadoop Summit
Real-time Stream Processing with Apache Flink @ Hadoop Summit
 
Apache Flink & Graph Processing
Apache Flink & Graph ProcessingApache Flink & Graph Processing
Apache Flink & Graph Processing
 
Apache spark - Architecture , Overview & libraries
Apache spark - Architecture , Overview & librariesApache spark - Architecture , Overview & libraries
Apache spark - Architecture , Overview & libraries
 

More from DataWorks Summit

Floating on a RAFT: HBase Durability with Apache Ratis
Floating on a RAFT: HBase Durability with Apache RatisFloating on a RAFT: HBase Durability with Apache Ratis
Floating on a RAFT: HBase Durability with Apache RatisDataWorks Summit
 
Tracking Crime as It Occurs with Apache Phoenix, Apache HBase and Apache NiFi
Tracking Crime as It Occurs with Apache Phoenix, Apache HBase and Apache NiFiTracking Crime as It Occurs with Apache Phoenix, Apache HBase and Apache NiFi
Tracking Crime as It Occurs with Apache Phoenix, Apache HBase and Apache NiFiDataWorks Summit
 
HBase Tales From the Trenches - Short stories about most common HBase operati...
HBase Tales From the Trenches - Short stories about most common HBase operati...HBase Tales From the Trenches - Short stories about most common HBase operati...
HBase Tales From the Trenches - Short stories about most common HBase operati...DataWorks Summit
 
Optimizing Geospatial Operations with Server-side Programming in HBase and Ac...
Optimizing Geospatial Operations with Server-side Programming in HBase and Ac...Optimizing Geospatial Operations with Server-side Programming in HBase and Ac...
Optimizing Geospatial Operations with Server-side Programming in HBase and Ac...DataWorks Summit
 
Managing the Dewey Decimal System
Managing the Dewey Decimal SystemManaging the Dewey Decimal System
Managing the Dewey Decimal SystemDataWorks Summit
 
Practical NoSQL: Accumulo's dirlist Example
Practical NoSQL: Accumulo's dirlist ExamplePractical NoSQL: Accumulo's dirlist Example
Practical NoSQL: Accumulo's dirlist ExampleDataWorks Summit
 
HBase Global Indexing to support large-scale data ingestion at Uber
HBase Global Indexing to support large-scale data ingestion at UberHBase Global Indexing to support large-scale data ingestion at Uber
HBase Global Indexing to support large-scale data ingestion at UberDataWorks Summit
 
Scaling Cloud-Scale Translytics Workloads with Omid and Phoenix
Scaling Cloud-Scale Translytics Workloads with Omid and PhoenixScaling Cloud-Scale Translytics Workloads with Omid and Phoenix
Scaling Cloud-Scale Translytics Workloads with Omid and PhoenixDataWorks Summit
 
Building the High Speed Cybersecurity Data Pipeline Using Apache NiFi
Building the High Speed Cybersecurity Data Pipeline Using Apache NiFiBuilding the High Speed Cybersecurity Data Pipeline Using Apache NiFi
Building the High Speed Cybersecurity Data Pipeline Using Apache NiFiDataWorks Summit
 
Supporting Apache HBase : Troubleshooting and Supportability Improvements
Supporting Apache HBase : Troubleshooting and Supportability ImprovementsSupporting Apache HBase : Troubleshooting and Supportability Improvements
Supporting Apache HBase : Troubleshooting and Supportability ImprovementsDataWorks Summit
 
Security Framework for Multitenant Architecture
Security Framework for Multitenant ArchitectureSecurity Framework for Multitenant Architecture
Security Framework for Multitenant ArchitectureDataWorks Summit
 
Presto: Optimizing Performance of SQL-on-Anything Engine
Presto: Optimizing Performance of SQL-on-Anything EnginePresto: Optimizing Performance of SQL-on-Anything Engine
Presto: Optimizing Performance of SQL-on-Anything EngineDataWorks Summit
 
Introducing MlFlow: An Open Source Platform for the Machine Learning Lifecycl...
Introducing MlFlow: An Open Source Platform for the Machine Learning Lifecycl...Introducing MlFlow: An Open Source Platform for the Machine Learning Lifecycl...
Introducing MlFlow: An Open Source Platform for the Machine Learning Lifecycl...DataWorks Summit
 
Extending Twitter's Data Platform to Google Cloud
Extending Twitter's Data Platform to Google CloudExtending Twitter's Data Platform to Google Cloud
Extending Twitter's Data Platform to Google CloudDataWorks Summit
 
Event-Driven Messaging and Actions using Apache Flink and Apache NiFi
Event-Driven Messaging and Actions using Apache Flink and Apache NiFiEvent-Driven Messaging and Actions using Apache Flink and Apache NiFi
Event-Driven Messaging and Actions using Apache Flink and Apache NiFiDataWorks Summit
 
Securing Data in Hybrid on-premise and Cloud Environments using Apache Ranger
Securing Data in Hybrid on-premise and Cloud Environments using Apache RangerSecuring Data in Hybrid on-premise and Cloud Environments using Apache Ranger
Securing Data in Hybrid on-premise and Cloud Environments using Apache RangerDataWorks Summit
 
Big Data Meets NVM: Accelerating Big Data Processing with Non-Volatile Memory...
Big Data Meets NVM: Accelerating Big Data Processing with Non-Volatile Memory...Big Data Meets NVM: Accelerating Big Data Processing with Non-Volatile Memory...
Big Data Meets NVM: Accelerating Big Data Processing with Non-Volatile Memory...DataWorks Summit
 
Computer Vision: Coming to a Store Near You
Computer Vision: Coming to a Store Near YouComputer Vision: Coming to a Store Near You
Computer Vision: Coming to a Store Near YouDataWorks Summit
 
Big Data Genomics: Clustering Billions of DNA Sequences with Apache Spark
Big Data Genomics: Clustering Billions of DNA Sequences with Apache SparkBig Data Genomics: Clustering Billions of DNA Sequences with Apache Spark
Big Data Genomics: Clustering Billions of DNA Sequences with Apache SparkDataWorks Summit
 

More from DataWorks Summit (20)

Data Science Crash Course
Data Science Crash CourseData Science Crash Course
Data Science Crash Course
 
Floating on a RAFT: HBase Durability with Apache Ratis
Floating on a RAFT: HBase Durability with Apache RatisFloating on a RAFT: HBase Durability with Apache Ratis
Floating on a RAFT: HBase Durability with Apache Ratis
 
Tracking Crime as It Occurs with Apache Phoenix, Apache HBase and Apache NiFi
Tracking Crime as It Occurs with Apache Phoenix, Apache HBase and Apache NiFiTracking Crime as It Occurs with Apache Phoenix, Apache HBase and Apache NiFi
Tracking Crime as It Occurs with Apache Phoenix, Apache HBase and Apache NiFi
 
HBase Tales From the Trenches - Short stories about most common HBase operati...
HBase Tales From the Trenches - Short stories about most common HBase operati...HBase Tales From the Trenches - Short stories about most common HBase operati...
HBase Tales From the Trenches - Short stories about most common HBase operati...
 
Optimizing Geospatial Operations with Server-side Programming in HBase and Ac...
Optimizing Geospatial Operations with Server-side Programming in HBase and Ac...Optimizing Geospatial Operations with Server-side Programming in HBase and Ac...
Optimizing Geospatial Operations with Server-side Programming in HBase and Ac...
 
Managing the Dewey Decimal System
Managing the Dewey Decimal SystemManaging the Dewey Decimal System
Managing the Dewey Decimal System
 
Practical NoSQL: Accumulo's dirlist Example
Practical NoSQL: Accumulo's dirlist ExamplePractical NoSQL: Accumulo's dirlist Example
Practical NoSQL: Accumulo's dirlist Example
 
HBase Global Indexing to support large-scale data ingestion at Uber
HBase Global Indexing to support large-scale data ingestion at UberHBase Global Indexing to support large-scale data ingestion at Uber
HBase Global Indexing to support large-scale data ingestion at Uber
 
Scaling Cloud-Scale Translytics Workloads with Omid and Phoenix
Scaling Cloud-Scale Translytics Workloads with Omid and PhoenixScaling Cloud-Scale Translytics Workloads with Omid and Phoenix
Scaling Cloud-Scale Translytics Workloads with Omid and Phoenix
 
Building the High Speed Cybersecurity Data Pipeline Using Apache NiFi
Building the High Speed Cybersecurity Data Pipeline Using Apache NiFiBuilding the High Speed Cybersecurity Data Pipeline Using Apache NiFi
Building the High Speed Cybersecurity Data Pipeline Using Apache NiFi
 
Supporting Apache HBase : Troubleshooting and Supportability Improvements
Supporting Apache HBase : Troubleshooting and Supportability ImprovementsSupporting Apache HBase : Troubleshooting and Supportability Improvements
Supporting Apache HBase : Troubleshooting and Supportability Improvements
 
Security Framework for Multitenant Architecture
Security Framework for Multitenant ArchitectureSecurity Framework for Multitenant Architecture
Security Framework for Multitenant Architecture
 
Presto: Optimizing Performance of SQL-on-Anything Engine
Presto: Optimizing Performance of SQL-on-Anything EnginePresto: Optimizing Performance of SQL-on-Anything Engine
Presto: Optimizing Performance of SQL-on-Anything Engine
 
Introducing MlFlow: An Open Source Platform for the Machine Learning Lifecycl...
Introducing MlFlow: An Open Source Platform for the Machine Learning Lifecycl...Introducing MlFlow: An Open Source Platform for the Machine Learning Lifecycl...
Introducing MlFlow: An Open Source Platform for the Machine Learning Lifecycl...
 
Extending Twitter's Data Platform to Google Cloud
Extending Twitter's Data Platform to Google CloudExtending Twitter's Data Platform to Google Cloud
Extending Twitter's Data Platform to Google Cloud
 
Event-Driven Messaging and Actions using Apache Flink and Apache NiFi
Event-Driven Messaging and Actions using Apache Flink and Apache NiFiEvent-Driven Messaging and Actions using Apache Flink and Apache NiFi
Event-Driven Messaging and Actions using Apache Flink and Apache NiFi
 
Securing Data in Hybrid on-premise and Cloud Environments using Apache Ranger
Securing Data in Hybrid on-premise and Cloud Environments using Apache RangerSecuring Data in Hybrid on-premise and Cloud Environments using Apache Ranger
Securing Data in Hybrid on-premise and Cloud Environments using Apache Ranger
 
Big Data Meets NVM: Accelerating Big Data Processing with Non-Volatile Memory...
Big Data Meets NVM: Accelerating Big Data Processing with Non-Volatile Memory...Big Data Meets NVM: Accelerating Big Data Processing with Non-Volatile Memory...
Big Data Meets NVM: Accelerating Big Data Processing with Non-Volatile Memory...
 
Computer Vision: Coming to a Store Near You
Computer Vision: Coming to a Store Near YouComputer Vision: Coming to a Store Near You
Computer Vision: Coming to a Store Near You
 
Big Data Genomics: Clustering Billions of DNA Sequences with Apache Spark
Big Data Genomics: Clustering Billions of DNA Sequences with Apache SparkBig Data Genomics: Clustering Billions of DNA Sequences with Apache Spark
Big Data Genomics: Clustering Billions of DNA Sequences with Apache Spark
 

Recently uploaded

Unraveling Multimodality with Large Language Models.pdf
Unraveling Multimodality with Large Language Models.pdfUnraveling Multimodality with Large Language Models.pdf
Unraveling Multimodality with Large Language Models.pdfAlex Barbosa Coqueiro
 
From Family Reminiscence to Scholarly Archive .
From Family Reminiscence to Scholarly Archive .From Family Reminiscence to Scholarly Archive .
From Family Reminiscence to Scholarly Archive .Alan Dix
 
Leverage Zilliz Serverless - Up to 50X Saving for Your Vector Storage Cost
Leverage Zilliz Serverless - Up to 50X Saving for Your Vector Storage CostLeverage Zilliz Serverless - Up to 50X Saving for Your Vector Storage Cost
Leverage Zilliz Serverless - Up to 50X Saving for Your Vector Storage CostZilliz
 
Advanced Test Driven-Development @ php[tek] 2024
Advanced Test Driven-Development @ php[tek] 2024Advanced Test Driven-Development @ php[tek] 2024
Advanced Test Driven-Development @ php[tek] 2024Scott Keck-Warren
 
DSPy a system for AI to Write Prompts and Do Fine Tuning
DSPy a system for AI to Write Prompts and Do Fine TuningDSPy a system for AI to Write Prompts and Do Fine Tuning
DSPy a system for AI to Write Prompts and Do Fine TuningLars Bell
 
Gen AI in Business - Global Trends Report 2024.pdf
Gen AI in Business - Global Trends Report 2024.pdfGen AI in Business - Global Trends Report 2024.pdf
Gen AI in Business - Global Trends Report 2024.pdfAddepto
 
"LLMs for Python Engineers: Advanced Data Analysis and Semantic Kernel",Oleks...
"LLMs for Python Engineers: Advanced Data Analysis and Semantic Kernel",Oleks..."LLMs for Python Engineers: Advanced Data Analysis and Semantic Kernel",Oleks...
"LLMs for Python Engineers: Advanced Data Analysis and Semantic Kernel",Oleks...Fwdays
 
Dev Dives: Streamline document processing with UiPath Studio Web
Dev Dives: Streamline document processing with UiPath Studio WebDev Dives: Streamline document processing with UiPath Studio Web
Dev Dives: Streamline document processing with UiPath Studio WebUiPathCommunity
 
SAP Build Work Zone - Overview L2-L3.pptx
SAP Build Work Zone - Overview L2-L3.pptxSAP Build Work Zone - Overview L2-L3.pptx
SAP Build Work Zone - Overview L2-L3.pptxNavinnSomaal
 
Hyperautomation and AI/ML: A Strategy for Digital Transformation Success.pdf
Hyperautomation and AI/ML: A Strategy for Digital Transformation Success.pdfHyperautomation and AI/ML: A Strategy for Digital Transformation Success.pdf
Hyperautomation and AI/ML: A Strategy for Digital Transformation Success.pdfPrecisely
 
Nell’iperspazio con Rocket: il Framework Web di Rust!
Nell’iperspazio con Rocket: il Framework Web di Rust!Nell’iperspazio con Rocket: il Framework Web di Rust!
Nell’iperspazio con Rocket: il Framework Web di Rust!Commit University
 
CloudStudio User manual (basic edition):
CloudStudio User manual (basic edition):CloudStudio User manual (basic edition):
CloudStudio User manual (basic edition):comworks
 
Anypoint Exchange: It’s Not Just a Repo!
Anypoint Exchange: It’s Not Just a Repo!Anypoint Exchange: It’s Not Just a Repo!
Anypoint Exchange: It’s Not Just a Repo!Manik S Magar
 
DevoxxFR 2024 Reproducible Builds with Apache Maven
DevoxxFR 2024 Reproducible Builds with Apache MavenDevoxxFR 2024 Reproducible Builds with Apache Maven
DevoxxFR 2024 Reproducible Builds with Apache MavenHervé Boutemy
 
DevEX - reference for building teams, processes, and platforms
DevEX - reference for building teams, processes, and platformsDevEX - reference for building teams, processes, and platforms
DevEX - reference for building teams, processes, and platformsSergiu Bodiu
 
Story boards and shot lists for my a level piece
Story boards and shot lists for my a level pieceStory boards and shot lists for my a level piece
Story boards and shot lists for my a level piececharlottematthew16
 
How to write a Business Continuity Plan
How to write a Business Continuity PlanHow to write a Business Continuity Plan
How to write a Business Continuity PlanDatabarracks
 
Unleash Your Potential - Namagunga Girls Coding Club
Unleash Your Potential - Namagunga Girls Coding ClubUnleash Your Potential - Namagunga Girls Coding Club
Unleash Your Potential - Namagunga Girls Coding ClubKalema Edgar
 

Recently uploaded (20)

Unraveling Multimodality with Large Language Models.pdf
Unraveling Multimodality with Large Language Models.pdfUnraveling Multimodality with Large Language Models.pdf
Unraveling Multimodality with Large Language Models.pdf
 
From Family Reminiscence to Scholarly Archive .
From Family Reminiscence to Scholarly Archive .From Family Reminiscence to Scholarly Archive .
From Family Reminiscence to Scholarly Archive .
 
Leverage Zilliz Serverless - Up to 50X Saving for Your Vector Storage Cost
Leverage Zilliz Serverless - Up to 50X Saving for Your Vector Storage CostLeverage Zilliz Serverless - Up to 50X Saving for Your Vector Storage Cost
Leverage Zilliz Serverless - Up to 50X Saving for Your Vector Storage Cost
 
Advanced Test Driven-Development @ php[tek] 2024
Advanced Test Driven-Development @ php[tek] 2024Advanced Test Driven-Development @ php[tek] 2024
Advanced Test Driven-Development @ php[tek] 2024
 
DSPy a system for AI to Write Prompts and Do Fine Tuning
DSPy a system for AI to Write Prompts and Do Fine TuningDSPy a system for AI to Write Prompts and Do Fine Tuning
DSPy a system for AI to Write Prompts and Do Fine Tuning
 
DMCC Future of Trade Web3 - Special Edition
DMCC Future of Trade Web3 - Special EditionDMCC Future of Trade Web3 - Special Edition
DMCC Future of Trade Web3 - Special Edition
 
E-Vehicle_Hacking_by_Parul Sharma_null_owasp.pptx
E-Vehicle_Hacking_by_Parul Sharma_null_owasp.pptxE-Vehicle_Hacking_by_Parul Sharma_null_owasp.pptx
E-Vehicle_Hacking_by_Parul Sharma_null_owasp.pptx
 
Gen AI in Business - Global Trends Report 2024.pdf
Gen AI in Business - Global Trends Report 2024.pdfGen AI in Business - Global Trends Report 2024.pdf
Gen AI in Business - Global Trends Report 2024.pdf
 
"LLMs for Python Engineers: Advanced Data Analysis and Semantic Kernel",Oleks...
"LLMs for Python Engineers: Advanced Data Analysis and Semantic Kernel",Oleks..."LLMs for Python Engineers: Advanced Data Analysis and Semantic Kernel",Oleks...
"LLMs for Python Engineers: Advanced Data Analysis and Semantic Kernel",Oleks...
 
Dev Dives: Streamline document processing with UiPath Studio Web
Dev Dives: Streamline document processing with UiPath Studio WebDev Dives: Streamline document processing with UiPath Studio Web
Dev Dives: Streamline document processing with UiPath Studio Web
 
SAP Build Work Zone - Overview L2-L3.pptx
SAP Build Work Zone - Overview L2-L3.pptxSAP Build Work Zone - Overview L2-L3.pptx
SAP Build Work Zone - Overview L2-L3.pptx
 
Hyperautomation and AI/ML: A Strategy for Digital Transformation Success.pdf
Hyperautomation and AI/ML: A Strategy for Digital Transformation Success.pdfHyperautomation and AI/ML: A Strategy for Digital Transformation Success.pdf
Hyperautomation and AI/ML: A Strategy for Digital Transformation Success.pdf
 
Nell’iperspazio con Rocket: il Framework Web di Rust!
Nell’iperspazio con Rocket: il Framework Web di Rust!Nell’iperspazio con Rocket: il Framework Web di Rust!
Nell’iperspazio con Rocket: il Framework Web di Rust!
 
CloudStudio User manual (basic edition):
CloudStudio User manual (basic edition):CloudStudio User manual (basic edition):
CloudStudio User manual (basic edition):
 
Anypoint Exchange: It’s Not Just a Repo!
Anypoint Exchange: It’s Not Just a Repo!Anypoint Exchange: It’s Not Just a Repo!
Anypoint Exchange: It’s Not Just a Repo!
 
DevoxxFR 2024 Reproducible Builds with Apache Maven
DevoxxFR 2024 Reproducible Builds with Apache MavenDevoxxFR 2024 Reproducible Builds with Apache Maven
DevoxxFR 2024 Reproducible Builds with Apache Maven
 
DevEX - reference for building teams, processes, and platforms
DevEX - reference for building teams, processes, and platformsDevEX - reference for building teams, processes, and platforms
DevEX - reference for building teams, processes, and platforms
 
Story boards and shot lists for my a level piece
Story boards and shot lists for my a level pieceStory boards and shot lists for my a level piece
Story boards and shot lists for my a level piece
 
How to write a Business Continuity Plan
How to write a Business Continuity PlanHow to write a Business Continuity Plan
How to write a Business Continuity Plan
 
Unleash Your Potential - Namagunga Girls Coding Club
Unleash Your Potential - Namagunga Girls Coding ClubUnleash Your Potential - Namagunga Girls Coding Club
Unleash Your Potential - Namagunga Girls Coding Club
 

Apache Flink Deep Dive

  • 1. Apache Flink™ deep-dive Unified Batch and Stream Processing Robert Metzger @rmetzger_
  • 2. Flink’s Recent History April 2014 April 2015Dec 2014 Top Level Project Graduation 0.70.60.5 0.90.9-m1
  • 3. What is Flink 3 Gelly Table ML SAMOA DataSet (Java/Scala) DataStream HadoopM/R Local Remote YARN Tez Embedded Dataflow Dataflow(WiP) MRQL Table Cascading(WiP) Streaming dataflow runtime Zeppelin
  • 4. Program compilation 4 case class Path (from: Long, to: Long) val tc = edges.iterate(10) { paths: DataSet[Path] => val next = paths .join(edges) .where("to") .equalTo("from") { (path, edge) => Path(path.from, edge.to) } .union(paths) .distinct() next } Optimizer Type extraction stack Task scheduling Dataflow metadata Pre-flight (Client) Master Workers Data Source orders.tbl Filter Map DataSource lineitem.tbl Join Hybrid Hash buildHT probe hash-part [0] hash-part [0] GroupRed sort forward Program Dataflow Graph Independent of batch or streaming job deploy operators track intermediate results  Layered Architecture allows plugging of components
  • 5. Native workload support 5 Flink Streaming topologies Long batch pipelines Machine Learning at scale How can an engine natively support all these workloads? And what does "native" mean? Graph Analysis  Low latency  resource utilization  iterative algorithms  Mutable state
  • 6. E.g.: Non-native iterations 6 Step Step Step Step Step Client for (int i = 0; i < maxIterations; i++) { // Execute MapReduce job }  Teaching an old elephant new tricks  Treat system as a black box
  • 7. E.g.: Non-native streaming 7 stream discretizer Job Job Job Job while (true) { // get next few records // issue batch job } Data Stream  Simulate stream processor with batch system
  • 8. Native workload support 8 Flink Streaming topologies Long batch pipelines Machine Learning at scale How can an engine natively support all these workloads? And what does "native" mean? Graph Analysis  Low latency  resource utilization  iterative algorithms  Mutable state
  • 9. Ingredients for “native” support 1. Execute everything as streams Pipelined execution, push model 2. Special code paths for batch Automatic job optimization, fault tolerance 3. Allow some iterative (cyclic) dataflows 4. Allow some mutable state 5. Operate on managed memory Make data processing on the JVM robust 9
  • 10. Flink by Use Case 10
  • 11. Stream data processing streaming dataflows 11 Full talk tomorrow: 3:10PM, Grand Ballroom 220A Stream processing with Flink
  • 12. Pipelined stream processor 12 Streaming Shuffle!  Low latency  Operators push data forward
  • 13. Expressive APIs 13 case class Word (word: String, frequency: Int) val lines: DataStream[String] = env.fromSocketStream(...) lines.flatMap {line => line.split(" ").map(word => Word(word,1))} .window(Time.of(5,SECONDS)).every(Time.of(1,SECONDS)) .groupBy("word").sum("frequency") .print() val lines: DataSet[String] = env.readTextFile(...) lines.flatMap {line => line.split(" ").map(word => Word(word,1))} .groupBy("word").sum("frequency") .print() DataSet API (batch): DataStream API (streaming):
  • 14. Checkpointing / Recovery 14 Chandy-Lamport Algorithm for consistent asynchronous distributed snapshots Pushes checkpoint barriers through the data flow Data Stream barrier Before barrier = part of the snapshot After barrier = Not in snapshot (backup till next snapshot)  Guarantees exactly-once processing
  • 16. Batch on an streaming engine 16 File in HDFS Filter Map Result 1 Map Result 2  Batch program, completely pipelined  Data is never materialized anywhere (in this example)
  • 17. Batch on an streaming engine Map Operator Map Operator Map Operator 17 Data Source (small) Stream Data Sink Data Sink Data Sink Join Operator in parallel Data Source (large) Data Sink in parallel (once build side finished) Map
  • 18. Batch processing requirements  Get the data processed as fast as possible • Automatic job optimizer • Efficient memory management  Robust processing • provide fault-tolerance • again, memory management 18
  • 19. Optimizer  Cost-based optimizer  Select data shipping strategy (forward, partition, broadcast)  Local execution (sort merge join/hash join)  Caching of loop invariant data (iterations) 19 case class Path (from: Long, to: Long) val tc = edges.iterate(10) { paths: DataSet[Path] => val next = paths .join(edges) .where("to") .equalTo("from") { (path, edge) => Path(path.from, edge.to) } .union(paths) .distinct() next } Optimizer Type extraction stack Pre-flight (Client) Data Source orders.tbl Filter Map DataSourc e lineitem.tbl Join Hybrid Hash build HT probe hash-part [0] hash-part [0] GroupRed sort forward Program Dataflow Graph
  • 20. Two execution plans 20 DataSource orders.tbl Filter Map DataSource lineitem.tbl Join Hybrid Hash buildHT probe broadcast forward Combine GroupRed sort DataSource orders.tbl Filter Map DataSource lineitem.tbl Join Hybrid Hash buildHT probe hash-part [0] hash-part [0] hash-part [0,1] GroupRed sort forward Best plan depends on relative sizes of input files
  • 22. Operators on managed memory 22
  • 23. Smooth out-of-core performance 23 More at: http://flink.apache.org/news/2015/03/13/peeking-into-Apache-Flinks-Engine-Room.html Blue bars are in-memory, orange bars (partially) out-of-core
  • 25. Iterate in the Dataflow 26  API and runtime support  Automatic caching of loop invariant data IterationState state = getInitialState(); while (!terminationCriterion()) { state = step(state); } setFinalState(state);
  • 26. Example: Matrix Factorization 27 Factorizing a matrix with 28 billion ratings for recommendations More at: http://data-artisans.com/computing-recommendations-with-flink.html Setups: • 40 medium instances ("n1-highmem-8" - 8 cores, 52 GB) • 40 large instances ("n1-highmem-16" - 16 cores, 104 GB)
  • 27. Flink ML – Machine Learning  Provide a complete toolchain • scikit-learn style pipelining • Data pre-processing  various algorithms • Recommendations: ALS • Supervised learning: Support Vector Machines • …  ML on streams: SAMOA. We are planning to add support for streaming into ML 28
  • 30. Iterate natively with state/deltas 31  Keep state in an controlled way by having a partitioned hash- map  Relax immutability assumption of batch processing
  • 31. … fast graph analysis 32More at: http://data-artisans.com/data-analysis-with-flink.html
  • 32. Gelly – Graph Processing API 33  Transformations: map, filter, subgraph, union, reverse, undirected  Mutations: add vertex/edge, remove …  Pregel style vertex centric iterations  Library of algorithms  Utilities: Special data types, loading, graph properties
  • 33. Gelly and Flink ML:  Available in Flink 0.9 (so far only beta release)  Still under heavy development  Seamlessly integrate with DataSet abstraction Preprocess data as needed Use results as needed  Easy entry point for new contributors 34
  • 35. Flink Meetup Groups  SF Spark and Friends • June 16, San Francisco  Bay Area Flink Meetup • June 17, Redwood City  Chicago Flink Meetup • June 30  Stockholm, Sweden  Berlin, Germany 36
  • 36. Flink Forward registration & call for abstracts is open now flink.apache.org 37 • 12/13 October 2015 • Meet developers and users of Flink! • With Flink Workshops / Trainings!
  • 39. Flink community 0 20 40 60 80 100 120 Aug-10 Feb-11 Sep-11 Apr-12 Oct-12 May-13 Nov-13 Jun-14 Dec-14 Jul-15 #unique contributor ids by git commits
  • 40. Flink Historic data Kafka, RabbitMQ, ... HDFS, JDBC, ... ETL, Graphs, Machine Learning Relational, … Low latency, windowing, aggregations, ... Event logs Real-time data streams What is Apache Flink? (master)
  • 41. Cornerpoints of Flink Design 42 Robust Algorithms on Managed Memory Pipelined Execution of Batch Programs  Better shuffle performance  No OutOfMemory Errors  Scales to very large JVMs  Efficient an robust processing Flexible Data Streaming Engine  Low Latency Steam Proc.  Highly flexible windows Native Iterations  Very fast Graph Processing  Stateful Iterations for ML High-level APIs, beyond key/value pairs  Java/Scala/Python (upcoming)  Relational-style optimizer  Graphs / Machine Learning  Streaming ML (coming)  Scales to very large groups Active Library Development
  • 42. Defining windows in Flink  Trigger policy • When to trigger the computation on current window  Eviction policy • When data points should leave the window • Defines window width/size  E.g., count-based policy • evict when #elements > n • start a new window every n-th element  Built-in: Count, Time, Delta policies 43
  • 44. 45
  • 45. 46
  • 47. A simple program 48 val orders = … val lineitems = … val filteredOrders = orders .filter(o => dataFormat.parse(l.shipDate).after(date)) .filter(o => o.shipPrio > 2) val lineitemsOfOrders = filteredOrders .join(lineitems) .where(“orderId”).equalTo(“orderId”) .apply((o,l) => new SelectedItem(o.orderDate, l.extdPrice)) val priceSums = lineitemsOfOrders .groupBy(“orderDate”).sum(“l.extdPrice”);
  • 48. Two execution plans 49 DataSource orders.tbl Filter Map DataSource lineitem.tbl Join Hybrid Hash buildHT probe broadcast forward Combine GroupRed sort DataSource orders.tbl Filter Map DataSource lineitem.tbl Join Hybrid Hash buildHT probe hash-part [0] hash-part [0] hash-part [0,1] GroupRed sort forwardBest plan depends on relative sizes of input files
  • 49. Examples of optimization  Task chaining • Coalesce map/filter/etc tasks  Join optimizations • Broadcast/partition, build/probe side, hash or sort-merge  Interesting properties • Re-use partitioning and sorting for later operations  Automatic caching • E.g., for iterations 50
  • 54. Batch processing 2 Batch on Streaming 55
  • 59. Smooth out-of-core performance 61 More at: http://flink.apache.org/news/2015/03/13/peeking-into-Apache-Flinks-Engine-Room.html Blue bars are in-memory, orange bars (partially) out-of-core

Editor's Notes

  1. 40 minutes
  2. Flink is an entire software stack the heart: streaming dataflow engine: think of programs as operators and data flows Kappa architecture: run batch programs on a streaming system Table API: logical representation, sql-style Samoa “on-line learners”
  3. toy program: native transitive closure type extraction: types that go in and out of each operator
  4. Flink is an analytical system streaming topology: real-time; low latency “native”: build-in support in the system, no working around, no black-box next slide: define native by some “non-native” examples
  5. Used for Machine Learning run the same job over the data multiple times to come up with parameters for a ml model this is how you do it when treating the engine as a black box
  6. If you only have a batch processor: do a lot of small batch jobs LIMITATION: state across the small jobs (batches)
  7. Flink is an analytical system streaming topology: real-time; low latency “native”: build-in support in the system, no working around, no black-box next slide: define native by some “non-native” examples
  8. Corner points / requirements for flink keep data in motion, avoid materialization even though it’s a streaming runtime, have special paths for batch: OPTIMIZER, CHECKPOINTING make the system aware of cyclic data flows, in a controlled way allow operators to have some state, in a controlled way (DELTA-ITERATIONS). relax “traditional” batch assumption flink runs in the jvm, but we want control over memory, not rely on GC
  9. Explain flink by use case
  10. pipelined execution: logical way you would go for low latency. no synchronization barriers, records keep flowing streaming shuffle (for example w/ hash code) push model maintain state inside long lived operators (!= mini batch)
  11. nice, fluent APIs known from the batch world window definitions
  12. Low overhead snapshots using “batched” snapshots Exactly once processing guarantees (without doing mini batches) How does it work: periodically push barriers through the streams. when a barrier reaches an operator, snapshot state when barrier reaches sinks, a checkpoint is completed (secured) multiple parallel checkpoints chop stream into generations (pre checkpointed, post checkpointed)
  13. structure, different title
  14. Measure effect of pipeline parallelism blocking happens in operators (join build side)
  15. in flink operators are running at the same time  need to control memory
  16. example: hash join robust in memory graceful behavior
  17. needed for machine learning
  18. function that encapsulate the transformation “Hadoop job” deploy this once, keep running across iterations (also you can keep state) allow to feed back data to the beginning
  19. 1 Tb of input data many terabytes of intermediate data 40 machines cluster @ google compute
  20. SVM = supervised learning ALS = Recommendation
  21. keep mutable state in a controlled way by having a hash map locally on each machine great documentation on flink website
  22. Weakly connected components, page rank, label propagation,
  23. dev list: 300-400 messages/month. record 1000 messages on
  24. structure, different title
  25. visualization of dataflow graph in flink Explain: sources, maps, binary operators (join, …) This is an actual example of a Flink job we’ve seen from an industry user candidate flights out of all available flights
  26. you need windows in stream programs: grouping for ex on infinite stream is impossible pipelined data ex: needed for low latency in batch: pipelined or blocking, if we are optimizing (for ex we don’t want all operators online at the same time)
  27. overlapping operators operators start as soon as data is available join and co-group start early, (co group starts sorting incoming data)
  28. in flink operators are running at the same time  need to control memory
  29. example: hash join robust in memory graceful behavior