Streaming

Streaming Analytics
with Apache Flink 1.0
Stephan Ewen
@stephanewen
Apache Flink Stack
Libraries
DataStream API
Stream Processing
DataSet API
Batch Processing
Runtime
Distributed Streaming Data Flow
Streaming and batch as first class citizens.
2
Today
Libraries
DataStream API
Stream Processing
DataSet API
Batch Processing
Runtime
Distributed Streaming Data Flow
Streaming and batch as first class citizens.
3
Streaming is the next programming paradigm
for data applications, and you need to start
thinking in terms of streams.
4
Streaming technology is enabling the obvious:
continuous processing on data that is
continuously produced
5
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
collect
store
analyze
serve
8
A brief History of Flink
January ‘10
April ‘14
v0.5
Project
Stratosphere
(Flink precursor)
Flink Project
Incubation
December ‘14
v0.6
v0.7
Top Level
Project
March ‘16
v0.8
v0.9
v0.10
Release
1.0
9
A brief History of Flink
The academia gap:
Reading/writing papers,
teaching, worrying about thesis
January ‘10
April ‘14
v0.5
Project
Stratosphere
(Flink precursor)
Flink Project
Incubation
December ‘14
v0.6
v0.7
Top Level
Project
March ‘16
v0.8
v0.9
v0.10
Release
1.0
Realizing this might be
interesting to people
beyond academia
(even more so, actually)
10
Programs and Dataflows
val lines: DataStream[String] = env.addSource(new FlinkKafkaConsumer09(…))
val events: DataStream[Event] = lines.map((line) => parse(line))
Source
Transformation
val stats: DataStream[Statistic] = stream
.keyBy("sensor")
.timeWindow(Time.seconds(5))
.sum(new MyAggregationFunction())
Transformation
stats.addSink(new RollingSink(path))
Sink
Source
[1]
Source
[2]
map()
[1]
map()
[2]
keyBy()/
window()/
apply()
[1]
keyBy()/
window()/
apply()
[2]
Sink
[1]
Streaming
Dataflow
11
What makes Flink flink?
Low latency
Make more sense of data
High Throughput
Well-behaved
flow control
(back pressure)
Windows &
user-defined state
Works on real-time
and historic data
True
Streaming
Event Time
Stateful
Streaming
APIs
Libraries
Exactly-once semantics
for fault tolerance
Globally consistent
savepoints
Complex Event Processing
Flexible windows
(time, count, session, roll-your own)
12
Streaming Analytics by Example
13
Time-Windowed Aggregations
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")
14
Time-Windowed Aggregations
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")
15
Session-Windowed Aggregations
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")
16
Session-Windowed Aggregations
case class Event(sensor: String, measure: Double)
val env = StreamExecutionEnvironment.getExecutionEnvironment
val stream: DataStream[Event] = env.addSource(…)
Flink 1.1 syntax
stream
.keyBy("sensor")
.window(EventTimeSessionWindows.withGap(Time.seconds(60)))
.max("measure")
17
Pattern Detection
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)
}
})
18
Pattern Detection
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
19
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
The biggest change in moving from
batch to streaming is
handling time explicitly
22
Example: Windowing by Time
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")
23
Example: Windowing by Time
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
Different Notions of Time
Event Producer
Message Queue
Flink
Data Source
Flink
Window Operator
partition 1
partition 2
Event
Time
Ingestion
Time
Window
Processing
Time
25
Event Time vs. Processing Time
Event Time
Episode Episode Episode Episode Episode Episode Episode
IV
V
VI
I
II
III
VII
1977
1980
1983
1999
2002
2005
2015
Processing Time
26
IoT / Mobile Applications
Queue / Log
Events occur on devices
Events stored in a log
Stream Analysis
Events analyzed in
a
data streaming
system
27
Out of order Streams
28
Out of order Streams
29
Out of order Streams
30
Out of order Streams
Out of order !!!
First burst of events
Second burst of events
31
Out of order Streams
Instant event-at-a-time
Arrival time windows
First burst of events
Second burst of events
Event time windows
32
Processing Time
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
33
Ingestion Time
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")
34
Event Time
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")
35
Event Time
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")
36
Event Time
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")
37
Watermarks
Stream (in order)
23
21
20
19
18 17 15
14
W(20)
11
10
9
9
7
W(11)
Event
Watermark
Event timestamp
Stream (out of order)
21
19
20
17
22 12
W(17)
17
14
12
9
15
11
7
W(11)
Watermark
Event
Event timestamp
38
Watermarks in Parallel
Event
Watermark
[id|timestamp]
33
Q|44 N|39 M|39
Source
(1)
29
map
(1)
K|35
B|31
A|30
W(33)
D|15
Event Time
at input streams
W(17)
E|30
R|37 O|23 L|22
17
Source
(2)
window
(1)
14
C|30
Watermark
Generation
14
29
29
17
H|20
map
(2)
G|18
F|15
W(17)
14
14
window
(2)
Event Time
at the operator
39
Mixing Event Time Processing Time
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")
40
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
.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;
}
42
Trigger example
.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;
}
43
Matters of State
(Fault Tolerance, Reinstatements, etc)
46
Back to the Aggregation Example
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
47
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
data stream
newer records
older records
event
State of the dataflow
at point Y
State of the dataflow
at point X
49
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
Savepoint B
Savepoint A
53
(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
2016-3-1
12:00 am
2016-3-1
1:00 am
2016-3-1
2:00 am
Collection of files, by ingestion time
…
2016-3-11
10:00pm
2016-3-11
11:00pm
2016-3-12
12:00am
2016-3-12
1:00am
To the batch
processor
54
Unclear Batch Boundaries
2016-3-1
12:00 am
2016-3-1
1:00 am
2016-3-1
2:00 am
…
2016-3-11
10:00pm
2016-3-11
11:00pm
2016-3-12
12:00am
2016-3-12
1:00am
To the batch
processor
?
?
What about sessions across batches?
55
(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)
Run new streaming
program from savepoint
Savepoint
56
Continuous Data Sources
Stream of Kafka Partitions
partition
partition
Savepoint
WIP (target: Flink 1.1)
2016-3-12
1:00am
Savepoint
2016-3-12
12:00am
2016-3-11
11:00pm
2016-3-11
10:00pm
Kafka offsets +
Operator state
File mod timestamp +
File position +
Operator state
…
2016-3-1
2:00 am
2016-3-1
1:00 am
2016-3-1
12:00 am
Stream view over sequence of files
57
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
30 billion events daily
See talks by
2 billion events in
10 1Gb machines
data integration & distribution
platform
at
66
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
I stream, do you?
68
Why does Flink stream flink?
Low latency
Make more sense of data
High Throughput
Well-behaved
flow control
(back pressure)
Windows &
user-defined state
Works on real-time
and historic data
True
Streaming
Event Time
Stateful
Streaming
APIs
Libraries
Exactly-once semantics
for fault tolerance
Globally consistent
savepoints
Complex Event Processing
Flexible windows
(time, count, session, roll-your own)
69
Addendum
70
Latency and Throughput
71
Low Latency and High Throughput
 Frequently though to be mutually exclusive
• Event-at-a-time  low latency, low throughput
• Mini batch  high latency, high throughput
 The above is not true!
 Very little latency has to be sacrificed for very high
throughput
72
Latency and Throughput
73
Latency and Throughput
74
The Effect of Buffering
 Network stack does not always
operate in event-at-a-time
mode
 Optional buffering adds
some milliseconds latency
but increases throughput
 No effect on application logic
75
On a technical level
Decouple all things
 Clocks
• Wall clock time (processing time)
• Event time (watermarks & punctuations)
• Consistency clock (logical checkpoint timestamps)
 Buffering
• Windows (application logic)
• Network (throughput tuning)
 …
76
Decoupling clocks
77
Stream Alignment
78
On exactly-once guarantees
 Giant topic of confusion
exactly what?
79
High Availability
80
High Availability Checkpoints
JobManager
Client
Apache Zookeeper™
1. Take snapshots
TaskManagers
81
High Availability Checkpoints
JobManager
Client
Apache Zookeeper™
1. Take snapshots
2. Persist snapshots
3. Send handles to JM
TaskManagers
82
High Availability Checkpoints
JobManager
Client
Apache Zookeeper™
1.
2.
3.
4.
Take snapshots
Persist snapshots
Send handles to JM
Create global checkpoint
TaskManagers
83
High Availability Checkpoints
JobManager
Client
Apache Zookeeper™
1.
2.
3.
4.
5.
Take snapshots
Persist snapshots
Send handles to JM
Create global checkpoint
Persist global checkpoint
TaskManagers
84
High Availability Checkpoints
JobManager
Client
Apache Zookeeper™
1.
2.
3.
4.
5.
6.
Take snapshots
Persist snapshots
Send handles to JM
Create global checkpoint
Persist global checkpoint
Write handle to ZooKeeper
TaskManagers
85
The Counting Pyramid of Needs
86