SlideShare a Scribd company logo
1 of 44
1
Kostas Kloudas
@KLOUBEN_K
meetup@ResearchGate
February16, 2017
Extending Flink’s Streaming APIs
2
Original creators of Apache
Flink®
Providers of the
dA Platform, a supported
Flink distribution
Additions in Flink 1.2
3
Additions in Flink 1.2
4
 Re-scalable State
 Low-level Stream Operations
 Asynchronous I/O
 Table API and SQL
 Externalized Checkpoints
 Queryable State
 Mesos Integration
 …and, of course, Documentation
Additions in Flink 1.2
5
 Re-scalable State
 Low-level Stream Operations
 Asynchronous I/O
 Table API and SQL
 Externalized Checkpoints
 Queryable State
 Mesos Integration
 …and, of course, Documentation
Low-level Stream Operations
6
Common Usecase Skeleton A
 On each incoming element:
• update some state
• register a callback for a moment in the future
 When that moment comes:
• Check a condition and perform a certain
action, e.g. emit an element
7
 Use built-in windowing:
• +Expressive
• +A lot of functionality out-of-the-box
• - Not always intuitive
• - An overkill for simple cases
 Write your own operator:
• - Too many things to account for in Flink 1.1
8
The Flink 1.1 way
The Flink 1.2 way: ProcessFunction
 Gives access to all basic building blocks:
• Events
• Fault-tolerant, Consistent State
• Timers (event- and processing-time)
9
The Flink 1.2 way: ProcessFunction
 Simple yet powerful API:
10
/**
* Process one element from the input stream.
*/
void processElement(I value, Context ctx, Collector<O> out) throws Exception;
/**
* Called when a timer set using {@link TimerService} fires.
*/
void onTimer(long timestamp, OnTimerContext ctx, Collector<O> out) throws Exception;
The Flink 1.2 way: ProcessFunction
 Simple yet powerful API:
11
/**
* Process one element from the input stream.
*/
void processElement(I value, Context ctx, Collector<O> out) throws Exception;
/**
* Called when a timer set using {@link TimerService} fires.
*/
void onTimer(long timestamp, OnTimerContext ctx, Collector<O> out) throws Exception;
A collector to emit result
values
The Flink 1.2 way: ProcessFunction
 Simple yet powerful API:
12
/**
* Process one element from the input stream.
*/
void processElement(I value, Context ctx, Collector<O> out) throws Exception;
/**
* Called when a timer set using {@link TimerService} fires.
*/
void onTimer(long timestamp, OnTimerContext ctx, Collector<O> out) throws Exception;
1. Get the timestamp of the element
2. Interact with the TimerService to:
• query the current time
• and register timers
1. Do the above
2. Query if we are operating on Event
or Processing time
ProcessFunction: example
 Requirements:
• maintain counts per incoming key, and
• emit the key/count pair if no element came for
the key in the last 100 ms (in event time)
13
ProcessFunction: example
14
 Implementation sketch:
• Store the count, key and last mod timestamp in
a ValueState (scoped by key)
• For each record:
• update the counter and the last mod timestamp
• register a timer 100ms from “now” (in event time)
• When the timer fires:
• check the callback’s timestamp against the last mod time for the
key and
• emit the key/count pair if they match
ProcessFunction: example
15
public class MyProcessFunction extends
RichProcessFunction<Tuple2<String, String>, Tuple2<String, Long>> {
@Override
public void open(Configuration parameters) throws Exception {
// register our state with the state backend
}
@Override
public void processElement(Tuple2<String, Long> value, Context ctx,
Collector<Tuple2<String, Long>> out) throws Exception {
// update our state and register a timer
}
@Override
public void onTimer(long timestamp, OnTimerContext ctx,
Collector<Tuple2<String, Long>> out) throws Exception {
// check the state for the key and emit a result if needed
}
}
ProcessFunction: example
16
public class MyProcessFunction extends
RichProcessFunction<Tuple2<String, String>, Tuple2<String, Long>> {
private ValueState<MyStateClass> state;
@Override
public void open(Configuration parameters) throws Exception {
state = getRuntimeContext().getState(
new ValueStateDescriptor<>("myState", MyStateClass.class));
}
}
ProcessFunction: example
17
public class MyProcessFunction extends
RichProcessFunction<Tuple2<String, String>, Tuple2<String, Long>> {
@Override
public void processElement(Tuple2<String, Long> value, Context ctx,
Collector<Tuple2<String, Long>> out) throws Exception {
CountWithTimestamp current = state.value();
if (current == null) {
current = new CountWithTimestamp();
current.key = value.f0;
}
current.count++;
current.lastModified = ctx.timestamp();
state.update(current);
ctx.timerService().registerEventTimeTimer(current.timestamp + 100);
}
}
ProcessFunction: example
18
public class MyProcessFunction extends
RichProcessFunction<Tuple2<String, String>, Tuple2<String, Long>> {
@Override
public void onTimer(long timestamp, OnTimerContext ctx,
Collector<Tuple2<String, Long>> out) throws Exception {
CountWithTimestamp result = state.value();
if (timestamp == result.lastModified) {
out.collect(new Tuple2<String, Long>(result.key, result.count)); }
}
}
ProcessFunction: example
19
 If your stream is not keyed, you can always
group on a dummy key
 BEWARE: parallelism of 1
stream.keyBy("id")
.process(new MyProcessFunction())
ProcessFunction: miscellaneous
20
 CoProcessFunction for low-level joins:
• Applied on two input streams
• Has two processElement() methods, one for each input stream
 Upcoming releases may further enhance the
ProcessFunction/CoProcessFunction
 Planning to transform all CEP operators to ProcessFunctions
Asynchronous I/O
21
Common Usecase Skeleton B
22
 On each incoming element:
• extract some info from the element (e.g. key)
• query an external storage system (DB or KV-
store) for additional info
• emit an enriched version of the input element
 Write a MapFunction that queries the DB:
• +Simple
• - Slow (synchronous access) or/and
• - Requires high parallelism (more tasks)
 Write your own operator:
• - Too many things to account for in Flink 1.1
23
The Flink 1.1 way
 Write a MapFunction that queries the DB:
• +Simple
• - Slow (synchronous access) or/and
• - Requires high parallelism (more tasks)
 Write your own operator:
• - Too many things to account for in Flink 1.1
24
The Flink 1.1 way
25
Synchronous Access
26
Synchronous Access
Communication delay can
dominate application
throughput and latency
27
Asynchronous Access
 Requirement:
• a client that supports asynchronous requests
 Flink handles the rest:
• integration of async IO with DataStream API
• fault-tolerance
• order of emitted elements
• correct time semantics (event/processing time)
28
The Flink 1.2 way: AsyncFunction
 Simple API:
/**
* Trigger async operation for each stream input.
*/
void asyncInvoke(IN input, AsyncCollector<OUT> collector) throws Exception;
 API call:
/**
* Example async function call.
*/
DataStream<...> result = AsyncDataStream.(un)orderedWait(stream,
new MyAsyncFunction(), 1000, TimeUnit.MILLISECONDS, 100);
29
The Flink 1.2 way: AsyncFunction
The Flink 1.2 way: AsyncFunction
30
Emitter
P2P3 P1P4
AsyncWaitOperator
E5
AsyncWaitOperator:
• a queue of “Promises”
• a separate thread (Emitter)
The Flink 1.2 way: AsyncFunction
31
Emitter
P2P3 P1P4
AsyncWaitOperator
• Wrap E5 in a “promise” P5
• Put P5 in the queue
• Call asyncInvoke(E5, P5)
E5
P5
asyncInvoke(E5, P5)P5
The Flink 1.2 way: AsyncFunction
32
Emitter
P2P3 P1P4
AsyncWaitOperator
E5
P5
asyncInvoke(E5, P5)P5
asyncInvoke(value, asyncCollector):
• a user-defined function
• value : the input element
• asyncCollector : the collector of the
result (when the query returns)
The Flink 1.2 way: AsyncFunction
33
Emitter
P2P3 P1P4
AsyncWaitOperator
E5
P5
asyncInvoke(E5, P5)P5
asyncInvoke(value, asyncCollector):
• a user-defined function
• value : the input element
• asyncCollector : the collector of the
result (when the query returns)
Future<String> future = client.query(E5);
future.thenAccept((String result) -> { P5.collect(
Collections.singleton(
new Tuple2<>(E5, result)));
});
The Flink 1.2 way: AsyncFunction
34
Emitter
P2P3 P1P4
AsyncWaitOperator
E5
P5
asyncInvoke(E5, P5)P5
asyncInvoke(value, asyncCollector):
• a user-defined function
• value : the input element
• asyncCollector : the collector of the
result (when the query returns)
Future<String> future = client.query(E5);
future.thenAccept((String result) -> { P5.collect(
Collections.singleton(
new Tuple2<>(E5, result)));
});
The Flink 1.2 way: AsyncFunction
35
Emitter
P2P3 P1P4
AsyncWaitOperator
E5
P5
asyncInvoke(E5, P5)P5
Emitter:
• separate thread
• polls queue for completed
promises (blocking)
• emits elements downstream
36
The Flink 1.2 way: AsyncFunction
DataStream<Tuple2<String, String>> result =
AsyncDataStream.(un)orderedWait(stream,
new MyAsyncFunction(),
1000, TimeUnit.MILLISECONDS,
100);
 our asyncFunction
 a timeout: max time until considered failed
 capacity: max number of in-flight requests
37
The Flink 1.2 way: AsyncFunction
DataStream<Tuple2<String, String>> result =
AsyncDataStream.(un)orderedWait(stream,
new MyAsyncFunction(),
1000, TimeUnit.MILLISECONDS,
100);
38
The Flink 1.2 way: AsyncFunction
DataStream<Tuple2<String, String>> result =
AsyncDataStream.(un)orderedWait(stream,
new MyAsyncFunction(),
1000, TimeUnit.MILLISECONDS,
100);
P2P3 P1P4E2E3 E1E4
Ideally... Emitter
39
The Flink 1.2 way: AsyncFunction
DataStream<Tuple2<String, String>> result =
AsyncDataStream.unorderedWait(stream,
new MyAsyncFunction(),
1000, TimeUnit.MILLISECONDS,
100);
P2P3 P1P4E2E3 E1E4
Reallistically... Emitter
...output ordered based on which request finished first
40
The Flink 1.2 way: AsyncFunction
P2P3 P1P4E2E3 E1E4
Emitter
 unorderedWait: emit results in order of completion
 orderedWait: emit results in order of arrival
 Always: watermarks never overpass elements and vice versa
