SlideShare una empresa de Scribd logo
1 de 36
Apache Flink® Training
DataStream API Basic
August 26, 2015
DataStream API
 Stream Processing
 Java and Scala
 All examples here in Java
 Documentation available at
flink.apache.org
 Currently labeled as beta – some API
changes are pending
• Noted in the slides with a warning
2
DataStream API by Example
3
Window WordCount: main Method
4
public static void main(String[] args) throws Exception {
// set up the execution environment
final StreamExecutionEnvironment env =
StreamExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple2<String, Integer>> counts = env
// read stream of words from socket
.socketTextStream("localhost", 9999)
// split up the lines in tuples containing: (word,1)
.flatMap(new Splitter())
// group by the tuple field "0"
.groupBy(0)
// keep the last 5 minute of data
.window(Time.of(5, TimeUnit.MINUTES))
//sum up tuple field "1"
.sum(1);
// print result in command line
counts.print();
// execute program
env.execute("Socket Incremental WordCount Example");
}
Stream Execution Environment
5
public static void main(String[] args) throws Exception {
// set up the execution environment
final StreamExecutionEnvironment env =
StreamExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple2<String, Integer>> counts = env
// read stream of words from socket
.socketTextStream("localhost", 9999)
// split up the lines in tuples containing: (word,1)
.flatMap(new Splitter())
// group by the tuple field "0"
.groupBy(0)
// keep the last 5 minute of data
.window(Time.of(5, TimeUnit.MINUTES))
//sum up tuple field "1"
.sum(1);
// print result in command line
counts.print();
// execute program
env.execute("Socket Incremental WordCount Example");
}
Data Sources
6
public static void main(String[] args) throws Exception {
// set up the execution environment
final StreamExecutionEnvironment env =
StreamExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple2<String, Integer>> counts = env
// read stream of words from socket
.socketTextStream("localhost", 9999)
// split up the lines in tuples containing: (word,1)
.flatMap(new Splitter())
// group by the tuple field "0"
.groupBy(0)
// keep the last 5 minute of data
.window(Time.of(5, TimeUnit.MINUTES))
//sum up tuple field "1"
.sum(1);
// print result in command line
counts.print();
// execute program
env.execute("Socket Incremental WordCount Example");
}
Data types
7
public static void main(String[] args) throws Exception {
// set up the execution environment
final StreamExecutionEnvironment env =
StreamExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple2<String, Integer>> counts = env
// read stream of words from socket
.socketTextStream("localhost", 9999)
// split up the lines in tuples containing: (word,1)
.flatMap(new Splitter())
// group by the tuple field "0"
.groupBy(0)
// keep the last 5 minute of data
.window(Time.of(5, TimeUnit.MINUTES))
//sum up tuple field "1"
.sum(1);
// print result in command line
counts.print();
// execute program
env.execute("Socket Incremental WordCount Example");
}
Transformations
8
public static void main(String[] args) throws Exception {
// set up the execution environment
final StreamExecutionEnvironment env =
StreamExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple2<String, Integer>> counts = env
// read stream of words from socket
.socketTextStream("localhost", 9999)
// split up the lines in tuples containing: (word,1)
.flatMap(new Splitter())
// group by the tuple field "0"
.groupBy(0)
// keep the last 5 minute of data
.window(Time.of(5, TimeUnit.MINUTES))
//sum up tuple field "1"
.sum(1);
// print result in command line
counts.print();
// execute program
env.execute("Socket Incremental WordCount Example");
}
User functions
9
public static void main(String[] args) throws Exception {
// set up the execution environment
final StreamExecutionEnvironment env =
StreamExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple2<String, Integer>> counts = env
// read stream of words from socket
.socketTextStream("localhost", 9999)
// split up the lines in tuples containing: (word,1)
.flatMap(new Splitter())
// group by the tuple field "0"
.groupBy(0)
// keep the last 5 minute of data
.window(Time.of(5, TimeUnit.MINUTES))
//sum up tuple field "1"
.sum(1);
// print result in command line
counts.print();
// execute program
env.execute("Socket Incremental WordCount Example");
}
DataSinks
10
public static void main(String[] args) throws Exception {
// set up the execution environment
final StreamExecutionEnvironment env =
StreamExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple2<String, Integer>> counts = env
// read stream of words from socket
.socketTextStream("localhost", 9999)
// split up the lines in tuples containing: (word,1)
.flatMap(new Splitter())
// group by the tuple field "0"
.groupBy(0)
// keep the last 5 minute of data
.window(Time.of(5, TimeUnit.MINUTES))
//sum up tuple field "1"
.sum(1);
// print result in command line
counts.print();
// execute program
env.execute("Socket Incremental WordCount Example");
}
Execute!
11
public static void main(String[] args) throws Exception {
// set up the execution environment
final StreamExecutionEnvironment env =
StreamExecutionEnvironment.getExecutionEnvironment();
DataSet<Tuple2<String, Integer>> counts = env
// read stream of words from socket
.socketTextStream("localhost", 9999)
// split up the lines in tuples containing: (word,1)
.flatMap(new Splitter())
// group by the tuple field "0"
.groupBy(0)
// keep the last 5 minute of data
.window(Time.of(5, TimeUnit.MINUTES))
//sum up tuple field "1"
.sum(1);
// print result in command line
counts.print();
// execute program
env.execute("Socket Incremental WordCount Example");
}
Window WordCount: FlatMap
public static class Splitter
implements FlatMapFunction<String, Tuple2<String, Integer>> {
@Override
public void flatMap(String value,
Collector<Tuple2<String, Integer>> out)
throws Exception {
// normalize and split the line
String[] tokens = value.toLowerCase().split("W+");
// emit the pairs
for (String token : tokens) {
if (token.length() > 0) {
out.collect(
new Tuple2<String, Integer>(token, 1));
}
}
}
}
12
WordCount: Map: Interface
13
public static class Splitter
implements FlatMapFunction<String, Tuple2<String, Integer>> {
@Override
public void flatMap(String value,
Collector<Tuple2<String, Integer>> out)
throws Exception {
// normalize and split the line
String[] tokens = value.toLowerCase().split("W+");
// emit the pairs
for (String token : tokens) {
if (token.length() > 0) {
out.collect(
new Tuple2<String, Integer>(token, 1));
}
}
}
}
WordCount: Map: Types
14
public static class Splitter
implements FlatMapFunction<String, Tuple2<String, Integer>> {
@Override
public void flatMap(String value,
Collector<Tuple2<String, Integer>> out)
throws Exception {
// normalize and split the line
String[] tokens = value.toLowerCase().split("W+");
// emit the pairs
for (String token : tokens) {
if (token.length() > 0) {
out.collect(
new Tuple2<String, Integer>(token, 1));
}
}
}
}
WordCount: Map: Collector
15
public static class Splitter
implements FlatMapFunction<String, Tuple2<String, Integer>> {
@Override
public void flatMap(String value,
Collector<Tuple2<String, Integer>> out)
throws Exception {
// normalize and split the line
String[] tokens = value.toLowerCase().split("W+");
// emit the pairs
for (String token : tokens) {
if (token.length() > 0) {
out.collect(
new Tuple2<String, Integer>(token, 1));
}
}
}
}
DataStream API Concepts
16
(Selected) Data Types
 Basic Java Types
