Interface TwoInputStreamOperator<IN1,IN2,OUT>
- Type Parameters:
IN1- The input type of the operatorIN2- The input type of the operatorOUT- The output type of the operator
- All Superinterfaces:
org.apache.flink.api.common.state.CheckpointListener,KeyContext,Serializable,StreamOperator<OUT>
- All Known Implementing Classes:
AsyncIntervalJoinOperator,BatchCoBroadcastWithKeyedOperator,BatchCoBroadcastWithNonKeyedOperator,CoBroadcastWithKeyedOperator,CoBroadcastWithNonKeyedOperator,CoProcessOperator,CoStreamFlatMap,CoStreamMap,IntervalJoinOperator,KeyedCoProcessOperator,LegacyKeyedCoProcessOperator
Interface for stream operators with two inputs. Use
AbstractStreamOperator as a base class if you want to
implement a custom operator.-
Method Summary
Modifier and TypeMethodDescriptionvoidprocessElement1(StreamRecord<IN1> element) Processes one element that arrived on the first input of this two-input operator.voidprocessElement2(StreamRecord<IN2> element) Processes one element that arrived on the second input of this two-input operator.voidprocessLatencyMarker1(LatencyMarker latencyMarker) Processes aLatencyMarkerthat arrived on the first input of this two-input operator.voidprocessLatencyMarker2(LatencyMarker latencyMarker) Processes aLatencyMarkerthat arrived on the second input of this two-input operator.default voidprocessRecordAttributes1(RecordAttributes recordAttributes) Processes aRecordAttributesthat arrived on the first input of this operator.default voidprocessRecordAttributes2(RecordAttributes recordAttributes) Processes aRecordAttributesthat arrived on the second input of this operator.default voidprocessWatermark1(WatermarkEvent watermark) Processes aWatermarkthat arrived on the first input of this operator.voidprocessWatermark1(Watermark mark) Processes aWatermarkthat arrived on the first input of this two-input operator.default voidprocessWatermark2(WatermarkEvent watermark) Processes aWatermarkthat arrived on the second input of this operator.voidprocessWatermark2(Watermark mark) Processes aWatermarkthat arrived on the second input of this two-input operator.voidprocessWatermarkStatus1(WatermarkStatus watermarkStatus) Processes aWatermarkStatusthat arrived on the first input of this two-input operator.voidprocessWatermarkStatus2(WatermarkStatus watermarkStatus) Processes aWatermarkStatusthat arrived on the second input of this two-input operator.Methods inherited from interface org.apache.flink.api.common.state.CheckpointListener
notifyCheckpointAborted, notifyCheckpointCompleteMethods inherited from interface org.apache.flink.streaming.api.operators.KeyContext
getCurrentKey, setCurrentKeyMethods inherited from interface org.apache.flink.streaming.api.operators.StreamOperator
close, finish, getMetricGroup, getOperatorAttributes, getOperatorID, initializeState, open, prepareSnapshotPreBarrier, setKeyContextElement1, setKeyContextElement2, snapshotState
-
Method Details
-
processElement1
Processes one element that arrived on the first input of this two-input operator. This method is guaranteed to not be called concurrently with other methods of the operator.- Throws:
Exception
-
processElement2
Processes one element that arrived on the second input of this two-input operator. This method is guaranteed to not be called concurrently with other methods of the operator.- Throws:
Exception
-
processWatermark1
Processes aWatermarkthat arrived on the first input of this two-input operator. This method is guaranteed to not be called concurrently with other methods of the operator. -
processWatermark2
Processes aWatermarkthat arrived on the second input of this two-input operator. This method is guaranteed to not be called concurrently with other methods of the operator. -
processLatencyMarker1
Processes aLatencyMarkerthat arrived on the first input of this two-input operator. This method is guaranteed to not be called concurrently with other methods of the operator.- Throws:
Exception- See Also:
-
processLatencyMarker2
Processes aLatencyMarkerthat arrived on the second input of this two-input operator. This method is guaranteed to not be called concurrently with other methods of the operator.- Throws:
Exception- See Also:
-
processWatermarkStatus1
Processes aWatermarkStatusthat arrived on the first input of this two-input operator. This method is guaranteed to not be called concurrently with other methods of the operator.- Throws:
Exception- See Also:
-
processWatermarkStatus2
Processes aWatermarkStatusthat arrived on the second input of this two-input operator. This method is guaranteed to not be called concurrently with other methods of the operator.- Throws:
Exception- See Also:
-
processRecordAttributes1
@Experimental default void processRecordAttributes1(RecordAttributes recordAttributes) throws Exception Processes aRecordAttributesthat arrived on the first input of this operator. This method is guaranteed to not be called concurrently with other methods of the operator.- Throws:
Exception
-
processRecordAttributes2
@Experimental default void processRecordAttributes2(RecordAttributes recordAttributes) throws Exception Processes aRecordAttributesthat arrived on the second input of this operator. This method is guaranteed to not be called concurrently with other methods of the operator.- Throws:
Exception
-
processWatermark1
Processes aWatermarkthat arrived on the first input of this operator.- Throws:
Exception
-
processWatermark2
Processes aWatermarkthat arrived on the second input of this operator.- Throws:
Exception
-