Interface StreamOperator<OUT>

Type Parameters:
OUT - The output type of the operator
All Superinterfaces:
org.apache.flink.api.common.state.CheckpointListener, KeyContext, Serializable
All Known Subinterfaces:
MultipleInputStreamOperator<OUT>, OneInputStreamOperator<IN,OUT>, TwoInputStreamOperator<IN1,IN2,OUT>, YieldingOperator<OUT>
All Known Implementing Classes:
AbstractAsyncStateStreamOperator, AbstractAsyncStateStreamOperatorV2, AbstractAsyncStateUdfStreamOperator, AbstractStreamOperator, AbstractStreamOperatorV2, AbstractUdfStreamOperator, AsyncEvictingWindowOperator, AsyncIntervalJoinOperator, AsyncKeyedProcessOperator, AsyncStreamFlatMap, AsyncWindowOperator, BatchCoBroadcastWithKeyedOperator, BatchCoBroadcastWithNonKeyedOperator, BatchGroupedReduceOperator, CacheTransformationTranslator.IdentityStreamOperator, CacheTransformationTranslator.NoOpStreamOperator, CoBroadcastWithKeyedOperator, CoBroadcastWithNonKeyedOperator, CollectSinkOperator, ContinuousFileReaderOperator, CoProcessOperator, CoStreamFlatMap, CoStreamMap, EvictingWindowOperator, GlobalCommitterOperator, IntervalJoinOperator, KeyedCoProcessOperator, KeyedProcessOperator, KeyedSortPartitionOperator, LegacyKeyedCoProcessOperator, LegacyKeyedProcessOperator, MapPartitionOperator, PartitionAggregateOperator, PartitionReduceOperator, ProcessOperator, QueryableAppendingStateOperator, QueryableValueStateOperator, SortPartitionOperator, SourceOperator, StreamFilter, StreamFlatMap, StreamGroupedReduceAsyncStateOperator, StreamGroupedReduceOperator, StreamMap, StreamProject, StreamSink, StreamSource, TimestampsAndWatermarksOperator, WindowOperator

@PublicEvolving public interface StreamOperator<OUT> extends org.apache.flink.api.common.state.CheckpointListener, KeyContext, Serializable
Basic interface for stream operators. Implementers would implement one of OneInputStreamOperator or TwoInputStreamOperator to create operators that process elements.

The class AbstractStreamOperator offers default implementation for the lifecycle and properties methods.

Methods of StreamOperator are guaranteed not to be called concurrently. Also, if using the timer service, timer callbacks are also guaranteed not to be called concurrently with methods on StreamOperator.

  • Method Details

    • open

      void open() throws Exception
      This method is called immediately before any elements are processed, it should contain the operator's initialization logic.
      Throws:
      Exception - An exception in this method causes the operator to fail.
    • finish

      void finish() throws Exception
      This method is called at the end of data processing.

      The method is expected to flush all remaining buffered data. Exceptions during this flushing of buffered data should be propagated, in order to cause the operation to be recognized as failed, because the last data items are not processed properly.

      After this method is called, no more records can be produced for the downstream operators.

      WARNING: It is not safe to use this method to commit any transactions or other side effects! You can use this method to flush any buffered data that can later on be committed e.g. in a CheckpointListener.notifyCheckpointComplete(long).

      NOTE:This method does not need to close any resources. You should release external resources in the close() method.

      Throws:
      Exception - An exception in this method causes the operator to fail.
    • close

      void close() throws Exception
      This method is called at the very end of the operator's life, both in the case of a successful completion of the operation, and in the case of a failure and canceling.

      This method is expected to make a thorough effort to release all resources that the operator has acquired.

      NOTE:It can not emit any records! If you need to emit records at the end of processing, do so in the finish() method.

      Throws:
      Exception
    • prepareSnapshotPreBarrier

      void prepareSnapshotPreBarrier(long checkpointId) throws Exception
      This method is called when the operator should do a snapshot, before it emits its own checkpoint barrier.

      This method is intended not for any actual state persistence, but only for emitting some data before emitting the checkpoint barrier. Operators that maintain some small transient state that is inefficient to checkpoint (especially when it would need to be checkpointed in a re-scalable way) but can simply be sent downstream before the checkpoint. An example are opportunistic pre-aggregation operators, which have small the pre-aggregation state that is frequently flushed downstream.

      Important: This method should not be used for any actual state snapshot logic, because it will inherently be within the synchronous part of the operator's checkpoint. If heavy work is done within this method, it will affect latency and downstream checkpoint alignments.

      Parameters:
      checkpointId - The ID of the checkpoint.
      Throws:
      Exception - Throwing an exception here causes the operator to fail and go into recovery.
    • snapshotState

      OperatorSnapshotFutures snapshotState(long checkpointId, long timestamp, CheckpointOptions checkpointOptions, CheckpointStreamFactory storageLocation) throws Exception
      Called to draw a state snapshot from the operator.
      Returns:
      a runnable future to the state handle that points to the snapshotted state. For synchronous implementations, the runnable might already be finished.
      Throws:
      Exception - exception that happened during snapshotting.
    • initializeState

      void initializeState(StreamTaskStateInitializer streamTaskStateManager) throws Exception
      Provides a context to initialize all state in the operator.
      Throws:
      Exception
    • setKeyContextElement1

      void setKeyContextElement1(StreamRecord<?> record) throws Exception
      Throws:
      Exception
    • setKeyContextElement2

      void setKeyContextElement2(StreamRecord<?> record) throws Exception
      Throws:
      Exception
    • getMetricGroup

      org.apache.flink.metrics.groups.OperatorMetricGroup getMetricGroup()
    • getOperatorID

      OperatorID getOperatorID()
    • getOperatorAttributes

      @Experimental default OperatorAttributes getOperatorAttributes()
      Called to get the OperatorAttributes of the operator. If there is no defined attribute, a default OperatorAttributes is built.
      Returns:
      OperatorAttributes of the operator.