Interface TwoInputStreamOperator<IN1,IN2,OUT>

Type Parameters:
IN1 - The input type of the operator
IN2 - The input type of the operator
OUT - 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

@PublicEvolving public interface TwoInputStreamOperator<IN1,IN2,OUT> extends StreamOperator<OUT>
Interface for stream operators with two inputs. Use AbstractStreamOperator as a base class if you want to implement a custom operator.
  • Method Details

    • processElement1

      void processElement1(StreamRecord<IN1> element) throws Exception
      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

      void processElement2(StreamRecord<IN2> element) throws Exception
      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

      void processWatermark1(Watermark mark) throws Exception
      Processes a Watermark 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
      See Also:
    • processWatermark2

      void processWatermark2(Watermark mark) throws Exception
      Processes a Watermark 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
      See Also:
    • processLatencyMarker1

      void processLatencyMarker1(LatencyMarker latencyMarker) throws Exception
      Processes a LatencyMarker 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
      See Also:
    • processLatencyMarker2

      void processLatencyMarker2(LatencyMarker latencyMarker) throws Exception
      Processes a LatencyMarker 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
      See Also:
    • processWatermarkStatus1

      void processWatermarkStatus1(WatermarkStatus watermarkStatus) throws Exception
      Processes a WatermarkStatus 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
      See Also:
    • processWatermarkStatus2

      void processWatermarkStatus2(WatermarkStatus watermarkStatus) throws Exception
      Processes a WatermarkStatus 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
      See Also:
    • processRecordAttributes1

      @Experimental default void processRecordAttributes1(RecordAttributes recordAttributes) throws Exception
      Processes a RecordAttributes that 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 a RecordAttributes that 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

      @Experimental default void processWatermark1(WatermarkEvent watermark) throws Exception
      Processes a Watermark that arrived on the first input of this operator.
      Throws:
      Exception
    • processWatermark2

      @Experimental default void processWatermark2(WatermarkEvent watermark) throws Exception
      Processes a Watermark that arrived on the second input of this operator.
      Throws:
      Exception