SlideShare a Scribd company logo
1 of 68
Stephan Ewen
@stephanewen
Streaming Analytics
with Apache Flink 1.0
Apache Flink Stack
2
DataStream API
Stream Processing
DataSet API
Batch Processing
Runtime
Distributed Streaming Data Flow
Libraries
Streaming and batch as first class citizens.
Today
3
Streaming and batch as first class citizens.
DataStream API
Stream Processing
DataSet API
Batch Processing
Runtime
Distributed Streaming Data Flow
Libraries
4
Streaming is the next programming paradigm
for data applications, and you need to start
thinking in terms of streams.
5
Streaming technology is enabling the obvious:
continuous processing on data that is
continuously produced
Continuous Processing with Batch
 Continuous
ingestion
 Periodic (e.g.,
hourly) files
 Periodic batch
jobs
6
λ Architecture
 "Batch layer": what
we had before
 "Stream layer":
approximate early
results
7
A Stream Processing Pipeline
8
collect log analyze serve & store
A brief History of Flink
9
January ‘10 December ‘14
v0.5 v0.6 v0.7
March ‘16
Flink Project
Incubation
Top Level
Project
v0.8 v0.10
Release
1.0
Project
Stratosphere
(Flink precursor)
v0.9
April ‘14
A brief History of Flink
10
January ‘10 December ‘14
v0.5 v0.6 v0.7
March ‘16
Flink Project
Incubation
Top Level
Project
v0.8 v0.10
Release
1.0
Project
Stratosphere
(Flink precursor)
v0.9
April ‘14
The academia gap:
Reading/writing papers,
teaching, worrying about thesis
Realizing this might be
interesting to people
beyond academia
(even more so, actually)
Programs and Dataflows
11
Source
Transformation
Transformation
Sink
val lines: DataStream[String] = env.addSource(new FlinkKafkaConsumer09(…))
val events: DataStream[Event] = lines.map((line) => parse(line))
val stats: DataStream[Statistic] = stream
.keyBy("sensor")
.timeWindow(Time.seconds(5))
.sum(new MyAggregationFunction())
stats.addSink(new RollingSink(path))
Source
[1]
map()
[1]
keyBy()/
window()/
apply()
[1]
Sink
[1]
Source
[2]
map()
[2]
keyBy()/
window()/
apply()
[2]
Streaming
Dataflow
What makes Flink flink?
12
Low latency
High Throughput
Well-behaved
flow control
(back pressure)
Make more sense of data
Works on real-time
and historic data
True
Streaming
Event Time
APIs
Libraries
Stateful
Streaming
Globally consistent
savepoints
Exactly-once semantics
for fault tolerance
Windows &
user-defined state
Flexible windows
(time, count, session, roll-your own)
Complex Event Processing
Streaming Analytics by Example
13
Time-Windowed Aggregations
14
case class Event(sensor: String, measure: Double)
val env = StreamExecutionEnvironment.getExecutionEnvironment
val stream: DataStream[Event] = env.addSource(…)
stream
.keyBy("sensor")
.timeWindow(Time.seconds(5))
.sum("measure")
Time-Windowed Aggregations
15
case class Event(sensor: String, measure: Double)
val env = StreamExecutionEnvironment.getExecutionEnvironment
val stream: DataStream[Event] = env.addSource(…)
stream
.keyBy("sensor")
.timeWindow(Time.seconds(60), Time.seconds(5))
.sum("measure")
Session-Windowed Aggregations
16
case class Event(sensor: String, measure: Double)
val env = StreamExecutionEnvironment.getExecutionEnvironment
val stream: DataStream[Event] = env.addSource(…)
stream
.keyBy("sensor")
.window(EventTimeSessionWindows.withGap(Time.seconds(60)))
.max("measure")
Session-Windowed Aggregations
17
case class Event(sensor: String, measure: Double)
val env = StreamExecutionEnvironment.getExecutionEnvironment
val stream: DataStream[Event] = env.addSource(…)
stream
.keyBy("sensor")
.window(EventTimeSessionWindows.withGap(Time.seconds(60)))
.max("measure")
Flink 1.1 syntax
Pattern Detection
18
case class Event(producer: String, evtType: Int, msg: String)
case class Alert(msg: String)
val stream: DataStream[Event] = env.addSource(…)
stream
.keyBy("producer")
.flatMap(new RichFlatMapFuncion[Event, Alert]() {
lazy val state: ValueState[Int] = getRuntimeContext.getState(…)
def flatMap(event: Event, out: Collector[Alert]) = {
val newState = state.value() match {
case 0 if (event.evtType == 0) => 1
case 1 if (event.evtType == 1) => 0
case x => out.collect(Alert(event.msg, x)); 0
}
state.update(newState)
}
})
Pattern Detection
19
case class Event(producer: String, evtType: Int, msg: String)
case class Alert(msg: String)
val stream: DataStream[Event] = env.addSource(…)
stream
.keyBy("producer")
.flatMap(new RichFlatMapFuncion[Event, Alert]() {
lazy val state: ValueState[Int] = getRuntimeContext.getState(…)
def flatMap(event: Event, out: Collector[Alert]) = {
val newState = state.value() match {
case 0 if (event.evtType == 0) => 1
case 1 if (event.evtType == 1) => 0
case x => out.collect(Alert(event.msg, x)); 0
}
state.update(newState)
}
})
Embedded key/value
state store
Many more
 Joining streams (e.g. combine readings from sensor)
 Detecting Patterns (CEP)
 Applying (changing) rules or models to events
 Training and applying online machine learning
models
 …