Documentation
 ProcessFunction:
https://ci.apache.org/projects/flink/flink-docs-release-
1.2/dev/stream/process_function.html
 AsyncIO:
https://ci.apache.org/projects/flink/flink-docs-release-1.2/dev/stream/asyncio.html
41
4
Thank you!
@KLOUBEN_K
@ApacheFlink
@dataArtisans
43
One day of hands-on Flink training
One day of conference
Tickets are on sale
Call for Papers is already open
Please visit our website:
http://sf.flink-forward.org
Follow us on Twitter:
@FlinkForward
We are hiring!
data-artisans.com/careers

More Related Content

What's hot

Tuning Apache Kafka Connectors for Flink.pptx
Tuning Apache Kafka Connectors for Flink.pptxTuning Apache Kafka Connectors for Flink.pptx
Tuning Apache Kafka Connectors for Flink.pptxFlink Forward
 
Squirreling Away $640 Billion: How Stripe Leverages Flink for Change Data Cap...
Squirreling Away $640 Billion: How Stripe Leverages Flink for Change Data Cap...Squirreling Away $640 Billion: How Stripe Leverages Flink for Change Data Cap...
Squirreling Away $640 Billion: How Stripe Leverages Flink for Change Data Cap...Flink Forward
 
The top 3 challenges running multi-tenant Flink at scale
The top 3 challenges running multi-tenant Flink at scaleThe top 3 challenges running multi-tenant Flink at scale
The top 3 challenges running multi-tenant Flink at scaleFlink Forward
 
Practical learnings from running thousands of Flink jobs
Practical learnings from running thousands of Flink jobsPractical learnings from running thousands of Flink jobs
Practical learnings from running thousands of Flink jobsFlink Forward
 
HTTP Analytics for 6M requests per second using ClickHouse, by Alexander Boc...
HTTP Analytics for 6M requests per second using ClickHouse, by  Alexander Boc...HTTP Analytics for 6M requests per second using ClickHouse, by  Alexander Boc...
HTTP Analytics for 6M requests per second using ClickHouse, by Alexander Boc...Altinity Ltd
 
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
 
Designing ETL Pipelines with Structured Streaming and Delta Lake—How to Archi...
Designing ETL Pipelines with Structured Streaming and Delta Lake—How to Archi...Designing ETL Pipelines with Structured Streaming and Delta Lake—How to Archi...
Designing ETL Pipelines with Structured Streaming and Delta Lake—How to Archi...Databricks
 
Streaming Event Time Partitioning with Apache Flink and Apache Iceberg - Juli...
Streaming Event Time Partitioning with Apache Flink and Apache Iceberg - Juli...Streaming Event Time Partitioning with Apache Flink and Apache Iceberg - Juli...
Streaming Event Time Partitioning with Apache Flink and Apache Iceberg - Juli...Flink Forward
 
Apache Pinot Case Study: Building Distributed Analytics Systems Using Apache ...
Apache Pinot Case Study: Building Distributed Analytics Systems Using Apache ...Apache Pinot Case Study: Building Distributed Analytics Systems Using Apache ...
Apache Pinot Case Study: Building Distributed Analytics Systems Using Apache ...HostedbyConfluent
 
From Message to Cluster: A Realworld Introduction to Kafka Capacity Planning
From Message to Cluster: A Realworld Introduction to Kafka Capacity PlanningFrom Message to Cluster: A Realworld Introduction to Kafka Capacity Planning
From Message to Cluster: A Realworld Introduction to Kafka Capacity Planningconfluent
 
Improving fault tolerance and scaling out in Kafka Streams with Bill Bejeck |...
Improving fault tolerance and scaling out in Kafka Streams with Bill Bejeck |...Improving fault tolerance and scaling out in Kafka Streams with Bill Bejeck |...
Improving fault tolerance and scaling out in Kafka Streams with Bill Bejeck |...HostedbyConfluent
 
Apache Arrow Flight Overview
Apache Arrow Flight OverviewApache Arrow Flight Overview
Apache Arrow Flight OverviewJacques Nadeau
 
One sink to rule them all: Introducing the new Async Sink
One sink to rule them all: Introducing the new Async SinkOne sink to rule them all: Introducing the new Async Sink
One sink to rule them all: Introducing the new Async SinkFlink Forward
 
Apache Flink Adoption at Shopify
Apache Flink Adoption at ShopifyApache Flink Adoption at Shopify
Apache Flink Adoption at ShopifyYaroslav Tkachenko
 
Distributed stream processing with Apache Kafka
Distributed stream processing with Apache KafkaDistributed stream processing with Apache Kafka
Distributed stream processing with Apache Kafkaconfluent
 
Finding attacks with these 6 events
Finding attacks with these 6 eventsFinding attacks with these 6 events
Finding attacks with these 6 eventsMichael Gough
 
How Uber scaled its Real Time Infrastructure to Trillion events per day
How Uber scaled its Real Time Infrastructure to Trillion events per dayHow Uber scaled its Real Time Infrastructure to Trillion events per day
How Uber scaled its Real Time Infrastructure to Trillion events per dayDataWorks Summit
 
Kafka Tutorial - Introduction to Apache Kafka (Part 1)
Kafka Tutorial - Introduction to Apache Kafka (Part 1)Kafka Tutorial - Introduction to Apache Kafka (Part 1)
Kafka Tutorial - Introduction to Apache Kafka (Part 1)Jean-Paul Azar
 
Hudi architecture, fundamentals and capabilities
Hudi architecture, fundamentals and capabilitiesHudi architecture, fundamentals and capabilities
Hudi architecture, fundamentals and capabilitiesNishith Agarwal
 

What's hot (20)

Tuning Apache Kafka Connectors for Flink.pptx
Tuning Apache Kafka Connectors for Flink.pptxTuning Apache Kafka Connectors for Flink.pptx
Tuning Apache Kafka Connectors for Flink.pptx
 
Squirreling Away $640 Billion: How Stripe Leverages Flink for Change Data Cap...
Squirreling Away $640 Billion: How Stripe Leverages Flink for Change Data Cap...Squirreling Away $640 Billion: How Stripe Leverages Flink for Change Data Cap...
Squirreling Away $640 Billion: How Stripe Leverages Flink for Change Data Cap...
 
The top 3 challenges running multi-tenant Flink at scale
The top 3 challenges running multi-tenant Flink at scaleThe top 3 challenges running multi-tenant Flink at scale
The top 3 challenges running multi-tenant Flink at scale
 
Practical learnings from running thousands of Flink jobs
Practical learnings from running thousands of Flink jobsPractical learnings from running thousands of Flink jobs
Practical learnings from running thousands of Flink jobs
 
HTTP Analytics for 6M requests per second using ClickHouse, by Alexander Boc...
HTTP Analytics for 6M requests per second using ClickHouse, by  Alexander Boc...HTTP Analytics for 6M requests per second using ClickHouse, by  Alexander Boc...
HTTP Analytics for 6M requests per second using ClickHouse, by Alexander Boc...
 
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...
 
Logstash
LogstashLogstash
Logstash
 
Designing ETL Pipelines with Structured Streaming and Delta Lake—How to Archi...
Designing ETL Pipelines with Structured Streaming and Delta Lake—How to Archi...Designing ETL Pipelines with Structured Streaming and Delta Lake—How to Archi...
Designing ETL Pipelines with Structured Streaming and Delta Lake—How to Archi...
 
Streaming Event Time Partitioning with Apache Flink and Apache Iceberg - Juli...
Streaming Event Time Partitioning with Apache Flink and Apache Iceberg - Juli...Streaming Event Time Partitioning with Apache Flink and Apache Iceberg - Juli...
Streaming Event Time Partitioning with Apache Flink and Apache Iceberg - Juli...
 
Apache Pinot Case Study: Building Distributed Analytics Systems Using Apache ...
Apache Pinot Case Study: Building Distributed Analytics Systems Using Apache ...Apache Pinot Case Study: Building Distributed Analytics Systems Using Apache ...
Apache Pinot Case Study: Building Distributed Analytics Systems Using Apache ...
 
From Message to Cluster: A Realworld Introduction to Kafka Capacity Planning
From Message to Cluster: A Realworld Introduction to Kafka Capacity PlanningFrom Message to Cluster: A Realworld Introduction to Kafka Capacity Planning
From Message to Cluster: A Realworld Introduction to Kafka Capacity Planning
 
Improving fault tolerance and scaling out in Kafka Streams with Bill Bejeck |...
Improving fault tolerance and scaling out in Kafka Streams with Bill Bejeck |...Improving fault tolerance and scaling out in Kafka Streams with Bill Bejeck |...
Improving fault tolerance and scaling out in Kafka Streams with Bill Bejeck |...
 
Apache Arrow Flight Overview
Apache Arrow Flight OverviewApache Arrow Flight Overview
Apache Arrow Flight Overview
 
One sink to rule them all: Introducing the new Async Sink
One sink to rule them all: Introducing the new Async SinkOne sink to rule them all: Introducing the new Async Sink
One sink to rule them all: Introducing the new Async Sink
 
Apache Flink Adoption at Shopify
Apache Flink Adoption at ShopifyApache Flink Adoption at Shopify
Apache Flink Adoption at Shopify
 
Distributed stream processing with Apache Kafka
Distributed stream processing with Apache KafkaDistributed stream processing with Apache Kafka
Distributed stream processing with Apache Kafka
 
Finding attacks with these 6 events
Finding attacks with these 6 eventsFinding attacks with these 6 events
Finding attacks with these 6 events
 
How Uber scaled its Real Time Infrastructure to Trillion events per day
How Uber scaled its Real Time Infrastructure to Trillion events per dayHow Uber scaled its Real Time Infrastructure to Trillion events per day
How Uber scaled its Real Time Infrastructure to Trillion events per day
 
