@InterfaceAudience.Private @InterfaceStability.Unstable public abstract class AbstractYarnScheduler<T extends SchedulerApplicationAttempt,N extends SchedulerNode> extends org.apache.hadoop.service.AbstractService implements ResourceScheduler
Modifier and Type | Field and Description |
---|---|
protected ActivitiesManager |
activitiesManager |
protected ConcurrentMap<org.apache.hadoop.yarn.api.records.ApplicationId,SchedulerApplication<T>> |
applications |
protected static Allocation |
EMPTY_ALLOCATION |
protected long |
lastNodeUpdateTime |
protected org.apache.hadoop.yarn.api.records.Resource |
minimumAllocation |
protected int |
nmExpireInterval |
protected long |
nmHeartbeatInterval |
protected ClusterNodeTracker<N> |
nodeTracker |
protected ReentrantReadWriteLock.ReadLock |
readLock |
protected RMContext |
rmContext |
protected SchedulerHealth |
schedulerHealth |
protected SchedulingMonitorManager |
schedulingMonitorManager |
protected long |
THREAD_JOIN_TIMEOUT_MS |
protected long |
updateInterval
To enable the update thread, subclasses should set updateInterval to a
positive value during
serviceInit(Configuration) . |
protected ReentrantReadWriteLock.WriteLock |
writeLock |
Constructor and Description |
---|
AbstractYarnScheduler(String name)
Construct the service.
|
Modifier and Type | Method and Description |
---|---|
void |
addQueue(Queue newQueue)
Add to the scheduler a new Queue.
|
void |
asyncContainerRelease(RMContainer container)
To be used to release a container via a Scheduler Event rather than
in the same thread.
|
boolean |
attemptAllocationOnNode(SchedulerApplicationAttempt appAttempt,
org.apache.hadoop.yarn.api.records.SchedulingRequest schedulingRequest,
SchedulerNode schedulerNode)
Default implementation.
|
long |
checkAndGetApplicationLifetime(String queueName,
long lifetime)
Verify whether a submitted application lifetime is valid as per configured
Queue lifetime.
|
org.apache.hadoop.yarn.api.records.Priority |
checkAndGetApplicationPriority(org.apache.hadoop.yarn.api.records.Priority priorityRequestedByApp,
org.apache.hadoop.security.UserGroupInformation user,
String queueName,
org.apache.hadoop.yarn.api.records.ApplicationId applicationId)
Verify whether a submitted application priority is valid as per configured
Queue
|
void |
clearPendingContainerCache() |
void |
completedContainer(RMContainer rmContainer,
org.apache.hadoop.yarn.api.records.ContainerStatus containerStatus,
RMContainerEventType event) |
protected abstract void |
completedContainerInternal(RMContainer rmContainer,
org.apache.hadoop.yarn.api.records.ContainerStatus containerStatus,
RMContainerEventType event) |
protected void |
containerIncreasedOnNode(org.apache.hadoop.yarn.api.records.ContainerId containerId,
SchedulerNode node,
org.apache.hadoop.yarn.api.records.Container increasedContainerReportedByNM) |
protected void |
containerLaunchedOnNode(org.apache.hadoop.yarn.api.records.ContainerId containerId,
SchedulerNode node) |
protected void |
createReleaseCache() |
protected List<SchedContainerChangeRequest> |
createSchedContainerChangeRequests(List<org.apache.hadoop.yarn.api.records.UpdateContainerRequest> changeRequests,
boolean increase) |
ActivitiesManager |
getActivitiesManager() |
T |
getApplicationAttempt(org.apache.hadoop.yarn.api.records.ApplicationAttemptId applicationAttemptId) |
org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport |
getAppResourceUsageReport(org.apache.hadoop.yarn.api.records.ApplicationAttemptId appAttemptId)
Get a resource usage report from a given app attempt ID.
|
List<N> |
getBlacklistedNodes(SchedulerApplicationAttempt app)
Add blacklisted NodeIds to the list that is passed.
|
org.apache.hadoop.yarn.util.Clock |
getClock() |
org.apache.hadoop.yarn.api.records.Resource |
getClusterResource()
Get the whole resource capacity of the cluster.
|
T |
getCurrentAttemptForContainer(org.apache.hadoop.yarn.api.records.ContainerId containerId) |
long |
getLastNodeUpdateTime() |
org.apache.hadoop.yarn.api.records.Priority |
getMaxClusterLevelAppPriority()
Get cluster max priority.
|
org.apache.hadoop.yarn.api.records.Resource |
getMaximumAllocation()
Get a Resource object with for the maximum allocation possible.
|
long |
getMaximumApplicationLifetime(String queueName)
Get maximum lifetime for a queue.
|
org.apache.hadoop.yarn.api.records.Resource |
getMaximumResourceCapability()
Get maximum allocatable
Resource at the cluster level. |
org.apache.hadoop.yarn.api.records.Resource |
getMaximumResourceCapability(String queueName)
Get maximum allocatable
Resource for the queue specified. |
org.apache.hadoop.yarn.api.records.Resource |
getMinimumAllocation() |
org.apache.hadoop.yarn.api.records.Resource |
getMinimumResourceCapability()
Get minimum allocatable
Resource . |
SchedulerNode |
getNode(org.apache.hadoop.yarn.api.records.NodeId nodeId) |
List<org.apache.hadoop.yarn.api.records.NodeId> |
getNodeIds(String resourceName)
Get the
NodeId available in the cluster by resource name. |
SchedulerNodeReport |
getNodeReport(org.apache.hadoop.yarn.api.records.NodeId nodeId)
Get node resource usage report.
|
List<N> |
getNodes(NodeFilter filter) |
ClusterNodeTracker<N> |
getNodeTracker() |
org.apache.hadoop.yarn.api.records.Resource |
getNormalizedResource(org.apache.hadoop.yarn.api.records.Resource requestedResource,
org.apache.hadoop.yarn.api.records.Resource maxResourceCapability)
Normalize a resource request using scheduler level maximum resource or
queue based maximum resource.
|
List<org.apache.hadoop.yarn.api.records.ResourceRequest> |
getPendingResourceRequestsForAttempt(org.apache.hadoop.yarn.api.records.ApplicationAttemptId attemptId) |
List<org.apache.hadoop.yarn.api.records.SchedulingRequest> |
getPendingSchedulingRequestsForAttempt(org.apache.hadoop.yarn.api.records.ApplicationAttemptId attemptId)
Get pending scheduling request for specified application attempt.
|
Set<String> |
getPlanQueues()
Gets the list of names for queues managed by the Reservation System
|
RMContainer |
getRMContainer(org.apache.hadoop.yarn.api.records.ContainerId containerId)
Get the container for the given containerId.
|
SchedulerAppReport |
getSchedulerAppInfo(org.apache.hadoop.yarn.api.records.ApplicationAttemptId appAttemptId)
Get the Scheduler app for a given app attempt Id.
|
Map<org.apache.hadoop.yarn.api.records.ApplicationId,SchedulerApplication<T>> |
getSchedulerApplications() |
SchedulerHealth |
getSchedulerHealth() |
N |
getSchedulerNode(org.apache.hadoop.yarn.api.records.NodeId nodeId)
Get SchedulerNode corresponds to nodeId.
|
SchedulingMonitorManager |
getSchedulingMonitorManager() |
EnumSet<org.apache.hadoop.yarn.proto.YarnServiceProtos.SchedulerResourceTypes> |
getSchedulingResourceTypes()
Return a collection of the resource types that are considered when
scheduling
|
List<org.apache.hadoop.yarn.api.records.Container> |
getTransferredContainers(org.apache.hadoop.yarn.api.records.ApplicationAttemptId currentAttempt)
Get previous attempts' live containers for work-preserving AM restart.
|
protected void |
handleContainerUpdates(SchedulerApplicationAttempt appAttempt,
ContainerUpdates updates) |
protected void |
initMaximumResourceCapability(org.apache.hadoop.yarn.api.records.Resource maximumAllocation) |
void |
killAllAppsInQueue(String queueName)
Terminate all applications in the specified queue.
|
abstract void |
killContainer(RMContainer container)
Kill a RMContainer.
|
void |
moveAllApps(String sourceQueue,
String destQueue)
Completely drain sourceQueue of applications, by moving all of them to
destQueue.
|
String |
moveApplication(org.apache.hadoop.yarn.api.records.ApplicationId appId,
String newQueue)
Moves the given application to the given queue
|
protected void |
nodeUpdate(RMNode nm)
Process a heartbeat update from a node.
|
protected void |
normalizeResourceRequests(List<org.apache.hadoop.yarn.api.records.ResourceRequest> asks)
Normalize a list of resource requests.
|
protected void |
normalizeResourceRequests(List<org.apache.hadoop.yarn.api.records.ResourceRequest> asks,
String queueName)
Normalize a list of resource requests
using queue maximum resource allocations.
|
void |
preValidateMoveApplication(org.apache.hadoop.yarn.api.records.ApplicationId appId,
String newQueue) |
void |
recoverContainersOnNode(List<org.apache.hadoop.yarn.server.api.protocolrecords.NMContainerStatus> containerReports,
RMNode nm) |
protected void |
refreshMaximumAllocation(org.apache.hadoop.yarn.api.records.Resource newMaxAlloc) |
void |
reinitialize(org.apache.hadoop.conf.Configuration conf,
RMContext rmContext)
Re-initialize the
ResourceScheduler . |
protected void |
releaseContainers(List<org.apache.hadoop.yarn.api.records.ContainerId> containers,
SchedulerApplicationAttempt attempt) |
void |
removeQueue(String queueName)
Remove an existing queue.
|
void |
resetSchedulerMetrics()
Reset scheduler metrics.
|
protected void |
rollbackContainerUpdate(org.apache.hadoop.yarn.api.records.ContainerId containerId)
Rollback container update after expiry.
|
void |
serviceInit(org.apache.hadoop.conf.Configuration conf) |
protected void |
serviceStart() |
protected void |
serviceStop() |
void |
setClock(org.apache.hadoop.yarn.util.Clock clock) |
void |
setClusterMaxPriority(org.apache.hadoop.conf.Configuration conf)
Set the cluster max priority
|
void |
setEntitlement(String queue,
QueueEntitlement entitlement)
This method increase the entitlement for current queue (must respect
invariants, e.g., no overcommit of parents, non negative, etc.).
|
protected void |
setLastNodeUpdateTime(long time) |
boolean |
shouldContainersBeAutoUpdated() |
protected void |
triggerUpdate()
Allows
UpdateThread to start processing without waiting till
updateInterval . |
void |
update()
Update internal state of the scheduler.
|
org.apache.hadoop.yarn.api.records.Priority |
updateApplicationPriority(org.apache.hadoop.yarn.api.records.Priority newPriority,
org.apache.hadoop.yarn.api.records.ApplicationId applicationId,
com.google.common.util.concurrent.SettableFuture<Object> future,
org.apache.hadoop.security.UserGroupInformation user)
Change application priority of a submitted application at runtime
|
void |
updateNodeResource(RMNode nm,
org.apache.hadoop.yarn.api.records.ResourceOption resourceOption)
Process resource update on a node.
|
protected void |
updateNodeResourceUtilization(RMNode nm,
SchedulerNode schedulerNode)
Update container and utilization information on the NodeManager.
|
protected void |
updateSchedulerHealthInformation(org.apache.hadoop.yarn.api.records.Resource releasedResources,
int releasedContainers)
Update schedulerHealth information.
|
close, getBlockers, getConfig, getFailureCause, getFailureState, getLifecycleHistory, getName, getServiceState, getStartTime, init, isInState, noteFailure, putBlocker, registerGlobalListener, registerServiceListener, removeBlocker, setConfig, start, stop, toString, unregisterGlobalListener, unregisterServiceListener, waitForServiceToStop
clone, equals, finalize, getClass, hashCode, notify, notifyAll, wait, wait, wait
setRMContext
allocate, checkAccess, getAppsInQueue, getNumClusterNodes, getQueueInfo, getQueueUserAclInfo, getResourceCalculator, getRootQueueMetrics
recover
protected final ClusterNodeTracker<N extends SchedulerNode> nodeTracker
protected org.apache.hadoop.yarn.api.records.Resource minimumAllocation
protected volatile RMContext rmContext
protected ActivitiesManager activitiesManager
protected SchedulerHealth schedulerHealth
protected volatile long lastNodeUpdateTime
protected final long THREAD_JOIN_TIMEOUT_MS
protected long updateInterval
serviceInit(Configuration)
.protected ConcurrentMap<org.apache.hadoop.yarn.api.records.ApplicationId,SchedulerApplication<T extends SchedulerApplicationAttempt>> applications
protected int nmExpireInterval
protected long nmHeartbeatInterval
protected static final Allocation EMPTY_ALLOCATION
protected final ReentrantReadWriteLock.ReadLock readLock
protected final ReentrantReadWriteLock.WriteLock writeLock
protected SchedulingMonitorManager schedulingMonitorManager
public AbstractYarnScheduler(String name)
name
- service namepublic void serviceInit(org.apache.hadoop.conf.Configuration conf) throws Exception
serviceInit
in class org.apache.hadoop.service.AbstractService
Exception
protected void serviceStart() throws Exception
serviceStart
in class org.apache.hadoop.service.AbstractService
Exception
protected void serviceStop() throws Exception
serviceStop
in class org.apache.hadoop.service.AbstractService
Exception
public ClusterNodeTracker<N> getNodeTracker()
public SchedulingMonitorManager getSchedulingMonitorManager()
public List<org.apache.hadoop.yarn.api.records.Container> getTransferredContainers(org.apache.hadoop.yarn.api.records.ApplicationAttemptId currentAttempt)
YarnScheduler
getTransferredContainers
in interface YarnScheduler
currentAttempt
- the id of the application attemptpublic Map<org.apache.hadoop.yarn.api.records.ApplicationId,SchedulerApplication<T>> getSchedulerApplications()
public List<N> getBlacklistedNodes(SchedulerApplicationAttempt app)
app
- application attempt.public List<N> getNodes(NodeFilter filter)
public boolean shouldContainersBeAutoUpdated()
public org.apache.hadoop.yarn.api.records.Resource getClusterResource()
YarnScheduler
getClusterResource
in interface YarnScheduler
public org.apache.hadoop.yarn.api.records.Resource getMinimumResourceCapability()
YarnScheduler
Resource
.getMinimumResourceCapability
in interface YarnScheduler
public org.apache.hadoop.yarn.api.records.Resource getMaximumResourceCapability()
YarnScheduler
Resource
at the cluster level.getMaximumResourceCapability
in interface YarnScheduler
public org.apache.hadoop.yarn.api.records.Resource getMaximumResourceCapability(String queueName)
YarnScheduler
Resource
for the queue specified.getMaximumResourceCapability
in interface YarnScheduler
queueName
- queue nameprotected void initMaximumResourceCapability(org.apache.hadoop.yarn.api.records.Resource maximumAllocation)
public SchedulerHealth getSchedulerHealth()
protected void setLastNodeUpdateTime(long time)
public long getLastNodeUpdateTime()
protected void containerLaunchedOnNode(org.apache.hadoop.yarn.api.records.ContainerId containerId, SchedulerNode node)
protected void containerIncreasedOnNode(org.apache.hadoop.yarn.api.records.ContainerId containerId, SchedulerNode node, org.apache.hadoop.yarn.api.records.Container increasedContainerReportedByNM)
public T getApplicationAttempt(org.apache.hadoop.yarn.api.records.ApplicationAttemptId applicationAttemptId)
public SchedulerAppReport getSchedulerAppInfo(org.apache.hadoop.yarn.api.records.ApplicationAttemptId appAttemptId)
YarnScheduler
getSchedulerAppInfo
in interface YarnScheduler
appAttemptId
- the id of the application attemptpublic org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport getAppResourceUsageReport(org.apache.hadoop.yarn.api.records.ApplicationAttemptId appAttemptId)
YarnScheduler
getAppResourceUsageReport
in interface YarnScheduler
appAttemptId
- the id of the application attemptpublic T getCurrentAttemptForContainer(org.apache.hadoop.yarn.api.records.ContainerId containerId)
public RMContainer getRMContainer(org.apache.hadoop.yarn.api.records.ContainerId containerId)
YarnScheduler
getRMContainer
in interface YarnScheduler
public SchedulerNodeReport getNodeReport(org.apache.hadoop.yarn.api.records.NodeId nodeId)
YarnScheduler
getNodeReport
in interface YarnScheduler
SchedulerNodeReport
for the node or null
if nodeId does not point to a defined node.public String moveApplication(org.apache.hadoop.yarn.api.records.ApplicationId appId, String newQueue) throws org.apache.hadoop.yarn.exceptions.YarnException
YarnScheduler
moveApplication
in interface YarnScheduler
org.apache.hadoop.yarn.exceptions.YarnException
- if the move cannot be carried outpublic void preValidateMoveApplication(org.apache.hadoop.yarn.api.records.ApplicationId appId, String newQueue) throws org.apache.hadoop.yarn.exceptions.YarnException
preValidateMoveApplication
in interface YarnScheduler
appId
- Application IDnewQueue
- Target QueueNameorg.apache.hadoop.yarn.exceptions.YarnException
- if the pre-validation for move cannot be carried outpublic void removeQueue(String queueName) throws org.apache.hadoop.yarn.exceptions.YarnException
YarnScheduler
removeQueue
in interface YarnScheduler
queueName
- name of the queue to removeorg.apache.hadoop.yarn.exceptions.YarnException
public void addQueue(Queue newQueue) throws org.apache.hadoop.yarn.exceptions.YarnException, IOException
YarnScheduler
addQueue
in interface YarnScheduler
newQueue
- the queue being added.org.apache.hadoop.yarn.exceptions.YarnException
IOException
public void setEntitlement(String queue, QueueEntitlement entitlement) throws org.apache.hadoop.yarn.exceptions.YarnException
YarnScheduler
setEntitlement
in interface YarnScheduler
queue
- the queue for which we change entitlemententitlement
- the new entitlement for the queue (capacity,
maxCapacity, etc..)org.apache.hadoop.yarn.exceptions.YarnException
public void recoverContainersOnNode(List<org.apache.hadoop.yarn.server.api.protocolrecords.NMContainerStatus> containerReports, RMNode nm)
protected void createReleaseCache()
public void clearPendingContainerCache()
@InterfaceAudience.Private public void completedContainer(RMContainer rmContainer, org.apache.hadoop.yarn.api.records.ContainerStatus containerStatus, RMContainerEventType event)
protected abstract void completedContainerInternal(RMContainer rmContainer, org.apache.hadoop.yarn.api.records.ContainerStatus containerStatus, RMContainerEventType event)
protected void releaseContainers(List<org.apache.hadoop.yarn.api.records.ContainerId> containers, SchedulerApplicationAttempt attempt)
public N getSchedulerNode(org.apache.hadoop.yarn.api.records.NodeId nodeId)
YarnScheduler
getSchedulerNode
in interface YarnScheduler
nodeId
- the node id of RMNodepublic void moveAllApps(String sourceQueue, String destQueue) throws org.apache.hadoop.yarn.exceptions.YarnException
YarnScheduler
moveAllApps
in interface YarnScheduler
org.apache.hadoop.yarn.exceptions.YarnException
public void killAllAppsInQueue(String queueName) throws org.apache.hadoop.yarn.exceptions.YarnException
YarnScheduler
killAllAppsInQueue
in interface YarnScheduler
queueName
- the name of queue to be drainedorg.apache.hadoop.yarn.exceptions.YarnException
public void updateNodeResource(RMNode nm, org.apache.hadoop.yarn.api.records.ResourceOption resourceOption)
public EnumSet<org.apache.hadoop.yarn.proto.YarnServiceProtos.SchedulerResourceTypes> getSchedulingResourceTypes()
getSchedulingResourceTypes
in interface YarnScheduler
public Set<String> getPlanQueues() throws org.apache.hadoop.yarn.exceptions.YarnException
YarnScheduler
getPlanQueues
in interface YarnScheduler
org.apache.hadoop.yarn.exceptions.YarnException
protected void refreshMaximumAllocation(org.apache.hadoop.yarn.api.records.Resource newMaxAlloc)
public List<org.apache.hadoop.yarn.api.records.ResourceRequest> getPendingResourceRequestsForAttempt(org.apache.hadoop.yarn.api.records.ApplicationAttemptId attemptId)
getPendingResourceRequestsForAttempt
in interface YarnScheduler
public List<org.apache.hadoop.yarn.api.records.SchedulingRequest> getPendingSchedulingRequestsForAttempt(org.apache.hadoop.yarn.api.records.ApplicationAttemptId attemptId)
YarnScheduler
getPendingSchedulingRequestsForAttempt
in interface YarnScheduler
attemptId
- the id of the application attemptpublic org.apache.hadoop.yarn.api.records.Priority checkAndGetApplicationPriority(org.apache.hadoop.yarn.api.records.Priority priorityRequestedByApp, org.apache.hadoop.security.UserGroupInformation user, String queueName, org.apache.hadoop.yarn.api.records.ApplicationId applicationId) throws org.apache.hadoop.yarn.exceptions.YarnException
YarnScheduler
checkAndGetApplicationPriority
in interface YarnScheduler
priorityRequestedByApp
- Submitted Application priority.user
- User who submitted the ApplicationqueueName
- Name of the QueueapplicationId
- Application IDorg.apache.hadoop.yarn.exceptions.YarnException
public org.apache.hadoop.yarn.api.records.Priority updateApplicationPriority(org.apache.hadoop.yarn.api.records.Priority newPriority, org.apache.hadoop.yarn.api.records.ApplicationId applicationId, com.google.common.util.concurrent.SettableFuture<Object> future, org.apache.hadoop.security.UserGroupInformation user) throws org.apache.hadoop.yarn.exceptions.YarnException
YarnScheduler
updateApplicationPriority
in interface YarnScheduler
newPriority
- Submitted Application priority.applicationId
- Application IDfuture
- Sets any type of exception happened from StateStoreuser
- who submitted the applicationorg.apache.hadoop.yarn.exceptions.YarnException
public org.apache.hadoop.yarn.api.records.Priority getMaxClusterLevelAppPriority()
YarnScheduler
getMaxClusterLevelAppPriority
in interface YarnScheduler
public void setClusterMaxPriority(org.apache.hadoop.conf.Configuration conf) throws org.apache.hadoop.yarn.exceptions.YarnException
YarnScheduler
setClusterMaxPriority
in interface YarnScheduler
org.apache.hadoop.yarn.exceptions.YarnException
protected List<SchedContainerChangeRequest> createSchedContainerChangeRequests(List<org.apache.hadoop.yarn.api.records.UpdateContainerRequest> changeRequests, boolean increase)
public ActivitiesManager getActivitiesManager()
public org.apache.hadoop.yarn.util.Clock getClock()
public void setClock(org.apache.hadoop.yarn.util.Clock clock)
@Lock(value=org.apache.hadoop.yarn.server.utils.Lock.NoLock.class) public SchedulerNode getNode(org.apache.hadoop.yarn.api.records.NodeId nodeId)
protected void updateSchedulerHealthInformation(org.apache.hadoop.yarn.api.records.Resource releasedResources, int releasedContainers)
releasedResources
- Reference resource object for completed containersreleasedContainers
- Count of released containersprotected void updateNodeResourceUtilization(RMNode nm, SchedulerNode schedulerNode)
nm
- The NodeManager to updateschedulerNode
- schedulerNodeprotected void nodeUpdate(RMNode nm)
nm
- The RMNode corresponding to the NodeManagerpublic org.apache.hadoop.yarn.api.records.Resource getNormalizedResource(org.apache.hadoop.yarn.api.records.Resource requestedResource, org.apache.hadoop.yarn.api.records.Resource maxResourceCapability)
YarnScheduler
getNormalizedResource
in interface YarnScheduler
requestedResource
- the resource to be normalizedmaxResourceCapability
- Maximum container allocation value, if null or
empty scheduler level maximum container allocation value will be
usedprotected void normalizeResourceRequests(List<org.apache.hadoop.yarn.api.records.ResourceRequest> asks)
asks
- resource requestsprotected void normalizeResourceRequests(List<org.apache.hadoop.yarn.api.records.ResourceRequest> asks, String queueName)
asks
- resource requestsprotected void handleContainerUpdates(SchedulerApplicationAttempt appAttempt, ContainerUpdates updates)
protected void rollbackContainerUpdate(org.apache.hadoop.yarn.api.records.ContainerId containerId)
containerId
- ContainerId.public List<org.apache.hadoop.yarn.api.records.NodeId> getNodeIds(String resourceName)
ResourceScheduler
NodeId
available in the cluster by resource name.getNodeIds
in interface ResourceScheduler
resourceName
- resource nameNodeId
by resource name.public void asyncContainerRelease(RMContainer container)
container
- Container.public org.apache.hadoop.yarn.api.records.Resource getMinimumAllocation()
public org.apache.hadoop.yarn.api.records.Resource getMaximumAllocation()
public long checkAndGetApplicationLifetime(String queueName, long lifetime)
YarnScheduler
checkAndGetApplicationLifetime
in interface YarnScheduler
queueName
- Name of the Queuelifetime
- configured application lifetimepublic long getMaximumApplicationLifetime(String queueName)
YarnScheduler
getMaximumApplicationLifetime
in interface YarnScheduler
queueName
- to get lifetimepublic abstract void killContainer(RMContainer container)
container
- the container to killpublic void update()
UpdateThread
every updateInterval
. By default, it will
not run; subclasses should set updateInterval
to a
positive value during serviceInit(Configuration)
if they want to
enable the thread.protected void triggerUpdate()
UpdateThread
to start processing without waiting till
updateInterval
.public void reinitialize(org.apache.hadoop.conf.Configuration conf, RMContext rmContext) throws IOException
ResourceScheduler
ResourceScheduler
.reinitialize
in interface ResourceScheduler
conf
- configurationIOException
public boolean attemptAllocationOnNode(SchedulerApplicationAttempt appAttempt, org.apache.hadoop.yarn.api.records.SchedulingRequest schedulingRequest, SchedulerNode schedulerNode)
attemptAllocationOnNode
in interface ResourceScheduler
appAttempt
- ApplicationAttempt.schedulingRequest
- SchedulingRequest.schedulerNode
- SchedulerNode.public void resetSchedulerMetrics()
ResourceScheduler
resetSchedulerMetrics
in interface ResourceScheduler
Copyright © 2008–2021 Apache Software Foundation. All rights reserved.