public class TaskExecutor extends RpcEndpoint implements TaskExecutorGateway
Task
.RpcEndpoint.MainThreadExecutor
Modifier and Type | Field and Description |
---|---|
static String |
TASK_MANAGER_NAME |
log, rpcServer
Constructor and Description |
---|
TaskExecutor(RpcService rpcService,
TaskManagerConfiguration taskManagerConfiguration,
HighAvailabilityServices haServices,
TaskManagerServices taskExecutorServices,
ExternalResourceInfoProvider externalResourceInfoProvider,
HeartbeatServices heartbeatServices,
TaskManagerMetricGroup taskManagerMetricGroup,
String metricQueryServiceAddress,
BlobCacheService blobCacheService,
FatalErrorHandler fatalErrorHandler,
TaskExecutorPartitionTracker partitionTracker,
BackPressureSampleService backPressureSampleService) |
Modifier and Type | Method and Description |
---|---|
CompletableFuture<Acknowledge> |
abortCheckpoint(ExecutionAttemptID executionAttemptID,
long checkpointId,
long checkpointTimestamp)
Abort a checkpoint for the given task.
|
CompletableFuture<Boolean> |
canBeReleased()
Checks whether the task executor can be released.
|
CompletableFuture<Acknowledge> |
cancelTask(ExecutionAttemptID executionAttemptID,
Time timeout)
Cancel the given task.
|
CompletableFuture<Acknowledge> |
confirmCheckpoint(ExecutionAttemptID executionAttemptID,
long checkpointId,
long checkpointTimestamp)
Confirm a checkpoint for the given task.
|
void |
disconnectJobManager(JobID jobId,
Exception cause)
Disconnects the given JobManager from the TaskManager.
|
void |
disconnectResourceManager(Exception cause)
Disconnects the ResourceManager from the TaskManager.
|
CompletableFuture<Acknowledge> |
freeSlot(AllocationID allocationId,
Throwable cause,
Time timeout)
Frees the slot with the given allocation ID.
|
ResourceID |
getResourceID() |
void |
heartbeatFromJobManager(ResourceID resourceID,
AllocatedSlotReport allocatedSlotReport)
Heartbeat request from the job manager.
|
void |
heartbeatFromResourceManager(ResourceID resourceID)
Heartbeat request from the resource manager.
|
void |
onStart()
User overridable callback which is called from
RpcEndpoint.internalCallOnStart() . |
CompletableFuture<Void> |
onStop()
Called to shut down the TaskManager.
|
CompletableFuture<Acknowledge> |
releaseClusterPartitions(Collection<IntermediateDataSetID> dataSetsToRelease,
Time timeout)
Releases all cluster partitions belong to any of the given data sets.
|
void |
releaseOrPromotePartitions(JobID jobId,
Set<ResultPartitionID> partitionToRelease,
Set<ResultPartitionID> partitionsToPromote)
Batch release/promote intermediate result partitions.
|
CompletableFuture<TransientBlobKey> |
requestFileUploadByName(String fileName,
Time timeout)
Requests the file upload of the specified name to the cluster's
BlobServer . |
CompletableFuture<TransientBlobKey> |
requestFileUploadByType(FileType fileType,
Time timeout)
Requests the file upload of the specified type to the cluster's
BlobServer . |
CompletableFuture<Collection<LogInfo>> |
requestLogList(Time timeout)
Requests for the historical log file names on the TaskManager.
|
CompletableFuture<SerializableOptional<String>> |
requestMetricQueryServiceAddress(Time timeout)
Returns the gateway of Metric Query Service on the TaskManager.
|
CompletableFuture<Acknowledge> |
requestSlot(SlotID slotId,
JobID jobId,
AllocationID allocationId,
ResourceProfile resourceProfile,
String targetAddress,
ResourceManagerId resourceManagerId,
Time timeout)
Requests a slot from the TaskManager.
|
CompletableFuture<TaskBackPressureResponse> |
requestTaskBackPressure(ExecutionAttemptID executionAttemptId,
int requestId,
Time timeout) |
CompletableFuture<ThreadDumpInfo> |
requestThreadDump(Time timeout)
Requests the thread dump from this TaskManager.
|
CompletableFuture<Acknowledge> |
sendOperatorEventToTask(ExecutionAttemptID executionAttemptID,
OperatorID operatorId,
SerializedValue<OperatorEvent> evt)
Sends an operator event to an operator in a task executed by the Task Manager (Task
Executor).
|
CompletableFuture<Acknowledge> |
submitTask(TaskDeploymentDescriptor tdd,
JobMasterId jobMasterId,
Time timeout)
Submit a
Task to the TaskExecutor . |
CompletableFuture<Acknowledge> |
triggerCheckpoint(ExecutionAttemptID executionAttemptID,
long checkpointId,
long checkpointTimestamp,
CheckpointOptions checkpointOptions)
Trigger the checkpoint for the given task.
|
CompletableFuture<Acknowledge> |
updatePartitions(ExecutionAttemptID executionAttemptID,
Iterable<PartitionInfo> partitionInfos,
Time timeout)
Update the task where the given partitions can be found.
|
callAsync, closeAsync, getAddress, getEndpointId, getHostname, getMainThreadExecutor, getRpcService, getSelfGateway, getTerminationFuture, internalCallOnStart, internalCallOnStop, isRunning, runAsync, scheduleRunAsync, scheduleRunAsync, start, stop, validateRunsInMainThread
clone, equals, finalize, getClass, hashCode, notify, notifyAll, toString, wait, wait, wait
getAddress, getHostname
close
public static final String TASK_MANAGER_NAME
public TaskExecutor(RpcService rpcService, TaskManagerConfiguration taskManagerConfiguration, HighAvailabilityServices haServices, TaskManagerServices taskExecutorServices, ExternalResourceInfoProvider externalResourceInfoProvider, HeartbeatServices heartbeatServices, TaskManagerMetricGroup taskManagerMetricGroup, @Nullable String metricQueryServiceAddress, BlobCacheService blobCacheService, FatalErrorHandler fatalErrorHandler, TaskExecutorPartitionTracker partitionTracker, BackPressureSampleService backPressureSampleService)
public CompletableFuture<Boolean> canBeReleased()
TaskExecutorGateway
canBeReleased
in interface TaskExecutorGateway
public CompletableFuture<Collection<LogInfo>> requestLogList(Time timeout)
TaskExecutorGateway
requestLogList
in interface TaskExecutorGateway
public void onStart() throws Exception
RpcEndpoint
RpcEndpoint.internalCallOnStart()
.
This method is called when the RpcEndpoint is being started. The method is guaranteed to be executed in the main thread context and can be used to start the rpc endpoint in the context of the rpc endpoint's main thread.
IMPORTANT: This method should never be called directly by the user.
onStart
in class RpcEndpoint
Exception
- indicating that the rpc endpoint could not be started. If an exception
occurs, then the rpc endpoint will automatically terminate.public CompletableFuture<Void> onStop()
onStop
in class RpcEndpoint
public CompletableFuture<TaskBackPressureResponse> requestTaskBackPressure(ExecutionAttemptID executionAttemptId, int requestId, Time timeout)
requestTaskBackPressure
in interface TaskExecutorGateway
public CompletableFuture<Acknowledge> submitTask(TaskDeploymentDescriptor tdd, JobMasterId jobMasterId, Time timeout)
TaskExecutorGateway
Task
to the TaskExecutor
.submitTask
in interface TaskExecutorGateway
tdd
- describing the task to submitjobMasterId
- identifying the submitting JobMastertimeout
- of the submit operationpublic CompletableFuture<Acknowledge> cancelTask(ExecutionAttemptID executionAttemptID, Time timeout)
TaskExecutorGateway
cancelTask
in interface TaskExecutorGateway
executionAttemptID
- identifying the tasktimeout
- for the cancel operationpublic CompletableFuture<Acknowledge> updatePartitions(ExecutionAttemptID executionAttemptID, Iterable<PartitionInfo> partitionInfos, Time timeout)
TaskExecutorGateway
updatePartitions
in interface TaskExecutorGateway
executionAttemptID
- identifying the taskpartitionInfos
- telling where the partition can be retrieved fromtimeout
- for the update partitions operationpublic void releaseOrPromotePartitions(JobID jobId, Set<ResultPartitionID> partitionToRelease, Set<ResultPartitionID> partitionsToPromote)
TaskExecutorGateway
releaseOrPromotePartitions
in interface TaskExecutorGateway
jobId
- id of the job that the partitions belong topartitionToRelease
- partition ids to releasepartitionsToPromote
- partitions ids to promotepublic CompletableFuture<Acknowledge> releaseClusterPartitions(Collection<IntermediateDataSetID> dataSetsToRelease, Time timeout)
TaskExecutorGateway
releaseClusterPartitions
in interface TaskExecutorGateway
dataSetsToRelease
- data sets for which all cluster partitions should be releasedtimeout
- for the partitions release operationpublic void heartbeatFromJobManager(ResourceID resourceID, AllocatedSlotReport allocatedSlotReport)
TaskExecutorGateway
heartbeatFromJobManager
in interface TaskExecutorGateway
resourceID
- unique id of the job managerpublic void heartbeatFromResourceManager(ResourceID resourceID)
TaskExecutorGateway
heartbeatFromResourceManager
in interface TaskExecutorGateway
resourceID
- unique id of the resource managerpublic CompletableFuture<Acknowledge> triggerCheckpoint(ExecutionAttemptID executionAttemptID, long checkpointId, long checkpointTimestamp, CheckpointOptions checkpointOptions)
TaskExecutorGateway
triggerCheckpoint
in interface TaskExecutorGateway
executionAttemptID
- identifying the taskcheckpointId
- unique id for the checkpointcheckpointTimestamp
- is the timestamp when the checkpoint has been initiatedcheckpointOptions
- for performing the checkpointpublic CompletableFuture<Acknowledge> confirmCheckpoint(ExecutionAttemptID executionAttemptID, long checkpointId, long checkpointTimestamp)
TaskExecutorGateway
confirmCheckpoint
in interface TaskExecutorGateway
executionAttemptID
- identifying the taskcheckpointId
- unique id for the checkpointcheckpointTimestamp
- is the timestamp when the checkpoint has been initiatedpublic CompletableFuture<Acknowledge> abortCheckpoint(ExecutionAttemptID executionAttemptID, long checkpointId, long checkpointTimestamp)
TaskExecutorGateway
abortCheckpoint
in interface TaskExecutorGateway
executionAttemptID
- identifying the taskcheckpointId
- unique id for the checkpointcheckpointTimestamp
- is the timestamp when the checkpoint has been initiatedpublic CompletableFuture<Acknowledge> requestSlot(SlotID slotId, JobID jobId, AllocationID allocationId, ResourceProfile resourceProfile, String targetAddress, ResourceManagerId resourceManagerId, Time timeout)
TaskExecutorGateway
requestSlot
in interface TaskExecutorGateway
slotId
- slot id for the requestjobId
- for which to request a slotallocationId
- id for the requestresourceProfile
- of requested slot, used only for dynamic slot allocation and will be
ignored otherwisetargetAddress
- to which to offer the requested slotsresourceManagerId
- current leader id of the ResourceManagertimeout
- for the operationpublic CompletableFuture<Acknowledge> freeSlot(AllocationID allocationId, Throwable cause, Time timeout)
TaskExecutorGateway
freeSlot
in interface TaskExecutorGateway
allocationId
- identifying the slot to freecause
- of the freeing operationtimeout
- for the operationpublic CompletableFuture<TransientBlobKey> requestFileUploadByType(FileType fileType, Time timeout)
TaskExecutorGateway
BlobServer
.requestFileUploadByType
in interface TaskExecutorGateway
fileType
- to uploadtimeout
- for the asynchronous operationTransientBlobKey
of the uploaded file.public CompletableFuture<TransientBlobKey> requestFileUploadByName(String fileName, Time timeout)
TaskExecutorGateway
BlobServer
.requestFileUploadByName
in interface TaskExecutorGateway
fileName
- to uploadtimeout
- for the asynchronous operationTransientBlobKey
of the uploaded file.public CompletableFuture<SerializableOptional<String>> requestMetricQueryServiceAddress(Time timeout)
TaskExecutorGateway
requestMetricQueryServiceAddress
in interface TaskExecutorGateway
public void disconnectJobManager(JobID jobId, Exception cause)
TaskExecutorGateway
disconnectJobManager
in interface TaskExecutorGateway
jobId
- JobID for which the JobManager was the leadercause
- for the disconnection from the JobManagerpublic void disconnectResourceManager(Exception cause)
TaskExecutorGateway
disconnectResourceManager
in interface TaskExecutorGateway
cause
- for the disconnection from the ResourceManagerpublic CompletableFuture<Acknowledge> sendOperatorEventToTask(ExecutionAttemptID executionAttemptID, OperatorID operatorId, SerializedValue<OperatorEvent> evt)
TaskExecutorOperatorEventGateway
The reception is acknowledged (future is completed) when the event has been dispatched to
the AbstractInvokable.dispatchOperatorEvent(OperatorID,
SerializedValue)
method. It is not guaranteed that the event is processed successfully
within the implementation. These cases are up to the task and event sender to handle (for
example with an explicit response message upon success, or by triggering failure/recovery
upon exception).
sendOperatorEventToTask
in interface TaskExecutorGateway
sendOperatorEventToTask
in interface TaskExecutorOperatorEventGateway
public CompletableFuture<ThreadDumpInfo> requestThreadDump(Time timeout)
TaskExecutorGateway
requestThreadDump
in interface TaskExecutorGateway
timeout
- timeout for the asynchronous operationThreadDumpInfo
for this TaskManager.public ResourceID getResourceID()
Copyright © 2014–2021 The Apache Software Foundation. All rights reserved.