SlideShare a Scribd company logo
1 of 37
Apache Flink® Training
DataStream API Advanced
August 26, 2015
Type System and Keys
What kind of data can Flink handle?
2
Note: Identical to
DataSet API
Apache Flink’s Type System
 Flink aims to support all data types
• Ease of programming
• Seamless integration with existing code
 Programs are analyzed before execution
• Used data types are identified
• Serializer & comparator are configured
3
Apache Flink’s Type System
 Data types are either
• Atomic types (like Java Primitives)
• Composite types (like Flink Tuples)
 Composite types nest other types
 Not all data types can be used as keys!
• Flink partitions DataStreams on keys
• Key types must be comparable
4
Atomic Types
Flink Type Java Type Can be used as key?
BasicType Java Primitives
(Integer, String, …)
Yes
ArrayType Arrays of Java primitives
or objects
No (Yes as of 0.10)
WritableType Implements Hadoop’s
Writable interface
Yes, if implements
WritableComparable
GenericType Any other type Yes, if implements
Comparable
5
Composite Types
 Are composed of fields with other types
• Fields types can be atomic or composite
 Fields can be addressed as keys
• Field type must be a key type!
 A composite type can be a key type
• All field types must be key types!
6
TupleType
 Java:
org.apache.flink.api.java.tuple.Tuple1 to Tuple25
 Scala:
use default Scala tuples (1 to 22 fields)
 Tuple fields are typed
Tuple3<Integer, String, Double> t3 =
new Tuple3<>(1, “2”, 3.0);
val t3: (Int, String, Double) = (1, ”2”, 3.0)
 Tuples give the best performance
7
TupleType
 Define keys by field position
DataStream<Tuple3<Integer, String, Double>> d = …
// group on String field
d.groupBy(1);
 Or field names
// group on Double field
d.groupBy(“f2”);
8
PojoType
 Any Java class that
• Has an empty default constructor
• Has publicly accessible fields
(public field or default getter & setter)
public class Person {
public int id;
public String name;
public Person() {};
public Person(int id, String name) {…};
}
DataStream<Person> p =
env.fromElements(new Person(1, ”Bob”)); 9
PojoType
 Define keys by field name
DataStream<Person> p = …
// group on “name” field
d.groupBy(“name”);
10
Scala CaseClasses
 Scala case classes are natively supported
