public class Task extends Object implements Runnable, TaskSlotPayload, TaskActions, PartitionProducerStateProvider, CheckpointListener, BackPressureSampleableTask
The Flink operators (implemented as subclasses of
AbstractInvokable 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.
PartitionProducerStateProvider.ResponseHandle| Constructor and Description |
|---|
Task(JobInformation jobInformation,
TaskInformation taskInformation,
ExecutionAttemptID executionAttemptID,
AllocationID slotAllocationId,
int subtaskIndex,
int attemptNumber,
Collection<ResultPartitionDeploymentDescriptor> resultPartitionDeploymentDescriptors,
Collection<InputGateDeploymentDescriptor> inputGateDeploymentDescriptors,
int targetSlotNumber,
MemoryManager memManager,
IOManager ioManager,
ShuffleEnvironment<?,?> shuffleEnvironment,
KvStateService kvStateService,
BroadcastVariableManager bcVarManager,
TaskEventDispatcher taskEventDispatcher,
TaskStateManager taskStateManager,
TaskManagerActions taskManagerActions,
InputSplitProvider inputSplitProvider,
CheckpointResponder checkpointResponder,
GlobalAggregateManager aggregateManager,
BlobCacheService blobService,
LibraryCacheManager libraryCache,
FileCache fileCache,
TaskManagerRuntimeInfo taskManagerConfig,
TaskMetricGroup metricGroup,
ResultPartitionConsumableNotifier resultPartitionConsumableNotifier,
PartitionProducerStateChecker partitionProducerStateChecker,
Executor executor)
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 |
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.
|
org.apache.flink.configuration.Configuration |
getJobConfiguration() |
org.apache.flink.api.common.JobID |
getJobID() |
JobVertexID |
getJobVertexId() |
TaskMetricGroup |
getMetricGroup() |
org.apache.flink.configuration.Configuration |
getTaskConfiguration() |
org.apache.flink.api.common.TaskInfo |
getTaskInfo() |
CompletableFuture<ExecutionState> |
getTerminationFuture() |
boolean |
isBackPressured() |
boolean |
isCanceledOrFailed()
Checks whether the task has failed, is canceled, or is being canceled at the moment.
|
boolean |
isRunning() |
void |
notifyCheckpointComplete(long checkpointID)
This method is called as a notification once a distributed checkpoint has been completed.
|
void |
requestPartitionProducerState(IntermediateDataSetID intermediateDataSetId,
ResultPartitionID resultPartitionId,
java.util.function.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,
boolean advanceToEndOfEventTime)
Calls the invokable to trigger a checkpoint.
|
public Task(JobInformation jobInformation, TaskInformation taskInformation, ExecutionAttemptID executionAttemptID, AllocationID slotAllocationId, int subtaskIndex, int attemptNumber, Collection<ResultPartitionDeploymentDescriptor> resultPartitionDeploymentDescriptors, Collection<InputGateDeploymentDescriptor> inputGateDeploymentDescriptors, int targetSlotNumber, MemoryManager memManager, IOManager ioManager, ShuffleEnvironment<?,?> shuffleEnvironment, KvStateService kvStateService, BroadcastVariableManager bcVarManager, TaskEventDispatcher taskEventDispatcher, TaskStateManager taskStateManager, TaskManagerActions taskManagerActions, InputSplitProvider inputSplitProvider, CheckpointResponder checkpointResponder, GlobalAggregateManager aggregateManager, BlobCacheService blobService, LibraryCacheManager libraryCache, FileCache fileCache, TaskManagerRuntimeInfo taskManagerConfig, @Nonnull TaskMetricGroup metricGroup, ResultPartitionConsumableNotifier resultPartitionConsumableNotifier, PartitionProducerStateChecker partitionProducerStateChecker, Executor executor)
IMPORTANT: This constructor may not start any work that would need to be undone in the case of a failing task deployment.
public org.apache.flink.api.common.JobID getJobID()
getJobID in interface TaskSlotPayloadpublic JobVertexID getJobVertexId()
public ExecutionAttemptID getExecutionId()
getExecutionId in interface TaskSlotPayloadpublic AllocationID getAllocationId()
getAllocationId in interface TaskSlotPayloadpublic org.apache.flink.api.common.TaskInfo getTaskInfo()
public org.apache.flink.configuration.Configuration getJobConfiguration()
public org.apache.flink.configuration.Configuration getTaskConfiguration()
public AccumulatorRegistry getAccumulatorRegistry()
public TaskMetricGroup getMetricGroup()
public Thread getExecutingThread()
public CompletableFuture<ExecutionState> getTerminationFuture()
getTerminationFuture in interface TaskSlotPayloadpublic boolean isBackPressured()
isBackPressured in interface BackPressureSampleableTaskpublic ExecutionState getExecutionState()
public boolean isCanceledOrFailed()
public boolean isRunning()
isRunning in interface BackPressureSampleableTaskpublic Throwable getFailureCause()
public void startTaskThread()
public void run()
@VisibleForTesting public static void setupPartitionsAndGates(ResultPartitionWriter[] producedPartitions, InputGate[] inputGates) throws IOException, InterruptedException
IOExceptionInterruptedExceptionpublic void cancelExecution()
This method never blocks.
public void failExternally(Throwable cause)
This method never blocks.
failExternally in interface TaskSlotPayloadfailExternally in interface TaskActionscause - of the failurepublic void requestPartitionProducerState(IntermediateDataSetID intermediateDataSetId, ResultPartitionID resultPartitionId, java.util.function.Consumer<? super PartitionProducerStateProvider.ResponseHandle> responseConsumer)
PartitionProducerStateProviderrequestPartitionProducerState in interface PartitionProducerStateProviderintermediateDataSetId - 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,
boolean advanceToEndOfEventTime)
checkpointID - The ID identifying the checkpoint.checkpointTimestamp - The timestamp associated with the checkpoint.checkpointOptions - Options for performing this checkpoint.advanceToEndOfEventTime - Flag indicating if the source should inject a MAX_WATERMARK in the pipeline
to fire any registered event-time timers.public void notifyCheckpointComplete(long checkpointID)
CheckpointListenernotifyCheckpointComplete in interface CheckpointListenercheckpointID - The ID of the checkpoint that has been completed.Copyright © 2014–2020 The Apache Software Foundation. All rights reserved.