SlideShare une entreprise Scribd logo
1  sur  52
Stream Processing with
Apache Flink
Robert Metzger
@rmetzger_
rmetzger@apache.org
GOTO Night Amsterdam,
March 10, 2016
Talk overview
 My take on the stream processing space,
and how it changes the way we think
about data
 Discussion of unique building blocks of
Flink
 Benchmarking Flink, by extending a
benchmark from Yahoo!
2
Apache Flink
 Apache Flink is an open source stream
processing framework
• Low latency
• High throughput
• Stateful
• Distributed
 Developed at the Apache Software
Foundation, 1.0.0 release available soon,
used in production
3
Entering the streaming era
4
5
Streaming is the biggest change in
data infrastructure since Hadoop
6
1. Radically simplified infrastructure
2. Do more with your data, faster
3. Can completely subsume batch
Traditional data processing
7
Web server
Logs
Web server
Logs
Web server
Logs
HDFS / S3
Periodic (custom) or
continuous ingestion
(Flume) into HDFS
Batch job(s) for
log analysis
Periodic log analysis
job
Serving
layer
 Log analysis example using a batch
processor
Job scheduler
(Oozie)
Traditional data processing
8
Web server
Logs
Web server
Logs
Web server
Logs
HDFS / S3
Periodic (custom) or
continuous ingestion
(Flume) into HDFS
Batch job(s) for
log analysis
Periodic log analysis
job
Serving
layer
 Latency from log event to serving layer
usually in the range of hours
every 2 hrs
Job scheduler
(Oozie)
Data processing without stream
processor
9
Web server
Logs
Web server
Logs
HDFS / S3
Batch job(s) for
log analysis
 This architecture is a hand-crafted micro-
batch model
Batch interval: ~2 hours
hours minutes milliseconds
Manually triggered
periodic batch job
Batch processor
with micro-batches
Latency
Approach
seconds
Stream processor
Downsides of stream processing with a
batch engine
 Very high latency (hours)
 Complex architecture required:
• Periodic job scheduler (e.g. Oozie)
• Data loading into HDFS (e.g. Flume)
• Batch processor
• (When using the “lambda architecture”: a stream
processor)
All these components need to be
implemented and maintained
 Backpressure: How does the pipeline handle
load spikes?
10
Log event analysis using a
stream processor
11
Web server
Web server
Web server
High throughput
publish/subscribe
bus
Serving
layer
 Stream processors allow to analyze
events with sub-second latency.
Options:
• Apache Kafka
• Amazon Kinesis
• MapR Streams
• Google Cloud Pub/Sub
Forward events
immediately to
pub/sub bus
Stream Processor
Options:
• Apache Flink
• Apache Beam
• Apache Samza
Process events in real
time & update
serving layer
12
Real-world data is produced in a
continuous fashion.
New systems like Flink and Kafka
embrace streaming nature of data.
Web server Kafka topic
Stream processor
What do we need for replacing
the “batch stack”?
13
Web server
Web server
Web server
High throughput
publish/subscribe
bus
Serving
layer
Options:
• Apache Kafka
• Amazon Kinesis
• MapR Streams
• Google Cloud Pub/Sub
Forward events
immediately to
pub/sub bus
Stream Processor
Options:
• Apache Flink
• Google Cloud
Dataflow
Process events in real
time & update
serving layer
Low latency
High throughput
State handling
Windowing / Out
of order events
Fault tolerance
and correctness
Apache Flink stack
15
Gelly
Table
ML
SAMOA
DataSet (Java/Scala)DataStream (Java / Scala)
HadoopM/R
LocalClusterYARN
ApacheBeam
ApacheBeam
Table
Cascading
Streaming dataflow runtime
StormAPI
Zeppelin
CEP
Needed for the use case
16
Gelly
Table
ML
SAMOA
DataSet (Java/Scala)DataStream (Java / Scala)
HadoopM/R
LocalClusterYARN
ApacheBeam
ApacheBeam
Table
Cascading
Streaming dataflow runtime
StormAPI
Zeppelin
CEP
Windowing / Out of order
events
17
Low latency
High throughput
State handling
Windowing / Out
of order events
Fault tolerance
and correctness
Building windows from a stream
18
 “Number of visitors in the last 5 minutes
per country”
Web server Kafka topic
Stream processor
// create stream from Kafka source
DataStream<LogEvent> stream = env.addSource(new KafkaConsumer());
// group by country
DataStream<LogEvent> keyedStream = stream.keyBy(“country“);
// window of size 5 minutes
keyedStream.timeWindow(Time.minutes(5))
// do operations per window
.apply(new CountPerWindowFunction());
Building windows: Execution
19
Kafka
Source
Window
Operator
S
S
S
W
W
W
group by
country
// window of size 5 minutes
keyedStream.timeWindow(Time.minutes(5));
Job plan Parallel execution on the cluster
Time
Window types in Flink
 Tumbling windows
 Sliding windows
 Custom windows with window assigners,
triggers and evictors
20Further reading: http://flink.apache.org/news/2015/12/04/Introducing-windows.html
Time-based windows
21
Stream
Time of event
Event data
{
“accessTime”: “1457002134”,
“userId”: “1337”,
“userLocation”: “UK”
}
 Windows are created based on the real
world time when the event occurred
A look at the reality of time
22
Kafka
Network delays
Out of sync clocks
33 11 21 15 9
 Events arrive out of order in the system
 Use-case specific low watermarks for time
tracking
Window between
0 and 15
Stream Processor
15
Guarantee that no event with time
<= 15 will arrive afterwards
Time characteristics in Apache Flink
 Event Time
• Users have to specify an event-time extractor +
watermark emitter
• Results are deterministic, but with latency
 Processing Time
• System time is used when evaluating windows
• low latency
 Ingestion Time
• Flink assigns current system time at the sources
 Pluggable, without window code changes
23
State handling
24
Low latency
High throughput
State handling
Windowing / Out
of order events
Fault tolerance
and correctness
State in streaming
 Where do we store the elements from our
windows?
 In stateless systems, an external state
store (e.g. Redis) is needed.
25
S
S
S
W
W
W
Time
Elements in windows
are state
Stream processor: Flink
Managed state in Flink
 Flink automatically backups and restores state
 State can be larger than the available memory
 State backends: (embedded) RocksDB, Heap
memory
26
Operator with windows
(large state)
State
backend
(local)
Distributed
File System
Periodic backup /
recovery
Web
server
Kafka
Managing the state
 How can we operate such a pipeline
24x7?
 Losing state (by stopping the system)
would require a replay of past events
 We need a way to store the state
somewhere!
27
Web server Kafka topic
Stream processor
Savepoints: Versioning state
 Savepoint: Create an addressable copy of a
job’s current state.
 Restart a job from any savepoint.
28
Further reading: http://data-artisans.com/how-apache-flink-enables-new-streaming-applications/
> flink savepoint <JobID>
HDFS
> hdfs:///flink-savepoints/2
> flink run –s hdfs:///flink-savepoints/2 <jar>
HDFS
Fault tolerance and
correctness
29
Low latency
High throughput
State handling
Windowing / Out
of order events
Fault tolerance
and correctness
Fault tolerance in streaming
 How do we ensure the results (number of
visitors) are always correct?
 Failures should not lead to data loss or
incorrect results
30
Web server Kafka topic
Stream processor
Fault tolerance in streaming
 at least once: ensure all operators see all
events
• Storm: Replay stream in failure case (acking
of individual records)
 Exactly once: ensure that operators do
not perform duplicate updates to their
state
• Flink: Distributed Snapshots
• Spark: Micro-batches on batch runtime
31
Flink’s Distributed Snapshots
 Lightweight approach of storing the state
of all operators without pausing the
execution
 Implemented using barriers flowing
through the topology
32
Data Stream
barrier
Before barrier =
part of the snapshot
After barrier =
Not in snapshot
Further reading: http://blog.acolyer.org/2015/08/19/asynchronous-distributed-snapshots-for-
distributed-dataflows/
Wrap-up: Log processing example
 How to do something with the data?
Windowing
 How does the system handle large windows?
Managed state
 How do operate such a system 24x7?
Safepoints
 How to ensure correct results across failures?
Checkpoints, Master HA
33
Web server Kafka topic
Stream processor
Performance:
Low Latency & High Throughput
34
Low latency
High throughput
State handling
Windowing / Out
of order events
Fault tolerance
and correctness
Performance: Introduction
 Performance always depends on your own
