Class AbstractAsyncStateStreamOperatorV2<OUT>
- java.lang.Object
-
- org.apache.flink.streaming.api.operators.AbstractStreamOperatorV2<OUT>
-
- org.apache.flink.runtime.asyncprocessing.operators.AbstractAsyncStateStreamOperatorV2<OUT>
-
- All Implemented Interfaces:
Serializable
,CheckpointListener
,KeyContext
,StreamOperator<OUT>
,StreamOperatorStateHandler.CheckpointedStreamOperator
,AsyncStateProcessing
,AsyncStateProcessingOperator
@Internal public abstract class AbstractAsyncStateStreamOperatorV2<OUT> extends AbstractStreamOperatorV2<OUT> implements AsyncStateProcessingOperator
This operator is an abstract class that give theAbstractStreamOperatorV2
the ability to performAsyncStateProcessing
. The aim is to make any subclass ofAbstractStreamOperatorV2
could manipulate async state with only a change of base class.- See Also:
- Serialized Form
-
-
Field Summary
-
Fields inherited from class org.apache.flink.streaming.api.operators.AbstractStreamOperatorV2
combinedWatermark, config, lastRecordAttributes, latencyStats, metrics, output, processingTimeService, stateHandler, timeServiceManager
-
-
Constructor Summary
Constructors Constructor Description AbstractAsyncStateStreamOperatorV2(StreamOperatorParameters<OUT> parameters, int numberOfInputs)
-
Method Summary
All Methods Instance Methods Concrete Methods Modifier and Type Method Description <K> void
asyncProcessWithKey(K key, ThrowingRunnable<Exception> processing)
Asynchronously process a code with a key provided.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.AsyncExecutionController<?>
getAsyncExecutionController()
RecordContext
getCurrentProcessingContext()
ElementOrder
getElementOrder()
Get theElementOrder
of this operator.<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.protected <N,S extends State,T>
SgetOrCreateKeyedState(N defaultNamespace, TypeSerializer<N> namespaceSerializer, StateDescriptor<T> stateDescriptor)
Create new state (v2) based on new state descriptor.<T> ThrowingConsumer<StreamRecord<T>,Exception>
getRecordProcessor(int inputId)
Get the record processor that could process record from input, which is the only entry for async processing.void
initializeState(StreamTaskStateInitializer streamTaskStateManager)
Initialize necessary state components forAbstractStreamOperatorV2
.boolean
isAsyncStateProcessingEnabled()
Indicates whether this operator is enabling the async state.void
postProcessElement()
A callback that will be triggered after an element finishesprocessElement
.void
prepareSnapshotPreBarrier(long checkpointId)
This method is called when the operator should do a snapshot, before it emits its own checkpoint barrier.void
preserveRecordOrderAndProcess(ThrowingRunnable<Exception> processing)
Check the order of same-key record, and then process the record.void
processRecordAttributes(RecordAttributes recordAttributes, int inputId)
void
processWatermark(Watermark mark)
void
processWatermarkStatus(WatermarkStatus watermarkStatus, int inputId)
protected void
reportOrForwardLatencyMarker(LatencyMarker marker)
<T> void
setAsyncKeyedContextElement(StreamRecord<T> record, KeySelector<T,?> keySelector)
Set key context for async state processing.-
Methods inherited from class org.apache.flink.streaming.api.operators.AbstractStreamOperatorV2
getCurrentKey, getExecutionConfig, getKeyedStateBackend, getKeyedStateStore, getMetricGroup, getOperatorConfig, getOperatorID, getOperatorName, getOperatorStateBackend, getOrCreateKeyedState, getPartitionedState, getPartitionedState, getProcessingTimeService, getRuntimeContext, getTimeServiceManager, getUserCodeClassloader, initializeState, internalSetKeyContextElement, isUsingCustomRawKeyedState, notifyCheckpointAborted, notifyCheckpointComplete, open, reportWatermark, setCurrentKey, setKeyContextElement1, setKeyContextElement2, snapshotState, snapshotState, useSplittableTimers
-
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.StreamOperator
getOperatorAttributes
-
-
-
-
Constructor Detail
-
AbstractAsyncStateStreamOperatorV2
public AbstractAsyncStateStreamOperatorV2(StreamOperatorParameters<OUT> parameters, int numberOfInputs)
-
-
Method Detail
-
initializeState
public final void initializeState(StreamTaskStateInitializer streamTaskStateManager) throws Exception
Initialize necessary state components forAbstractStreamOperatorV2
.- Specified by:
initializeState
in interfaceStreamOperator<OUT>
- Overrides:
initializeState
in classAbstractStreamOperatorV2<OUT>
- Throws:
Exception
-
isAsyncStateProcessingEnabled
public boolean isAsyncStateProcessingEnabled()
Description copied from class:AbstractStreamOperatorV2
Indicates whether this operator is enabling the async state. Can be overridden by subclasses.- Specified by:
isAsyncStateProcessingEnabled
in interfaceAsyncStateProcessing
- Overrides:
isAsyncStateProcessingEnabled
in classAbstractStreamOperatorV2<OUT>
- Returns:
- ture if async state processing is enabled.
-
getElementOrder
public ElementOrder getElementOrder()
Description copied from interface:AsyncStateProcessingOperator
Get theElementOrder
of this operator.- Specified by:
getElementOrder
in interfaceAsyncStateProcessingOperator
-
setAsyncKeyedContextElement
public final <T> void setAsyncKeyedContextElement(StreamRecord<T> record, KeySelector<T,?> keySelector) throws Exception
Description copied from interface:AsyncStateProcessingOperator
Set key context for async state processing.- Specified by:
setAsyncKeyedContextElement
in interfaceAsyncStateProcessingOperator
- Type Parameters:
T
- the type of the record.- Parameters:
record
- the record.keySelector
- the key selector to select a key from record.- Throws:
Exception
-
postProcessElement
public final void postProcessElement()
Description copied from interface:AsyncStateProcessingOperator
A callback that will be triggered after an element finishesprocessElement
.- Specified by:
postProcessElement
in interfaceAsyncStateProcessingOperator
-
preserveRecordOrderAndProcess
public final void preserveRecordOrderAndProcess(ThrowingRunnable<Exception> processing)
Description copied from interface:AsyncStateProcessingOperator
Check the order of same-key record, and then process the record. Mainly used when theAsyncStateProcessingOperator.getElementOrder()
returnsElementOrder.RECORD_ORDER
.- Specified by:
preserveRecordOrderAndProcess
in interfaceAsyncStateProcessingOperator
- Parameters:
processing
- the record processing logic.
-
asyncProcessWithKey
public <K> void asyncProcessWithKey(K key, ThrowingRunnable<Exception> processing)
Description copied from interface:AsyncStateProcessingOperator
Asynchronously process a code with a key provided.- Specified by:
asyncProcessWithKey
in interfaceAsyncStateProcessingOperator
- Type Parameters:
K
- the type of key.- Parameters:
key
- the specified key.processing
- the process logic.
-
getRecordProcessor
public final <T> ThrowingConsumer<StreamRecord<T>,Exception> getRecordProcessor(int inputId)
Description copied from interface:AsyncStateProcessing
Get the record processor that could process record from input, which is the only entry for async processing.- Specified by:
getRecordProcessor
in interfaceAsyncStateProcessing
- Parameters:
inputId
- the input identifier, start from 1. Borrow the design fromorg.apache.flink.streaming.api.operators.AbstractInput#inputId
. This is only relevant if there is multiple inputs for the instance.
-
getOrCreateKeyedState
protected <N,S extends State,T> S getOrCreateKeyedState(@Nonnull N defaultNamespace, @Nonnull TypeSerializer<N> namespaceSerializer, @Nonnull StateDescriptor<T> stateDescriptor) throws Exception
Create new state (v2) based on new state descriptor.- 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>
- Overrides:
prepareSnapshotPreBarrier
in classAbstractStreamOperatorV2<OUT>
- Parameters:
checkpointId
- The ID of the checkpoint.- Throws:
Exception
- Throwing an exception here causes the operator to fail and go into recovery.
-
getInternalTimerService
public <K,N> InternalTimerService<N> getInternalTimerService(String name, TypeSerializer<N> namespaceSerializer, Triggerable<K,N> triggerable)
Description copied from class:AbstractStreamOperatorV2
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.- Overrides:
getInternalTimerService
in classAbstractStreamOperatorV2<OUT>
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
-
reportOrForwardLatencyMarker
protected void reportOrForwardLatencyMarker(LatencyMarker marker)
- Overrides:
reportOrForwardLatencyMarker
in classAbstractStreamOperatorV2<OUT>
-
processWatermark
public void processWatermark(Watermark mark) throws Exception
- Overrides:
processWatermark
in classAbstractStreamOperatorV2<OUT>
- Throws:
Exception
-
processWatermarkStatus
public void processWatermarkStatus(WatermarkStatus watermarkStatus, int inputId) throws Exception
- Overrides:
processWatermarkStatus
in classAbstractStreamOperatorV2<OUT>
- Throws:
Exception
-
processRecordAttributes
public void processRecordAttributes(RecordAttributes recordAttributes, int inputId) throws Exception
- Overrides:
processRecordAttributes
in classAbstractStreamOperatorV2<OUT>
- Throws:
Exception
-
getAsyncExecutionController
@VisibleForTesting public AsyncExecutionController<?> getAsyncExecutionController()
-
getCurrentProcessingContext
@VisibleForTesting public RecordContext getCurrentProcessingContext()
-
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>
- Overrides:
finish
in classAbstractStreamOperatorV2<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>
- Overrides:
close
in classAbstractStreamOperatorV2<OUT>
- Throws:
Exception
-
-