OUT
- OP
- @Internal public abstract class StreamTask<OUT,OP extends StreamOperator<OUT>> extends Object implements TaskInvokable, CheckpointableTask, CoordinatedTask, AsyncExceptionHandler, ContainingTaskDetails
StreamOperator
s which form the
Task's operator chain. Operators that are chained together execute synchronously in the same
thread and hence on the same stream partition. A common case for these chains are successive
map/flatmap/filter tasks.
The task chain contains one "head" operator and multiple chained operators. The StreamTask is specialized for the type of the head operator: one-input and two-input tasks, as well as for sources, iteration heads and iteration tails.
The Task class deals with the setup of the streams read by the head operator, and the streams produced by the operators at the ends of the operator chain. Note that the chain may fork and thus have multiple ends.
The life cycle of the task is set up as follows:
-- setInitialState -> provides state of all operators in the chain
-- invoke()
|
+----> Create basic utils (config, etc) and load the chain of operators
+----> operators.setup()
+----> task specific init()
+----> initialize-operator-states()
+----> open-operators()
+----> run()
+----> finish-operators()
+----> close-operators()
+----> common cleanup
+----> task specific cleanup()
The StreamTask
has a lock object called lock
. All calls to methods on a StreamOperator
must be synchronized on this lock object to ensure that no methods are called
concurrently.
Modifier and Type | Field and Description |
---|---|
protected CheckpointStorage |
checkpointStorage
Our checkpoint storage.
|
protected StreamConfig |
configuration
The configuration of this streaming task.
|
protected StreamInputProcessor |
inputProcessor
The input processor.
|
protected static org.slf4j.Logger |
LOG
The logger used by the StreamTask and its subclasses.
|
protected MailboxProcessor |
mailboxProcessor |
protected OP |
mainOperator
the main operator that consumes the input streams of this task.
|
protected OperatorChain<OUT,OP> |
operatorChain
The chain of operators executed by this task.
|
protected RecordWriterDelegate<SerializationDelegate<StreamRecord<OUT>>> |
recordWriter |
protected StateBackend |
stateBackend
Our state backend.
|
protected TimerService |
systemTimerService
In contrast to
timerService we should not register any user timers here. |
protected TimerService |
timerService
The internal
TimerService used to define the current processing time (default =
System.currentTimeMillis() ) and register timers for tasks to be executed in the
future. |
static ThreadGroup |
TRIGGER_THREAD_GROUP
The thread group that holds all trigger timer threads.
|
Modifier | Constructor and Description |
---|---|
protected |
StreamTask(Environment env)
Constructor for initialization, possibly with initial state (recovery / savepoint / etc).
|
protected |
StreamTask(Environment env,
TimerService timerService)
Constructor for initialization, possibly with initial state (recovery / savepoint / etc).
|
protected |
StreamTask(Environment environment,
TimerService timerService,
Thread.UncaughtExceptionHandler uncaughtExceptionHandler) |
protected |
StreamTask(Environment environment,
TimerService timerService,
Thread.UncaughtExceptionHandler uncaughtExceptionHandler,
StreamTaskActionExecutor actionExecutor)
Constructor for initialization, possibly with initial state (recovery / savepoint / etc).
|
protected |
StreamTask(Environment environment,
TimerService timerService,
Thread.UncaughtExceptionHandler uncaughtExceptionHandler,
StreamTaskActionExecutor actionExecutor,
TaskMailbox mailbox) |
Modifier and Type | Method and Description |
---|---|
void |
abortCheckpointOnBarrier(long checkpointId,
CheckpointException cause)
Aborts a checkpoint as the result of receiving possibly some checkpoint barriers, but at
least one
CancelCheckpointMarker . |
protected void |
advanceToEndOfEventTime()
Emits the
MAX_WATERMARK so that all registered timers are fired. |
protected void |
afterInvoke() |
void |
cancel()
This method is called when a task is canceled either as a result of a user abort or an
execution failure.
|
protected void |
cancelTask() |
void |
cleanUp(Throwable throwable)
Cleanup any resources used in
TaskInvokable.invoke() OR TaskInvokable.restore() . |
protected void |
cleanUpInternal() |
static <OUT> RecordWriterDelegate<SerializationDelegate<StreamRecord<OUT>>> |
createRecordWriterDelegate(StreamConfig configuration,
Environment environment) |
StreamTaskStateInitializer |
createStreamTaskStateInitializer() |
protected void |
declineCheckpoint(long checkpointId) |
void |
dispatchOperatorEvent(OperatorID operator,
SerializedValue<OperatorEvent> event) |
protected void |
endData(StopMode mode) |
protected void |
finalize()
The finalize method shuts down the timer.
|
protected long |
getAsyncCheckpointStartDelayNanos() |
ExecutorService |
getAsyncOperationsThreadPool() |
CloseableRegistry |
getCancelables() |
protected Optional<CheckpointBarrierHandler> |
getCheckpointBarrierHandler()
Acquires the optional
CheckpointBarrierHandler associated with this stream task. |
CheckpointStorageWorkerView |
getCheckpointStorage() |
protected CompletableFuture<Void> |
getCompletionFuture() |
StreamConfig |
getConfiguration() |
Environment |
getEnvironment() |
MailboxExecutorFactory |
getMailboxExecutorFactory() |
String |
getName()
Gets the name of the task, in the form "taskname (2/5)".
|
ProcessingTimeServiceFactory |
getProcessingTimeServiceFactory() |
void |
handleAsyncException(String message,
Throwable exception)
Handles an exception thrown by another thread (e.g.
|
protected abstract void |
init() |
void |
invoke()
Starts the execution.
|
boolean |
isCanceled() |
boolean |
isFailing() |
boolean |
isMailboxLoopRunning() |
boolean |
isRunning() |
boolean |
isUsingNonBlockingInput() |
void |
maybeInterruptOnCancel(Thread toInterrupt,
String taskName,
Long timeout)
Checks whether the task should be interrupted during cancellation and if so, execute the
specified
Runnable interruptAction . |
Future<Void> |
notifyCheckpointAbortAsync(long checkpointId,
long latestCompletedCheckpointId)
Invoked when a checkpoint has been aborted, i.e., when the checkpoint coordinator has
received a decline message from one task and try to abort the targeted checkpoint by
notification.
|
Future<Void> |
notifyCheckpointCompleteAsync(long checkpointId)
Invoked when a checkpoint has been completed, i.e., when the checkpoint coordinator has
received the notification from all participating tasks.
|
Future<Void> |
notifyCheckpointSubsumedAsync(long checkpointId)
Invoked when a checkpoint has been subsumed, i.e., when the checkpoint coordinator has
confirmed one checkpoint has been finished, and try to remove the first previous checkpoint.
|
protected void |
processInput(MailboxDefaultAction.Controller controller)
This method implements the default action of the task (e.g.
|
void |
restore()
This method can be called before
TaskInvokable.invoke() to restore an invokable object for the
last valid state, if it has it. |
void |
runMailboxLoop() |
boolean |
runMailboxStep() |
protected void |
setSynchronousSavepoint(long checkpointId) |
protected Counter |
setupNumRecordsInCounter(StreamOperator streamOperator) |
String |
toString() |
CompletableFuture<Boolean> |
triggerCheckpointAsync(CheckpointMetaData checkpointMetaData,
CheckpointOptions checkpointOptions)
This method is called to trigger a checkpoint, asynchronously by the checkpoint coordinator.
|
void |
triggerCheckpointOnBarrier(CheckpointMetaData checkpointMetaData,
CheckpointOptions checkpointOptions,
CheckpointMetricsBuilder checkpointMetrics)
This method is called when a checkpoint is triggered as a result of receiving checkpoint
barriers on all input streams.
|
clone, equals, getClass, hashCode, notify, notifyAll, wait, wait, wait
getExecutionConfig, getIndexInSubtaskGroup, getJobConfiguration, getUserCodeClassLoader
public static final ThreadGroup TRIGGER_THREAD_GROUP
protected static final org.slf4j.Logger LOG
@Nullable protected StreamInputProcessor inputProcessor
init()
method.protected OP extends StreamOperator<OUT> mainOperator
protected OperatorChain<OUT,OP extends StreamOperator<OUT>> operatorChain
protected final StreamConfig configuration
protected final StateBackend stateBackend
protected final CheckpointStorage checkpointStorage
protected final TimerService timerService
TimerService
used to define the current processing time (default =
System.currentTimeMillis()
) and register timers for tasks to be executed in the
future.protected final TimerService systemTimerService
timerService
we should not register any user timers here. It should
be used only for system level timers.protected final RecordWriterDelegate<SerializationDelegate<StreamRecord<OUT>>> recordWriter
protected final MailboxProcessor mailboxProcessor
protected StreamTask(Environment env) throws Exception
env
- The task environment for this task.Exception
protected StreamTask(Environment env, @Nullable TimerService timerService) throws Exception
env
- The task environment for this task.timerService
- Optionally, a specific timer service to use.Exception
protected StreamTask(Environment environment, @Nullable TimerService timerService, Thread.UncaughtExceptionHandler uncaughtExceptionHandler) throws Exception
Exception
protected StreamTask(Environment environment, @Nullable TimerService timerService, Thread.UncaughtExceptionHandler uncaughtExceptionHandler, StreamTaskActionExecutor actionExecutor) throws Exception
This constructor accepts a special TimerService
. By default (and if null is passes
for the timer service) a DefaultTimerService
will be
used.
environment
- The task environment for this task.timerService
- Optionally, a specific timer service to use.uncaughtExceptionHandler
- to handle uncaught exceptions in the async operations thread
poolactionExecutor
- a mean to wrap all actions performed by this task thread. Currently,
only SynchronizedActionExecutor can be used to preserve locking semantics.Exception
protected StreamTask(Environment environment, @Nullable TimerService timerService, Thread.UncaughtExceptionHandler uncaughtExceptionHandler, StreamTaskActionExecutor actionExecutor, TaskMailbox mailbox) throws Exception
Exception
protected void processInput(MailboxDefaultAction.Controller controller) throws Exception
controller
- controller object for collaborative interaction between the action and the
stream task.Exception
- on any problems in the action.protected void setSynchronousSavepoint(long checkpointId)
protected void advanceToEndOfEventTime() throws Exception
MAX_WATERMARK
so that all registered timers are fired.
This is used by the source task when the job is TERMINATED
. In the case, we want
all the timers registered throughout the pipeline to fire and the related state (e.g.
windows) to be flushed.
For tasks other than the source task, this method does nothing.
Exception
public StreamTaskStateInitializer createStreamTaskStateInitializer()
protected Counter setupNumRecordsInCounter(StreamOperator streamOperator)
public final void restore() throws Exception
TaskInvokable
TaskInvokable.invoke()
to restore an invokable object for the
last valid state, if it has it.
If TaskInvokable.invoke()
is not called after this method for some reason (e.g. task
cancellation); then all resources should be cleaned up by calling TaskInvokable.cleanUp(Throwable)
()} after the method returns.
restore
in interface TaskInvokable
Exception
public final void invoke() throws Exception
TaskInvokable
This method is called by the task manager when the actual execution of the task starts.
All resources should be cleaned up by calling TaskInvokable.cleanUp(Throwable)
()} after the
method returns.
invoke
in interface TaskInvokable
Exception
@VisibleForTesting public boolean runMailboxStep() throws Exception
Exception
@VisibleForTesting public boolean isMailboxLoopRunning()
public final void cleanUp(Throwable throwable) throws Exception
TaskInvokable
TaskInvokable.invoke()
OR TaskInvokable.restore()
. This method must be
called regardless whether the aforementioned calls succeeded or failed.cleanUp
in interface TaskInvokable
throwable
- iff failure happened during the execution of TaskInvokable.restore()
or TaskInvokable.invoke()
, null otherwise.
ATTENTION: CancelTaskException
should not be treated as a failure.
Exception
protected CompletableFuture<Void> getCompletionFuture()
public final void cancel() throws Exception
TaskInvokable
cancel
in interface TaskInvokable
Exception
public MailboxExecutorFactory getMailboxExecutorFactory()
public final boolean isRunning()
public final boolean isCanceled()
public final boolean isFailing()
protected void finalize() throws Throwable
This should not be relied upon! It will cause shutdown to happen much later than if manual shutdown is attempted, and cause threads to linger for longer than needed.
public final String getName()
public CheckpointStorageWorkerView getCheckpointStorage()
public StreamConfig getConfiguration()
public CompletableFuture<Boolean> triggerCheckpointAsync(CheckpointMetaData checkpointMetaData, CheckpointOptions checkpointOptions)
CheckpointableTask
This method is called for tasks that start the checkpoints by injecting the initial
barriers, i.e., the source tasks. In contrast, checkpoints on downstream operators, which are
the result of receiving checkpoint barriers, invoke the CheckpointableTask.triggerCheckpointOnBarrier(CheckpointMetaData, CheckpointOptions, CheckpointMetricsBuilder)
method.
triggerCheckpointAsync
in interface CheckpointableTask
checkpointMetaData
- Meta data for about this checkpointcheckpointOptions
- Options for performing this checkpointfalse
if the checkpoint was not carried out, true
otherwiseprotected Optional<CheckpointBarrierHandler> getCheckpointBarrierHandler()
CheckpointBarrierHandler
associated with this stream task. The
CheckpointBarrierHandler
should exist if the task has data inputs and requires to
align the barriers.public void triggerCheckpointOnBarrier(CheckpointMetaData checkpointMetaData, CheckpointOptions checkpointOptions, CheckpointMetricsBuilder checkpointMetrics) throws IOException
CheckpointableTask
triggerCheckpointOnBarrier
in interface CheckpointableTask
checkpointMetaData
- Meta data for about this checkpointcheckpointOptions
- Options for performing this checkpointcheckpointMetrics
- Metrics about this checkpointIOException
- Exceptions thrown as the result of triggering a checkpoint are forwarded.public void abortCheckpointOnBarrier(long checkpointId, CheckpointException cause) throws IOException
CheckpointableTask
CancelCheckpointMarker
.
This requires implementing tasks to forward a CancelCheckpointMarker
to their outputs.
abortCheckpointOnBarrier
in interface CheckpointableTask
checkpointId
- The ID of the checkpoint to be aborted.cause
- The reason why the checkpoint was aborted during alignmentIOException
protected void declineCheckpoint(long checkpointId)
public final ExecutorService getAsyncOperationsThreadPool()
public Future<Void> notifyCheckpointCompleteAsync(long checkpointId)
CheckpointableTask
notifyCheckpointCompleteAsync
in interface CheckpointableTask
checkpointId
- The ID of the checkpoint that is complete.public Future<Void> notifyCheckpointAbortAsync(long checkpointId, long latestCompletedCheckpointId)
CheckpointableTask
notifyCheckpointAbortAsync
in interface CheckpointableTask
checkpointId
- The ID of the checkpoint that is aborted.latestCompletedCheckpointId
- The ID of the latest completed checkpoint.public Future<Void> notifyCheckpointSubsumedAsync(long checkpointId)
CheckpointableTask
notifyCheckpointSubsumedAsync
in interface CheckpointableTask
checkpointId
- The ID of the checkpoint that is subsumed.public void dispatchOperatorEvent(OperatorID operator, SerializedValue<OperatorEvent> event) throws FlinkException
dispatchOperatorEvent
in interface CoordinatedTask
FlinkException
public ProcessingTimeServiceFactory getProcessingTimeServiceFactory()
public void handleAsyncException(String message, Throwable exception)
In more detail, it marks task execution failed for an external reason (a reason other than the task code itself throwing an exception). If the task is already in a terminal state (such as FINISHED, CANCELED, FAILED), or if the task is already canceling this does nothing. Otherwise it sets the state to FAILED, and, if the invokable code is running, starts an asynchronous thread that aborts that code.
This method never blocks.
handleAsyncException
in interface AsyncExceptionHandler
public final CloseableRegistry getCancelables()
@VisibleForTesting public static <OUT> RecordWriterDelegate<SerializationDelegate<StreamRecord<OUT>>> createRecordWriterDelegate(StreamConfig configuration, Environment environment)
protected long getAsyncCheckpointStartDelayNanos()
public boolean isUsingNonBlockingInput()
isUsingNonBlockingInput
in interface TaskInvokable
InputGate.getNext()
is used (as opposed to
InputGate.pollNext()
. To be removed together with the DataSet API.public void maybeInterruptOnCancel(Thread toInterrupt, @Nullable String taskName, @Nullable Long timeout)
TaskInvokable
Runnable interruptAction
.maybeInterruptOnCancel
in interface TaskInvokable
taskName
- optional taskName to log stack tracetimeout
- optional timeout to log stack tracepublic final Environment getEnvironment()
Copyright © 2014–2024 The Apache Software Foundation. All rights reserved.