Kafka Tutorial - Introduction to Apache Kafka (Part 1)
Kafka Tutorial - Introduction to Apache Kafka (Part 1)Kafka Tutorial - Introduction to Apache Kafka (Part 1)
Kafka Tutorial - Introduction to Apache Kafka (Part 1)
 
Hudi architecture, fundamentals and capabilities
Hudi architecture, fundamentals and capabilitiesHudi architecture, fundamentals and capabilities
Hudi architecture, fundamentals and capabilities
 

Viewers also liked

Timo Walther - Table & SQL API - unified APIs for batch and stream processing
Timo Walther - Table & SQL API - unified APIs for batch and stream processingTimo Walther - Table & SQL API - unified APIs for batch and stream processing
Timo Walther - Table & SQL API - unified APIs for batch and stream processingVerverica
 
Apache Flink Meetup: Sanjar Akhmedov - Joining Infinity – Windowless Stream ...
Apache Flink Meetup:  Sanjar Akhmedov - Joining Infinity – Windowless Stream ...Apache Flink Meetup:  Sanjar Akhmedov - Joining Infinity – Windowless Stream ...
Apache Flink Meetup: Sanjar Akhmedov - Joining Infinity – Windowless Stream ...Ververica
 
Stefan Richter - A look at Flink 1.2 and beyond @ Berlin Meetup
Stefan Richter - A look at Flink 1.2 and beyond @ Berlin Meetup Stefan Richter - A look at Flink 1.2 and beyond @ Berlin Meetup
Stefan Richter - A look at Flink 1.2 and beyond @ Berlin Meetup Ververica
 
Kostas Tzoumas - Stream Processing with Apache Flink®
Kostas Tzoumas - Stream Processing with Apache Flink®Kostas Tzoumas - Stream Processing with Apache Flink®
Kostas Tzoumas - Stream Processing with Apache Flink®Ververica
 
Stephan Ewen - Stream Processing as a Foundational Paradigm and Apache Flink'...
Stephan Ewen - Stream Processing as a Foundational Paradigm and Apache Flink'...Stephan Ewen - Stream Processing as a Foundational Paradigm and Apache Flink'...
Stephan Ewen - Stream Processing as a Foundational Paradigm and Apache Flink'...Ververica
 
Aljoscha Krettek - Apache Flink for IoT: How Event-Time Processing Enables Ea...
Aljoscha Krettek - Apache Flink for IoT: How Event-Time Processing Enables Ea...Aljoscha Krettek - Apache Flink for IoT: How Event-Time Processing Enables Ea...
Aljoscha Krettek - Apache Flink for IoT: How Event-Time Processing Enables Ea...Ververica
 
Robert Metzger - Apache Flink Community Updates November 2016 @ Berlin Meetup
Robert Metzger - Apache Flink Community Updates November 2016 @ Berlin Meetup Robert Metzger - Apache Flink Community Updates November 2016 @ Berlin Meetup
Robert Metzger - Apache Flink Community Updates November 2016 @ Berlin Meetup Ververica
 
Keynote: Stephan Ewen - Stream Processing as a Foundational Paradigm and Apac...
Keynote: Stephan Ewen - Stream Processing as a Foundational Paradigm and Apac...Keynote: Stephan Ewen - Stream Processing as a Foundational Paradigm and Apac...
Keynote: Stephan Ewen - Stream Processing as a Foundational Paradigm and Apac...Ververica
 
Stream Analytics with SQL on Apache Flink
Stream Analytics with SQL on Apache FlinkStream Analytics with SQL on Apache Flink
Stream Analytics with SQL on Apache FlinkFabian Hueske
 
Fundamentals of Stream Processing with Apache Beam, Tyler Akidau, Frances Perry
Fundamentals of Stream Processing with Apache Beam, Tyler Akidau, Frances Perry Fundamentals of Stream Processing with Apache Beam, Tyler Akidau, Frances Perry
Fundamentals of Stream Processing with Apache Beam, Tyler Akidau, Frances Perry confluent
 
Apache Flink's Table & SQL API - unified APIs for batch and stream processing
Apache Flink's Table & SQL API - unified APIs for batch and stream processingApache Flink's Table & SQL API - unified APIs for batch and stream processing
Apache Flink's Table & SQL API - unified APIs for batch and stream processingTimo Walther
 
Gilbert: Declarative Sparse Linear Algebra on Massively Parallel Dataflow Sys...
Gilbert: Declarative Sparse Linear Algebra on Massively Parallel Dataflow Sys...Gilbert: Declarative Sparse Linear Algebra on Massively Parallel Dataflow Sys...
Gilbert: Declarative Sparse Linear Algebra on Massively Parallel Dataflow Sys...Till Rohrmann
 
Doctor data de-duplication and profiling
Doctor data de-duplication and profilingDoctor data de-duplication and profiling
Doctor data de-duplication and profilingKADAMBINI SHREE
 
Boletín ultimas incorporacións bibliográfica. Biblioteca Provincial da Coruña...
Boletín ultimas incorporacións bibliográfica. Biblioteca Provincial da Coruña...Boletín ultimas incorporacións bibliográfica. Biblioteca Provincial da Coruña...
Boletín ultimas incorporacións bibliográfica. Biblioteca Provincial da Coruña...Bibliotecadicoruna
 
How to keep your heart healthy
How to keep your heart healthyHow to keep your heart healthy
How to keep your heart healthycuretips24
 
Cs2017 gary allemann presentation
Cs2017 gary allemann presentationCs2017 gary allemann presentation
Cs2017 gary allemann presentationGary Allemann
 
VIETNAMESE MARKET ANALYSIS Practical and Legal Perspective
VIETNAMESE MARKET ANALYSIS Practical and Legal PerspectiveVIETNAMESE MARKET ANALYSIS Practical and Legal Perspective
VIETNAMESE MARKET ANALYSIS Practical and Legal PerspectiveDr. Oliver Massmann
 
Knowledge & Enlightenment: Indian Mythological Perspective
Knowledge & Enlightenment: Indian Mythological PerspectiveKnowledge & Enlightenment: Indian Mythological Perspective
Knowledge & Enlightenment: Indian Mythological PerspectiveGautam Awasthi
 

Viewers also liked (20)

Timo Walther - Table & SQL API - unified APIs for batch and stream processing
Timo Walther - Table & SQL API - unified APIs for batch and stream processingTimo Walther - Table & SQL API - unified APIs for batch and stream processing
Timo Walther - Table & SQL API - unified APIs for batch and stream processing
 
Apache Flink Meetup: Sanjar Akhmedov - Joining Infinity – Windowless Stream ...
Apache Flink Meetup:  Sanjar Akhmedov - Joining Infinity – Windowless Stream ...Apache Flink Meetup:  Sanjar Akhmedov - Joining Infinity – Windowless Stream ...
Apache Flink Meetup: Sanjar Akhmedov - Joining Infinity – Windowless Stream ...
 
Stefan Richter - A look at Flink 1.2 and beyond @ Berlin Meetup
Stefan Richter - A look at Flink 1.2 and beyond @ Berlin Meetup Stefan Richter - A look at Flink 1.2 and beyond @ Berlin Meetup
Stefan Richter - A look at Flink 1.2 and beyond @ Berlin Meetup
 
Kostas Tzoumas - Stream Processing with Apache Flink®
Kostas Tzoumas - Stream Processing with Apache Flink®Kostas Tzoumas - Stream Processing with Apache Flink®
Kostas Tzoumas - Stream Processing with Apache Flink®
 
Stephan Ewen - Stream Processing as a Foundational Paradigm and Apache Flink'...
Stephan Ewen - Stream Processing as a Foundational Paradigm and Apache Flink'...Stephan Ewen - Stream Processing as a Foundational Paradigm and Apache Flink'...
Stephan Ewen - Stream Processing as a Foundational Paradigm and Apache Flink'...
 
Aljoscha Krettek - Apache Flink for IoT: How Event-Time Processing Enables Ea...
Aljoscha Krettek - Apache Flink for IoT: How Event-Time Processing Enables Ea...Aljoscha Krettek - Apache Flink for IoT: How Event-Time Processing Enables Ea...
Aljoscha Krettek - Apache Flink for IoT: How Event-Time Processing Enables Ea...
 
Robert Metzger - Apache Flink Community Updates November 2016 @ Berlin Meetup
Robert Metzger - Apache Flink Community Updates November 2016 @ Berlin Meetup Robert Metzger - Apache Flink Community Updates November 2016 @ Berlin Meetup
Robert Metzger - Apache Flink Community Updates November 2016 @ Berlin Meetup
 
Keynote: Stephan Ewen - Stream Processing as a Foundational Paradigm and Apac...
Keynote: Stephan Ewen - Stream Processing as a Foundational Paradigm and Apac...Keynote: Stephan Ewen - Stream Processing as a Foundational Paradigm and Apac...
Keynote: Stephan Ewen - Stream Processing as a Foundational Paradigm and Apac...
 
Stream Analytics with SQL on Apache Flink
Stream Analytics with SQL on Apache FlinkStream Analytics with SQL on Apache Flink
Stream Analytics with SQL on Apache Flink
 
Fundamentals of Stream Processing with Apache Beam, Tyler Akidau, Frances Perry
Fundamentals of Stream Processing with Apache Beam, Tyler Akidau, Frances Perry Fundamentals of Stream Processing with Apache Beam, Tyler Akidau, Frances Perry
Fundamentals of Stream Processing with Apache Beam, Tyler Akidau, Frances Perry
 
Apache Flink's Table & SQL API - unified APIs for batch and stream processing
Apache Flink's Table & SQL API - unified APIs for batch and stream processingApache Flink's Table & SQL API - unified APIs for batch and stream processing
Apache Flink's Table & SQL API - unified APIs for batch and stream processing
 
Gilbert: Declarative Sparse Linear Algebra on Massively Parallel Dataflow Sys...
Gilbert: Declarative Sparse Linear Algebra on Massively Parallel Dataflow Sys...Gilbert: Declarative Sparse Linear Algebra on Massively Parallel Dataflow Sys...
Gilbert: Declarative Sparse Linear Algebra on Massively Parallel Dataflow Sys...
 
