Interface JobMasterGateway
- All Superinterfaces:
BlocklistListener,CheckpointCoordinatorGateway,org.apache.flink.runtime.rpc.FencedRpcGateway<JobMasterId>,JobMasterOperatorEventGateway,KvStateLocationOracle,KvStateRegistryGateway,org.apache.flink.runtime.rpc.RpcGateway
- All Known Implementing Classes:
JobMaster
public interface JobMasterGateway
extends CheckpointCoordinatorGateway, org.apache.flink.runtime.rpc.FencedRpcGateway<JobMasterId>, KvStateLocationOracle, KvStateRegistryGateway, JobMasterOperatorEventGateway, BlocklistListener
JobMaster rpc gateway interface.-
Method Summary
Modifier and TypeMethodDescriptionCancels the currently executed job.deliverCoordinationRequestToCoordinator(OperatorID operatorId, org.apache.flink.util.SerializedValue<CoordinationRequest> serializedRequest, Duration timeout) Deliver a coordination request to a specified coordinator and return the response.voiddisconnectResourceManager(ResourceManagerId resourceManagerId, Exception cause) Disconnects the resource manager from the job manager because of the given cause.disconnectTaskManager(ResourceID resourceID, Exception cause) Disconnects the givenTaskExecutorfrom theJobMaster.voidfailSlot(ResourceID taskManagerId, AllocationID allocationId, Exception cause) Fails the slot with the given allocation id and cause.default CompletableFuture<Collection<PartitionWithMetrics>>getPartitionWithMetrics(Duration timeout, Set<ResultPartitionID> expectedPartitions) Get specified partitions and their metrics (identified byexpectedPartitions), the metrics include sizes of sub-partitions in a result partition.heartbeatFromResourceManager(ResourceID resourceID) Sends heartbeat request from the resource manager.heartbeatFromTaskManager(ResourceID resourceID, TaskExecutorToJobManagerHeartbeatPayload payload) Sends the heartbeat to job manager from task manager.voidnotifyEndOfData(ExecutionAttemptID executionAttempt) Notifies that the task has reached the end of data.voidnotifyNotEnoughResourcesAvailable(Collection<ResourceRequirement> acquiredResources) Notifies that not enough resources are available to fulfill the resource requirements of a job.offerSlots(ResourceID taskManagerId, Collection<SlotOffer> slots, Duration timeout) Offers the given slots to the job manager.registerTaskManager(org.apache.flink.api.common.JobID jobId, TaskManagerRegistrationInformation taskManagerRegistrationInformation, Duration timeout) Registers the task manager at the job manager.requestCheckpointStats(Duration timeout) Requests theCheckpointStatsSnapshotof the job.requestJob(Duration timeout) Requests theExecutionGraphInfoof the executed job.Read currentjob resource requirements.CompletableFuture<org.apache.flink.api.common.JobStatus>requestJobStatus(Duration timeout) Requests the current job status.requestNextInputSplit(JobVertexID vertexID, ExecutionAttemptID executionAttempt) Requests the next input split for theExecutionJobVertex.requestPartitionState(IntermediateDataSetID intermediateResultId, ResultPartitionID partitionId) Requests the current state of the partition.default voidNotify jobMaster to fetch and retain partitions on task managers.stopTrackingAndReleasePartitions(Collection<ResultPartitionID> partitionIds) Notifies theJobMasterPartitionTrackerto stop tracking the target result partitions and release the locally occupied resources onTaskExecutors if any.stopWithSavepoint(String targetDirectory, org.apache.flink.core.execution.SavepointFormatType formatType, boolean terminate, Duration timeout) Stops the job with a savepoint.default CompletableFuture<String>triggerCheckpoint(Duration timeout) Triggers taking a checkpoint of the executed job.triggerCheckpoint(org.apache.flink.core.execution.CheckpointType checkpointType, Duration timeout) Triggers taking a checkpoint of the executed job.triggerSavepoint(String targetDirectory, boolean cancelJob, org.apache.flink.core.execution.SavepointFormatType formatType, Duration timeout) Triggers taking a savepoint of the executed job.updateGlobalAggregate(String aggregateName, Object aggregand, byte[] serializedAggregationFunction) Update the aggregate and return the new value.updateJobResourceRequirements(JobResourceRequirements jobResourceRequirements) Updatejob resource requirements.updateTaskExecutionState(TaskExecutionState taskExecutionState) Updates the task execution state for a given task.Methods inherited from interface org.apache.flink.runtime.blocklist.BlocklistListener
notifyNewBlockedNodesMethods inherited from interface org.apache.flink.runtime.checkpoint.CheckpointCoordinatorGateway
acknowledgeCheckpoint, declineCheckpoint, reportCheckpointMetrics, reportInitializationMetricsMethods inherited from interface org.apache.flink.runtime.rpc.FencedRpcGateway
getFencingTokenMethods inherited from interface org.apache.flink.runtime.jobmaster.JobMasterOperatorEventGateway
sendOperatorEventToCoordinator, sendRequestToCoordinatorMethods inherited from interface org.apache.flink.runtime.jobmaster.KvStateLocationOracle
requestKvStateLocationMethods inherited from interface org.apache.flink.runtime.jobmaster.KvStateRegistryGateway
notifyKvStateRegistered, notifyKvStateUnregisteredMethods inherited from interface org.apache.flink.runtime.rpc.RpcGateway
getAddress, getHostname
-
Method Details
-
cancel
Cancels the currently executed job.- Parameters:
timeout- of this operation- Returns:
- Future acknowledge of the operation
-
updateTaskExecutionState
Updates the task execution state for a given task.- Parameters:
taskExecutionState- New task execution state for a given task- Returns:
- Future flag of the task execution state update result
-
requestNextInputSplit
CompletableFuture<SerializedInputSplit> requestNextInputSplit(JobVertexID vertexID, ExecutionAttemptID executionAttempt) Requests the next input split for theExecutionJobVertex. The next input split is sent back to the sender as aSerializedInputSplitmessage.- Parameters:
vertexID- The job vertex idexecutionAttempt- The execution attempt id- Returns:
- The future of the input split. If there is no further input split, will return an empty object.
-
requestPartitionState
CompletableFuture<ExecutionState> requestPartitionState(IntermediateDataSetID intermediateResultId, ResultPartitionID partitionId) Requests the current state of the partition. The state of a partition is currently bound to the state of the producing execution.- Parameters:
intermediateResultId- The execution attempt ID of the task requesting the partition state.partitionId- The partition ID of the partition to request the state of.- Returns:
- The future of the partition state
-
disconnectTaskManager
Disconnects the givenTaskExecutorfrom theJobMaster.- Parameters:
resourceID- identifying the TaskManager to disconnectcause- for the disconnection of the TaskManager- Returns:
- Future acknowledge once the JobMaster has been disconnected from the TaskManager
-
disconnectResourceManager
Disconnects the resource manager from the job manager because of the given cause.- Parameters:
resourceManagerId- identifying the resource manager leader idcause- of the disconnect
-
offerSlots
CompletableFuture<Collection<SlotOffer>> offerSlots(ResourceID taskManagerId, Collection<SlotOffer> slots, Duration timeout) Offers the given slots to the job manager. The response contains the set of accepted slots.- Parameters:
taskManagerId- identifying the task managerslots- to offer to the job managertimeout- for the rpc call- Returns:
- Future set of accepted slots.
-
failSlot
Fails the slot with the given allocation id and cause.- Parameters:
taskManagerId- identifying the task managerallocationId- identifying the slot to failcause- of the failing
-
registerTaskManager
CompletableFuture<RegistrationResponse> registerTaskManager(org.apache.flink.api.common.JobID jobId, TaskManagerRegistrationInformation taskManagerRegistrationInformation, Duration timeout) Registers the task manager at the job manager.- Parameters:
jobId- jobId specifying the job for which the JobMaster should be responsibletaskManagerRegistrationInformation- the information for registering a task manager at the job managertimeout- for the rpc call- Returns:
- Future registration response indicating whether the registration was successful or not
-
heartbeatFromTaskManager
CompletableFuture<Void> heartbeatFromTaskManager(ResourceID resourceID, TaskExecutorToJobManagerHeartbeatPayload payload) Sends the heartbeat to job manager from task manager.- Parameters:
resourceID- unique id of the task managerpayload- report payload- Returns:
- future which is completed exceptionally if the operation fails
-
heartbeatFromResourceManager
Sends heartbeat request from the resource manager.- Parameters:
resourceID- unique id of the resource manager- Returns:
- future which is completed exceptionally if the operation fails
-
requestJobStatus
Requests the current job status.- Parameters:
timeout- for the rpc call- Returns:
- Future containing the current job status
-
requestJob
Requests theExecutionGraphInfoof the executed job.- Parameters:
timeout- for the rpc call- Returns:
- Future which is completed with the
ExecutionGraphInfoof the executed job
-
requestCheckpointStats
Requests theCheckpointStatsSnapshotof the job.- Parameters:
timeout- for the rpc call- Returns:
- Future which is completed with the
CheckpointStatsSnapshotof the job
-
triggerSavepoint
CompletableFuture<String> triggerSavepoint(@Nullable String targetDirectory, boolean cancelJob, org.apache.flink.core.execution.SavepointFormatType formatType, Duration timeout) Triggers taking a savepoint of the executed job.- Parameters:
targetDirectory- to which to write the savepoint data or null if the default savepoint directory should be usedformatType- binary format for the savepointtimeout- for the rpc call- Returns:
- Future which is completed with the savepoint path once completed
-
triggerCheckpoint
CompletableFuture<CompletedCheckpoint> triggerCheckpoint(org.apache.flink.core.execution.CheckpointType checkpointType, Duration timeout) Triggers taking a checkpoint of the executed job.- Parameters:
checkpointType- to determine how checkpoint should be takentimeout- for the rpc call- Returns:
- Future which is completed with the CompletedCheckpoint once completed
-
triggerCheckpoint
Triggers taking a checkpoint of the executed job.- Parameters:
timeout- for the rpc call- Returns:
- Future which is completed with the checkpoint path once completed
-
stopWithSavepoint
CompletableFuture<String> stopWithSavepoint(@Nullable String targetDirectory, org.apache.flink.core.execution.SavepointFormatType formatType, boolean terminate, Duration timeout) Stops the job with a savepoint.- Parameters:
targetDirectory- to which to write the savepoint data or null if the default savepoint directory should be usedterminate- flag indicating if the job should terminate or just suspendtimeout- for the rpc call- Returns:
- Future which is completed with the savepoint path once completed
-
notifyNotEnoughResourcesAvailable
Notifies that not enough resources are available to fulfill the resource requirements of a job.- Parameters:
acquiredResources- the resources that have been acquired for the job
-
updateGlobalAggregate
CompletableFuture<Object> updateGlobalAggregate(String aggregateName, Object aggregand, byte[] serializedAggregationFunction) Update the aggregate and return the new value.- Parameters:
aggregateName- The name of the aggregate to updateaggregand- The value to add to the aggregateserializedAggregationFunction- The function to apply to the current aggregate and aggregand to obtain the new aggregate value, this should be of typeAggregateFunction- Returns:
- The updated aggregate
-
deliverCoordinationRequestToCoordinator
CompletableFuture<CoordinationResponse> deliverCoordinationRequestToCoordinator(OperatorID operatorId, org.apache.flink.util.SerializedValue<CoordinationRequest> serializedRequest, Duration timeout) Deliver a coordination request to a specified coordinator and return the response.- Parameters:
operatorId- identifying the coordinator to receive the requestserializedRequest- serialized request to deliver- Returns:
- A future containing the response. The response will fail with a
FlinkExceptionif the task is not running, or no operator/coordinator exists for the given ID, or the coordinator cannot handle client events.
-
stopTrackingAndReleasePartitions
Notifies theJobMasterPartitionTrackerto stop tracking the target result partitions and release the locally occupied resources onTaskExecutors if any. -
getPartitionWithMetrics
default CompletableFuture<Collection<PartitionWithMetrics>> getPartitionWithMetrics(Duration timeout, Set<ResultPartitionID> expectedPartitions) Get specified partitions and their metrics (identified byexpectedPartitions), the metrics include sizes of sub-partitions in a result partition.- Parameters:
timeout- The timeout used for retrieve the specified partitions.expectedPartitions- The set of identifiers for the result partitions whose metrics are to be fetched.- Returns:
- A future will contain a collection of the partitions with their metrics that could be retrieved from the expected partitions within the specified timeout period.
-
startFetchAndRetainPartitionWithMetricsOnTaskManager
default void startFetchAndRetainPartitionWithMetricsOnTaskManager()Notify jobMaster to fetch and retain partitions on task managers. It will process for future TaskManager registrations and already registered TaskManagers. -
requestJobResourceRequirements
CompletableFuture<JobResourceRequirements> requestJobResourceRequirements()Read currentjob resource requirements.- Returns:
- Future which that contains current resource requirements.
-
updateJobResourceRequirements
CompletableFuture<Acknowledge> updateJobResourceRequirements(JobResourceRequirements jobResourceRequirements) Updatejob resource requirements.- Parameters:
jobResourceRequirements- new resource requirements- Returns:
- Future which is completed successfully when requirements are updated
-
notifyEndOfData
Notifies that the task has reached the end of data.- Parameters:
executionAttempt- The execution attempt id.
-