public class Task extends Object implements Runnable, TaskSlotPayload, TaskActions, PartitionProducerStateProvider
The Flink operators (implemented as subclasses of TaskInvokable
have only data
readers, writers, and certain event callbacks. The task connects those to the network stack and
actor messages, and tracks the state of the execution and handles exceptions.
Tasks have no knowledge about how they relate to other tasks, or whether they are the first attempt to execute the task, or a repeated attempt. All of that is only known to the JobManager. All the task knows are its own runnable code, the task's configuration, and the IDs of the intermediate results to consume and produce (if any).
Each Task is run by one dedicated thread.
Modifier and Type | Class and Description |
---|---|
static class |
Task.NotifyCheckpointOperation
Various operation of notify checkpoint.
|
PartitionProducerStateProvider.ResponseHandle
Constructor and Description |
---|
Task(JobInformation jobInformation,
TaskInformation taskInformation,
ExecutionAttemptID executionAttemptID,
AllocationID slotAllocationId,
List<ResultPartitionDeploymentDescriptor> resultPartitionDeploymentDescriptors,
List<InputGateDeploymentDescriptor> inputGateDeploymentDescriptors,
MemoryManager memManager,
SharedResources sharedResources,
IOManager ioManager,
ShuffleEnvironment<?,?> shuffleEnvironment,
KvStateService kvStateService,
BroadcastVariableManager bcVarManager,
TaskEventDispatcher taskEventDispatcher,
ExternalResourceInfoProvider externalResourceInfoProvider,
TaskStateManager taskStateManager,
TaskManagerActions taskManagerActions,
InputSplitProvider inputSplitProvider,
CheckpointResponder checkpointResponder,
TaskOperatorEventGateway operatorCoordinatorEventGateway,
GlobalAggregateManager aggregateManager,
LibraryCacheManager.ClassLoaderHandle classLoaderHandle,
FileCache fileCache,
TaskManagerRuntimeInfo taskManagerConfig,
TaskMetricGroup metricGroup,
PartitionProducerStateChecker partitionProducerStateChecker,
Executor executor,
ChannelStateWriteRequestExecutorFactory channelStateExecutorFactory)
IMPORTANT: This constructor may not start any work that would need to be undone in the
case of a failing task deployment.
|
Modifier and Type | Method and Description |
---|---|
void |
cancelExecution()
Cancels the task execution.
|
void |
deliverOperatorEvent(OperatorID operator,
SerializedValue<OperatorEvent> evt)
Dispatches an operator event to the invokable task.
|
void |
failExternally(Throwable cause)
Marks task execution failed for an external reason (a reason other than the task code itself
throwing an exception).
|
AccumulatorRegistry |
getAccumulatorRegistry() |
AllocationID |
getAllocationId() |
Thread |
getExecutingThread() |
ExecutionAttemptID |
getExecutionId() |
ExecutionState |
getExecutionState()
Returns the current execution state of the task.
|
Throwable |
getFailureCause()
If the task has failed, this method gets the exception that caused this task to fail.
|
Configuration |
getJobConfiguration() |
JobID |
getJobID() |
JobVertexID |
getJobVertexId() |
TaskMetricGroup |
getMetricGroup() |
Configuration |
getTaskConfiguration() |
TaskInfo |
getTaskInfo() |
CompletableFuture<ExecutionState> |
getTerminationFuture() |
boolean |
isBackPressured() |
boolean |
isCanceledOrFailed()
Checks whether the task has failed, is canceled, or is being canceled at the moment.
|
static void |
logTaskThreadStackTrace(Thread thread,
String taskName,
long timeoutMs,
String action) |
void |
notifyCheckpointAborted(long checkpointID,
long latestCompletedCheckpointId) |
void |
notifyCheckpointComplete(long checkpointID) |
void |
notifyCheckpointSubsumed(long checkpointID) |
void |
requestPartitionProducerState(IntermediateDataSetID intermediateDataSetId,
ResultPartitionID resultPartitionId,
Consumer<? super PartitionProducerStateProvider.ResponseHandle> responseConsumer)
Trigger the producer execution state request.
|
void |
run()
The core work method that bootstraps the task and executes its code.
|
static void |
setupPartitionsAndGates(ResultPartitionWriter[] producedPartitions,
InputGate[] inputGates) |
void |
startTaskThread()
Starts the task's thread.
|
String |
toString() |
void |
triggerCheckpointBarrier(long checkpointID,
long checkpointTimestamp,
CheckpointOptions checkpointOptions)
Calls the invokable to trigger a checkpoint.
|
public Task(JobInformation jobInformation, TaskInformation taskInformation, ExecutionAttemptID executionAttemptID, AllocationID slotAllocationId, List<ResultPartitionDeploymentDescriptor> resultPartitionDeploymentDescriptors, List<InputGateDeploymentDescriptor> inputGateDeploymentDescriptors, MemoryManager memManager, SharedResources sharedResources, IOManager ioManager, ShuffleEnvironment<?,?> shuffleEnvironment, KvStateService kvStateService, BroadcastVariableManager bcVarManager, TaskEventDispatcher taskEventDispatcher, ExternalResourceInfoProvider externalResourceInfoProvider, TaskStateManager taskStateManager, TaskManagerActions taskManagerActions, InputSplitProvider inputSplitProvider, CheckpointResponder checkpointResponder, TaskOperatorEventGateway operatorCoordinatorEventGateway, GlobalAggregateManager aggregateManager, LibraryCacheManager.ClassLoaderHandle classLoaderHandle, FileCache fileCache, TaskManagerRuntimeInfo taskManagerConfig, @Nonnull TaskMetricGroup metricGroup, PartitionProducerStateChecker partitionProducerStateChecker, Executor executor, ChannelStateWriteRequestExecutorFactory channelStateExecutorFactory)
public JobID getJobID()
getJobID
in interface TaskSlotPayload
public JobVertexID getJobVertexId()
public ExecutionAttemptID getExecutionId()
getExecutionId
in interface TaskSlotPayload
public AllocationID getAllocationId()
getAllocationId
in interface TaskSlotPayload
public TaskInfo getTaskInfo()
public Configuration getJobConfiguration()
public Configuration getTaskConfiguration()
public AccumulatorRegistry getAccumulatorRegistry()
public TaskMetricGroup getMetricGroup()
public Thread getExecutingThread()
public CompletableFuture<ExecutionState> getTerminationFuture()
getTerminationFuture
in interface TaskSlotPayload
public boolean isBackPressured()
public ExecutionState getExecutionState()
public boolean isCanceledOrFailed()
public Throwable getFailureCause()
public void startTaskThread()
public void run()
@VisibleForTesting public static void setupPartitionsAndGates(ResultPartitionWriter[] producedPartitions, InputGate[] inputGates) throws IOException
IOException
public void cancelExecution()
This method never blocks.
public void failExternally(Throwable cause)
This method never blocks.
failExternally
in interface TaskSlotPayload
failExternally
in interface TaskActions
cause
- of the failurepublic void requestPartitionProducerState(IntermediateDataSetID intermediateDataSetId, ResultPartitionID resultPartitionId, Consumer<? super PartitionProducerStateProvider.ResponseHandle> responseConsumer)
PartitionProducerStateProvider
requestPartitionProducerState
in interface PartitionProducerStateProvider
intermediateDataSetId
- ID of the parent intermediate data set.resultPartitionId
- ID of the result partition to check. This identifies the producing
execution and partition.responseConsumer
- consumer for the response handle.public void triggerCheckpointBarrier(long checkpointID, long checkpointTimestamp, CheckpointOptions checkpointOptions)
checkpointID
- The ID identifying the checkpoint.checkpointTimestamp
- The timestamp associated with the checkpoint.checkpointOptions
- Options for performing this checkpoint.public void notifyCheckpointComplete(long checkpointID)
public void notifyCheckpointAborted(long checkpointID, long latestCompletedCheckpointId)
public void notifyCheckpointSubsumed(long checkpointID)
public void deliverOperatorEvent(OperatorID operator, SerializedValue<OperatorEvent> evt) throws FlinkException
If the event delivery did not succeed, this method throws an exception. Callers can use that exception for error reporting, but need not react with failing this task (this method takes care of that).
FlinkException
- This method throws exceptions indicating the reason why delivery did
not succeed.Copyright © 2014–2024 The Apache Software Foundation. All rights reserved.