IN
- Input type for the operator.OUT
- Output type for the operator.@Internal public class AsyncWaitOperator<IN,OUT> extends AbstractUdfStreamOperator<OUT,AsyncFunction<IN,OUT>> implements OneInputStreamOperator<IN,OUT>, OperatorActions
AsyncWaitOperator
allows to asynchronously process incoming stream records. For that
the operator creates an ResultFuture
which is passed to an AsyncFunction
.
Within the async function, the user can complete the async collector arbitrarily. Once the async
collector has been completed, the result is emitted by the operator's emitter to downstream
operators.
The operator offers different output modes depending on the chosen
AsyncDataStream.OutputMode
. In order to give exactly once processing guarantees, the
operator stores all currently in-flight StreamElement
in it's operator state. Upon
recovery the recorded set of stream elements is replayed.
In case of chaining of this operator, it has to be made sure that the operators in the chain are
opened tail to head. The reason for this is that an opened AsyncWaitOperator
starts
already emitting recovered StreamElement
to downstream operators.
AbstractStreamOperator.CountingOutput
Modifier and Type | Field and Description |
---|---|
protected Object |
checkpointingLock |
userFunction
chainingStrategy, config, latencyStats, LOG, metrics, output, timeServiceManager
Constructor and Description |
---|
AsyncWaitOperator(AsyncFunction<IN,OUT> asyncFunction,
long timeout,
int capacity,
AsyncDataStream.OutputMode outputMode) |
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.
|
void |
failOperator(Throwable throwable)
Fail the respective stream operator with the given throwable.
|
void |
initializeState(StateInitializationContext context)
Stream operators with state which can be restored need to override this hook method.
|
void |
open()
This method is called immediately before any elements are processed, it should contain the
operator's initialization logic, e.g.
|
void |
processElement(StreamRecord<IN> element)
Processes one element that arrived at this operator.
|
void |
processWatermark(Watermark mark)
Processes a
Watermark . |
void |
setup(StreamTask<?,?> containingTask,
StreamConfig config,
Output<StreamRecord<OUT>> output)
Initializes the operator.
|
void |
snapshotState(StateSnapshotContext context)
Stream operators with state, which want to participate in a snapshot need to override this hook method.
|
getUserFunction, getUserFunctionParameters, notifyCheckpointComplete, setOutputType
getChainingStrategy, getContainingTask, getCurrentKey, getExecutionConfig, getInternalTimerService, getKeyedStateBackend, getKeyedStateStore, getMetricGroup, getOperatorConfig, getOperatorID, getOperatorName, getOperatorStateBackend, getOrCreateKeyedState, getPartitionedState, getPartitionedState, getProcessingTimeService, getRuntimeContext, getUserCodeClassloader, initializeState, numEventTimeTimers, numProcessingTimeTimers, processLatencyMarker, processLatencyMarker1, processLatencyMarker2, processWatermark1, processWatermark2, reportOrForwardLatencyMarker, setChainingStrategy, setCurrentKey, setKeyContextElement1, setKeyContextElement2, snapshotState
clone, equals, finalize, getClass, hashCode, notify, notifyAll, toString, wait, wait, wait
processLatencyMarker
getChainingStrategy, getMetricGroup, getOperatorID, initializeState, setChainingStrategy, setKeyContextElement1, setKeyContextElement2, snapshotState
notifyCheckpointComplete
getCurrentKey, setCurrentKey
protected transient Object checkpointingLock
public AsyncWaitOperator(AsyncFunction<IN,OUT> asyncFunction, long timeout, int capacity, AsyncDataStream.OutputMode outputMode)
public void setup(StreamTask<?,?> containingTask, StreamConfig config, Output<StreamRecord<OUT>> output)
StreamOperator
setup
in interface StreamOperator<OUT>
setup
in class AbstractUdfStreamOperator<OUT,AsyncFunction<IN,OUT>>
public void open() throws Exception
AbstractStreamOperator
The default implementation does nothing.
open
in interface StreamOperator<OUT>
open
in class AbstractUdfStreamOperator<OUT,AsyncFunction<IN,OUT>>
Exception
- An exception in this method causes the operator to fail.public void processElement(StreamRecord<IN> element) throws Exception
OneInputStreamOperator
processElement
in interface OneInputStreamOperator<IN,OUT>
Exception
public void processWatermark(Watermark mark) throws Exception
OneInputStreamOperator
Watermark
.
This method is guaranteed to not be called concurrently with other methods of the operator.processWatermark
in interface OneInputStreamOperator<IN,OUT>
processWatermark
in class AbstractStreamOperator<OUT>
Exception
Watermark
public void snapshotState(StateSnapshotContext context) throws Exception
AbstractStreamOperator
snapshotState
in class AbstractUdfStreamOperator<OUT,AsyncFunction<IN,OUT>>
context
- context that provides information and means required for taking a snapshotException
public void initializeState(StateInitializationContext context) throws Exception
AbstractStreamOperator
initializeState
in class AbstractUdfStreamOperator<OUT,AsyncFunction<IN,OUT>>
context
- context that allows to register different states.Exception
public void close() throws Exception
AbstractStreamOperator
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>
close
in class AbstractUdfStreamOperator<OUT,AsyncFunction<IN,OUT>>
Exception
- An exception in this method causes the operator to fail.public void dispose() throws Exception
AbstractStreamOperator
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
dispose
in class AbstractUdfStreamOperator<OUT,AsyncFunction<IN,OUT>>
Exception
- if something goes wrong during disposal.public void failOperator(Throwable throwable)
OperatorActions
failOperator
in interface OperatorActions
throwable
- to fail the stream operator withCopyright © 2014–2019 The Apache Software Foundation. All rights reserved.