OUT
- The output type of the operator@Experimental public abstract class AbstractStreamOperatorV2<OUT> extends Object implements StreamOperator<OUT>, StreamOperatorStateHandler.CheckpointedStreamOperator
AbstractStreamOperator
. Currently intended to work smoothly just with MultipleInputStreamOperator
.
One note-able difference in comparison to AbstractStreamOperator
is lack of AbstractStreamOperator.setup(StreamTask, StreamConfig, Output)
in favor of initialisation in the
constructor, and removed some tight coupling with classes like StreamTask
.
Methods are guaranteed not to be called concurrently.
Modifier and Type | Field and Description |
---|---|
protected StreamConfig |
config |
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 |
Constructor and Description |
---|
AbstractStreamOperatorV2(StreamOperatorParameters<OUT> parameters,
int numberOfInputs) |
Modifier and Type | Method and 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.
|
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 a
InternalTimerService that can be used to query current processing time and
event time and to set timers. |
<K> KeyedStateBackend<K> |
getKeyedStateBackend() |
Optional<KeyedStateStore> |
getKeyedStateStore() |
OperatorMetricGroup |
getMetricGroup() |
StreamConfig |
getOperatorConfig() |
OperatorID |
getOperatorID() |
protected String |
getOperatorName()
Return the operator name.
|
OperatorStateBackend |
getOperatorStateBackend() |
protected <N,S extends State,T> |
getOrCreateKeyedState(TypeSerializer<N> namespaceSerializer,
StateDescriptor<S,T> stateDescriptor) |
protected <S extends State,N> |
getPartitionedState(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> |
getPartitionedState(StateDescriptor<S,?> stateDescriptor)
Creates a partitioned state handle, using the state backend configured for this task.
|
ProcessingTimeService |
getProcessingTimeService()
Returns the
ProcessingTimeService 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 Optional<InternalTimeServiceManager<?>> |
getTimeServiceManager() |
ClassLoader |
getUserCodeClassloader() |
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 <T> void |
internalSetKeyContextElement(StreamRecord<T> record,
KeySelector<T,?> selector) |
protected boolean |
isUsingCustomRawKeyedState()
Indicates whether or not implementations of this class is writing to the raw keyed state
streams on snapshots, using
snapshotState(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 given
checkpointId 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 |
processWatermark(Watermark mark) |
void |
processWatermarkStatus(WatermarkStatus watermarkStatus,
int inputId) |
protected void |
reportOrForwardLatencyMarker(LatencyMarker marker) |
protected void |
reportWatermark(Watermark mark,
int inputId) |
void |
setCurrentKey(Object key) |
void |
setKeyContextElement1(StreamRecord<?> record) |
void |
setKeyContextElement2(StreamRecord<?> record) |
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.
|
protected static final org.slf4j.Logger LOG
protected final StreamConfig config
protected final Output<StreamRecord<OUT>> output
protected final InternalOperatorMetricGroup metrics
protected final LatencyStats latencyStats
protected final ProcessingTimeService processingTimeService
public AbstractStreamOperatorV2(StreamOperatorParameters<OUT> parameters, int numberOfInputs)
public OperatorMetricGroup getMetricGroup()
getMetricGroup
in interface StreamOperator<OUT>
public final void initializeState(StreamTaskStateInitializer streamTaskStateManager) throws Exception
StreamOperator
initializeState
in interface StreamOperator<OUT>
Exception
@Internal protected boolean isUsingCustomRawKeyedState()
snapshotState(StateSnapshotContext)
. If yes, subclasses
should override this method to return true
.
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 to true
, this
allows the AbstractStreamOperator
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.
snapshotState(StateSnapshotContext)
.public void open() throws Exception
The default implementation does nothing.
open
in interface StreamOperator<OUT>
Exception
- An exception in this method causes the operator to fail.public void finish() throws Exception
StreamOperator
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.
finish
in interface StreamOperator<OUT>
Exception
- An exception in this method causes the operator to fail.public void close() throws Exception
StreamOperator
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.
close
in interface StreamOperator<OUT>
Exception
public void prepareSnapshotPreBarrier(long checkpointId) throws Exception
StreamOperator
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.
prepareSnapshotPreBarrier
in interface StreamOperator<OUT>
checkpointId
- The ID of the checkpoint.Exception
- Throwing an exception here causes the operator to fail and go into
recovery.public final OperatorSnapshotFutures snapshotState(long checkpointId, long timestamp, CheckpointOptions checkpointOptions, CheckpointStreamFactory factory) throws Exception
StreamOperator
snapshotState
in interface StreamOperator<OUT>
Exception
- exception that happened during snapshotting.public void snapshotState(StateSnapshotContext context) throws Exception
snapshotState
in interface StreamOperatorStateHandler.CheckpointedStreamOperator
context
- context that provides information and means required for taking a snapshotException
public void initializeState(StateInitializationContext context) throws Exception
initializeState
in interface StreamOperatorStateHandler.CheckpointedStreamOperator
context
- context that allows to register different states.Exception
public void notifyCheckpointComplete(long checkpointId) throws Exception
CheckpointListener
checkpointId
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.
notifyCheckpointComplete
in interface CheckpointListener
checkpointId
- The ID of the checkpoint that has been completed.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.public void notifyCheckpointAborted(long checkpointId) throws Exception
CheckpointListener
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.
notifyCheckpointAborted
in interface CheckpointListener
checkpointId
- The ID of the checkpoint that has been aborted.Exception
- This method can propagate exceptions, which leads to a failure/recovery for
the task or job.public ExecutionConfig getExecutionConfig()
public StreamConfig getOperatorConfig()
public ClassLoader getUserCodeClassloader()
protected String getOperatorName()
public StreamingRuntimeContext getRuntimeContext()
public <K> KeyedStateBackend<K> getKeyedStateBackend()
@VisibleForTesting public OperatorStateBackend getOperatorStateBackend()
@VisibleForTesting public ProcessingTimeService getProcessingTimeService()
ProcessingTimeService
responsible for getting the current processing time
and registering timers.protected <S extends State> S getPartitionedState(StateDescriptor<S,?> stateDescriptor) throws Exception
IllegalStateException
- Thrown, if the key/value state was already initialized.Exception
- Thrown, if the state backend cannot create the key/value state.protected <N,S extends State,T> S getOrCreateKeyedState(TypeSerializer<N> namespaceSerializer, StateDescriptor<S,T> stateDescriptor) throws Exception
Exception
protected <S extends State,N> S getPartitionedState(N namespace, TypeSerializer<N> namespaceSerializer, StateDescriptor<S,?> stateDescriptor) throws Exception
IllegalStateException
- Thrown, if the key/value state was already initialized.Exception
- Thrown, if the state backend cannot create the key/value state.protected <T> void internalSetKeyContextElement(StreamRecord<T> record, KeySelector<T,?> selector) throws Exception
Exception
public void setCurrentKey(Object key)
setCurrentKey
in interface KeyContext
public Object getCurrentKey()
getCurrentKey
in interface KeyContext
public Optional<KeyedStateStore> getKeyedStateStore()
protected void reportOrForwardLatencyMarker(LatencyMarker marker)
@VisibleForTesting public <K,N> InternalTimerService<N> getInternalTimerService(String name, TypeSerializer<N> namespaceSerializer, Triggerable<K,N> triggerable)
InternalTimerService
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.
N
- The type of the timer namespace.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
- The Triggerable
that should be invoked when timers fireprotected void reportWatermark(Watermark mark, int inputId) throws Exception
Exception
public final void processWatermarkStatus(WatermarkStatus watermarkStatus, int inputId) throws Exception
Exception
public OperatorID getOperatorID()
getOperatorID
in interface StreamOperator<OUT>
public void setKeyContextElement1(StreamRecord<?> record) throws Exception
setKeyContextElement1
in interface StreamOperator<OUT>
Exception
public void setKeyContextElement2(StreamRecord<?> record) throws Exception
setKeyContextElement2
in interface StreamOperator<OUT>
Exception
protected Optional<InternalTimeServiceManager<?>> getTimeServiceManager()
Copyright © 2014–2024 The Apache Software Foundation. All rights reserved.