@InterfaceAudience.Public @InterfaceStability.Stable public interface ApplicationClientProtocol extends ApplicationBaseProtocol
The protocol between clients and the ResourceManager
to submit/abort jobs and to get information on applications, cluster metrics,
nodes, queues and ACLs.
Modifier and Type | Method and Description |
---|---|
ReservationDeleteResponse |
deleteReservation(ReservationDeleteRequest request)
The interface used by clients to remove an existing Reservation.
|
FailApplicationAttemptResponse |
failApplicationAttempt(FailApplicationAttemptRequest request)
The interface used by clients to request the
ResourceManager to fail an application attempt. |
KillApplicationResponse |
forceKillApplication(KillApplicationRequest request)
The interface used by clients to request the
ResourceManager to abort submitted application. |
GetAttributesToNodesResponse |
getAttributesToNodes(GetAttributesToNodesRequest request)
The interface used by client to get attributes to nodes mappings
available in ResourceManager.
|
GetClusterMetricsResponse |
getClusterMetrics(GetClusterMetricsRequest request)
The interface used by clients to get metrics about the cluster from
the
ResourceManager . |
GetClusterNodeAttributesResponse |
getClusterNodeAttributes(GetClusterNodeAttributesRequest request)
The interface used by client to get node attributes available in
ResourceManager.
|
GetClusterNodeLabelsResponse |
getClusterNodeLabels(GetClusterNodeLabelsRequest request)
The interface used by client to get node labels in the cluster
|
GetClusterNodesResponse |
getClusterNodes(GetClusterNodesRequest request)
The interface used by clients to get a report of all nodes
in the cluster from the
ResourceManager . |
GetLabelsToNodesResponse |
getLabelsToNodes(GetLabelsToNodesRequest request)
The interface used by client to get labels to nodes mappings
in existing cluster
|
GetNewApplicationResponse |
getNewApplication(GetNewApplicationRequest request)
The interface used by clients to obtain a new
ApplicationId for
submitting new applications. |
GetNewReservationResponse |
getNewReservation(GetNewReservationRequest request)
The interface used by clients to obtain a new
ReservationId for
submitting new reservations. |
GetNodesToAttributesResponse |
getNodesToAttributes(GetNodesToAttributesRequest request)
The interface used by client to get node to attributes mappings.
|
GetNodesToLabelsResponse |
getNodeToLabels(GetNodesToLabelsRequest request)
The interface used by client to get node to labels mappings in existing cluster
|
GetQueueInfoResponse |
getQueueInfo(GetQueueInfoRequest request)
The interface used by clients to get information about queues
from the
ResourceManager . |
GetQueueUserAclsInfoResponse |
getQueueUserAcls(GetQueueUserAclsInfoRequest request)
The interface used by clients to get information about queue
acls for current user from the
ResourceManager . |
GetResourceProfileResponse |
getResourceProfile(GetResourceProfileRequest request)
The interface to get the details for a specific resource profile.
|
GetAllResourceProfilesResponse |
getResourceProfiles(GetAllResourceProfilesRequest request)
The interface used by clients to get all the resource profiles that are
available on the ResourceManager.
|
GetAllResourceTypeInfoResponse |
getResourceTypeInfo(GetAllResourceTypeInfoRequest request)
The interface to get the details for a specific resource profile.
|
ReservationListResponse |
listReservations(ReservationListRequest request)
The interface used by clients to get the list of reservations in a plan.
|
MoveApplicationAcrossQueuesResponse |
moveApplicationAcrossQueues(MoveApplicationAcrossQueuesRequest request)
Move an application to a new queue.
|
SignalContainerResponse |
signalToContainer(SignalContainerRequest request)
The interface used by clients to request the
ResourceManager to signal a container. |
SubmitApplicationResponse |
submitApplication(SubmitApplicationRequest request)
The interface used by clients to submit a new application to the
ResourceManager. |
ReservationSubmissionResponse |
submitReservation(ReservationSubmissionRequest request)
The interface used by clients to submit a new reservation to the
ResourceManager . |
UpdateApplicationPriorityResponse |
updateApplicationPriority(UpdateApplicationPriorityRequest request)
The interface used by client to set priority of an application.
|
UpdateApplicationTimeoutsResponse |
updateApplicationTimeouts(UpdateApplicationTimeoutsRequest request)
The interface used by client to set ApplicationTimeouts of an application.
|
ReservationUpdateResponse |
updateReservation(ReservationUpdateRequest request)
The interface used by clients to update an existing Reservation.
|
cancelDelegationToken, getApplicationAttemptReport, getApplicationAttempts, getApplicationReport, getApplications, getContainerReport, getContainers, getDelegationToken, renewDelegationToken
@InterfaceAudience.Public @InterfaceStability.Stable GetNewApplicationResponse getNewApplication(GetNewApplicationRequest request) throws YarnException, IOException
The interface used by clients to obtain a new ApplicationId
for
submitting new applications.
The ResourceManager
responds with a new, monotonically
increasing, ApplicationId
which is used by the client to submit
a new application.
The ResourceManager
also responds with details such
as maximum resource capabilities in the cluster as specified in
GetNewApplicationResponse
.
request
- request to get a new ApplicationId
ApplicationId
to be used
to submit an applicationYarnException
IOException
submitApplication(SubmitApplicationRequest)
@InterfaceAudience.Public @InterfaceStability.Stable SubmitApplicationResponse submitApplication(SubmitApplicationRequest request) throws YarnException, IOException
The interface used by clients to submit a new application to the
ResourceManager.
The client is required to provide details such as queue,
Resource
required to run the ApplicationMaster
,
the equivalent of ContainerLaunchContext
for launching
the ApplicationMaster
etc. via the
SubmitApplicationRequest
.
Currently the ResourceManager
sends an immediate (empty)
SubmitApplicationResponse
on accepting the submission and throws
an exception if it rejects the submission. However, this call needs to be
followed by ApplicationBaseProtocol.getApplicationReport(GetApplicationReportRequest)
to make sure that the application gets properly submitted - obtaining a
SubmitApplicationResponse
from ResourceManager doesn't guarantee
that RM 'remembers' this application beyond failover or restart. If RM
failover or RM restart happens before ResourceManager saves the
application's state successfully, the subsequent
ApplicationBaseProtocol.getApplicationReport(GetApplicationReportRequest)
will throw
a ApplicationNotFoundException
. The Clients need to re-submit
the application with the same ApplicationSubmissionContext
when
it encounters the ApplicationNotFoundException
on the
ApplicationBaseProtocol.getApplicationReport(GetApplicationReportRequest)
call.
During the submission process, it checks whether the application already exists. If the application exists, it will simply return SubmitApplicationResponse
In secure mode,the ResourceManager
verifies access to
queues etc. before accepting the application submission.
request
- request to submit a new applicationYarnException
IOException
getNewApplication(GetNewApplicationRequest)
@InterfaceAudience.Public @InterfaceStability.Unstable FailApplicationAttemptResponse failApplicationAttempt(FailApplicationAttemptRequest request) throws YarnException, IOException
The interface used by clients to request the
ResourceManager
to fail an application attempt.
The client, via FailApplicationAttemptRequest
provides the
ApplicationAttemptId
of the attempt to be failed.
In secure mode,the ResourceManager
verifies access to the
application, queue etc. before failing the attempt.
Currently, the ResourceManager
returns an empty response
on success and throws an exception on rejecting the request.
request
- request to fail an attemptResourceManager
returns an empty response
on success and throws an exception on rejecting the requestYarnException
IOException
getQueueUserAcls(GetQueueUserAclsInfoRequest)
@InterfaceAudience.Public @InterfaceStability.Stable KillApplicationResponse forceKillApplication(KillApplicationRequest request) throws YarnException, IOException
The interface used by clients to request the
ResourceManager
to abort submitted application.
The client, via KillApplicationRequest
provides the
ApplicationId
of the application to be aborted.
In secure mode,the ResourceManager
verifies access to the
application, queue etc. before terminating the application.
Currently, the ResourceManager
returns an empty response
on success and throws an exception on rejecting the request.
request
- request to abort a submitted applicationResourceManager
returns an empty response
on success and throws an exception on rejecting the requestYarnException
IOException
getQueueUserAcls(GetQueueUserAclsInfoRequest)
@InterfaceAudience.Public @InterfaceStability.Stable GetClusterMetricsResponse getClusterMetrics(GetClusterMetricsRequest request) throws YarnException, IOException
The interface used by clients to get metrics about the cluster from
the ResourceManager
.
The ResourceManager
responds with a
GetClusterMetricsResponse
which includes the
YarnClusterMetrics
with details such as number of current
nodes in the cluster.
request
- request for cluster metricsYarnException
IOException
@InterfaceAudience.Public @InterfaceStability.Stable GetClusterNodesResponse getClusterNodes(GetClusterNodesRequest request) throws YarnException, IOException
The interface used by clients to get a report of all nodes
in the cluster from the ResourceManager
.
The ResourceManager
responds with a
GetClusterNodesResponse
which includes the
NodeReport
for all the nodes in the cluster.
request
- request for report on all nodesYarnException
IOException
@InterfaceAudience.Public @InterfaceStability.Stable GetQueueInfoResponse getQueueInfo(GetQueueInfoRequest request) throws YarnException, IOException
The interface used by clients to get information about queues
from the ResourceManager
.
The client, via GetQueueInfoRequest
, can ask for details such
as used/total resources, child queues, running applications etc.
In secure mode,the ResourceManager
verifies access before
providing the information.
request
- request to get queue informationYarnException
IOException
@InterfaceAudience.Public @InterfaceStability.Stable GetQueueUserAclsInfoResponse getQueueUserAcls(GetQueueUserAclsInfoRequest request) throws YarnException, IOException
The interface used by clients to get information about queue
acls for current user from the ResourceManager
.
The ResourceManager
responds with queue acls for all
existing queues.
request
- request to get queue acls for current userYarnException
IOException
@InterfaceAudience.Public @InterfaceStability.Unstable MoveApplicationAcrossQueuesResponse moveApplicationAcrossQueues(MoveApplicationAcrossQueuesRequest request) throws YarnException, IOException
request
- the application ID and the target queueYarnException
IOException
@InterfaceAudience.Public @InterfaceStability.Unstable GetNewReservationResponse getNewReservation(GetNewReservationRequest request) throws YarnException, IOException
The interface used by clients to obtain a new ReservationId
for
submitting new reservations.
The ResourceManager
responds with a new, unique,
ReservationId
which is used by the client to submit
a new reservation.
request
- to get a new ReservationId
ReservationId
to be used
to submit a new reservationYarnException
- if the reservation system is not enabled.IOException
- on IO failures.submitReservation(ReservationSubmissionRequest)
@InterfaceAudience.Public @InterfaceStability.Unstable ReservationSubmissionResponse submitReservation(ReservationSubmissionRequest request) throws YarnException, IOException
The interface used by clients to submit a new reservation to the
ResourceManager
.
The client packages all details of its request in a
ReservationSubmissionRequest
object. This contains information
about the amount of capacity, temporal constraints, and concurrency needs.
Furthermore, the reservation might be composed of multiple stages, with
ordering dependencies among them.
In order to respond, a new admission control component in the
ResourceManager
performs an analysis of the resources that have
been committed over the period of time the user is requesting, verify that
the user requests can be fulfilled, and that it respect a sharing policy
(e.g., CapacityOverTimePolicy
). Once it has positively determined
that the ReservationSubmissionRequest is satisfiable the
ResourceManager
answers with a
ReservationSubmissionResponse
that include a non-null
ReservationId
. Upon failure to find a valid allocation the response
is an exception with the reason.
On application submission the client can use this ReservationId
to
obtain access to the reserved resources.
The system guarantees that during the time-range specified by the user, the
reservationID will be corresponding to a valid reservation. The amount of
capacity dedicated to such queue can vary overtime, depending of the
allocation that has been determined. But it is guaranteed to satisfy all
the constraint expressed by the user in the
ReservationSubmissionRequest
.
request
- the request to submit a new ReservationReservationId
on accepting the submissionYarnException
- if the request is invalid or reservation cannot be
created successfullyIOException
@InterfaceAudience.Public @InterfaceStability.Unstable ReservationUpdateResponse updateReservation(ReservationUpdateRequest request) throws YarnException, IOException
The interface used by clients to update an existing Reservation. This is referred to as a re-negotiation process, in which a user that has previously submitted a Reservation.
The allocation is attempted by virtually substituting all previous
allocations related to this Reservation with new ones, that satisfy the new
ReservationUpdateRequest
. Upon success the previous allocation is
substituted by the new one, and on failure (i.e., if the system cannot find
a valid allocation for the updated request), the previous allocation
remains valid.
The ReservationId
is not changed, and applications currently
running within this reservation will automatically receive the resources
based on the new allocation.
request
- to update an existing Reservation (the ReservationRequest
should refer to an existing valid ReservationId
)YarnException
- if the request is invalid or reservation cannot be
updated successfullyIOException
@InterfaceAudience.Public @InterfaceStability.Unstable ReservationDeleteResponse deleteReservation(ReservationDeleteRequest request) throws YarnException, IOException
The interface used by clients to remove an existing Reservation. Upon deletion of a reservation applications running with this reservation, are automatically downgraded to normal jobs running without any dedicated reservation.
request
- to remove an existing Reservation (the ReservationRequest
should refer to an existing valid ReservationId
)YarnException
- if the request is invalid or reservation cannot be
deleted successfullyIOException
@InterfaceAudience.Public @InterfaceStability.Unstable ReservationListResponse listReservations(ReservationListRequest request) throws YarnException, IOException
The interface used by clients to get the list of reservations in a plan. The reservationId will be used to search for reservations to list if it is provided. Otherwise, it will select active reservations within the startTime and endTime (inclusive).
request
- to list reservations in a plan. Contains fields to select
String queue, ReservationId reservationId, long startTime,
long endTime, and a bool includeReservationAllocations.
queue: Required. Cannot be null or empty. Refers to the
reservable queue in the scheduler that was selected when
creating a reservation submission
ReservationSubmissionRequest
.
reservationId: Optional. If provided, other fields will
be ignored.
startTime: Optional. If provided, only reservations that
end after the startTime will be selected. This defaults
to 0 if an invalid number is used.
endTime: Optional. If provided, only reservations that
start on or before endTime will be selected. This defaults
to Long.MAX_VALUE if an invalid number is used.
includeReservationAllocations: Optional. Flag that
determines whether the entire reservation allocations are
to be returned. Reservation allocations are subject to
change in the event of re-planning as described by
ReservationDefinition
.YarnException
- if the request is invalidIOException
- on IO failures@InterfaceAudience.Public @InterfaceStability.Unstable GetNodesToLabelsResponse getNodeToLabels(GetNodesToLabelsRequest request) throws YarnException, IOException
The interface used by client to get node to labels mappings in existing cluster
request
- YarnException
IOException
@InterfaceAudience.Public @InterfaceStability.Unstable GetLabelsToNodesResponse getLabelsToNodes(GetLabelsToNodesRequest request) throws YarnException, IOException
The interface used by client to get labels to nodes mappings in existing cluster
request
- YarnException
IOException
@InterfaceAudience.Public @InterfaceStability.Unstable GetClusterNodeLabelsResponse getClusterNodeLabels(GetClusterNodeLabelsRequest request) throws YarnException, IOException
The interface used by client to get node labels in the cluster
request
- to get node labels collection of this clusterYarnException
IOException
@InterfaceAudience.Public @InterfaceStability.Unstable UpdateApplicationPriorityResponse updateApplicationPriority(UpdateApplicationPriorityRequest request) throws YarnException, IOException
The interface used by client to set priority of an application.
request
- to set priority of an applicationYarnException
IOException
@InterfaceAudience.Public @InterfaceStability.Unstable SignalContainerResponse signalToContainer(SignalContainerRequest request) throws YarnException, IOException
The interface used by clients to request the
ResourceManager
to signal a container. For example,
the client can send command OUTPUT_THREAD_DUMP to dump threads of the
container.
The client, via SignalContainerRequest
provides the
id of the container and the signal command.
In secure mode,the ResourceManager
verifies access to the
application before signaling the container.
The user needs to have MODIFY_APP
permission.
Currently, the ResourceManager
returns an empty response
on success and throws an exception on rejecting the request.
request
- request to signal a containerResourceManager
returns an empty response
on success and throws an exception on rejecting the requestYarnException
IOException
@InterfaceAudience.Public @InterfaceStability.Unstable UpdateApplicationTimeoutsResponse updateApplicationTimeouts(UpdateApplicationTimeoutsRequest request) throws YarnException, IOException
The interface used by client to set ApplicationTimeouts of an application. The UpdateApplicationTimeoutsRequest should have timeout value with absolute time with ISO8601 format yyyy-MM-dd'T'HH:mm:ss.SSSZ.
Note: If application timeout value is less than or equal to current time then update application throws YarnException.request
- to set ApplicationTimeouts of an applicationYarnException
- if update request has empty values or application is
in completing states.IOException
- on IO failures@InterfaceAudience.Public @InterfaceStability.Unstable GetAllResourceProfilesResponse getResourceProfiles(GetAllResourceProfilesRequest request) throws YarnException, IOException
The interface used by clients to get all the resource profiles that are available on the ResourceManager.
request
- request to get all the resource profilesYARNFeatureNotEnabledException
- if resource-profile is disabledYarnException
- if any error happens inside YARNIOException
- in case of other errors@InterfaceAudience.Public @InterfaceStability.Unstable GetResourceProfileResponse getResourceProfile(GetResourceProfileRequest request) throws YarnException, IOException
The interface to get the details for a specific resource profile.
request
- request to get the details of a resource profileYARNFeatureNotEnabledException
- if resource-profile is disabledYarnException
- if any error happens inside YARNIOException
- in case of other errors@InterfaceAudience.Public @InterfaceStability.Unstable GetAllResourceTypeInfoResponse getResourceTypeInfo(GetAllResourceTypeInfoRequest request) throws YarnException, IOException
The interface to get the details for a specific resource profile.
request
- request to get the details of a resource profileYarnException
- if any error happens inside YARNIOException
- in case of other errors@InterfaceAudience.Public @InterfaceStability.Unstable GetAttributesToNodesResponse getAttributesToNodes(GetAttributesToNodesRequest request) throws YarnException, IOException
The interface used by client to get attributes to nodes mappings available in ResourceManager.
request
- request to get details of attributes to nodes mapping.YarnException
- if any error happens inside YARNIOException
- incase of other errors@InterfaceAudience.Public @InterfaceStability.Unstable GetClusterNodeAttributesResponse getClusterNodeAttributes(GetClusterNodeAttributesRequest request) throws YarnException, IOException
The interface used by client to get node attributes available in ResourceManager.
request
- request to get node attributes collection of this cluster.YarnException
- if any error happens inside YARN.IOException
- incase of other errors.@InterfaceAudience.Public @InterfaceStability.Unstable GetNodesToAttributesResponse getNodesToAttributes(GetNodesToAttributesRequest request) throws YarnException, IOException
The interface used by client to get node to attributes mappings. in existing cluster.
request
- request to get nodes to attributes mapping.YarnException
- if any error happens inside YARN.IOException
Copyright © 2008–2021 Apache Software Foundation. All rights reserved.