Class AbstractStreamOperator<OUT>
- java.lang.Object
-
- org.apache.flink.streaming.api.operators.AbstractStreamOperator<OUT>
-
- Type Parameters:
OUT
- The output type of the operator.
- All Implemented Interfaces:
Serializable
,CheckpointListener
,KeyContext
,KeyContextHandler
,StreamOperator<OUT>
,StreamOperatorStateHandler.CheckpointedStreamOperator
,YieldingOperator<OUT>
- Direct Known Subclasses:
AbstractAsyncStateStreamOperator
,AbstractMapBundleOperator
,AbstractPythonFunctionOperator
,AbstractStreamingJoinOperator
,AbstractStreamingWriter
,AbstractUdfStreamOperator
,BaseTwoInputStreamOperatorWithStateRetention
,BatchCompactCoordinator
,BatchCompactOperator
,BatchFileWriter
,BootstrapStreamTaskRunner
,CacheTransformationTranslator.IdentityStreamOperator
,CacheTransformationTranslator.NoOpStreamOperator
,CompactCoordinator
,CompactCoordinator
,CompactCoordinatorStateHandler
,CompactOperator
,CompactorOperator
,CompactorOperatorStateHandler
,ContinuousFileReaderOperator
,DynamicFilteringDataCollectorOperator
,GenericWriteAheadSink
,KeyedSortPartitionOperator
,LocalSlicingWindowAggOperator
,PartitionCommitter
,ProcTimeMiniBatchAssignerOperator
,ProcTimeSortOperator
,QueryableAppendingStateOperator
,QueryableValueStateOperator
,RowTimeMiniBatchAssginerOperator
,RowTimeSortOperator
,SortPartitionOperator
,SourceOperator
,StreamProject
,TableStreamOperator
,TimestampsAndWatermarksOperator
,WatermarkAssignerOperator
,WindowOperator
@PublicEvolving public abstract class AbstractStreamOperator<OUT> extends Object implements StreamOperator<OUT>, YieldingOperator<OUT>, StreamOperatorStateHandler.CheckpointedStreamOperator, KeyContextHandler, Serializable
Base class for all stream operators. Operators that contain a user function should extend the classAbstractUdfStreamOperator
instead (which is a specialized subclass of this class).For concrete implementations, one of the following two interfaces must also be implemented, to mark the operator as unary or binary:
OneInputStreamOperator
orTwoInputStreamOperator
.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 onStreamOperator
.Note, this class is going to be removed and replaced in the future by
AbstractStreamOperatorV2
. However asAbstractStreamOperatorV2
is currently experimental,AbstractStreamOperator
has not been deprecated just yet.- See Also:
- Serialized Form
-
-
Field Summary
Fields Modifier and Type Field Description protected StreamConfig
config
protected RecordAttributes
lastRecordAttributes1
protected RecordAttributes
lastRecordAttributes2
protected LatencyStats
latencyStats
protected static org.slf4j.Logger
LOG
The logger used by the operator class and its subclasses.protected InternalOperatorMetricGroup
metrics
Metric group for the operator.protected Output<StreamRecord<OUT>>
output
protected ProcessingTimeService
processingTimeService
protected StreamOperatorStateHandler
stateHandler
protected KeySelector<?,?>
stateKeySelector1
KeySelector
for extracting a key from an element being processed.protected KeySelector<?,?>
stateKeySelector2
KeySelector
for extracting a key from an element being processed.protected InternalTimeServiceManager<?>
timeServiceManager
-
Constructor Summary
Constructors Constructor Description AbstractStreamOperator()
AbstractStreamOperator(StreamOperatorParameters<OUT> parameters)
-
Method Summary
All Methods Instance Methods Concrete Methods Modifier and Type Method Description void
close()
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.void
finish()
This method is called at the end of data processing.StreamTask<?,?>
getContainingTask()
Object
getCurrentKey()
ExecutionConfig
getExecutionConfig()
Gets the execution config defined on the execution environment of the job to which this operator belongs.<K,N>
InternalTimerService<N>getInternalTimerService(String name, TypeSerializer<N> namespaceSerializer, Triggerable<K,N> triggerable)
Returns aInternalTimerService
that can be used to query current processing time and event time and to set timers.<K> KeyedStateBackend<K>
getKeyedStateBackend()
KeyedStateStore
getKeyedStateStore()
OperatorMetricGroup
getMetricGroup()
StreamConfig
getOperatorConfig()
OperatorID
getOperatorID()
protected String
getOperatorName()
Return the operator name.OperatorStateBackend
getOperatorStateBackend()
protected <N,S extends State,T>
SgetOrCreateKeyedState(TypeSerializer<N> namespaceSerializer, StateDescriptor<S,T> stateDescriptor)
protected <S extends State,N>
SgetPartitionedState(N namespace, TypeSerializer<N> namespaceSerializer, StateDescriptor<S,?> stateDescriptor)
Creates a partitioned state handle, using the state backend configured for this task.protected <S extends State>
SgetPartitionedState(StateDescriptor<S,?> stateDescriptor)
Creates a partitioned state handle, using the state backend configured for this task.ProcessingTimeService
getProcessingTimeService()
Returns theProcessingTimeService
responsible for getting the current processing time and registering timers.StreamingRuntimeContext
getRuntimeContext()
Returns a context that allows the operator to query information about the execution and also to interact with systems such as broadcast variables and managed state.protected KeySelector<?,?>
getStateKeySelector1()
protected KeySelector<?,?>
getStateKeySelector2()
protected Optional<InternalTimeServiceManager<?>>
getTimeServiceManager()
ClassLoader
getUserCodeClassloader()
boolean
hasKeyContext1()
Whether the first input ofStreamOperator
has "KeyContext".boolean
hasKeyContext2()
Whether the second input ofStreamOperator
has "KeyContext".void
initializeState(StateInitializationContext context)
Stream operators with state which can be restored need to override this hook method.void
initializeState(StreamTaskStateInitializer streamTaskStateManager)
Provides a context to initialize all state in the operator.protected boolean
isUsingCustomRawKeyedState()
Indicates whether or not implementations of this class is writing to the raw keyed state streams on snapshots, usingsnapshotState(StateSnapshotContext)
.void
notifyCheckpointAborted(long checkpointId)
This method is called as a notification once a distributed checkpoint has been aborted.void
notifyCheckpointComplete(long checkpointId)
Notifies the listener that the checkpoint with the givencheckpointId
completed and was committed.void
open()
This method is called immediately before any elements are processed, it should contain the operator's initialization logic, e.g. state initialization.void
prepareSnapshotPreBarrier(long checkpointId)
This method is called when the operator should do a snapshot, before it emits its own checkpoint barrier.void
processLatencyMarker(LatencyMarker latencyMarker)
void
processLatencyMarker1(LatencyMarker latencyMarker)
void
processLatencyMarker2(LatencyMarker latencyMarker)
void
processRecordAttributes(RecordAttributes recordAttributes)
void
processRecordAttributes1(RecordAttributes recordAttributes)
void
processRecordAttributes2(RecordAttributes recordAttributes)
void
processWatermark(Watermark mark)
void
processWatermark1(Watermark mark)
void
processWatermark2(Watermark mark)
void
processWatermarkStatus(WatermarkStatus watermarkStatus)
void
processWatermarkStatus1(WatermarkStatus watermarkStatus)
void
processWatermarkStatus2(WatermarkStatus watermarkStatus)
protected void
reportOrForwardLatencyMarker(LatencyMarker marker)
void
setCurrentKey(Object key)
void
setKeyContextElement1(StreamRecord record)
void
setKeyContextElement2(StreamRecord record)
void
setMailboxExecutor(MailboxExecutor mailboxExecutor)
protected void
setProcessingTimeService(ProcessingTimeService processingTimeService)
protected void
setup(StreamTask<?,?> containingTask, StreamConfig config, Output<StreamRecord<OUT>> output)
OperatorSnapshotFutures
snapshotState(long checkpointId, long timestamp, CheckpointOptions checkpointOptions, CheckpointStreamFactory factory)
Called to draw a state snapshot from the operator.void
snapshotState(StateSnapshotContext context)
Stream operators with state, which want to participate in a snapshot need to override this hook method.boolean
useSplittableTimers()
Can be overridden to disable splittable timers for this particular operator even if config option is enabled.-
Methods inherited from class java.lang.Object
clone, equals, finalize, getClass, hashCode, notify, notifyAll, toString, wait, wait, wait
-
Methods inherited from interface org.apache.flink.streaming.api.operators.KeyContextHandler
hasKeyContext
-
Methods inherited from interface org.apache.flink.streaming.api.operators.StreamOperator
getOperatorAttributes
-
-
-
-
Field Detail
-
LOG
protected static final org.slf4j.Logger LOG
The logger used by the operator class and its subclasses.
-
config
protected transient StreamConfig config
-
output
protected transient Output<StreamRecord<OUT>> output
-
stateKeySelector1
protected transient KeySelector<?,?> stateKeySelector1
KeySelector
for extracting a key from an element being processed. This is used to scope keyed state to a key. This is null if the operator is not a keyed operator.This is for elements from the first input.
-
stateKeySelector2
protected transient KeySelector<?,?> stateKeySelector2
KeySelector
for extracting a key from an element being processed. This is used to scope keyed state to a key. This is null if the operator is not a keyed operator.This is for elements from the second input.
-
stateHandler
protected transient StreamOperatorStateHandler stateHandler
-
timeServiceManager
protected transient InternalTimeServiceManager<?> timeServiceManager
-
metrics
protected transient InternalOperatorMetricGroup metrics
Metric group for the operator.
-
latencyStats
protected transient LatencyStats latencyStats
-
processingTimeService
protected transient ProcessingTimeService processingTimeService
-
lastRecordAttributes1
protected transient RecordAttributes lastRecordAttributes1
-
lastRecordAttributes2
protected transient RecordAttributes lastRecordAttributes2
-
-
Constructor Detail
-
AbstractStreamOperator
public AbstractStreamOperator()
-
AbstractStreamOperator
public AbstractStreamOperator(StreamOperatorParameters<OUT> parameters)
-
-
Method Detail
-
setup
protected void setup(StreamTask<?,?> containingTask, StreamConfig config, Output<StreamRecord<OUT>> output)
-
setProcessingTimeService
protected void setProcessingTimeService(ProcessingTimeService processingTimeService)
-
getMetricGroup
public OperatorMetricGroup getMetricGroup()
- Specified by:
getMetricGroup
in interfaceStreamOperator<OUT>
-
initializeState
public void initializeState(StreamTaskStateInitializer streamTaskStateManager) throws Exception
Description copied from interface:StreamOperator
Provides a context to initialize all state in the operator.- Specified by:
initializeState
in interfaceStreamOperator<OUT>
- Throws:
Exception
-
isUsingCustomRawKeyedState
@Internal protected boolean isUsingCustomRawKeyedState()
Indicates whether or not implementations of this class is writing to the raw keyed state streams on snapshots, usingsnapshotState(StateSnapshotContext)
. If yes, subclasses should override this method to returntrue
.Subclasses need to explicitly indicate the use of raw keyed state because, internally, the
AbstractStreamOperator
may attempt to read from it as well to restore heap-based timers and ultimately fail with read errors. By setting this flag totrue
, this allows theAbstractStreamOperator
to know that the data written in the raw keyed states were not written by the timer services, and skips the timer restore attempt.Please refer to FLINK-19741 for further details.
TODO: this method can be removed once all timers are moved to be managed by state backends.
- Returns:
- flag indicating whether or not this operator is writing to raw keyed state via
snapshotState(StateSnapshotContext)
.
-
setMailboxExecutor
@Internal public void setMailboxExecutor(MailboxExecutor mailboxExecutor)
- Specified by:
setMailboxExecutor
in interfaceYieldingOperator<OUT>
-
useSplittableTimers
@Internal public boolean useSplittableTimers()
Can be overridden to disable splittable timers for this particular operator even if config option is enabled. By default, splittable timers are disabled.- Returns:
true
if splittable timers should be used (subject toStreamConfig.isUnalignedCheckpointsEnabled()
andStreamConfig.isUnalignedCheckpointsSplittableTimersEnabled()
.false
if splittable timers should never be used.
-
open
public void open() throws Exception
This method is called immediately before any elements are processed, it should contain the operator's initialization logic, e.g. state initialization.The default implementation does nothing.
- Specified by:
open
in interfaceStreamOperator<OUT>
- Throws:
Exception
- An exception in this method causes the operator to fail.
-
finish
public void finish() throws Exception
Description copied from interface:StreamOperator
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
StreamOperator.close()
method.- Specified by:
finish
in interfaceStreamOperator<OUT>
- Throws:
Exception
- An exception in this method causes the operator to fail.
-
close
public void close() throws Exception
Description copied from interface:StreamOperator
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
StreamOperator.finish()
method.- Specified by:
close
in interfaceStreamOperator<OUT>
- Throws:
Exception
-
prepareSnapshotPreBarrier
public void prepareSnapshotPreBarrier(long checkpointId) throws Exception
Description copied from interface:StreamOperator
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.
- Specified by:
prepareSnapshotPreBarrier
in interfaceStreamOperator<OUT>
- Parameters:
checkpointId
- The ID of the checkpoint.- Throws:
Exception
- Throwing an exception here causes the operator to fail and go into recovery.
-
snapshotState
public OperatorSnapshotFutures snapshotState(long checkpointId, long timestamp, CheckpointOptions checkpointOptions, CheckpointStreamFactory factory) throws Exception
Description copied from interface:StreamOperator
Called to draw a state snapshot from the operator.- Specified by:
snapshotState
in interfaceStreamOperator<OUT>
- 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.
-
snapshotState
public void snapshotState(StateSnapshotContext context) throws Exception
Stream operators with state, which want to participate in a snapshot need to override this hook method.- Specified by:
snapshotState
in interfaceStreamOperatorStateHandler.CheckpointedStreamOperator
- Parameters:
context
- context that provides information and means required for taking a snapshot- Throws:
Exception
-
initializeState
public void initializeState(StateInitializationContext context) throws Exception
Stream operators with state which can be restored need to override this hook method.- Specified by:
initializeState
in interfaceStreamOperatorStateHandler.CheckpointedStreamOperator
- Parameters:
context
- context that allows to register different states.- Throws:
Exception
-
notifyCheckpointComplete
public void notifyCheckpointComplete(long checkpointId) throws Exception
Description copied from interface:CheckpointListener
Notifies the listener that the checkpoint with the givencheckpointId
completed and was committed.These notifications are "best effort", meaning they can sometimes be skipped. To behave properly, implementers need to follow the "Checkpoint Subsuming Contract". Please see the
class-level JavaDocs
for details.Please note that checkpoints may generally overlap, so you cannot assume that the
notifyCheckpointComplete()
call is always for the latest prior checkpoint (or snapshot) that was taken on the function/operator implementing this interface. It might be for a checkpoint that was triggered earlier. Implementing the "Checkpoint Subsuming Contract" (see above) properly handles this situation correctly as well.Please note that throwing exceptions from this method will not cause the completed checkpoint to be revoked. Throwing exceptions will typically cause task/job failure and trigger recovery.
- Specified by:
notifyCheckpointComplete
in interfaceCheckpointListener
- Parameters:
checkpointId
- The ID of the checkpoint that has been completed.- Throws:
Exception
- This method can propagate exceptions, which leads to a failure/recovery for the task. Note that this will NOT lead to the checkpoint being revoked.
-
notifyCheckpointAborted
public void notifyCheckpointAborted(long checkpointId) throws Exception
Description copied from interface:CheckpointListener
This method is called as a notification once a distributed checkpoint has been aborted.Important: The fact that a checkpoint has been aborted does NOT mean that the data and artifacts produced between the previous checkpoint and the aborted checkpoint are to be discarded. The expected behavior is as if this checkpoint was never triggered in the first place, and the next successful checkpoint simply covers a longer time span. See the "Checkpoint Subsuming Contract" in the
class-level JavaDocs
for details.These notifications are "best effort", meaning they can sometimes be skipped.
This method is very rarely necessary to implement. The "best effort" guarantee, together with the fact that this method should not result in discarding any data (per the "Checkpoint Subsuming Contract") means it is mainly useful for earlier cleanups of auxiliary resources. One example is to pro-actively clear a local per-checkpoint state cache upon checkpoint failure.
- Specified by:
notifyCheckpointAborted
in interfaceCheckpointListener
- Parameters:
checkpointId
- The ID of the checkpoint that has been aborted.- Throws:
Exception
- This method can propagate exceptions, which leads to a failure/recovery for the task or job.
-
getExecutionConfig
public ExecutionConfig getExecutionConfig()
Gets the execution config defined on the execution environment of the job to which this operator belongs.- Returns:
- The job's execution config.
-
getOperatorConfig
public StreamConfig getOperatorConfig()
-
getContainingTask
public StreamTask<?,?> getContainingTask()
-
getUserCodeClassloader
public ClassLoader getUserCodeClassloader()
-
getOperatorName
protected String getOperatorName()
Return the operator name. If the runtime context has been set, then the task name with subtask index is returned. Otherwise, the simple class name is returned.- Returns:
- If runtime context is set, then return task name with subtask index. Otherwise return simple class name.
-
getRuntimeContext
@VisibleForTesting public StreamingRuntimeContext getRuntimeContext()
Returns a context that allows the operator to query information about the execution and also to interact with systems such as broadcast variables and managed state. This also allows to register timers.
-
getKeyedStateBackend
public <K> KeyedStateBackend<K> getKeyedStateBackend()
-
getOperatorStateBackend
@VisibleForTesting public OperatorStateBackend getOperatorStateBackend()
-
getProcessingTimeService
@VisibleForTesting public ProcessingTimeService getProcessingTimeService()
Returns theProcessingTimeService
responsible for getting the current processing time and registering timers.
-
getPartitionedState
protected <S extends State> S getPartitionedState(StateDescriptor<S,?> stateDescriptor) throws Exception
Creates a partitioned state handle, using the state backend configured for this task.- Throws:
IllegalStateException
- Thrown, if the key/value state was already initialized.Exception
- Thrown, if the state backend cannot create the key/value state.
-
getOrCreateKeyedState
protected <N,S extends State,T> S getOrCreateKeyedState(TypeSerializer<N> namespaceSerializer, StateDescriptor<S,T> stateDescriptor) throws Exception
- Throws:
Exception
-
getPartitionedState
protected <S extends State,N> S getPartitionedState(N namespace, TypeSerializer<N> namespaceSerializer, StateDescriptor<S,?> stateDescriptor) throws Exception
Creates a partitioned state handle, using the state backend configured for this task.- Throws:
IllegalStateException
- Thrown, if the key/value state was already initialized.Exception
- Thrown, if the state backend cannot create the key/value state.
-
setKeyContextElement1
public void setKeyContextElement1(StreamRecord record) throws Exception
- Specified by:
setKeyContextElement1
in interfaceStreamOperator<OUT>
- Throws:
Exception
-
setKeyContextElement2
public void setKeyContextElement2(StreamRecord record) throws Exception
- Specified by:
setKeyContextElement2
in interfaceStreamOperator<OUT>
- Throws:
Exception
-
hasKeyContext1
@Internal public boolean hasKeyContext1()
Description copied from interface:KeyContextHandler
Whether the first input ofStreamOperator
has "KeyContext". If false, we can omit the call ofStreamOperator.setKeyContextElement1(org.apache.flink.streaming.runtime.streamrecord.StreamRecord<?>)
for each record arrived on the first input.- Specified by:
hasKeyContext1
in interfaceKeyContextHandler
- Returns:
- True if the first input has "KeyContext", false otherwise.
-
hasKeyContext2
@Internal public boolean hasKeyContext2()
Description copied from interface:KeyContextHandler
Whether the second input ofStreamOperator
has "KeyContext". If false, we can omit the call ofStreamOperator.setKeyContextElement1(org.apache.flink.streaming.runtime.streamrecord.StreamRecord<?>)
for each record arrived on the second input.- Specified by:
hasKeyContext2
in interfaceKeyContextHandler
- Returns:
- True if the second input has "KeyContext", false otherwise.
-
setCurrentKey
public void setCurrentKey(Object key)
- Specified by:
setCurrentKey
in interfaceKeyContext
-
getCurrentKey
public Object getCurrentKey()
- Specified by:
getCurrentKey
in interfaceKeyContext
-
getKeyedStateStore
public KeyedStateStore getKeyedStateStore()
-
getStateKeySelector1
protected KeySelector<?,?> getStateKeySelector1()
-
getStateKeySelector2
protected KeySelector<?,?> getStateKeySelector2()
-
processLatencyMarker
public void processLatencyMarker(LatencyMarker latencyMarker) throws Exception
- Throws:
Exception
-
processLatencyMarker1
public void processLatencyMarker1(LatencyMarker latencyMarker) throws Exception
- Throws:
Exception
-
processLatencyMarker2
public void processLatencyMarker2(LatencyMarker latencyMarker) throws Exception
- Throws:
Exception
-
reportOrForwardLatencyMarker
protected void reportOrForwardLatencyMarker(LatencyMarker marker)
-
getInternalTimerService
public <K,N> InternalTimerService<N> getInternalTimerService(String name, TypeSerializer<N> namespaceSerializer, Triggerable<K,N> triggerable)
Returns aInternalTimerService
that can be used to query current processing time and event time and to set timers. An operator can have several timer services, where each has its own namespace serializer. Timer services are differentiated by the string key that is given when requesting them, if you call this method with the same key multiple times you will get the same timer service instance in subsequent requests.Timers are always scoped to a key, the currently active key of a keyed stream operation. When a timer fires, this key will also be set as the currently active key.
Each timer has attached metadata, the namespace. Different timer services can have a different namespace type. If you don't need namespace differentiation you can use
VoidNamespaceSerializer
as the namespace serializer.- Type Parameters:
N
- The type of the timer namespace.- Parameters:
name
- The name of the requested timer service. If no service exists under the given name a new one will be created and returned.namespaceSerializer
-TypeSerializer
for the timer namespace.triggerable
- TheTriggerable
that should be invoked when timers fire
-
processWatermarkStatus
public void processWatermarkStatus(WatermarkStatus watermarkStatus) throws Exception
- Throws:
Exception
-
processWatermarkStatus1
public final void processWatermarkStatus1(WatermarkStatus watermarkStatus) throws Exception
- Throws:
Exception
-
processWatermarkStatus2
public final void processWatermarkStatus2(WatermarkStatus watermarkStatus) throws Exception
- Throws:
Exception
-
getOperatorID
public OperatorID getOperatorID()
- Specified by:
getOperatorID
in interfaceStreamOperator<OUT>
-
getTimeServiceManager
protected Optional<InternalTimeServiceManager<?>> getTimeServiceManager()
-
processRecordAttributes
@Experimental public void processRecordAttributes(RecordAttributes recordAttributes) throws Exception
- Throws:
Exception
-
processRecordAttributes1
@Experimental public void processRecordAttributes1(RecordAttributes recordAttributes)
-
processRecordAttributes2
@Experimental public void processRecordAttributes2(RecordAttributes recordAttributes)
-
-