OUT
- The output type of the operator@PublicEvolving public abstract class AbstractStreamOperator<OUT> extends Object implements StreamOperator<OUT>, Serializable
AbstractUdfStreamOperator
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
or TwoInputStreamOperator
.
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
.
Modifier and Type | Class and Description |
---|---|
class |
AbstractStreamOperator.CountingOutput
Wrapping
Output that updates metrics on the number of emitted elements. |
Modifier and Type | Field and Description |
---|---|
protected ChainingStrategy |
chainingStrategy |
protected StreamConfig |
config |
protected LatencyStats |
latencyStats |
protected static org.slf4j.Logger |
LOG
The logger used by the operator class and its subclasses.
|
protected OperatorMetricGroup |
metrics
Metric group for the operator.
|
protected Output<StreamRecord<OUT>> |
output |
protected InternalTimeServiceManager<?,?> |
timeServiceManager |
Constructor and Description |
---|
AbstractStreamOperator() |
Modifier and Type | Method and Description |
---|---|
void |
close()
This method is called after all records have been added to the operators via the methods
OneInputStreamOperator.processElement(StreamRecord) , or
TwoInputStreamOperator.processElement1(StreamRecord) and
TwoInputStreamOperator.processElement2(StreamRecord) . |
void |
dispose()
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.
|
ChainingStrategy |
getChainingStrategy() |
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 a
InternalTimerService that can be used to query current processing time
and event time and to set timers. |
<K> KeyedStateBackend<K> |
getKeyedStateBackend() |
KeyedStateStore |
getKeyedStateStore() |
MetricGroup |
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.
|
protected 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.
|
ClassLoader |
getUserCodeClassloader() |
void |
initializeState()
Provides a context to initialize all state in the operator.
|
void |
initializeState(StateInitializationContext context)
Stream operators with state which can be restored need to override this hook method.
|
void |
notifyCheckpointComplete(long checkpointId)
This method is called as a notification once a distributed checkpoint has been completed.
|
int |
numEventTimeTimers() |
int |
numProcessingTimeTimers() |
void |
open()
This method is called immediately before any elements are processed, it should contain the
operator's initialization logic, e.g.
|
void |
processLatencyMarker(LatencyMarker latencyMarker) |
void |
processLatencyMarker1(LatencyMarker latencyMarker) |
void |
processLatencyMarker2(LatencyMarker latencyMarker) |
void |
processWatermark(Watermark mark) |
void |
processWatermark1(Watermark mark) |
void |
processWatermark2(Watermark mark) |
protected void |
reportOrForwardLatencyMarker(LatencyMarker marker) |
void |
setChainingStrategy(ChainingStrategy strategy) |
void |
setCurrentKey(Object key) |
void |
setKeyContextElement1(StreamRecord record) |
void |
setKeyContextElement2(StreamRecord record) |
void |
setup(StreamTask<?,?> containingTask,
StreamConfig config,
Output<StreamRecord<OUT>> output)
Initializes the operator.
|
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 ChainingStrategy chainingStrategy
protected transient StreamConfig config
protected transient Output<StreamRecord<OUT>> output
protected transient OperatorMetricGroup metrics
protected transient LatencyStats latencyStats
protected transient InternalTimeServiceManager<?,?> timeServiceManager
public void setup(StreamTask<?,?> containingTask, StreamConfig config, Output<StreamRecord<OUT>> output)
StreamOperator
setup
in interface StreamOperator<OUT>
public MetricGroup getMetricGroup()
getMetricGroup
in interface StreamOperator<OUT>
public final void initializeState() throws Exception
StreamOperator
initializeState
in interface StreamOperator<OUT>
Exception
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 close() throws Exception
OneInputStreamOperator.processElement(StreamRecord)
, or
TwoInputStreamOperator.processElement1(StreamRecord)
and
TwoInputStreamOperator.processElement2(StreamRecord)
.
The method is expected to flush all remaining buffered data. Exceptions during this flushing of buffered should be propagated, in order to cause the operation to be recognized asa failed, because the last data items are not processed properly.
close
in interface StreamOperator<OUT>
Exception
- An exception in this method causes the operator to fail.public void dispose() throws Exception
This method is expected to make a thorough effort to release all resources that the operator has acquired.
dispose
in interface StreamOperator<OUT>
dispose
in interface Disposable
Exception
- if something goes wrong during disposal.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
context
- context that provides information and means required for taking a snapshotException
public void initializeState(StateInitializationContext context) throws Exception
context
- context that allows to register different states.Exception
public void notifyCheckpointComplete(long checkpointId) throws Exception
CheckpointListener
notifyCheckpointComplete
in interface CheckpointListener
checkpointId
- The ID of the checkpoint that has been completed.Exception
public ExecutionConfig getExecutionConfig()
public StreamConfig getOperatorConfig()
public StreamTask<?,?> getContainingTask()
public ClassLoader getUserCodeClassloader()
protected String getOperatorName()
public StreamingRuntimeContext getRuntimeContext()
public <K> KeyedStateBackend<K> getKeyedStateBackend()
public OperatorStateBackend getOperatorStateBackend()
protected 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.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
public void setCurrentKey(Object key)
setCurrentKey
in interface KeyContext
public Object getCurrentKey()
getCurrentKey
in interface KeyContext
public KeyedStateStore getKeyedStateStore()
public final void setChainingStrategy(ChainingStrategy strategy)
setChainingStrategy
in interface StreamOperator<OUT>
public final ChainingStrategy getChainingStrategy()
getChainingStrategy
in interface StreamOperator<OUT>
public void processLatencyMarker(LatencyMarker latencyMarker) throws Exception
Exception
public void processLatencyMarker1(LatencyMarker latencyMarker) throws Exception
Exception
public void processLatencyMarker2(LatencyMarker latencyMarker) throws Exception
Exception
protected void reportOrForwardLatencyMarker(LatencyMarker marker)
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 firepublic OperatorID getOperatorID()
getOperatorID
in interface StreamOperator<OUT>
@VisibleForTesting public int numProcessingTimeTimers()
@VisibleForTesting public int numEventTimeTimers()
Copyright © 2014–2019 The Apache Software Foundation. All rights reserved.