OUT
- The output type of the operator@PublicEvolving public interface StreamOperator<OUT> extends CheckpointListener, KeyContext, Disposable, Serializable
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
.
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() |
MetricGroup |
getMetricGroup() |
OperatorID |
getOperatorID() |
void |
initializeState()
Provides a context to initialize all state in the operator.
|
void |
open()
This method is called immediately before any elements are processed, it should contain the
operator's initialization logic.
|
void |
prepareSnapshotPreBarrier(long checkpointId)
This method is called when the operator should do a snapshot, before it emits its
own checkpoint barrier.
|
void |
setChainingStrategy(ChainingStrategy strategy) |
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 storageLocation)
Called to draw a state snapshot from the operator.
|
notifyCheckpointComplete
getCurrentKey, setCurrentKey
void setup(StreamTask<?,?> containingTask, StreamConfig config, Output<StreamRecord<OUT>> output)
void open() throws Exception
Exception
- An exception in this method causes the operator to fail.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 as failed, because the last data items are not processed properly.
Exception
- An exception in this method causes the operator to fail.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 Disposable
Exception
- if something goes wrong during disposal.void prepareSnapshotPreBarrier(long checkpointId) throws Exception
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.
checkpointId
- The ID of the checkpoint.Exception
- Throwing an exception here causes the operator to fail and go into recovery.OperatorSnapshotFutures snapshotState(long checkpointId, long timestamp, CheckpointOptions checkpointOptions, CheckpointStreamFactory storageLocation) throws Exception
Exception
- exception that happened during snapshotting.void initializeState() throws Exception
Exception
void setKeyContextElement1(StreamRecord<?> record) throws Exception
Exception
void setKeyContextElement2(StreamRecord<?> record) throws Exception
Exception
ChainingStrategy getChainingStrategy()
void setChainingStrategy(ChainingStrategy strategy)
MetricGroup getMetricGroup()
OperatorID getOperatorID()
Copyright © 2014–2020 The Apache Software Foundation. All rights reserved.