• String, Long, Integer, Boolean,…
• Arrays
 Composite Types
• Tuples
• Many more (covered in the advanced slides)
17
Tuples
 The easiest and most lightweight way of
encapsulating data in Flink
 Tuple1 up to Tuple25
Tuple2<String, String> person = new Tuple2<>("Max", "Mustermann”);
Tuple3<String, String, Integer> person = new Tuple3<>("Max", "Mustermann", 42);
Tuple4<String, String, Integer, Boolean> person =
new Tuple4<>("Max", "Mustermann", 42, true);
// zero based index!
String firstName = person.f0;
String secondName = person.f1;
Integer age = person.f2;
Boolean fired = person.f3;
18
Transformations: Map
DataStream<Integer> integers = env.fromElements(1, 2, 3, 4);
// Regular Map - Takes one element and produces one element
DataStream<Integer> doubleIntegers =
integers.map(new MapFunction<Integer, Integer>() {
@Override
public Integer map(Integer value) {
return value * 2;
}
});
doubleIntegers.print();
> 2, 4, 6, 8
// Flat Map - Takes one element and produces zero, one, or more elements.
DataStream<Integer> doubleIntegers2 =
integers.flatMap(new FlatMapFunction<Integer, Integer>() {
@Override
public void flatMap(Integer value, Collector<Integer> out) {
out.collect(value * 2);
}
});
doubleIntegers2.print();
> 2, 4, 6, 8
19
Transformations: Filter
// The DataStream
DataStream<Integer> integers = env.fromElements(1, 2, 3, 4);
DataStream<Integer> filtered =
integers.filter(new FilterFunction<Integer>() {
@Override
public boolean filter(Integer value) {
return value != 3;
}
});
integers.print();
> 1, 2, 4
20
Transformations: Partitioning
 DataStreams can be partitioned by a key
21
// (name, age) of employees
DataStream<Tuple2<String, Integer>> passengers = …
// group by second field (age)
DataStream<Integer, Integer> grouped = passengers.groupBy(1)
Stephan, 18 Fabian, 23
Julia, 27 Anna, 18
Romeo, 27
Anna, 18 Stephan, 18
Julia, 27 Romeo, 27
Fabian, 23
Warning: Possible
renaming in next
releasesBen, 25
Ben, 25
Data Shipping Strategies
 Optionally, you can specify how data is shipped
between two transformations
 Forward: stream.forward()
• Only local communication
 Rebalance: stream.rebalance()
• Round-robin partitioning
 Partition by hash: stream.partitionByHash(...)
 Custom partitioning: stream.partitionCustom(...)
 Broadcast: stream.broadcast()
• Broadcast to all nodes
22
Data Sources
Collection
 fromCollection(collection)
 fromElements(1,2,3,4,5)
23
Data Sources (2)
Text socket
 socketTextStream("hostname",port)
Text file
 readFileStream(“/path/to/file”, 1000,
WatchType.PROCESS_ONLY_APPENDED)
Connectors
 E.g., Apache Kafka, RabbitMQ, …
24
Data Sources: Collections
StreamExecutionEnvironment env =
StreamExecutionEnvironment.getExecutionEnvironment();
// read from elements
DataStream<String> names = env.fromElements(“Some”, “Example”, “Strings”);
// read from Java collection
List<String> list = new ArrayList<String>();
list.add(“Some”);
list.add(“Example”);
list.add(“Strings”);
DataStream<String> names = env.fromCollection(list);
25
Data Sources: Files,Sockets,Connectors
StreamExecutionEnvironment env =
StreamExecutionEnvironment.getExecutionEnvironment();
// read text socket from port
DataStream<String> socketLines = env
.socketTextStream(”localhost", 9999);
// read a text file ingesting new elements every 100 milliseconds
DataStream<String> localLines = env
.readFileStream(”/path/to/file", 1000,
WatchType.PROCESS_ONLY_APPENDED);
26
Data Sinks
Text
 writeAsText(“/path/to/file”)
CSV
 writeAsCsv(“/path/to/file”)
Return data to the Client
 print()
27
Note: Identical to
DataSet API
Data Sinks (2)
Socket
 writeToSocket(hostname, port, SerializationSchema)
Connectors
 E.g., Apache Kafka, Elasticsearch,
Rolling HDFS Files
28
Data Sinks
 Lazily executed when env.execute() is called
DataStream<…> result;
// nothing happens
result.writeToSocket(...);
// nothing happens
result.writeAsText("/path/to/file", "n", "|");
// Execution really starts here
env.execute();
29
Fault Tolerance
30
Fault Tolerance in Flink
 Flink provides recovery by taking a consistent checkpoint every N
milliseconds and rolling back to the checkpointed state
• https://ci.apache.org/projects/flink/flink-docs-
master/internals/stream_checkpointing.html
 Exactly once (default)
• // Take checkpoint every 5000 milliseconds
env.enableCheckpointing (5000)
 At least once (for lower latency)
• // Take checkpoint every 5000 milliseconds
env.enableCheckpointing (5000, CheckpointingMode.AT_LEAST_ONCE)
 Setting the interval to few seconds should be good for most
applications
 If checkpointing is not enabled, no recovery guarantees are provided
31
Best Practices
32
Some advice
 Use env.fromElements(..) or env.fromCollection(..) to
quickly get a DataStream to experiment
with
 Use print() to quickly print a DataStream
33
Update Guide
34
From 0.9 to 0.10
 groupBy(…) -> keyBy(…)
 DataStream renames:
• KeyedDataStream -> KeyedStream
• WindowedDataStream -> WindowedStream
• ConnectedDataStream -> ConnectedStream
• JoinOperator -> JoinedStreams
35
36

Más contenido relacionado

La actualidad más candente

Streams and Tables: Two Sides of the Same Coin (BIRTE 2018)
Streams and Tables: Two Sides of the Same Coin (BIRTE 2018)Streams and Tables: Two Sides of the Same Coin (BIRTE 2018)
Streams and Tables: Two Sides of the Same Coin (BIRTE 2018)confluent
 
Fundamentals of Apache Kafka
Fundamentals of Apache KafkaFundamentals of Apache Kafka
Fundamentals of Apache KafkaChhavi Parasher
 
Introduction to Kafka connect
Introduction to Kafka connectIntroduction to Kafka connect
Introduction to Kafka connectKnoldus Inc.
 
Demystifying flink memory allocation and tuning - Roshan Naik, Uber
Demystifying flink memory allocation and tuning - Roshan Naik, UberDemystifying flink memory allocation and tuning - Roshan Naik, Uber
Demystifying flink memory allocation and tuning - Roshan Naik, UberFlink Forward
 
KSQL Deep Dive - The Open Source Streaming Engine for Apache Kafka
KSQL Deep Dive - The Open Source Streaming Engine for Apache KafkaKSQL Deep Dive - The Open Source Streaming Engine for Apache Kafka
KSQL Deep Dive - The Open Source Streaming Engine for Apache KafkaKai Wähner
 
Apache Beam: A unified model for batch and stream processing data
Apache Beam: A unified model for batch and stream processing dataApache Beam: A unified model for batch and stream processing data
Apache Beam: A unified model for batch and stream processing dataDataWorks Summit/Hadoop Summit
 
No data loss pipeline with apache kafka
No data loss pipeline with apache kafkaNo data loss pipeline with apache kafka
No data loss pipeline with apache kafkaJiangjie Qin
 
SQL Extensions to Support Streaming Data With Fabian Hueske | Current 2022
SQL Extensions to Support Streaming Data With Fabian Hueske | Current 2022SQL Extensions to Support Streaming Data With Fabian Hueske | Current 2022
SQL Extensions to Support Streaming Data With Fabian Hueske | Current 2022HostedbyConfluent
 
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 Kafka - Martin Podval
Apache Kafka - Martin PodvalApache Kafka - Martin Podval
Apache Kafka - Martin PodvalMartin Podval
 
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
 
Introduction to Apache Flink
Introduction to Apache FlinkIntroduction to Apache Flink
Introduction to Apache Flinkdatamantra
 
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
 
Producer Performance Tuning for Apache Kafka
Producer Performance Tuning for Apache KafkaProducer Performance Tuning for Apache Kafka
Producer Performance Tuning for Apache KafkaJiangjie Qin
 
A Thorough Comparison of Delta Lake, Iceberg and Hudi
A Thorough Comparison of Delta Lake, Iceberg and HudiA Thorough Comparison of Delta Lake, Iceberg and Hudi
A Thorough Comparison of Delta Lake, Iceberg and HudiDatabricks
 
Apache Flink internals
Apache Flink internalsApache Flink internals
Apache Flink internalsKostas Tzoumas
 
From cache to in-memory data grid. Introduction to Hazelcast.
From cache to in-memory data grid. Introduction to Hazelcast.From cache to in-memory data grid. Introduction to Hazelcast.
From cache to in-memory data grid. Introduction to Hazelcast.Taras Matyashovsky
 
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
 
Advanced Streaming Analytics with Apache Flink and Apache Kafka, Stephan Ewen
Advanced Streaming Analytics with Apache Flink and Apache Kafka, Stephan EwenAdvanced Streaming Analytics with Apache Flink and Apache Kafka, Stephan Ewen
Advanced Streaming Analytics with Apache Flink and Apache Kafka, Stephan Ewenconfluent
 
Deploying Confluent Platform for Production
Deploying Confluent Platform for ProductionDeploying Confluent Platform for Production
Deploying Confluent Platform for Productionconfluent
 

La actualidad más candente (20)

Streams and Tables: Two Sides of the Same Coin (BIRTE 2018)
Streams and Tables: Two Sides of the Same Coin (BIRTE 2018)Streams and Tables: Two Sides of the Same Coin (BIRTE 2018)
Streams and Tables: Two Sides of the Same Coin (BIRTE 2018)
 
Fundamentals of Apache Kafka
Fundamentals of Apache KafkaFundamentals of Apache Kafka
Fundamentals of Apache Kafka
 
Introduction to Kafka connect
Introduction to Kafka connectIntroduction to Kafka connect
Introduction to Kafka connect
 
Demystifying flink memory allocation and tuning - Roshan Naik, Uber
Demystifying flink memory allocation and tuning - Roshan Naik, UberDemystifying flink memory allocation and tuning - Roshan Naik, Uber
Demystifying flink memory allocation and tuning - Roshan Naik, Uber
 
KSQL Deep Dive - The Open Source Streaming Engine for Apache Kafka
KSQL Deep Dive - The Open Source Streaming Engine for Apache KafkaKSQL Deep Dive - The Open Source Streaming Engine for Apache Kafka
KSQL Deep Dive - The Open Source Streaming Engine for Apache Kafka
 
Apache Beam: A unified model for batch and stream processing data
Apache Beam: A unified model for batch and stream processing dataApache Beam: A unified model for batch and stream processing data
Apache Beam: A unified model for batch and stream processing data
 
No data loss pipeline with apache kafka
No data loss pipeline with apache kafkaNo data loss pipeline with apache kafka
No data loss pipeline with apache kafka
 
SQL Extensions to Support Streaming Data With Fabian Hueske | Current 2022
SQL Extensions to Support Streaming Data With Fabian Hueske | Current 2022SQL Extensions to Support Streaming Data With Fabian Hueske | Current 2022
SQL Extensions to Support Streaming Data With Fabian Hueske | Current 2022
 
Stephan Ewen - Scaling to large State
Stephan Ewen - Scaling to large StateStephan Ewen - Scaling to large State
Stephan Ewen - Scaling to large State
 
Apache Kafka - Martin Podval
Apache Kafka - Martin PodvalApache Kafka - Martin Podval
Apache Kafka - Martin Podval
 
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
 
Introduction to Apache Flink
Introduction to Apache FlinkIntroduction to Apache Flink
Introduction to Apache Flink
 
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
 
Producer Performance Tuning for Apache Kafka
Producer Performance Tuning for Apache KafkaProducer Performance Tuning for Apache Kafka
Producer Performance Tuning for Apache Kafka
 
A Thorough Comparison of Delta Lake, Iceberg and Hudi
A Thorough Comparison of Delta Lake, Iceberg and HudiA Thorough Comparison of Delta Lake, Iceberg and Hudi
A Thorough Comparison of Delta Lake, Iceberg and Hudi
 
Apache Flink internals
Apache Flink internalsApache Flink internals
Apache Flink internals
 
From cache to in-memory data grid. Introduction to Hazelcast.
From cache to in-memory data grid. Introduction to Hazelcast.From cache to in-memory data grid. Introduction to Hazelcast.
From cache to in-memory data grid. Introduction to Hazelcast.
 
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
 
Advanced Streaming Analytics with Apache Flink and Apache Kafka, Stephan Ewen
Advanced Streaming Analytics with Apache Flink and Apache Kafka, Stephan EwenAdvanced Streaming Analytics with Apache Flink and Apache Kafka, Stephan Ewen
Advanced Streaming Analytics with Apache Flink and Apache Kafka, Stephan Ewen
 
Deploying Confluent Platform for Production
Deploying Confluent Platform for ProductionDeploying Confluent Platform for Production
Deploying Confluent Platform for Production
 

Destacado

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
 
Slim Baltagi – Flink vs. Spark
Slim Baltagi – Flink vs. SparkSlim Baltagi – Flink vs. Spark
Slim Baltagi – Flink vs. SparkFlink Forward
 
Introduction to Apache Flink - Fast and reliable big data processing
Introduction to Apache Flink - Fast and reliable big data processingIntroduction to Apache Flink - Fast and reliable big data processing
Introduction to Apache Flink - Fast and reliable big data processingTill Rohrmann
 
Mohamed Amine Abdessemed – Real-time Data Integration with Apache Flink & Kafka
Mohamed Amine Abdessemed – Real-time Data Integration with Apache Flink & KafkaMohamed Amine Abdessemed – Real-time Data Integration with Apache Flink & Kafka
Mohamed Amine Abdessemed – Real-time Data Integration with Apache Flink & KafkaFlink Forward
 
Marton Balassi – Stateful Stream Processing
Marton Balassi – Stateful Stream ProcessingMarton Balassi – Stateful Stream Processing
Marton Balassi – Stateful Stream ProcessingFlink Forward
 
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
 
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 Training: DataStream API Part 2 Advanced
Apache Flink Training: DataStream API Part 2 Advanced Apache Flink Training: DataStream API Part 2 Advanced
Apache Flink Training: DataStream API Part 2 Advanced Flink Forward
 
Kamal Hakimzadeh – Reproducible Distributed Experiments
Kamal Hakimzadeh – Reproducible Distributed ExperimentsKamal Hakimzadeh – Reproducible Distributed Experiments
Kamal Hakimzadeh – Reproducible Distributed ExperimentsFlink Forward
 
Jim Dowling – Interactive Flink analytics with HopsWorks and Zeppelin
Jim Dowling – Interactive Flink analytics with HopsWorks and ZeppelinJim Dowling – Interactive Flink analytics with HopsWorks and Zeppelin
Jim Dowling – Interactive Flink analytics with HopsWorks and ZeppelinFlink Forward
 
Fabian Hueske – Cascading on Flink
Fabian Hueske – Cascading on FlinkFabian Hueske – Cascading on Flink
Fabian Hueske – Cascading on FlinkFlink Forward
 
Anwar Rizal – Streaming & Parallel Decision Tree in Flink
Anwar Rizal – Streaming & Parallel Decision Tree in FlinkAnwar Rizal – Streaming & Parallel Decision Tree in Flink
Anwar Rizal – Streaming & Parallel Decision Tree in FlinkFlink 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 - Hadoop MapReduce Compatibility
Apache Flink - Hadoop MapReduce CompatibilityApache Flink - Hadoop MapReduce Compatibility
Apache Flink - Hadoop MapReduce CompatibilityFabian Hueske
 
Till Rohrmann – Fault Tolerance and Job Recovery in Apache Flink
Till Rohrmann – Fault Tolerance and Job Recovery in Apache FlinkTill Rohrmann – Fault Tolerance and Job Recovery in Apache Flink
Till Rohrmann – Fault Tolerance and Job Recovery in Apache FlinkFlink 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
 
Simon Laws – Apache Flink Cluster Deployment on Docker and Docker-Compose
Simon Laws – Apache Flink Cluster Deployment on Docker and Docker-ComposeSimon Laws – Apache Flink Cluster Deployment on Docker and Docker-Compose
Simon Laws – Apache Flink Cluster Deployment on Docker and Docker-ComposeFlink 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
 
Vyacheslav Zholudev – Flink, a Convenient Abstraction Layer for Yarn?
Vyacheslav Zholudev – Flink, a Convenient Abstraction Layer for Yarn?Vyacheslav Zholudev – Flink, a Convenient Abstraction Layer for Yarn?
Vyacheslav Zholudev – Flink, a Convenient Abstraction Layer for Yarn?Flink Forward
 
S. Bartoli & F. Pompermaier – A Semantic Big Data Companion
S. Bartoli & F. Pompermaier – A Semantic Big Data CompanionS. Bartoli & F. Pompermaier – A Semantic Big Data Companion
S. Bartoli & F. Pompermaier – A Semantic Big Data CompanionFlink Forward
 

Destacado (20)

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
 
Slim Baltagi – Flink vs. Spark
Slim Baltagi – Flink vs. SparkSlim Baltagi – Flink vs. Spark
Slim Baltagi – Flink vs. Spark
 
Introduction to Apache Flink - Fast and reliable big data processing
Introduction to Apache Flink - Fast and reliable big data processingIntroduction to Apache Flink - Fast and reliable big data processing
Introduction to Apache Flink - Fast and reliable big data processing
 
Mohamed Amine Abdessemed – Real-time Data Integration with Apache Flink & Kafka
Mohamed Amine Abdessemed – Real-time Data Integration with Apache Flink & KafkaMohamed Amine Abdessemed – Real-time Data Integration with Apache Flink & Kafka
Mohamed Amine Abdessemed – Real-time Data Integration with Apache Flink & Kafka
 
Marton Balassi – Stateful Stream Processing
Marton Balassi – Stateful Stream ProcessingMarton Balassi – Stateful Stream Processing
Marton Balassi – Stateful Stream Processing
 
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
 
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 Training: DataStream API Part 2 Advanced
Apache Flink Training: DataStream API Part 2 Advanced Apache Flink Training: DataStream API Part 2 Advanced
Apache Flink Training: DataStream API Part 2 Advanced
 
Kamal Hakimzadeh – Reproducible Distributed Experiments
Kamal Hakimzadeh – Reproducible Distributed ExperimentsKamal Hakimzadeh – Reproducible Distributed Experiments
Kamal Hakimzadeh – Reproducible Distributed Experiments
 
Jim Dowling – Interactive Flink analytics with HopsWorks and Zeppelin
Jim Dowling – Interactive Flink analytics with HopsWorks and ZeppelinJim Dowling – Interactive Flink analytics with HopsWorks and Zeppelin
Jim Dowling – Interactive Flink analytics with HopsWorks and Zeppelin
 
Fabian Hueske – Cascading on Flink
Fabian Hueske – Cascading on FlinkFabian Hueske – Cascading on Flink
Fabian Hueske – Cascading on Flink
 
Anwar Rizal – Streaming & Parallel Decision Tree in Flink
Anwar Rizal – Streaming & Parallel Decision Tree in FlinkAnwar Rizal – Streaming & Parallel Decision Tree in Flink
Anwar Rizal – Streaming & Parallel Decision Tree in Flink
 
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 - Hadoop MapReduce Compatibility
Apache Flink - Hadoop MapReduce CompatibilityApache Flink - Hadoop MapReduce Compatibility
Apache Flink - Hadoop MapReduce Compatibility
 
Till Rohrmann – Fault Tolerance and Job Recovery in Apache Flink
Till Rohrmann – Fault Tolerance and Job Recovery in Apache FlinkTill Rohrmann – Fault Tolerance and Job Recovery in Apache Flink
Till Rohrmann – Fault Tolerance and Job Recovery in Apache Flink
 
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
 
Simon Laws – Apache Flink Cluster Deployment on Docker and Docker-Compose
Simon Laws – Apache Flink Cluster Deployment on Docker and Docker-ComposeSimon Laws – Apache Flink Cluster Deployment on Docker and Docker-Compose
Simon Laws – Apache Flink Cluster Deployment on Docker and Docker-Compose
 
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
 
Vyacheslav Zholudev – Flink, a Convenient Abstraction Layer for Yarn?
Vyacheslav Zholudev – Flink, a Convenient Abstraction Layer for Yarn?Vyacheslav Zholudev – Flink, a Convenient Abstraction Layer for Yarn?
Vyacheslav Zholudev – Flink, a Convenient Abstraction Layer for Yarn?
 
S. Bartoli & F. Pompermaier – A Semantic Big Data Companion
S. Bartoli & F. Pompermaier – A Semantic Big Data CompanionS. Bartoli & F. Pompermaier – A Semantic Big Data Companion
S. Bartoli & F. Pompermaier – A Semantic Big Data Companion
 

Similar a Apache Flink Training: DataStream API Part 1 Basic

Apache Flink Training: DataSet API Basics
Apache Flink Training: DataSet API BasicsApache Flink Training: DataSet API Basics
Apache Flink Training: DataSet API BasicsFlink Forward
 
Flink Batch Processing and Iterations
Flink Batch Processing and IterationsFlink Batch Processing and Iterations
Flink Batch Processing and IterationsSameer Wadkar
 
Apache Flink Stream Processing
Apache Flink Stream ProcessingApache Flink Stream Processing
Apache Flink Stream ProcessingSuneel Marthi
 
Introduction to Apache Flink
Introduction to Apache FlinkIntroduction to Apache Flink
Introduction to Apache Flinkmxmxm
 
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
 
실시간 인벤트 처리
실시간 인벤트 처리실시간 인벤트 처리
실시간 인벤트 처리Byeongweon Moon
 
Frequency .java Word frequency counter package frequ.pdf
Frequency .java  Word frequency counter  package frequ.pdfFrequency .java  Word frequency counter  package frequ.pdf
Frequency .java Word frequency counter package frequ.pdfarshiartpalace
 
Lecture#6 functions in c++
Lecture#6 functions in c++Lecture#6 functions in c++
Lecture#6 functions in c++NUST Stuff
 
Router Queue Simulation in C++ in MMNN and MM1 conditions
Router Queue Simulation in C++ in MMNN and MM1 conditionsRouter Queue Simulation in C++ in MMNN and MM1 conditions
Router Queue Simulation in C++ in MMNN and MM1 conditionsMorteza Mahdilar
 
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
 
Parallel Programming With Dot Net
Parallel Programming With Dot NetParallel Programming With Dot Net
Parallel Programming With Dot NetNeeraj Kaushik
 
ikh331-06-distributed-programming
ikh331-06-distributed-programmingikh331-06-distributed-programming
ikh331-06-distributed-programmingAnung Ariwibowo
 
httplinux.die.netman3execfork() creates a new process by.docx
httplinux.die.netman3execfork() creates a new process by.docxhttplinux.die.netman3execfork() creates a new process by.docx
httplinux.die.netman3execfork() creates a new process by.docxadampcarr67227
 
Taming Cloud APIs with Swift
Taming Cloud APIs with SwiftTaming Cloud APIs with Swift
Taming Cloud APIs with SwiftTim Burks
 
Program Assignment Process ManagementObjective This program a.docx
Program Assignment  Process ManagementObjective This program a.docxProgram Assignment  Process ManagementObjective This program a.docx
Program Assignment Process ManagementObjective This program a.docxwkyra78
 
Threaded-Execution and CPS Provide Smooth Switching Between Execution Modes
Threaded-Execution and CPS Provide Smooth Switching Between Execution ModesThreaded-Execution and CPS Provide Smooth Switching Between Execution Modes
Threaded-Execution and CPS Provide Smooth Switching Between Execution ModesESUG
 

Similar a Apache Flink Training: DataStream API Part 1 Basic (20)

Apache Flink Training: DataSet API Basics
Apache Flink Training: DataSet API BasicsApache Flink Training: DataSet API Basics
Apache Flink Training: DataSet API Basics
 
Flink Batch Processing and Iterations
Flink Batch Processing and IterationsFlink Batch Processing and Iterations
Flink Batch Processing and Iterations
 
Apache Flink Stream Processing
Apache Flink Stream ProcessingApache Flink Stream Processing
Apache Flink Stream Processing
 
Introduction to Apache Flink
Introduction to Apache FlinkIntroduction to Apache Flink
Introduction to Apache Flink
 
srgoc
srgocsrgoc
srgoc
 
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
 
C Exam Help
C Exam Help C Exam Help
C Exam Help
 
C Homework Help
C Homework HelpC Homework Help
C Homework Help
 
실시간 인벤트 처리
실시간 인벤트 처리실시간 인벤트 처리
실시간 인벤트 처리
 
Frequency .java Word frequency counter package frequ.pdf
Frequency .java  Word frequency counter  package frequ.pdfFrequency .java  Word frequency counter  package frequ.pdf
Frequency .java Word frequency counter package frequ.pdf
 
Lecture#6 functions in c++
Lecture#6 functions in c++Lecture#6 functions in c++
Lecture#6 functions in c++
 
Microkernel Development
Microkernel DevelopmentMicrokernel Development
Microkernel Development
 
Router Queue Simulation in C++ in MMNN and MM1 conditions
Router Queue Simulation in C++ in MMNN and MM1 conditionsRouter Queue Simulation in C++ in MMNN and MM1 conditions
Router Queue Simulation in C++ in MMNN and MM1 conditions
 
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
 
Parallel Programming With Dot Net
Parallel Programming With Dot NetParallel Programming With Dot Net
Parallel Programming With Dot Net
 
ikh331-06-distributed-programming
ikh331-06-distributed-programmingikh331-06-distributed-programming
ikh331-06-distributed-programming
 
httplinux.die.netman3execfork() creates a new process by.docx
httplinux.die.netman3execfork() creates a new process by.docxhttplinux.die.netman3execfork() creates a new process by.docx
httplinux.die.netman3execfork() creates a new process by.docx
 
Taming Cloud APIs with Swift
Taming Cloud APIs with SwiftTaming Cloud APIs with Swift
Taming Cloud APIs with Swift
 
Program Assignment Process ManagementObjective This program a.docx
Program Assignment  Process ManagementObjective This program a.docxProgram Assignment  Process ManagementObjective This program a.docx
Program Assignment Process ManagementObjective This program a.docx
 
Threaded-Execution and CPS Provide Smooth Switching Between Execution Modes
Threaded-Execution and CPS Provide Smooth Switching Between Execution ModesThreaded-Execution and CPS Provide Smooth Switching Between Execution Modes
Threaded-Execution and CPS Provide Smooth Switching Between Execution Modes
 

Más de 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
 
Autoscaling Flink with Reactive Mode
Autoscaling Flink with Reactive ModeAutoscaling Flink with Reactive Mode
Autoscaling Flink with Reactive ModeFlink 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
 
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
 
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
 
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
 
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
 

Más de 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
 
Autoscaling Flink with Reactive Mode
Autoscaling Flink with Reactive ModeAutoscaling Flink with Reactive Mode
Autoscaling Flink with Reactive Mode
 
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
 
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
 
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
 
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
 
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!
 

Último

Passkey Providers and Enabling Portability: FIDO Paris Seminar.pptx
Passkey Providers and Enabling Portability: FIDO Paris Seminar.pptxPasskey Providers and Enabling Portability: FIDO Paris Seminar.pptx
Passkey Providers and Enabling Portability: FIDO Paris Seminar.pptxLoriGlavin3
 
Advanced Computer Architecture – An Introduction
Advanced Computer Architecture – An IntroductionAdvanced Computer Architecture – An Introduction
Advanced Computer Architecture – An IntroductionDilum Bandara
 
Unleash Your Potential - Namagunga Girls Coding Club
Unleash Your Potential - Namagunga Girls Coding ClubUnleash Your Potential - Namagunga Girls Coding Club
Unleash Your Potential - Namagunga Girls Coding ClubKalema Edgar
 
The Fit for Passkeys for Employee and Consumer Sign-ins: FIDO Paris Seminar.pptx
The Fit for Passkeys for Employee and Consumer Sign-ins: FIDO Paris Seminar.pptxThe Fit for Passkeys for Employee and Consumer Sign-ins: FIDO Paris Seminar.pptx
The Fit for Passkeys for Employee and Consumer Sign-ins: FIDO Paris Seminar.pptxLoriGlavin3
 
"Debugging python applications inside k8s environment", Andrii Soldatenko
"Debugging python applications inside k8s environment", Andrii Soldatenko"Debugging python applications inside k8s environment", Andrii Soldatenko
"Debugging python applications inside k8s environment", Andrii SoldatenkoFwdays
 
Streamlining Python Development: A Guide to a Modern Project Setup
Streamlining Python Development: A Guide to a Modern Project SetupStreamlining Python Development: A Guide to a Modern Project Setup
Streamlining Python Development: A Guide to a Modern Project SetupFlorian Wilhelm
 
Connect Wave/ connectwave Pitch Deck Presentation
Connect Wave/ connectwave Pitch Deck PresentationConnect Wave/ connectwave Pitch Deck Presentation
Connect Wave/ connectwave Pitch Deck PresentationSlibray Presentation
 
Digital Identity is Under Attack: FIDO Paris Seminar.pptx
Digital Identity is Under Attack: FIDO Paris Seminar.pptxDigital Identity is Under Attack: FIDO Paris Seminar.pptx
Digital Identity is Under Attack: FIDO Paris Seminar.pptxLoriGlavin3
 
Merck Moving Beyond Passwords: FIDO Paris Seminar.pptx
Merck Moving Beyond Passwords: FIDO Paris Seminar.pptxMerck Moving Beyond Passwords: FIDO Paris Seminar.pptx
Merck Moving Beyond Passwords: FIDO Paris Seminar.pptxLoriGlavin3
 
TeamStation AI System Report LATAM IT Salaries 2024
TeamStation AI System Report LATAM IT Salaries 2024TeamStation AI System Report LATAM IT Salaries 2024
TeamStation AI System Report LATAM IT Salaries 2024Lonnie McRorey
 
Are Multi-Cloud and Serverless Good or Bad?
Are Multi-Cloud and Serverless Good or Bad?Are Multi-Cloud and Serverless Good or Bad?
Are Multi-Cloud and Serverless Good or Bad?Mattias Andersson
 
Ensuring Technical Readiness For Copilot in Microsoft 365
Ensuring Technical Readiness For Copilot in Microsoft 365Ensuring Technical Readiness For Copilot in Microsoft 365
Ensuring Technical Readiness For Copilot in Microsoft 3652toLead Limited
 
Scanning the Internet for External Cloud Exposures via SSL Certs
Scanning the Internet for External Cloud Exposures via SSL CertsScanning the Internet for External Cloud Exposures via SSL Certs
Scanning the Internet for External Cloud Exposures via SSL CertsRizwan Syed
 
DevEX - reference for building teams, processes, and platforms
DevEX - reference for building teams, processes, and platformsDevEX - reference for building teams, processes, and platforms
DevEX - reference for building teams, processes, and platformsSergiu Bodiu
 
The Ultimate Guide to Choosing WordPress Pros and Cons
The Ultimate Guide to Choosing WordPress Pros and ConsThe Ultimate Guide to Choosing WordPress Pros and Cons
The Ultimate Guide to Choosing WordPress Pros and ConsPixlogix Infotech
 
How to write a Business Continuity Plan
How to write a Business Continuity PlanHow to write a Business Continuity Plan
How to write a Business Continuity PlanDatabarracks
 
DSPy a system for AI to Write Prompts and Do Fine Tuning
DSPy a system for AI to Write Prompts and Do Fine TuningDSPy a system for AI to Write Prompts and Do Fine Tuning
DSPy a system for AI to Write Prompts and Do Fine TuningLars Bell
 
From Family Reminiscence to Scholarly Archive .
From Family Reminiscence to Scholarly Archive .From Family Reminiscence to Scholarly Archive .
From Family Reminiscence to Scholarly Archive .Alan Dix
 
Transcript: New from BookNet Canada for 2024: Loan Stars - Tech Forum 2024
Transcript: New from BookNet Canada for 2024: Loan Stars - Tech Forum 2024Transcript: New from BookNet Canada for 2024: Loan Stars - Tech Forum 2024
Transcript: New from BookNet Canada for 2024: Loan Stars - Tech Forum 2024BookNet Canada
 
Tampa BSides - Chef's Tour of Microsoft Security Adoption Framework (SAF)
Tampa BSides - Chef's Tour of Microsoft Security Adoption Framework (SAF)Tampa BSides - Chef's Tour of Microsoft Security Adoption Framework (SAF)
Tampa BSides - Chef's Tour of Microsoft Security Adoption Framework (SAF)Mark Simos
 

Último (20)

Passkey Providers and Enabling Portability: FIDO Paris Seminar.pptx
Passkey Providers and Enabling Portability: FIDO Paris Seminar.pptxPasskey Providers and Enabling Portability: FIDO Paris Seminar.pptx
Passkey Providers and Enabling Portability: FIDO Paris Seminar.pptx
 
Advanced Computer Architecture – An Introduction
Advanced Computer Architecture – An IntroductionAdvanced Computer Architecture – An Introduction
Advanced Computer Architecture – An Introduction
 
Unleash Your Potential - Namagunga Girls Coding Club
Unleash Your Potential - Namagunga Girls Coding ClubUnleash Your Potential - Namagunga Girls Coding Club
Unleash Your Potential - Namagunga Girls Coding Club
 
The Fit for Passkeys for Employee and Consumer Sign-ins: FIDO Paris Seminar.pptx
The Fit for Passkeys for Employee and Consumer Sign-ins: FIDO Paris Seminar.pptxThe Fit for Passkeys for Employee and Consumer Sign-ins: FIDO Paris Seminar.pptx
The Fit for Passkeys for Employee and Consumer Sign-ins: FIDO Paris Seminar.pptx
 
"Debugging python applications inside k8s environment", Andrii Soldatenko
"Debugging python applications inside k8s environment", Andrii Soldatenko"Debugging python applications inside k8s environment", Andrii Soldatenko
"Debugging python applications inside k8s environment", Andrii Soldatenko
 
Streamlining Python Development: A Guide to a Modern Project Setup
Streamlining Python Development: A Guide to a Modern Project SetupStreamlining Python Development: A Guide to a Modern Project Setup
Streamlining Python Development: A Guide to a Modern Project Setup
 
Connect Wave/ connectwave Pitch Deck Presentation
Connect Wave/ connectwave Pitch Deck PresentationConnect Wave/ connectwave Pitch Deck Presentation
Connect Wave/ connectwave Pitch Deck Presentation
 
Digital Identity is Under Attack: FIDO Paris Seminar.pptx
Digital Identity is Under Attack: FIDO Paris Seminar.pptxDigital Identity is Under Attack: FIDO Paris Seminar.pptx
Digital Identity is Under Attack: FIDO Paris Seminar.pptx
 
Merck Moving Beyond Passwords: FIDO Paris Seminar.pptx
Merck Moving Beyond Passwords: FIDO Paris Seminar.pptxMerck Moving Beyond Passwords: FIDO Paris Seminar.pptx
Merck Moving Beyond Passwords: FIDO Paris Seminar.pptx
 
TeamStation AI System Report LATAM IT Salaries 2024
TeamStation AI System Report LATAM IT Salaries 2024TeamStation AI System Report LATAM IT Salaries 2024
TeamStation AI System Report LATAM IT Salaries 2024
 
Are Multi-Cloud and Serverless Good or Bad?
Are Multi-Cloud and Serverless Good or Bad?Are Multi-Cloud and Serverless Good or Bad?
Are Multi-Cloud and Serverless Good or Bad?
 
Ensuring Technical Readiness For Copilot in Microsoft 365
Ensuring Technical Readiness For Copilot in Microsoft 365Ensuring Technical Readiness For Copilot in Microsoft 365
Ensuring Technical Readiness For Copilot in Microsoft 365
 
Scanning the Internet for External Cloud Exposures via SSL Certs
Scanning the Internet for External Cloud Exposures via SSL CertsScanning the Internet for External Cloud Exposures via SSL Certs
Scanning the Internet for External Cloud Exposures via SSL Certs
 
DevEX - reference for building teams, processes, and platforms
DevEX - reference for building teams, processes, and platformsDevEX - reference for building teams, processes, and platforms
DevEX - reference for building teams, processes, and platforms
 
The Ultimate Guide to Choosing WordPress Pros and Cons
The Ultimate Guide to Choosing WordPress Pros and ConsThe Ultimate Guide to Choosing WordPress Pros and Cons
The Ultimate Guide to Choosing WordPress Pros and Cons
 
How to write a Business Continuity Plan
How to write a Business Continuity PlanHow to write a Business Continuity Plan
How to write a Business Continuity Plan
 
DSPy a system for AI to Write Prompts and Do Fine Tuning
DSPy a system for AI to Write Prompts and Do Fine TuningDSPy a system for AI to Write Prompts and Do Fine Tuning
DSPy a system for AI to Write Prompts and Do Fine Tuning
 
From Family Reminiscence to Scholarly Archive .
From Family Reminiscence to Scholarly Archive .From Family Reminiscence to Scholarly Archive .
From Family Reminiscence to Scholarly Archive .
 
Transcript: New from BookNet Canada for 2024: Loan Stars - Tech Forum 2024
Transcript: New from BookNet Canada for 2024: Loan Stars - Tech Forum 2024Transcript: New from BookNet Canada for 2024: Loan Stars - Tech Forum 2024
Transcript: New from BookNet Canada for 2024: Loan Stars - Tech Forum 2024
 
Tampa BSides - Chef's Tour of Microsoft Security Adoption Framework (SAF)
Tampa BSides - Chef's Tour of Microsoft Security Adoption Framework (SAF)Tampa BSides - Chef's Tour of Microsoft Security Adoption Framework (SAF)
Tampa BSides - Chef's Tour of Microsoft Security Adoption Framework (SAF)
 

Apache Flink Training: DataStream API Part 1 Basic

  • 1. Apache Flink® Training DataStream API Basic August 26, 2015
  • 2. DataStream API  Stream Processing  Java and Scala  All examples here in Java  Documentation available at flink.apache.org  Currently labeled as beta – some API changes are pending • Noted in the slides with a warning 2
  • 3. DataStream API by Example 3
  • 4. Window WordCount: main Method 4 public static void main(String[] args) throws Exception { // set up the execution environment final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); DataSet<Tuple2<String, Integer>> counts = env // read stream of words from socket .socketTextStream("localhost", 9999) // split up the lines in tuples containing: (word,1) .flatMap(new Splitter()) // group by the tuple field "0" .groupBy(0) // keep the last 5 minute of data .window(Time.of(5, TimeUnit.MINUTES)) //sum up tuple field "1" .sum(1); // print result in command line counts.print(); // execute program env.execute("Socket Incremental WordCount Example"); }
  • 5. Stream Execution Environment 5 public static void main(String[] args) throws Exception { // set up the execution environment final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); DataSet<Tuple2<String, Integer>> counts = env // read stream of words from socket .socketTextStream("localhost", 9999) // split up the lines in tuples containing: (word,1) .flatMap(new Splitter()) // group by the tuple field "0" .groupBy(0) // keep the last 5 minute of data .window(Time.of(5, TimeUnit.MINUTES)) //sum up tuple field "1" .sum(1); // print result in command line counts.print(); // execute program env.execute("Socket Incremental WordCount Example"); }
  • 6. Data Sources 6 public static void main(String[] args) throws Exception { // set up the execution environment final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); DataSet<Tuple2<String, Integer>> counts = env // read stream of words from socket .socketTextStream("localhost", 9999) // split up the lines in tuples containing: (word,1) .flatMap(new Splitter()) // group by the tuple field "0" .groupBy(0) // keep the last 5 minute of data .window(Time.of(5, TimeUnit.MINUTES)) //sum up tuple field "1" .sum(1); // print result in command line counts.print(); // execute program env.execute("Socket Incremental WordCount Example"); }
  • 7. Data types 7 public static void main(String[] args) throws Exception { // set up the execution environment final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); DataSet<Tuple2<String, Integer>> counts = env // read stream of words from socket .socketTextStream("localhost", 9999) // split up the lines in tuples containing: (word,1) .flatMap(new Splitter()) // group by the tuple field "0" .groupBy(0) // keep the last 5 minute of data .window(Time.of(5, TimeUnit.MINUTES)) //sum up tuple field "1" .sum(1); // print result in command line counts.print(); // execute program env.execute("Socket Incremental WordCount Example"); }
  • 8. Transformations 8 public static void main(String[] args) throws Exception { // set up the execution environment final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); DataSet<Tuple2<String, Integer>> counts = env // read stream of words from socket .socketTextStream("localhost", 9999) // split up the lines in tuples containing: (word,1) .flatMap(new Splitter()) // group by the tuple field "0" .groupBy(0) // keep the last 5 minute of data .window(Time.of(5, TimeUnit.MINUTES)) //sum up tuple field "1" .sum(1); // print result in command line counts.print(); // execute program env.execute("Socket Incremental WordCount Example"); }
  • 9. User functions 9 public static void main(String[] args) throws Exception { // set up the execution environment final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); DataSet<Tuple2<String, Integer>> counts = env // read stream of words from socket .socketTextStream("localhost", 9999) // split up the lines in tuples containing: (word,1) .flatMap(new Splitter()) // group by the tuple field "0" .groupBy(0) // keep the last 5 minute of data .window(Time.of(5, TimeUnit.MINUTES)) //sum up tuple field "1" .sum(1); // print result in command line counts.print(); // execute program env.execute("Socket Incremental WordCount Example"); }
  • 10. DataSinks 10 public static void main(String[] args) throws Exception { // set up the execution environment final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); DataSet<Tuple2<String, Integer>> counts = env // read stream of words from socket .socketTextStream("localhost", 9999) // split up the lines in tuples containing: (word,1) .flatMap(new Splitter()) // group by the tuple field "0" .groupBy(0) // keep the last 5 minute of data .window(Time.of(5, TimeUnit.MINUTES)) //sum up tuple field "1" .sum(1); // print result in command line counts.print(); // execute program env.execute("Socket Incremental WordCount Example"); }
  • 11. Execute! 11 public static void main(String[] args) throws Exception { // set up the execution environment final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); DataSet<Tuple2<String, Integer>> counts = env // read stream of words from socket .socketTextStream("localhost", 9999) // split up the lines in tuples containing: (word,1) .flatMap(new Splitter()) // group by the tuple field "0" .groupBy(0) // keep the last 5 minute of data .window(Time.of(5, TimeUnit.MINUTES)) //sum up tuple field "1" .sum(1); // print result in command line counts.print(); // execute program env.execute("Socket Incremental WordCount Example"); }
  • 12. Window WordCount: FlatMap public static class Splitter implements FlatMapFunction<String, Tuple2<String, Integer>> { @Override public void flatMap(String value, Collector<Tuple2<String, Integer>> out) throws Exception { // normalize and split the line String[] tokens = value.toLowerCase().split("W+"); // emit the pairs for (String token : tokens) { if (token.length() > 0) { out.collect( new Tuple2<String, Integer>(token, 1)); } } } } 12
  • 13. WordCount: Map: Interface 13 public static class Splitter implements FlatMapFunction<String, Tuple2<String, Integer>> { @Override public void flatMap(String value, Collector<Tuple2<String, Integer>> out) throws Exception { // normalize and split the line String[] tokens = value.toLowerCase().split("W+"); // emit the pairs for (String token : tokens) { if (token.length() > 0) { out.collect( new Tuple2<String, Integer>(token, 1)); } } } }
  • 14. WordCount: Map: Types 14 public static class Splitter implements FlatMapFunction<String, Tuple2<String, Integer>> { @Override public void flatMap(String value, Collector<Tuple2<String, Integer>> out) throws Exception { // normalize and split the line String[] tokens = value.toLowerCase().split("W+"); // emit the pairs for (String token : tokens) { if (token.length() > 0) { out.collect( new Tuple2<String, Integer>(token, 1)); } } } }
  • 15. WordCount: Map: Collector 15 public static class Splitter implements FlatMapFunction<String, Tuple2<String, Integer>> { @Override public void flatMap(String value, Collector<Tuple2<String, Integer>> out) throws Exception { // normalize and split the line String[] tokens = value.toLowerCase().split("W+"); // emit the pairs for (String token : tokens) { if (token.length() > 0) { out.collect( new Tuple2<String, Integer>(token, 1)); } } } }
  • 17. (Selected) Data Types  Basic Java Types • String, Long, Integer, Boolean,… • Arrays  Composite Types • Tuples • Many more (covered in the advanced slides) 17
  • 18. Tuples  The easiest and most lightweight way of encapsulating data in Flink  Tuple1 up to Tuple25 Tuple2<String, String> person = new Tuple2<>("Max", "Mustermann”); Tuple3<String, String, Integer> person = new Tuple3<>("Max", "Mustermann", 42); Tuple4<String, String, Integer, Boolean> person = new Tuple4<>("Max", "Mustermann", 42, true); // zero based index! String firstName = person.f0; String secondName = person.f1; Integer age = person.f2; Boolean fired = person.f3; 18
  • 19. Transformations: Map DataStream<Integer> integers = env.fromElements(1, 2, 3, 4); // Regular Map - Takes one element and produces one element DataStream<Integer> doubleIntegers = integers.map(new MapFunction<Integer, Integer>() { @Override public Integer map(Integer value) { return value * 2; } }); doubleIntegers.print(); > 2, 4, 6, 8 // Flat Map - Takes one element and produces zero, one, or more elements. DataStream<Integer> doubleIntegers2 = integers.flatMap(new FlatMapFunction<Integer, Integer>() { @Override public void flatMap(Integer value, Collector<Integer> out) { out.collect(value * 2); } }); doubleIntegers2.print(); > 2, 4, 6, 8 19
  • 20. Transformations: Filter // The DataStream DataStream<Integer> integers = env.fromElements(1, 2, 3, 4); DataStream<Integer> filtered = integers.filter(new FilterFunction<Integer>() { @Override public boolean filter(Integer value) { return value != 3; } }); integers.print(); > 1, 2, 4 20
  • 21. Transformations: Partitioning  DataStreams can be partitioned by a key 21 // (name, age) of employees DataStream<Tuple2<String, Integer>> passengers = … // group by second field (age) DataStream<Integer, Integer> grouped = passengers.groupBy(1) Stephan, 18 Fabian, 23 Julia, 27 Anna, 18 Romeo, 27 Anna, 18 Stephan, 18 Julia, 27 Romeo, 27 Fabian, 23 Warning: Possible renaming in next releasesBen, 25 Ben, 25
  • 22. Data Shipping Strategies  Optionally, you can specify how data is shipped between two transformations  Forward: stream.forward() • Only local communication  Rebalance: stream.rebalance() • Round-robin partitioning  Partition by hash: stream.partitionByHash(...)  Custom partitioning: stream.partitionCustom(...)  Broadcast: stream.broadcast() • Broadcast to all nodes 22
  • 24. Data Sources (2) Text socket  socketTextStream("hostname",port) Text file  readFileStream(“/path/to/file”, 1000, WatchType.PROCESS_ONLY_APPENDED) Connectors  E.g., Apache Kafka, RabbitMQ, … 24
  • 25. Data Sources: Collections StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); // read from elements DataStream<String> names = env.fromElements(“Some”, “Example”, “Strings”); // read from Java collection List<String> list = new ArrayList<String>(); list.add(“Some”); list.add(“Example”); list.add(“Strings”); DataStream<String> names = env.fromCollection(list); 25
  • 26. Data Sources: Files,Sockets,Connectors StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); // read text socket from port DataStream<String> socketLines = env .socketTextStream(”localhost", 9999); // read a text file ingesting new elements every 100 milliseconds DataStream<String> localLines = env .readFileStream(”/path/to/file", 1000, WatchType.PROCESS_ONLY_APPENDED); 26
  • 27. Data Sinks Text  writeAsText(“/path/to/file”) CSV  writeAsCsv(“/path/to/file”) Return data to the Client  print() 27 Note: Identical to DataSet API
  • 28. Data Sinks (2) Socket  writeToSocket(hostname, port, SerializationSchema) Connectors  E.g., Apache Kafka, Elasticsearch, Rolling HDFS Files 28
  • 29. Data Sinks  Lazily executed when env.execute() is called DataStream<…> result; // nothing happens result.writeToSocket(...); // nothing happens result.writeAsText("/path/to/file", "n", "|"); // Execution really starts here env.execute(); 29
  • 31. Fault Tolerance in Flink  Flink provides recovery by taking a consistent checkpoint every N milliseconds and rolling back to the checkpointed state • https://ci.apache.org/projects/flink/flink-docs- master/internals/stream_checkpointing.html  Exactly once (default) • // Take checkpoint every 5000 milliseconds env.enableCheckpointing (5000)  At least once (for lower latency) • // Take checkpoint every 5000 milliseconds env.enableCheckpointing (5000, CheckpointingMode.AT_LEAST_ONCE)  Setting the interval to few seconds should be good for most applications  If checkpointing is not enabled, no recovery guarantees are provided 31
  • 33. Some advice  Use env.fromElements(..) or env.fromCollection(..) to quickly get a DataStream to experiment with  Use print() to quickly print a DataStream 33
  • 35. From 0.9 to 0.10  groupBy(…) -> keyBy(…)  DataStream renames: • KeyedDataStream -> KeyedStream • WindowedDataStream -> WindowedStream • ConnectedDataStream -> ConnectedStream • JoinOperator -> JoinedStreams 35
  • 36. 36