DataStream programs in Flink are regular programs that implement transformations on data streams (e.g., filtering, updating state, defining windows, aggregating). The data streams are initially created from various sources (e.g., message queues, socket streams, files). Results are returned via sinks, which may for example write the data to files, or to standard output (for example the command line terminal). Flink programs run in a variety of contexts, standalone, or embedded in other programs. The execution can happen in a local JVM, or on clusters of many machines.
In order to create your own Flink DataStream program, we encourage you to start with anatomy of a Flink Program and gradually add your own stream transformations. The remaining sections act as references for additional operations and advanced features.
The DataStream API gets its name from the special DataStream
class that is
used to represent a collection of data in a Flink program. You can think of
them as immutable collections of data that can contain duplicates. This data
can either be finite or unbounded, the API that you use to work on them is the
same.
A DataStream
is similar to a regular Java Collection
in terms of usage but
is quite different in some key ways. They are immutable, meaning that once they
are created you cannot add or remove elements. You can also not simply inspect
the elements inside but only work on them using the DataStream
API
operations, which are also called transformations.
You can create an initial DataStream
by adding a source in a Flink program.
Then you can derive new streams from this and combine them by using API methods
such as map
, filter
, and so on.
Flink programs look like regular programs that transform DataStreams
. Each
program consists of the same basic parts:
execution environment
,We will now give an overview of each of those steps, please refer to the respective sections for more details. Note that all core classes of the Java DataStream API can be found in org.apache.flink.streaming.api.
The StreamExecutionEnvironment
is the basis for all Flink programs. You can
obtain one using these static methods on StreamExecutionEnvironment
:
Typically, you only need to use getExecutionEnvironment()
, since this will do
the right thing depending on the context: if you are executing your program
inside an IDE or as a regular Java program it will create a local environment
that will execute your program on your local machine. If you created a JAR file
from your program, and invoke it through the command line, the Flink cluster manager will execute your main method and
getExecutionEnvironment()
will return an execution environment for executing
your program on a cluster.
For specifying data sources the execution environment has several methods to read from files using various methods: you can just read them line by line, as CSV files, or using any of the other provided sources. To just read a text file as a sequence of lines, you can use:
This will give you a DataStream on which you can then apply transformations to create new derived DataStreams.
You apply transformations by calling methods on DataStream with a transformation functions. For example, a map transformation looks like this:
This will create a new DataStream by converting every String in the original collection to an Integer.
Once you have a DataStream containing your final results, you can write it to an outside system by creating a sink. These are just some example methods for creating a sink:
We will now give an overview of each of those steps, please refer to the respective sections for more details. Note that all core classes of the Scala DataStream API can be found in org.apache.flink.streaming.api.scala.
The StreamExecutionEnvironment
is the basis for all Flink programs. You can
obtain one using these static methods on StreamExecutionEnvironment
:
Typically, you only need to use getExecutionEnvironment()
, since this will do
the right thing depending on the context: if you are executing your program
inside an IDE or as a regular Java program it will create a local environment
that will execute your program on your local machine. If you created a JAR file
from your program, and invoke it through the command line, the Flink cluster manager will execute your main method and
getExecutionEnvironment()
will return an execution environment for executing
your program on a cluster.
For specifying data sources the execution environment has several methods to read from files using various methods: you can just read them line by line, as CSV files, or using any of the other provided sources. To just read a text file as a sequence of lines, you can use:
This will give you a DataStream on which you can then apply transformations to create new derived DataStreams.
You apply transformations by calling methods on DataStream with a transformation functions. For example, a map transformation looks like this:
This will create a new DataStream by converting every String in the original collection to an Integer.
Once you have a DataStream containing your final results, you can write it to an outside system by creating a sink. These are just some example methods for creating a sink:
Once you specified the complete program you need to trigger the program
execution by calling execute()
on the StreamExecutionEnvironment
.
Depending on the type of the ExecutionEnvironment
the execution will be
triggered on your local machine or submit your program for execution on a
cluster.
The execute()
method will wait for the job to finish and then return a
JobExecutionResult
, this contains execution times and accumulator results.
If you don’t want to wait for the job to finish, you can trigger asynchronous
job execution by calling executeAysnc()
on the StreamExecutionEnvironment
.
It will return a JobClient
with which you can communicate with the job you
just submitted. For instance, here is how to implement the semantics of
execute()
by using executeAsync()
.
That last part about program execution is crucial to understanding when and how
Flink operations are executed. All Flink programs are executed lazily: When the
program’s main method is executed, the data loading and transformations do not
happen directly. Rather, each operation is created and added to a dataflow
graph. The operations are actually executed when the execution is explicitly
triggered by an execute()
call on the execution environment. Whether the
program is executed locally or on a cluster depends on the type of execution
environment
The lazy evaluation lets you construct sophisticated programs that Flink executes as one holistically planned unit.
The following program is a complete, working example of streaming window word count application, that counts the words coming from a web socket in 5 second windows. You can copy & paste the code to run it locally.
To run the example program, start the input stream with netcat first from a terminal:
Just type some words hitting return for a new word. These will be the input to the word count program. If you want to see counts greater than 1, type the same word again and again within 5 seconds (increase the window size from 5 seconds if you cannot type that fast ☺).
Sources are where your program reads its input from. You can attach a source to your program by
using StreamExecutionEnvironment.addSource(sourceFunction)
. Flink comes with a number of pre-implemented
source functions, but you can always write your own custom sources by implementing the SourceFunction
for non-parallel sources, or by implementing the ParallelSourceFunction
interface or extending the
RichParallelSourceFunction
for parallel sources.
There are several predefined stream sources accessible from the StreamExecutionEnvironment
:
File-based:
readTextFile(path)
- Reads text files, i.e. files that respect the TextInputFormat
specification, line-by-line and returns them as Strings.
readFile(fileInputFormat, path)
- Reads (once) files as dictated by the specified file input format.
readFile(fileInputFormat, path, watchType, interval, pathFilter, typeInfo)
- This is the method called internally by the two previous ones. It reads files in the path
based on the given fileInputFormat
. Depending on the provided watchType
, this source may periodically monitor (every interval
ms) the path for new data (FileProcessingMode.PROCESS_CONTINUOUSLY
), or process once the data currently in the path and exit (FileProcessingMode.PROCESS_ONCE
). Using the pathFilter
, the user can further exclude files from being processed.
IMPLEMENTATION:
Under the hood, Flink splits the file reading process into two sub-tasks, namely directory monitoring and data reading. Each of these sub-tasks is implemented by a separate entity. Monitoring is implemented by a single, non-parallel (parallelism = 1) task, while reading is performed by multiple tasks running in parallel. The parallelism of the latter is equal to the job parallelism. The role of the single monitoring task is to scan the directory (periodically or only once depending on the watchType
), find the files to be processed, divide them in splits, and assign these splits to the downstream readers. The readers are the ones who will read the actual data. Each split is read by only one reader, while a reader can read multiple splits, one-by-one.
IMPORTANT NOTES:
If the watchType
is set to FileProcessingMode.PROCESS_CONTINUOUSLY
, when a file is modified, its contents are re-processed entirely. This can break the “exactly-once” semantics, as appending data at the end of a file will lead to all its contents being re-processed.
If the watchType
is set to FileProcessingMode.PROCESS_ONCE
, the source scans the path once and exits, without waiting for the readers to finish reading the file contents. Of course the readers will continue reading until all file contents are read. Closing the source leads to no more checkpoints after that point. This may lead to slower recovery after a node failure, as the job will resume reading from the last checkpoint.
Socket-based:
socketTextStream
- Reads from a socket. Elements can be separated by a delimiter.Collection-based:
fromCollection(Collection)
- Creates a data stream from the Java Java.util.Collection. All elements
in the collection must be of the same type.
fromCollection(Iterator, Class)
- Creates a data stream from an iterator. The class specifies the
data type of the elements returned by the iterator.
fromElements(T ...)
- Creates a data stream from the given sequence of objects. All objects must be
of the same type.
fromParallelCollection(SplittableIterator, Class)
- Creates a data stream from an iterator, in
parallel. The class specifies the data type of the elements returned by the iterator.
generateSequence(from, to)
- Generates the sequence of numbers in the given interval, in
parallel.
Custom:
addSource
- Attach a new source function. For example, to read from Apache Kafka you can use
addSource(new FlinkKafkaConsumer<>(...))
. See connectors for more details.Sources are where your program reads its input from. You can attach a source to your program by
using StreamExecutionEnvironment.addSource(sourceFunction)
. Flink comes with a number of pre-implemented
source functions, but you can always write your own custom sources by implementing the SourceFunction
for non-parallel sources, or by implementing the ParallelSourceFunction
interface or extending the
RichParallelSourceFunction
for parallel sources.
There are several predefined stream sources accessible from the StreamExecutionEnvironment
:
File-based:
readTextFile(path)
- Reads text files, i.e. files that respect the TextInputFormat
specification, line-by-line and returns them as Strings.
readFile(fileInputFormat, path)
- Reads (once) files as dictated by the specified file input format.
readFile(fileInputFormat, path, watchType, interval, pathFilter)
- This is the method called internally by the two previous ones. It reads files in the path
based on the given fileInputFormat
. Depending on the provided watchType
, this source may periodically monitor (every interval
ms) the path for new data (FileProcessingMode.PROCESS_CONTINUOUSLY
), or process once the data currently in the path and exit (FileProcessingMode.PROCESS_ONCE
). Using the pathFilter
, the user can further exclude files from being processed.
IMPLEMENTATION:
Under the hood, Flink splits the file reading process into two sub-tasks, namely directory monitoring and data reading. Each of these sub-tasks is implemented by a separate entity. Monitoring is implemented by a single, non-parallel (parallelism = 1) task, while reading is performed by multiple tasks running in parallel. The parallelism of the latter is equal to the job parallelism. The role of the single monitoring task is to scan the directory (periodically or only once depending on the watchType
), find the files to be processed, divide them in splits, and assign these splits to the downstream readers. The readers are the ones who will read the actual data. Each split is read by only one reader, while a reader can read multiple splits, one-by-one.
IMPORTANT NOTES:
If the watchType
is set to FileProcessingMode.PROCESS_CONTINUOUSLY
, when a file is modified, its contents are re-processed entirely. This can break the “exactly-once” semantics, as appending data at the end of a file will lead to all its contents being re-processed.
If the watchType
is set to FileProcessingMode.PROCESS_ONCE
, the source scans the path once and exits, without waiting for the readers to finish reading the file contents. Of course the readers will continue reading until all file contents are read. Closing the source leads to no more checkpoints after that point. This may lead to slower recovery after a node failure, as the job will resume reading from the last checkpoint.
Socket-based:
socketTextStream
- Reads from a socket. Elements can be separated by a delimiter.Collection-based:
fromCollection(Seq)
- Creates a data stream from the Java Java.util.Collection. All elements
in the collection must be of the same type.
fromCollection(Iterator)
- Creates a data stream from an iterator. The class specifies the
data type of the elements returned by the iterator.
fromElements(elements: _*)
- Creates a data stream from the given sequence of objects. All objects must be
of the same type.
fromParallelCollection(SplittableIterator)
- Creates a data stream from an iterator, in
parallel. The class specifies the data type of the elements returned by the iterator.
generateSequence(from, to)
- Generates the sequence of numbers in the given interval, in
parallel.
Custom:
addSource
- Attach a new source function. For example, to read from Apache Kafka you can use
addSource(new FlinkKafkaConsumer<>(...))
. See connectors for more details.Please see operators for an overview of the available stream transformations.
Data sinks consume DataStreams and forward them to files, sockets, external systems, or print them. Flink comes with a variety of built-in output formats that are encapsulated behind operations on the DataStreams:
writeAsText()
/ TextOutputFormat
- Writes elements line-wise as Strings. The Strings are
obtained by calling the toString() method of each element.
writeAsCsv(...)
/ CsvOutputFormat
- Writes tuples as comma-separated value files. Row and field
delimiters are configurable. The value for each field comes from the toString() method of the objects.
print()
/ printToErr()
- Prints the toString() value
of each element on the standard out / standard error stream. Optionally, a prefix (msg) can be provided which is
prepended to the output. This can help to distinguish between different calls to print. If the parallelism is
greater than 1, the output will also be prepended with the identifier of the task which produced the output.
writeUsingOutputFormat()
/ FileOutputFormat
- Method and base class for custom file outputs. Supports
custom object-to-bytes conversion.
writeToSocket
- Writes elements to a socket according to a SerializationSchema
addSink
- Invokes a custom sink function. Flink comes bundled with connectors to other systems (such as
Apache Kafka) that are implemented as sink functions.
Data sinks consume DataStreams and forward them to files, sockets, external systems, or print them. Flink comes with a variety of built-in output formats that are encapsulated behind operations on the DataStreams:
writeAsText()
/ TextOutputFormat
- Writes elements line-wise as Strings. The Strings are
obtained by calling the toString() method of each element.
writeAsCsv(...)
/ CsvOutputFormat
- Writes tuples as comma-separated value files. Row and field
delimiters are configurable. The value for each field comes from the toString() method of the objects.
print()
/ printToErr()
- Prints the toString() value
of each element on the standard out / standard error stream. Optionally, a prefix (msg) can be provided which is
prepended to the output. This can help to distinguish between different calls to print. If the parallelism is
greater than 1, the output will also be prepended with the identifier of the task which produced the output.
writeUsingOutputFormat()
/ FileOutputFormat
- Method and base class for custom file outputs. Supports
custom object-to-bytes conversion.
writeToSocket
- Writes elements to a socket according to a SerializationSchema
addSink
- Invokes a custom sink function. Flink comes bundled with connectors to other systems (such as
Apache Kafka) that are implemented as sink functions.
Note that the write*()
methods on DataStream
are mainly intended for debugging purposes.
They are not participating in Flink’s checkpointing, this means these functions usually have
at-least-once semantics. The data flushing to the target system depends on the implementation of the
OutputFormat. This means that not all elements send to the OutputFormat are immediately showing up
in the target system. Also, in failure cases, those records might be lost.
For reliable, exactly-once delivery of a stream into a file system, use the StreamingFileSink
.
Also, custom implementations through the .addSink(...)
method can participate in Flink’s checkpointing
for exactly-once semantics.
Iterative streaming programs implement a step function and embed it into an IterativeStream
. As a DataStream
program may never finish, there is no maximum number of iterations. Instead, you need to specify which part
of the stream is fed back to the iteration and which part is forwarded downstream using a side output
or a filter
. Here, we show an example using filters. First, we define an IterativeStream
Then, we specify the logic that will be executed inside the loop using a series of transformations (here
a simple map
transformation)
To close an iteration and define the iteration tail, call the closeWith(feedbackStream)
method of the IterativeStream
.
The DataStream given to the closeWith
function will be fed back to the iteration head.
A common pattern is to use a filter to separate the part of the stream that is fed back,
and the part of the stream which is propagated forward. These filters can, e.g., define
the “termination” logic, where an element is allowed to propagate downstream rather
than being fed back.
For example, here is program that continuously subtracts 1 from a series of integers until they reach zero:
Iterative streaming programs implement a step function and embed it into an IterativeStream
. As a DataStream
program may never finish, there is no maximum number of iterations. Instead, you need to specify which part
of the stream is fed back to the iteration and which part is forwarded downstream using a side output
or a filter
. Here, we show an example iteration where the body (the part of the computation that is repeated)
is a simple map transformation, and the elements that are fed back are distinguished by the elements that
are forwarded downstream using filters.
For example, here is program that continuously subtracts 1 from a series of integers until they reach zero:
The StreamExecutionEnvironment
contains the ExecutionConfig
which allows to set job specific configuration values for the runtime.
Please refer to execution configuration for an explanation of most parameters. These parameters pertain specifically to the DataStream API:
setAutoWatermarkInterval(long milliseconds)
: Set the interval for automatic watermark emission. You can
get the current value with long getAutoWatermarkInterval()
State & Checkpointing describes how to enable and configure Flink’s checkpointing mechanism.
By default, elements are not transferred on the network one-by-one (which would cause unnecessary network traffic)
but are buffered. The size of the buffers (which are actually transferred between machines) can be set in the Flink config files.
While this method is good for optimizing throughput, it can cause latency issues when the incoming stream is not fast enough.
To control throughput and latency, you can use env.setBufferTimeout(timeoutMillis)
on the execution environment
(or on individual operators) to set a maximum wait time for the buffers to fill up. After this time, the
buffers are sent automatically even if they are not full. The default value for this timeout is 100 ms.
Usage:
To maximize throughput, set setBufferTimeout(-1)
which will remove the timeout and buffers will only be
flushed when they are full. To minimize latency, set the timeout to a value close to 0 (for example 5 or 10 ms).
A buffer timeout of 0 should be avoided, because it can cause severe performance degradation.
Before running a streaming program in a distributed cluster, it is a good idea to make sure that the implemented algorithm works as desired. Hence, implementing data analysis programs is usually an incremental process of checking results, debugging, and improving.
Flink provides features to significantly ease the development process of data analysis programs by supporting local debugging from within an IDE, injection of test data, and collection of result data. This section give some hints how to ease the development of Flink programs.
A LocalStreamEnvironment
starts a Flink system within the same JVM process it was created in. If you
start the LocalEnvironment from an IDE, you can set breakpoints in your code and easily debug your
program.
A LocalEnvironment is created and used as follows:
Flink provides special data sources which are backed by Java collections to ease testing. Once a program has been tested, the sources and sinks can be easily replaced by sources and sinks that read from / write to external systems.
Collection data sources can be used as follows:
Note: Currently, the collection data source requires that data types and iterators implement
Serializable
. Furthermore, collection data sources can not be executed in parallel (
parallelism = 1).
Flink also provides a sink to collect DataStream results for testing and debugging purposes. It can be used as follows:
Note: flink-streaming-contrib
module is removed from Flink 1.5.0.
Its classes have been moved into flink-streaming-java
and flink-streaming-scala
.