ConnectedStreams

@Public
class ConnectedStreams[IN1, IN2](javaStream: ConnectedStreams[IN1, IN2])

ConnectedStreams represents two connected streams of (possibly) different data types. Connected streams are useful for cases where operations on one stream directly affect the operations on the other stream, usually via shared state between the streams.

An example for the use of connected streams would be to apply rules that change over time onto another stream. One of the connected streams has the rules, the other stream the elements to apply the rules to. The operation on the connected stream maintains the current set of rules in the state. It may receive either a rule update and update the state or a data element and apply the rules in the state to the element.

The connected stream can be conceptually viewed as a union stream of an Either type, that holds either the first stream's type or the second stream's type.

class Object
trait Matchable
class Any

Value members

Concrete methods

def flatMap[R : TypeInformation](coFlatMapper: CoFlatMapFunction[IN1, IN2, R]): DataStream[R]

Applies a CoFlatMap transformation on these connected streams.

Applies a CoFlatMap transformation on these connected streams.

The transformation calls CoFlatMapFunction#flatMap1 for each element in the first stream and CoFlatMapFunction#flatMap2 for each element of the second stream.

On can pass a subclass of org.apache.flink.streaming.api.functions.co.RichCoFlatMapFunction to gain access to the org.apache.flink.api.common.functions.RuntimeContext and to additional life cycle methods.

Value parameters:
coFlatMapper

The CoFlatMapFunction used to transform the two connected streams

Returns:

The resulting data stream.

def flatMap[R : TypeInformation](fun1: (IN1, Collector[R]) => Unit, fun2: (IN2, Collector[R]) => Unit): DataStream[R]

Applies a CoFlatMap transformation on the connected streams.

Applies a CoFlatMap transformation on the connected streams.

The transformation consists of two separate functions, where the first one is called for each element of the first connected stream, and the second one is called for each element of the second connected stream.

Value parameters:
fun1

Function called per element of the first input.

fun2

Function called per element of the second input.

Returns:

The resulting data stream.

def flatMap[R : TypeInformation](fun1: IN1 => IterableOnce[R], fun2: IN2 => IterableOnce[R]): DataStream[R]

Applies a CoFlatMap transformation on the connected streams.

Applies a CoFlatMap transformation on the connected streams.

The transformation consists of two separate functions, where the first one is called for each element of the first connected stream, and the second one is called for each element of the second connected stream.

Value parameters:
fun1

Function called per element of the first input.

fun2

Function called per element of the second input.

Returns:

The resulting data stream.

def keyBy(keyPosition1: Int, keyPosition2: Int): ConnectedStreams[IN1, IN2]

Keys the two connected streams together. After this operation, all elements with the same key from both streams will be sent to the same parallel instance of the transformation functions.

Keys the two connected streams together. After this operation, all elements with the same key from both streams will be sent to the same parallel instance of the transformation functions.

Value parameters:
keyPosition1

The first stream's key field

keyPosition2

The second stream's key field

Returns:

The key-grouped connected streams

def keyBy(keyPositions1: Array[Int], keyPositions2: Array[Int]): ConnectedStreams[IN1, IN2]

Keys the two connected streams together. After this operation, all elements with the same key from both streams will be sent to the same parallel instance of the transformation functions.

Keys the two connected streams together. After this operation, all elements with the same key from both streams will be sent to the same parallel instance of the transformation functions.

Value parameters:
keyPositions1

The first stream's key fields

keyPositions2

The second stream's key fields

Returns:

The key-grouped connected streams

def keyBy(field1: String, field2: String): ConnectedStreams[IN1, IN2]

Keys the two connected streams together. After this operation, all elements with the same key from both streams will be sent to the same parallel instance of the transformation functions.

Keys the two connected streams together. After this operation, all elements with the same key from both streams will be sent to the same parallel instance of the transformation functions.

Value parameters:
field1

The first stream's key expression