use cases, so test it yourself!
 We based our experiments on a recent
benchmark published by Yahoo!
 They benchmarked Storm, Spark
Streaming and Flink with a production use-
case (counting ad impressions)
35
Full Yahoo! article: https://yahooeng.tumblr.com/post/135321837876/benchmarking-streaming-
computation-engines-at
Yahoo! Benchmark
 Count ad impressions grouped by
campaign
 Compute aggregates over a 10 second
window
 Emit current value of window aggregates
to Redis every second for query
36
Full Yahoo! article: https://yahooeng.tumblr.com/post/135321837876/benchmarking-streaming-
computation-engines-at
Yahoo’s Results
“Storm […] and Flink […] show sub-second
latencies at relatively high throughputs with
Storm having the lowest 99th percentile
latency. Spark streaming 1.5.1 supports high
throughputs, but at a relatively higher
latency.”
(Quote from the blog post’s executive summary)
37
Full Yahoo! article: https://yahooeng.tumblr.com/post/135321837876/benchmarking-streaming-
computation-engines-at
Extending the benchmark
 Benchmark stops at Storm’s throughput
limits. Where is Flink’s limit?
 How will Flink’s own window
implementation perform compared to
Yahoo’s “state in redis windowing”
approach?
38
Full Yahoo! article: https://yahooeng.tumblr.com/post/135321837876/benchmarking-streaming-
computation-engines-at
Windowing with state in Redis
39
KafkaConsumer
map()
filter()
group
windowing &
caching code
realtime queries
Rewrite to use Flink’s own window
40
KafkaConsumer
map()
filter()
group
Flink event
time
windows
realtime queries
Results after rewrite
41
0 750,000 1,500,000 2,250,000 3,000,000 3,750,000
Storm
Flink
Throughput: msgs/sec
400k msgs/sec
Can we even go further?
42
KafkaConsumer
map()
filter()
group
Flink event
time
windows
Network link to
Kafka cluster is
bottleneck!
(1GigE)
Data Generator
map()
filter()
group
Flink event
time
windows
Solution: Move
data generator
into job (10 GigE)
Results without network bottleneck
43
0 4,000,000 8,000,000 12,000,000 16,000,000
Storm
Flink
Flink (10 GigE)
Throughput: msgs/sec
10 GigE end-to-end
15m msgs/sec
400k msgs/sec
3m msgs/sec
Benchmark summary
 Flink achieves throughput of 15 million
messages/second on 10 machines
 35x higher throughput compared to
Storm (80x compared to Yahoo’s runs)
 Flink ran with exactly once guarantees,
Storm with at least once.
 Read the full report: http://data-
artisans.com/extending-the-yahoo-
streaming-benchmark/
44
Closing
45
Other notable features
 Expressive DataStream API (similar to high
level APIs known from the batch world)
 Flink is a full-fledged batch processor with
an optimizer, managed memory, memory-
aware algorithms, build-in iterations
 Many libraries: Complex Event Processing
(CEP), Graph Processing, Machine Learning
 Integration with YARN, HBase,
ElasticSearch, Kafka, MapReduce, …
46
Questions?
 Ask now!
 eMail: rmetzger@apache.org
 Twitter: @rmetzger_
 Follow: @ApacheFlink
 Read: flink.apache.org/blog, data-
artisans.com/blog/
 Mailinglists: (news | user | dev)@flink.apache.org
47
Apache Flink stack
48
Gelly
Table
ML
SAMOA
DataSet (Java/Scala)DataStream (Java / Scala)
HadoopM/R
LocalClusterYARN
ApacheBeam
ApacheBeam
Table
Cascading
Streaming dataflow runtime
StormAPI
Zeppelin
CEP
Appendix
49
Roadmap 2016
50
 SQL / StreamSQL
 CEP Library
 Managed Operator State
 Dynamic Scaling
 Miscellaneous
Miscellaneous
 Support for Apache Mesos
 Security
• Over-the-wire encryption of RPC (akka) and data
transfers (netty)
 More connectors
• Apache Cassandra
• Amazon Kinesis
 Enhance metrics
• Throughput / Latencies
• Backpressure monitoring
• Spilling / Out of Core
51
Handling Backpressure
52
Slow down
upstream
operators
Backpressure might occur when:
• Operators create checkpoints
• Windows are evaluated
• Operators depend on external
resources
• JVMs do Garbage Collection
Operator not able
to process
incoming data
immediately
Handling Backpressure
53
Sender
Sender
Receiver
Receiver
Sender does not have any
empty buffers available:
Slowdown
Network transfer (Netty) or
local buffer exchange
(when S and R are on the
same machine)
• Data sources slow down pulling data from their underlying
system (Kafka or similar queues)
Full buffer
Empty buffer

Contenu connexe

Tendances

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
 
Stream Processing with Apache Flink
Stream Processing with Apache FlinkStream Processing with Apache Flink
Stream Processing with Apache FlinkC4Media
 
Apache Flink Berlin Meetup May 2016
Apache Flink Berlin Meetup May 2016Apache Flink Berlin Meetup May 2016
Apache Flink Berlin Meetup May 2016Stephan Ewen
 
Stream Processing with Apache Flink (Flink.tw Meetup 2016/07/19)
Stream Processing with Apache Flink (Flink.tw Meetup 2016/07/19)Stream Processing with Apache Flink (Flink.tw Meetup 2016/07/19)
Stream Processing with Apache Flink (Flink.tw Meetup 2016/07/19)Apache Flink Taiwan User Group
 
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
 
Data Stream Processing with Apache Flink
Data Stream Processing with Apache FlinkData Stream Processing with Apache Flink
Data Stream Processing with Apache FlinkFabian Hueske
 
Flink history, roadmap and vision
Flink history, roadmap and visionFlink history, roadmap and vision
Flink history, roadmap and visionStephan Ewen
 
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
 
Taking a look under the hood of Apache Flink's relational APIs.
Taking a look under the hood of Apache Flink's relational APIs.Taking a look under the hood of Apache Flink's relational APIs.
Taking a look under the hood of Apache Flink's relational APIs.Fabian Hueske
 
Till Rohrmann - Dynamic Scaling - How Apache Flink adapts to changing workloads
Till Rohrmann - Dynamic Scaling - How Apache Flink adapts to changing workloadsTill Rohrmann - Dynamic Scaling - How Apache Flink adapts to changing workloads
Till Rohrmann - Dynamic Scaling - How Apache Flink adapts to changing workloadsFlink Forward
 
data Artisans Product Announcement
data Artisans Product Announcementdata Artisans Product Announcement
data Artisans Product AnnouncementFlink Forward
 
Christian Kreuzfeld – Static vs Dynamic Stream Processing
Christian Kreuzfeld – Static vs Dynamic Stream ProcessingChristian Kreuzfeld – Static vs Dynamic Stream Processing
Christian Kreuzfeld – Static vs Dynamic Stream ProcessingFlink Forward
 
Marton Balassi – Stateful Stream Processing
Marton Balassi – Stateful Stream ProcessingMarton Balassi – Stateful Stream Processing
Marton Balassi – Stateful Stream ProcessingFlink Forward
 
Apache Flink: Streaming Done Right @ FOSDEM 2016
Apache Flink: Streaming Done Right @ FOSDEM 2016Apache Flink: Streaming Done Right @ FOSDEM 2016
Apache Flink: Streaming Done Right @ FOSDEM 2016Till Rohrmann
 
