OUT
- OP
- @Internal public abstract class StreamTask<OUT,OP extends StreamOperator<OUT>> extends AbstractInvokable implements AsyncExceptionHandler
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()
+----> close-operators()
+----> dispose-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 | Class and Description |
---|---|
protected static class |
StreamTask.AsyncCheckpointRunnable
This runnable executes the asynchronous parts of all involved backend snapshots for the subtask.
|
Modifier and Type | Field and Description |
---|---|
protected StreamConfig |
configuration
The configuration of this streaming task.
|
protected OP |
headOperator
the head operator that consumes the input streams of this task.
|
protected OperatorChain<OUT,OP> |
operatorChain
The chain of operators executed by this task.
|
protected StateBackend |
stateBackend
Our state backend.
|
protected ProcessingTimeService |
timerService
The internal
ProcessingTimeService 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 environment,
ProcessingTimeService timeProvider)
Constructor for initialization, possibly with initial state (recovery / savepoint / etc).
|
Modifier and Type | Method and Description |
---|---|
void |
abortCheckpointOnBarrier(long checkpointId,
Throwable cause)
Aborts a checkpoint as the result of receiving possibly some checkpoint barriers,
but at least one
CancelCheckpointMarker . |
void |
cancel()
This method is called when a task is canceled either as a result of a user abort or an execution failure.
|
protected abstract void |
cancelTask() |
protected abstract void |
cleanup() |
protected CheckpointExceptionHandlerFactory |
createCheckpointExceptionHandlerFactory() |
static <OUT> List<StreamRecordWriter<SerializationDelegate<StreamRecord<OUT>>>> |
createStreamRecordWriters(StreamConfig configuration,
Environment environment) |
StreamTaskStateInitializer |
createStreamTaskStateInitializer() |
protected void |
finalize()
The finalize method shuts down the timer.
|
Map<String,Accumulator<?,?>> |
getAccumulatorMap() |
ExecutorService |
getAsyncOperationsThreadPool() |
CloseableRegistry |
getCancelables() |
Object |
getCheckpointLock()
Gets the lock object on which all operations that involve data and state mutation have to lock.
|
CheckpointStorage |
getCheckpointStorage() |
StreamConfig |
getConfiguration() |
String |
getName()
Gets the name of the task, in the form "taskname (2/5)".
|
ProcessingTimeService |
getProcessingTimeService()
Returns the
ProcessingTimeService responsible for telling the current
processing time and registering timers. |
StreamStatusMaintainer |
getStreamStatusMaintainer() |
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 |
isRunning() |
void |
notifyCheckpointComplete(long checkpointId)
Invoked when a checkpoint has been completed, i.e., when the checkpoint coordinator has received
the notification from all participating tasks.
|
protected abstract void |
run() |
String |
toString() |
boolean |
triggerCheckpoint(CheckpointMetaData checkpointMetaData,
CheckpointOptions checkpointOptions)
This method is called to trigger a checkpoint, asynchronously by the checkpoint
coordinator.
|
void |
triggerCheckpointOnBarrier(CheckpointMetaData checkpointMetaData,
CheckpointOptions checkpointOptions,
CheckpointMetrics checkpointMetrics)
This method is called when a checkpoint is triggered as a result of receiving checkpoint
barriers on all input streams.
|
getCurrentNumberOfSubtasks, getEnvironment, getExecutionConfig, getIndexInSubtaskGroup, getJobConfiguration, getTaskConfiguration, getUserCodeClassLoader
public static final ThreadGroup TRIGGER_THREAD_GROUP
protected OP extends StreamOperator<OUT> headOperator
protected OperatorChain<OUT,OP extends StreamOperator<OUT>> operatorChain
protected final StreamConfig configuration
protected StateBackend stateBackend
protected ProcessingTimeService timerService
ProcessingTimeService
used to define the current
processing time (default = System.currentTimeMillis()
) and
register timers for tasks to be executed in the future.protected StreamTask(Environment env)
env
- The task environment for this task.protected StreamTask(Environment environment, @Nullable ProcessingTimeService timeProvider)
This constructor accepts a special ProcessingTimeService
. By default (and if
null is passes for the time provider) a DefaultTimerService
will be used.
environment
- The task environment for this task.timeProvider
- Optionally, a specific time provider to use.public StreamTaskStateInitializer createStreamTaskStateInitializer()
public final void invoke() throws Exception
AbstractInvokable
Must be overwritten by the concrete task implementation. This method is called by the task manager when the actual execution of the task starts.
All resources should be cleaned up when the method returns. Make sure
to guard the code with try-finally
blocks where necessary.
invoke
in class AbstractInvokable
Exception
- Tasks may forward their exceptions for the TaskManager to handle through failure/recovery.public final void cancel() throws Exception
AbstractInvokable
cancel
in class AbstractInvokable
Exception
- thrown if any exception occurs during the execution of the user codepublic final boolean isRunning()
public final boolean isCanceled()
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 String getName()
public Object getCheckpointLock()
public CheckpointStorage getCheckpointStorage()
public StreamConfig getConfiguration()
public Map<String,Accumulator<?,?>> getAccumulatorMap()
public StreamStatusMaintainer getStreamStatusMaintainer()
public boolean triggerCheckpoint(CheckpointMetaData checkpointMetaData, CheckpointOptions checkpointOptions) throws Exception
AbstractInvokable
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 AbstractInvokable.triggerCheckpointOnBarrier(CheckpointMetaData, CheckpointOptions, CheckpointMetrics)
method.
triggerCheckpoint
in class AbstractInvokable
checkpointMetaData
- Meta data for about this checkpointcheckpointOptions
- Options for performing this checkpointfalse
if the checkpoint can not be carried out, true
otherwiseException
public void triggerCheckpointOnBarrier(CheckpointMetaData checkpointMetaData, CheckpointOptions checkpointOptions, CheckpointMetrics checkpointMetrics) throws Exception
AbstractInvokable
triggerCheckpointOnBarrier
in class AbstractInvokable
checkpointMetaData
- Meta data for about this checkpointcheckpointOptions
- Options for performing this checkpointcheckpointMetrics
- Metrics about this checkpointException
- Exceptions thrown as the result of triggering a checkpoint are forwarded.public void abortCheckpointOnBarrier(long checkpointId, Throwable cause) throws Exception
AbstractInvokable
CancelCheckpointMarker
.
This requires implementing tasks to forward a
CancelCheckpointMarker
to their outputs.
abortCheckpointOnBarrier
in class AbstractInvokable
checkpointId
- The ID of the checkpoint to be aborted.cause
- The reason why the checkpoint was aborted during alignmentException
public ExecutorService getAsyncOperationsThreadPool()
public void notifyCheckpointComplete(long checkpointId) throws Exception
AbstractInvokable
notifyCheckpointComplete
in class AbstractInvokable
checkpointId
- The ID of the checkpoint that is complete..Exception
- The notification method may forward its exceptions.protected CheckpointExceptionHandlerFactory createCheckpointExceptionHandlerFactory()
public ProcessingTimeService getProcessingTimeService()
ProcessingTimeService
responsible for telling the current
processing time and registering timers.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 CloseableRegistry getCancelables()
@VisibleForTesting public static <OUT> List<StreamRecordWriter<SerializationDelegate<StreamRecord<OUT>>>> createStreamRecordWriters(StreamConfig configuration, Environment environment)
Copyright © 2014–2019 The Apache Software Foundation. All rights reserved.