Doctor data de-duplication and profiling
Doctor data de-duplication and profilingDoctor data de-duplication and profiling
Doctor data de-duplication and profiling
 
Boletín ultimas incorporacións bibliográfica. Biblioteca Provincial da Coruña...
Boletín ultimas incorporacións bibliográfica. Biblioteca Provincial da Coruña...Boletín ultimas incorporacións bibliográfica. Biblioteca Provincial da Coruña...
Boletín ultimas incorporacións bibliográfica. Biblioteca Provincial da Coruña...
 
How to keep your heart healthy
How to keep your heart healthyHow to keep your heart healthy
How to keep your heart healthy
 
Hamlet & Bakistan
Hamlet & BakistanHamlet & Bakistan
Hamlet & Bakistan
 
O acoso é desigualdade, é violencia: ERRADIQUÉMOLO!
O acoso é desigualdade, é violencia: ERRADIQUÉMOLO!O acoso é desigualdade, é violencia: ERRADIQUÉMOLO!
O acoso é desigualdade, é violencia: ERRADIQUÉMOLO!
 
Cs2017 gary allemann presentation
Cs2017 gary allemann presentationCs2017 gary allemann presentation
Cs2017 gary allemann presentation
 
VIETNAMESE MARKET ANALYSIS Practical and Legal Perspective
VIETNAMESE MARKET ANALYSIS Practical and Legal PerspectiveVIETNAMESE MARKET ANALYSIS Practical and Legal Perspective
VIETNAMESE MARKET ANALYSIS Practical and Legal Perspective
 
Knowledge & Enlightenment: Indian Mythological Perspective
Knowledge & Enlightenment: Indian Mythological PerspectiveKnowledge & Enlightenment: Indian Mythological Perspective
Knowledge & Enlightenment: Indian Mythological Perspective
 

Similar to Kostas Kloudas - Extending Flink's Streaming APIs

Flink Forward SF 2017: Konstantinos Kloudas - Extending Flink’s Streaming APIs
Flink Forward SF 2017: Konstantinos Kloudas -  Extending Flink’s Streaming APIsFlink Forward SF 2017: Konstantinos Kloudas -  Extending Flink’s Streaming APIs
Flink Forward SF 2017: Konstantinos Kloudas - Extending Flink’s Streaming APIsFlink Forward
 
Treasure Data Summer Internship 2016
Treasure Data Summer Internship 2016Treasure Data Summer Internship 2016
Treasure Data Summer Internship 2016Yuta Iwama
 
Exactly-Once Made Easy: Transactional Messaging Improvement for Usability and...
Exactly-Once Made Easy: Transactional Messaging Improvement for Usability and...Exactly-Once Made Easy: Transactional Messaging Improvement for Usability and...
Exactly-Once Made Easy: Transactional Messaging Improvement for Usability and...HostedbyConfluent
 
Exactly-Once Made Easy: Transactional Messaging Improvement for Usability and...
Exactly-Once Made Easy: Transactional Messaging Improvement for Usability and...Exactly-Once Made Easy: Transactional Messaging Improvement for Usability and...
Exactly-Once Made Easy: Transactional Messaging Improvement for Usability and...Guozhang Wang
 
Java/Scala Lab: Анатолий Кметюк - Scala SubScript: Алгебра для реактивного пр...
Java/Scala Lab: Анатолий Кметюк - Scala SubScript: Алгебра для реактивного пр...Java/Scala Lab: Анатолий Кметюк - Scala SubScript: Алгебра для реактивного пр...
Java/Scala Lab: Анатолий Кметюк - Scala SubScript: Алгебра для реактивного пр...GeeksLab Odessa
 
History of asynchronous in .NET
History of asynchronous in .NETHistory of asynchronous in .NET
History of asynchronous in .NETMarcin Tyborowski
 