Dynamic Scaling: How Apache Flink Adapts to Changing Workloads (at FlinkForwa...
Dynamic Scaling: How Apache Flink Adapts to Changing Workloads (at FlinkForwa...Dynamic Scaling: How Apache Flink Adapts to Changing Workloads (at FlinkForwa...
Dynamic Scaling: How Apache Flink Adapts to Changing Workloads (at FlinkForwa...Till Rohrmann
 
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
 
The Evolution of (Open Source) Data Processing
The Evolution of (Open Source) Data ProcessingThe Evolution of (Open Source) Data Processing
The Evolution of (Open Source) Data ProcessingAljoscha Krettek
 
Dongwon Kim – A Comparative Performance Evaluation of Flink
Dongwon Kim – A Comparative Performance Evaluation of FlinkDongwon Kim – A Comparative Performance Evaluation of Flink
Dongwon Kim – A Comparative Performance Evaluation of FlinkFlink Forward
 
Extending the Yahoo Streaming Benchmark
Extending the Yahoo Streaming BenchmarkExtending the Yahoo Streaming Benchmark
Extending the Yahoo Streaming BenchmarkJamie Grier
 
Kostas Tzoumas_Stephan Ewen - Keynote -The maturing data streaming ecosystem ...
Kostas Tzoumas_Stephan Ewen - Keynote -The maturing data streaming ecosystem ...Kostas Tzoumas_Stephan Ewen - Keynote -The maturing data streaming ecosystem ...
Kostas Tzoumas_Stephan Ewen - Keynote -The maturing data streaming ecosystem ...Flink Forward
 

Tendances (20)

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...
 
Stream Processing with Apache Flink
Stream Processing with Apache FlinkStream Processing with Apache Flink
Stream Processing with Apache Flink
 
Apache Flink Berlin Meetup May 2016
Apache Flink Berlin Meetup May 2016Apache Flink Berlin Meetup May 2016
Apache Flink Berlin Meetup May 2016
 
Stream Processing with Apache Flink (Flink.tw Meetup 2016/07/19)
Stream Processing with Apache Flink (Flink.tw Meetup 2016/07/19)Stream Processing with Apache Flink (Flink.tw Meetup 2016/07/19)
Stream Processing with Apache Flink (Flink.tw Meetup 2016/07/19)
 
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
 
Data Stream Processing with Apache Flink
Data Stream Processing with Apache FlinkData Stream Processing with Apache Flink
Data Stream Processing with Apache Flink
 
Flink history, roadmap and vision
Flink history, roadmap and visionFlink history, roadmap and vision
Flink history, roadmap and vision
 
Apache Flink - Overview and Use cases of a Distributed Dataflow System (at pr...
Apache Flink - Overview and Use cases of a Distributed Dataflow System (at pr...Apache Flink - Overview and Use cases of a Distributed Dataflow System (at pr...
Apache Flink - Overview and Use cases of a Distributed Dataflow System (at pr...
 
Taking a look under the hood of Apache Flink's relational APIs.
Taking a look under the hood of Apache Flink's relational APIs.Taking a look under the hood of Apache Flink's relational APIs.
Taking a look under the hood of Apache Flink's relational APIs.
 
Till Rohrmann - Dynamic Scaling - How Apache Flink adapts to changing workloads
Till Rohrmann - Dynamic Scaling - How Apache Flink adapts to changing workloadsTill Rohrmann - Dynamic Scaling - How Apache Flink adapts to changing workloads
Till Rohrmann - Dynamic Scaling - How Apache Flink adapts to changing workloads
 
data Artisans Product Announcement
data Artisans Product Announcementdata Artisans Product Announcement
data Artisans Product Announcement
 
Christian Kreuzfeld – Static vs Dynamic Stream Processing
Christian Kreuzfeld – Static vs Dynamic Stream ProcessingChristian Kreuzfeld – Static vs Dynamic Stream Processing
Christian Kreuzfeld – Static vs Dynamic Stream Processing
 
Marton Balassi – Stateful Stream Processing
Marton Balassi – Stateful Stream ProcessingMarton Balassi – Stateful Stream Processing
Marton Balassi – Stateful Stream Processing
 
Apache Flink: Streaming Done Right @ FOSDEM 2016
Apache Flink: Streaming Done Right @ FOSDEM 2016Apache Flink: Streaming Done Right @ FOSDEM 2016
Apache Flink: Streaming Done Right @ FOSDEM 2016
 
Dynamic Scaling: How Apache Flink Adapts to Changing Workloads (at FlinkForwa...
Dynamic Scaling: How Apache Flink Adapts to Changing Workloads (at FlinkForwa...Dynamic Scaling: How Apache Flink Adapts to Changing Workloads (at FlinkForwa...
Dynamic Scaling: How Apache Flink Adapts to Changing Workloads (at FlinkForwa...
 
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
 
The Evolution of (Open Source) Data Processing
The Evolution of (Open Source) Data ProcessingThe Evolution of (Open Source) Data Processing
The Evolution of (Open Source) Data Processing
 
Dongwon Kim – A Comparative Performance Evaluation of Flink
Dongwon Kim – A Comparative Performance Evaluation of FlinkDongwon Kim – A Comparative Performance Evaluation of Flink
Dongwon Kim – A Comparative Performance Evaluation of Flink
 
Extending the Yahoo Streaming Benchmark
Extending the Yahoo Streaming BenchmarkExtending the Yahoo Streaming Benchmark
Extending the Yahoo Streaming Benchmark
 
Kostas Tzoumas_Stephan Ewen - Keynote -The maturing data streaming ecosystem ...
Kostas Tzoumas_Stephan Ewen - Keynote -The maturing data streaming ecosystem ...Kostas Tzoumas_Stephan Ewen - Keynote -The maturing data streaming ecosystem ...
Kostas Tzoumas_Stephan Ewen - Keynote -The maturing data streaming ecosystem ...
 

En vedette

Data-Ed Webinar: Data Governance Strategies
Data-Ed Webinar: Data Governance StrategiesData-Ed Webinar: Data Governance Strategies
Data-Ed Webinar: Data Governance StrategiesDATAVERSITY
 
DevOps Powered by Splunk
DevOps Powered by SplunkDevOps Powered by Splunk
DevOps Powered by SplunkSplunk
 
Tips to achieve continuous integration/delivery using HP ALM, Jenkins, and S...
 Tips to achieve continuous integration/delivery using HP ALM, Jenkins, and S... Tips to achieve continuous integration/delivery using HP ALM, Jenkins, and S...
Tips to achieve continuous integration/delivery using HP ALM, Jenkins, and S...Skytap Cloud
 
Business intelligence 3.0 and the data lake
Business intelligence 3.0 and the data lakeBusiness intelligence 3.0 and the data lake
Business intelligence 3.0 and the data lakeData Science Thailand
 
Log Mining: Beyond Log Analysis
Log Mining: Beyond Log AnalysisLog Mining: Beyond Log Analysis
Log Mining: Beyond Log AnalysisAnton Chuvakin
 
Realtime Reporting using Spark Streaming
Realtime Reporting using Spark StreamingRealtime Reporting using Spark Streaming
Realtime Reporting using Spark StreamingSantosh Sahoo
 
Production and Beyond: Deploying and Managing Machine Learning Models
Production and Beyond: Deploying and Managing Machine Learning ModelsProduction and Beyond: Deploying and Managing Machine Learning Models
Production and Beyond: Deploying and Managing Machine Learning ModelsTuri, Inc.
 
Real Time Data Processing using Spark Streaming | Data Day Texas 2015
Real Time Data Processing using Spark Streaming | Data Day Texas 2015Real Time Data Processing using Spark Streaming | Data Day Texas 2015
Real Time Data Processing using Spark Streaming | Data Day Texas 2015Cloudera, Inc.
 
Real-Time Analytics with Apache Cassandra and Apache Spark
Real-Time Analytics with Apache Cassandra and Apache SparkReal-Time Analytics with Apache Cassandra and Apache Spark
Real-Time Analytics with Apache Cassandra and Apache SparkGuido Schmutz
 

En vedette (9)

Data-Ed Webinar: Data Governance Strategies
Data-Ed Webinar: Data Governance StrategiesData-Ed Webinar: Data Governance Strategies
Data-Ed Webinar: Data Governance Strategies
 
DevOps Powered by Splunk
DevOps Powered by SplunkDevOps Powered by Splunk
DevOps Powered by Splunk
 
Tips to achieve continuous integration/delivery using HP ALM, Jenkins, and S...
 Tips to achieve continuous integration/delivery using HP ALM, Jenkins, and S... Tips to achieve continuous integration/delivery using HP ALM, Jenkins, and S...
Tips to achieve continuous integration/delivery using HP ALM, Jenkins, and S...
 
Business intelligence 3.0 and the data lake
Business intelligence 3.0 and the data lakeBusiness intelligence 3.0 and the data lake
Business intelligence 3.0 and the data lake
 
Log Mining: Beyond Log Analysis
Log Mining: Beyond Log AnalysisLog Mining: Beyond Log Analysis
Log Mining: Beyond Log Analysis
 
Realtime Reporting using Spark Streaming
Realtime Reporting using Spark StreamingRealtime Reporting using Spark Streaming
Realtime Reporting using Spark Streaming
 
Production and Beyond: Deploying and Managing Machine Learning Models
Production and Beyond: Deploying and Managing Machine Learning ModelsProduction and Beyond: Deploying and Managing Machine Learning Models
Production and Beyond: Deploying and Managing Machine Learning Models
 
Real Time Data Processing using Spark Streaming | Data Day Texas 2015
Real Time Data Processing using Spark Streaming | Data Day Texas 2015Real Time Data Processing using Spark Streaming | Data Day Texas 2015
Real Time Data Processing using Spark Streaming | Data Day Texas 2015
 
Real-Time Analytics with Apache Cassandra and Apache Spark
Real-Time Analytics with Apache Cassandra and Apache SparkReal-Time Analytics with Apache Cassandra and Apache Spark
Real-Time Analytics with Apache Cassandra and Apache Spark
 

Similaire à GOTO Night Amsterdam - Stream processing with Apache Flink

Flexible and Real-Time Stream Processing with Apache Flink
Flexible and Real-Time Stream Processing with Apache FlinkFlexible and Real-Time Stream Processing with Apache Flink
Flexible and Real-Time Stream Processing with Apache FlinkDataWorks Summit
 
Flink Streaming @BudapestData
Flink Streaming @BudapestDataFlink Streaming @BudapestData
Flink Streaming @BudapestDataGyula Fóra
 
Apache Flink(tm) - A Next-Generation Stream Processor
Apache Flink(tm) - A Next-Generation Stream ProcessorApache Flink(tm) - A Next-Generation Stream Processor
Apache Flink(tm) - A Next-Generation Stream ProcessorAljoscha Krettek
 
Modern Stream Processing With Apache Flink @ GOTO Berlin 2017
Modern Stream Processing With Apache Flink @ GOTO Berlin 2017Modern Stream Processing With Apache Flink @ GOTO Berlin 2017
Modern Stream Processing With Apache Flink @ GOTO Berlin 2017Till Rohrmann
 
Counting Elements in Streams
Counting Elements in StreamsCounting Elements in Streams
Counting Elements in StreamsJamie Grier
 
Flink forward-2017-netflix keystones-paas
Flink forward-2017-netflix keystones-paasFlink forward-2017-netflix keystones-paas
Flink forward-2017-netflix keystones-paasMonal Daxini
 
Flink 0.10 - Upcoming Features
Flink 0.10 - Upcoming FeaturesFlink 0.10 - Upcoming Features
Flink 0.10 - Upcoming FeaturesAljoscha Krettek
 
Stephan Ewen - Experiences running Flink at Very Large Scale
Stephan Ewen -  Experiences running Flink at Very Large ScaleStephan Ewen -  Experiences running Flink at Very Large Scale
Stephan Ewen - Experiences running Flink at Very Large ScaleVerverica
 
Analitica de datos en tiempo real con Apache Flink y Apache BEAM
Analitica de datos en tiempo real con Apache Flink y Apache BEAMAnalitica de datos en tiempo real con Apache Flink y Apache BEAM
Analitica de datos en tiempo real con Apache Flink y Apache BEAMjavier ramirez
 
Apache Flink at Strata San Jose 2016
Apache Flink at Strata San Jose 2016Apache Flink at Strata San Jose 2016
Apache Flink at Strata San Jose 2016Kostas Tzoumas
 
Stream Data Processing at Big Data Landscape by Oleksandr Fedirko
Stream Data Processing at Big Data Landscape by Oleksandr Fedirko Stream Data Processing at Big Data Landscape by Oleksandr Fedirko
Stream Data Processing at Big Data Landscape by Oleksandr Fedirko GlobalLogic Ukraine
 
Intro to Apache Apex - Next Gen Platform for Ingest and Transform
Intro to Apache Apex - Next Gen Platform for Ingest and TransformIntro to Apache Apex - Next Gen Platform for Ingest and Transform
Intro to Apache Apex - Next Gen Platform for Ingest and TransformApache Apex
 
2018-04 Kafka Summit London: Stephan Ewen - "Apache Flink and Apache Kafka fo...
2018-04 Kafka Summit London: Stephan Ewen - "Apache Flink and Apache Kafka fo...2018-04 Kafka Summit London: Stephan Ewen - "Apache Flink and Apache Kafka fo...
2018-04 Kafka Summit London: Stephan Ewen - "Apache Flink and Apache Kafka fo...Ververica
 
Architecture of Flink's Streaming Runtime @ ApacheCon EU 2015
Architecture of Flink's Streaming Runtime @ ApacheCon EU 2015Architecture of Flink's Streaming Runtime @ ApacheCon EU 2015
Architecture of Flink's Streaming Runtime @ ApacheCon EU 2015Robert Metzger
 
Intro to Apache Apex (next gen Hadoop) & comparison to Spark Streaming
Intro to Apache Apex (next gen Hadoop) & comparison to Spark StreamingIntro to Apache Apex (next gen Hadoop) & comparison to Spark Streaming
Intro to Apache Apex (next gen Hadoop) & comparison to Spark StreamingApache Apex
 
Large-Scale Stream Processing in the Hadoop Ecosystem - Hadoop Summit 2016
Large-Scale Stream Processing in the Hadoop Ecosystem - Hadoop Summit 2016Large-Scale Stream Processing in the Hadoop Ecosystem - Hadoop Summit 2016
Large-Scale Stream Processing in the Hadoop Ecosystem - Hadoop Summit 2016Gyula Fóra
 
Large-Scale Stream Processing in the Hadoop Ecosystem
Large-Scale Stream Processing in the Hadoop Ecosystem Large-Scale Stream Processing in the Hadoop Ecosystem
Large-Scale Stream Processing in the Hadoop Ecosystem DataWorks Summit/Hadoop Summit
 
Chicago Flink Meetup: Flink's streaming architecture
Chicago Flink Meetup: Flink's streaming architectureChicago Flink Meetup: Flink's streaming architecture
Chicago Flink Meetup: Flink's streaming architectureRobert Metzger
 
Kostas Tzoumas - Apache Flink®: State of the Union and What's Next
Kostas Tzoumas - Apache Flink®: State of the Union and What's NextKostas Tzoumas - Apache Flink®: State of the Union and What's Next
Kostas Tzoumas - Apache Flink®: State of the Union and What's NextVerverica
 

Similaire à GOTO Night Amsterdam - Stream processing with Apache Flink (20)

Flexible and Real-Time Stream Processing with Apache Flink
Flexible and Real-Time Stream Processing with Apache FlinkFlexible and Real-Time Stream Processing with Apache Flink
Flexible and Real-Time Stream Processing with Apache Flink
 
Debunking Common Myths in Stream Processing
Debunking Common Myths in Stream ProcessingDebunking Common Myths in Stream Processing
Debunking Common Myths in Stream Processing
 
Flink Streaming @BudapestData
Flink Streaming @BudapestDataFlink Streaming @BudapestData
Flink Streaming @BudapestData
 
Apache Flink(tm) - A Next-Generation Stream Processor
Apache Flink(tm) - A Next-Generation Stream ProcessorApache Flink(tm) - A Next-Generation Stream Processor
Apache Flink(tm) - A Next-Generation Stream Processor
 
Modern Stream Processing With Apache Flink @ GOTO Berlin 2017
Modern Stream Processing With Apache Flink @ GOTO Berlin 2017Modern Stream Processing With Apache Flink @ GOTO Berlin 2017
Modern Stream Processing With Apache Flink @ GOTO Berlin 2017
 
Counting Elements in Streams
Counting Elements in StreamsCounting Elements in Streams
Counting Elements in Streams
 
Flink forward-2017-netflix keystones-paas
Flink forward-2017-netflix keystones-paasFlink forward-2017-netflix keystones-paas
Flink forward-2017-netflix keystones-paas
 
Flink 0.10 - Upcoming Features
Flink 0.10 - Upcoming FeaturesFlink 0.10 - Upcoming Features
Flink 0.10 - Upcoming Features
 
Stephan Ewen - Experiences running Flink at Very Large Scale
Stephan Ewen -  Experiences running Flink at Very Large ScaleStephan Ewen -  Experiences running Flink at Very Large Scale
Stephan Ewen - Experiences running Flink at Very Large Scale
 
Analitica de datos en tiempo real con Apache Flink y Apache BEAM
Analitica de datos en tiempo real con Apache Flink y Apache BEAMAnalitica de datos en tiempo real con Apache Flink y Apache BEAM
Analitica de datos en tiempo real con Apache Flink y Apache BEAM
 
Apache Flink at Strata San Jose 2016
Apache Flink at Strata San Jose 2016Apache Flink at Strata San Jose 2016
Apache Flink at Strata San Jose 2016
 
Stream Data Processing at Big Data Landscape by Oleksandr Fedirko
Stream Data Processing at Big Data Landscape by Oleksandr Fedirko Stream Data Processing at Big Data Landscape by Oleksandr Fedirko
Stream Data Processing at Big Data Landscape by Oleksandr Fedirko
 
Intro to Apache Apex - Next Gen Platform for Ingest and Transform
Intro to Apache Apex - Next Gen Platform for Ingest and TransformIntro to Apache Apex - Next Gen Platform for Ingest and Transform
Intro to Apache Apex - Next Gen Platform for Ingest and Transform
 
2018-04 Kafka Summit London: Stephan Ewen - "Apache Flink and Apache Kafka fo...
2018-04 Kafka Summit London: Stephan Ewen - "Apache Flink and Apache Kafka fo...2018-04 Kafka Summit London: Stephan Ewen - "Apache Flink and Apache Kafka fo...
2018-04 Kafka Summit London: Stephan Ewen - "Apache Flink and Apache Kafka fo...
 
Architecture of Flink's Streaming Runtime @ ApacheCon EU 2015
Architecture of Flink's Streaming Runtime @ ApacheCon EU 2015Architecture of Flink's Streaming Runtime @ ApacheCon EU 2015
Architecture of Flink's Streaming Runtime @ ApacheCon EU 2015
 
Intro to Apache Apex (next gen Hadoop) & comparison to Spark Streaming
Intro to Apache Apex (next gen Hadoop) & comparison to Spark StreamingIntro to Apache Apex (next gen Hadoop) & comparison to Spark Streaming
Intro to Apache Apex (next gen Hadoop) & comparison to Spark Streaming
 
Large-Scale Stream Processing in the Hadoop Ecosystem - Hadoop Summit 2016
Large-Scale Stream Processing in the Hadoop Ecosystem - Hadoop Summit 2016Large-Scale Stream Processing in the Hadoop Ecosystem - Hadoop Summit 2016
Large-Scale Stream Processing in the Hadoop Ecosystem - Hadoop Summit 2016
 
Large-Scale Stream Processing in the Hadoop Ecosystem
Large-Scale Stream Processing in the Hadoop Ecosystem Large-Scale Stream Processing in the Hadoop Ecosystem
Large-Scale Stream Processing in the Hadoop Ecosystem
 
Chicago Flink Meetup: Flink's streaming architecture
Chicago Flink Meetup: Flink's streaming architectureChicago Flink Meetup: Flink's streaming architecture
Chicago Flink Meetup: Flink's streaming architecture
 
Kostas Tzoumas - Apache Flink®: State of the Union and What's Next
Kostas Tzoumas - Apache Flink®: State of the Union and What's NextKostas Tzoumas - Apache Flink®: State of the Union and What's Next
Kostas Tzoumas - Apache Flink®: State of the Union and What's Next
 

Plus de Robert Metzger

How to Contribute to Apache Flink (and Flink at the Apache Software Foundation)
How to Contribute to Apache Flink (and Flink at the Apache Software Foundation)How to Contribute to Apache Flink (and Flink at the Apache Software Foundation)
How to Contribute to Apache Flink (and Flink at the Apache Software Foundation)Robert Metzger
 
Apache Flink Community Updates November 2016 @ Berlin Meetup
Apache Flink Community Updates November 2016 @ Berlin MeetupApache Flink Community Updates November 2016 @ Berlin Meetup
Apache Flink Community Updates November 2016 @ Berlin MeetupRobert Metzger
 
Flink September 2015 Community Update
Flink September 2015 Community UpdateFlink September 2015 Community Update
Flink September 2015 Community UpdateRobert Metzger
 
Click-Through Example for Flink’s KafkaConsumer Checkpointing
Click-Through Example for Flink’s KafkaConsumer CheckpointingClick-Through Example for Flink’s KafkaConsumer Checkpointing
Click-Through Example for Flink’s KafkaConsumer CheckpointingRobert Metzger
 
August Flink Community Update
August Flink Community UpdateAugust Flink Community Update
August Flink Community UpdateRobert Metzger
 
Flink Cummunity Update July (Berlin Meetup)
Flink Cummunity Update July (Berlin Meetup)Flink Cummunity Update July (Berlin Meetup)
Flink Cummunity Update July (Berlin Meetup)Robert Metzger
 
Apache Flink First Half of 2015 Community Update
Apache Flink First Half of 2015 Community UpdateApache Flink First Half of 2015 Community Update
Apache Flink First Half of 2015 Community UpdateRobert Metzger
 
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
 
Berlin Apache Flink Meetup May 2015, Community Update
Berlin Apache Flink Meetup May 2015, Community UpdateBerlin Apache Flink Meetup May 2015, Community Update
Berlin Apache Flink Meetup May 2015, Community UpdateRobert Metzger
 
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
 
Flink Community Update April 2015
Flink Community Update April 2015Flink Community Update April 2015
Flink Community Update April 2015Robert Metzger
 
Apache Flink Community Update March 2015
Apache Flink Community Update March 2015Apache Flink Community Update March 2015
Apache Flink Community Update March 2015Robert Metzger
 
Flink Community Update February 2015
Flink Community Update February 2015Flink Community Update February 2015
Flink Community Update February 2015Robert Metzger
 
Compute "Closeness" in Graphs using Apache Giraph.
Compute "Closeness" in Graphs using Apache Giraph.Compute "Closeness" in Graphs using Apache Giraph.
Compute "Closeness" in Graphs using Apache Giraph.Robert Metzger
 
Stratosphere System Overview Big Data Beers Berlin. 20.11.2013
Stratosphere System Overview Big Data Beers Berlin. 20.11.2013Stratosphere System Overview Big Data Beers Berlin. 20.11.2013
Stratosphere System Overview Big Data Beers Berlin. 20.11.2013Robert Metzger
 
Stratosphere Intro (Java and Scala Interface)
Stratosphere Intro (Java and Scala Interface)Stratosphere Intro (Java and Scala Interface)
Stratosphere Intro (Java and Scala Interface)Robert Metzger
 

Plus de Robert Metzger (18)

How to Contribute to Apache Flink (and Flink at the Apache Software Foundation)
How to Contribute to Apache Flink (and Flink at the Apache Software Foundation)How to Contribute to Apache Flink (and Flink at the Apache Software Foundation)
How to Contribute to Apache Flink (and Flink at the Apache Software Foundation)
 
dA Platform Overview
dA Platform OverviewdA Platform Overview
dA Platform Overview
 
Apache Flink Community Updates November 2016 @ Berlin Meetup
Apache Flink Community Updates November 2016 @ Berlin MeetupApache Flink Community Updates November 2016 @ Berlin Meetup
Apache Flink Community Updates November 2016 @ Berlin Meetup
 
Flink September 2015 Community Update
Flink September 2015 Community UpdateFlink September 2015 Community Update
Flink September 2015 Community Update
 
Click-Through Example for Flink’s KafkaConsumer Checkpointing
Click-Through Example for Flink’s KafkaConsumer CheckpointingClick-Through Example for Flink’s KafkaConsumer Checkpointing
Click-Through Example for Flink’s KafkaConsumer Checkpointing
 
August Flink Community Update
August Flink Community UpdateAugust Flink Community Update
August Flink Community Update
 
Flink Cummunity Update July (Berlin Meetup)
Flink Cummunity Update July (Berlin Meetup)Flink Cummunity Update July (Berlin Meetup)
Flink Cummunity Update July (Berlin Meetup)
 
Apache Flink First Half of 2015 Community Update
Apache Flink First Half of 2015 Community UpdateApache Flink First Half of 2015 Community Update
Apache Flink First Half of 2015 Community Update
 
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 Hands On
Apache Flink Hands OnApache Flink Hands On
Apache Flink Hands On
 
Berlin Apache Flink Meetup May 2015, Community Update
Berlin Apache Flink Meetup May 2015, Community UpdateBerlin Apache Flink Meetup May 2015, Community Update
Berlin Apache Flink Meetup May 2015, Community Update
 
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
 
Flink Community Update April 2015
Flink Community Update April 2015Flink Community Update April 2015
Flink Community Update April 2015
 
Apache Flink Community Update March 2015
Apache Flink Community Update March 2015Apache Flink Community Update March 2015
Apache Flink Community Update March 2015
 
Flink Community Update February 2015
Flink Community Update February 2015Flink Community Update February 2015
Flink Community Update February 2015
 
Compute "Closeness" in Graphs using Apache Giraph.
Compute "Closeness" in Graphs using Apache Giraph.Compute "Closeness" in Graphs using Apache Giraph.
Compute "Closeness" in Graphs using Apache Giraph.
 
Stratosphere System Overview Big Data Beers Berlin. 20.11.2013
Stratosphere System Overview Big Data Beers Berlin. 20.11.2013Stratosphere System Overview Big Data Beers Berlin. 20.11.2013
Stratosphere System Overview Big Data Beers Berlin. 20.11.2013
 
Stratosphere Intro (Java and Scala Interface)
Stratosphere Intro (Java and Scala Interface)Stratosphere Intro (Java and Scala Interface)
Stratosphere Intro (Java and Scala Interface)
 

Dernier

A Year of the Servo Reboot: Where Are We Now?
A Year of the Servo Reboot: Where Are We Now?A Year of the Servo Reboot: Where Are We Now?
A Year of the Servo Reboot: Where Are We Now?Igalia
 
2024: Domino Containers - The Next Step. News from the Domino Container commu...
2024: Domino Containers - The Next Step. News from the Domino Container commu...2024: Domino Containers - The Next Step. News from the Domino Container commu...
2024: Domino Containers - The Next Step. News from the Domino Container commu...Martijn de Jong
 
Apidays Singapore 2024 - Building Digital Trust in a Digital Economy by Veron...
Apidays Singapore 2024 - Building Digital Trust in a Digital Economy by Veron...Apidays Singapore 2024 - Building Digital Trust in a Digital Economy by Veron...
Apidays Singapore 2024 - Building Digital Trust in a Digital Economy by Veron...apidays
 
Axa Assurance Maroc - Insurer Innovation Award 2024
Axa Assurance Maroc - Insurer Innovation Award 2024Axa Assurance Maroc - Insurer Innovation Award 2024
Axa Assurance Maroc - Insurer Innovation Award 2024The Digital Insurer
 
A Domino Admins Adventures (Engage 2024)
A Domino Admins Adventures (Engage 2024)A Domino Admins Adventures (Engage 2024)
A Domino Admins Adventures (Engage 2024)Gabriella Davis
 
What Are The Drone Anti-jamming Systems Technology?
What Are The Drone Anti-jamming Systems Technology?What Are The Drone Anti-jamming Systems Technology?
What Are The Drone Anti-jamming Systems Technology?Antenna Manufacturer Coco
 
The 7 Things I Know About Cyber Security After 25 Years | April 2024
The 7 Things I Know About Cyber Security After 25 Years | April 2024The 7 Things I Know About Cyber Security After 25 Years | April 2024
The 7 Things I Know About Cyber Security After 25 Years | April 2024Rafal Los
 
Driving Behavioral Change for Information Management through Data-Driven Gree...
Driving Behavioral Change for Information Management through Data-Driven Gree...Driving Behavioral Change for Information Management through Data-Driven Gree...
Driving Behavioral Change for Information Management through Data-Driven Gree...Enterprise Knowledge
 
Scaling API-first – The story of a global engineering organization
Scaling API-first – The story of a global engineering organizationScaling API-first – The story of a global engineering organization
Scaling API-first – The story of a global engineering organizationRadu Cotescu
 
Raspberry Pi 5: Challenges and Solutions in Bringing up an OpenGL/Vulkan Driv...
Raspberry Pi 5: Challenges and Solutions in Bringing up an OpenGL/Vulkan Driv...Raspberry Pi 5: Challenges and Solutions in Bringing up an OpenGL/Vulkan Driv...
Raspberry Pi 5: Challenges and Solutions in Bringing up an OpenGL/Vulkan Driv...Igalia
 
[2024]Digital Global Overview Report 2024 Meltwater.pdf
[2024]Digital Global Overview Report 2024 Meltwater.pdf[2024]Digital Global Overview Report 2024 Meltwater.pdf
[2024]Digital Global Overview Report 2024 Meltwater.pdfhans926745
 
A Call to Action for Generative AI in 2024
A Call to Action for Generative AI in 2024A Call to Action for Generative AI in 2024
A Call to Action for Generative AI in 2024Results
 
Mastering MySQL Database Architecture: Deep Dive into MySQL Shell and MySQL R...
Mastering MySQL Database Architecture: Deep Dive into MySQL Shell and MySQL R...Mastering MySQL Database Architecture: Deep Dive into MySQL Shell and MySQL R...
Mastering MySQL Database Architecture: Deep Dive into MySQL Shell and MySQL R...Miguel Araújo
 
08448380779 Call Girls In Civil Lines Women Seeking Men
08448380779 Call Girls In Civil Lines Women Seeking Men08448380779 Call Girls In Civil Lines Women Seeking Men
08448380779 Call Girls In Civil Lines Women Seeking MenDelhi Call girls
 
Exploring the Future Potential of AI-Enabled Smartphone Processors
Exploring the Future Potential of AI-Enabled Smartphone ProcessorsExploring the Future Potential of AI-Enabled Smartphone Processors
Exploring the Future Potential of AI-Enabled Smartphone Processorsdebabhi2
 
08448380779 Call Girls In Diplomatic Enclave Women Seeking Men
08448380779 Call Girls In Diplomatic Enclave Women Seeking Men08448380779 Call Girls In Diplomatic Enclave Women Seeking Men
08448380779 Call Girls In Diplomatic Enclave Women Seeking MenDelhi Call girls
 
04-2024-HHUG-Sales-and-Marketing-Alignment.pptx
04-2024-HHUG-Sales-and-Marketing-Alignment.pptx04-2024-HHUG-Sales-and-Marketing-Alignment.pptx
04-2024-HHUG-Sales-and-Marketing-Alignment.pptxHampshireHUG
 
From Event to Action: Accelerate Your Decision Making with Real-Time Automation
From Event to Action: Accelerate Your Decision Making with Real-Time AutomationFrom Event to Action: Accelerate Your Decision Making with Real-Time Automation
From Event to Action: Accelerate Your Decision Making with Real-Time AutomationSafe Software
 
Strategies for Unlocking Knowledge Management in Microsoft 365 in the Copilot...
Strategies for Unlocking Knowledge Management in Microsoft 365 in the Copilot...Strategies for Unlocking Knowledge Management in Microsoft 365 in the Copilot...
Strategies for Unlocking Knowledge Management in Microsoft 365 in the Copilot...Drew Madelung
 
Powerful Google developer tools for immediate impact! (2023-24 C)
Powerful Google developer tools for immediate impact! (2023-24 C)Powerful Google developer tools for immediate impact! (2023-24 C)
Powerful Google developer tools for immediate impact! (2023-24 C)wesley chun
 

Dernier (20)

A Year of the Servo Reboot: Where Are We Now?
A Year of the Servo Reboot: Where Are We Now?A Year of the Servo Reboot: Where Are We Now?
A Year of the Servo Reboot: Where Are We Now?
 
2024: Domino Containers - The Next Step. News from the Domino Container commu...
2024: Domino Containers - The Next Step. News from the Domino Container commu...2024: Domino Containers - The Next Step. News from the Domino Container commu...
2024: Domino Containers - The Next Step. News from the Domino Container commu...
 
Apidays Singapore 2024 - Building Digital Trust in a Digital Economy by Veron...
Apidays Singapore 2024 - Building Digital Trust in a Digital Economy by Veron...Apidays Singapore 2024 - Building Digital Trust in a Digital Economy by Veron...
Apidays Singapore 2024 - Building Digital Trust in a Digital Economy by Veron...
 
Axa Assurance Maroc - Insurer Innovation Award 2024
Axa Assurance Maroc - Insurer Innovation Award 2024Axa Assurance Maroc - Insurer Innovation Award 2024
Axa Assurance Maroc - Insurer Innovation Award 2024
 
A Domino Admins Adventures (Engage 2024)
A Domino Admins Adventures (Engage 2024)A Domino Admins Adventures (Engage 2024)
A Domino Admins Adventures (Engage 2024)
 
What Are The Drone Anti-jamming Systems Technology?
What Are The Drone Anti-jamming Systems Technology?What Are The Drone Anti-jamming Systems Technology?
What Are The Drone Anti-jamming Systems Technology?
 
The 7 Things I Know About Cyber Security After 25 Years | April 2024
The 7 Things I Know About Cyber Security After 25 Years | April 2024The 7 Things I Know About Cyber Security After 25 Years | April 2024
The 7 Things I Know About Cyber Security After 25 Years | April 2024
 
Driving Behavioral Change for Information Management through Data-Driven Gree...
Driving Behavioral Change for Information Management through Data-Driven Gree...Driving Behavioral Change for Information Management through Data-Driven Gree...
Driving Behavioral Change for Information Management through Data-Driven Gree...
 
Scaling API-first – The story of a global engineering organization
Scaling API-first – The story of a global engineering organizationScaling API-first – The story of a global engineering organization
Scaling API-first – The story of a global engineering organization
 
Raspberry Pi 5: Challenges and Solutions in Bringing up an OpenGL/Vulkan Driv...
Raspberry Pi 5: Challenges and Solutions in Bringing up an OpenGL/Vulkan Driv...Raspberry Pi 5: Challenges and Solutions in Bringing up an OpenGL/Vulkan Driv...
Raspberry Pi 5: Challenges and Solutions in Bringing up an OpenGL/Vulkan Driv...
 
[2024]Digital Global Overview Report 2024 Meltwater.pdf
[2024]Digital Global Overview Report 2024 Meltwater.pdf[2024]Digital Global Overview Report 2024 Meltwater.pdf
[2024]Digital Global Overview Report 2024 Meltwater.pdf
 
A Call to Action for Generative AI in 2024
A Call to Action for Generative AI in 2024A Call to Action for Generative AI in 2024
A Call to Action for Generative AI in 2024
 
Mastering MySQL Database Architecture: Deep Dive into MySQL Shell and MySQL R...
Mastering MySQL Database Architecture: Deep Dive into MySQL Shell and MySQL R...Mastering MySQL Database Architecture: Deep Dive into MySQL Shell and MySQL R...
Mastering MySQL Database Architecture: Deep Dive into MySQL Shell and MySQL R...
 
08448380779 Call Girls In Civil Lines Women Seeking Men
08448380779 Call Girls In Civil Lines Women Seeking Men08448380779 Call Girls In Civil Lines Women Seeking Men
08448380779 Call Girls In Civil Lines Women Seeking Men
 
Exploring the Future Potential of AI-Enabled Smartphone Processors
Exploring the Future Potential of AI-Enabled Smartphone ProcessorsExploring the Future Potential of AI-Enabled Smartphone Processors
Exploring the Future Potential of AI-Enabled Smartphone Processors
 
08448380779 Call Girls In Diplomatic Enclave Women Seeking Men
08448380779 Call Girls In Diplomatic Enclave Women Seeking Men08448380779 Call Girls In Diplomatic Enclave Women Seeking Men
08448380779 Call Girls In Diplomatic Enclave Women Seeking Men
 
04-2024-HHUG-Sales-and-Marketing-Alignment.pptx
04-2024-HHUG-Sales-and-Marketing-Alignment.pptx04-2024-HHUG-Sales-and-Marketing-Alignment.pptx
04-2024-HHUG-Sales-and-Marketing-Alignment.pptx
 
From Event to Action: Accelerate Your Decision Making with Real-Time Automation
From Event to Action: Accelerate Your Decision Making with Real-Time AutomationFrom Event to Action: Accelerate Your Decision Making with Real-Time Automation
From Event to Action: Accelerate Your Decision Making with Real-Time Automation
 
Strategies for Unlocking Knowledge Management in Microsoft 365 in the Copilot...
Strategies for Unlocking Knowledge Management in Microsoft 365 in the Copilot...Strategies for Unlocking Knowledge Management in Microsoft 365 in the Copilot...
Strategies for Unlocking Knowledge Management in Microsoft 365 in the Copilot...
 
Powerful Google developer tools for immediate impact! (2023-24 C)
Powerful Google developer tools for immediate impact! (2023-24 C)Powerful Google developer tools for immediate impact! (2023-24 C)
Powerful Google developer tools for immediate impact! (2023-24 C)
 

GOTO Night Amsterdam - Stream processing with Apache Flink

  • 1. Stream Processing with Apache Flink Robert Metzger @rmetzger_ rmetzger@apache.org GOTO Night Amsterdam, March 10, 2016
  • 2. Talk overview  My take on the stream processing space, and how it changes the way we think about data  Discussion of unique building blocks of Flink  Benchmarking Flink, by extending a benchmark from Yahoo! 2
  • 3. Apache Flink  Apache Flink is an open source stream processing framework • Low latency • High throughput • Stateful • Distributed  Developed at the Apache Software Foundation, 1.0.0 release available soon, used in production 3
  • 5. 5 Streaming is the biggest change in data infrastructure since Hadoop
  • 6. 6 1. Radically simplified infrastructure 2. Do more with your data, faster 3. Can completely subsume batch
  • 7. Traditional data processing 7 Web server Logs Web server Logs Web server Logs HDFS / S3 Periodic (custom) or continuous ingestion (Flume) into HDFS Batch job(s) for log analysis Periodic log analysis job Serving layer  Log analysis example using a batch processor Job scheduler (Oozie)
  • 8. Traditional data processing 8 Web server Logs Web server Logs Web server Logs HDFS / S3 Periodic (custom) or continuous ingestion (Flume) into HDFS Batch job(s) for log analysis Periodic log analysis job Serving layer  Latency from log event to serving layer usually in the range of hours every 2 hrs Job scheduler (Oozie)
  • 9. Data processing without stream processor 9 Web server Logs Web server Logs HDFS / S3 Batch job(s) for log analysis  This architecture is a hand-crafted micro- batch model Batch interval: ~2 hours hours minutes milliseconds Manually triggered periodic batch job Batch processor with micro-batches Latency Approach seconds Stream processor
  • 10. Downsides of stream processing with a batch engine  Very high latency (hours)  Complex architecture required: • Periodic job scheduler (e.g. Oozie) • Data loading into HDFS (e.g. Flume) • Batch processor • (When using the “lambda architecture”: a stream processor) All these components need to be implemented and maintained  Backpressure: How does the pipeline handle load spikes? 10
  • 11. Log event analysis using a stream processor 11 Web server Web server Web server High throughput publish/subscribe bus Serving layer  Stream processors allow to analyze events with sub-second latency. Options: • Apache Kafka • Amazon Kinesis • MapR Streams • Google Cloud Pub/Sub Forward events immediately to pub/sub bus Stream Processor Options: • Apache Flink • Apache Beam • Apache Samza Process events in real time & update serving layer
  • 12. 12 Real-world data is produced in a continuous fashion. New systems like Flink and Kafka embrace streaming nature of data. Web server Kafka topic Stream processor
  • 13. What do we need for replacing the “batch stack”? 13 Web server Web server Web server High throughput publish/subscribe bus Serving layer Options: • Apache Kafka • Amazon Kinesis • MapR Streams • Google Cloud Pub/Sub Forward events immediately to pub/sub bus Stream Processor Options: • Apache Flink • Google Cloud Dataflow Process events in real time & update serving layer Low latency High throughput State handling Windowing / Out of order events Fault tolerance and correctness
  • 14. Apache Flink stack 15 Gelly Table ML SAMOA DataSet (Java/Scala)DataStream (Java / Scala) HadoopM/R LocalClusterYARN ApacheBeam ApacheBeam Table Cascading Streaming dataflow runtime StormAPI Zeppelin CEP
  • 15. Needed for the use case 16 Gelly Table ML SAMOA DataSet (Java/Scala)DataStream (Java / Scala) HadoopM/R LocalClusterYARN ApacheBeam ApacheBeam Table Cascading Streaming dataflow runtime StormAPI Zeppelin CEP
  • 16. Windowing / Out of order events 17 Low latency High throughput State handling Windowing / Out of order events Fault tolerance and correctness
  • 17. Building windows from a stream 18  “Number of visitors in the last 5 minutes per country” Web server Kafka topic Stream processor // create stream from Kafka source DataStream<LogEvent> stream = env.addSource(new KafkaConsumer()); // group by country DataStream<LogEvent> keyedStream = stream.keyBy(“country“); // window of size 5 minutes keyedStream.timeWindow(Time.minutes(5)) // do operations per window .apply(new CountPerWindowFunction());
  • 18. Building windows: Execution 19 Kafka Source Window Operator S S S W W W group by country // window of size 5 minutes keyedStream.timeWindow(Time.minutes(5)); Job plan Parallel execution on the cluster Time
  • 19. Window types in Flink  Tumbling windows  Sliding windows  Custom windows with window assigners, triggers and evictors 20Further reading: http://flink.apache.org/news/2015/12/04/Introducing-windows.html
  • 20. Time-based windows 21 Stream Time of event Event data { “accessTime”: “1457002134”, “userId”: “1337”, “userLocation”: “UK” }  Windows are created based on the real world time when the event occurred
  • 21. A look at the reality of time 22 Kafka Network delays Out of sync clocks 33 11 21 15 9  Events arrive out of order in the system  Use-case specific low watermarks for time tracking Window between 0 and 15 Stream Processor 15 Guarantee that no event with time <= 15 will arrive afterwards
  • 22. Time characteristics in Apache Flink  Event Time • Users have to specify an event-time extractor + watermark emitter • Results are deterministic, but with latency  Processing Time • System time is used when evaluating windows • low latency  Ingestion Time • Flink assigns current system time at the sources  Pluggable, without window code changes 23
  • 23. State handling 24 Low latency High throughput State handling Windowing / Out of order events Fault tolerance and correctness
  • 24. State in streaming  Where do we store the elements from our windows?  In stateless systems, an external state store (e.g. Redis) is needed. 25 S S S W W W Time Elements in windows are state
  • 25. Stream processor: Flink Managed state in Flink  Flink automatically backups and restores state  State can be larger than the available memory  State backends: (embedded) RocksDB, Heap memory 26 Operator with windows (large state) State backend (local) Distributed File System Periodic backup / recovery Web server Kafka
  • 26. Managing the state  How can we operate such a pipeline 24x7?  Losing state (by stopping the system) would require a replay of past events  We need a way to store the state somewhere! 27 Web server Kafka topic Stream processor
  • 27. Savepoints: Versioning state  Savepoint: Create an addressable copy of a job’s current state.  Restart a job from any savepoint. 28 Further reading: http://data-artisans.com/how-apache-flink-enables-new-streaming-applications/ > flink savepoint <JobID> HDFS > hdfs:///flink-savepoints/2 > flink run –s hdfs:///flink-savepoints/2 <jar> HDFS
  • 28. Fault tolerance and correctness 29 Low latency High throughput State handling Windowing / Out of order events Fault tolerance and correctness
  • 29. Fault tolerance in streaming  How do we ensure the results (number of visitors) are always correct?  Failures should not lead to data loss or incorrect results 30 Web server Kafka topic Stream processor
  • 30. Fault tolerance in streaming  at least once: ensure all operators see all events • Storm: Replay stream in failure case (acking of individual records)  Exactly once: ensure that operators do not perform duplicate updates to their state • Flink: Distributed Snapshots • Spark: Micro-batches on batch runtime 31
  • 31. Flink’s Distributed Snapshots  Lightweight approach of storing the state of all operators without pausing the execution  Implemented using barriers flowing through the topology 32 Data Stream barrier Before barrier = part of the snapshot After barrier = Not in snapshot Further reading: http://blog.acolyer.org/2015/08/19/asynchronous-distributed-snapshots-for- distributed-dataflows/
  • 32. Wrap-up: Log processing example  How to do something with the data? Windowing  How does the system handle large windows? Managed state  How do operate such a system 24x7? Safepoints  How to ensure correct results across failures? Checkpoints, Master HA 33 Web server Kafka topic Stream processor
  • 33. Performance: Low Latency & High Throughput 34 Low latency High throughput State handling Windowing / Out of order events Fault tolerance and correctness
  • 34. Performance: Introduction  Performance always depends on your own use cases, so test it yourself!  We based our experiments on a recent benchmark published by Yahoo!  They benchmarked Storm, Spark Streaming and Flink with a production use- case (counting ad impressions) 35 Full Yahoo! article: https://yahooeng.tumblr.com/post/135321837876/benchmarking-streaming- computation-engines-at
  • 35. Yahoo! Benchmark  Count ad impressions grouped by campaign  Compute aggregates over a 10 second window  Emit current value of window aggregates to Redis every second for query 36 Full Yahoo! article: https://yahooeng.tumblr.com/post/135321837876/benchmarking-streaming- computation-engines-at
  • 36. Yahoo’s Results “Storm […] and Flink […] show sub-second latencies at relatively high throughputs with Storm having the lowest 99th percentile latency. Spark streaming 1.5.1 supports high throughputs, but at a relatively higher latency.” (Quote from the blog post’s executive summary) 37 Full Yahoo! article: https://yahooeng.tumblr.com/post/135321837876/benchmarking-streaming- computation-engines-at
  • 37. Extending the benchmark  Benchmark stops at Storm’s throughput limits. Where is Flink’s limit?  How will Flink’s own window implementation perform compared to Yahoo’s “state in redis windowing” approach? 38 Full Yahoo! article: https://yahooeng.tumblr.com/post/135321837876/benchmarking-streaming- computation-engines-at
  • 38. Windowing with state in Redis 39 KafkaConsumer map() filter() group windowing & caching code realtime queries
  • 39. Rewrite to use Flink’s own window 40 KafkaConsumer map() filter() group Flink event time windows realtime queries
  • 40. Results after rewrite 41 0 750,000 1,500,000 2,250,000 3,000,000 3,750,000 Storm Flink Throughput: msgs/sec 400k msgs/sec
  • 41. Can we even go further? 42 KafkaConsumer map() filter() group Flink event time windows Network link to Kafka cluster is bottleneck! (1GigE) Data Generator map() filter() group Flink event time windows Solution: Move data generator into job (10 GigE)
  • 42. Results without network bottleneck 43 0 4,000,000 8,000,000 12,000,000 16,000,000 Storm Flink Flink (10 GigE) Throughput: msgs/sec 10 GigE end-to-end 15m msgs/sec 400k msgs/sec 3m msgs/sec
  • 43. Benchmark summary  Flink achieves throughput of 15 million messages/second on 10 machines  35x higher throughput compared to Storm (80x compared to Yahoo’s runs)  Flink ran with exactly once guarantees, Storm with at least once.  Read the full report: http://data- artisans.com/extending-the-yahoo- streaming-benchmark/ 44
  • 45. Other notable features  Expressive DataStream API (similar to high level APIs known from the batch world)  Flink is a full-fledged batch processor with an optimizer, managed memory, memory- aware algorithms, build-in iterations  Many libraries: Complex Event Processing (CEP), Graph Processing, Machine Learning  Integration with YARN, HBase, ElasticSearch, Kafka, MapReduce, … 46
  • 46. Questions?  Ask now!  eMail: rmetzger@apache.org  Twitter: @rmetzger_  Follow: @ApacheFlink  Read: flink.apache.org/blog, data- artisans.com/blog/  Mailinglists: (news | user | dev)@flink.apache.org 47
  • 47. Apache Flink stack 48 Gelly Table ML SAMOA DataSet (Java/Scala)DataStream (Java / Scala) HadoopM/R LocalClusterYARN ApacheBeam ApacheBeam Table Cascading Streaming dataflow runtime StormAPI Zeppelin CEP
  • 49. Roadmap 2016 50  SQL / StreamSQL  CEP Library  Managed Operator State  Dynamic Scaling  Miscellaneous
  • 50. Miscellaneous  Support for Apache Mesos  Security • Over-the-wire encryption of RPC (akka) and data transfers (netty)  More connectors • Apache Cassandra • Amazon Kinesis  Enhance metrics • Throughput / Latencies • Backpressure monitoring • Spilling / Out of Core 51
  • 51. Handling Backpressure 52 Slow down upstream operators Backpressure might occur when: • Operators create checkpoints • Windows are evaluated • Operators depend on external resources • JVMs do Garbage Collection Operator not able to process incoming data immediately
  • 52. Handling Backpressure 53 Sender Sender Receiver Receiver Sender does not have any empty buffers available: Slowdown Network transfer (Netty) or local buffer exchange (when S and R are on the same machine) • Data sources slow down pulling data from their underlying system (Kafka or similar queues) Full buffer Empty buffer

Notes de l'éditeur

  1. Test note
  2. Test1
  3. Test1
  4. But what is the importance of streaming, what can you do with it? First, streaming radically simplifies the data infrastructure, by serving many use cases out of the stream processor in real time. This is connected to broader trends like the move to more microservice-based organizations. Second, streaming is the style of processing that is needed by new applications. These include Internet of Things, and demand-driven services like Uber. Third, streaming is just a better way to do many of the traditional use cases because it subsumes batch.