case class Person(id: Int, name: String)
d: DataStream[Person] =
env.fromElements(Person(1, “Bob”)
 Define keys by field name
// use field “name” as key
d.groupBy(“name”)
11
Composite & Nested Keys
DataStream<Tuple3<String, Person, Double>> d;
 Composite keys are supported
// group on both long fields
d.groupBy(0, 1);
 Nested fields can be used as types
// group on nested “name” field
d.groupBy(“f1.name”);
 Full types can be used as key using “*” wildcard
// group on complete nested Pojo field
d.groupBy(“f1.*”);
• “*” wildcard can also be used for atomic types 12
KeySelectors
 Keys can be computed using KeySelectors
public class SumKeySelector implements
KeySelector<Tuple2<Long, Long>, Long> {
public Long getKey(Tuple2<Long, Long> t) {
return t.f0 + t.f1;
}}
DataStream<Tuple2<Long,Long>> d = …
d.groupBy(new SumKeySelector());
13
Windows and Aggregates
14
Windows
 Aggregations on DataStreams are different
from aggregations on DataSets
• e.g., it is not possible to count all elements of a
DataStream – they are infinite
 DataStream aggregations make sense on
windowed streams
• i.e., a window of the "latest" elements of a stream
 Windows can be defined on grouped and
partitioned streams
15
Windows (2)
16
// (name, age) of passengers
DataStream<Tuple2<String, Integer>> passengers = …
// group by second field (age) and keep last 1 minute
// worth of data sliding the window every 10 seconds
passengers
.groupBy(1)
.window(Time.of(1, TimeUnit.MINUTES))
.every(Time.of(10, TimeUnit.SECONDS))
Types of Windows
 Tumbling time window
• .window(Time.of(1, TimeUnit.MINUTES))
 Sliding time window
• .window(Time.of(60, TimeUnit.SECONDS))
.every(Time.of(10, TimeUnit.SECONDS))
 Count-based sliding window
• .window(Count.of(1000))
.every(Count.of(10))
17
Aggregations on Windowed Streams
18
// (name, age) of passengers
DataStream<Tuple2<String, Integer>> passengers = …
// group by second field (age) and keep last 1 minute
// worth of data sliding the window every 10 seconds
// count passengers
employees
.groupBy(1)
.window(Time.of(1, TimeUnit.MINUTES))
.every(Time.of(10, TimeUnit.SECONDS))
.mapWindow(new CountSameAge());
Warning: 0.9 -> 0.10
mapWindow becomes apply
MapWindow
public static class CountSameAge implements WindowMapFunction<Tuple2<String,
Integer>, Tuple2<Integer, Integer>> {
@Override
public void mapWindow(Iterable<Tuple2<String, Integer>> persons,
Collector<Tuple2<Integer, Integer>> out) {
Integer ageGroup = 0;
Integer countsInGroup = 0;
for (Tuple2<String, Integer> person : persons) {
ageGroup = person.f1;
countsInGroup++;
}
out.collect(new Tuple2<>(ageGroup, countsInGroup));
}
}
19
Warning: 0.9 -> 0.10
WindowMapFunction becomes WindowFunction
Operations on Windowed Streams
 mapWindow
• Do something over the whole window
 reduceWindow
• Apply a functional reduce function to the
window
 Aggregates: sum, min, max, and others
 flatten
• Get back a regular DataStream
20
Warning: 0.9 -> 0.10
mapWindow becomes apply
reduceWindow becomes reduce
Working With Multiple Streams
21
Connecting Streams
 Sometimes several DataStreams need to
be correlated with each other and share
state
 You can connect or join two DataStreams
22
DataStream<String> strings = …
DataStream<Integer> ints = …
// Create a ConnectedDataStream
strings.connect(ints);
Map on Connected Streams
23
DataStream<String> strings = …
DataStream<Integer> ints = …
// Create a ConnectedDataStream
strings.connect(ints)
.map(new CoMapFunction<Integer,String,Boolean> {
@Override
public Boolean map1 (Integer value) {
return true;
}
@Override
public Boolean map2 (String value) {
return false;
}
});
FlatMap on Connected Streams
24
DataStream<String> strings = …
DataStream<Integer> ints = …
// Create a ConnectedDataStream
strings.connect(ints)
.flatMap(new CoFlatMapFunction<Integer,String,String> {
@Override
public void flatMap1 (Integer value, Collector<String> out) {
out.collect(value.toString());
}
@Override
public void flatMap2 (String value, Collector<String> out) {
for (String word: value.split(" ")) {
out.collect(word)
}
}
});
Rich functions and state
25
RichFunctions
 Function interfaces have only one method
• Single abstract method (SAM)
• Support for Java8 Lambda functions
 There is a “Rich” variant for each function.
• RichFlatMapFunction, …
• Additional methods
• open(Configuration c)
• close()
• getRuntimeContext()
26
RichFunctions & RuntimeContext
 RuntimeContext has useful methods:
• getIndexOfThisSubtask ()
• getNumberOfParallelSubtasks()
• getExecutionConfig()
 Give access to partitioned state
flink.apache.org 27
Stateful Computations
 All DataStream transformations can be stateful
• State is mutable and lives as long as the streaming
job is running
• State is recovered with exactly-once semantics by
Flink after a failure
 You can define two kinds of state
• Local state: each parallel task can register some local
variables to take part in Flink’s checkpointing
• Partitioned by key state: an operator on a partitioned
by key stream can access and update state
corresponding to its key
• Partitioned state will be available in Flink 0.10
28
Defining Local State
29
DataSet<String> aStream;
DataStream<Long> lengths = aStream.map (new MapWithCounter());
public static class MapWithCounter implements MapFunction<String,Long>, Checkpointed<Long> {
private long totalLength = 0;
@Override
public Long map (String value) {
totalLength += value.length();8
return (Long) value.length();
}
@Override
public Serializable snapshotState(
long checkpointId,
long checkpointTimestamp) throws Exception {
return new Long (totalLength);
}
@Override
public void restoreState (Serializable state) throws Exception {
totalLength = (Long) state;
}
}
Defining Partitioned State
30
DataSet<Tuple2<String,String>> aStream;
DataStream<Long> lengths = aStream.groupBy(0).map (new MapWithCounter());
public static class MapWithCounter implements RichMapFunction<Tuple2<String,String>,Long> {
private OperatorState<Long> totalLengthByKey;
@Override
public Long map (Tuple2<String,String> value) {
totalLengthByKey.update(totalLengthByKey.update.value() + 1);
return (Long) value.f1.length();
}
@Override
public void open (Configuration conf) {
totalLengthByKey = getRuntimeContext()
.getOperatorState("totalLengthByKey", 0L, true);
}
}
Note: Will be available
in Flink 0.10
Connecting to Apache Kafka
31
Kafka and Flink
 “Apache Kafka is a distributed, partitioned, replicated
commit log service”
 Kafka uses Apache Zookeeper for coordination
 Kafka maintains feeds of messages in categories
called topics
 A Kafka topic can be read by Flink to produce a
DataStream, and a DataStream can be written to a
Kafka topic
 Flink coordinates with Kafka to provide recovery in the
case of failures
32
Reading Data from Kafka
 Enable checkpointing
E.g., env.enableCheckpointing(5000);
 Add a DataStream source from a Kafka topic
Properties props = new Properties();
props.setProperty("zookeeper.connect", “localhost:2181”);
props.setProperty("bootstrap.servers", “localhost:9092”);
props.setProperty("group.id", “myGroup”);
// create a data source
DataStream<TaxiRide> rides = env.addSource(
new FlinkKafkaConsumer082<TaxiRide>(
“myTopic”,
new TaxiRideSchema(),
props)
);
33
Writing Data to Kafka
 Add a Kafka sink to a DataStream by
providing
• The broker address
• The topic name
• A serialization schema
DataStream<String> aStream = …
aStream.addSink(
new KafkaSink<String>(
“localhost:9092”, // default local broker
“myTopic”,
new SimpleStringSchema));
34
More API Features
35
Not Covered Here
 Iterations (feedback edges)
• Very useful for Machine Learning
 More transformations
• union, join, ...
36
37

More Related Content

What's hot

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
 
Introduction To Flink
Introduction To FlinkIntroduction To Flink
Introduction To FlinkKnoldus Inc.
 
Using the New Apache Flink Kubernetes Operator in a Production Deployment
Using the New Apache Flink Kubernetes Operator in a Production DeploymentUsing the New Apache Flink Kubernetes Operator in a Production Deployment
Using the New Apache Flink Kubernetes Operator in a Production DeploymentFlink Forward
 
Apache Flink internals
Apache Flink internalsApache Flink internals
Apache Flink internalsKostas Tzoumas
 
Real-time Stream Processing with Apache Flink
Real-time Stream Processing with Apache FlinkReal-time Stream Processing with Apache Flink
Real-time Stream Processing with Apache FlinkDataWorks Summit
 
Stream Processing Frameworks
Stream Processing FrameworksStream Processing Frameworks
Stream Processing FrameworksSirKetchup
 
Apache kafka 모니터링을 위한 Metrics 이해 및 최적화 방안
Apache kafka 모니터링을 위한 Metrics 이해 및 최적화 방안Apache kafka 모니터링을 위한 Metrics 이해 및 최적화 방안
Apache kafka 모니터링을 위한 Metrics 이해 및 최적화 방안SANG WON PARK
 
Apache Flink Stream Processing
Apache Flink Stream ProcessingApache Flink Stream Processing
Apache Flink Stream ProcessingSuneel Marthi
 
Performance Optimizations in Apache Impala
Performance Optimizations in Apache ImpalaPerformance Optimizations in Apache Impala
Performance Optimizations in Apache ImpalaCloudera, Inc.
 
Best practices and lessons learnt from Running Apache NiFi at Renault
Best practices and lessons learnt from Running Apache NiFi at RenaultBest practices and lessons learnt from Running Apache NiFi at Renault
Best practices and lessons learnt from Running Apache NiFi at RenaultDataWorks Summit
 
Autoscaling Flink with Reactive Mode
Autoscaling Flink with Reactive ModeAutoscaling Flink with Reactive Mode
Autoscaling Flink with Reactive ModeFlink Forward
 
Stephan Ewen - Scaling to large State
Stephan Ewen - Scaling to large StateStephan Ewen - Scaling to large State
Stephan Ewen - Scaling to large StateFlink Forward
 
Apache Flink @ NYC Flink Meetup
Apache Flink @ NYC Flink MeetupApache Flink @ NYC Flink Meetup
Apache Flink @ NYC Flink MeetupStephan Ewen
 
Introduction and Overview of Apache Kafka, TriHUG July 23, 2013
Introduction and Overview of Apache Kafka, TriHUG July 23, 2013Introduction and Overview of Apache Kafka, TriHUG July 23, 2013
Introduction and Overview of Apache Kafka, TriHUG July 23, 2013mumrah
 
Introduction to Apache NiFi dws19 DWS - DC 2019
Introduction to Apache NiFi   dws19 DWS - DC 2019Introduction to Apache NiFi   dws19 DWS - DC 2019
Introduction to Apache NiFi dws19 DWS - DC 2019Timothy Spann
 
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 SparkDatabricks
 
Hive ICDE 2010
Hive ICDE 2010Hive ICDE 2010
Hive ICDE 2010ragho
 
XStream: stream processing platform at facebook
XStream:  stream processing platform at facebookXStream:  stream processing platform at facebook
XStream: stream processing platform at facebookAniket Mokashi
 
Processing Semantically-Ordered Streams in Financial Services
Processing Semantically-Ordered Streams in Financial ServicesProcessing Semantically-Ordered Streams in Financial Services
Processing Semantically-Ordered Streams in Financial ServicesFlink Forward
 
Reactive Programming for a demanding world: building event-driven and respons...
Reactive Programming for a demanding world: building event-driven and respons...Reactive Programming for a demanding world: building event-driven and respons...
Reactive Programming for a demanding world: building event-driven and respons...Mario Fusco
 

What's hot (20)

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
 
Introduction To Flink
Introduction To FlinkIntroduction To Flink
Introduction To Flink
 
Using the New Apache Flink Kubernetes Operator in a Production Deployment
Using the New Apache Flink Kubernetes Operator in a Production DeploymentUsing the New Apache Flink Kubernetes Operator in a Production Deployment
Using the New Apache Flink Kubernetes Operator in a Production Deployment
 
Apache Flink internals
Apache Flink internalsApache Flink internals
Apache Flink internals
 
Real-time Stream Processing with Apache Flink
Real-time Stream Processing with Apache FlinkReal-time Stream Processing with Apache Flink
Real-time Stream Processing with Apache Flink
 
Stream Processing Frameworks
Stream Processing FrameworksStream Processing Frameworks
Stream Processing Frameworks
 
Apache kafka 모니터링을 위한 Metrics 이해 및 최적화 방안
Apache kafka 모니터링을 위한 Metrics 이해 및 최적화 방안Apache kafka 모니터링을 위한 Metrics 이해 및 최적화 방안
Apache kafka 모니터링을 위한 Metrics 이해 및 최적화 방안
 
Apache Flink Stream Processing
Apache Flink Stream ProcessingApache Flink Stream Processing
Apache Flink Stream Processing
 
Performance Optimizations in Apache Impala
Performance Optimizations in Apache ImpalaPerformance Optimizations in Apache Impala
Performance Optimizations in Apache Impala
 
Best practices and lessons learnt from Running Apache NiFi at Renault
Best practices and lessons learnt from Running Apache NiFi at RenaultBest practices and lessons learnt from Running Apache NiFi at Renault
Best practices and lessons learnt from Running Apache NiFi at Renault
 
Autoscaling Flink with Reactive Mode
Autoscaling Flink with Reactive ModeAutoscaling Flink with Reactive Mode
Autoscaling Flink with Reactive Mode
 
Stephan Ewen - Scaling to large State
Stephan Ewen - Scaling to large StateStephan Ewen - Scaling to large State
Stephan Ewen - Scaling to large State
 
Apache Flink @ NYC Flink Meetup
Apache Flink @ NYC Flink MeetupApache Flink @ NYC Flink Meetup
Apache Flink @ NYC Flink Meetup
 
Introduction and Overview of Apache Kafka, TriHUG July 23, 2013
Introduction and Overview of Apache Kafka, TriHUG July 23, 2013Introduction and Overview of Apache Kafka, TriHUG July 23, 2013
Introduction and Overview of Apache Kafka, TriHUG July 23, 2013
 
Introduction to Apache NiFi dws19 DWS - DC 2019
Introduction to Apache NiFi   dws19 DWS - DC 2019Introduction to Apache NiFi   dws19 DWS - DC 2019
Introduction to Apache NiFi dws19 DWS - DC 2019
 
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
 
Hive ICDE 2010
Hive ICDE 2010Hive ICDE 2010
Hive ICDE 2010
 
XStream: stream processing platform at facebook
XStream:  stream processing platform at facebookXStream:  stream processing platform at facebook
XStream: stream processing platform at facebook
 
Processing Semantically-Ordered Streams in Financial Services
Processing Semantically-Ordered Streams in Financial ServicesProcessing Semantically-Ordered Streams in Financial Services
Processing Semantically-Ordered Streams in Financial Services
 
Reactive Programming for a demanding world: building event-driven and respons...
Reactive Programming for a demanding world: building event-driven and respons...Reactive Programming for a demanding world: building event-driven and respons...
Reactive Programming for a demanding world: building event-driven and respons...
 

Viewers also liked

Matthias J. Sax – A Tale of Squirrels and Storms
Matthias J. Sax – A Tale of Squirrels and StormsMatthias J. Sax – A Tale of Squirrels and Storms
Matthias J. Sax – A Tale of Squirrels and StormsFlink Forward
 
K. Tzoumas & S. Ewen – Flink Forward Keynote
K. Tzoumas & S. Ewen – Flink Forward KeynoteK. Tzoumas & S. Ewen – Flink Forward Keynote
K. Tzoumas & S. Ewen – Flink Forward KeynoteFlink Forward
 
Apache Flink Training: DataSet API Basics
Apache Flink Training: DataSet API BasicsApache Flink Training: DataSet API Basics
Apache Flink Training: DataSet API BasicsFlink Forward
 
Streaming Data Flow with Apache Flink @ Paris Flink Meetup 2015
Streaming Data Flow with Apache Flink @ Paris Flink Meetup 2015Streaming Data Flow with Apache Flink @ Paris Flink Meetup 2015
Streaming Data Flow with Apache Flink @ Paris Flink Meetup 2015Till Rohrmann
 
Michael Häusler – Everyday flink
Michael Häusler – Everyday flinkMichael Häusler – Everyday flink
Michael Häusler – Everyday flinkFlink 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 FlinkFlink 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-timeFlink Forward
 
Assaf Araki – Real Time Analytics at Scale
Assaf Araki – Real Time Analytics at ScaleAssaf Araki – Real Time Analytics at Scale
Assaf Araki – Real Time Analytics at ScaleFlink Forward
 
Apache Flink Training: System Overview
Apache Flink Training: System OverviewApache Flink Training: System Overview
Apache Flink Training: System OverviewFlink Forward
 
Apache Flink: API, runtime, and project roadmap
Apache Flink: API, runtime, and project roadmapApache Flink: API, runtime, and project roadmap
Apache Flink: API, runtime, and project roadmapKostas Tzoumas
 
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 SystemFlink 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 FlinkFlink Forward
 
Vasia Kalavri – Training: Gelly School
Vasia Kalavri – Training: Gelly School Vasia Kalavri – Training: Gelly School
Vasia Kalavri – Training: Gelly School 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 FlinkFlink Forward
 
Christian Kreuzfeld – Static vs Dynamic Stream Processing
Christian Kreuzfeld – Static vs Dynamic Stream ProcessingChristian Kreuzfeld – Static vs Dynamic Stream Processing
Christian Kreuzfeld – Static vs Dynamic Stream ProcessingFlink Forward
 
Kamal Hakimzadeh – Reproducible Distributed Experiments
Kamal Hakimzadeh – Reproducible Distributed ExperimentsKamal Hakimzadeh – Reproducible Distributed Experiments
Kamal Hakimzadeh – Reproducible Distributed ExperimentsFlink Forward
 
Fabian Hueske – Juggling with Bits and Bytes
Fabian Hueske – Juggling with Bits and BytesFabian Hueske – Juggling with Bits and Bytes
Fabian Hueske – Juggling with Bits and BytesFlink 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 DSLFlink Forward
 
Fabian Hueske – Cascading on Flink
Fabian Hueske – Cascading on FlinkFabian Hueske – Cascading on Flink
Fabian Hueske – Cascading on FlinkFlink Forward
 
Apache Flink - Hadoop MapReduce Compatibility
Apache Flink - Hadoop MapReduce CompatibilityApache Flink - Hadoop MapReduce Compatibility
Apache Flink - Hadoop MapReduce CompatibilityFabian Hueske
 

Viewers also liked (20)

Matthias J. Sax – A Tale of Squirrels and Storms
Matthias J. Sax – A Tale of Squirrels and StormsMatthias J. Sax – A Tale of Squirrels and Storms
Matthias J. Sax – A Tale of Squirrels and Storms
 
K. Tzoumas & S. Ewen – Flink Forward Keynote
K. Tzoumas & S. Ewen – Flink Forward KeynoteK. Tzoumas & S. Ewen – Flink Forward Keynote
K. Tzoumas & S. Ewen – Flink Forward Keynote
 
Apache Flink Training: DataSet API Basics
Apache Flink Training: DataSet API BasicsApache Flink Training: DataSet API Basics
Apache Flink Training: DataSet API Basics
 
Streaming Data Flow with Apache Flink @ Paris Flink Meetup 2015
Streaming Data Flow with Apache Flink @ Paris Flink Meetup 2015Streaming Data Flow with Apache Flink @ Paris Flink Meetup 2015
Streaming Data Flow with Apache Flink @ Paris Flink Meetup 2015
 
Michael Häusler – Everyday flink
Michael Häusler – Everyday flinkMichael Häusler – Everyday flink
Michael Häusler – Everyday flink
 
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
 
Assaf Araki – Real Time Analytics at Scale
Assaf Araki – Real Time Analytics at ScaleAssaf Araki – Real Time Analytics at Scale
Assaf Araki – Real Time Analytics at Scale
 
Apache Flink Training: System Overview
Apache Flink Training: System OverviewApache Flink Training: System Overview
Apache Flink Training: System Overview
 
Apache Flink: API, runtime, and project roadmap
Apache Flink: API, runtime, and project roadmapApache Flink: API, runtime, and project roadmap
Apache Flink: API, runtime, and project roadmap
 
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
 
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
 
Vasia Kalavri – Training: Gelly School
Vasia Kalavri – Training: Gelly School Vasia Kalavri – Training: Gelly School
Vasia Kalavri – Training: Gelly School
 
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
 
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
 
Kamal Hakimzadeh – Reproducible Distributed Experiments
Kamal Hakimzadeh – Reproducible Distributed ExperimentsKamal Hakimzadeh – Reproducible Distributed Experiments
Kamal Hakimzadeh – Reproducible Distributed Experiments
 
Fabian Hueske – Juggling with Bits and Bytes
Fabian Hueske – Juggling with Bits and BytesFabian Hueske – Juggling with Bits and Bytes
Fabian Hueske – Juggling with Bits and Bytes
 
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
 
Fabian Hueske – Cascading on Flink
Fabian Hueske – Cascading on FlinkFabian Hueske – Cascading on Flink
Fabian Hueske – Cascading on Flink
 
Apache Flink - Hadoop MapReduce Compatibility
Apache Flink - Hadoop MapReduce CompatibilityApache Flink - Hadoop MapReduce Compatibility
Apache Flink - Hadoop MapReduce Compatibility
 

Similar to Apache Flink Training: DataStream API Part 2 Advanced

ikh331-06-distributed-programming
ikh331-06-distributed-programmingikh331-06-distributed-programming
ikh331-06-distributed-programmingAnung Ariwibowo
 
Apache Hadoop Java API
Apache Hadoop Java APIApache Hadoop Java API
Apache Hadoop Java APIAdam Kawa
 
Flink Batch Processing and Iterations
Flink Batch Processing and IterationsFlink Batch Processing and Iterations
Flink Batch Processing and IterationsSameer Wadkar
 
Java 8 Lambda Built-in Functional Interfaces
Java 8 Lambda Built-in Functional InterfacesJava 8 Lambda Built-in Functional Interfaces
Java 8 Lambda Built-in Functional InterfacesGanesh Samarthyam
 
Kotlin Bytecode Generation and Runtime Performance
Kotlin Bytecode Generation and Runtime PerformanceKotlin Bytecode Generation and Runtime Performance
Kotlin Bytecode Generation and Runtime Performanceintelliyole
 
Creat Shape classes from scratch DETAILS You will create 3 shape cla.pdf
Creat Shape classes from scratch DETAILS You will create 3 shape cla.pdfCreat Shape classes from scratch DETAILS You will create 3 shape cla.pdf
Creat Shape classes from scratch DETAILS You will create 3 shape cla.pdfaromanets
 
Object oriented programming using c++
Object oriented programming using c++Object oriented programming using c++
Object oriented programming using c++Hoang Nguyen
 
Java design patterns
Java design patternsJava design patterns
Java design patternsShawn Brito
 
Use of Apache Commons and Utilities
Use of Apache Commons and UtilitiesUse of Apache Commons and Utilities
Use of Apache Commons and UtilitiesPramod Kumar
 
K is for Kotlin
K is for KotlinK is for Kotlin
K is for KotlinTechMagic
 
[2019-07] GraphQL in depth (serverside)
[2019-07] GraphQL in depth (serverside)[2019-07] GraphQL in depth (serverside)
[2019-07] GraphQL in depth (serverside)croquiscom
 
Introduction to Apache Flink
Introduction to Apache FlinkIntroduction to Apache Flink
Introduction to Apache Flinkmxmxm
 
2017 02-07 - elastic & spark. building a search geo locator
2017 02-07 - elastic & spark. building a search geo locator2017 02-07 - elastic & spark. building a search geo locator
2017 02-07 - elastic & spark. building a search geo locatorAlberto Paro
 
2017 02-07 - elastic & spark. building a search geo locator
2017 02-07 - elastic & spark. building a search geo locator2017 02-07 - elastic & spark. building a search geo locator
2017 02-07 - elastic & spark. building a search geo locatorAlberto Paro
 

Similar to Apache Flink Training: DataStream API Part 2 Advanced (20)

ikh331-06-distributed-programming
ikh331-06-distributed-programmingikh331-06-distributed-programming
ikh331-06-distributed-programming
 
Apache Hadoop Java API
Apache Hadoop Java APIApache Hadoop Java API
Apache Hadoop Java API
 
Flink Batch Processing and Iterations
Flink Batch Processing and IterationsFlink Batch Processing and Iterations
Flink Batch Processing and Iterations
 
Hw09 Hadoop + Clojure
Hw09   Hadoop + ClojureHw09   Hadoop + Clojure
Hw09 Hadoop + Clojure
 
Hadoop + Clojure
Hadoop + ClojureHadoop + Clojure
Hadoop + Clojure
 
Anti patterns
Anti patternsAnti patterns
Anti patterns
 
Java 8 Lambda Built-in Functional Interfaces
Java 8 Lambda Built-in Functional InterfacesJava 8 Lambda Built-in Functional Interfaces
Java 8 Lambda Built-in Functional Interfaces
 
Kotlin Bytecode Generation and Runtime Performance
Kotlin Bytecode Generation and Runtime PerformanceKotlin Bytecode Generation and Runtime Performance
Kotlin Bytecode Generation and Runtime Performance
 
TechTalk - Dotnet
TechTalk - DotnetTechTalk - Dotnet
TechTalk - Dotnet
 
Creat Shape classes from scratch DETAILS You will create 3 shape cla.pdf
Creat Shape classes from scratch DETAILS You will create 3 shape cla.pdfCreat Shape classes from scratch DETAILS You will create 3 shape cla.pdf
Creat Shape classes from scratch DETAILS You will create 3 shape cla.pdf
 
Java 8 Workshop
Java 8 WorkshopJava 8 Workshop
Java 8 Workshop
 
Object oriented programming using c++
Object oriented programming using c++Object oriented programming using c++
Object oriented programming using c++
 
Java design patterns
Java design patternsJava design patterns
Java design patterns
 
Use of Apache Commons and Utilities
Use of Apache Commons and UtilitiesUse of Apache Commons and Utilities
Use of Apache Commons and Utilities
 
K is for Kotlin
K is for KotlinK is for Kotlin
K is for Kotlin
 
[2019-07] GraphQL in depth (serverside)
[2019-07] GraphQL in depth (serverside)[2019-07] GraphQL in depth (serverside)
[2019-07] GraphQL in depth (serverside)
 
Introduction to Apache Flink
Introduction to Apache FlinkIntroduction to Apache Flink
Introduction to Apache Flink
 
2017 02-07 - elastic & spark. building a search geo locator
2017 02-07 - elastic & spark. building a search geo locator2017 02-07 - elastic & spark. building a search geo locator
2017 02-07 - elastic & spark. building a search geo locator
 
2017 02-07 - elastic & spark. building a search geo locator
2017 02-07 - elastic & spark. building a search geo locator2017 02-07 - elastic & spark. building a search geo locator
2017 02-07 - elastic & spark. building a search geo locator
 
srgoc
srgocsrgoc
srgoc
 

More from Flink Forward

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
 
Evening out the uneven: dealing with skew in Flink
Evening out the uneven: dealing with skew in FlinkEvening out the uneven: dealing with skew in Flink
Evening out the uneven: dealing with skew in FlinkFlink Forward
 
“Alexa, be quiet!”: End-to-end near-real time model building and evaluation i...
“Alexa, be quiet!”: End-to-end near-real time model building and evaluation i...“Alexa, be quiet!”: End-to-end near-real time model building and evaluation i...
“Alexa, be quiet!”: End-to-end near-real time model building and evaluation i...Flink Forward
 
Introducing BinarySortedMultiMap - A new Flink state primitive to boost your ...
Introducing BinarySortedMultiMap - A new Flink state primitive to boost your ...Introducing BinarySortedMultiMap - A new Flink state primitive to boost your ...
Introducing BinarySortedMultiMap - A new Flink state primitive to boost your ...Flink Forward
 
Introducing the Apache Flink Kubernetes Operator
Introducing the Apache Flink Kubernetes OperatorIntroducing the Apache Flink Kubernetes Operator
Introducing the Apache Flink Kubernetes OperatorFlink Forward
 
Dynamically Scaling Data Streams across Multiple Kafka Clusters with Zero Fli...
Dynamically Scaling Data Streams across Multiple Kafka Clusters with Zero Fli...Dynamically Scaling Data Streams across Multiple Kafka Clusters with Zero Fli...
Dynamically Scaling Data Streams across Multiple Kafka Clusters with Zero Fli...Flink Forward
 
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
 
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
 
Flink powered stream processing platform at Pinterest
Flink powered stream processing platform at PinterestFlink powered stream processing platform at Pinterest
Flink powered stream processing platform at PinterestFlink Forward
 
Apache Flink in the Cloud-Native Era
Apache Flink in the Cloud-Native EraApache Flink in the Cloud-Native Era
Apache Flink in the Cloud-Native EraFlink Forward
 
Where is my bottleneck? Performance troubleshooting in Flink
Where is my bottleneck? Performance troubleshooting in FlinkWhere is my bottleneck? Performance troubleshooting in Flink
Where is my bottleneck? Performance troubleshooting in FlinkFlink Forward
 
The Current State of Table API in 2022
The Current State of Table API in 2022The Current State of Table API in 2022
The Current State of Table API in 2022Flink Forward
 
Flink SQL on Pulsar made easy
Flink SQL on Pulsar made easyFlink SQL on Pulsar made easy
Flink SQL on Pulsar made easyFlink Forward
 
Dynamic Rule-based Real-time Market Data Alerts
Dynamic Rule-based Real-time Market Data AlertsDynamic Rule-based Real-time Market Data Alerts
Dynamic Rule-based Real-time Market Data AlertsFlink Forward
 
Exactly-Once Financial Data Processing at Scale with Flink and Pinot
Exactly-Once Financial Data Processing at Scale with Flink and PinotExactly-Once Financial Data Processing at Scale with Flink and Pinot
Exactly-Once Financial Data Processing at Scale with Flink and PinotFlink Forward
 
Tame the small files problem and optimize data layout for streaming ingestion...
Tame the small files problem and optimize data layout for streaming ingestion...Tame the small files problem and optimize data layout for streaming ingestion...
Tame the small files problem and optimize data layout for streaming ingestion...Flink Forward
 
Batch Processing at Scale with Flink & Iceberg
Batch Processing at Scale with Flink & IcebergBatch Processing at Scale with Flink & Iceberg
Batch Processing at Scale with Flink & IcebergFlink Forward
 
Welcome to the Flink Community!
Welcome to the Flink Community!Welcome to the Flink Community!
Welcome to the Flink Community!Flink 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
 
Extending Flink SQL for stream processing use cases
Extending Flink SQL for stream processing use casesExtending Flink SQL for stream processing use cases
Extending Flink SQL for stream processing use casesFlink Forward
 

More from Flink Forward (20)

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...
 
Evening out the uneven: dealing with skew in Flink
Evening out the uneven: dealing with skew in FlinkEvening out the uneven: dealing with skew in Flink
Evening out the uneven: dealing with skew in Flink
 
“Alexa, be quiet!”: End-to-end near-real time model building and evaluation i...
“Alexa, be quiet!”: End-to-end near-real time model building and evaluation i...“Alexa, be quiet!”: End-to-end near-real time model building and evaluation i...
“Alexa, be quiet!”: End-to-end near-real time model building and evaluation i...
 
Introducing BinarySortedMultiMap - A new Flink state primitive to boost your ...
Introducing BinarySortedMultiMap - A new Flink state primitive to boost your ...Introducing BinarySortedMultiMap - A new Flink state primitive to boost your ...
Introducing BinarySortedMultiMap - A new Flink state primitive to boost your ...
 
Introducing the Apache Flink Kubernetes Operator
Introducing the Apache Flink Kubernetes OperatorIntroducing the Apache Flink Kubernetes Operator
Introducing the Apache Flink Kubernetes Operator
 
Dynamically Scaling Data Streams across Multiple Kafka Clusters with Zero Fli...
Dynamically Scaling Data Streams across Multiple Kafka Clusters with Zero Fli...Dynamically Scaling Data Streams across Multiple Kafka Clusters with Zero Fli...
Dynamically Scaling Data Streams across Multiple Kafka Clusters with Zero Fli...
 
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
 
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
 
Flink powered stream processing platform at Pinterest
Flink powered stream processing platform at PinterestFlink powered stream processing platform at Pinterest
Flink powered stream processing platform at Pinterest
 
Apache Flink in the Cloud-Native Era
Apache Flink in the Cloud-Native EraApache Flink in the Cloud-Native Era
Apache Flink in the Cloud-Native Era
 
Where is my bottleneck? Performance troubleshooting in Flink
Where is my bottleneck? Performance troubleshooting in FlinkWhere is my bottleneck? Performance troubleshooting in Flink
Where is my bottleneck? Performance troubleshooting in Flink
 
The Current State of Table API in 2022
The Current State of Table API in 2022The Current State of Table API in 2022
The Current State of Table API in 2022
 
Flink SQL on Pulsar made easy
Flink SQL on Pulsar made easyFlink SQL on Pulsar made easy
Flink SQL on Pulsar made easy
 
Dynamic Rule-based Real-time Market Data Alerts
Dynamic Rule-based Real-time Market Data AlertsDynamic Rule-based Real-time Market Data Alerts
Dynamic Rule-based Real-time Market Data Alerts
 
Exactly-Once Financial Data Processing at Scale with Flink and Pinot
Exactly-Once Financial Data Processing at Scale with Flink and PinotExactly-Once Financial Data Processing at Scale with Flink and Pinot
Exactly-Once Financial Data Processing at Scale with Flink and Pinot
 
Tame the small files problem and optimize data layout for streaming ingestion...
Tame the small files problem and optimize data layout for streaming ingestion...Tame the small files problem and optimize data layout for streaming ingestion...
Tame the small files problem and optimize data layout for streaming ingestion...
 
Batch Processing at Scale with Flink & Iceberg
Batch Processing at Scale with Flink & IcebergBatch Processing at Scale with Flink & Iceberg
Batch Processing at Scale with Flink & Iceberg
 
Welcome to the Flink Community!
Welcome to the Flink Community!Welcome to the Flink Community!
Welcome to the Flink Community!
 
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
 
Extending Flink SQL for stream processing use cases
Extending Flink SQL for stream processing use casesExtending Flink SQL for stream processing use cases
Extending Flink SQL for stream processing use cases
 

Recently uploaded

Tata AIG General Insurance Company - Insurer Innovation Award 2024
Tata AIG General Insurance Company - Insurer Innovation Award 2024Tata AIG General Insurance Company - Insurer Innovation Award 2024
Tata AIG General Insurance Company - Insurer Innovation Award 2024The Digital Insurer
 
The Codex of Business Writing Software for Real-World Solutions 2.pptx
The Codex of Business Writing Software for Real-World Solutions 2.pptxThe Codex of Business Writing Software for Real-World Solutions 2.pptx
The Codex of Business Writing Software for Real-World Solutions 2.pptxMalak Abu Hammad
 
Axa Assurance Maroc - Insurer Innovation Award 2024
Axa Assurance Maroc - Insurer Innovation Award 2024Axa Assurance Maroc - Insurer Innovation Award 2024
Axa Assurance Maroc - Insurer Innovation Award 2024The Digital Insurer
 
Breaking the Kubernetes Kill Chain: Host Path Mount
Breaking the Kubernetes Kill Chain: Host Path MountBreaking the Kubernetes Kill Chain: Host Path Mount
Breaking the Kubernetes Kill Chain: Host Path MountPuma Security, LLC
 
Unblocking The Main Thread Solving ANRs and Frozen Frames
Unblocking The Main Thread Solving ANRs and Frozen FramesUnblocking The Main Thread Solving ANRs and Frozen Frames
Unblocking The Main Thread Solving ANRs and Frozen FramesSinan KOZAK
 
Mastering MySQL Database Architecture: Deep Dive into MySQL Shell and MySQL R...
Mastering MySQL Database Architecture: Deep Dive into MySQL Shell and MySQL R...Mastering MySQL Database Architecture: Deep Dive into MySQL Shell and MySQL R...
Mastering MySQL Database Architecture: Deep Dive into MySQL Shell and MySQL R...Miguel Araújo
 
Neo4j - How KGs are shaping the future of Generative AI at AWS Summit London ...
Neo4j - How KGs are shaping the future of Generative AI at AWS Summit London ...Neo4j - How KGs are shaping the future of Generative AI at AWS Summit London ...
Neo4j - How KGs are shaping the future of Generative AI at AWS Summit London ...Neo4j
 
Scaling API-first – The story of a global engineering organization
Scaling API-first – The story of a global engineering organizationScaling API-first – The story of a global engineering organization
Scaling API-first – The story of a global engineering organizationRadu Cotescu
 
Handwritten Text Recognition for manuscripts and early printed texts
Handwritten Text Recognition for manuscripts and early printed textsHandwritten Text Recognition for manuscripts and early printed texts
Handwritten Text Recognition for manuscripts and early printed textsMaria Levchenko
 
The Role of Taxonomy and Ontology in Semantic Layers - Heather Hedden.pdf
The Role of Taxonomy and Ontology in Semantic Layers - Heather Hedden.pdfThe Role of Taxonomy and Ontology in Semantic Layers - Heather Hedden.pdf
The Role of Taxonomy and Ontology in Semantic Layers - Heather Hedden.pdfEnterprise Knowledge
 
Salesforce Community Group Quito, Salesforce 101
Salesforce Community Group Quito, Salesforce 101Salesforce Community Group Quito, Salesforce 101
Salesforce Community Group Quito, Salesforce 101Paola De la Torre
 
Boost PC performance: How more available memory can improve productivity
Boost PC performance: How more available memory can improve productivityBoost PC performance: How more available memory can improve productivity
Boost PC performance: How more available memory can improve productivityPrincipled Technologies
 
How to convert PDF to text with Nanonets
How to convert PDF to text with NanonetsHow to convert PDF to text with Nanonets
How to convert PDF to text with Nanonetsnaman860154
 
08448380779 Call Girls In Friends Colony Women Seeking Men
08448380779 Call Girls In Friends Colony Women Seeking Men08448380779 Call Girls In Friends Colony Women Seeking Men
08448380779 Call Girls In Friends Colony Women Seeking MenDelhi Call girls
 
A Call to Action for Generative AI in 2024
A Call to Action for Generative AI in 2024A Call to Action for Generative AI in 2024
A Call to Action for Generative AI in 2024Results
 
08448380779 Call Girls In Greater Kailash - I Women Seeking Men
08448380779 Call Girls In Greater Kailash - I Women Seeking Men08448380779 Call Girls In Greater Kailash - I Women Seeking Men
08448380779 Call Girls In Greater Kailash - I Women Seeking MenDelhi Call girls
 
Finology Group – Insurtech Innovation Award 2024
Finology Group – Insurtech Innovation Award 2024Finology Group – Insurtech Innovation Award 2024
Finology Group – Insurtech Innovation Award 2024The Digital Insurer
 
Exploring the Future Potential of AI-Enabled Smartphone Processors
Exploring the Future Potential of AI-Enabled Smartphone ProcessorsExploring the Future Potential of AI-Enabled Smartphone Processors
Exploring the Future Potential of AI-Enabled Smartphone Processorsdebabhi2
 
How to Troubleshoot Apps for the Modern Connected Worker
How to Troubleshoot Apps for the Modern Connected WorkerHow to Troubleshoot Apps for the Modern Connected Worker
How to Troubleshoot Apps for the Modern Connected WorkerThousandEyes
 
GenCyber Cyber Security Day Presentation
GenCyber Cyber Security Day PresentationGenCyber Cyber Security Day Presentation
GenCyber Cyber Security Day PresentationMichael W. Hawkins
 

Recently uploaded (20)

Tata AIG General Insurance Company - Insurer Innovation Award 2024
Tata AIG General Insurance Company - Insurer Innovation Award 2024Tata AIG General Insurance Company - Insurer Innovation Award 2024
Tata AIG General Insurance Company - Insurer Innovation Award 2024
 
The Codex of Business Writing Software for Real-World Solutions 2.pptx
The Codex of Business Writing Software for Real-World Solutions 2.pptxThe Codex of Business Writing Software for Real-World Solutions 2.pptx
The Codex of Business Writing Software for Real-World Solutions 2.pptx
 
Axa Assurance Maroc - Insurer Innovation Award 2024
Axa Assurance Maroc - Insurer Innovation Award 2024Axa Assurance Maroc - Insurer Innovation Award 2024
Axa Assurance Maroc - Insurer Innovation Award 2024
 
Breaking the Kubernetes Kill Chain: Host Path Mount
Breaking the Kubernetes Kill Chain: Host Path MountBreaking the Kubernetes Kill Chain: Host Path Mount
Breaking the Kubernetes Kill Chain: Host Path Mount
 
Unblocking The Main Thread Solving ANRs and Frozen Frames
Unblocking The Main Thread Solving ANRs and Frozen FramesUnblocking The Main Thread Solving ANRs and Frozen Frames
Unblocking The Main Thread Solving ANRs and Frozen Frames
 
Mastering MySQL Database Architecture: Deep Dive into MySQL Shell and MySQL R...
Mastering MySQL Database Architecture: Deep Dive into MySQL Shell and MySQL R...Mastering MySQL Database Architecture: Deep Dive into MySQL Shell and MySQL R...
Mastering MySQL Database Architecture: Deep Dive into MySQL Shell and MySQL R...
 
Neo4j - How KGs are shaping the future of Generative AI at AWS Summit London ...
Neo4j - How KGs are shaping the future of Generative AI at AWS Summit London ...Neo4j - How KGs are shaping the future of Generative AI at AWS Summit London ...
Neo4j - How KGs are shaping the future of Generative AI at AWS Summit London ...
 
Scaling API-first – The story of a global engineering organization
Scaling API-first – The story of a global engineering organizationScaling API-first – The story of a global engineering organization
Scaling API-first – The story of a global engineering organization
 
Handwritten Text Recognition for manuscripts and early printed texts
Handwritten Text Recognition for manuscripts and early printed textsHandwritten Text Recognition for manuscripts and early printed texts
Handwritten Text Recognition for manuscripts and early printed texts
 
The Role of Taxonomy and Ontology in Semantic Layers - Heather Hedden.pdf
The Role of Taxonomy and Ontology in Semantic Layers - Heather Hedden.pdfThe Role of Taxonomy and Ontology in Semantic Layers - Heather Hedden.pdf
The Role of Taxonomy and Ontology in Semantic Layers - Heather Hedden.pdf
 
Salesforce Community Group Quito, Salesforce 101
Salesforce Community Group Quito, Salesforce 101Salesforce Community Group Quito, Salesforce 101
Salesforce Community Group Quito, Salesforce 101
 
Boost PC performance: How more available memory can improve productivity
Boost PC performance: How more available memory can improve productivityBoost PC performance: How more available memory can improve productivity
Boost PC performance: How more available memory can improve productivity
 
How to convert PDF to text with Nanonets
How to convert PDF to text with NanonetsHow to convert PDF to text with Nanonets
How to convert PDF to text with Nanonets
 
08448380779 Call Girls In Friends Colony Women Seeking Men
08448380779 Call Girls In Friends Colony Women Seeking Men08448380779 Call Girls In Friends Colony Women Seeking Men
08448380779 Call Girls In Friends Colony Women Seeking Men
 
A Call to Action for Generative AI in 2024
A Call to Action for Generative AI in 2024A Call to Action for Generative AI in 2024
A Call to Action for Generative AI in 2024
 
08448380779 Call Girls In Greater Kailash - I Women Seeking Men
08448380779 Call Girls In Greater Kailash - I Women Seeking Men08448380779 Call Girls In Greater Kailash - I Women Seeking Men
08448380779 Call Girls In Greater Kailash - I Women Seeking Men
 
Finology Group – Insurtech Innovation Award 2024
Finology Group – Insurtech Innovation Award 2024Finology Group – Insurtech Innovation Award 2024
Finology Group – Insurtech Innovation Award 2024
 
Exploring the Future Potential of AI-Enabled Smartphone Processors
Exploring the Future Potential of AI-Enabled Smartphone ProcessorsExploring the Future Potential of AI-Enabled Smartphone Processors
Exploring the Future Potential of AI-Enabled Smartphone Processors
 
How to Troubleshoot Apps for the Modern Connected Worker
How to Troubleshoot Apps for the Modern Connected WorkerHow to Troubleshoot Apps for the Modern Connected Worker
How to Troubleshoot Apps for the Modern Connected Worker
 
GenCyber Cyber Security Day Presentation
GenCyber Cyber Security Day PresentationGenCyber Cyber Security Day Presentation
GenCyber Cyber Security Day Presentation
 

Apache Flink Training: DataStream API Part 2 Advanced

  • 1. Apache Flink® Training DataStream API Advanced August 26, 2015
  • 2. Type System and Keys What kind of data can Flink handle? 2 Note: Identical to DataSet API
  • 3. Apache Flink’s Type System  Flink aims to support all data types • Ease of programming • Seamless integration with existing code  Programs are analyzed before execution • Used data types are identified • Serializer & comparator are configured 3
  • 4. Apache Flink’s Type System  Data types are either • Atomic types (like Java Primitives) • Composite types (like Flink Tuples)  Composite types nest other types  Not all data types can be used as keys! • Flink partitions DataStreams on keys • Key types must be comparable 4
  • 5. Atomic Types Flink Type Java Type Can be used as key? BasicType Java Primitives (Integer, String, …) Yes ArrayType Arrays of Java primitives or objects No (Yes as of 0.10) WritableType Implements Hadoop’s Writable interface Yes, if implements WritableComparable GenericType Any other type Yes, if implements Comparable 5
  • 6. Composite Types  Are composed of fields with other types • Fields types can be atomic or composite  Fields can be addressed as keys • Field type must be a key type!  A composite type can be a key type • All field types must be key types! 6
  • 7. TupleType  Java: org.apache.flink.api.java.tuple.Tuple1 to Tuple25  Scala: use default Scala tuples (1 to 22 fields)  Tuple fields are typed Tuple3<Integer, String, Double> t3 = new Tuple3<>(1, “2”, 3.0); val t3: (Int, String, Double) = (1, ”2”, 3.0)  Tuples give the best performance 7
  • 8. TupleType  Define keys by field position DataStream<Tuple3<Integer, String, Double>> d = … // group on String field d.groupBy(1);  Or field names // group on Double field d.groupBy(“f2”); 8
  • 9. PojoType  Any Java class that • Has an empty default constructor • Has publicly accessible fields (public field or default getter & setter) public class Person { public int id; public String name; public Person() {}; public Person(int id, String name) {…}; } DataStream<Person> p = env.fromElements(new Person(1, ”Bob”)); 9
  • 10. PojoType  Define keys by field name DataStream<Person> p = … // group on “name” field d.groupBy(“name”); 10
  • 11. Scala CaseClasses  Scala case classes are natively supported case class Person(id: Int, name: String) d: DataStream[Person] = env.fromElements(Person(1, “Bob”)  Define keys by field name // use field “name” as key d.groupBy(“name”) 11
  • 12. Composite & Nested Keys DataStream<Tuple3<String, Person, Double>> d;  Composite keys are supported // group on both long fields d.groupBy(0, 1);  Nested fields can be used as types // group on nested “name” field d.groupBy(“f1.name”);  Full types can be used as key using “*” wildcard // group on complete nested Pojo field d.groupBy(“f1.*”); • “*” wildcard can also be used for atomic types 12
  • 13. KeySelectors  Keys can be computed using KeySelectors public class SumKeySelector implements KeySelector<Tuple2<Long, Long>, Long> { public Long getKey(Tuple2<Long, Long> t) { return t.f0 + t.f1; }} DataStream<Tuple2<Long,Long>> d = … d.groupBy(new SumKeySelector()); 13
  • 15. Windows  Aggregations on DataStreams are different from aggregations on DataSets • e.g., it is not possible to count all elements of a DataStream – they are infinite  DataStream aggregations make sense on windowed streams • i.e., a window of the "latest" elements of a stream  Windows can be defined on grouped and partitioned streams 15
  • 16. Windows (2) 16 // (name, age) of passengers DataStream<Tuple2<String, Integer>> passengers = … // group by second field (age) and keep last 1 minute // worth of data sliding the window every 10 seconds passengers .groupBy(1) .window(Time.of(1, TimeUnit.MINUTES)) .every(Time.of(10, TimeUnit.SECONDS))
  • 17. Types of Windows  Tumbling time window • .window(Time.of(1, TimeUnit.MINUTES))  Sliding time window • .window(Time.of(60, TimeUnit.SECONDS)) .every(Time.of(10, TimeUnit.SECONDS))  Count-based sliding window • .window(Count.of(1000)) .every(Count.of(10)) 17
  • 18. Aggregations on Windowed Streams 18 // (name, age) of passengers DataStream<Tuple2<String, Integer>> passengers = … // group by second field (age) and keep last 1 minute // worth of data sliding the window every 10 seconds // count passengers employees .groupBy(1) .window(Time.of(1, TimeUnit.MINUTES)) .every(Time.of(10, TimeUnit.SECONDS)) .mapWindow(new CountSameAge()); Warning: 0.9 -> 0.10 mapWindow becomes apply
  • 19. MapWindow public static class CountSameAge implements WindowMapFunction<Tuple2<String, Integer>, Tuple2<Integer, Integer>> { @Override public void mapWindow(Iterable<Tuple2<String, Integer>> persons, Collector<Tuple2<Integer, Integer>> out) { Integer ageGroup = 0; Integer countsInGroup = 0; for (Tuple2<String, Integer> person : persons) { ageGroup = person.f1; countsInGroup++; } out.collect(new Tuple2<>(ageGroup, countsInGroup)); } } 19 Warning: 0.9 -> 0.10 WindowMapFunction becomes WindowFunction
  • 20. Operations on Windowed Streams  mapWindow • Do something over the whole window  reduceWindow • Apply a functional reduce function to the window  Aggregates: sum, min, max, and others  flatten • Get back a regular DataStream 20 Warning: 0.9 -> 0.10 mapWindow becomes apply reduceWindow becomes reduce
  • 21. Working With Multiple Streams 21
  • 22. Connecting Streams  Sometimes several DataStreams need to be correlated with each other and share state  You can connect or join two DataStreams 22 DataStream<String> strings = … DataStream<Integer> ints = … // Create a ConnectedDataStream strings.connect(ints);
  • 23. Map on Connected Streams 23 DataStream<String> strings = … DataStream<Integer> ints = … // Create a ConnectedDataStream strings.connect(ints) .map(new CoMapFunction<Integer,String,Boolean> { @Override public Boolean map1 (Integer value) { return true; } @Override public Boolean map2 (String value) { return false; } });
  • 24. FlatMap on Connected Streams 24 DataStream<String> strings = … DataStream<Integer> ints = … // Create a ConnectedDataStream strings.connect(ints) .flatMap(new CoFlatMapFunction<Integer,String,String> { @Override public void flatMap1 (Integer value, Collector<String> out) { out.collect(value.toString()); } @Override public void flatMap2 (String value, Collector<String> out) { for (String word: value.split(" ")) { out.collect(word) } } });
  • 25. Rich functions and state 25
  • 26. RichFunctions  Function interfaces have only one method • Single abstract method (SAM) • Support for Java8 Lambda functions  There is a “Rich” variant for each function. • RichFlatMapFunction, … • Additional methods • open(Configuration c) • close() • getRuntimeContext() 26
  • 27. RichFunctions & RuntimeContext  RuntimeContext has useful methods: • getIndexOfThisSubtask () • getNumberOfParallelSubtasks() • getExecutionConfig()  Give access to partitioned state flink.apache.org 27
  • 28. Stateful Computations  All DataStream transformations can be stateful • State is mutable and lives as long as the streaming job is running • State is recovered with exactly-once semantics by Flink after a failure  You can define two kinds of state • Local state: each parallel task can register some local variables to take part in Flink’s checkpointing • Partitioned by key state: an operator on a partitioned by key stream can access and update state corresponding to its key • Partitioned state will be available in Flink 0.10 28
  • 29. Defining Local State 29 DataSet<String> aStream; DataStream<Long> lengths = aStream.map (new MapWithCounter()); public static class MapWithCounter implements MapFunction<String,Long>, Checkpointed<Long> { private long totalLength = 0; @Override public Long map (String value) { totalLength += value.length();8 return (Long) value.length(); } @Override public Serializable snapshotState( long checkpointId, long checkpointTimestamp) throws Exception { return new Long (totalLength); } @Override public void restoreState (Serializable state) throws Exception { totalLength = (Long) state; } }
  • 30. Defining Partitioned State 30 DataSet<Tuple2<String,String>> aStream; DataStream<Long> lengths = aStream.groupBy(0).map (new MapWithCounter()); public static class MapWithCounter implements RichMapFunction<Tuple2<String,String>,Long> { private OperatorState<Long> totalLengthByKey; @Override public Long map (Tuple2<String,String> value) { totalLengthByKey.update(totalLengthByKey.update.value() + 1); return (Long) value.f1.length(); } @Override public void open (Configuration conf) { totalLengthByKey = getRuntimeContext() .getOperatorState("totalLengthByKey", 0L, true); } } Note: Will be available in Flink 0.10
  • 32. Kafka and Flink  “Apache Kafka is a distributed, partitioned, replicated commit log service”  Kafka uses Apache Zookeeper for coordination  Kafka maintains feeds of messages in categories called topics  A Kafka topic can be read by Flink to produce a DataStream, and a DataStream can be written to a Kafka topic  Flink coordinates with Kafka to provide recovery in the case of failures 32
  • 33. Reading Data from Kafka  Enable checkpointing E.g., env.enableCheckpointing(5000);  Add a DataStream source from a Kafka topic Properties props = new Properties(); props.setProperty("zookeeper.connect", “localhost:2181”); props.setProperty("bootstrap.servers", “localhost:9092”); props.setProperty("group.id", “myGroup”); // create a data source DataStream<TaxiRide> rides = env.addSource( new FlinkKafkaConsumer082<TaxiRide>( “myTopic”, new TaxiRideSchema(), props) ); 33
  • 34. Writing Data to Kafka  Add a Kafka sink to a DataStream by providing • The broker address • The topic name • A serialization schema DataStream<String> aStream = … aStream.addSink( new KafkaSink<String>( “localhost:9092”, // default local broker “myTopic”, new SimpleStringSchema)); 34
  • 36. Not Covered Here  Iterations (feedback edges) • Very useful for Machine Learning  More transformations • union, join, ... 36
  • 37. 37