Running a Scalable And Reliable Symfony2 Application in Cloud (Symfony Sweden...
Running a Scalable And Reliable Symfony2 Application in Cloud (Symfony Sweden...Running a Scalable And Reliable Symfony2 Application in Cloud (Symfony Sweden...
Running a Scalable And Reliable Symfony2 Application in Cloud (Symfony Sweden...Ville Mattila
 
Stream processing with Apache Flink - Maximilian Michels Data Artisans
Stream processing with Apache Flink - Maximilian Michels Data ArtisansStream processing with Apache Flink - Maximilian Michels Data Artisans
Stream processing with Apache Flink - Maximilian Michels Data ArtisansEvention
 
Go Concurrency Patterns
Go Concurrency PatternsGo Concurrency Patterns
Go Concurrency PatternsElifTech
 
Apache Flink Stream Processing
Apache Flink Stream ProcessingApache Flink Stream Processing
Apache Flink Stream ProcessingSuneel Marthi
 
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 FlinkDataWorks Summit/Hadoop Summit
 
Anton Moldovan "Load testing which you always wanted"
Anton Moldovan "Load testing which you always wanted"Anton Moldovan "Load testing which you always wanted"
Anton Moldovan "Load testing which you always wanted"Fwdays
 
Fabian Hueske_Till Rohrmann - Declarative stream processing with StreamSQL an...
Fabian Hueske_Till Rohrmann - Declarative stream processing with StreamSQL an...Fabian Hueske_Till Rohrmann - Declarative stream processing with StreamSQL an...
Fabian Hueske_Till Rohrmann - Declarative stream processing with StreamSQL an...Flink Forward
 
Flink Forward Berlin 2017: Patrick Gunia - Migration of a realtime stats prod...
Flink Forward Berlin 2017: Patrick Gunia - Migration of a realtime stats prod...Flink Forward Berlin 2017: Patrick Gunia - Migration of a realtime stats prod...
Flink Forward Berlin 2017: Patrick Gunia - Migration of a realtime stats prod...Flink Forward
 
Async programming and python
Async programming and pythonAsync programming and python
Async programming and pythonChetan Giridhar
 
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 @ Tel Aviv / Herzliya Meetup
Apache Flink @ Tel Aviv / Herzliya MeetupApache Flink @ Tel Aviv / Herzliya Meetup
Apache Flink @ Tel Aviv / Herzliya MeetupRobert Metzger
 

Similar to Kostas Kloudas - Extending Flink's Streaming APIs (20)

Flink Forward SF 2017: Konstantinos Kloudas - Extending Flink’s Streaming APIs
Flink Forward SF 2017: Konstantinos Kloudas -  Extending Flink’s Streaming APIsFlink Forward SF 2017: Konstantinos Kloudas -  Extending Flink’s Streaming APIs
Flink Forward SF 2017: Konstantinos Kloudas - Extending Flink’s Streaming APIs
 
Treasure Data Summer Internship 2016
Treasure Data Summer Internship 2016Treasure Data Summer Internship 2016
Treasure Data Summer Internship 2016
 
Exactly-Once Made Easy: Transactional Messaging Improvement for Usability and...
Exactly-Once Made Easy: Transactional Messaging Improvement for Usability and...Exactly-Once Made Easy: Transactional Messaging Improvement for Usability and...
Exactly-Once Made Easy: Transactional Messaging Improvement for Usability and...
 
Exactly-Once Made Easy: Transactional Messaging Improvement for Usability and...
Exactly-Once Made Easy: Transactional Messaging Improvement for Usability and...Exactly-Once Made Easy: Transactional Messaging Improvement for Usability and...
Exactly-Once Made Easy: Transactional Messaging Improvement for Usability and...
 
Java/Scala Lab: Анатолий Кметюк - Scala SubScript: Алгебра для реактивного пр...
Java/Scala Lab: Анатолий Кметюк - Scala SubScript: Алгебра для реактивного пр...Java/Scala Lab: Анатолий Кметюк - Scala SubScript: Алгебра для реактивного пр...
Java/Scala Lab: Анатолий Кметюк - Scala SubScript: Алгебра для реактивного пр...
 
History of asynchronous in .NET
History of asynchronous in .NETHistory of asynchronous in .NET
History of asynchronous in .NET
 
Running a Scalable And Reliable Symfony2 Application in Cloud (Symfony Sweden...
Running a Scalable And Reliable Symfony2 Application in Cloud (Symfony Sweden...Running a Scalable And Reliable Symfony2 Application in Cloud (Symfony Sweden...
Running a Scalable And Reliable Symfony2 Application in Cloud (Symfony Sweden...
 
Big Data Warsaw
Big Data WarsawBig Data Warsaw
Big Data Warsaw
 
Stream processing with Apache Flink - Maximilian Michels Data Artisans
Stream processing with Apache Flink - Maximilian Michels Data ArtisansStream processing with Apache Flink - Maximilian Michels Data Artisans
Stream processing with Apache Flink - Maximilian Michels Data Artisans
 
Go Concurrency Patterns
Go Concurrency PatternsGo Concurrency Patterns
Go Concurrency Patterns
 
Apache Flink Stream Processing
Apache Flink Stream ProcessingApache Flink Stream Processing
Apache Flink Stream Processing
 
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
 
Anton Moldovan "Load testing which you always wanted"
Anton Moldovan "Load testing which you always wanted"Anton Moldovan "Load testing which you always wanted"
Anton Moldovan "Load testing which you always wanted"
 
Fabian Hueske_Till Rohrmann - Declarative stream processing with StreamSQL an...
Fabian Hueske_Till Rohrmann - Declarative stream processing with StreamSQL an...Fabian Hueske_Till Rohrmann - Declarative stream processing with StreamSQL an...
Fabian Hueske_Till Rohrmann - Declarative stream processing with StreamSQL an...
 
Flink Forward Berlin 2017: Patrick Gunia - Migration of a realtime stats prod...
Flink Forward Berlin 2017: Patrick Gunia - Migration of a realtime stats prod...Flink Forward Berlin 2017: Patrick Gunia - Migration of a realtime stats prod...
Flink Forward Berlin 2017: Patrick Gunia - Migration of a realtime stats prod...
 
Unified Stream and Batch Processing with Apache Flink
Unified Stream and Batch Processing with Apache FlinkUnified Stream and Batch Processing with Apache Flink
Unified Stream and Batch Processing with Apache Flink
 
Async programming and python
Async programming and pythonAsync programming and python
Async programming and python
 
Python, do you even async?
Python, do you even async?Python, do you even async?
Python, do you even async?
 
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 @ Tel Aviv / Herzliya Meetup
Apache Flink @ Tel Aviv / Herzliya MeetupApache Flink @ Tel Aviv / Herzliya Meetup
Apache Flink @ Tel Aviv / Herzliya Meetup
 

More from Ververica

2020-05-06 Apache Flink Meetup London: The Easiest Way to Get Operational wit...
2020-05-06 Apache Flink Meetup London: The Easiest Way to Get Operational wit...2020-05-06 Apache Flink Meetup London: The Easiest Way to Get Operational wit...
2020-05-06 Apache Flink Meetup London: The Easiest Way to Get Operational wit...Ververica
 
Webinar: How to contribute to Apache Flink - Robert Metzger
Webinar:  How to contribute to Apache Flink - Robert MetzgerWebinar:  How to contribute to Apache Flink - Robert Metzger
Webinar: How to contribute to Apache Flink - Robert MetzgerVerverica
 
Webinar: Deep Dive on Apache Flink State - Seth Wiesman
Webinar: Deep Dive on Apache Flink State - Seth WiesmanWebinar: Deep Dive on Apache Flink State - Seth Wiesman
Webinar: Deep Dive on Apache Flink State - Seth WiesmanVerverica
 
Webinar: 99 Ways to Enrich Streaming Data with Apache Flink - Konstantin Knauf
Webinar: 99 Ways to Enrich Streaming Data with Apache Flink - Konstantin KnaufWebinar: 99 Ways to Enrich Streaming Data with Apache Flink - Konstantin Knauf
Webinar: 99 Ways to Enrich Streaming Data with Apache Flink - Konstantin KnaufVerverica
 
Webinar: Detecting row patterns with Flink SQL - Dawid Wysakowicz
Webinar:  Detecting row patterns with Flink SQL - Dawid WysakowiczWebinar:  Detecting row patterns with Flink SQL - Dawid Wysakowicz
Webinar: Detecting row patterns with Flink SQL - Dawid WysakowiczVerverica
 
Deploying Flink on Kubernetes - David Anderson
 Deploying Flink on Kubernetes - David Anderson Deploying Flink on Kubernetes - David Anderson
Deploying Flink on Kubernetes - David AndersonVerverica
 
Webinar: Flink SQL in Action - Fabian Hueske
 Webinar: Flink SQL in Action - Fabian Hueske Webinar: Flink SQL in Action - Fabian Hueske
Webinar: Flink SQL in Action - Fabian HueskeVerverica
 
2018-04 Kafka Summit London: Stephan Ewen - "Apache Flink and Apache Kafka fo...
2018-04 Kafka Summit London: Stephan Ewen - "Apache Flink and Apache Kafka fo...2018-04 Kafka Summit London: Stephan Ewen - "Apache Flink and Apache Kafka fo...
2018-04 Kafka Summit London: Stephan Ewen - "Apache Flink and Apache Kafka fo...Ververica
 
2018-01 Seattle Apache Flink Meetup at OfferUp, Opening Remarks and Talk 2
2018-01 Seattle Apache Flink Meetup at OfferUp, Opening Remarks and Talk 22018-01 Seattle Apache Flink Meetup at OfferUp, Opening Remarks and Talk 2
2018-01 Seattle Apache Flink Meetup at OfferUp, Opening Remarks and Talk 2Ververica
 
Stephan Ewen - Experiences running Flink at Very Large Scale
Stephan Ewen -  Experiences running Flink at Very Large ScaleStephan Ewen -  Experiences running Flink at Very Large Scale
Stephan Ewen - Experiences running Flink at Very Large ScaleVerverica
 
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
 
Tzu-Li (Gordon) Tai - Stateful Stream Processing with Apache Flink
Tzu-Li (Gordon) Tai - Stateful Stream Processing with Apache FlinkTzu-Li (Gordon) Tai - Stateful Stream Processing with Apache Flink
Tzu-Li (Gordon) Tai - Stateful Stream Processing with Apache FlinkVerverica
 
Kostas Kloudas - Complex Event Processing with Flink: the state of FlinkCEP
Kostas Kloudas - Complex Event Processing with Flink: the state of FlinkCEP Kostas Kloudas - Complex Event Processing with Flink: the state of FlinkCEP
Kostas Kloudas - Complex Event Processing with Flink: the state of FlinkCEP Ververica
 
Aljoscha Krettek - Portable stateful big data processing in Apache Beam
Aljoscha Krettek - Portable stateful big data processing in Apache BeamAljoscha Krettek - Portable stateful big data processing in Apache Beam
Aljoscha Krettek - Portable stateful big data processing in Apache BeamVerverica
 
Aljoscha Krettek - Apache Flink® and IoT: How Stateful Event-Time Processing ...
Aljoscha Krettek - Apache Flink® and IoT: How Stateful Event-Time Processing ...Aljoscha Krettek - Apache Flink® and IoT: How Stateful Event-Time Processing ...
Aljoscha Krettek - Apache Flink® and IoT: How Stateful Event-Time Processing ...Ververica
 
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
 
Kostas Tzoumas - Apache Flink®: State of the Union and What's Next
Kostas Tzoumas - Apache Flink®: State of the Union and What's NextKostas Tzoumas - Apache Flink®: State of the Union and What's Next
Kostas Tzoumas - Apache Flink®: State of the Union and What's NextVerverica
 

More from Ververica (17)

2020-05-06 Apache Flink Meetup London: The Easiest Way to Get Operational wit...
2020-05-06 Apache Flink Meetup London: The Easiest Way to Get Operational wit...2020-05-06 Apache Flink Meetup London: The Easiest Way to Get Operational wit...
2020-05-06 Apache Flink Meetup London: The Easiest Way to Get Operational wit...
 
Webinar: How to contribute to Apache Flink - Robert Metzger
Webinar:  How to contribute to Apache Flink - Robert MetzgerWebinar:  How to contribute to Apache Flink - Robert Metzger
Webinar: How to contribute to Apache Flink - Robert Metzger
 
Webinar: Deep Dive on Apache Flink State - Seth Wiesman
Webinar: Deep Dive on Apache Flink State - Seth WiesmanWebinar: Deep Dive on Apache Flink State - Seth Wiesman
Webinar: Deep Dive on Apache Flink State - Seth Wiesman
 
Webinar: 99 Ways to Enrich Streaming Data with Apache Flink - Konstantin Knauf
Webinar: 99 Ways to Enrich Streaming Data with Apache Flink - Konstantin KnaufWebinar: 99 Ways to Enrich Streaming Data with Apache Flink - Konstantin Knauf
Webinar: 99 Ways to Enrich Streaming Data with Apache Flink - Konstantin Knauf
 
Webinar: Detecting row patterns with Flink SQL - Dawid Wysakowicz
Webinar:  Detecting row patterns with Flink SQL - Dawid WysakowiczWebinar:  Detecting row patterns with Flink SQL - Dawid Wysakowicz
Webinar: Detecting row patterns with Flink SQL - Dawid Wysakowicz
 
Deploying Flink on Kubernetes - David Anderson
 Deploying Flink on Kubernetes - David Anderson Deploying Flink on Kubernetes - David Anderson
Deploying Flink on Kubernetes - David Anderson
 
Webinar: Flink SQL in Action - Fabian Hueske
 Webinar: Flink SQL in Action - Fabian Hueske Webinar: Flink SQL in Action - Fabian Hueske
Webinar: Flink SQL in Action - Fabian Hueske
 
2018-04 Kafka Summit London: Stephan Ewen - "Apache Flink and Apache Kafka fo...
2018-04 Kafka Summit London: Stephan Ewen - "Apache Flink and Apache Kafka fo...2018-04 Kafka Summit London: Stephan Ewen - "Apache Flink and Apache Kafka fo...
2018-04 Kafka Summit London: Stephan Ewen - "Apache Flink and Apache Kafka fo...
 
2018-01 Seattle Apache Flink Meetup at OfferUp, Opening Remarks and Talk 2
2018-01 Seattle Apache Flink Meetup at OfferUp, Opening Remarks and Talk 22018-01 Seattle Apache Flink Meetup at OfferUp, Opening Remarks and Talk 2
2018-01 Seattle Apache Flink Meetup at OfferUp, Opening Remarks and Talk 2
 
Stephan Ewen - Experiences running Flink at Very Large Scale
Stephan Ewen -  Experiences running Flink at Very Large ScaleStephan Ewen -  Experiences running Flink at Very Large Scale
Stephan Ewen - Experiences running Flink at Very Large Scale
 
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
 
Tzu-Li (Gordon) Tai - Stateful Stream Processing with Apache Flink
Tzu-Li (Gordon) Tai - Stateful Stream Processing with Apache FlinkTzu-Li (Gordon) Tai - Stateful Stream Processing with Apache Flink
Tzu-Li (Gordon) Tai - Stateful Stream Processing with Apache Flink
 
Kostas Kloudas - Complex Event Processing with Flink: the state of FlinkCEP
Kostas Kloudas - Complex Event Processing with Flink: the state of FlinkCEP Kostas Kloudas - Complex Event Processing with Flink: the state of FlinkCEP
Kostas Kloudas - Complex Event Processing with Flink: the state of FlinkCEP
 
Aljoscha Krettek - Portable stateful big data processing in Apache Beam
Aljoscha Krettek - Portable stateful big data processing in Apache BeamAljoscha Krettek - Portable stateful big data processing in Apache Beam
Aljoscha Krettek - Portable stateful big data processing in Apache Beam
 
Aljoscha Krettek - Apache Flink® and IoT: How Stateful Event-Time Processing ...
Aljoscha Krettek - Apache Flink® and IoT: How Stateful Event-Time Processing ...Aljoscha Krettek - Apache Flink® and IoT: How Stateful Event-Time Processing ...
Aljoscha Krettek - Apache Flink® and IoT: How Stateful Event-Time Processing ...
 
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
 
Kostas Tzoumas - Apache Flink®: State of the Union and What's Next
Kostas Tzoumas - Apache Flink®: State of the Union and What's NextKostas Tzoumas - Apache Flink®: State of the Union and What's Next
Kostas Tzoumas - Apache Flink®: State of the Union and What's Next
 

Recently uploaded

Escorts Service Kumaraswamy Layout ☎ 7737669865☎ Book Your One night Stand (B...
Escorts Service Kumaraswamy Layout ☎ 7737669865☎ Book Your One night Stand (B...Escorts Service Kumaraswamy Layout ☎ 7737669865☎ Book Your One night Stand (B...
Escorts Service Kumaraswamy Layout ☎ 7737669865☎ Book Your One night Stand (B...amitlee9823
 
➥🔝 7737669865 🔝▻ Sambalpur Call-girls in Women Seeking Men 🔝Sambalpur🔝 Esc...
➥🔝 7737669865 🔝▻ Sambalpur Call-girls in Women Seeking Men  🔝Sambalpur🔝   Esc...➥🔝 7737669865 🔝▻ Sambalpur Call-girls in Women Seeking Men  🔝Sambalpur🔝   Esc...
➥🔝 7737669865 🔝▻ Sambalpur Call-girls in Women Seeking Men 🔝Sambalpur🔝 Esc...amitlee9823
 
Just Call Vip call girls roorkee Escorts ☎️9352988975 Two shot with one girl ...
Just Call Vip call girls roorkee Escorts ☎️9352988975 Two shot with one girl ...Just Call Vip call girls roorkee Escorts ☎️9352988975 Two shot with one girl ...
Just Call Vip call girls roorkee Escorts ☎️9352988975 Two shot with one girl ...gajnagarg
 
Call Girls Jalahalli Just Call 👗 7737669865 👗 Top Class Call Girl Service Ban...
Call Girls Jalahalli Just Call 👗 7737669865 👗 Top Class Call Girl Service Ban...Call Girls Jalahalli Just Call 👗 7737669865 👗 Top Class Call Girl Service Ban...
Call Girls Jalahalli Just Call 👗 7737669865 👗 Top Class Call Girl Service Ban...amitlee9823
 
Just Call Vip call girls Mysore Escorts ☎️9352988975 Two shot with one girl (...
Just Call Vip call girls Mysore Escorts ☎️9352988975 Two shot with one girl (...Just Call Vip call girls Mysore Escorts ☎️9352988975 Two shot with one girl (...
Just Call Vip call girls Mysore Escorts ☎️9352988975 Two shot with one girl (...gajnagarg
 
Chintamani Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore ...
Chintamani Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore ...Chintamani Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore ...
Chintamani Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore ...amitlee9823
 
Call Girls Bommasandra Just Call 👗 7737669865 👗 Top Class Call Girl Service B...
Call Girls Bommasandra Just Call 👗 7737669865 👗 Top Class Call Girl Service B...Call Girls Bommasandra Just Call 👗 7737669865 👗 Top Class Call Girl Service B...
Call Girls Bommasandra Just Call 👗 7737669865 👗 Top Class Call Girl Service B...amitlee9823
 
Call Girls Indiranagar Just Call 👗 7737669865 👗 Top Class Call Girl Service B...
Call Girls Indiranagar Just Call 👗 7737669865 👗 Top Class Call Girl Service B...Call Girls Indiranagar Just Call 👗 7737669865 👗 Top Class Call Girl Service B...
Call Girls Indiranagar Just Call 👗 7737669865 👗 Top Class Call Girl Service B...amitlee9823
 
Just Call Vip call girls Erode Escorts ☎️9352988975 Two shot with one girl (E...
Just Call Vip call girls Erode Escorts ☎️9352988975 Two shot with one girl (E...Just Call Vip call girls Erode Escorts ☎️9352988975 Two shot with one girl (E...
Just Call Vip call girls Erode Escorts ☎️9352988975 Two shot with one girl (E...gajnagarg
 
Call Girls In Attibele ☎ 7737669865 🥵 Book Your One night Stand
Call Girls In Attibele ☎ 7737669865 🥵 Book Your One night StandCall Girls In Attibele ☎ 7737669865 🥵 Book Your One night Stand
Call Girls In Attibele ☎ 7737669865 🥵 Book Your One night Standamitlee9823
 
👉 Amritsar Call Girl 👉📞 6367187148 👉📞 Just📲 Call Ruhi Call Girl Phone No Amri...
👉 Amritsar Call Girl 👉📞 6367187148 👉📞 Just📲 Call Ruhi Call Girl Phone No Amri...👉 Amritsar Call Girl 👉📞 6367187148 👉📞 Just📲 Call Ruhi Call Girl Phone No Amri...
👉 Amritsar Call Girl 👉📞 6367187148 👉📞 Just📲 Call Ruhi Call Girl Phone No Amri...karishmasinghjnh
 
Just Call Vip call girls kakinada Escorts ☎️9352988975 Two shot with one girl...
Just Call Vip call girls kakinada Escorts ☎️9352988975 Two shot with one girl...Just Call Vip call girls kakinada Escorts ☎️9352988975 Two shot with one girl...
Just Call Vip call girls kakinada Escorts ☎️9352988975 Two shot with one girl...gajnagarg
 
Call Girls In Doddaballapur Road ☎ 7737669865 🥵 Book Your One night Stand
Call Girls In Doddaballapur Road ☎ 7737669865 🥵 Book Your One night StandCall Girls In Doddaballapur Road ☎ 7737669865 🥵 Book Your One night Stand
Call Girls In Doddaballapur Road ☎ 7737669865 🥵 Book Your One night Standamitlee9823
 
Call Girls In Shivaji Nagar ☎ 7737669865 🥵 Book Your One night Stand
Call Girls In Shivaji Nagar ☎ 7737669865 🥵 Book Your One night StandCall Girls In Shivaji Nagar ☎ 7737669865 🥵 Book Your One night Stand
Call Girls In Shivaji Nagar ☎ 7737669865 🥵 Book Your One night Standamitlee9823
 
Vip Mumbai Call Girls Thane West Call On 9920725232 With Body to body massage...
Vip Mumbai Call Girls Thane West Call On 9920725232 With Body to body massage...Vip Mumbai Call Girls Thane West Call On 9920725232 With Body to body massage...
Vip Mumbai Call Girls Thane West Call On 9920725232 With Body to body massage...amitlee9823
 
Call Girls Hsr Layout Just Call 👗 7737669865 👗 Top Class Call Girl Service Ba...
Call Girls Hsr Layout Just Call 👗 7737669865 👗 Top Class Call Girl Service Ba...Call Girls Hsr Layout Just Call 👗 7737669865 👗 Top Class Call Girl Service Ba...
Call Girls Hsr Layout Just Call 👗 7737669865 👗 Top Class Call Girl Service Ba...amitlee9823
 
➥🔝 7737669865 🔝▻ malwa Call-girls in Women Seeking Men 🔝malwa🔝 Escorts Ser...
➥🔝 7737669865 🔝▻ malwa Call-girls in Women Seeking Men  🔝malwa🔝   Escorts Ser...➥🔝 7737669865 🔝▻ malwa Call-girls in Women Seeking Men  🔝malwa🔝   Escorts Ser...
➥🔝 7737669865 🔝▻ malwa Call-girls in Women Seeking Men 🔝malwa🔝 Escorts Ser...amitlee9823
 
Call Girls In Nandini Layout ☎ 7737669865 🥵 Book Your One night Stand
Call Girls In Nandini Layout ☎ 7737669865 🥵 Book Your One night StandCall Girls In Nandini Layout ☎ 7737669865 🥵 Book Your One night Stand
Call Girls In Nandini Layout ☎ 7737669865 🥵 Book Your One night Standamitlee9823
 

Recently uploaded (20)

Escorts Service Kumaraswamy Layout ☎ 7737669865☎ Book Your One night Stand (B...
Escorts Service Kumaraswamy Layout ☎ 7737669865☎ Book Your One night Stand (B...Escorts Service Kumaraswamy Layout ☎ 7737669865☎ Book Your One night Stand (B...
Escorts Service Kumaraswamy Layout ☎ 7737669865☎ Book Your One night Stand (B...
 
➥🔝 7737669865 🔝▻ Sambalpur Call-girls in Women Seeking Men 🔝Sambalpur🔝 Esc...
➥🔝 7737669865 🔝▻ Sambalpur Call-girls in Women Seeking Men  🔝Sambalpur🔝   Esc...➥🔝 7737669865 🔝▻ Sambalpur Call-girls in Women Seeking Men  🔝Sambalpur🔝   Esc...
➥🔝 7737669865 🔝▻ Sambalpur Call-girls in Women Seeking Men 🔝Sambalpur🔝 Esc...
 
Just Call Vip call girls roorkee Escorts ☎️9352988975 Two shot with one girl ...
Just Call Vip call girls roorkee Escorts ☎️9352988975 Two shot with one girl ...Just Call Vip call girls roorkee Escorts ☎️9352988975 Two shot with one girl ...
Just Call Vip call girls roorkee Escorts ☎️9352988975 Two shot with one girl ...
 
Call Girls Jalahalli Just Call 👗 7737669865 👗 Top Class Call Girl Service Ban...
Call Girls Jalahalli Just Call 👗 7737669865 👗 Top Class Call Girl Service Ban...Call Girls Jalahalli Just Call 👗 7737669865 👗 Top Class Call Girl Service Ban...
Call Girls Jalahalli Just Call 👗 7737669865 👗 Top Class Call Girl Service Ban...
 
Just Call Vip call girls Mysore Escorts ☎️9352988975 Two shot with one girl (...
Just Call Vip call girls Mysore Escorts ☎️9352988975 Two shot with one girl (...Just Call Vip call girls Mysore Escorts ☎️9352988975 Two shot with one girl (...
Just Call Vip call girls Mysore Escorts ☎️9352988975 Two shot with one girl (...
 
Chintamani Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore ...
Chintamani Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore ...Chintamani Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore ...
Chintamani Call Girls: 🍓 7737669865 🍓 High Profile Model Escorts | Bangalore ...
 
Call Girls Bommasandra Just Call 👗 7737669865 👗 Top Class Call Girl Service B...
Call Girls Bommasandra Just Call 👗 7737669865 👗 Top Class Call Girl Service B...Call Girls Bommasandra Just Call 👗 7737669865 👗 Top Class Call Girl Service B...
Call Girls Bommasandra Just Call 👗 7737669865 👗 Top Class Call Girl Service B...
 
Call Girls Indiranagar Just Call 👗 7737669865 👗 Top Class Call Girl Service B...
Call Girls Indiranagar Just Call 👗 7737669865 👗 Top Class Call Girl Service B...Call Girls Indiranagar Just Call 👗 7737669865 👗 Top Class Call Girl Service B...
Call Girls Indiranagar Just Call 👗 7737669865 👗 Top Class Call Girl Service B...
 
Abortion pills in Doha Qatar (+966572737505 ! Get Cytotec
Abortion pills in Doha Qatar (+966572737505 ! Get CytotecAbortion pills in Doha Qatar (+966572737505 ! Get Cytotec
Abortion pills in Doha Qatar (+966572737505 ! Get Cytotec
 
Just Call Vip call girls Erode Escorts ☎️9352988975 Two shot with one girl (E...
Just Call Vip call girls Erode Escorts ☎️9352988975 Two shot with one girl (E...Just Call Vip call girls Erode Escorts ☎️9352988975 Two shot with one girl (E...
Just Call Vip call girls Erode Escorts ☎️9352988975 Two shot with one girl (E...
 
Call Girls In Attibele ☎ 7737669865 🥵 Book Your One night Stand
Call Girls In Attibele ☎ 7737669865 🥵 Book Your One night StandCall Girls In Attibele ☎ 7737669865 🥵 Book Your One night Stand
Call Girls In Attibele ☎ 7737669865 🥵 Book Your One night Stand
 
👉 Amritsar Call Girl 👉📞 6367187148 👉📞 Just📲 Call Ruhi Call Girl Phone No Amri...
👉 Amritsar Call Girl 👉📞 6367187148 👉📞 Just📲 Call Ruhi Call Girl Phone No Amri...👉 Amritsar Call Girl 👉📞 6367187148 👉📞 Just📲 Call Ruhi Call Girl Phone No Amri...
👉 Amritsar Call Girl 👉📞 6367187148 👉📞 Just📲 Call Ruhi Call Girl Phone No Amri...
 
Just Call Vip call girls kakinada Escorts ☎️9352988975 Two shot with one girl...
Just Call Vip call girls kakinada Escorts ☎️9352988975 Two shot with one girl...Just Call Vip call girls kakinada Escorts ☎️9352988975 Two shot with one girl...
Just Call Vip call girls kakinada Escorts ☎️9352988975 Two shot with one girl...
 
Call Girls In Doddaballapur Road ☎ 7737669865 🥵 Book Your One night Stand
Call Girls In Doddaballapur Road ☎ 7737669865 🥵 Book Your One night StandCall Girls In Doddaballapur Road ☎ 7737669865 🥵 Book Your One night Stand
Call Girls In Doddaballapur Road ☎ 7737669865 🥵 Book Your One night Stand
 
Call Girls In Shivaji Nagar ☎ 7737669865 🥵 Book Your One night Stand
Call Girls In Shivaji Nagar ☎ 7737669865 🥵 Book Your One night StandCall Girls In Shivaji Nagar ☎ 7737669865 🥵 Book Your One night Stand
Call Girls In Shivaji Nagar ☎ 7737669865 🥵 Book Your One night Stand
 
Vip Mumbai Call Girls Thane West Call On 9920725232 With Body to body massage...
Vip Mumbai Call Girls Thane West Call On 9920725232 With Body to body massage...Vip Mumbai Call Girls Thane West Call On 9920725232 With Body to body massage...
Vip Mumbai Call Girls Thane West Call On 9920725232 With Body to body massage...
 
Call Girls In Shalimar Bagh ( Delhi) 9953330565 Escorts Service
Call Girls In Shalimar Bagh ( Delhi) 9953330565 Escorts ServiceCall Girls In Shalimar Bagh ( Delhi) 9953330565 Escorts Service
Call Girls In Shalimar Bagh ( Delhi) 9953330565 Escorts Service
 
Call Girls Hsr Layout Just Call 👗 7737669865 👗 Top Class Call Girl Service Ba...
Call Girls Hsr Layout Just Call 👗 7737669865 👗 Top Class Call Girl Service Ba...Call Girls Hsr Layout Just Call 👗 7737669865 👗 Top Class Call Girl Service Ba...
Call Girls Hsr Layout Just Call 👗 7737669865 👗 Top Class Call Girl Service Ba...
 
➥🔝 7737669865 🔝▻ malwa Call-girls in Women Seeking Men 🔝malwa🔝 Escorts Ser...
➥🔝 7737669865 🔝▻ malwa Call-girls in Women Seeking Men  🔝malwa🔝   Escorts Ser...➥🔝 7737669865 🔝▻ malwa Call-girls in Women Seeking Men  🔝malwa🔝   Escorts Ser...
➥🔝 7737669865 🔝▻ malwa Call-girls in Women Seeking Men 🔝malwa🔝 Escorts Ser...
 
Call Girls In Nandini Layout ☎ 7737669865 🥵 Book Your One night Stand
Call Girls In Nandini Layout ☎ 7737669865 🥵 Book Your One night StandCall Girls In Nandini Layout ☎ 7737669865 🥵 Book Your One night Stand
Call Girls In Nandini Layout ☎ 7737669865 🥵 Book Your One night Stand
 

Kostas Kloudas - Extending Flink's Streaming APIs

  • 2. 2 Original creators of Apache Flink® Providers of the dA Platform, a supported Flink distribution
  • 4. Additions in Flink 1.2 4  Re-scalable State  Low-level Stream Operations  Asynchronous I/O  Table API and SQL  Externalized Checkpoints  Queryable State  Mesos Integration  …and, of course, Documentation
  • 5. Additions in Flink 1.2 5  Re-scalable State  Low-level Stream Operations  Asynchronous I/O  Table API and SQL  Externalized Checkpoints  Queryable State  Mesos Integration  …and, of course, Documentation
  • 7. Common Usecase Skeleton A  On each incoming element: • update some state • register a callback for a moment in the future  When that moment comes: • Check a condition and perform a certain action, e.g. emit an element 7
  • 8.  Use built-in windowing: • +Expressive • +A lot of functionality out-of-the-box • - Not always intuitive • - An overkill for simple cases  Write your own operator: • - Too many things to account for in Flink 1.1 8 The Flink 1.1 way
  • 9. The Flink 1.2 way: ProcessFunction  Gives access to all basic building blocks: • Events • Fault-tolerant, Consistent State • Timers (event- and processing-time) 9
  • 10. The Flink 1.2 way: ProcessFunction  Simple yet powerful API: 10 /** * Process one element from the input stream. */ void processElement(I value, Context ctx, Collector<O> out) throws Exception; /** * Called when a timer set using {@link TimerService} fires. */ void onTimer(long timestamp, OnTimerContext ctx, Collector<O> out) throws Exception;
  • 11. The Flink 1.2 way: ProcessFunction  Simple yet powerful API: 11 /** * Process one element from the input stream. */ void processElement(I value, Context ctx, Collector<O> out) throws Exception; /** * Called when a timer set using {@link TimerService} fires. */ void onTimer(long timestamp, OnTimerContext ctx, Collector<O> out) throws Exception; A collector to emit result values
  • 12. The Flink 1.2 way: ProcessFunction  Simple yet powerful API: 12 /** * Process one element from the input stream. */ void processElement(I value, Context ctx, Collector<O> out) throws Exception; /** * Called when a timer set using {@link TimerService} fires. */ void onTimer(long timestamp, OnTimerContext ctx, Collector<O> out) throws Exception; 1. Get the timestamp of the element 2. Interact with the TimerService to: • query the current time • and register timers 1. Do the above 2. Query if we are operating on Event or Processing time
  • 13. ProcessFunction: example  Requirements: • maintain counts per incoming key, and • emit the key/count pair if no element came for the key in the last 100 ms (in event time) 13
  • 14. ProcessFunction: example 14  Implementation sketch: • Store the count, key and last mod timestamp in a ValueState (scoped by key) • For each record: • update the counter and the last mod timestamp • register a timer 100ms from “now” (in event time) • When the timer fires: • check the callback’s timestamp against the last mod time for the key and • emit the key/count pair if they match
  • 15. ProcessFunction: example 15 public class MyProcessFunction extends RichProcessFunction<Tuple2<String, String>, Tuple2<String, Long>> { @Override public void open(Configuration parameters) throws Exception { // register our state with the state backend } @Override public void processElement(Tuple2<String, Long> value, Context ctx, Collector<Tuple2<String, Long>> out) throws Exception { // update our state and register a timer } @Override public void onTimer(long timestamp, OnTimerContext ctx, Collector<Tuple2<String, Long>> out) throws Exception { // check the state for the key and emit a result if needed } }
  • 16. ProcessFunction: example 16 public class MyProcessFunction extends RichProcessFunction<Tuple2<String, String>, Tuple2<String, Long>> { private ValueState<MyStateClass> state; @Override public void open(Configuration parameters) throws Exception { state = getRuntimeContext().getState( new ValueStateDescriptor<>("myState", MyStateClass.class)); } }
  • 17. ProcessFunction: example 17 public class MyProcessFunction extends RichProcessFunction<Tuple2<String, String>, Tuple2<String, Long>> { @Override public void processElement(Tuple2<String, Long> value, Context ctx, Collector<Tuple2<String, Long>> out) throws Exception { CountWithTimestamp current = state.value(); if (current == null) { current = new CountWithTimestamp(); current.key = value.f0; } current.count++; current.lastModified = ctx.timestamp(); state.update(current); ctx.timerService().registerEventTimeTimer(current.timestamp + 100); } }
  • 18. ProcessFunction: example 18 public class MyProcessFunction extends RichProcessFunction<Tuple2<String, String>, Tuple2<String, Long>> { @Override public void onTimer(long timestamp, OnTimerContext ctx, Collector<Tuple2<String, Long>> out) throws Exception { CountWithTimestamp result = state.value(); if (timestamp == result.lastModified) { out.collect(new Tuple2<String, Long>(result.key, result.count)); } } }
  • 19. ProcessFunction: example 19  If your stream is not keyed, you can always group on a dummy key  BEWARE: parallelism of 1 stream.keyBy("id") .process(new MyProcessFunction())
  • 20. ProcessFunction: miscellaneous 20  CoProcessFunction for low-level joins: • Applied on two input streams • Has two processElement() methods, one for each input stream  Upcoming releases may further enhance the ProcessFunction/CoProcessFunction  Planning to transform all CEP operators to ProcessFunctions
  • 22. Common Usecase Skeleton B 22  On each incoming element: • extract some info from the element (e.g. key) • query an external storage system (DB or KV- store) for additional info • emit an enriched version of the input element
  • 23.  Write a MapFunction that queries the DB: • +Simple • - Slow (synchronous access) or/and • - Requires high parallelism (more tasks)  Write your own operator: • - Too many things to account for in Flink 1.1 23 The Flink 1.1 way
  • 24.  Write a MapFunction that queries the DB: • +Simple • - Slow (synchronous access) or/and • - Requires high parallelism (more tasks)  Write your own operator: • - Too many things to account for in Flink 1.1 24 The Flink 1.1 way
  • 26. 26 Synchronous Access Communication delay can dominate application throughput and latency
  • 28.  Requirement: • a client that supports asynchronous requests  Flink handles the rest: • integration of async IO with DataStream API • fault-tolerance • order of emitted elements • correct time semantics (event/processing time) 28 The Flink 1.2 way: AsyncFunction
  • 29.  Simple API: /** * Trigger async operation for each stream input. */ void asyncInvoke(IN input, AsyncCollector<OUT> collector) throws Exception;  API call: /** * Example async function call. */ DataStream<...> result = AsyncDataStream.(un)orderedWait(stream, new MyAsyncFunction(), 1000, TimeUnit.MILLISECONDS, 100); 29 The Flink 1.2 way: AsyncFunction
  • 30. The Flink 1.2 way: AsyncFunction 30 Emitter P2P3 P1P4 AsyncWaitOperator E5 AsyncWaitOperator: • a queue of “Promises” • a separate thread (Emitter)
  • 31. The Flink 1.2 way: AsyncFunction 31 Emitter P2P3 P1P4 AsyncWaitOperator • Wrap E5 in a “promise” P5 • Put P5 in the queue • Call asyncInvoke(E5, P5) E5 P5 asyncInvoke(E5, P5)P5
  • 32. The Flink 1.2 way: AsyncFunction 32 Emitter P2P3 P1P4 AsyncWaitOperator E5 P5 asyncInvoke(E5, P5)P5 asyncInvoke(value, asyncCollector): • a user-defined function • value : the input element • asyncCollector : the collector of the result (when the query returns)
  • 33. The Flink 1.2 way: AsyncFunction 33 Emitter P2P3 P1P4 AsyncWaitOperator E5 P5 asyncInvoke(E5, P5)P5 asyncInvoke(value, asyncCollector): • a user-defined function • value : the input element • asyncCollector : the collector of the result (when the query returns) Future<String> future = client.query(E5); future.thenAccept((String result) -> { P5.collect( Collections.singleton( new Tuple2<>(E5, result))); });
  • 34. The Flink 1.2 way: AsyncFunction 34 Emitter P2P3 P1P4 AsyncWaitOperator E5 P5 asyncInvoke(E5, P5)P5 asyncInvoke(value, asyncCollector): • a user-defined function • value : the input element • asyncCollector : the collector of the result (when the query returns) Future<String> future = client.query(E5); future.thenAccept((String result) -> { P5.collect( Collections.singleton( new Tuple2<>(E5, result))); });
  • 35. The Flink 1.2 way: AsyncFunction 35 Emitter P2P3 P1P4 AsyncWaitOperator E5 P5 asyncInvoke(E5, P5)P5 Emitter: • separate thread • polls queue for completed promises (blocking) • emits elements downstream
  • 36. 36 The Flink 1.2 way: AsyncFunction DataStream<Tuple2<String, String>> result = AsyncDataStream.(un)orderedWait(stream, new MyAsyncFunction(), 1000, TimeUnit.MILLISECONDS, 100);  our asyncFunction  a timeout: max time until considered failed  capacity: max number of in-flight requests
  • 37. 37 The Flink 1.2 way: AsyncFunction DataStream<Tuple2<String, String>> result = AsyncDataStream.(un)orderedWait(stream, new MyAsyncFunction(), 1000, TimeUnit.MILLISECONDS, 100);
  • 38. 38 The Flink 1.2 way: AsyncFunction DataStream<Tuple2<String, String>> result = AsyncDataStream.(un)orderedWait(stream, new MyAsyncFunction(), 1000, TimeUnit.MILLISECONDS, 100); P2P3 P1P4E2E3 E1E4 Ideally... Emitter
  • 39. 39 The Flink 1.2 way: AsyncFunction DataStream<Tuple2<String, String>> result = AsyncDataStream.unorderedWait(stream, new MyAsyncFunction(), 1000, TimeUnit.MILLISECONDS, 100); P2P3 P1P4E2E3 E1E4 Reallistically... Emitter ...output ordered based on which request finished first
  • 40. 40 The Flink 1.2 way: AsyncFunction P2P3 P1P4E2E3 E1E4 Emitter  unorderedWait: emit results in order of completion  orderedWait: emit results in order of arrival  Always: watermarks never overpass elements and vice versa
  • 43. 43 One day of hands-on Flink training One day of conference Tickets are on sale Call for Papers is already open Please visit our website: http://sf.flink-forward.org Follow us on Twitter: @FlinkForward

Editor's Notes

  1. My name is Kostas Kloudas and I am here to talk to you about some of the latest extensions of Flink’s streaming APIs. I bit about me, I am a Flink committer and a software engineer at data Artisans...
  2. So, enough with the introductions, let’s cut to the chase. As the streaming space and the Flink community grow, Flink grows with them. This has led to a number of cool new features being added in Flink 1.2. These features range from: engine enhancements to support better performance and features like rescaling and low-level abstractions that allow for easier interaction with Flink’s the low level mechanisms Higher level API enhancements that make the implementation of common cases easier
  3. In this talk I will focus on the two highlighted features, namely: ....
  4. An example could be that you have your recommendation system, and users that navigate from an item to its related/recommended ones. In this case, and to adjust your recommendation algorithm, you can have a “rule” that says if the user does not purchase the related clicked item within X sec, send a signal to the recommendation system that the recommendation was not good For those of you familiar with the Flink APIs, you can imagine this as a flatMap with the ability to register and react to timers.
  5. Not always intuitive: You do not want to think about assigners, triggers, functions etc when all you want to do is as simple as a flatmap with a timer For 2nd: but if windowing is an overkill, imagine a custom operator.
  6. So, that was in Flink 1.1 and these remain valid approaches also in Flink 1.2. But, to make things easier, Flink 1.2 ships with a new abstraction called the ProcessFunction, that was introduced to cover precisely these cases. The ProcessFunction is a low-level stream processing operation, which gives access to the basic building blocks of all (acyclic) streaming applications: events (stream elements) state (fault tolerant, consistent) timers (event time and processing time) Again, you can imagine it as a flatmap with access to state and timers.
  7. Focusing on the arguments of each of the calls:...
  8. Emphasize that time stands for both event and processing time.
  9. This example is copied from our documentation for which I will provide a link at the end of the slides (but you can always use your favorite search engine to look for ProcessFunction in Flink)
  10. ... Following the same pattern as before...
  11. Let’s focus a bit on the “synchronous access” part and see what this stands for.
  12. As shown in the figure, synchronous access means that after sending a request for key a, you have to wait for the response, before being able to send the next request for key b. In the figure, with brown we show the waiting time, and we can see that this can easily dominate throughput and latency.
  13. Let’s focus a bit on the “synchronous access” part and see what this stands for. As shown in the figure, synchronous access means that after sending a request for key a, you have to wait for the response, before being able to send the next request for key b. In the figure, with brown we show the waiting time, and we can see that this can easily dominate throughput and latency.
  14. To face the problems of synchronous access, the asynchronous pattern allows for multiplexing requests and responses so that you send a request for a, b, c, etc and, in the same time, you receive the responses as they arrive, without waiting between consecutive requests. This is exactly the pattern that AsyncIO implements. And in order to leverage its capabilities, the only requirement it imposes is:
  15. If you have this, then Flink will provide the rest, such as...
  16. The API of the async function requires the implementation of a single method ... Which is the one that triggers an async operation for each input element. And to integrate it into your program, you will have to write something like the following: We will see more about the details of these methods in the following slides. So now that we have the 10000 feet view of the async io, let’s see a little bit how this works:
  17. This is the diagram of our AsyncWaitOperator, the operator that runs our asyncFunction. As we can see, it is composed of a queue of ”Promises” and a separate Thread, the “Emitter”, which is responsible for sending Elements (e.g. the received responses) downstream. A ”promise” is an asynchronous abstraction which “promises” to have a value in the future. This queue is the queue of PENDING promises, e.g. our pending requests.
  18. A ”promise” is an asynchronous abstraction which “promises” to have a value in the future. On this promise, we can attach a callback, which will be triggered upon completion of the requested action, i.e. When the promise has a concrete value (or completes with an exception)
  19. CLIENT should be asynchronous. If not, then the call will block in the query() and we will have the same synchronous pattern as before.
  20. CLIENT should be asynchronous. If not, then the call will block in the query() and we will have the same synchronous pattern as before.
  21. A ”promise” is an asynchronous abstraction which “promises” to have a value in the future. On this promise, we can attach a callback, which will be triggered upon completion of the requested action, i.e. When the promise has a concrete value (or completes with an exception)
  22. Let’s focus a bit on the “synchronous access” part and see what this stands for...
  23. As operations are served asynchronously, the order of the output elements will not be the same as the one of their respective input elements. This in fact depends on how fast the storage system serves each of the individual requests. To control the order of the emitted events, Flink can operate on 2 modes:
  24. As operations are served asynchronously, the order of the output elements will not be the same as the one of their respective input elements. This in fact depends on how fast the storage system serves each of the individual requests. To control the order of the emitted events, Flink can operate on 2 modes:
  25. As operations are served asynchronously, the order of the output elements will not be the same as the one of their respective input elements. This in fact depends on how fast the storage system serves each of the individual requests. To control the order of the emitted events, Flink can operate on 2 modes: