Class SourceOperator<OUT,SplitT extends SourceSplit>
- java.lang.Object
-
- org.apache.flink.streaming.api.operators.AbstractStreamOperator<OUT>
-
- org.apache.flink.streaming.api.operators.SourceOperator<OUT,SplitT>
-
- Type Parameters:
OUT
- The output type of the operator.
- All Implemented Interfaces:
Serializable
,CheckpointListener
,AvailabilityProvider
,OperatorEventHandler
,KeyContext
,KeyContextHandler
,TimestampsAndWatermarks.WatermarkUpdateListener
,StreamOperator<OUT>
,StreamOperatorStateHandler.CheckpointedStreamOperator
,YieldingOperator<OUT>
,PushingAsyncDataInput<OUT>
@Internal public class SourceOperator<OUT,SplitT extends SourceSplit> extends AbstractStreamOperator<OUT> implements OperatorEventHandler, PushingAsyncDataInput<OUT>, TimestampsAndWatermarks.WatermarkUpdateListener
Base source operator only used for integrating the source reader which is proposed by FLIP-27. It implements the interface ofPushingAsyncDataInput
which is naturally compatible with one input processing in runtime stack.Important Note on Serialization: The SourceOperator inherits the
Serializable
interface from the StreamOperator, but is in fact NOT serializable. The operator must only be instantiated in the StreamTask from its factory.- See Also:
- Serialized Form
-
-
Nested Class Summary
-
Nested classes/interfaces inherited from interface org.apache.flink.runtime.io.AvailabilityProvider
AvailabilityProvider.AvailabilityHelper
-
Nested classes/interfaces inherited from interface org.apache.flink.streaming.runtime.io.PushingAsyncDataInput
PushingAsyncDataInput.DataOutput<T>
-
-
Field Summary
-
Fields inherited from class org.apache.flink.streaming.api.operators.AbstractStreamOperator
combinedWatermark, config, lastRecordAttributes1, lastRecordAttributes2, latencyStats, LOG, metrics, output, processingTimeService, stateHandler, stateKeySelector1, stateKeySelector2, timeServiceManager
-
Fields inherited from interface org.apache.flink.runtime.io.AvailabilityProvider
AVAILABLE
-
-
Constructor Summary
Constructors Constructor Description SourceOperator(StreamOperatorParameters<OUT> parameters, FunctionWithException<SourceReaderContext,SourceReader<OUT,SplitT>,Exception> readerFactory, OperatorEventGateway operatorEventGateway, SimpleVersionedSerializer<SplitT> splitSerializer, WatermarkStrategy<OUT> watermarkStrategy, ProcessingTimeService timeService, Configuration configuration, String localHostname, boolean emitProgressiveWatermarks, StreamTask.CanEmitBatchOfRecordsChecker canEmitBatchOfRecords)
-
Method Summary
All Methods Instance Methods Concrete Methods Modifier and Type Method 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.DataInputStatus
emitNext(PushingAsyncDataInput.DataOutput<OUT> output)
Pushes elements to the output from current data input, and returns the input status to indicate whether there are more available data in current input.void
finish()
This method is called at the end of data processing.CompletableFuture<?>
getAvailableFuture()
InternalSourceReaderMetricGroup
getSourceMetricGroup()
SourceReader<OUT,SplitT>
getSourceReader()
void
handleOperatorEvent(OperatorEvent event)
void
initializeState(StateInitializationContext context)
Stream operators with state which can be restored need to override this hook method.void
initReader()
Initializes the reader.protected void
initSourceMetricGroup()
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 givencheckpointId
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.protected void
setup(StreamTask<?,?> containingTask, StreamConfig config, Output<StreamRecord<OUT>> output)
void
snapshotState(StateSnapshotContext context)
Stream operators with state, which want to participate in a snapshot need to override this hook method.void
splitFinished(String splitId)
Notifies that split has finished.CompletableFuture<Void>
stop(StopMode mode)
void
updateCurrentEffectiveWatermark(long watermark)
Update the effective watermark.void
updateCurrentSplitWatermark(String splitId, long watermark)
Notifies about changes to per split watermarks.void
updateIdle(boolean isIdle)
It should be called once the idle is changed.-
Methods inherited from class org.apache.flink.streaming.api.operators.AbstractStreamOperator
getContainingTask, getCurrentKey, getExecutionConfig, getInternalTimerService, getKeyedStateBackend, getKeyedStateStore, getMetricGroup, getOperatorConfig, getOperatorID, getOperatorName, getOperatorStateBackend, getOrCreateKeyedState, getPartitionedState, getPartitionedState, getProcessingTimeService, getRuntimeContext, getStateKeySelector1, getStateKeySelector2, getTimeServiceManager, getUserCodeClassloader, hasKeyContext1, hasKeyContext2, initializeState, isAsyncStateProcessingEnabled, isUsingCustomRawKeyedState, prepareSnapshotPreBarrier, processLatencyMarker, processLatencyMarker1, processLatencyMarker2, processRecordAttributes, processRecordAttributes1, processRecordAttributes2, processWatermark, processWatermark1, processWatermark2, processWatermarkStatus, processWatermarkStatus, processWatermarkStatus1, processWatermarkStatus2, reportOrForwardLatencyMarker, setCurrentKey, setKeyContextElement1, setKeyContextElement2, setMailboxExecutor, setProcessingTimeService, 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.runtime.io.AvailabilityProvider
isApproximatelyAvailable, isAvailable
-
Methods inherited from interface org.apache.flink.streaming.api.operators.KeyContextHandler
hasKeyContext
-
Methods inherited from interface org.apache.flink.streaming.api.operators.StreamOperator
getOperatorAttributes
-
-
-
-
Constructor Detail
-
SourceOperator
public SourceOperator(StreamOperatorParameters<OUT> parameters, FunctionWithException<SourceReaderContext,SourceReader<OUT,SplitT>,Exception> readerFactory, OperatorEventGateway operatorEventGateway, SimpleVersionedSerializer<SplitT> splitSerializer, WatermarkStrategy<OUT> watermarkStrategy, ProcessingTimeService timeService, Configuration configuration, String localHostname, boolean emitProgressiveWatermarks, StreamTask.CanEmitBatchOfRecordsChecker canEmitBatchOfRecords)
-
-
Method Detail
-
setup
protected void setup(StreamTask<?,?> containingTask, StreamConfig config, Output<StreamRecord<OUT>> output)
- Overrides:
setup
in classAbstractStreamOperator<OUT>
-
initSourceMetricGroup
@VisibleForTesting protected void initSourceMetricGroup()
-
initReader
public void initReader() throws Exception
Initializes the reader. The code from this method should ideally happen in the constructor or in the operator factory even. It has to happen here at a slightly later stage, because of the lazy metric initialization.Calling this method explicitly is an optional way to have the reader initialization a bit earlier than in open(), as needed by the
SourceOperatorStreamTask
This code should move to the constructor once the metric groups are available at task setup time.
- Throws:
Exception
-
getSourceMetricGroup
public InternalSourceReaderMetricGroup getSourceMetricGroup()
-
open
public void open() throws Exception
Description copied from class:AbstractStreamOperator
This method is called immediately before any elements are processed, it should contain the operator's initialization logic, e.g. state initialization.The default implementation does nothing.
- Specified by:
open
in interfaceStreamOperator<OUT>
- Overrides:
open
in classAbstractStreamOperator<OUT>
- Throws:
Exception
- An exception in this method causes the operator to fail.
-
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 classAbstractStreamOperator<OUT>
- Throws:
Exception
- An exception in this method causes the operator to fail.
-
stop
public CompletableFuture<Void> stop(StopMode mode)
-
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 classAbstractStreamOperator<OUT>
- Throws:
Exception
-
emitNext
public DataInputStatus emitNext(PushingAsyncDataInput.DataOutput<OUT> output) throws Exception
Description copied from interface:PushingAsyncDataInput
Pushes elements to the output from current data input, and returns the input status to indicate whether there are more available data in current input.This method should be non blocking.
- Specified by:
emitNext
in interfacePushingAsyncDataInput<OUT>
- Throws:
Exception
-
snapshotState
public void snapshotState(StateSnapshotContext context) throws Exception
Description copied from class:AbstractStreamOperator
Stream operators with state, which want to participate in a snapshot need to override this hook method.- Specified by:
snapshotState
in interfaceStreamOperatorStateHandler.CheckpointedStreamOperator
- Overrides:
snapshotState
in classAbstractStreamOperator<OUT>
- Parameters:
context
- context that provides information and means required for taking a snapshot- Throws:
Exception
-
getAvailableFuture
public CompletableFuture<?> getAvailableFuture()
- Specified by:
getAvailableFuture
in interfaceAvailabilityProvider
- Returns:
- a future that is completed if the respective provider is available.
-
initializeState
public void initializeState(StateInitializationContext context) throws Exception
Description copied from class:AbstractStreamOperator
Stream operators with state which can be restored need to override this hook method.- Specified by:
initializeState
in interfaceStreamOperatorStateHandler.CheckpointedStreamOperator
- Overrides:
initializeState
in classAbstractStreamOperator<OUT>
- Parameters:
context
- context that allows to register different states.- Throws:
Exception
-
notifyCheckpointComplete
public void notifyCheckpointComplete(long checkpointId) throws Exception
Description copied from interface:CheckpointListener
Notifies the listener that the checkpoint with the givencheckpointId
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.
- Specified by:
notifyCheckpointComplete
in interfaceCheckpointListener
- Overrides:
notifyCheckpointComplete
in classAbstractStreamOperator<OUT>
- Parameters:
checkpointId
- The ID of the checkpoint that has been completed.- Throws:
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.
-
notifyCheckpointAborted
public void notifyCheckpointAborted(long checkpointId) throws Exception
Description copied from interface:CheckpointListener
This method is called as a notification once a distributed checkpoint has been aborted.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.
- Specified by:
notifyCheckpointAborted
in interfaceCheckpointListener
- Overrides:
notifyCheckpointAborted
in classAbstractStreamOperator<OUT>
- Parameters:
checkpointId
- The ID of the checkpoint that has been aborted.- Throws:
Exception
- This method can propagate exceptions, which leads to a failure/recovery for the task or job.
-
handleOperatorEvent
public void handleOperatorEvent(OperatorEvent event)
- Specified by:
handleOperatorEvent
in interfaceOperatorEventHandler
-
updateIdle
public void updateIdle(boolean isIdle)
Description copied from interface:TimestampsAndWatermarks.WatermarkUpdateListener
It should be called once the idle is changed.- Specified by:
updateIdle
in interfaceTimestampsAndWatermarks.WatermarkUpdateListener
-
updateCurrentEffectiveWatermark
public void updateCurrentEffectiveWatermark(long watermark)
Description copied from interface:TimestampsAndWatermarks.WatermarkUpdateListener
Update the effective watermark. If an output becomes idle, please call {@link this#updateIdle} instead of update the watermark toLong.MAX_VALUE
. Because the output needs to distinguish between idle and real watermark.- Specified by:
updateCurrentEffectiveWatermark
in interfaceTimestampsAndWatermarks.WatermarkUpdateListener
-
updateCurrentSplitWatermark
public void updateCurrentSplitWatermark(String splitId, long watermark)
Description copied from interface:TimestampsAndWatermarks.WatermarkUpdateListener
Notifies about changes to per split watermarks.- Specified by:
updateCurrentSplitWatermark
in interfaceTimestampsAndWatermarks.WatermarkUpdateListener
-
splitFinished
public void splitFinished(String splitId)
Description copied from interface:TimestampsAndWatermarks.WatermarkUpdateListener
Notifies that split has finished.- Specified by:
splitFinished
in interfaceTimestampsAndWatermarks.WatermarkUpdateListener
-
getSourceReader
@VisibleForTesting public SourceReader<OUT,SplitT> getSourceReader()
-
-