Operators transform one or more DataStreams into a new DataStream. Programs can combine multiple transformations into sophisticated dataflow topologies.
This section gives a description of the basic transformations, the effective physical partitioning after applying those as well as insights into Flink’s operator chaining.
Transformation | Description |
---|---|
MapDataStream → DataStream |
Takes one element and produces one element. A map function that doubles the values of the input stream: |
FlatMapDataStream → DataStream |
Takes one element and produces zero, one, or more elements. A flatmap function that splits sentences to words: |
FilterDataStream → DataStream |
Evaluates a boolean function for each element and retains those for which the function returns true. A filter that filters out zero values: |
KeyByDataStream → KeyedStream |
Logically partitions a stream into disjoint partitions. All records with the same key are assigned to the same partition. Internally, keyBy() is implemented with hash partitioning. There are different ways to specify keys. This transformation returns a KeyedStream, which is, among other things, required to use keyed state. Attention A type cannot be a key if:
|
ReduceKeyedStream → DataStream |
A "rolling" reduce on a keyed data stream. Combines the current element with the last reduced value and
emits the new value.
A reduce function that creates a stream of partial sums: |
AggregationsKeyedStream → DataStream |
Rolling aggregations on a keyed data stream. The difference between min and minBy is that min returns the minimum value, whereas minBy returns the element that has the minimum value in this field (same for max and maxBy). |
WindowKeyedStream → WindowedStream |
Windows can be defined on already partitioned KeyedStreams. Windows group the data in each key according to some characteristic (e.g., the data that arrived within the last 5 seconds). See windows for a complete description of windows. |
WindowAllDataStream → AllWindowedStream |
Windows can be defined on regular DataStreams. Windows group all the stream events according to some characteristic (e.g., the data that arrived within the last 5 seconds). See windows for a complete description of windows. WARNING: This is in many cases a non-parallel transformation. All records will be gathered in one task for the windowAll operator. |
Window ApplyWindowedStream → DataStreamAllWindowedStream → DataStream |
Applies a general function to the window as a whole. Below is a function that manually sums the elements of a window. Note: If you are using a windowAll transformation, you need to use an AllWindowFunction instead. |
Window ReduceWindowedStream → DataStream |
Applies a functional reduce function to the window and returns the reduced value. |
Aggregations on windowsWindowedStream → DataStream |
Aggregates the contents of a window. The difference between min and minBy is that min returns the minimum value, whereas minBy returns the element that has the minimum value in this field (same for max and maxBy). |
UnionDataStream* → DataStream |
Union of two or more data streams creating a new stream containing all the elements from all the streams. Note: If you union a data stream with itself you will get each element twice in the resulting stream. |
Window JoinDataStream,DataStream → DataStream |
Join two data streams on a given key and a common window. |
Interval JoinKeyedStream,KeyedStream → DataStream |
Join two elements e1 and e2 of two keyed streams with a common key over a given time interval, so that e1.timestamp + lowerBound <= e2.timestamp <= e1.timestamp + upperBound |
Window CoGroupDataStream,DataStream → DataStream |
Cogroups two data streams on a given key and a common window. |
ConnectDataStream,DataStream → ConnectedStreams |
"Connects" two data streams retaining their types. Connect allowing for shared state between the two streams. |
CoMap, CoFlatMapConnectedStreams → DataStream |
Similar to map and flatMap on a connected data stream |
IterateDataStream → IterativeStream → DataStream |
Creates a "feedback" loop in the flow, by redirecting the output of one operator to some previous operator. This is especially useful for defining algorithms that continuously update a model. The following code starts with a stream and applies the iteration body continuously. Elements that are greater than 0 are sent back to the feedback channel, and the rest of the elements are forwarded downstream. |
Transformation | Description |
---|---|
MapDataStream → DataStream |
Takes one element and produces one element. A map function that doubles the values of the input stream: |
FlatMapDataStream → DataStream |
Takes one element and produces zero, one, or more elements. A flatmap function that splits sentences to words: |
FilterDataStream → DataStream |
Evaluates a boolean function for each element and retains those for which the function returns true. A filter that filters out zero values: |
KeyByDataStream → KeyedStream |
Logically partitions a stream into disjoint partitions, each partition containing elements of the same key. Internally, this is implemented with hash partitioning. See keys on how to specify keys. This transformation returns a KeyedStream. |
ReduceKeyedStream → DataStream |
A "rolling" reduce on a keyed data stream. Combines the current element with the last reduced value and
emits the new value.
|
AggregationsKeyedStream → DataStream |
Rolling aggregations on a keyed data stream. The difference between min and minBy is that min returns the minimum value, whereas minBy returns the element that has the minimum value in this field (same for max and maxBy). |
WindowKeyedStream → WindowedStream |
Windows can be defined on already partitioned KeyedStreams. Windows group the data in each key according to some characteristic (e.g., the data that arrived within the last 5 seconds). See windows for a description of windows. |
WindowAllDataStream → AllWindowedStream |
Windows can be defined on regular DataStreams. Windows group all the stream events according to some characteristic (e.g., the data that arrived within the last 5 seconds). See windows for a complete description of windows. WARNING: This is in many cases a non-parallel transformation. All records will be gathered in one task for the windowAll operator. |
Window ApplyWindowedStream → DataStreamAllWindowedStream → DataStream |
Applies a general function to the window as a whole. Below is a function that manually sums the elements of a window. Note: If you are using a windowAll transformation, you need to use an AllWindowFunction instead. |
Window ReduceWindowedStream → DataStream |
Applies a functional reduce function to the window and returns the reduced value. |
Aggregations on windowsWindowedStream → DataStream |
Aggregates the contents of a window. The difference between min and minBy is that min returns the minimum value, whereas minBy returns the element that has the minimum value in this field (same for max and maxBy). |
UnionDataStream* → DataStream |
Union of two or more data streams creating a new stream containing all the elements from all the streams. Note: If you union a data stream with itself you will get each element twice in the resulting stream. |
Window JoinDataStream,DataStream → DataStream |
Join two data streams on a given key and a common window. |
Window CoGroupDataStream,DataStream → DataStream |
Cogroups two data streams on a given key and a common window. |
ConnectDataStream,DataStream → ConnectedStreams |
"Connects" two data streams retaining their types, allowing for shared state between the two streams. |
CoMap, CoFlatMapConnectedStreams → DataStream |
Similar to map and flatMap on a connected data stream |
IterateDataStream → IterativeStream → DataStream |
Creates a "feedback" loop in the flow, by redirecting the output of one operator to some previous operator. This is especially useful for defining algorithms that continuously update a model. The following code starts with a stream and applies the iteration body continuously. Elements that are greater than 0 are sent back to the feedback channel, and the rest of the elements are forwarded downstream. |
Extraction from tuples, case classes and collections via anonymous pattern matching, like the following:
is not supported by the API out-of-the-box. To use this feature, you should use a Scala API extension.
Transformation | Description |
---|---|
MapDataStream → DataStream |
Takes one element and produces one element. A map function that doubles the values of the input stream: |
FlatMapDataStream → DataStream |
Takes one element and produces zero, one, or more elements. A flatmap function that splits sentences to words: |
FilterDataStream → DataStream |
Evaluates a boolean function for each element and retains those for which the function returns true. A filter that filters out zero values: |
KeyByDataStream → KeyedStream |
Logically partitions a stream into disjoint partitions. All records with the same key are assigned to the same partition. Internally, keyBy() is implemented with hash partitioning. There are different ways to specify keys. This transformation returns a KeyedStream, which is, among other things, required to use keyed state. Attention A type cannot be a key if it is an array of any type. |
ReduceKeyedStream → DataStream |
A "rolling" reduce on a keyed data stream. Combines the current element with the last reduced value and
emits the new value.
A reduce function that creates a stream of partial sums: |
UnionDataStream* → DataStream |
Union of two or more data streams creating a new stream containing all the elements from all the streams. Note: If you union a data stream with itself you will get each element twice in the resulting stream. |
ConnectDataStream,DataStream → ConnectedStreams |
"Connects" two data streams retaining their types, allowing for shared state between the two streams. |
CoMap, CoFlatMapConnectedStreams → DataStream |
Similar to map and flatMap on a connected data stream |
The following transformations are available on data streams of Tuples:
Transformation | Description |
---|---|
ProjectDataStream → DataStream |
Selects a subset of fields from the tuples |
Transformation | Description |
---|---|
ProjectDataStream → DataStream |
Selects a subset of fields from the tuples |
Flink also gives low-level control (if desired) on the exact stream partitioning after a transformation, via the following functions.
Transformation | Description |
---|---|
Custom partitioningDataStream → DataStream |
Uses a user-defined Partitioner to select the target task for each element. |
Random partitioningDataStream → DataStream |
Partitions elements randomly according to a uniform distribution. |
Rebalancing (Round-robin partitioning)DataStream → DataStream |
Partitions elements round-robin, creating equal load per partition. Useful for performance optimization in the presence of data skew. |
RescalingDataStream → DataStream |
Partitions elements, round-robin, to a subset of downstream operations. This is useful if you want to have pipelines where you, for example, fan out from each parallel instance of a source to a subset of several mappers to distribute load but don't want the full rebalance that rebalance() would incur. This would require only local data transfers instead of transferring data over network, depending on other configuration values such as the number of slots of TaskManagers. The subset of downstream operations to which the upstream operation sends elements depends on the degree of parallelism of both the upstream and downstream operation. For example, if the upstream operation has parallelism 2 and the downstream operation has parallelism 6, then one upstream operation would distribute elements to three downstream operations while the other upstream operation would distribute to the other three downstream operations. If, on the other hand, the downstream operation has parallelism 2 while the upstream operation has parallelism 6 then three upstream operations would distribute to one downstream operation while the other three upstream operations would distribute to the other downstream operation. In cases where the different parallelisms are not multiples of each other one or several downstream operations will have a differing number of inputs from upstream operations. Please see this figure for a visualization of the connection pattern in the above example:
|
BroadcastingDataStream → DataStream |
Broadcasts elements to every partition. |
Transformation | Description |
---|---|
Custom partitioningDataStream → DataStream |
Uses a user-defined Partitioner to select the target task for each element. |
Random partitioningDataStream → DataStream |
Partitions elements randomly according to a uniform distribution. |
Rebalancing (Round-robin partitioning)DataStream → DataStream |
Partitions elements round-robin, creating equal load per partition. Useful for performance optimization in the presence of data skew. |
RescalingDataStream → DataStream |
Partitions elements, round-robin, to a subset of downstream operations. This is useful if you want to have pipelines where you, for example, fan out from each parallel instance of a source to a subset of several mappers to distribute load but don't want the full rebalance that rebalance() would incur. This would require only local data transfers instead of transferring data over network, depending on other configuration values such as the number of slots of TaskManagers. The subset of downstream operations to which the upstream operation sends elements depends on the degree of parallelism of both the upstream and downstream operation. For example, if the upstream operation has parallelism 2 and the downstream operation has parallelism 4, then one upstream operation would distribute elements to two downstream operations while the other upstream operation would distribute to the other two downstream operations. If, on the other hand, the downstream operation has parallelism 2 while the upstream operation has parallelism 4 then two upstream operations would distribute to one downstream operation while the other two upstream operations would distribute to the other downstream operations. In cases where the different parallelisms are not multiples of each other one or several downstream operations will have a differing number of inputs from upstream operations. Please see this figure for a visualization of the connection pattern in the above example:
|
BroadcastingDataStream → DataStream |
Broadcasts elements to every partition. |
Transformation | Description |
---|---|
Custom partitioningDataStream → DataStream |
Uses a user-defined Partitioner to select the target task for each element. |
Random partitioningDataStream → DataStream |
Partitions elements randomly according to a uniform distribution. |
Rebalancing (Round-robin partitioning)DataStream → DataStream |
Partitions elements round-robin, creating equal load per partition. Useful for performance optimization in the presence of data skew. |
RescalingDataStream → DataStream |
Partitions elements, round-robin, to a subset of downstream operations. This is useful if you want to have pipelines where you, for example, fan out from each parallel instance of a source to a subset of several mappers to distribute load but don't want the full rebalance that rebalance() would incur. This would require only local data transfers instead of transferring data over network, depending on other configuration values such as the number of slots of TaskManagers. The subset of downstream operations to which the upstream operation sends elements depends on the degree of parallelism of both the upstream and downstream operation. For example, if the upstream operation has parallelism 2 and the downstream operation has parallelism 6, then one upstream operation would distribute elements to three downstream operations while the other upstream operation would distribute to the other three downstream operations. If, on the other hand, the downstream operation has parallelism 2 while the upstream operation has parallelism 6 then three upstream operations would distribute to one downstream operation while the other three upstream operations would distribute to the other downstream operation. In cases where the different parallelisms are not multiples of each other one or several downstream operations will have a differing number of inputs from upstream operations. Please see this figure for a visualization of the connection pattern in the above example:
|
BroadcastingDataStream → DataStream |
Broadcasts elements to every partition. |
Chaining two subsequent transformations means co-locating them within the same thread for better performance. Flink by default chains operators if this is possible (e.g., two subsequent map transformations). The API gives fine-grained control over chaining if desired:
Use StreamExecutionEnvironment.disableOperatorChaining()
if you want to disable chaining in
the whole job. For more fine grained control, the following functions are available. Note that
these functions can only be used right after a DataStream transformation as they refer to the
previous transformation. For example, you can use someStream.map(...).startNewChain()
, but
you cannot use someStream.startNewChain()
.
A resource group is a slot in Flink, see slots. You can manually isolate operators in separate slots if desired.
Transformation | Description |
---|---|
Start new chain |
Begin a new chain, starting with this operator. The two mappers will be chained, and filter will not be chained to the first mapper. |
Disable chaining |
Do not chain the map operator |
Set slot sharing group |
Set the slot sharing group of an operation. Flink will put operations with the same slot sharing group into the same slot while keeping operations that don't have the slot sharing group in other slots. This can be used to isolate slots. The slot sharing group is inherited from input operations if all input operations are in the same slot sharing group. The name of the default slot sharing group is "default", operations can explicitly be put into this group by calling slotSharingGroup("default"). |
Chaining two subsequent transformations means co-locating them within the same thread for better performance. Flink by default chains operators if this is possible (e.g., two subsequent map transformations). The API gives fine-grained control over chaining if desired:
Use StreamExecutionEnvironment.disableOperatorChaining()
if you want to disable chaining in
the whole job. For more fine grained control, the following functions are available. Note that
these functions can only be used right after a DataStream transformation as they refer to the
previous transformation. For example, you can use someStream.map(...).startNewChain()
, but
you cannot use someStream.startNewChain()
.
A resource group is a slot in Flink, see slots. You can manually isolate operators in separate slots if desired.
Transformation | Description |
---|---|
Start new chain |
Begin a new chain, starting with this operator. The two mappers will be chained, and filter will not be chained to the first mapper. |
Disable chaining |
Do not chain the map operator |
Set slot sharing group |
Set the slot sharing group of an operation. Flink will put operations with the same slot sharing group into the same slot while keeping operations that don't have the slot sharing group in other slots. This can be used to isolate slots. The slot sharing group is inherited from input operations if all input operations are in the same slot sharing group. The name of the default slot sharing group is "default", operations can explicitly be put into this group by calling slotSharingGroup("default"). |
Chaining two subsequent transformations means co-locating them within the same thread for better performance. Flink by default chains operators if this is possible (e.g., two subsequent map transformations). The API gives fine-grained control over chaining if desired:
Use stream_execution_environment.disable_operator_chaining()
if you want to disable chaining in
the whole job. For more fine grained control, the following functions are available. Note that
these functions can only be used right after a DataStream transformation as they refer to the
previous transformation. For example, you can use some_stream.map(...).start_new_chain()
, but
you cannot use some_stream.start_new_chain()
.
A resource group is a slot in Flink, see slots. You can manually isolate operators in separate slots if desired.
Transformation | Description |
---|---|
Start new chain |
Begin a new chain, starting with this operator. The two mappers will be chained, and filter will not be chained to the first mapper. |
Disable chaining |
Do not chain the map operator |
Set slot sharing group |
Set the slot sharing group of an operation. Flink will put operations with the same slot sharing group into the same slot while keeping operations that don't have the slot sharing group in other slots. This can be used to isolate slots. The slot sharing group is inherited from input operations if all input operations are in the same slot sharing group. The name of the default slot sharing group is "default", operations can explicitly be put into this group by calling slotSharingGroup("default"). |