Class PatternStream<T>
- Type Parameters:
T- Type of the events
NFA. In order to process the detected sequences, the user has to
specify a PatternSelectFunction or a PatternFlatSelectFunction.
Additionally it allows to handle partially matched event patterns which have timed out. For
this the user has to specify a PatternTimeoutFunction or a PatternFlatTimeoutFunction.
-
Method Summary
Modifier and TypeMethodDescription<R> org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator<R>flatSelect(PatternFlatSelectFunction<T, R> patternFlatSelectFunction) Applies a flat select function to the detected pattern sequence.<R> org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator<R>flatSelect(PatternFlatSelectFunction<T, R> patternFlatSelectFunction, org.apache.flink.api.common.typeinfo.TypeInformation<R> outTypeInfo) Applies a flat select function to the detected pattern sequence.<L,R> org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator<R> flatSelect(org.apache.flink.util.OutputTag<L> timedOutPartialMatchesTag, PatternFlatTimeoutFunction<T, L> patternFlatTimeoutFunction, org.apache.flink.api.common.typeinfo.TypeInformation<R> outTypeInfo, PatternFlatSelectFunction<T, R> patternFlatSelectFunction) Applies a flat select function to the detected pattern sequence.<L,R> org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator<R> flatSelect(org.apache.flink.util.OutputTag<L> timedOutPartialMatchesTag, PatternFlatTimeoutFunction<T, L> patternFlatTimeoutFunction, PatternFlatSelectFunction<T, R> patternFlatSelectFunction) Applies a flat select function to the detected pattern sequence.Sets the time characteristic to event time.Sets the time characteristic to processing time.<R> org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator<R>process(PatternProcessFunction<T, R> patternProcessFunction) Applies a process function to the detected pattern sequence.<R> org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator<R>process(PatternProcessFunction<T, R> patternProcessFunction, org.apache.flink.api.common.typeinfo.TypeInformation<R> outTypeInfo) Applies a process function to the detected pattern sequence.<R> org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator<R>select(PatternSelectFunction<T, R> patternSelectFunction) Applies a select function to the detected pattern sequence.<R> org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator<R>select(PatternSelectFunction<T, R> patternSelectFunction, org.apache.flink.api.common.typeinfo.TypeInformation<R> outTypeInfo) Applies a select function to the detected pattern sequence.<L,R> org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator<R> select(org.apache.flink.util.OutputTag<L> timedOutPartialMatchesTag, PatternTimeoutFunction<T, L> patternTimeoutFunction, org.apache.flink.api.common.typeinfo.TypeInformation<R> outTypeInfo, PatternSelectFunction<T, R> patternSelectFunction) Applies a select function to the detected pattern sequence.<L,R> org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator<R> select(org.apache.flink.util.OutputTag<L> timedOutPartialMatchesTag, PatternTimeoutFunction<T, L> patternTimeoutFunction, PatternSelectFunction<T, R> patternSelectFunction) Applies a select function to the detected pattern sequence.sideOutputLateData(org.apache.flink.util.OutputTag<T> lateDataOutputTag) Send late arriving data to the side output identified by the givenOutputTag.
-
Method Details
-
sideOutputLateData
Send late arriving data to the side output identified by the givenOutputTag. A record is considered late after the watermark has passed its timestamp.You can get the stream of late data using
SingleOutputStreamOperator.getSideOutput(OutputTag)on theSingleOutputStreamOperatorresulting from the pattern processing operations. -
inProcessingTime
Sets the time characteristic to processing time. -
inEventTime
Sets the time characteristic to event time. -
process
public <R> org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator<R> process(PatternProcessFunction<T, R> patternProcessFunction) Applies a process function to the detected pattern sequence. For each pattern sequence the providedPatternProcessFunctionis called. In order to process timed out partial matches as well one can useTimedOutPartialMatchHandleras additional interface.- Type Parameters:
R- Type of the resulting elements- Parameters:
patternProcessFunction- The pattern process function which is called for each detected pattern sequence.- Returns:
DataStreamwhich contains the resulting elements from the pattern process function.
-
process
public <R> org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator<R> process(PatternProcessFunction<T, R> patternProcessFunction, org.apache.flink.api.common.typeinfo.TypeInformation<R> outTypeInfo) Applies a process function to the detected pattern sequence. For each pattern sequence the providedPatternProcessFunctionis called. In order to process timed out partial matches as well one can useTimedOutPartialMatchHandleras additional interface.- Type Parameters:
R- Type of the resulting elements- Parameters:
patternProcessFunction- The pattern process function which is called for each detected pattern sequence.outTypeInfo- Explicit specification of output type.- Returns:
DataStreamwhich contains the resulting elements from the pattern process function.
-
select
public <R> org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator<R> select(PatternSelectFunction<T, R> patternSelectFunction) Applies a select function to the detected pattern sequence. For each pattern sequence the providedPatternSelectFunctionis called. The pattern select function can produce exactly one resulting element.- Type Parameters:
R- Type of the resulting elements- Parameters:
patternSelectFunction- The pattern select function which is called for each detected pattern sequence.- Returns:
DataStreamwhich contains the resulting elements from the pattern select function.
-
select
public <R> org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator<R> select(PatternSelectFunction<T, R> patternSelectFunction, org.apache.flink.api.common.typeinfo.TypeInformation<R> outTypeInfo) Applies a select function to the detected pattern sequence. For each pattern sequence the providedPatternSelectFunctionis called. The pattern select function can produce exactly one resulting element.- Type Parameters:
R- Type of the resulting elements- Parameters:
patternSelectFunction- The pattern select function which is called for each detected pattern sequence.outTypeInfo- Explicit specification of output type.- Returns:
DataStreamwhich contains the resulting elements from the pattern select function.
-
select
public <L,R> org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator<R> select(org.apache.flink.util.OutputTag<L> timedOutPartialMatchesTag, PatternTimeoutFunction<T, L> patternTimeoutFunction, PatternSelectFunction<T, R> patternSelectFunction) Applies a select function to the detected pattern sequence. For each pattern sequence the providedPatternSelectFunctionis called. The pattern select function can produce exactly one resulting element.Applies a timeout function to a partial pattern sequence which has timed out. For each partial pattern sequence the provided
PatternTimeoutFunctionis called. The pattern timeout function can produce exactly one resulting element.You can get the stream of timed-out data resulting from the
SingleOutputStreamOperator.getSideOutput(OutputTag)on theSingleOutputStreamOperatorresulting from the select operation with the sameOutputTag.- Type Parameters:
L- Type of the resulting timeout elementsR- Type of the resulting elements- Parameters:
timedOutPartialMatchesTag-OutputTagthat identifies side output with timed out patternspatternTimeoutFunction- The pattern timeout function which is called for each partial pattern sequence which has timed out.patternSelectFunction- The pattern select function which is called for each detected pattern sequence.- Returns:
DataStreamwhich contains the resulting elements with the resulting timeout elements in a side output.
-
select
public <L,R> org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator<R> select(org.apache.flink.util.OutputTag<L> timedOutPartialMatchesTag, PatternTimeoutFunction<T, L> patternTimeoutFunction, org.apache.flink.api.common.typeinfo.TypeInformation<R> outTypeInfo, PatternSelectFunction<T, R> patternSelectFunction) Applies a select function to the detected pattern sequence. For each pattern sequence the providedPatternSelectFunctionis called. The pattern select function can produce exactly one resulting element.Applies a timeout function to a partial pattern sequence which has timed out. For each partial pattern sequence the provided
PatternTimeoutFunctionis called. The pattern timeout function can produce exactly one resulting element.You can get the stream of timed-out data resulting from the
SingleOutputStreamOperator.getSideOutput(OutputTag)on theSingleOutputStreamOperatorresulting from the select operation with the sameOutputTag.- Type Parameters:
L- Type of the resulting timeout elementsR- Type of the resulting elements- Parameters:
timedOutPartialMatchesTag-OutputTagthat identifies side output with timed out patternspatternTimeoutFunction- The pattern timeout function which is called for each partial pattern sequence which has timed out.outTypeInfo- Explicit specification of output type.patternSelectFunction- The pattern select function which is called for each detected pattern sequence.- Returns:
DataStreamwhich contains the resulting elements with the resulting timeout elements in a side output.
-
flatSelect
public <R> org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator<R> flatSelect(PatternFlatSelectFunction<T, R> patternFlatSelectFunction) Applies a flat select function to the detected pattern sequence. For each pattern sequence the providedPatternFlatSelectFunctionis called. The pattern flat select function can produce an arbitrary number of resulting elements.- Type Parameters:
R- Type of the resulting elements- Parameters:
patternFlatSelectFunction- The pattern flat select function which is called for each detected pattern sequence.- Returns:
DataStreamwhich contains the resulting elements from the pattern flat select function.
-
flatSelect
public <R> org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator<R> flatSelect(PatternFlatSelectFunction<T, R> patternFlatSelectFunction, org.apache.flink.api.common.typeinfo.TypeInformation<R> outTypeInfo) Applies a flat select function to the detected pattern sequence. For each pattern sequence the providedPatternFlatSelectFunctionis called. The pattern flat select function can produce an arbitrary number of resulting elements.- Type Parameters:
R- Type of the resulting elements- Parameters:
patternFlatSelectFunction- The pattern flat select function which is called for each detected pattern sequence.outTypeInfo- Explicit specification of output type.- Returns:
DataStreamwhich contains the resulting elements from the pattern flat select function.
-
flatSelect
public <L,R> org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator<R> flatSelect(org.apache.flink.util.OutputTag<L> timedOutPartialMatchesTag, PatternFlatTimeoutFunction<T, L> patternFlatTimeoutFunction, PatternFlatSelectFunction<T, R> patternFlatSelectFunction) Applies a flat select function to the detected pattern sequence. For each pattern sequence the providedPatternFlatSelectFunctionis called. The pattern select function can produce exactly one resulting element.Applies a timeout function to a partial pattern sequence which has timed out. For each partial pattern sequence the provided
PatternFlatTimeoutFunctionis called. The pattern timeout function can produce exactly one resulting element.You can get the stream of timed-out data resulting from the
SingleOutputStreamOperator.getSideOutput(OutputTag)on theSingleOutputStreamOperatorresulting from the select operation with the sameOutputTag.- Type Parameters:
L- Type of the resulting timeout elementsR- Type of the resulting elements- Parameters:
timedOutPartialMatchesTag-OutputTagthat identifies side output with timed out patternspatternFlatTimeoutFunction- The pattern timeout function which is called for each partial pattern sequence which has timed out.patternFlatSelectFunction- The pattern select function which is called for each detected pattern sequence.- Returns:
DataStreamwhich contains the resulting elements with the resulting timeout elements in a side output.
-
flatSelect
public <L,R> org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator<R> flatSelect(org.apache.flink.util.OutputTag<L> timedOutPartialMatchesTag, PatternFlatTimeoutFunction<T, L> patternFlatTimeoutFunction, org.apache.flink.api.common.typeinfo.TypeInformation<R> outTypeInfo, PatternFlatSelectFunction<T, R> patternFlatSelectFunction) Applies a flat select function to the detected pattern sequence. For each pattern sequence the providedPatternFlatSelectFunctionis called. The pattern select function can produce exactly one resulting element.Applies a timeout function to a partial pattern sequence which has timed out. For each partial pattern sequence the provided
PatternFlatTimeoutFunctionis called. The pattern timeout function can produce exactly one resulting element.You can get the stream of timed-out data resulting from the
SingleOutputStreamOperator.getSideOutput(OutputTag)on theSingleOutputStreamOperatorresulting from the select operation with the sameOutputTag.- Type Parameters:
L- Type of the resulting timeout elementsR- Type of the resulting elements- Parameters:
timedOutPartialMatchesTag-OutputTagthat identifies side output with timed out patternspatternFlatTimeoutFunction- The pattern timeout function which is called for each partial pattern sequence which has timed out.patternFlatSelectFunction- The pattern select function which is called for each detected pattern sequence.outTypeInfo- Explicit specification of output type.- Returns:
DataStreamwhich contains the resulting elements with the resulting timeout elements in a side output.
-