20
(It's) About Time
21
22
The biggest change in moving from
batch to streaming is
handling time explicitly
Example: Windowing by Time
23
case class Event(id: String, measure: Double, timestamp: Long)
val env = StreamExecutionEnvironment.getExecutionEnvironment
val stream: DataStream[Event] = env.addSource(…)
stream
.keyBy("id")
.timeWindow(Time.seconds(15), Time.seconds(5))
.sum("measure")
Example: Windowing by Time
24
case class Event(id: String, measure: Double, timestamp: Long)
val env = StreamExecutionEnvironment.getExecutionEnvironment
val stream: DataStream[Event] = env.addSource(…)
stream
.keyBy("id")
.timeWindow(Time.seconds(15), Time.seconds(5))
.sum("measure")
Different Notions of Time
25
Event Producer Message Queue
Flink
Data Source
Flink
Window Operator
partition 1
partition 2
Event
Time
Ingestion
Time
Window
Processing
Time
1977 1980 1983 1999 2002 2005 2015
Processing Time
Episode
IV
Episode
V
Episode
VI
Episode
I
Episode
II
Episode
III
Episode
VII
Event Time
Event Time vs. Processing Time
26
Out of order Streams
27
Events occur on devices
Queue / Log
Events analyzed in
a
data streaming
system
Stream Analysis
Events stored in a log
Out of order Streams
28
Out of order Streams
29
Out of order Streams
30
Out of order Streams
31
Out of order !!!
First burst of events
Second burst of events
Out of order Streams
32
Event time windows
Arrival time windows
Instant event-at-a-time
First burst of events
Second burst of events
Processing Time
33
case class Event(id: String, measure: Double, timestamp: Long)
val env = StreamExecutionEnvironment.getExecutionEnvironment
env.setStreamTimeCharacteristic(ProcessingTime)
val stream: DataStream[Event] = env.addSource(…)
stream
.keyBy("id")
.timeWindow(Time.seconds(15), Time.seconds(5))
.sum("measure")
Window by operator's processing time
Ingestion Time
34
case class Event(id: String, measure: Double, timestamp: Long)
val env = StreamExecutionEnvironment.getExecutionEnvironment
env.setStreamTimeCharacteristic(IngestionTime)
val stream: DataStream[Event] = env.addSource(…)
stream
.keyBy("id")
.timeWindow(Time.seconds(15), Time.seconds(5))
.sum("measure")
Event Time
35
case class Event(id: String, measure: Double, timestamp: Long)
val env = StreamExecutionEnvironment.getExecutionEnvironment
env.setStreamTimeCharacteristic(EventTime)
val stream: DataStream[Event] = env.addSource(…)
stream
.keyBy("id")
.timeWindow(Time.seconds(15), Time.seconds(5))
.sum("measure")
Event Time
36
case class Event(id: String, measure: Double, timestamp: Long)
val env = StreamExecutionEnvironment.getExecutionEnvironment
env.setStreamTimeCharacteristic(EventTime)
val stream: DataStream[Event] = env.addSource(…)
val tsStream = stream.assignAscendingTimestamps(_.timestamp)
tsStream
.keyBy("id")
.timeWindow(Time.seconds(15), Time.seconds(5))
.sum("measure")
Event Time
37
case class Event(id: String, measure: Double, timestamp: Long)
val env = StreamExecutionEnvironment.getExecutionEnvironment
env.setStreamTimeCharacteristic(EventTime)
val stream: DataStream[Event] = env.addSource(…)
val tsStream = stream.assignTimestampsAndWatermarks(
new MyTimestampsAndWatermarkGenerator())
tsStream
.keyBy("id")
.timeWindow(Time.seconds(15), Time.seconds(5))
.sum("measure")
Watermarks
38
7
W(11)W(17)
11159121417122220 171921
Watermark
Event
Event timestamp
Stream (in order)
7
W(11)W(20)
Watermark
991011141517
Event
Event timestamp
1820 192123
Stream (out of order)
Watermarks in Parallel
39
Source
(1)
Source
(2)
map
(1)
map
(2)
window
(1)
window
(2)
29
29
17
14
14
29
14
14
W(33)
W(17)
W(17)
A|30B|31
C|30
D|15
E|30
F|15G|18H|20
K|35
Watermark
Event Time
at the operator
Event
[id|timestamp]
Event Time
at input streams
33
17
Watermark
Generation
M|39N|39Q|44
L|22O|23R|37
Mixing Event Time Processing Time
40
case class Event(id: String, measure: Double, timestamp: Long)
val env = StreamExecutionEnvironment.getExecutionEnvironment
env.setStreamTimeCharacteristic(EventTime)
val stream: DataStream[Event] = env.addSource(…)
val tsStream = stream.assignAscendingTimestamps(_.timestamp)
tsStream
.keyBy("id")
.window(SlidingEventTimeWindows.of(seconds(15), seconds(5))
.trigger(new MyTrigger())
.sum("measure")
Window Triggers
 React to any combination of
• Event Time
• Processing Time
• Event data
 Example of a mixed EventTime / Proc. Time Trigger:
• Trigger when event time reaches window end
OR
• When processing time reaches window end plus 30 secs.
41
Trigger example
42
.sum("measure")
public class EventTimeTrigger extends Trigger<Object, TimeWindow> {
public TriggerResult onElement(Object evt, long time,
TimeWindow window, TriggerContext ctx) {
ctx.registerEventTimeTimer(window.maxTimestamp());
ctx.registerProcessingTimeTimer(window.maxTimestamp() + 30000);
return TriggerResult.CONTINUE;
}
public TriggerResult onEventTime(long time, TimeWindow w, TriggerContext ctx) {
return TriggerResult.FIRE_AND_PURGE;
}
public TriggerResult onProcessingTime(long time, TimeWindow w, TriggerContext c) {
return TriggerResult.FIRE_AND_PURGE;
}
Trigger example
43
.sum("measure")
public class EventTimeTrigger extends Trigger<Object, TimeWindow> {
public TriggerResult onElement(Object evt, long time,
TimeWindow window, TriggerContext ctx) {
ctx.registerEventTimeTimer(window.maxTimestamp());
ctx.registerProcessingTimeTimer(window.maxTimestamp() + 30000);
return TriggerResult.CONTINUE;
}
public TriggerResult onEventTime(long time, TimeWindow w, TriggerContext ctx) {
return TriggerResult.FIRE_AND_PURGE;
}
public TriggerResult onProcessingTime(long time, TimeWindow w, TriggerContext c) {
return TriggerResult.FIRE_AND_CONTINUE;
}
Per Kafka Partition Watermarks
44
Source
(1)
Source
(2)
map
(1)
map
(2)
window
(1)
window
(2)
33
17
29
29
17
14
14
29
14
14
W(33)
W(17)
W(17)
A|30B|73
C|33
D|18
E|31
F|15G|91H|94
K|77
Watermark
Generation
L|35N|39
O|97 M|89
I|21Q|23
T|99 S|97
Per Kafka Partition Watermarks
45
val env = StreamExecutionEnvironment.getExecutionEnvironment
env.setStreamTimeCharacteristic(EventTime)
val kafka = new FlinkKafkaConsumer09(topic, schema, props)
kafka.assignTimestampsAndWatermarks(
new MyTimestampsAndWatermarkGenerator())
val stream: DataStream[Event] = env.addSource(kafka)
stream
.keyBy("id")
.timeWindow(Time.seconds(15), Time.seconds(5))
.sum("measure")
Matters of State
(Fault Tolerance, Reinstatements, etc)
46
Back to the Aggregation Example
47
case class Event(id: String, measure: Double, timestamp: Long)
val env = StreamExecutionEnvironment.getExecutionEnvironment
val stream: DataStream[Event] = env.addSource(
new FlinkKafkaConsumer09(topic, schema, properties))
stream
.keyBy("id")
.timeWindow(Time.seconds(15), Time.seconds(5))
.sum("measure")
Stateful
Fault Tolerance
 Prevent data loss (reprocess lost in-flight events)
 Recover state consistency (exactly-once semantics)
• Pending windows & user-defined (key/value) state
 Checkpoint based fault tolerance
• Periodicaly create checkpoints
• Recovery: resume from last completed checkpoint
• Async. Barrier Snapshots (ABS) Algorithm 
48
Checkpoints
49
data stream
event
newer records older records
State of the dataflow
at point Y
State of the dataflow
at point X
Checkpoint Barriers
 Markers, injected into the streams
50
Checkpoint Procedure
51
Checkpoint Procedure
52
Savepoints
 A "Checkpoint" is a globally consistent point-in-time snapshot
of the streaming program (point in stream, state)
 A "Savepoint" is a user-triggered retained checkpoint
 Streaming programs can start from a savepoint
53
Savepoint B Savepoint A
(Re)processing data (in batch)
 Re-processing data (what-if exploration, to correct bugs, etc.)
 Usually by running a batch job with a set of old files
 Tools that map files to times
54
2016-3-1
12:00 am
2016-3-1
1:00 am
2016-3-1
2:00 am
2016-3-11
11:00pm
2016-3-12
12:00am
2016-3-12
1:00am…
Collection of files, by ingestion time
2016-3-11
10:00pm
To the batch
processor
Unclear Batch Boundaries
55
2016-3-1
12:00 am
2016-3-1
1:00 am
2016-3-1
2:00 am
2016-3-11
11:00pm
2016-3-12
12:00am
2016-3-12
1:00am…
2016-3-11
10:00pm
To the batch
processor
?
?
What about sessions across batches?
(Re)processing data (streaming)
 Draw savepoints at times that you will want to start new jobs
from (daily, hourly, …)
 Reprocess by starting a new job from a savepoint
• Defines start position in stream (for example Kafka offsets)
• Initializes pending state (like partial sessions)
56
Savepoint
Run new streaming
program from savepoint
Continuous Data Sources
57
2016-3-1
12:00 am
2016-3-1
1:00 am
2016-3-1
2:00 am
2016-3-11
11:00pm
2016-3-12
12:00am
2016-3-12
1:00am
2016-3-11
10:00pm …
partition
partition
Savepoint
Savepoint
Stream of Kafka Partitions
Stream view over sequence of files
Kafka offsets +
Operator state
File mod timestamp +
File position +
Operator state
WIP (target: Flink 1.1)
Upgrading Programs
 A program starting from a savepoint can differ from the
program that created the savepoint
• Unique operator names match state and operator
 Mechanism be used to fix bugs in programs, to evolve
programs, parameters, libraries, …
58
State Backends
 Large state is a collection of key/value pairs
 State backend defines what data structure holds the
state, plus how it is snapshotted
 Most common choices
• Main memory – snapshots to master
• Main memory – snapshots to dist. filesystem
• RocksDB – snapshots to dist. filesystem
59
Complex Event Processing Primer
60
Example: Temperature Monitoring
 Receiving temperature an power events
from sensors
 Looking for temperatures repeatedly
exceeding thresholds within a
short time period (10 secs)
61
Event Types
62
Defining Patterns
63
Generating Alerts
64
An Outlook on Things to Come
65
Flink in the wild
66
30 billion events daily 2 billion events in
10 1Gb machines
data integration & distribution
platform
See talks by at
Roadmap
 Dynamic Scaling, Resource Elasticity
 Stream SQL
 CEP enhancements
 Incremental & asynchronous state snapshotting
 Mesos support
 More connectors, end-to-end exactly once
 API enhancements (e.g., joins, slowly changing inputs)
 Security (data encryption, Kerberos with Kafka)
67
68
I stream, do you?

More Related Content

What's hot

Apache Flink Training: System Overview
Apache Flink Training: System OverviewApache Flink Training: System Overview
Apache Flink Training: System OverviewFlink Forward
 
Streaming Data Lakes using Kafka Connect + Apache Hudi | Vinoth Chandar, Apac...
Streaming Data Lakes using Kafka Connect + Apache Hudi | Vinoth Chandar, Apac...Streaming Data Lakes using Kafka Connect + Apache Hudi | Vinoth Chandar, Apac...
Streaming Data Lakes using Kafka Connect + Apache Hudi | Vinoth Chandar, Apac...HostedbyConfluent
 
Apache Flink and what it is used for
Apache Flink and what it is used forApache Flink and what it is used for
Apache Flink and what it is used forAljoscha Krettek
 
Fundamentals of Apache Kafka
Fundamentals of Apache KafkaFundamentals of Apache Kafka
Fundamentals of Apache KafkaChhavi Parasher
 
Building a fully managed stream processing platform on Flink at scale for Lin...
Building a fully managed stream processing platform on Flink at scale for Lin...Building a fully managed stream processing platform on Flink at scale for Lin...
Building a fully managed stream processing platform on Flink at scale for Lin...Flink Forward
 
Introduction to Apache Flink
Introduction to Apache FlinkIntroduction to Apache Flink
Introduction to Apache Flinkdatamantra
 
Making Apache Spark Better with Delta Lake
Making Apache Spark Better with Delta LakeMaking Apache Spark Better with Delta Lake
Making Apache Spark Better with Delta LakeDatabricks
 
Unlocking the Power of Apache Flink: An Introduction in 4 Acts
Unlocking the Power of Apache Flink: An Introduction in 4 ActsUnlocking the Power of Apache Flink: An Introduction in 4 Acts
Unlocking the Power of Apache Flink: An Introduction in 4 ActsHostedbyConfluent
 
An Introduction to Confluent Cloud: Apache Kafka as a Service
An Introduction to Confluent Cloud: Apache Kafka as a ServiceAn Introduction to Confluent Cloud: Apache Kafka as a Service
An Introduction to Confluent Cloud: Apache Kafka as a Serviceconfluent
 
ksqlDB: A Stream-Relational Database System
ksqlDB: A Stream-Relational Database SystemksqlDB: A Stream-Relational Database System
ksqlDB: A Stream-Relational Database Systemconfluent
 
Serverless Kafka on AWS as Part of a Cloud-native Data Lake Architecture
Serverless Kafka on AWS as Part of a Cloud-native Data Lake ArchitectureServerless Kafka on AWS as Part of a Cloud-native Data Lake Architecture
Serverless Kafka on AWS as Part of a Cloud-native Data Lake ArchitectureKai Wähner
 
Introduction to Apache Flink - Fast and reliable big data processing
Introduction to Apache Flink - Fast and reliable big data processingIntroduction to Apache Flink - Fast and reliable big data processing
Introduction to Apache Flink - Fast and reliable big data processingTill Rohrmann
 
Confluent Workshop Series: ksqlDB로 스트리밍 앱 빌드
Confluent Workshop Series: ksqlDB로 스트리밍 앱 빌드Confluent Workshop Series: ksqlDB로 스트리밍 앱 빌드
Confluent Workshop Series: ksqlDB로 스트리밍 앱 빌드confluent
 
Serverless Kafka and Spark in a Multi-Cloud Lakehouse Architecture
Serverless Kafka and Spark in a Multi-Cloud Lakehouse ArchitectureServerless Kafka and Spark in a Multi-Cloud Lakehouse Architecture
Serverless Kafka and Spark in a Multi-Cloud Lakehouse ArchitectureKai Wähner
 
Apache kafka 모니터링을 위한 Metrics 이해 및 최적화 방안
Apache kafka 모니터링을 위한 Metrics 이해 및 최적화 방안Apache kafka 모니터링을 위한 Metrics 이해 및 최적화 방안
Apache kafka 모니터링을 위한 Metrics 이해 및 최적화 방안SANG WON PARK
 

What's hot (20)

Apache Flink Training: System Overview
Apache Flink Training: System OverviewApache Flink Training: System Overview
Apache Flink Training: System Overview
 
Streaming Data Lakes using Kafka Connect + Apache Hudi | Vinoth Chandar, Apac...
Streaming Data Lakes using Kafka Connect + Apache Hudi | Vinoth Chandar, Apac...Streaming Data Lakes using Kafka Connect + Apache Hudi | Vinoth Chandar, Apac...
Streaming Data Lakes using Kafka Connect + Apache Hudi | Vinoth Chandar, Apac...
 
Apache Flink and what it is used for
Apache Flink and what it is used forApache Flink and what it is used for
Apache Flink and what it is used for
 
Flink vs. Spark
Flink vs. SparkFlink vs. Spark
Flink vs. Spark
 
Fundamentals of Apache Kafka
Fundamentals of Apache KafkaFundamentals of Apache Kafka
Fundamentals of Apache Kafka
 
Building a fully managed stream processing platform on Flink at scale for Lin...
Building a fully managed stream processing platform on Flink at scale for Lin...Building a fully managed stream processing platform on Flink at scale for Lin...
Building a fully managed stream processing platform on Flink at scale for Lin...
 
Apache Kafka at LinkedIn
Apache Kafka at LinkedInApache Kafka at LinkedIn
Apache Kafka at LinkedIn
 
Introduction to Apache Flink
Introduction to Apache FlinkIntroduction to Apache Flink
Introduction to Apache Flink
 
Apache flink
Apache flinkApache flink
Apache flink
 
Making Apache Spark Better with Delta Lake
Making Apache Spark Better with Delta LakeMaking Apache Spark Better with Delta Lake
Making Apache Spark Better with Delta Lake
 
Unlocking the Power of Apache Flink: An Introduction in 4 Acts
Unlocking the Power of Apache Flink: An Introduction in 4 ActsUnlocking the Power of Apache Flink: An Introduction in 4 Acts
Unlocking the Power of Apache Flink: An Introduction in 4 Acts
 
Kafka 101
Kafka 101Kafka 101
Kafka 101
 
An Introduction to Confluent Cloud: Apache Kafka as a Service
An Introduction to Confluent Cloud: Apache Kafka as a ServiceAn Introduction to Confluent Cloud: Apache Kafka as a Service
An Introduction to Confluent Cloud: Apache Kafka as a Service
 
ksqlDB: A Stream-Relational Database System
ksqlDB: A Stream-Relational Database SystemksqlDB: A Stream-Relational Database System
ksqlDB: A Stream-Relational Database System
 
Serverless Kafka on AWS as Part of a Cloud-native Data Lake Architecture
Serverless Kafka on AWS as Part of a Cloud-native Data Lake ArchitectureServerless Kafka on AWS as Part of a Cloud-native Data Lake Architecture
Serverless Kafka on AWS as Part of a Cloud-native Data Lake Architecture
 
Apache Flink Deep Dive
Apache Flink Deep DiveApache Flink Deep Dive
Apache Flink Deep Dive
 
Introduction to Apache Flink - Fast and reliable big data processing
Introduction to Apache Flink - Fast and reliable big data processingIntroduction to Apache Flink - Fast and reliable big data processing
Introduction to Apache Flink - Fast and reliable big data processing
 
Confluent Workshop Series: ksqlDB로 스트리밍 앱 빌드
Confluent Workshop Series: ksqlDB로 스트리밍 앱 빌드Confluent Workshop Series: ksqlDB로 스트리밍 앱 빌드
Confluent Workshop Series: ksqlDB로 스트리밍 앱 빌드
 
Serverless Kafka and Spark in a Multi-Cloud Lakehouse Architecture
Serverless Kafka and Spark in a Multi-Cloud Lakehouse ArchitectureServerless Kafka and Spark in a Multi-Cloud Lakehouse Architecture
Serverless Kafka and Spark in a Multi-Cloud Lakehouse Architecture
 
Apache kafka 모니터링을 위한 Metrics 이해 및 최적화 방안
Apache kafka 모니터링을 위한 Metrics 이해 및 최적화 방안Apache kafka 모니터링을 위한 Metrics 이해 및 최적화 방안
Apache kafka 모니터링을 위한 Metrics 이해 및 최적화 방안
 

Viewers also liked

Debunking Common Myths in Stream Processing
Debunking Common Myths in Stream ProcessingDebunking Common Myths in Stream Processing
Debunking Common Myths in Stream ProcessingKostas Tzoumas
 
Apache Flink: API, runtime, and project roadmap
Apache Flink: API, runtime, and project roadmapApache Flink: API, runtime, and project roadmap
Apache Flink: API, runtime, and project roadmapKostas Tzoumas
 
Inside Parquet Format
Inside Parquet FormatInside Parquet Format
Inside Parquet FormatYue Chen
 
Streaming Analytics & CEP - Two sides of the same coin?
Streaming Analytics & CEP - Two sides of the same coin?Streaming Analytics & CEP - Two sides of the same coin?
Streaming Analytics & CEP - Two sides of the same coin?Till Rohrmann
 
Fabian Hueske - Stream Analytics with SQL on Apache Flink
Fabian Hueske - Stream Analytics with SQL on Apache FlinkFabian Hueske - Stream Analytics with SQL on Apache Flink
Fabian Hueske - Stream Analytics with SQL on Apache FlinkVerverica
 
Inside HDFS Append
Inside HDFS AppendInside HDFS Append
Inside HDFS AppendYue Chen
 

Viewers also liked (6)

Debunking Common Myths in Stream Processing
Debunking Common Myths in Stream ProcessingDebunking Common Myths in Stream Processing
Debunking Common Myths in Stream Processing
 
Apache Flink: API, runtime, and project roadmap
Apache Flink: API, runtime, and project roadmapApache Flink: API, runtime, and project roadmap
Apache Flink: API, runtime, and project roadmap
 
Inside Parquet Format
Inside Parquet FormatInside Parquet Format
Inside Parquet Format
 
Streaming Analytics & CEP - Two sides of the same coin?
Streaming Analytics & CEP - Two sides of the same coin?Streaming Analytics & CEP - Two sides of the same coin?
Streaming Analytics & CEP - Two sides of the same coin?
 
Fabian Hueske - Stream Analytics with SQL on Apache Flink
Fabian Hueske - Stream Analytics with SQL on Apache FlinkFabian Hueske - Stream Analytics with SQL on Apache Flink
Fabian Hueske - Stream Analytics with SQL on Apache Flink
 
Inside HDFS Append
Inside HDFS AppendInside HDFS Append
Inside HDFS Append
 

Similar to Streaming Analytics with Apache Flink 1.0 Explained

Advanced Streaming Analytics with Apache Flink and Apache Kafka, Stephan Ewen
Advanced Streaming Analytics with Apache Flink and Apache Kafka, Stephan EwenAdvanced Streaming Analytics with Apache Flink and Apache Kafka, Stephan Ewen
Advanced Streaming Analytics with Apache Flink and Apache Kafka, Stephan Ewenconfluent
 
Flink forward SF 2017: Ufuk Celebi - The Stream Processor as a Database: Buil...
Flink forward SF 2017: Ufuk Celebi - The Stream Processor as a Database: Buil...Flink forward SF 2017: Ufuk Celebi - The Stream Processor as a Database: Buil...
Flink forward SF 2017: Ufuk Celebi - The Stream Processor as a Database: Buil...Flink Forward
 
Spark Streaming - Meetup Data Analysis
Spark Streaming - Meetup Data AnalysisSpark Streaming - Meetup Data Analysis
Spark Streaming - Meetup Data AnalysisSushmanth Sagala
 
Flink 0.10 @ Bay Area Meetup (October 2015)
Flink 0.10 @ Bay Area Meetup (October 2015)Flink 0.10 @ Bay Area Meetup (October 2015)
Flink 0.10 @ Bay Area Meetup (October 2015)Stephan Ewen
 
Apache flink: data streaming as a basis for all analytics by Kostas Tzoumas a...
Apache flink: data streaming as a basis for all analytics by Kostas Tzoumas a...Apache flink: data streaming as a basis for all analytics by Kostas Tzoumas a...
Apache flink: data streaming as a basis for all analytics by Kostas Tzoumas a...Big Data Spain
 
Meet the squirrel @ #CSHUG
Meet the squirrel @ #CSHUGMeet the squirrel @ #CSHUG
Meet the squirrel @ #CSHUGMárton Balassi
 
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
 
K. Tzoumas & S. Ewen – Flink Forward Keynote
K. Tzoumas & S. Ewen – Flink Forward KeynoteK. Tzoumas & S. Ewen – Flink Forward Keynote
K. Tzoumas & S. Ewen – Flink Forward KeynoteFlink Forward
 
Apache Flink @ Tel Aviv / Herzliya Meetup
Apache Flink @ Tel Aviv / Herzliya MeetupApache Flink @ Tel Aviv / Herzliya Meetup
Apache Flink @ Tel Aviv / Herzliya MeetupRobert Metzger
 
Unified Stream & Batch Processing with Apache Flink (Hadoop Summit Dublin 2016)
Unified Stream & Batch Processing with Apache Flink (Hadoop Summit Dublin 2016)Unified Stream & Batch Processing with Apache Flink (Hadoop Summit Dublin 2016)
Unified Stream & Batch Processing with Apache Flink (Hadoop Summit Dublin 2016)ucelebi
 
Go Concurrency Patterns
Go Concurrency PatternsGo Concurrency Patterns
Go Concurrency PatternsElifTech
 
Continuous Processing with Apache Flink - Strata London 2016
Continuous Processing with Apache Flink - Strata London 2016Continuous Processing with Apache Flink - Strata London 2016
Continuous Processing with Apache Flink - Strata London 2016Stephan Ewen
 
A formalization of complex event stream processing
A formalization of complex event stream processingA formalization of complex event stream processing
A formalization of complex event stream processingSylvain Hallé
 
FIWARE Global Summit - Real-time Processing of Historic Context Information u...
FIWARE Global Summit - Real-time Processing of Historic Context Information u...FIWARE Global Summit - Real-time Processing of Historic Context Information u...
FIWARE Global Summit - Real-time Processing of Historic Context Information u...FIWARE
 
Streaming Data Flow with Apache Flink @ Paris Flink Meetup 2015
Streaming Data Flow with Apache Flink @ Paris Flink Meetup 2015Streaming Data Flow with Apache Flink @ Paris Flink Meetup 2015
Streaming Data Flow with Apache Flink @ Paris Flink Meetup 2015Till Rohrmann
 
Building Applications with Streams and Snapshots
Building Applications with Streams and SnapshotsBuilding Applications with Streams and Snapshots
Building Applications with Streams and SnapshotsJ On The Beach
 
Wprowadzenie do technologii Big Data / Intro to Big Data Ecosystem
Wprowadzenie do technologii Big Data / Intro to Big Data EcosystemWprowadzenie do technologii Big Data / Intro to Big Data Ecosystem
Wprowadzenie do technologii Big Data / Intro to Big Data EcosystemSages
 

Similar to Streaming Analytics with Apache Flink 1.0 Explained (20)

Advanced Streaming Analytics with Apache Flink and Apache Kafka, Stephan Ewen
Advanced Streaming Analytics with Apache Flink and Apache Kafka, Stephan EwenAdvanced Streaming Analytics with Apache Flink and Apache Kafka, Stephan Ewen
Advanced Streaming Analytics with Apache Flink and Apache Kafka, Stephan Ewen
 
Flink forward SF 2017: Ufuk Celebi - The Stream Processor as a Database: Buil...
Flink forward SF 2017: Ufuk Celebi - The Stream Processor as a Database: Buil...Flink forward SF 2017: Ufuk Celebi - The Stream Processor as a Database: Buil...
Flink forward SF 2017: Ufuk Celebi - The Stream Processor as a Database: Buil...
 
Spark Streaming - Meetup Data Analysis
Spark Streaming - Meetup Data AnalysisSpark Streaming - Meetup Data Analysis
Spark Streaming - Meetup Data Analysis
 
Flink 0.10 @ Bay Area Meetup (October 2015)
Flink 0.10 @ Bay Area Meetup (October 2015)Flink 0.10 @ Bay Area Meetup (October 2015)
Flink 0.10 @ Bay Area Meetup (October 2015)
 
Apache flink: data streaming as a basis for all analytics by Kostas Tzoumas a...
Apache flink: data streaming as a basis for all analytics by Kostas Tzoumas a...Apache flink: data streaming as a basis for all analytics by Kostas Tzoumas a...
Apache flink: data streaming as a basis for all analytics by Kostas Tzoumas a...
 
Meet the squirrel @ #CSHUG
Meet the squirrel @ #CSHUGMeet the squirrel @ #CSHUG
Meet the squirrel @ #CSHUG
 
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
 
K. Tzoumas & S. Ewen – Flink Forward Keynote
K. Tzoumas & S. Ewen – Flink Forward KeynoteK. Tzoumas & S. Ewen – Flink Forward Keynote
K. Tzoumas & S. Ewen – Flink Forward Keynote
 
Apache Flink @ Tel Aviv / Herzliya Meetup
Apache Flink @ Tel Aviv / Herzliya MeetupApache Flink @ Tel Aviv / Herzliya Meetup
Apache Flink @ Tel Aviv / Herzliya Meetup
 
Unified Stream & Batch Processing with Apache Flink (Hadoop Summit Dublin 2016)
Unified Stream & Batch Processing with Apache Flink (Hadoop Summit Dublin 2016)Unified Stream & Batch Processing with Apache Flink (Hadoop Summit Dublin 2016)
Unified Stream & Batch Processing with Apache Flink (Hadoop Summit Dublin 2016)
 
About time
About timeAbout time
About time
 
Go on!
Go on!Go on!
Go on!
 
Go Concurrency Patterns
Go Concurrency PatternsGo Concurrency Patterns
Go Concurrency Patterns
 
Continuous Processing with Apache Flink - Strata London 2016
Continuous Processing with Apache Flink - Strata London 2016Continuous Processing with Apache Flink - Strata London 2016
Continuous Processing with Apache Flink - Strata London 2016
 
A formalization of complex event stream processing
A formalization of complex event stream processingA formalization of complex event stream processing
A formalization of complex event stream processing
 
Reactive Extensions
Reactive ExtensionsReactive Extensions
Reactive Extensions
 
FIWARE Global Summit - Real-time Processing of Historic Context Information u...
FIWARE Global Summit - Real-time Processing of Historic Context Information u...FIWARE Global Summit - Real-time Processing of Historic Context Information u...
FIWARE Global Summit - Real-time Processing of Historic Context Information u...
 
Streaming Data Flow with Apache Flink @ Paris Flink Meetup 2015
Streaming Data Flow with Apache Flink @ Paris Flink Meetup 2015Streaming Data Flow with Apache Flink @ Paris Flink Meetup 2015
Streaming Data Flow with Apache Flink @ Paris Flink Meetup 2015
 
Building Applications with Streams and Snapshots
Building Applications with Streams and SnapshotsBuilding Applications with Streams and Snapshots
Building Applications with Streams and Snapshots
 
Wprowadzenie do technologii Big Data / Intro to Big Data Ecosystem
Wprowadzenie do technologii Big Data / Intro to Big Data EcosystemWprowadzenie do technologii Big Data / Intro to Big Data Ecosystem
Wprowadzenie do technologii Big Data / Intro to Big Data Ecosystem
 

More from Stephan Ewen

The Stream Processor as the Database - Apache Flink @ Berlin buzzwords
The Stream Processor as the Database - Apache Flink @ Berlin buzzwords   The Stream Processor as the Database - Apache Flink @ Berlin buzzwords
The Stream Processor as the Database - Apache Flink @ Berlin buzzwords Stephan Ewen
 
Apache Flink Berlin Meetup May 2016
Apache Flink Berlin Meetup May 2016Apache Flink Berlin Meetup May 2016
Apache Flink Berlin Meetup May 2016Stephan Ewen
 
Flink use cases @ bay area meetup (october 2015)
Flink use cases @ bay area meetup (october 2015)Flink use cases @ bay area meetup (october 2015)
Flink use cases @ bay area meetup (october 2015)Stephan Ewen
 
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
 
Apache Flink@ Strata & Hadoop World London
Apache Flink@ Strata & Hadoop World LondonApache Flink@ Strata & Hadoop World London
Apache Flink@ Strata & Hadoop World LondonStephan 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
 
Flink history, roadmap and vision
Flink history, roadmap and visionFlink history, roadmap and vision
Flink history, roadmap and visionStephan Ewen
 

More from Stephan Ewen (7)

The Stream Processor as the Database - Apache Flink @ Berlin buzzwords
The Stream Processor as the Database - Apache Flink @ Berlin buzzwords   The Stream Processor as the Database - Apache Flink @ Berlin buzzwords
The Stream Processor as the Database - Apache Flink @ Berlin buzzwords
 
Apache Flink Berlin Meetup May 2016
Apache Flink Berlin Meetup May 2016Apache Flink Berlin Meetup May 2016
Apache Flink Berlin Meetup May 2016
 
Flink use cases @ bay area meetup (october 2015)
Flink use cases @ bay area meetup (october 2015)Flink use cases @ bay area meetup (october 2015)
Flink use cases @ bay area meetup (october 2015)
 
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
 
Apache Flink@ Strata & Hadoop World London
Apache Flink@ Strata & Hadoop World LondonApache Flink@ Strata & Hadoop World London
Apache Flink@ Strata & Hadoop World London
 
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...
 
Flink history, roadmap and vision
Flink history, roadmap and visionFlink history, roadmap and vision
Flink history, roadmap and vision
 

Recently uploaded

"ML in Production",Oleksandr Bagan
"ML in Production",Oleksandr Bagan"ML in Production",Oleksandr Bagan
"ML in Production",Oleksandr BaganFwdays
 
Benefits Of Flutter Compared To Other Frameworks
Benefits Of Flutter Compared To Other FrameworksBenefits Of Flutter Compared To Other Frameworks
Benefits Of Flutter Compared To Other FrameworksSoftradix Technologies
 
Install Stable Diffusion in windows machine
Install Stable Diffusion in windows machineInstall Stable Diffusion in windows machine
Install Stable Diffusion in windows machinePadma Pradeep
 
Key Features Of Token Development (1).pptx
Key  Features Of Token  Development (1).pptxKey  Features Of Token  Development (1).pptx
Key Features Of Token Development (1).pptxLBM Solutions
 
Understanding the Laravel MVC Architecture
Understanding the Laravel MVC ArchitectureUnderstanding the Laravel MVC Architecture
Understanding the Laravel MVC ArchitecturePixlogix Infotech
 
WordPress Websites for Engineers: Elevate Your Brand
WordPress Websites for Engineers: Elevate Your BrandWordPress Websites for Engineers: Elevate Your Brand
WordPress Websites for Engineers: Elevate Your Brandgvaughan
 
Are Multi-Cloud and Serverless Good or Bad?
Are Multi-Cloud and Serverless Good or Bad?Are Multi-Cloud and Serverless Good or Bad?
Are Multi-Cloud and Serverless Good or Bad?Mattias Andersson
 
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
 
Pigging Solutions Piggable Sweeping Elbows
Pigging Solutions Piggable Sweeping ElbowsPigging Solutions Piggable Sweeping Elbows
Pigging Solutions Piggable Sweeping ElbowsPigging Solutions
 
"Debugging python applications inside k8s environment", Andrii Soldatenko
"Debugging python applications inside k8s environment", Andrii Soldatenko"Debugging python applications inside k8s environment", Andrii Soldatenko
"Debugging python applications inside k8s environment", Andrii SoldatenkoFwdays
 
Transcript: New from BookNet Canada for 2024: BNC BiblioShare - Tech Forum 2024
Transcript: New from BookNet Canada for 2024: BNC BiblioShare - Tech Forum 2024Transcript: New from BookNet Canada for 2024: BNC BiblioShare - Tech Forum 2024
Transcript: New from BookNet Canada for 2024: BNC BiblioShare - Tech Forum 2024BookNet Canada
 
New from BookNet Canada for 2024: BNC BiblioShare - Tech Forum 2024
New from BookNet Canada for 2024: BNC BiblioShare - Tech Forum 2024New from BookNet Canada for 2024: BNC BiblioShare - Tech Forum 2024
New from BookNet Canada for 2024: BNC BiblioShare - Tech Forum 2024BookNet Canada
 
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
 
Scanning the Internet for External Cloud Exposures via SSL Certs
Scanning the Internet for External Cloud Exposures via SSL CertsScanning the Internet for External Cloud Exposures via SSL Certs
Scanning the Internet for External Cloud Exposures via SSL CertsRizwan Syed
 
Human Factors of XR: Using Human Factors to Design XR Systems
Human Factors of XR: Using Human Factors to Design XR SystemsHuman Factors of XR: Using Human Factors to Design XR Systems
Human Factors of XR: Using Human Factors to Design XR SystemsMark Billinghurst
 
Kotlin Multiplatform & Compose Multiplatform - Starter kit for pragmatics
Kotlin Multiplatform & Compose Multiplatform - Starter kit for pragmaticsKotlin Multiplatform & Compose Multiplatform - Starter kit for pragmatics
Kotlin Multiplatform & Compose Multiplatform - Starter kit for pragmaticscarlostorres15106
 
SIP trunking in Janus @ Kamailio World 2024
SIP trunking in Janus @ Kamailio World 2024SIP trunking in Janus @ Kamailio World 2024
SIP trunking in Janus @ Kamailio World 2024Lorenzo Miniero
 
Designing IA for AI - Information Architecture Conference 2024
Designing IA for AI - Information Architecture Conference 2024Designing IA for AI - Information Architecture Conference 2024
Designing IA for AI - Information Architecture Conference 2024Enterprise Knowledge
 
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
 

Recently uploaded (20)

"ML in Production",Oleksandr Bagan
"ML in Production",Oleksandr Bagan"ML in Production",Oleksandr Bagan
"ML in Production",Oleksandr Bagan
 
Benefits Of Flutter Compared To Other Frameworks
Benefits Of Flutter Compared To Other FrameworksBenefits Of Flutter Compared To Other Frameworks
Benefits Of Flutter Compared To Other Frameworks
 
Install Stable Diffusion in windows machine
Install Stable Diffusion in windows machineInstall Stable Diffusion in windows machine
Install Stable Diffusion in windows machine
 
Key Features Of Token Development (1).pptx
Key  Features Of Token  Development (1).pptxKey  Features Of Token  Development (1).pptx
Key Features Of Token Development (1).pptx
 
Understanding the Laravel MVC Architecture
Understanding the Laravel MVC ArchitectureUnderstanding the Laravel MVC Architecture
Understanding the Laravel MVC Architecture
 
WordPress Websites for Engineers: Elevate Your Brand
WordPress Websites for Engineers: Elevate Your BrandWordPress Websites for Engineers: Elevate Your Brand
WordPress Websites for Engineers: Elevate Your Brand
 
Are Multi-Cloud and Serverless Good or Bad?
Are Multi-Cloud and Serverless Good or Bad?Are Multi-Cloud and Serverless Good or Bad?
Are Multi-Cloud and Serverless Good or Bad?
 
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
 
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
 
Pigging Solutions Piggable Sweeping Elbows
Pigging Solutions Piggable Sweeping ElbowsPigging Solutions Piggable Sweeping Elbows
Pigging Solutions Piggable Sweeping Elbows
 
"Debugging python applications inside k8s environment", Andrii Soldatenko
"Debugging python applications inside k8s environment", Andrii Soldatenko"Debugging python applications inside k8s environment", Andrii Soldatenko
"Debugging python applications inside k8s environment", Andrii Soldatenko
 
Transcript: New from BookNet Canada for 2024: BNC BiblioShare - Tech Forum 2024
Transcript: New from BookNet Canada for 2024: BNC BiblioShare - Tech Forum 2024Transcript: New from BookNet Canada for 2024: BNC BiblioShare - Tech Forum 2024
Transcript: New from BookNet Canada for 2024: BNC BiblioShare - Tech Forum 2024
 
New from BookNet Canada for 2024: BNC BiblioShare - Tech Forum 2024
New from BookNet Canada for 2024: BNC BiblioShare - Tech Forum 2024New from BookNet Canada for 2024: BNC BiblioShare - Tech Forum 2024
New from BookNet Canada for 2024: BNC BiblioShare - Tech Forum 2024
 
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
 
Scanning the Internet for External Cloud Exposures via SSL Certs
Scanning the Internet for External Cloud Exposures via SSL CertsScanning the Internet for External Cloud Exposures via SSL Certs
Scanning the Internet for External Cloud Exposures via SSL Certs
 
Human Factors of XR: Using Human Factors to Design XR Systems
Human Factors of XR: Using Human Factors to Design XR SystemsHuman Factors of XR: Using Human Factors to Design XR Systems
Human Factors of XR: Using Human Factors to Design XR Systems
 
Kotlin Multiplatform & Compose Multiplatform - Starter kit for pragmatics
Kotlin Multiplatform & Compose Multiplatform - Starter kit for pragmaticsKotlin Multiplatform & Compose Multiplatform - Starter kit for pragmatics
Kotlin Multiplatform & Compose Multiplatform - Starter kit for pragmatics
 
SIP trunking in Janus @ Kamailio World 2024
SIP trunking in Janus @ Kamailio World 2024SIP trunking in Janus @ Kamailio World 2024
SIP trunking in Janus @ Kamailio World 2024
 
Designing IA for AI - Information Architecture Conference 2024
Designing IA for AI - Information Architecture Conference 2024Designing IA for AI - Information Architecture Conference 2024
Designing IA for AI - Information Architecture Conference 2024
 
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
 

Streaming Analytics with Apache Flink 1.0 Explained

  • 2. Apache Flink Stack 2 DataStream API Stream Processing DataSet API Batch Processing Runtime Distributed Streaming Data Flow Libraries Streaming and batch as first class citizens.
  • 3. Today 3 Streaming and batch as first class citizens. DataStream API Stream Processing DataSet API Batch Processing Runtime Distributed Streaming Data Flow Libraries
  • 4. 4 Streaming is the next programming paradigm for data applications, and you need to start thinking in terms of streams.
  • 5. 5 Streaming technology is enabling the obvious: continuous processing on data that is continuously produced
  • 6. Continuous Processing with Batch  Continuous ingestion  Periodic (e.g., hourly) files  Periodic batch jobs 6
  • 7. λ Architecture  "Batch layer": what we had before  "Stream layer": approximate early results 7
  • 8. A Stream Processing Pipeline 8 collect log analyze serve & store
  • 9. A brief History of Flink 9 January ‘10 December ‘14 v0.5 v0.6 v0.7 March ‘16 Flink Project Incubation Top Level Project v0.8 v0.10 Release 1.0 Project Stratosphere (Flink precursor) v0.9 April ‘14
  • 10. A brief History of Flink 10 January ‘10 December ‘14 v0.5 v0.6 v0.7 March ‘16 Flink Project Incubation Top Level Project v0.8 v0.10 Release 1.0 Project Stratosphere (Flink precursor) v0.9 April ‘14 The academia gap: Reading/writing papers, teaching, worrying about thesis Realizing this might be interesting to people beyond academia (even more so, actually)
  • 11. Programs and Dataflows 11 Source Transformation Transformation Sink val lines: DataStream[String] = env.addSource(new FlinkKafkaConsumer09(…)) val events: DataStream[Event] = lines.map((line) => parse(line)) val stats: DataStream[Statistic] = stream .keyBy("sensor") .timeWindow(Time.seconds(5)) .sum(new MyAggregationFunction()) stats.addSink(new RollingSink(path)) Source [1] map() [1] keyBy()/ window()/ apply() [1] Sink [1] Source [2] map() [2] keyBy()/ window()/ apply() [2] Streaming Dataflow
  • 12. What makes Flink flink? 12 Low latency High Throughput Well-behaved flow control (back pressure) Make more sense of data Works on real-time and historic data True Streaming Event Time APIs Libraries Stateful Streaming Globally consistent savepoints Exactly-once semantics for fault tolerance Windows & user-defined state Flexible windows (time, count, session, roll-your own) Complex Event Processing
  • 14. Time-Windowed Aggregations 14 case class Event(sensor: String, measure: Double) val env = StreamExecutionEnvironment.getExecutionEnvironment val stream: DataStream[Event] = env.addSource(…) stream .keyBy("sensor") .timeWindow(Time.seconds(5)) .sum("measure")
  • 15. Time-Windowed Aggregations 15 case class Event(sensor: String, measure: Double) val env = StreamExecutionEnvironment.getExecutionEnvironment val stream: DataStream[Event] = env.addSource(…) stream .keyBy("sensor") .timeWindow(Time.seconds(60), Time.seconds(5)) .sum("measure")
  • 16. Session-Windowed Aggregations 16 case class Event(sensor: String, measure: Double) val env = StreamExecutionEnvironment.getExecutionEnvironment val stream: DataStream[Event] = env.addSource(…) stream .keyBy("sensor") .window(EventTimeSessionWindows.withGap(Time.seconds(60))) .max("measure")
  • 17. Session-Windowed Aggregations 17 case class Event(sensor: String, measure: Double) val env = StreamExecutionEnvironment.getExecutionEnvironment val stream: DataStream[Event] = env.addSource(…) stream .keyBy("sensor") .window(EventTimeSessionWindows.withGap(Time.seconds(60))) .max("measure") Flink 1.1 syntax
  • 18. Pattern Detection 18 case class Event(producer: String, evtType: Int, msg: String) case class Alert(msg: String) val stream: DataStream[Event] = env.addSource(…) stream .keyBy("producer") .flatMap(new RichFlatMapFuncion[Event, Alert]() { lazy val state: ValueState[Int] = getRuntimeContext.getState(…) def flatMap(event: Event, out: Collector[Alert]) = { val newState = state.value() match { case 0 if (event.evtType == 0) => 1 case 1 if (event.evtType == 1) => 0 case x => out.collect(Alert(event.msg, x)); 0 } state.update(newState) } })
  • 19. Pattern Detection 19 case class Event(producer: String, evtType: Int, msg: String) case class Alert(msg: String) val stream: DataStream[Event] = env.addSource(…) stream .keyBy("producer") .flatMap(new RichFlatMapFuncion[Event, Alert]() { lazy val state: ValueState[Int] = getRuntimeContext.getState(…) def flatMap(event: Event, out: Collector[Alert]) = { val newState = state.value() match { case 0 if (event.evtType == 0) => 1 case 1 if (event.evtType == 1) => 0 case x => out.collect(Alert(event.msg, x)); 0 } state.update(newState) } }) Embedded key/value state store
  • 20. Many more  Joining streams (e.g. combine readings from sensor)  Detecting Patterns (CEP)  Applying (changing) rules or models to events  Training and applying online machine learning models  … 20
  • 22. 22 The biggest change in moving from batch to streaming is handling time explicitly
  • 23. Example: Windowing by Time 23 case class Event(id: String, measure: Double, timestamp: Long) val env = StreamExecutionEnvironment.getExecutionEnvironment val stream: DataStream[Event] = env.addSource(…) stream .keyBy("id") .timeWindow(Time.seconds(15), Time.seconds(5)) .sum("measure")
  • 24. Example: Windowing by Time 24 case class Event(id: String, measure: Double, timestamp: Long) val env = StreamExecutionEnvironment.getExecutionEnvironment val stream: DataStream[Event] = env.addSource(…) stream .keyBy("id") .timeWindow(Time.seconds(15), Time.seconds(5)) .sum("measure")
  • 25. Different Notions of Time 25 Event Producer Message Queue Flink Data Source Flink Window Operator partition 1 partition 2 Event Time Ingestion Time Window Processing Time
  • 26. 1977 1980 1983 1999 2002 2005 2015 Processing Time Episode IV Episode V Episode VI Episode I Episode II Episode III Episode VII Event Time Event Time vs. Processing Time 26
  • 27. Out of order Streams 27 Events occur on devices Queue / Log Events analyzed in a data streaming system Stream Analysis Events stored in a log
  • 28. Out of order Streams 28
  • 29. Out of order Streams 29
  • 30. Out of order Streams 30
  • 31. Out of order Streams 31 Out of order !!! First burst of events Second burst of events
  • 32. Out of order Streams 32 Event time windows Arrival time windows Instant event-at-a-time First burst of events Second burst of events
  • 33. Processing Time 33 case class Event(id: String, measure: Double, timestamp: Long) val env = StreamExecutionEnvironment.getExecutionEnvironment env.setStreamTimeCharacteristic(ProcessingTime) val stream: DataStream[Event] = env.addSource(…) stream .keyBy("id") .timeWindow(Time.seconds(15), Time.seconds(5)) .sum("measure") Window by operator's processing time
  • 34. Ingestion Time 34 case class Event(id: String, measure: Double, timestamp: Long) val env = StreamExecutionEnvironment.getExecutionEnvironment env.setStreamTimeCharacteristic(IngestionTime) val stream: DataStream[Event] = env.addSource(…) stream .keyBy("id") .timeWindow(Time.seconds(15), Time.seconds(5)) .sum("measure")
  • 35. Event Time 35 case class Event(id: String, measure: Double, timestamp: Long) val env = StreamExecutionEnvironment.getExecutionEnvironment env.setStreamTimeCharacteristic(EventTime) val stream: DataStream[Event] = env.addSource(…) stream .keyBy("id") .timeWindow(Time.seconds(15), Time.seconds(5)) .sum("measure")
  • 36. Event Time 36 case class Event(id: String, measure: Double, timestamp: Long) val env = StreamExecutionEnvironment.getExecutionEnvironment env.setStreamTimeCharacteristic(EventTime) val stream: DataStream[Event] = env.addSource(…) val tsStream = stream.assignAscendingTimestamps(_.timestamp) tsStream .keyBy("id") .timeWindow(Time.seconds(15), Time.seconds(5)) .sum("measure")
  • 37. Event Time 37 case class Event(id: String, measure: Double, timestamp: Long) val env = StreamExecutionEnvironment.getExecutionEnvironment env.setStreamTimeCharacteristic(EventTime) val stream: DataStream[Event] = env.addSource(…) val tsStream = stream.assignTimestampsAndWatermarks( new MyTimestampsAndWatermarkGenerator()) tsStream .keyBy("id") .timeWindow(Time.seconds(15), Time.seconds(5)) .sum("measure")
  • 38. Watermarks 38 7 W(11)W(17) 11159121417122220 171921 Watermark Event Event timestamp Stream (in order) 7 W(11)W(20) Watermark 991011141517 Event Event timestamp 1820 192123 Stream (out of order)
  • 39. Watermarks in Parallel 39 Source (1) Source (2) map (1) map (2) window (1) window (2) 29 29 17 14 14 29 14 14 W(33) W(17) W(17) A|30B|31 C|30 D|15 E|30 F|15G|18H|20 K|35 Watermark Event Time at the operator Event [id|timestamp] Event Time at input streams 33 17 Watermark Generation M|39N|39Q|44 L|22O|23R|37
  • 40. Mixing Event Time Processing Time 40 case class Event(id: String, measure: Double, timestamp: Long) val env = StreamExecutionEnvironment.getExecutionEnvironment env.setStreamTimeCharacteristic(EventTime) val stream: DataStream[Event] = env.addSource(…) val tsStream = stream.assignAscendingTimestamps(_.timestamp) tsStream .keyBy("id") .window(SlidingEventTimeWindows.of(seconds(15), seconds(5)) .trigger(new MyTrigger()) .sum("measure")
  • 41. Window Triggers  React to any combination of • Event Time • Processing Time • Event data  Example of a mixed EventTime / Proc. Time Trigger: • Trigger when event time reaches window end OR • When processing time reaches window end plus 30 secs. 41
  • 42. Trigger example 42 .sum("measure") public class EventTimeTrigger extends Trigger<Object, TimeWindow> { public TriggerResult onElement(Object evt, long time, TimeWindow window, TriggerContext ctx) { ctx.registerEventTimeTimer(window.maxTimestamp()); ctx.registerProcessingTimeTimer(window.maxTimestamp() + 30000); return TriggerResult.CONTINUE; } public TriggerResult onEventTime(long time, TimeWindow w, TriggerContext ctx) { return TriggerResult.FIRE_AND_PURGE; } public TriggerResult onProcessingTime(long time, TimeWindow w, TriggerContext c) { return TriggerResult.FIRE_AND_PURGE; }
  • 43. Trigger example 43 .sum("measure") public class EventTimeTrigger extends Trigger<Object, TimeWindow> { public TriggerResult onElement(Object evt, long time, TimeWindow window, TriggerContext ctx) { ctx.registerEventTimeTimer(window.maxTimestamp()); ctx.registerProcessingTimeTimer(window.maxTimestamp() + 30000); return TriggerResult.CONTINUE; } public TriggerResult onEventTime(long time, TimeWindow w, TriggerContext ctx) { return TriggerResult.FIRE_AND_PURGE; } public TriggerResult onProcessingTime(long time, TimeWindow w, TriggerContext c) { return TriggerResult.FIRE_AND_CONTINUE; }
  • 44. Per Kafka Partition Watermarks 44 Source (1) Source (2) map (1) map (2) window (1) window (2) 33 17 29 29 17 14 14 29 14 14 W(33) W(17) W(17) A|30B|73 C|33 D|18 E|31 F|15G|91H|94 K|77 Watermark Generation L|35N|39 O|97 M|89 I|21Q|23 T|99 S|97
  • 45. Per Kafka Partition Watermarks 45 val env = StreamExecutionEnvironment.getExecutionEnvironment env.setStreamTimeCharacteristic(EventTime) val kafka = new FlinkKafkaConsumer09(topic, schema, props) kafka.assignTimestampsAndWatermarks( new MyTimestampsAndWatermarkGenerator()) val stream: DataStream[Event] = env.addSource(kafka) stream .keyBy("id") .timeWindow(Time.seconds(15), Time.seconds(5)) .sum("measure")
  • 46. Matters of State (Fault Tolerance, Reinstatements, etc) 46
  • 47. Back to the Aggregation Example 47 case class Event(id: String, measure: Double, timestamp: Long) val env = StreamExecutionEnvironment.getExecutionEnvironment val stream: DataStream[Event] = env.addSource( new FlinkKafkaConsumer09(topic, schema, properties)) stream .keyBy("id") .timeWindow(Time.seconds(15), Time.seconds(5)) .sum("measure") Stateful
  • 48. Fault Tolerance  Prevent data loss (reprocess lost in-flight events)  Recover state consistency (exactly-once semantics) • Pending windows & user-defined (key/value) state  Checkpoint based fault tolerance • Periodicaly create checkpoints • Recovery: resume from last completed checkpoint • Async. Barrier Snapshots (ABS) Algorithm  48
  • 49. Checkpoints 49 data stream event newer records older records State of the dataflow at point Y State of the dataflow at point X
  • 50. Checkpoint Barriers  Markers, injected into the streams 50
  • 53. Savepoints  A "Checkpoint" is a globally consistent point-in-time snapshot of the streaming program (point in stream, state)  A "Savepoint" is a user-triggered retained checkpoint  Streaming programs can start from a savepoint 53 Savepoint B Savepoint A
  • 54. (Re)processing data (in batch)  Re-processing data (what-if exploration, to correct bugs, etc.)  Usually by running a batch job with a set of old files  Tools that map files to times 54 2016-3-1 12:00 am 2016-3-1 1:00 am 2016-3-1 2:00 am 2016-3-11 11:00pm 2016-3-12 12:00am 2016-3-12 1:00am… Collection of files, by ingestion time 2016-3-11 10:00pm To the batch processor
  • 55. Unclear Batch Boundaries 55 2016-3-1 12:00 am 2016-3-1 1:00 am 2016-3-1 2:00 am 2016-3-11 11:00pm 2016-3-12 12:00am 2016-3-12 1:00am… 2016-3-11 10:00pm To the batch processor ? ? What about sessions across batches?
  • 56. (Re)processing data (streaming)  Draw savepoints at times that you will want to start new jobs from (daily, hourly, …)  Reprocess by starting a new job from a savepoint • Defines start position in stream (for example Kafka offsets) • Initializes pending state (like partial sessions) 56 Savepoint Run new streaming program from savepoint
  • 57. Continuous Data Sources 57 2016-3-1 12:00 am 2016-3-1 1:00 am 2016-3-1 2:00 am 2016-3-11 11:00pm 2016-3-12 12:00am 2016-3-12 1:00am 2016-3-11 10:00pm … partition partition Savepoint Savepoint Stream of Kafka Partitions Stream view over sequence of files Kafka offsets + Operator state File mod timestamp + File position + Operator state WIP (target: Flink 1.1)
  • 58. Upgrading Programs  A program starting from a savepoint can differ from the program that created the savepoint • Unique operator names match state and operator  Mechanism be used to fix bugs in programs, to evolve programs, parameters, libraries, … 58
  • 59. State Backends  Large state is a collection of key/value pairs  State backend defines what data structure holds the state, plus how it is snapshotted  Most common choices • Main memory – snapshots to master • Main memory – snapshots to dist. filesystem • RocksDB – snapshots to dist. filesystem 59
  • 61. Example: Temperature Monitoring  Receiving temperature an power events from sensors  Looking for temperatures repeatedly exceeding thresholds within a short time period (10 secs) 61
  • 65. An Outlook on Things to Come 65
  • 66. Flink in the wild 66 30 billion events daily 2 billion events in 10 1Gb machines data integration & distribution platform See talks by at
  • 67. Roadmap  Dynamic Scaling, Resource Elasticity  Stream SQL  CEP enhancements  Incremental & asynchronous state snapshotting  Mesos support  More connectors, end-to-end exactly once  API enhancements (e.g., joins, slowly changing inputs)  Security (data encryption, Kerberos with Kafka) 67