SlideShare uma empresa Scribd logo
1 de 46
Stephan Ewen, Kostas Tzoumas
Flink committers
co-founders @ data Artisans
@StephanEwen, @kostas_tzoumas
Flink-0.10
What is Flink
2
Gelly
Table
ML
SAMOA
DataSet (Java/Scala) DataStream (Java/Scala)
HadoopM/R
Local Remote Yarn Tez Embedded
Dataflow
Dataflow(WiP)
MRQL
Table
Cascading(WiP)
Streaming dataflow runtime
Flink 0.10 Summary
 Focus on operational readiness
• high availability
• monitoring
• integration with other systems
 First-class support for event time
 Refined DataStream API: easy and powerful
3
Improved DataStream API
 Stream data analysis differs from batch data
analysis by introducing time
 Streams are unbounded and produce data over
time
 Simple as batch API if handling time in a simple
way
 Powerful if you want to handle time in an
advanced way (out-of-order records,
preliminary results, etc)
4
Improved DataStream API
5
case class Event(location: Location, numVehicles: Long)
val stream: DataStream[Event] = …;
stream
.filter { evt => isIntersection(evt.location) }
Improved DataStream API
6
case class Event(location: Location, numVehicles: Long)
val stream: DataStream[Event] = …;
stream
.filter { evt => isIntersection(evt.location) }
.keyBy("location")
.timeWindow(Time.of(15, MINUTES), Time.of(5, MINUTES))
.sum("numVehicles")
Improved DataStream API
7
case class Event(location: Location, numVehicles: Long)
val stream: DataStream[Event] = …;
stream
.filter { evt => isIntersection(evt.location) }
.keyBy("location")
.timeWindow(Time.of(15, MINUTES), Time.of(5, MINUTES))
.trigger(new Threshold(200))
.sum("numVehicles")
Improved DataStream API
8
case class Event(location: Location, numVehicles: Long)
val stream: DataStream[Event] = …;
stream
.filter { evt => isIntersection(evt.location) }
.keyBy("location")
.timeWindow(Time.of(15, MINUTES), Time.of(5, MINUTES))
.trigger(new Threshold(200))
.sum("numVehicles")
.keyBy( evt => evt.location.grid )
.mapWithState { (evt, state: Option[Model]) => {
val model = state.orElse(new Model())
(model.classify(evt), Some(model.update(evt)))
}}
IoT / Mobile Applications
9
Events occur on devices
Queue / Log
Events analyzed in a
data streaming
system
Stream Analysis
Events stored in a log
IoT / Mobile Applications
10
IoT / Mobile Applications
11
IoT / Mobile Applications
12
IoT / Mobile Applications
13
Out of order !!!
First burst of events
Second burst of events
IoT / Mobile Applications
14
Event time windows
Arrival time windows
Instant event-at-a-time
Flink supports out of order time (event time) windows,
arrival time windows (and mixtures) plus low latency processing.
First burst of events
Second burst of events
15
We need a
Global Clock
that runs on
event time
instead of
processing time.
16
This is a source
This is our window operator
1
0
0
0 0
1
2
1
2
1
1
This is the current event-time time
2
2
2
2
2
This is a watermark.
17
StreamExecutionEnvironment env =
StreamExecutionEnvironment.getExecutionEnvironment();
env.setStreamTimeCharacteristic(ProcessingTime);
DataStream<Tweet> text = env.addSource(new TwitterSrc());
DataStream<Tuple2<String, Integer>> counts = text
.flatMap(new ExtractHashtags())
.keyBy(“name”)
.timeWindow(Time.of(5, MINUTES)
.apply(new HashtagCounter());
Processing Time
18
Event Time
StreamExecutionEnvironment env =
StreamExecutionEnvironment.getExecutionEnvironment();
env.setStreamTimeCharacteristic(EventTime);
DataStream<Tweet> text = env.addSource(new TwitterSrc());
text = text.assignTimestamps(new MyTimestampExtractor());
DataStream<Tuple2<String, Integer>> counts = text
.flatMap(new ExtractHashtags())
.keyBy(“name”)
.timeWindow(Time.of(5, MINUTES)
.apply(new HashtagCounter());
32-35
24-27
20-23
8-110-3
4-7
19
Tumbling Windows of 4 Seconds
123412
4
59
9 0
20
20
22212326323321
26
353642
20
High Availability and Consistency
21
No Single-Point-Of-Failure
any more
Exactly-once processing semantics
across pipeline
Checkpoints/Fault Tolerance is decoupled from windows
 Allows for highly flexible window implementations
ZooKeeper
ensemble
Multiple
Masters
failover
Operator State
 Stateless operators
 System state
 User defined state
22
ds.filter(_ != 0)
ds.keyBy("id").timeWindow(Time.of(5, SECONDS)).reduce(…)
public class CounterSum implements RichReduceFunction<Long> {
private OperatorState<Long> counter;
@Override public Long reduce(Long v1, Long v2) throws Exception {
counter.update(counter.value() + 1);
return v1 + v2;
}
@Override public void open(Configuration config) {
counter = getRuntimeContext().getOperatorState(“counter”, 0L, false);
}
}
Checkpoints
 Consistent snapshots of distributed data
stream and operator state
23
Barriers
 Markers for checkpoints
 Injected in the data flow
24
25
 Alignment for multi-input operators
High Availability and Consistency
26
No Single-Point-Of-Failure
any more
Exactly-once processing semantics
across pipeline
Checkpoints/Fault Tolerance is decoupled from windows
 Allows for highly flexible window implementations
ZooKeeper
ensemble
Multiple
Masters
failover
Without high availability
27
JobManager
TaskManager
With high availability
28
JobManager
TaskManager
Stand-by
JobManager
Apache Zookeeper™
KEEP GOING
Persisting jobs
29
JobManager
Client
TaskManagers
Apache Zookeeper™
Job
1. Submit job
Persisting jobs
30
JobManager
Client
TaskManagers
Apache Zookeeper™
1. Submit job
2. Persist execution graph
Persisting jobs
31
JobManager
Client
TaskManagers
Apache Zookeeper™
1. Submit job
2. Persist execution graph
3. Write handle to ZooKeeper
Persisting jobs
32
JobManager
Client
TaskManagers
Apache Zookeeper™
1. Submit job
2. Persist execution graph
3. Write handle to ZooKeeper
4. Deploy tasks
Handling checkpoints
33
JobManager
Client
TaskManagers
Apache Zookeeper™
1. Take snapshots
Handling checkpoints
34
JobManager
Client
TaskManagers
Apache Zookeeper™
1. Take snapshots
2. Persist snapshots
3. Send handles to JM
Handling checkpoints
35
JobManager
Client
TaskManagers
Apache Zookeeper™
1. Take snapshots
2. Persist snapshots
3. Send handles to JM
4. Create global checkpoint
Handling checkpoints
36
JobManager
Client
TaskManagers
Apache Zookeeper™
1. Take snapshots
2. Persist snapshots
3. Send handles to JM
4. Create global checkpoint
5. Persist global checkpoint
Handling checkpoints
37
JobManager
Client
TaskManagers
Apache Zookeeper™
1. Take snapshots
2. Persist snapshots
3. Send handles to JM
4. Create global checkpoint
5. Persist global checkpoint
6. Write handle to ZooKeeper
Performance
38
Continuous
streaming
Latency-bound
buffering
Distributed
Snapshots
High Throughput &
Low Latency
With configurable throughput/latency tradeoff
Batch and Streaming
39
case class WordCount(word: String, count: Int)
val text: DataStream[String] = …;
text
.flatMap { line => line.split(" ") }
.map { word => new WordCount(word, 1) }
.keyBy("word")
.window(GlobalWindows.create())
.trigger(new EOFTrigger())
.sum("count")
Batch Word Count in the DataStream API
Batch and Streaming
40
Batch Word Count in the DataSet API
case class WordCount(word: String, count: Int)
val text: DataStream[String] = …;
text
.flatMap { line => line.split(" ") }
.map { word => new WordCount(word, 1) }
.keyBy("word")
.window(GlobalWindows.create())
.trigger(new EOFTrigger())
.sum("count")
val text: DataSet[String] = …;
text
.flatMap { line => line.split(" ") }
.map { word => new WordCount(word, 1) }
.groupBy("word")
.sum("count")
Batch and Streaming
41
Pipelined and
blocking operators Streaming Dataflow Runtime
Batch Parameters
DataSet DataStream
Relational
Optimizer
Window
Optimization
Pipelined and
windowed operators
Schedule lazily
Schedule eagerly
Recompute whole
operators Periodic checkpoints
Streaming data movement
Stateful operations
DAG recovery
Fully buffered streams DAG resource management
Streaming
Parameters
Batch and Streaming
42
A full-fledged batch processor as well
Batch and Streaming
43
A full-fledged batch processor as well
More details at Dongwon Kim's Talk
"A comparative performance evaluation of Flink"
Integration (picture not complete)
44
POSIX Java/Scala
Collections
POSIX
Monitoring
45
Life system metrics and
user-defined accumulators/statistics
Get http://flink-m:8081/jobs/7684be6004e4e955c2a558a9bc463f65/accumulators
Monitoring REST API for
custom monitoring tools
{ "id": "dceafe2df1f57a1206fcb907cb38ad97", "user-accumulators": [
{ "name":"avglen", "type":"DoubleCounter", "value":"123.03259440000001" },
{ "name":"genwords", "type":"LongCounter", "value":"75000000" } ] }
Flink 0.10 Summary
 Focus on operational readiness
• high availability
• monitoring
• integration with other systems
 First-class support for event time
 Refined DataStream API: easy and powerful
46

Mais conteúdo relacionado

Mais procurados

Flink Gelly - Karlsruhe - June 2015
Flink Gelly - Karlsruhe - June 2015Flink Gelly - Karlsruhe - June 2015
Flink Gelly - Karlsruhe - June 2015
Andra Lungu
 

Mais procurados (20)

Arbitrary Stateful Aggregations using Structured Streaming in Apache Spark
Arbitrary Stateful Aggregations using Structured Streaming in Apache SparkArbitrary Stateful Aggregations using Structured Streaming in Apache Spark
Arbitrary Stateful Aggregations using Structured Streaming in Apache Spark
 
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)
 
Apache Flink Deep Dive
Apache Flink Deep DiveApache Flink Deep Dive
Apache Flink Deep Dive
 
First Flink Bay Area meetup
First Flink Bay Area meetupFirst Flink Bay Area meetup
First Flink Bay Area meetup
 
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
 
Batch and Stream Graph Processing with Apache Flink
Batch and Stream Graph Processing with Apache FlinkBatch and Stream Graph Processing with Apache Flink
Batch and Stream Graph Processing with Apache Flink
 
Flink Gelly - Karlsruhe - June 2015
Flink Gelly - Karlsruhe - June 2015Flink Gelly - Karlsruhe - June 2015
Flink Gelly - Karlsruhe - June 2015
 
Apache Flink Training: System Overview
Apache Flink Training: System OverviewApache Flink Training: System Overview
Apache Flink Training: System Overview
 
Machine Learning with Apache Flink at Stockholm Machine Learning Group
Machine Learning with Apache Flink at Stockholm Machine Learning GroupMachine Learning with Apache Flink at Stockholm Machine Learning Group
Machine Learning with Apache Flink at Stockholm Machine Learning Group
 
Flink internals web
Flink internals web Flink internals web
Flink internals web
 
Apache Flink Training: DataSet API Basics
Apache Flink Training: DataSet API BasicsApache Flink Training: DataSet API Basics
Apache Flink Training: DataSet API Basics
 
Don't Cross The Streams - Data Streaming And Apache Flink
Don't Cross The Streams  - Data Streaming And Apache FlinkDon't Cross The Streams  - Data Streaming And Apache Flink
Don't Cross The Streams - Data Streaming And Apache Flink
 
Deep dive into stateful stream processing in structured streaming by Tathaga...
Deep dive into stateful stream processing in structured streaming  by Tathaga...Deep dive into stateful stream processing in structured streaming  by Tathaga...
Deep dive into stateful stream processing in structured streaming by Tathaga...
 
What's new with Apache Spark's Structured Streaming?
What's new with Apache Spark's Structured Streaming?What's new with Apache Spark's Structured Streaming?
What's new with Apache Spark's Structured Streaming?
 
Predictive Datacenter Analytics with Strymon
Predictive Datacenter Analytics with StrymonPredictive Datacenter Analytics with Strymon
Predictive Datacenter Analytics with Strymon
 
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
 
Flink Forward SF 2017: Timo Walther - Table & SQL API – unified APIs for bat...
Flink Forward SF 2017: Timo Walther -  Table & SQL API – unified APIs for bat...Flink Forward SF 2017: Timo Walther -  Table & SQL API – unified APIs for bat...
Flink Forward SF 2017: Timo Walther - Table & SQL API – unified APIs for bat...
 
Gelly-Stream: Single-Pass Graph Streaming Analytics with Apache Flink
Gelly-Stream: Single-Pass Graph Streaming Analytics with Apache FlinkGelly-Stream: Single-Pass Graph Streaming Analytics with Apache Flink
Gelly-Stream: Single-Pass Graph Streaming Analytics with Apache Flink
 
Vasia Kalavri – Training: Gelly School
Vasia Kalavri – Training: Gelly School Vasia Kalavri – Training: Gelly School
Vasia Kalavri – Training: Gelly School
 
FastR+Apache Flink
FastR+Apache FlinkFastR+Apache Flink
FastR+Apache Flink
 

Destaque

Mohamed Amine Abdessemed – Real-time Data Integration with Apache Flink & Kafka
Mohamed Amine Abdessemed – Real-time Data Integration with Apache Flink & KafkaMohamed Amine Abdessemed – Real-time Data Integration with Apache Flink & Kafka
Mohamed Amine Abdessemed – Real-time Data Integration with Apache Flink & Kafka
Flink Forward
 
William Vambenepe – Google Cloud Dataflow and Flink , Stream Processing by De...
William Vambenepe – Google Cloud Dataflow and Flink , Stream Processing by De...William Vambenepe – Google Cloud Dataflow and Flink , Stream Processing by De...
William Vambenepe – Google Cloud Dataflow and Flink , Stream Processing by De...
Flink Forward
 
Martin Junghans – Gradoop: Scalable Graph Analytics with Apache Flink
Martin Junghans – Gradoop: Scalable Graph Analytics with Apache FlinkMartin Junghans – Gradoop: Scalable Graph Analytics with Apache Flink
Martin Junghans – Gradoop: Scalable Graph Analytics with Apache Flink
Flink 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 Processing
Flink Forward
 
Albert Bifet – Apache Samoa: Mining Big Data Streams with Apache Flink
Albert Bifet – Apache Samoa: Mining Big Data Streams with Apache FlinkAlbert Bifet – Apache Samoa: Mining Big Data Streams with Apache Flink
Albert Bifet – Apache Samoa: Mining Big Data Streams with Apache Flink
Flink Forward
 
Suneel Marthi – BigPetStore Flink: A Comprehensive Blueprint for Apache Flink
Suneel Marthi – BigPetStore Flink: A Comprehensive Blueprint for Apache FlinkSuneel Marthi – BigPetStore Flink: A Comprehensive Blueprint for Apache Flink
Suneel Marthi – BigPetStore Flink: A Comprehensive Blueprint for Apache Flink
Flink Forward
 
Chris Hillman – Beyond Mapreduce Scientific Data Processing in Real-time
Chris Hillman – Beyond Mapreduce Scientific Data Processing in Real-timeChris Hillman – Beyond Mapreduce Scientific Data Processing in Real-time
Chris Hillman – Beyond Mapreduce Scientific Data Processing in Real-time
Flink Forward
 
Sebastian Schelter – Distributed Machine Learing with the Samsara DSL
Sebastian Schelter – Distributed Machine Learing with the Samsara DSLSebastian Schelter – Distributed Machine Learing with the Samsara DSL
Sebastian Schelter – Distributed Machine Learing with the Samsara DSL
Flink Forward
 
Apache Flink Training: DataStream API Part 1 Basic
 Apache Flink Training: DataStream API Part 1 Basic Apache Flink Training: DataStream API Part 1 Basic
Apache Flink Training: DataStream API Part 1 Basic
Flink Forward
 
Moon soo Lee – Data Science Lifecycle with Apache Flink and Apache Zeppelin
Moon soo Lee – Data Science Lifecycle with Apache Flink and Apache ZeppelinMoon soo Lee – Data Science Lifecycle with Apache Flink and Apache Zeppelin
Moon soo Lee – Data Science Lifecycle with Apache Flink and Apache Zeppelin
Flink Forward
 
Tran Nam-Luc – Stale Synchronous Parallel Iterations on Flink
Tran Nam-Luc – Stale Synchronous Parallel Iterations on FlinkTran Nam-Luc – Stale Synchronous Parallel Iterations on Flink
Tran Nam-Luc – Stale Synchronous Parallel Iterations on Flink
Flink Forward
 
Alexander Kolb – Flink. Yet another Streaming Framework?
Alexander Kolb – Flink. Yet another Streaming Framework?Alexander Kolb – Flink. Yet another Streaming Framework?
Alexander Kolb – Flink. Yet another Streaming Framework?
Flink Forward
 

Destaque (20)

Mohamed Amine Abdessemed – Real-time Data Integration with Apache Flink & Kafka
Mohamed Amine Abdessemed – Real-time Data Integration with Apache Flink & KafkaMohamed Amine Abdessemed – Real-time Data Integration with Apache Flink & Kafka
Mohamed Amine Abdessemed – Real-time Data Integration with Apache Flink & Kafka
 
Fabian Hueske – Cascading on Flink
Fabian Hueske – Cascading on FlinkFabian Hueske – Cascading on Flink
Fabian Hueske – Cascading on Flink
 
William Vambenepe – Google Cloud Dataflow and Flink , Stream Processing by De...
William Vambenepe – Google Cloud Dataflow and Flink , Stream Processing by De...William Vambenepe – Google Cloud Dataflow and Flink , Stream Processing by De...
William Vambenepe – Google Cloud Dataflow and Flink , Stream Processing by De...
 
Martin Junghans – Gradoop: Scalable Graph Analytics with Apache Flink
Martin Junghans – Gradoop: Scalable Graph Analytics with Apache FlinkMartin Junghans – Gradoop: Scalable Graph Analytics with Apache Flink
Martin Junghans – Gradoop: Scalable Graph Analytics with Apache Flink
 
Aljoscha Krettek – Notions of Time
Aljoscha Krettek – Notions of TimeAljoscha Krettek – Notions of Time
Aljoscha Krettek – Notions of Time
 
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
 
Flink Apachecon Presentation
Flink Apachecon PresentationFlink Apachecon Presentation
Flink Apachecon Presentation
 
Apache Flink internals
Apache Flink internalsApache Flink internals
Apache Flink internals
 
Ufuc Celebi – Stream & Batch Processing in one System
Ufuc Celebi – Stream & Batch Processing in one SystemUfuc Celebi – Stream & Batch Processing in one System
Ufuc Celebi – Stream & Batch Processing in one System
 
Albert Bifet – Apache Samoa: Mining Big Data Streams with Apache Flink
Albert Bifet – Apache Samoa: Mining Big Data Streams with Apache FlinkAlbert Bifet – Apache Samoa: Mining Big Data Streams with Apache Flink
Albert Bifet – Apache Samoa: Mining Big Data Streams with Apache Flink
 
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
 
Suneel Marthi – BigPetStore Flink: A Comprehensive Blueprint for Apache Flink
Suneel Marthi – BigPetStore Flink: A Comprehensive Blueprint for Apache FlinkSuneel Marthi – BigPetStore Flink: A Comprehensive Blueprint for Apache Flink
Suneel Marthi – BigPetStore Flink: A Comprehensive Blueprint for Apache Flink
 
Chris Hillman – Beyond Mapreduce Scientific Data Processing in Real-time
Chris Hillman – Beyond Mapreduce Scientific Data Processing in Real-timeChris Hillman – Beyond Mapreduce Scientific Data Processing in Real-time
Chris Hillman – Beyond Mapreduce Scientific Data Processing in Real-time
 
Sebastian Schelter – Distributed Machine Learing with the Samsara DSL
Sebastian Schelter – Distributed Machine Learing with the Samsara DSLSebastian Schelter – Distributed Machine Learing with the Samsara DSL
Sebastian Schelter – Distributed Machine Learing with the Samsara DSL
 
Apache Flink Training: DataStream API Part 1 Basic
 Apache Flink Training: DataStream API Part 1 Basic Apache Flink Training: DataStream API Part 1 Basic
Apache Flink Training: DataStream API Part 1 Basic
 
Moon soo Lee – Data Science Lifecycle with Apache Flink and Apache Zeppelin
Moon soo Lee – Data Science Lifecycle with Apache Flink and Apache ZeppelinMoon soo Lee – Data Science Lifecycle with Apache Flink and Apache Zeppelin
Moon soo Lee – Data Science Lifecycle with Apache Flink and Apache Zeppelin
 
Tran Nam-Luc – Stale Synchronous Parallel Iterations on Flink
Tran Nam-Luc – Stale Synchronous Parallel Iterations on FlinkTran Nam-Luc – Stale Synchronous Parallel Iterations on Flink
Tran Nam-Luc – Stale Synchronous Parallel Iterations on Flink
 
Alexander Kolb – Flink. Yet another Streaming Framework?
Alexander Kolb – Flink. Yet another Streaming Framework?Alexander Kolb – Flink. Yet another Streaming Framework?
Alexander Kolb – Flink. Yet another Streaming Framework?
 
Marton Balassi – Stateful Stream Processing
Marton Balassi – Stateful Stream ProcessingMarton Balassi – Stateful Stream Processing
Marton Balassi – Stateful Stream Processing
 
Flink Case Study: Bouygues Telecom
Flink Case Study: Bouygues TelecomFlink Case Study: Bouygues Telecom
Flink Case Study: Bouygues Telecom
 

Semelhante a Flink 0.10 @ Bay Area Meetup (October 2015)

Flink Streaming Hadoop Summit San Jose
Flink Streaming Hadoop Summit San JoseFlink Streaming Hadoop Summit San Jose
Flink Streaming Hadoop Summit San Jose
Kostas Tzoumas
 
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
confluent
 
Developing streaming applications with apache apex (strata + hadoop world)
Developing streaming applications with apache apex (strata + hadoop world)Developing streaming applications with apache apex (strata + hadoop world)
Developing streaming applications with apache apex (strata + hadoop world)
Apache Apex
 

Semelhante a Flink 0.10 @ Bay Area Meetup (October 2015) (20)

Apache Flink Stream Processing
Apache Flink Stream ProcessingApache Flink Stream Processing
Apache Flink Stream Processing
 
Flink Streaming Hadoop Summit San Jose
Flink Streaming Hadoop Summit San JoseFlink Streaming Hadoop Summit San Jose
Flink Streaming Hadoop Summit San Jose
 
So you think you can stream.pptx
So you think you can stream.pptxSo you think you can stream.pptx
So you think you can stream.pptx
 
Dataservices: Processing (Big) Data the Microservice Way
Dataservices: Processing (Big) Data the Microservice WayDataservices: Processing (Big) Data the Microservice Way
Dataservices: Processing (Big) Data the Microservice Way
 
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 @ Tel Aviv / Herzliya Meetup
Apache Flink @ Tel Aviv / Herzliya MeetupApache Flink @ Tel Aviv / Herzliya Meetup
Apache Flink @ Tel Aviv / Herzliya Meetup
 
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
 
strata_spark_streaming.ppt
strata_spark_streaming.pptstrata_spark_streaming.ppt
strata_spark_streaming.ppt
 
Unifying Stream, SWL and CEP for Declarative Stream Processing with Apache Flink
Unifying Stream, SWL and CEP for Declarative Stream Processing with Apache FlinkUnifying Stream, SWL and CEP for Declarative Stream Processing with Apache Flink
Unifying Stream, SWL and CEP for Declarative Stream Processing with Apache Flink
 
Taking Spark Streaming to the Next Level with Datasets and DataFrames
Taking Spark Streaming to the Next Level with Datasets and DataFramesTaking Spark Streaming to the Next Level with Datasets and DataFrames
Taking Spark Streaming to the Next Level with Datasets and DataFrames
 
Spark streaming
Spark streamingSpark streaming
Spark streaming
 
Distributed Real-Time Stream Processing: Why and How 2.0
Distributed Real-Time Stream Processing:  Why and How 2.0Distributed Real-Time Stream Processing:  Why and How 2.0
Distributed Real-Time Stream Processing: Why and How 2.0
 
About time
About timeAbout time
About time
 
Strata Singapore: Gearpump Real time DAG-Processing with Akka at Scale
Strata Singapore: GearpumpReal time DAG-Processing with Akka at ScaleStrata Singapore: GearpumpReal time DAG-Processing with Akka at Scale
Strata Singapore: Gearpump Real time DAG-Processing with Akka at Scale
 
Continuous Application with Structured Streaming 2.0
Continuous Application with Structured Streaming 2.0Continuous Application with Structured Streaming 2.0
Continuous Application with Structured Streaming 2.0
 
Distributed Real-Time Stream Processing: Why and How: Spark Summit East talk ...
Distributed Real-Time Stream Processing: Why and How: Spark Summit East talk ...Distributed Real-Time Stream Processing: Why and How: Spark Summit East talk ...
Distributed Real-Time Stream Processing: Why and How: Spark Summit East talk ...
 
Distributed Stream Processing - Spark Summit East 2017
Distributed Stream Processing - Spark Summit East 2017Distributed Stream Processing - Spark Summit East 2017
Distributed Stream Processing - Spark Summit East 2017
 
Spark streaming: Best Practices
Spark streaming: Best PracticesSpark streaming: Best Practices
Spark streaming: Best Practices
 
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
 
Developing streaming applications with apache apex (strata + hadoop world)
Developing streaming applications with apache apex (strata + hadoop world)Developing streaming applications with apache apex (strata + hadoop world)
Developing streaming applications with apache apex (strata + hadoop world)
 

Último

%+27788225528 love spells in Atlanta Psychic Readings, Attraction spells,Brin...
%+27788225528 love spells in Atlanta Psychic Readings, Attraction spells,Brin...%+27788225528 love spells in Atlanta Psychic Readings, Attraction spells,Brin...
%+27788225528 love spells in Atlanta Psychic Readings, Attraction spells,Brin...
masabamasaba
 
%+27788225528 love spells in new york Psychic Readings, Attraction spells,Bri...
%+27788225528 love spells in new york Psychic Readings, Attraction spells,Bri...%+27788225528 love spells in new york Psychic Readings, Attraction spells,Bri...
%+27788225528 love spells in new york Psychic Readings, Attraction spells,Bri...
masabamasaba
 
%+27788225528 love spells in Knoxville Psychic Readings, Attraction spells,Br...
%+27788225528 love spells in Knoxville Psychic Readings, Attraction spells,Br...%+27788225528 love spells in Knoxville Psychic Readings, Attraction spells,Br...
%+27788225528 love spells in Knoxville Psychic Readings, Attraction spells,Br...
masabamasaba
 

Último (20)

%+27788225528 love spells in Atlanta Psychic Readings, Attraction spells,Brin...
%+27788225528 love spells in Atlanta Psychic Readings, Attraction spells,Brin...%+27788225528 love spells in Atlanta Psychic Readings, Attraction spells,Brin...
%+27788225528 love spells in Atlanta Psychic Readings, Attraction spells,Brin...
 
Define the academic and professional writing..pdf
Define the academic and professional writing..pdfDefine the academic and professional writing..pdf
Define the academic and professional writing..pdf
 
%in kempton park+277-882-255-28 abortion pills for sale in kempton park
%in kempton park+277-882-255-28 abortion pills for sale in kempton park %in kempton park+277-882-255-28 abortion pills for sale in kempton park
%in kempton park+277-882-255-28 abortion pills for sale in kempton park
 
tonesoftg
tonesoftgtonesoftg
tonesoftg
 
%+27788225528 love spells in new york Psychic Readings, Attraction spells,Bri...
%+27788225528 love spells in new york Psychic Readings, Attraction spells,Bri...%+27788225528 love spells in new york Psychic Readings, Attraction spells,Bri...
%+27788225528 love spells in new york Psychic Readings, Attraction spells,Bri...
 
%in Harare+277-882-255-28 abortion pills for sale in Harare
%in Harare+277-882-255-28 abortion pills for sale in Harare%in Harare+277-882-255-28 abortion pills for sale in Harare
%in Harare+277-882-255-28 abortion pills for sale in Harare
 
Introducing Microsoft’s new Enterprise Work Management (EWM) Solution
Introducing Microsoft’s new Enterprise Work Management (EWM) SolutionIntroducing Microsoft’s new Enterprise Work Management (EWM) Solution
Introducing Microsoft’s new Enterprise Work Management (EWM) Solution
 
WSO2Con2024 - From Code To Cloud: Fast Track Your Cloud Native Journey with C...
WSO2Con2024 - From Code To Cloud: Fast Track Your Cloud Native Journey with C...WSO2Con2024 - From Code To Cloud: Fast Track Your Cloud Native Journey with C...
WSO2Con2024 - From Code To Cloud: Fast Track Your Cloud Native Journey with C...
 
%in Hazyview+277-882-255-28 abortion pills for sale in Hazyview
%in Hazyview+277-882-255-28 abortion pills for sale in Hazyview%in Hazyview+277-882-255-28 abortion pills for sale in Hazyview
%in Hazyview+277-882-255-28 abortion pills for sale in Hazyview
 
call girls in Vaishali (Ghaziabad) 🔝 >༒8448380779 🔝 genuine Escort Service 🔝✔️✔️
call girls in Vaishali (Ghaziabad) 🔝 >༒8448380779 🔝 genuine Escort Service 🔝✔️✔️call girls in Vaishali (Ghaziabad) 🔝 >༒8448380779 🔝 genuine Escort Service 🔝✔️✔️
call girls in Vaishali (Ghaziabad) 🔝 >༒8448380779 🔝 genuine Escort Service 🔝✔️✔️
 
WSO2Con2024 - Enabling Transactional System's Exponential Growth With Simplicity
WSO2Con2024 - Enabling Transactional System's Exponential Growth With SimplicityWSO2Con2024 - Enabling Transactional System's Exponential Growth With Simplicity
WSO2Con2024 - Enabling Transactional System's Exponential Growth With Simplicity
 
%+27788225528 love spells in Knoxville Psychic Readings, Attraction spells,Br...
%+27788225528 love spells in Knoxville Psychic Readings, Attraction spells,Br...%+27788225528 love spells in Knoxville Psychic Readings, Attraction spells,Br...
%+27788225528 love spells in Knoxville Psychic Readings, Attraction spells,Br...
 
OpenChain - The Ramifications of ISO/IEC 5230 and ISO/IEC 18974 for Legal Pro...
OpenChain - The Ramifications of ISO/IEC 5230 and ISO/IEC 18974 for Legal Pro...OpenChain - The Ramifications of ISO/IEC 5230 and ISO/IEC 18974 for Legal Pro...
OpenChain - The Ramifications of ISO/IEC 5230 and ISO/IEC 18974 for Legal Pro...
 
WSO2Con2024 - WSO2's IAM Vision: Identity-Led Digital Transformation
WSO2Con2024 - WSO2's IAM Vision: Identity-Led Digital TransformationWSO2Con2024 - WSO2's IAM Vision: Identity-Led Digital Transformation
WSO2Con2024 - WSO2's IAM Vision: Identity-Led Digital Transformation
 
AI & Machine Learning Presentation Template
AI & Machine Learning Presentation TemplateAI & Machine Learning Presentation Template
AI & Machine Learning Presentation Template
 
W01_panagenda_Navigating-the-Future-with-The-Hitchhikers-Guide-to-Notes-and-D...
W01_panagenda_Navigating-the-Future-with-The-Hitchhikers-Guide-to-Notes-and-D...W01_panagenda_Navigating-the-Future-with-The-Hitchhikers-Guide-to-Notes-and-D...
W01_panagenda_Navigating-the-Future-with-The-Hitchhikers-Guide-to-Notes-and-D...
 
MarTech Trend 2024 Book : Marketing Technology Trends (2024 Edition) How Data...
MarTech Trend 2024 Book : Marketing Technology Trends (2024 Edition) How Data...MarTech Trend 2024 Book : Marketing Technology Trends (2024 Edition) How Data...
MarTech Trend 2024 Book : Marketing Technology Trends (2024 Edition) How Data...
 
Software Quality Assurance Interview Questions
Software Quality Assurance Interview QuestionsSoftware Quality Assurance Interview Questions
Software Quality Assurance Interview Questions
 
Microsoft AI Transformation Partner Playbook.pdf
Microsoft AI Transformation Partner Playbook.pdfMicrosoft AI Transformation Partner Playbook.pdf
Microsoft AI Transformation Partner Playbook.pdf
 
WSO2CON 2024 - Cloud Native Middleware: Domain-Driven Design, Cell-Based Arch...
WSO2CON 2024 - Cloud Native Middleware: Domain-Driven Design, Cell-Based Arch...WSO2CON 2024 - Cloud Native Middleware: Domain-Driven Design, Cell-Based Arch...
WSO2CON 2024 - Cloud Native Middleware: Domain-Driven Design, Cell-Based Arch...
 

Flink 0.10 @ Bay Area Meetup (October 2015)

Notas do Editor

  1. People previously made the case that high throughput and low latency are mutually exclusive