@InterfaceAudience.Public @InterfaceStability.Stable public abstract class YarnClient extends AbstractService
Modifier | Constructor and Description |
---|---|
protected |
YarnClient(String name) |
Modifier and Type | Method and Description |
---|---|
abstract YarnClientApplication |
createApplication()
Obtain a
YarnClientApplication for a new application,
which in turn contains the ApplicationSubmissionContext and
GetNewApplicationResponse
objects. |
static YarnClient |
createYarnClient()
Create a new instance of YarnClient.
|
abstract ReservationDeleteResponse |
deleteReservation(ReservationDeleteRequest request)
The interface used by clients to remove an existing Reservation.
|
abstract List<QueueInfo> |
getAllQueues()
Get information (
QueueInfo ) about all queues, recursively if there
is a hierarchy |
abstract org.apache.hadoop.security.token.Token<AMRMTokenIdentifier> |
getAMRMToken(ApplicationId appId)
Get the AMRM token of the application.
|
abstract ApplicationAttemptReport |
getApplicationAttemptReport(ApplicationAttemptId applicationAttemptId)
Get a report of the given ApplicationAttempt.
|
abstract List<ApplicationAttemptReport> |
getApplicationAttempts(ApplicationId applicationId)
Get a report of all (ApplicationAttempts) of Application in the cluster.
|
abstract ApplicationReport |
getApplicationReport(ApplicationId appId)
Get a report of the given Application.
|
abstract List<ApplicationReport> |
getApplications()
Get a report (ApplicationReport) of all Applications in the cluster.
|
abstract List<ApplicationReport> |
getApplications(EnumSet<YarnApplicationState> applicationStates)
Get a report (ApplicationReport) of Applications matching the given
application states in the cluster.
|
abstract List<ApplicationReport> |
getApplications(Set<String> applicationTypes)
Get a report (ApplicationReport) of Applications
matching the given application types in the cluster.
|
abstract List<ApplicationReport> |
getApplications(Set<String> applicationTypes,
EnumSet<YarnApplicationState> applicationStates)
Get a report (ApplicationReport) of Applications matching the given
application types and application states in the cluster.
|
abstract List<QueueInfo> |
getChildQueueInfos(String parent)
Get information (
QueueInfo ) about all the immediate children queues
of the given queue |
abstract Set<String> |
getClusterNodeLabels()
The interface used by client to get node labels in the cluster
|
abstract ContainerReport |
getContainerReport(ContainerId containerId)
Get a report of the given Container.
|
abstract List<ContainerReport> |
getContainers(ApplicationAttemptId applicationAttemptId)
Get a report of all (Containers) of ApplicationAttempt in the cluster.
|
abstract Map<String,Set<NodeId>> |
getLabelsToNodes()
The interface used by client to get labels to nodes mapping
in existing cluster
|
abstract Map<String,Set<NodeId>> |
getLabelsToNodes(Set<String> labels)
The interface used by client to get labels to nodes mapping
for specified labels in existing cluster
|
abstract List<NodeReport> |
getNodeReports(NodeState... states)
Get a report of nodes (
NodeReport ) in the cluster. |
abstract Map<NodeId,Set<String>> |
getNodeToLabels()
The interface used by client to get node to labels mappings in existing cluster
|
abstract List<QueueUserACLInfo> |
getQueueAclsInfo()
Get information about acls for current user on all the
existing queues.
|
abstract QueueInfo |
getQueueInfo(String queueName)
Get information (
QueueInfo ) about a given queue. |
abstract Token |
getRMDelegationToken(Text renewer)
Get a delegation token so as to be able to talk to YARN using those tokens.
|
abstract List<QueueInfo> |
getRootQueueInfos()
Get information (
QueueInfo ) about top level queues. |
abstract YarnClusterMetrics |
getYarnClusterMetrics()
Get metrics (
YarnClusterMetrics ) about the cluster. |
abstract void |
killApplication(ApplicationId applicationId)
Kill an application identified by given ID.
|
abstract void |
moveApplicationAcrossQueues(ApplicationId appId,
String queue)
Attempts to move the given application to the given queue.
|
abstract ApplicationId |
submitApplication(ApplicationSubmissionContext appContext)
Submit a new application to
YARN. It is a blocking call - it
will not return ApplicationId until the submitted application is
submitted successfully and accepted by the ResourceManager. |
abstract ReservationSubmissionResponse |
submitReservation(ReservationSubmissionRequest request)
The interface used by clients to submit a new reservation to the
ResourceManager . |
abstract ReservationUpdateResponse |
updateReservation(ReservationUpdateRequest request)
The interface used by clients to update an existing Reservation.
|
close, getBlockers, getConfig, getFailureCause, getFailureState, getLifecycleHistory, getName, getServiceState, getStartTime, init, isInState, noteFailure, putBlocker, registerGlobalListener, registerServiceListener, removeBlocker, serviceInit, serviceStart, serviceStop, setConfig, start, stop, toString, unregisterGlobalListener, unregisterServiceListener, waitForServiceToStop
@InterfaceAudience.Private protected YarnClient(String name)
@InterfaceAudience.Public public static YarnClient createYarnClient()
public abstract YarnClientApplication createApplication() throws YarnException, IOException
Obtain a YarnClientApplication
for a new application,
which in turn contains the ApplicationSubmissionContext
and
GetNewApplicationResponse
objects.
YarnClientApplication
built for a new applicationYarnException
IOException
public abstract ApplicationId submitApplication(ApplicationSubmissionContext appContext) throws YarnException, IOException
Submit a new application to YARN.
It is a blocking call - it
will not return ApplicationId
until the submitted application is
submitted successfully and accepted by the ResourceManager.
Users should provide an ApplicationId
as part of the parameter
ApplicationSubmissionContext
when submitting a new application,
otherwise it will throw the ApplicationIdNotProvidedException
.
This internally calls (SubmitApplicationRequest)
, and after that, it internally invokes
(GetApplicationReportRequest)
and waits till it can make sure that the
application gets properly submitted. If RM fails over or RM restart
happens before ResourceManager saves the application's state,
#getApplicationReport(GetApplicationReportRequest)
will throw
the ApplicationNotFoundException
. This API automatically resubmits
the application with the same ApplicationSubmissionContext
when it
catches the ApplicationNotFoundException
appContext
- ApplicationSubmissionContext
containing all the details
needed to submit a new applicationApplicationId
of the accepted applicationYarnException
IOException
createApplication()
public abstract void killApplication(ApplicationId applicationId) throws YarnException, IOException
Kill an application identified by given ID.
applicationId
- ApplicationId
of the application that needs to be killedYarnException
- in case of errors or if YARN rejects the request due to
access-control restrictions.IOException
getQueueAclsInfo()
public abstract ApplicationReport getApplicationReport(ApplicationId appId) throws YarnException, IOException
Get a report of the given Application.
In secure mode, YARN
verifies access to the application, queue
etc. before accepting the request.
If the user does not have VIEW_APP
access then the following
fields in the report will be set to stubbed values:
appId
- ApplicationId
of the application that needs a reportYarnException
IOException
public abstract org.apache.hadoop.security.token.Token<AMRMTokenIdentifier> getAMRMToken(ApplicationId appId) throws YarnException, IOException
The AMRM token is required for AM to RM scheduling operations. For
managed Application Masters Yarn takes care of injecting it. For unmanaged
Applications Masters, the token must be obtained via this method and set
in the UserGroupInformation
of the
current user.
The AMRM token will be returned only if all the following conditions are met:
appId
- ApplicationId
of the application to get the AMRM tokenYarnException
IOException
public abstract List<ApplicationReport> getApplications() throws YarnException, IOException
Get a report (ApplicationReport) of all Applications in the cluster.
If the user does not have VIEW_APP
access for an application
then the corresponding report will be filtered as described in
getApplicationReport(ApplicationId)
.
YarnException
IOException
public abstract List<ApplicationReport> getApplications(Set<String> applicationTypes) throws YarnException, IOException
Get a report (ApplicationReport) of Applications matching the given application types in the cluster.
If the user does not have VIEW_APP
access for an application
then the corresponding report will be filtered as described in
getApplicationReport(ApplicationId)
.
applicationTypes
- YarnException
IOException
public abstract List<ApplicationReport> getApplications(EnumSet<YarnApplicationState> applicationStates) throws YarnException, IOException
Get a report (ApplicationReport) of Applications matching the given application states in the cluster.
If the user does not have VIEW_APP
access for an application
then the corresponding report will be filtered as described in
getApplicationReport(ApplicationId)
.
applicationStates
- YarnException
IOException
public abstract List<ApplicationReport> getApplications(Set<String> applicationTypes, EnumSet<YarnApplicationState> applicationStates) throws YarnException, IOException
Get a report (ApplicationReport) of Applications matching the given application types and application states in the cluster.
If the user does not have VIEW_APP
access for an application
then the corresponding report will be filtered as described in
getApplicationReport(ApplicationId)
.
applicationTypes
- applicationStates
- YarnException
IOException
public abstract YarnClusterMetrics getYarnClusterMetrics() throws YarnException, IOException
Get metrics (YarnClusterMetrics
) about the cluster.
YarnException
IOException
public abstract List<NodeReport> getNodeReports(NodeState... states) throws YarnException, IOException
Get a report of nodes (NodeReport
) in the cluster.
states
- The NodeState
s to filter on. If no filter states are
given, nodes in all states will be returned.YarnException
IOException
public abstract Token getRMDelegationToken(Text renewer) throws YarnException, IOException
Get a delegation token so as to be able to talk to YARN using those tokens.
renewer
- Address of the renewer who can renew these tokens when needed by
securely talking to YARN.Token
) that can be used to
talk to YARNYarnException
IOException
public abstract QueueInfo getQueueInfo(String queueName) throws YarnException, IOException
Get information (QueueInfo
) about a given queue.
queueName
- Name of the queue whose information is neededYarnException
- in case of errors or if YARN rejects the request due to
access-control restrictions.IOException
public abstract List<QueueInfo> getAllQueues() throws YarnException, IOException
Get information (QueueInfo
) about all queues, recursively if there
is a hierarchy
YarnException
IOException
public abstract List<QueueInfo> getRootQueueInfos() throws YarnException, IOException
Get information (QueueInfo
) about top level queues.
YarnException
IOException
public abstract List<QueueInfo> getChildQueueInfos(String parent) throws YarnException, IOException
Get information (QueueInfo
) about all the immediate children queues
of the given queue
parent
- Name of the queue whose child-queues' information is neededYarnException
IOException
public abstract List<QueueUserACLInfo> getQueueAclsInfo() throws YarnException, IOException
Get information about acls for current user on all the existing queues.
QueueUserACLInfo
) for
current userYarnException
IOException
public abstract ApplicationAttemptReport getApplicationAttemptReport(ApplicationAttemptId applicationAttemptId) throws YarnException, IOException
Get a report of the given ApplicationAttempt.
In secure mode, YARN
verifies access to the application, queue
etc. before accepting the request.
applicationAttemptId
- ApplicationAttemptId
of the application attempt that needs
a reportYarnException
ApplicationAttemptNotFoundException
- if application attempt
not foundIOException
public abstract List<ApplicationAttemptReport> getApplicationAttempts(ApplicationId applicationId) throws YarnException, IOException
Get a report of all (ApplicationAttempts) of Application in the cluster.
applicationId
- YarnException
IOException
public abstract ContainerReport getContainerReport(ContainerId containerId) throws YarnException, IOException
Get a report of the given Container.
In secure mode, YARN
verifies access to the application, queue
etc. before accepting the request.
containerId
- ContainerId
of the container that needs a reportYarnException
ContainerNotFoundException
- if container not found.IOException
public abstract List<ContainerReport> getContainers(ApplicationAttemptId applicationAttemptId) throws YarnException, IOException
Get a report of all (Containers) of ApplicationAttempt in the cluster.
applicationAttemptId
- YarnException
IOException
public abstract void moveApplicationAcrossQueues(ApplicationId appId, String queue) throws YarnException, IOException
Attempts to move the given application to the given queue.
appId
- Application to move.queue
- Queue to place it in to.YarnException
IOException
@InterfaceAudience.Public @InterfaceStability.Unstable public abstract 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 gang 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 ReservationRequest is satisfiable the ResourceManager
answers with a ReservationSubmissionResponse
that includes a
ReservationId
. Upon failure to find a valid allocation the response
is an exception with the message detailing the reason of failure.
The semantics guarantees that the ReservationId
returned,
corresponds to a valid reservation existing in the time-range request by
the user. The amount of capacity dedicated to such reservation 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
ReservationDefinition
request
- request to submit a new ReservationReservationId
on accepting the
submissionYarnException
- if the reservation cannot be created successfullyIOException
@InterfaceAudience.Public @InterfaceStability.Unstable public abstract 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
ReservationDefinition
. Upon success the previous allocation is
atomically 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.
request
- to update an existing Reservation (the
ReservationUpdateRequest
should refer to an existing valid
ReservationId
)YarnException
- if the request is invalid or reservation cannot be
updated successfullyIOException
@InterfaceAudience.Public @InterfaceStability.Unstable public abstract ReservationDeleteResponse deleteReservation(ReservationDeleteRequest request) throws YarnException, IOException
The interface used by clients to remove an existing Reservation.
request
- to remove an existing Reservation (the
ReservationDeleteRequest
should refer to an existing valid
ReservationId
)YarnException
- if the request is invalid or reservation cannot be
deleted successfullyIOException
@InterfaceAudience.Public @InterfaceStability.Unstable public abstract Map<NodeId,Set<String>> getNodeToLabels() throws YarnException, IOException
The interface used by client to get node to labels mappings in existing cluster
YarnException
IOException
@InterfaceAudience.Public @InterfaceStability.Unstable public abstract Map<String,Set<NodeId>> getLabelsToNodes() throws YarnException, IOException
The interface used by client to get labels to nodes mapping in existing cluster
YarnException
IOException
@InterfaceAudience.Public @InterfaceStability.Unstable public abstract Map<String,Set<NodeId>> getLabelsToNodes(Set<String> labels) throws YarnException, IOException
The interface used by client to get labels to nodes mapping for specified labels in existing cluster
labels
- labels for which labels to nodes mapping has to be retrievedYarnException
IOException
@InterfaceAudience.Public @InterfaceStability.Unstable public abstract Set<String> getClusterNodeLabels() throws YarnException, IOException
The interface used by client to get node labels in the cluster
YarnException
IOException
Copyright © 2018 Apache Software Foundation. All rights reserved.