field2

The second stream's key expression

Returns:

The key-grouped connected streams

def keyBy(fields1: Array[String], fields2: Array[String]): ConnectedStreams[IN1, IN2]

Keys the two connected streams together. After this operation, all elements with the same key from both streams will be sent to the same parallel instance of the transformation functions.

Keys the two connected streams together. After this operation, all elements with the same key from both streams will be sent to the same parallel instance of the transformation functions.

Value parameters:
fields1

The first stream's key expressions

fields2

The second stream's key expressions

Returns:

The key-grouped connected streams

def keyBy[KEY : TypeInformation](fun1: IN1 => KEY, fun2: IN2 => KEY): ConnectedStreams[IN1, IN2]

Keys the two connected streams together. After this operation, all elements with the same key from both streams will be sent to the same parallel instance of the transformation functions.

Keys the two connected streams together. After this operation, all elements with the same key from both streams will be sent to the same parallel instance of the transformation functions.

Value parameters:
fun1

The first stream's key function

fun2

The second stream's key function

Returns:

The key-grouped connected streams

def map[R : TypeInformation](fun1: IN1 => R, fun2: IN2 => R): DataStream[R]

Applies a CoMap transformation on the connected streams.

Applies a CoMap transformation on the connected streams.

The transformation consists of two separate functions, where the first one is called for each element of the first connected stream, and the second one is called for each element of the second connected stream.

Value parameters:
fun1

Function called per element of the first input.

fun2

Function called per element of the second input.

Returns:

The resulting data stream.

def map[R : TypeInformation](coMapper: CoMapFunction[IN1, IN2, R]): DataStream[R]

Applies a CoMap transformation on these connected streams.

Applies a CoMap transformation on these connected streams.

The transformation calls CoMapFunction#map1 for each element in the first stream and CoMapFunction#map2 for each element of the second stream.

On can pass a subclass of org.apache.flink.streaming.api.functions.co.RichCoMapFunction to gain access to the org.apache.flink.api.common.functions.RuntimeContext and to additional life cycle methods.

Value parameters:
coMapper

The CoMapFunction used to transform the two connected streams

Returns:

The resulting data stream

@PublicEvolving
def process[R : TypeInformation](coProcessFunction: CoProcessFunction[IN1, IN2, R]): DataStream[R]

Applies the given CoProcessFunction on the connected input streams, thereby creating a transformed output stream.

Applies the given CoProcessFunction on the connected input streams, thereby creating a transformed output stream.

The function will be called for every element in the input streams and can produce zero or more output elements. Contrary to the flatMap function, this function can also query the time and set timers. When reacting to the firing of set timers the function can directly emit elements and/or register yet more timers.

Value parameters:
coProcessFunction

The CoProcessFunction that is called for each element in the stream.

Returns:

The transformed DataStream.

@PublicEvolving
def process[K, R : TypeInformation](keyedCoProcessFunction: KeyedCoProcessFunction[K, IN1, IN2, R]): DataStream[R]

Applies the given KeyedCoProcessFunction on the connected input keyed streams, thereby creating a transformed output stream.

Applies the given KeyedCoProcessFunction on the connected input keyed streams, thereby creating a transformed output stream.

The function will be called for every element in the input keyed streams and can produce zero or more output elements. Contrary to the flatMap function, this function can also query the time and set timers. When reacting to the firing of set timers the function can directly emit elements and/or register yet more timers.

Value parameters:
keyedCoProcessFunction

The KeyedCoProcessFunction that is called for each element in the stream.

Returns:

The transformed DataStream.

@PublicEvolving
def transform[R : TypeInformation](functionName: String, operator: TwoInputStreamOperator[IN1, IN2, R]): DataStream[R]
@PublicEvolving
def transform[R : TypeInformation](functionName: String, factory: TwoInputStreamOperatorFactory[IN1, IN2, R]): DataStream[R]