YARN-6040. Introduce api independent PendingAsk to replace usage of ResourceRequest within Scheduler classes. (Wangda Tan via asuresh)
Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/2977bc6a Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/2977bc6a Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/2977bc6a Branch: refs/heads/trunk Commit: 2977bc6a141041ef7579efc416e93fc55e0c2a1a Parents: 4a659ff Author: Arun Suresh <asur...@apache.org> Authored: Fri Jan 6 09:59:57 2017 -0800 Committer: Arun Suresh <asur...@apache.org> Committed: Fri Jan 6 09:59:57 2017 -0800 ---------------------------------------------------------------------- .../scheduler/AppSchedulingInfo.java | 163 +++++++------ .../scheduler/SchedulerApplicationAttempt.java | 44 ++-- .../allocator/AbstractContainerAllocator.java | 4 + .../allocator/RegularContainerAllocator.java | 132 +++++----- .../scheduler/common/PendingAsk.java | 57 +++++ .../scheduler/common/fica/FiCaSchedulerApp.java | 55 +++-- .../scheduler/fair/FSAppAttempt.java | 160 +++++++------ .../scheduler/fair/FSPreemptionThread.java | 11 +- .../scheduler/fifo/FifoAppAttempt.java | 10 +- .../scheduler/fifo/FifoScheduler.java | 121 +++++----- .../LocalitySchedulingPlacementSet.java | 129 ++++++++-- .../placement/SchedulingPlacementSet.java | 71 +++++- .../scheduler/TestAbstractYarnScheduler.java | 15 +- .../scheduler/TestAppSchedulingInfo.java | 4 +- .../TestSchedulerApplicationAttempt.java | 3 +- .../capacity/TestCapacityScheduler.java | 238 +++++++++---------- .../scheduler/capacity/TestLeafQueue.java | 67 +++--- .../TestNodeLabelContainerAllocation.java | 65 +++-- .../scheduler/capacity/TestReservations.java | 22 +- 19 files changed, 803 insertions(+), 568 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/hadoop/blob/2977bc6a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java ---------------------------------------------------------------------- diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java index d901d90..e2ff082 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java @@ -34,16 +34,18 @@ import org.apache.hadoop.yarn.server.resourcemanager.RMServerUtils; import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.SchedulingMode; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.LocalitySchedulingPlacementSet; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.ResourceRequestUpdateResult; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.SchedulingPlacementSet; import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey; + +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.PendingAsk; import org.apache.hadoop.yarn.util.resource.Resources; import java.util.ArrayList; import java.util.Collection; -import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -582,16 +584,10 @@ public class AppSchedulingInfo { return schedulerKeys.keySet(); } - @SuppressWarnings("unchecked") - public Map<String, ResourceRequest> getResourceRequests( - SchedulerRequestKey schedulerKey) { - SchedulingPlacementSet ps = schedulerKeyToPlacementSets.get(schedulerKey); - if (null != ps) { - return ps.getResourceRequests(); - } - return Collections.emptyMap(); - } - + /** + * Used by REST API to fetch ResourceRequest + * @return All pending ResourceRequests. + */ public List<ResourceRequest> getAllResourceRequests() { List<ResourceRequest> ret = new ArrayList<>(); try { @@ -605,53 +601,51 @@ public class AppSchedulingInfo { return ret; } - public ResourceRequest getResourceRequest(SchedulerRequestKey schedulerKey, - String resourceName) { + public SchedulingPlacementSet getFirstSchedulingPlacementSet() { try { - this.readLock.lock(); - SchedulingPlacementSet ps = - schedulerKeyToPlacementSets.get(schedulerKey); - return (ps == null) ? null : ps.getResourceRequest(resourceName); + readLock.lock(); + for (SchedulerRequestKey key : schedulerKeys.keySet()) { + SchedulingPlacementSet ps = schedulerKeyToPlacementSets.get(key); + if (null != ps) { + return ps; + } + } + return null; } finally { - this.readLock.unlock(); + readLock.unlock(); } + } - public Resource getResource(SchedulerRequestKey schedulerKey) { + public PendingAsk getNextPendingAsk() { try { - this.readLock.lock(); - ResourceRequest request = - getResourceRequest(schedulerKey, ResourceRequest.ANY); - return (request == null) ? null : request.getCapability(); + readLock.lock(); + SchedulerRequestKey firstRequestKey = schedulerKeys.firstKey(); + return getPendingAsk(firstRequestKey, ResourceRequest.ANY); } finally { - this.readLock.unlock(); + readLock.unlock(); } + } - /** - * Method to return the next resource request to be serviced. - * - * In the initial implementation, we just pick any {@link ResourceRequest} - * corresponding to the highest priority. - * - * @return next {@link ResourceRequest} to allocate resources for. - */ - @Unstable - public synchronized ResourceRequest getNextResourceRequest() { - SchedulingPlacementSet<SchedulerNode> ps = schedulerKeyToPlacementSets.get( - schedulerKeys.firstKey()); - if (null != ps) { - for (ResourceRequest rr : ps.getResourceRequests().values()) { - return rr; - } - } + public PendingAsk getPendingAsk(SchedulerRequestKey schedulerKey) { + return getPendingAsk(schedulerKey, ResourceRequest.ANY); + } - return null; + public PendingAsk getPendingAsk(SchedulerRequestKey schedulerKey, + String resourceName) { + try { + this.readLock.lock(); + SchedulingPlacementSet ps = schedulerKeyToPlacementSets.get(schedulerKey); + return (ps == null) ? PendingAsk.ZERO : ps.getPendingAsk(resourceName); + } finally { + this.readLock.unlock(); + } } /** * Returns if the place (node/rack today) is either blacklisted by the - * application (user) or the system + * application (user) or the system. * * @param resourceName * the resourcename @@ -724,7 +718,6 @@ public class AppSchedulingInfo { public List<ResourceRequest> allocate(NodeType type, SchedulerNode node, SchedulerRequestKey schedulerKey, - ResourceRequest request, Container containerAllocated) { try { writeLock.lock(); @@ -733,19 +726,13 @@ public class AppSchedulingInfo { updateMetricsForAllocatedContainer(type, containerAllocated); } - return schedulerKeyToPlacementSets.get(schedulerKey) - .allocate(schedulerKey, type, node, request); + return schedulerKeyToPlacementSets.get(schedulerKey).allocate( + schedulerKey, type, node); } finally { writeLock.unlock(); } } - public List<ResourceRequest> allocate(NodeType type, - SchedulerNode node, SchedulerRequestKey schedulerKey, - Container containerAllocated) { - return allocate(type, node, schedulerKey, null, containerAllocated); - } - public void checkForDeactivation() { if (schedulerKeys.isEmpty()) { activeUsersManager.deactivateApplication(user, applicationId); @@ -758,18 +745,20 @@ public class AppSchedulingInfo { QueueMetrics oldMetrics = queue.getMetrics(); QueueMetrics newMetrics = newQueue.getMetrics(); for (SchedulingPlacementSet ps : schedulerKeyToPlacementSets.values()) { - ResourceRequest request = ps.getResourceRequest(ResourceRequest.ANY); - if (request != null && request.getNumContainers() > 0) { - oldMetrics.decrPendingResources(user, request.getNumContainers(), - request.getCapability()); - newMetrics.incrPendingResources(user, request.getNumContainers(), - request.getCapability()); - - Resource delta = Resources.multiply(request.getCapability(), - request.getNumContainers()); + PendingAsk ask = ps.getPendingAsk(ResourceRequest.ANY); + if (ask.getCount() > 0) { + oldMetrics.decrPendingResources(user, ask.getCount(), + ask.getPerAllocationResource()); + newMetrics.incrPendingResources(user, ask.getCount(), + ask.getPerAllocationResource()); + + Resource delta = Resources.multiply(ask.getPerAllocationResource(), + ask.getCount()); // Update Queue - queue.decPendingResource(request.getNodeLabelExpression(), delta); - newQueue.incPendingResource(request.getNodeLabelExpression(), delta); + queue.decPendingResource( + ps.getPrimaryRequestedNodePartition(), delta); + newQueue.incPendingResource( + ps.getPrimaryRequestedNodePartition(), delta); } } oldMetrics.moveAppFrom(this); @@ -789,16 +778,16 @@ public class AppSchedulingInfo { this.writeLock.lock(); QueueMetrics metrics = queue.getMetrics(); for (SchedulingPlacementSet ps : schedulerKeyToPlacementSets.values()) { - ResourceRequest request = ps.getResourceRequest(ResourceRequest.ANY); - if (request != null && request.getNumContainers() > 0) { - metrics.decrPendingResources(user, request.getNumContainers(), - request.getCapability()); + PendingAsk ask = ps.getPendingAsk(ResourceRequest.ANY); + if (ask.getCount() > 0) { + metrics.decrPendingResources(user, ask.getCount(), + ask.getPerAllocationResource()); // Update Queue queue.decPendingResource( - request.getNodeLabelExpression(), - Resources.multiply(request.getCapability(), - request.getNumContainers())); + ps.getPrimaryRequestedNodePartition(), + Resources.multiply(ask.getPerAllocationResource(), + ask.getCount())); } } metrics.finishAppAttempt(applicationId, pending, user); @@ -906,4 +895,38 @@ public class AppSchedulingInfo { return (SchedulingPlacementSet<N>) schedulerKeyToPlacementSets.get( schedulerkey); } + + /** + * Can delay to next?. + * + * @param schedulerKey schedulerKey + * @param resourceName resourceName + * + * @return If request exists, return {relaxLocality} + * Otherwise, return true. + */ + public boolean canDelayTo( + SchedulerRequestKey schedulerKey, String resourceName) { + try { + this.readLock.lock(); + SchedulingPlacementSet ps = + schedulerKeyToPlacementSets.get(schedulerKey); + return (ps == null) || ps.canDelayTo(resourceName); + } finally { + this.readLock.unlock(); + } + } + + public boolean acceptNodePartition(SchedulerRequestKey schedulerKey, + String nodePartition, SchedulingMode schedulingMode) { + try { + this.readLock.lock(); + SchedulingPlacementSet ps = + schedulerKeyToPlacementSets.get(schedulerKey); + return (ps != null) && ps.acceptNodePartition(nodePartition, + schedulingMode); + } finally { + this.readLock.unlock(); + } + } } http://git-wip-us.apache.org/repos/asf/hadoop/blob/2977bc6a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java ---------------------------------------------------------------------- diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java index 4a8b2da..1f6bd1f 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java @@ -18,7 +18,6 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.HashMap; @@ -78,6 +77,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.Scheduli import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.SchedulingPlacementSet; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy.SchedulableEntity; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.PendingAsk; import org.apache.hadoop.yarn.server.scheduler.OpportunisticContainerContext; import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey; import org.apache.hadoop.yarn.state.InvalidStateTransitionException; @@ -283,11 +283,6 @@ public class SchedulerApplicationAttempt implements SchedulableEntity { return appSchedulingInfo.getUser(); } - public Map<String, ResourceRequest> getResourceRequests( - SchedulerRequestKey schedulerKey) { - return appSchedulingInfo.getResourceRequests(schedulerKey); - } - public Set<ContainerId> getPendingRelease() { return this.pendingRelease; } @@ -299,34 +294,28 @@ public class SchedulerApplicationAttempt implements SchedulableEntity { public Collection<SchedulerRequestKey> getSchedulerKeys() { return appSchedulingInfo.getSchedulerKeys(); } - - public ResourceRequest getResourceRequest( + + public PendingAsk getPendingAsk( SchedulerRequestKey schedulerKey, String resourceName) { try { readLock.lock(); - return appSchedulingInfo.getResourceRequest(schedulerKey, resourceName); + return appSchedulingInfo.getPendingAsk(schedulerKey, resourceName); } finally { readLock.unlock(); } - } - public int getTotalRequiredResources( - SchedulerRequestKey schedulerKey) { - try { - readLock.lock(); - ResourceRequest request = - getResourceRequest(schedulerKey, ResourceRequest.ANY); - return request == null ? 0 : request.getNumContainers(); - } finally { - readLock.unlock(); - } + public int getOutstandingAsksCount(SchedulerRequestKey schedulerKey) { + return getOutstandingAsksCount(schedulerKey, ResourceRequest.ANY); } - public Resource getResource(SchedulerRequestKey schedulerKey) { + public int getOutstandingAsksCount(SchedulerRequestKey schedulerKey, + String resourceName) { try { readLock.lock(); - return appSchedulingInfo.getResource(schedulerKey); + SchedulingPlacementSet ps = appSchedulingInfo.getSchedulingPlacementSet( + schedulerKey); + return ps == null ? 0 : ps.getOutstandingAsksCount(resourceName); } finally { readLock.unlock(); } @@ -625,16 +614,13 @@ public class SchedulerApplicationAttempt implements SchedulableEntity { try { readLock.lock(); for (SchedulerRequestKey schedulerKey : getSchedulerKeys()) { - Map<String, ResourceRequest> requests = getResourceRequests( - schedulerKey); - if (requests != null) { + SchedulingPlacementSet ps = getSchedulingPlacementSet(schedulerKey); + if (ps != null && + ps.getOutstandingAsksCount(ResourceRequest.ANY) > 0) { LOG.debug("showRequests:" + " application=" + getApplicationId() + " headRoom=" + getHeadroom() + " currentConsumption=" + attemptResourceUsage.getUsed().getMemorySize()); - for (ResourceRequest request : requests.values()) { - LOG.debug("showRequests:" + " application=" + getApplicationId() - + " request=" + request); - } + ps.showRequests(); } } } finally { http://git-wip-us.apache.org/repos/asf/hadoop/blob/2977bc6a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/AbstractContainerAllocator.java ---------------------------------------------------------------------- diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/AbstractContainerAllocator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/AbstractContainerAllocator.java index 5bb91e2..a411f17 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/AbstractContainerAllocator.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/AbstractContainerAllocator.java @@ -24,6 +24,7 @@ import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.server.resourcemanager.RMContext; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AppSchedulingInfo; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.ActivitiesLogger; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.ActivitiesManager; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.ActivityDiagnosticConstant; @@ -46,6 +47,7 @@ public abstract class AbstractContainerAllocator { private static final Log LOG = LogFactory.getLog(AbstractContainerAllocator.class); FiCaSchedulerApp application; + AppSchedulingInfo appInfo; final ResourceCalculator rc; final RMContext rmContext; ActivitiesManager activitiesManager; @@ -59,6 +61,8 @@ public abstract class AbstractContainerAllocator { ResourceCalculator rc, RMContext rmContext, ActivitiesManager activitiesManager) { this.application = application; + this.appInfo = + application == null ? null : application.getAppSchedulingInfo(); this.rc = rc; this.rmContext = rmContext; this.activitiesManager = activitiesManager; http://git-wip-us.apache.org/repos/asf/hadoop/blob/2977bc6a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/RegularContainerAllocator.java ---------------------------------------------------------------------- diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/RegularContainerAllocator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/RegularContainerAllocator.java index eeb0815..8078bcd 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/RegularContainerAllocator.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/RegularContainerAllocator.java @@ -39,7 +39,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerAppUtils; import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.ActivityDiagnosticConstant; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.ActivitiesLogger; @@ -54,6 +53,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaS import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.PlacementSet; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.PlacementSetUtils; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.SchedulingPlacementSet; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.PendingAsk; import org.apache.hadoop.yarn.server.utils.BuilderUtils; import org.apache.hadoop.yarn.util.resource.ResourceCalculator; import org.apache.hadoop.yarn.util.resource.Resources; @@ -64,8 +64,6 @@ import org.apache.hadoop.yarn.util.resource.Resources; */ public class RegularContainerAllocator extends AbstractContainerAllocator { private static final Log LOG = LogFactory.getLog(RegularContainerAllocator.class); - - private ResourceRequest lastResourceRequest = null; public RegularContainerAllocator(FiCaSchedulerApp application, ResourceCalculator rc, RMContext rmContext, @@ -103,9 +101,10 @@ public class RegularContainerAllocator extends AbstractContainerAllocator { Priority priority = schedulerKey.getPriority(); FiCaSchedulerNode node = PlacementSetUtils.getSingleNode(ps); - ResourceRequest anyRequest = - application.getResourceRequest(schedulerKey, ResourceRequest.ANY); - if (null == anyRequest) { + PendingAsk offswitchPendingAsk = application.getPendingAsk(schedulerKey, + ResourceRequest.ANY); + + if (offswitchPendingAsk.getCount() <= 0) { ActivitiesLogger.APP.recordSkippedAppActivityWithoutAllocation( activitiesManager, node, application, priority, ActivityDiagnosticConstant.PRIORITY_SKIPPED_BECAUSE_NULL_ANY_REQUEST); @@ -113,10 +112,10 @@ public class RegularContainerAllocator extends AbstractContainerAllocator { } // Required resource - Resource required = anyRequest.getCapability(); + Resource required = offswitchPendingAsk.getPerAllocationResource(); // Do we need containers at this 'priority'? - if (application.getTotalRequiredResources(schedulerKey) <= 0) { + if (application.getOutstandingAsksCount(schedulerKey) <= 0) { ActivitiesLogger.APP.recordSkippedAppActivityWithoutAllocation( activitiesManager, node, application, priority, ActivityDiagnosticConstant.APPLICATION_PRIORITY_DO_NOT_NEED_RESOURCE); @@ -141,11 +140,9 @@ public class RegularContainerAllocator extends AbstractContainerAllocator { } } - // Is the node-label-expression of this offswitch resource request - // matches the node's label? + // Is the nodePartition of pending request matches the node's partition // If not match, jump to next priority. - if (!SchedulerUtils.checkResourceRequestMatchingNodePartition( - anyRequest.getNodeLabelExpression(), ps.getPartition(), + if (!appInfo.acceptNodePartition(schedulerKey, node.getPartition(), schedulingMode)) { ActivitiesLogger.APP.recordSkippedAppActivityWithoutAllocation( activitiesManager, node, application, priority, @@ -182,8 +179,11 @@ public class RegularContainerAllocator extends AbstractContainerAllocator { // This is to make sure non-partitioned-resource-request will prefer // to be allocated to non-partitioned nodes int missedNonPartitionedRequestSchedulingOpportunity = 0; - if (anyRequest.getNodeLabelExpression() - .equals(RMNodeLabelsManager.NO_LABEL)) { + // Only do this when request associated with given scheduler key accepts + // NO_LABEL under RESPECT_EXCLUSIVITY mode + if (StringUtils.equals(RMNodeLabelsManager.NO_LABEL, + appInfo.getSchedulingPlacementSet(schedulerKey) + .getPrimaryRequestedNodePartition())) { missedNonPartitionedRequestSchedulingOpportunity = application.addMissedNonPartitionedRequestSchedulingOpportunity( schedulerKey); @@ -264,8 +264,9 @@ public class RegularContainerAllocator extends AbstractContainerAllocator { public float getLocalityWaitFactor( SchedulerRequestKey schedulerKey, int clusterNodes) { // Estimate: Required unique resources (i.e. hosts + racks) - int requiredResources = - Math.max(application.getResourceRequests(schedulerKey).size() - 1, 0); + int requiredResources = Math.max( + application.getSchedulingPlacementSet(schedulerKey) + .getUniqueLocationAsks() - 1, 0); // waitFactor can't be more than '1' // i.e. no point skipping more than clustersize opportunities @@ -287,11 +288,10 @@ public class RegularContainerAllocator extends AbstractContainerAllocator { } // 'Delay' off-switch - ResourceRequest offSwitchRequest = - application.getResourceRequest(schedulerKey, ResourceRequest.ANY); long missedOpportunities = application.getSchedulingOpportunities(schedulerKey); - long requiredContainers = offSwitchRequest.getNumContainers(); + long requiredContainers = application.getOutstandingAsksCount( + schedulerKey); float localityWaitFactor = getLocalityWaitFactor(schedulerKey, rmContext.getScheduler() @@ -304,9 +304,8 @@ public class RegularContainerAllocator extends AbstractContainerAllocator { } // Check if we need containers on this rack - ResourceRequest rackLocalRequest = - application.getResourceRequest(schedulerKey, node.getRackName()); - if (rackLocalRequest == null || rackLocalRequest.getNumContainers() <= 0) { + if (application.getOutstandingAsksCount(schedulerKey, node.getRackName()) + <= 0) { return false; } @@ -321,24 +320,21 @@ public class RegularContainerAllocator extends AbstractContainerAllocator { // Check if we need containers on this host if (type == NodeType.NODE_LOCAL) { // Now check if we need containers on this host... - ResourceRequest nodeLocalRequest = - application.getResourceRequest(schedulerKey, node.getNodeName()); - if (nodeLocalRequest != null) { - return nodeLocalRequest.getNumContainers() > 0; - } + return application.getOutstandingAsksCount(schedulerKey, + node.getNodeName()) > 0; } return false; } private ContainerAllocation assignNodeLocalContainers( - Resource clusterResource, ResourceRequest nodeLocalResourceRequest, + Resource clusterResource, PendingAsk nodeLocalAsk, FiCaSchedulerNode node, SchedulerRequestKey schedulerKey, RMContainer reservedContainer, SchedulingMode schedulingMode, ResourceLimits currentResoureLimits) { if (canAssign(schedulerKey, node, NodeType.NODE_LOCAL, reservedContainer)) { return assignContainer(clusterResource, node, schedulerKey, - nodeLocalResourceRequest, NodeType.NODE_LOCAL, reservedContainer, + nodeLocalAsk, NodeType.NODE_LOCAL, reservedContainer, schedulingMode, currentResoureLimits); } @@ -350,13 +346,13 @@ public class RegularContainerAllocator extends AbstractContainerAllocator { } private ContainerAllocation assignRackLocalContainers( - Resource clusterResource, ResourceRequest rackLocalResourceRequest, + Resource clusterResource, PendingAsk rackLocalAsk, FiCaSchedulerNode node, SchedulerRequestKey schedulerKey, RMContainer reservedContainer, SchedulingMode schedulingMode, ResourceLimits currentResoureLimits) { if (canAssign(schedulerKey, node, NodeType.RACK_LOCAL, reservedContainer)) { return assignContainer(clusterResource, node, schedulerKey, - rackLocalResourceRequest, NodeType.RACK_LOCAL, reservedContainer, + rackLocalAsk, NodeType.RACK_LOCAL, reservedContainer, schedulingMode, currentResoureLimits); } @@ -368,13 +364,13 @@ public class RegularContainerAllocator extends AbstractContainerAllocator { } private ContainerAllocation assignOffSwitchContainers( - Resource clusterResource, ResourceRequest offSwitchResourceRequest, + Resource clusterResource, PendingAsk offSwitchAsk, FiCaSchedulerNode node, SchedulerRequestKey schedulerKey, RMContainer reservedContainer, SchedulingMode schedulingMode, ResourceLimits currentResoureLimits) { if (canAssign(schedulerKey, node, NodeType.OFF_SWITCH, reservedContainer)) { return assignContainer(clusterResource, node, schedulerKey, - offSwitchResourceRequest, NodeType.OFF_SWITCH, reservedContainer, + offSwitchAsk, NodeType.OFF_SWITCH, reservedContainer, schedulingMode, currentResoureLimits); } @@ -396,12 +392,12 @@ public class RegularContainerAllocator extends AbstractContainerAllocator { NodeType requestLocalityType = null; // Data-local - ResourceRequest nodeLocalResourceRequest = - application.getResourceRequest(schedulerKey, node.getNodeName()); - if (nodeLocalResourceRequest != null) { + PendingAsk nodeLocalAsk = + application.getPendingAsk(schedulerKey, node.getNodeName()); + if (nodeLocalAsk.getCount() > 0) { requestLocalityType = NodeType.NODE_LOCAL; allocation = - assignNodeLocalContainers(clusterResource, nodeLocalResourceRequest, + assignNodeLocalContainers(clusterResource, nodeLocalAsk, node, schedulerKey, reservedContainer, schedulingMode, currentResoureLimits); if (Resources.greaterThan(rc, clusterResource, @@ -412,10 +408,10 @@ public class RegularContainerAllocator extends AbstractContainerAllocator { } // Rack-local - ResourceRequest rackLocalResourceRequest = - application.getResourceRequest(schedulerKey, node.getRackName()); - if (rackLocalResourceRequest != null) { - if (!rackLocalResourceRequest.getRelaxLocality()) { + PendingAsk rackLocalAsk = + application.getPendingAsk(schedulerKey, node.getRackName()); + if (rackLocalAsk.getCount() > 0) { + if (!appInfo.canDelayTo(schedulerKey, node.getRackName())) { ActivitiesLogger.APP.recordSkippedAppActivityWithoutAllocation( activitiesManager, node, application, priority, ActivityDiagnosticConstant.SKIP_PRIORITY_BECAUSE_OF_RELAX_LOCALITY); @@ -427,7 +423,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator { requestLocalityType; allocation = - assignRackLocalContainers(clusterResource, rackLocalResourceRequest, + assignRackLocalContainers(clusterResource, rackLocalAsk, node, schedulerKey, reservedContainer, schedulingMode, currentResoureLimits); if (Resources.greaterThan(rc, clusterResource, @@ -438,10 +434,10 @@ public class RegularContainerAllocator extends AbstractContainerAllocator { } // Off-switch - ResourceRequest offSwitchResourceRequest = - application.getResourceRequest(schedulerKey, ResourceRequest.ANY); - if (offSwitchResourceRequest != null) { - if (!offSwitchResourceRequest.getRelaxLocality()) { + PendingAsk offSwitchAsk = + application.getPendingAsk(schedulerKey, ResourceRequest.ANY); + if (offSwitchAsk.getCount() > 0) { + if (!appInfo.canDelayTo(schedulerKey, ResourceRequest.ANY)) { ActivitiesLogger.APP.recordSkippedAppActivityWithoutAllocation( activitiesManager, node, application, priority, ActivityDiagnosticConstant.SKIP_PRIORITY_BECAUSE_OF_RELAX_LOCALITY); @@ -453,7 +449,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator { requestLocalityType; allocation = - assignOffSwitchContainers(clusterResource, offSwitchResourceRequest, + assignOffSwitchContainers(clusterResource, offSwitchAsk, node, schedulerKey, reservedContainer, schedulingMode, currentResoureLimits); @@ -474,41 +470,25 @@ public class RegularContainerAllocator extends AbstractContainerAllocator { private ContainerAllocation assignContainer(Resource clusterResource, FiCaSchedulerNode node, SchedulerRequestKey schedulerKey, - ResourceRequest request, NodeType type, RMContainer rmContainer, + PendingAsk pendingAsk, NodeType type, RMContainer rmContainer, SchedulingMode schedulingMode, ResourceLimits currentResoureLimits) { Priority priority = schedulerKey.getPriority(); - lastResourceRequest = request; - + if (LOG.isDebugEnabled()) { LOG.debug("assignContainers: node=" + node.getNodeName() + " application=" + application.getApplicationId() + " priority=" + schedulerKey.getPriority() - + " request=" + request + " type=" + type); - } - - // check if the resource request can access the label - if (!SchedulerUtils.checkResourceRequestMatchingNodePartition( - request.getNodeLabelExpression(), node.getPartition(), - schedulingMode)) { - // this is a reserved container, but we cannot allocate it now according - // to label not match. This can be caused by node label changed - // We should un-reserve this container. - ActivitiesLogger.APP.recordAppActivityWithoutAllocation(activitiesManager, - node, application, priority, - ActivityDiagnosticConstant.REQUEST_CAN_NOT_ACCESS_NODE_LABEL, - ActivityState.REJECTED); - return new ContainerAllocation(rmContainer, null, - AllocationState.LOCALITY_SKIPPED); + + " pendingAsk=" + pendingAsk + " type=" + type); } - Resource capability = request.getCapability(); + Resource capability = pendingAsk.getPerAllocationResource(); Resource available = node.getUnallocatedResource(); Resource totalResource = node.getTotalResource(); if (!Resources.lessThanOrEqual(rc, clusterResource, capability, totalResource)) { LOG.warn("Node : " + node.getNodeID() - + " does not have sufficient resource for request : " + request + + " does not have sufficient resource for ask : " + pendingAsk + " node total capability : " + node.getTotalResource()); // Skip this locality request ActivitiesLogger.APP.recordSkippedAppActivityWithoutAllocation( @@ -600,9 +580,8 @@ public class RegularContainerAllocator extends AbstractContainerAllocator { } } - ContainerAllocation result = - new ContainerAllocation(unreservedContainer, request.getCapability(), - AllocationState.ALLOCATED); + ContainerAllocation result = new ContainerAllocation(unreservedContainer, + pendingAsk.getPerAllocationResource(), AllocationState.ALLOCATED); result.containerNodeType = type; result.setToKillContainers(toKillContainers); return result; @@ -626,9 +605,8 @@ public class RegularContainerAllocator extends AbstractContainerAllocator { } } - ContainerAllocation result = - new ContainerAllocation(null, request.getCapability(), - AllocationState.RESERVED); + ContainerAllocation result = new ContainerAllocation(null, + pendingAsk.getPerAllocationResource(), AllocationState.RESERVED); result.containerNodeType = type; result.setToKillContainers(null); return result; @@ -644,7 +622,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator { boolean shouldAllocOrReserveNewContainer( SchedulerRequestKey schedulerKey, Resource required) { int requiredContainers = - application.getTotalRequiredResources(schedulerKey); + application.getOutstandingAsksCount(schedulerKey); int reservedContainers = application.getNumReservedContainers(schedulerKey); int starvation = 0; if (reservedContainers > 0) { @@ -699,7 +677,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator { SchedulerRequestKey schedulerKey, Container container) { // Inform the application RMContainer allocatedContainer = application.allocate(node, schedulerKey, - lastResourceRequest, container); + container); allocationResult.updatedContainer = allocatedContainer; @@ -803,7 +781,7 @@ public class RegularContainerAllocator extends AbstractContainerAllocator { } } else { // pre-check when allocating reserved container - if (application.getTotalRequiredResources(schedulerKey) == 0) { + if (application.getOutstandingAsksCount(schedulerKey) == 0) { // Release return new ContainerAllocation(reservedContainer, null, AllocationState.QUEUE_SKIPPED); http://git-wip-us.apache.org/repos/asf/hadoop/blob/2977bc6a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/PendingAsk.java ---------------------------------------------------------------------- diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/PendingAsk.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/PendingAsk.java new file mode 100644 index 0000000..85d8715 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/PendingAsk.java @@ -0,0 +1,57 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.yarn.server.resourcemanager.scheduler.common; + +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.util.resource.Resources; + +/** + * {@link PendingAsk} is the class to include minimal information of how much + * resource to ask under constraints (e.g. on one host / rack / node-attributes) + * , etc. + */ +public class PendingAsk { + private final Resource perAllocationResource; + private final int count; + public final static PendingAsk ZERO = new PendingAsk(Resources.none(), 0); + + public PendingAsk(Resource res, int num) { + this.perAllocationResource = res; + this.count = num; + } + + public Resource getPerAllocationResource() { + return perAllocationResource; + } + + public int getCount() { + return count; + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append("<per-allocation-resource="); + sb.append(getPerAllocationResource()); + sb.append(",repeat="); + sb.append(getCount()); + sb.append(">"); + return sb.toString(); + } +} http://git-wip-us.apache.org/repos/asf/hadoop/blob/2977bc6a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java ---------------------------------------------------------------------- diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java index 809446f..4329335 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java @@ -70,6 +70,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.Placeme import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.SchedulingPlacementSet; import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey; + +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.PendingAsk; import org.apache.hadoop.yarn.server.utils.BuilderUtils; import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator; import org.apache.hadoop.yarn.util.resource.ResourceCalculator; @@ -206,8 +208,7 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt { } public RMContainer allocate(FiCaSchedulerNode node, - SchedulerRequestKey schedulerKey, ResourceRequest request, - Container container) { + SchedulerRequestKey schedulerKey, Container container) { try { readLock.lock(); @@ -217,7 +218,16 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt { // Required sanity check - AM can call 'allocate' to update resource // request without locking the scheduler, hence we need to check - if (getTotalRequiredResources(schedulerKey) <= 0) { + if (getOutstandingAsksCount(schedulerKey) <= 0) { + return null; + } + + SchedulingPlacementSet<FiCaSchedulerNode> ps = + appSchedulingInfo.getSchedulingPlacementSet(schedulerKey); + if (null == ps) { + LOG.warn("Failed to get " + SchedulingPlacementSet.class.getName() + + " for application=" + getApplicationId() + " schedulerRequestKey=" + + schedulerKey); return null; } @@ -225,7 +235,7 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt { RMContainer rmContainer = new RMContainerImpl(container, schedulerKey, this.getApplicationAttemptId(), node.getNodeID(), appSchedulingInfo.getUser(), this.rmContext, - request.getNodeLabelExpression()); + ps.getPrimaryRequestedNodePartition()); ((RMContainerImpl) rmContainer).setQueueName(this.getQueueName()); // FIXME, should set when confirmed @@ -694,21 +704,36 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt { return false; } - public synchronized Map<String, Resource> getTotalPendingRequestsPerPartition() { + public Map<String, Resource> getTotalPendingRequestsPerPartition() { + try { + readLock.lock(); + + Map<String, Resource> ret = new HashMap<>(); + for (SchedulerRequestKey schedulerKey : appSchedulingInfo + .getSchedulerKeys()) { + SchedulingPlacementSet<FiCaSchedulerNode> ps = + appSchedulingInfo.getSchedulingPlacementSet(schedulerKey); + + String nodePartition = ps.getPrimaryRequestedNodePartition(); + Resource res = ret.get(nodePartition); + if (null == res) { + res = Resources.createResource(0); + ret.put(nodePartition, res); + } - Map<String, Resource> ret = new HashMap<String, Resource>(); - Resource res = null; - for (SchedulerRequestKey key : appSchedulingInfo.getSchedulerKeys()) { - ResourceRequest rr = appSchedulingInfo.getResourceRequest(key, "*"); - if ((res = ret.get(rr.getNodeLabelExpression())) == null) { - res = Resources.createResource(0, 0); - ret.put(rr.getNodeLabelExpression(), res); + PendingAsk ask = ps.getPendingAsk(ResourceRequest.ANY); + if (ask.getCount() > 0) { + Resources.addTo(res, Resources + .multiply(ask.getPerAllocationResource(), + ask.getCount())); + } } - Resources.addTo(res, - Resources.multiply(rr.getCapability(), rr.getNumContainers())); + return ret; + } finally { + readLock.unlock(); } - return ret; + } public void markContainerForPreemption(ContainerId cont) { http://git-wip-us.apache.org/repos/asf/hadoop/blob/2977bc6a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSAppAttempt.java ---------------------------------------------------------------------- diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSAppAttempt.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSAppAttempt.java index 0daa8a4..d7ed7d1 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSAppAttempt.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSAppAttempt.java @@ -18,16 +18,6 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair; -import java.text.DecimalFormat; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; - import com.google.common.annotations.VisibleForTesting; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -56,11 +46,22 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Queue; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.PendingAsk; import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey; import org.apache.hadoop.yarn.server.utils.BuilderUtils; import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator; import org.apache.hadoop.yarn.util.resource.Resources; +import java.text.DecimalFormat; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + /** * Represents an application attempt from the viewpoint of the Fair Scheduler. */ @@ -416,7 +417,7 @@ public class FSAppAttempt extends SchedulerApplicationAttempt } public RMContainer allocate(NodeType type, FSSchedulerNode node, - SchedulerRequestKey schedulerKey, ResourceRequest request, + SchedulerRequestKey schedulerKey, PendingAsk pendingAsk, Container reservedContainer) { RMContainer rmContainer; Container container; @@ -437,13 +438,13 @@ public class FSAppAttempt extends SchedulerApplicationAttempt // Required sanity check - AM can call 'allocate' to update resource // request without locking the scheduler, hence we need to check - if (getTotalRequiredResources(schedulerKey) <= 0) { + if (getOutstandingAsksCount(schedulerKey) <= 0) { return null; } container = reservedContainer; if (container == null) { - container = createContainer(node, request.getCapability(), + container = createContainer(node, pendingAsk.getPerAllocationResource(), schedulerKey); } @@ -459,7 +460,7 @@ public class FSAppAttempt extends SchedulerApplicationAttempt // Update consumption and track allocations List<ResourceRequest> resourceRequestList = appSchedulingInfo.allocate( - type, node, schedulerKey, request, container); + type, node, schedulerKey, container); this.attemptResourceUsage.incUsed(container.getResource()); // Update resource requests related to "request" and store in RMContainer @@ -632,7 +633,7 @@ public class FSAppAttempt extends SchedulerApplicationAttempt * in {@link FSSchedulerNode}.. * return whether reservation was possible with the current threshold limits */ - private boolean reserve(ResourceRequest request, FSSchedulerNode node, + private boolean reserve(Resource perAllocationResource, FSSchedulerNode node, Container reservedContainer, NodeType type, SchedulerRequestKey schedulerKey) { @@ -641,7 +642,7 @@ public class FSAppAttempt extends SchedulerApplicationAttempt " app_id=" + getApplicationId()); if (reservedContainer == null) { reservedContainer = - createContainer(node, request.getCapability(), + createContainer(node, perAllocationResource, schedulerKey); getMetrics().reserveResource(getUser(), reservedContainer.getResource()); @@ -763,8 +764,8 @@ public class FSAppAttempt extends SchedulerApplicationAttempt * * @param node * The node to try placing the container on. - * @param request - * The ResourceRequest we're trying to satisfy. + * @param pendingAsk + * The {@link PendingAsk} we're trying to satisfy. * @param type * The locality of the assignment. * @param reserved @@ -776,11 +777,11 @@ public class FSAppAttempt extends SchedulerApplicationAttempt * made, returns an empty resource. */ private Resource assignContainer( - FSSchedulerNode node, ResourceRequest request, NodeType type, + FSSchedulerNode node, PendingAsk pendingAsk, NodeType type, boolean reserved, SchedulerRequestKey schedulerKey) { // How much does this request need? - Resource capability = request.getCapability(); + Resource capability = pendingAsk.getPerAllocationResource(); // How much does the node have? Resource available = node.getUnallocatedResource(); @@ -794,7 +795,7 @@ public class FSAppAttempt extends SchedulerApplicationAttempt if (Resources.fitsIn(capability, available)) { // Inform the application of the new container for this request RMContainer allocatedContainer = - allocate(type, node, schedulerKey, request, + allocate(type, node, schedulerKey, pendingAsk, reservedContainer); if (allocatedContainer == null) { // Did the application need this resource? @@ -825,8 +826,9 @@ public class FSAppAttempt extends SchedulerApplicationAttempt } // The desired container won't fit here, so reserve - if (isReservable(capability) && - reserve(request, node, reservedContainer, type, schedulerKey)) { + if (isReservable(capability) && reserve( + pendingAsk.getPerAllocationResource(), node, reservedContainer, type, + schedulerKey)) { if (isWaitingForAMContainer()) { updateAMDiagnosticMsg(capability, " exceed the available resources of the node and the request is" @@ -841,7 +843,7 @@ public class FSAppAttempt extends SchedulerApplicationAttempt } if (LOG.isDebugEnabled()) { LOG.debug("Couldn't creating reservation for " + - getName() + ",at priority " + request.getPriority()); + getName() + ",at priority " + schedulerKey.getPriority()); } return Resources.none(); } @@ -852,19 +854,16 @@ public class FSAppAttempt extends SchedulerApplicationAttempt getQueue().getPolicy().getResourceCalculator(), capacity); } - private boolean hasNodeOrRackLocalRequests(SchedulerRequestKey schedulerKey) { - return getResourceRequests(schedulerKey).size() > 1; - } - /** * Whether the AM container for this app is over maxAMShare limit. */ private boolean isOverAMShareLimit() { // Check the AM resource usage for the leaf queue if (!isAmRunning() && !getUnmanagedAM()) { - List<ResourceRequest> ask = appSchedulingInfo.getAllResourceRequests(); - if (ask.isEmpty() || !getQueue().canRunAppAM( - ask.get(0).getCapability())) { + // Return true if we have not ask, or queue is not be able to run app's AM + PendingAsk ask = appSchedulingInfo.getNextPendingAsk(); + if (ask.getCount() == 0 || !getQueue().canRunAppAM( + ask.getPerAllocationResource())) { return true; } } @@ -886,6 +885,11 @@ public class FSAppAttempt extends SchedulerApplicationAttempt // (not scheduled) in order to promote better locality. try { writeLock.lock(); + + // TODO (wandga): All logics in this method should be added to + // SchedulerPlacement#canDelayTo which is independent from scheduler. + // Scheduler can choose to use various/pluggable delay-scheduling + // implementation. for (SchedulerRequestKey schedulerKey : keysToTry) { // Skip it for reserved container, since // we already check it in isValidReservation. @@ -895,14 +899,16 @@ public class FSAppAttempt extends SchedulerApplicationAttempt addSchedulingOpportunity(schedulerKey); - ResourceRequest rackLocalRequest = getResourceRequest(schedulerKey, + PendingAsk rackLocalPendingAsk = getPendingAsk(schedulerKey, node.getRackName()); - ResourceRequest localRequest = getResourceRequest(schedulerKey, + PendingAsk nodeLocalPendingAsk = getPendingAsk(schedulerKey, node.getNodeName()); - if (localRequest != null && !localRequest.getRelaxLocality()) { + if (nodeLocalPendingAsk.getCount() > 0 + && !appSchedulingInfo.canDelayTo(schedulerKey, + node.getNodeName())) { LOG.warn("Relax locality off is not supported on local request: " - + localRequest); + + nodeLocalPendingAsk); } NodeType allowedLocality; @@ -918,23 +924,23 @@ public class FSAppAttempt extends SchedulerApplicationAttempt scheduler.getRackLocalityThreshold()); } - if (rackLocalRequest != null && rackLocalRequest.getNumContainers() != 0 - && localRequest != null && localRequest.getNumContainers() != 0) { + if (rackLocalPendingAsk.getCount() > 0 + && nodeLocalPendingAsk.getCount() > 0) { if (LOG.isTraceEnabled()) { LOG.trace("Assign container on " + node.getNodeName() + " node, assignType: NODE_LOCAL" + ", allowedLocality: " + allowedLocality + ", priority: " + schedulerKey.getPriority() + ", app attempt id: " + this.attemptId); } - return assignContainer(node, localRequest, NodeType.NODE_LOCAL, + return assignContainer(node, nodeLocalPendingAsk, NodeType.NODE_LOCAL, reserved, schedulerKey); } - if (rackLocalRequest != null && !rackLocalRequest.getRelaxLocality()) { + if (!appSchedulingInfo.canDelayTo(schedulerKey, node.getRackName())) { continue; } - if (rackLocalRequest != null && rackLocalRequest.getNumContainers() != 0 + if (rackLocalPendingAsk.getCount() > 0 && (allowedLocality.equals(NodeType.RACK_LOCAL) || allowedLocality .equals(NodeType.OFF_SWITCH))) { if (LOG.isTraceEnabled()) { @@ -943,27 +949,26 @@ public class FSAppAttempt extends SchedulerApplicationAttempt + allowedLocality + ", priority: " + schedulerKey.getPriority() + ", app attempt id: " + this.attemptId); } - return assignContainer(node, rackLocalRequest, NodeType.RACK_LOCAL, + return assignContainer(node, rackLocalPendingAsk, NodeType.RACK_LOCAL, reserved, schedulerKey); } - ResourceRequest offSwitchRequest = getResourceRequest(schedulerKey, + PendingAsk offswitchAsk = getPendingAsk(schedulerKey, ResourceRequest.ANY); - if (offSwitchRequest != null && !offSwitchRequest.getRelaxLocality()) { + if (!appSchedulingInfo.canDelayTo(schedulerKey, ResourceRequest.ANY)) { continue; } - if (offSwitchRequest != null - && offSwitchRequest.getNumContainers() != 0) { - if (!hasNodeOrRackLocalRequests(schedulerKey) || allowedLocality - .equals(NodeType.OFF_SWITCH)) { + if (offswitchAsk.getCount() > 0) { + if (getSchedulingPlacementSet(schedulerKey).getUniqueLocationAsks() + <= 1 || allowedLocality.equals(NodeType.OFF_SWITCH)) { if (LOG.isTraceEnabled()) { LOG.trace("Assign container on " + node.getNodeName() + " node, assignType: OFF_SWITCH" + ", allowedLocality: " + allowedLocality + ", priority: " + schedulerKey.getPriority() + ", app attempt id: " + this.attemptId); } - return assignContainer(node, offSwitchRequest, NodeType.OFF_SWITCH, + return assignContainer(node, offswitchAsk, NodeType.OFF_SWITCH, reserved, schedulerKey); } } @@ -988,29 +993,35 @@ public class FSAppAttempt extends SchedulerApplicationAttempt */ private boolean hasContainerForNode(SchedulerRequestKey key, FSSchedulerNode node) { - ResourceRequest anyRequest = getResourceRequest(key, ResourceRequest.ANY); - ResourceRequest rackRequest = getResourceRequest(key, node.getRackName()); - ResourceRequest nodeRequest = getResourceRequest(key, node.getNodeName()); + PendingAsk offswitchAsk = getPendingAsk(key, ResourceRequest.ANY); + Resource resource = offswitchAsk.getPerAllocationResource(); + boolean hasRequestForOffswitch = + offswitchAsk.getCount() > 0; + boolean hasRequestForRack = getOutstandingAsksCount(key, + node.getRackName()) > 0; + boolean hasRequestForNode = getOutstandingAsksCount(key, + node.getNodeName()) > 0; boolean ret = true; if (!(// There must be outstanding requests at the given priority: - anyRequest != null && anyRequest.getNumContainers() > 0 && - // If locality relaxation is turned off at *-level, there must be a - // non-zero request for the node's rack: - (anyRequest.getRelaxLocality() || - (rackRequest != null && rackRequest.getNumContainers() > 0)) && - // If locality relaxation is turned off at rack-level, there must be a - // non-zero request at the node: - (rackRequest == null || rackRequest.getRelaxLocality() || - (nodeRequest != null && nodeRequest.getNumContainers() > 0)) && - // The requested container must be able to fit on the node: - Resources.lessThanOrEqual(RESOURCE_CALCULATOR, null, - anyRequest.getCapability(), node.getRMNode().getTotalCapability()))) { + hasRequestForOffswitch && + // If locality relaxation is turned off at *-level, there must be a + // non-zero request for the node's rack: + (appSchedulingInfo.canDelayTo(key, ResourceRequest.ANY) || + (hasRequestForRack)) && + // If locality relaxation is turned off at rack-level, + // there must be a non-zero request at the node: + (!hasRequestForRack || appSchedulingInfo.canDelayTo(key, + node.getRackName()) || (hasRequestForNode)) && + // The requested container must be able to fit on the node: + Resources.lessThanOrEqual(RESOURCE_CALCULATOR, null, + resource, + node.getRMNode().getTotalCapability()))) { ret = false; - } else if (!getQueue().fitsInMaxShare(anyRequest.getCapability())) { + } else if (!getQueue().fitsInMaxShare(resource)) { // The requested container must fit in queue maximum share if (isWaitingForAMContainer()) { - updateAMDiagnosticMsg(anyRequest.getCapability(), + updateAMDiagnosticMsg(resource, " exceeds current queue or its parents maximum resource allowed)."); } ret = false; @@ -1091,10 +1102,6 @@ public class FSAppAttempt extends SchedulerApplicationAttempt return this.fairshareStarvation; } - ResourceRequest getNextResourceRequest() { - return appSchedulingInfo.getNextResourceRequest(); - } - /** * Helper method that captures if this app is identified to be starved. * @return true if the app is starved for fairshare, false otherwise @@ -1174,10 +1181,11 @@ public class FSAppAttempt extends SchedulerApplicationAttempt try { writeLock.lock(); for (SchedulerRequestKey k : getSchedulerKeys()) { - ResourceRequest r = getResourceRequest(k, ResourceRequest.ANY); - if (r != null) { - Resources.multiplyAndAddTo(demand, r.getCapability(), - r.getNumContainers()); + PendingAsk pendingAsk = getPendingAsk(k, ResourceRequest.ANY); + if (pendingAsk.getCount() > 0) { + Resources.multiplyAndAddTo(demand, + pendingAsk.getPerAllocationResource(), + pendingAsk.getCount()); } } } finally { @@ -1189,9 +1197,9 @@ public class FSAppAttempt extends SchedulerApplicationAttempt public Resource assignContainer(FSSchedulerNode node) { if (isOverAMShareLimit()) { if (isWaitingForAMContainer()) { - List<ResourceRequest> ask = appSchedulingInfo.getAllResourceRequests(); - updateAMDiagnosticMsg(ask.get(0).getCapability(), " exceeds maximum " - + "AM resource allowed)."); + PendingAsk amAsk = appSchedulingInfo.getNextPendingAsk(); + updateAMDiagnosticMsg(amAsk.getPerAllocationResource(), + " exceeds maximum AM resource allowed)."); } if (LOG.isDebugEnabled()) { http://git-wip-us.apache.org/repos/asf/hadoop/blob/2977bc6a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSPreemptionThread.java ---------------------------------------------------------------------- diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSPreemptionThread.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSPreemptionThread.java index 3579857..f432484 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSPreemptionThread.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSPreemptionThread.java @@ -26,6 +26,8 @@ import org.apache.hadoop.yarn.api.records.ResourceRequest; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement.SchedulingPlacementSet; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.PendingAsk; import org.apache.hadoop.yarn.util.resource.Resources; import java.util.ArrayList; @@ -90,14 +92,17 @@ class FSPreemptionThread extends Thread { List<RMContainer> containers = new ArrayList<>(); // return value // Find the nodes that match the next resource request - ResourceRequest request = starvedApp.getNextResourceRequest(); + SchedulingPlacementSet nextPs = + starvedApp.getAppSchedulingInfo().getFirstSchedulingPlacementSet(); + PendingAsk firstPendingAsk = nextPs.getPendingAsk(ResourceRequest.ANY); // TODO (KK): Should we check other resource requests if we can't match // the first one? - Resource requestCapability = request.getCapability(); + Resource requestCapability = firstPendingAsk.getPerAllocationResource(); + List<FSSchedulerNode> potentialNodes = scheduler.getNodeTracker().getNodesByResourceName( - request.getResourceName()); + nextPs.getAcceptedResouceNames().next().toString()); // From the potential nodes, pick a node that has enough containers // from apps over their fairshare http://git-wip-us.apache.org/repos/asf/hadoop/blob/2977bc6a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoAppAttempt.java ---------------------------------------------------------------------- diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoAppAttempt.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoAppAttempt.java index fa61710..d932e0e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoAppAttempt.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoAppAttempt.java @@ -51,8 +51,7 @@ public class FifoAppAttempt extends FiCaSchedulerApp { } public RMContainer allocate(NodeType type, FiCaSchedulerNode node, - SchedulerRequestKey schedulerKey, ResourceRequest request, - Container container) { + SchedulerRequestKey schedulerKey, Container container) { try { writeLock.lock(); @@ -62,15 +61,14 @@ public class FifoAppAttempt extends FiCaSchedulerApp { // Required sanity check - AM can call 'allocate' to update resource // request without locking the scheduler, hence we need to check - if (getTotalRequiredResources(schedulerKey) <= 0) { + if (getOutstandingAsksCount(schedulerKey) <= 0) { return null; } // Create RMContainer RMContainer rmContainer = new RMContainerImpl(container, schedulerKey, this.getApplicationAttemptId(), node.getNodeID(), - appSchedulingInfo.getUser(), this.rmContext, - request.getNodeLabelExpression()); + appSchedulingInfo.getUser(), this.rmContext, node.getPartition()); ((RMContainerImpl) rmContainer).setQueueName(this.getQueueName()); updateAMContainerDiagnostics(AMState.ASSIGNED, null); @@ -83,7 +81,7 @@ public class FifoAppAttempt extends FiCaSchedulerApp { // Update consumption and track allocations List<ResourceRequest> resourceRequestList = appSchedulingInfo.allocate( - type, node, schedulerKey, request, container); + type, node, schedulerKey, container); attemptResourceUsage.incUsed(node.getPartition(), container.getResource()); http://git-wip-us.apache.org/repos/asf/hadoop/blob/2977bc6a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java ---------------------------------------------------------------------- diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java index f4ab9c8..d52f538 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java @@ -18,16 +18,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.ConcurrentSkipListMap; - +import com.google.common.annotations.VisibleForTesting; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate; @@ -90,6 +81,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeRemoved import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeResourceUpdateSchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.PendingAsk; import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey; import org.apache.hadoop.yarn.server.utils.BuilderUtils; import org.apache.hadoop.yarn.server.utils.Lock; @@ -97,7 +89,15 @@ import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator; import org.apache.hadoop.yarn.util.resource.ResourceCalculator; import org.apache.hadoop.yarn.util.resource.Resources; -import com.google.common.annotations.VisibleForTesting; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentSkipListMap; @LimitedPrivate("yarn") @Evolving @@ -545,35 +545,32 @@ public class FifoScheduler extends private int getMaxAllocatableContainers(FifoAppAttempt application, SchedulerRequestKey schedulerKey, FiCaSchedulerNode node, NodeType type) { - int maxContainers = 0; - - ResourceRequest offSwitchRequest = - application.getResourceRequest(schedulerKey, ResourceRequest.ANY); - if (offSwitchRequest != null) { - maxContainers = offSwitchRequest.getNumContainers(); - } + PendingAsk offswitchAsk = application.getPendingAsk(schedulerKey, + ResourceRequest.ANY); + int maxContainers = offswitchAsk.getCount(); if (type == NodeType.OFF_SWITCH) { return maxContainers; } if (type == NodeType.RACK_LOCAL) { - ResourceRequest rackLocalRequest = - application.getResourceRequest(schedulerKey, node.getRMNode() - .getRackName()); - if (rackLocalRequest == null) { + PendingAsk rackLocalAsk = application.getPendingAsk(schedulerKey, + node.getRackName()); + if (rackLocalAsk.getCount() <= 0) { return maxContainers; } - maxContainers = Math.min(maxContainers, rackLocalRequest.getNumContainers()); + maxContainers = Math.min(maxContainers, + rackLocalAsk.getCount()); } if (type == NodeType.NODE_LOCAL) { - ResourceRequest nodeLocalRequest = - application.getResourceRequest(schedulerKey, node.getRMNode() - .getNodeAddress()); - if (nodeLocalRequest != null) { - maxContainers = Math.min(maxContainers, nodeLocalRequest.getNumContainers()); + PendingAsk nodeLocalAsk = application.getPendingAsk(schedulerKey, + node.getRMNode().getHostName()); + + if (nodeLocalAsk.getCount() > 0) { + maxContainers = Math.min(maxContainers, + nodeLocalAsk.getCount()); } } @@ -611,25 +608,21 @@ public class FifoScheduler extends private int assignNodeLocalContainers(FiCaSchedulerNode node, FifoAppAttempt application, SchedulerRequestKey schedulerKey) { int assignedContainers = 0; - ResourceRequest request = - application.getResourceRequest(schedulerKey, node.getNodeName()); - if (request != null) { + PendingAsk nodeLocalAsk = application.getPendingAsk(schedulerKey, + node.getNodeName()); + if (nodeLocalAsk.getCount() > 0) { // Don't allocate on this node if we don't need containers on this rack - ResourceRequest rackRequest = - application.getResourceRequest(schedulerKey, - node.getRMNode().getRackName()); - if (rackRequest == null || rackRequest.getNumContainers() <= 0) { + if (application.getOutstandingAsksCount(schedulerKey, + node.getRackName()) <= 0) { return 0; } - - int assignableContainers = - Math.min( - getMaxAllocatableContainers(application, schedulerKey, node, - NodeType.NODE_LOCAL), - request.getNumContainers()); + + int assignableContainers = Math.min( + getMaxAllocatableContainers(application, schedulerKey, node, + NodeType.NODE_LOCAL), nodeLocalAsk.getCount()); assignedContainers = - assignContainer(node, application, schedulerKey, - assignableContainers, request, NodeType.NODE_LOCAL); + assignContainer(node, application, schedulerKey, assignableContainers, + nodeLocalAsk.getPerAllocationResource(), NodeType.NODE_LOCAL); } return assignedContainers; } @@ -637,25 +630,21 @@ public class FifoScheduler extends private int assignRackLocalContainers(FiCaSchedulerNode node, FifoAppAttempt application, SchedulerRequestKey schedulerKey) { int assignedContainers = 0; - ResourceRequest request = - application.getResourceRequest(schedulerKey, node.getRMNode() - .getRackName()); - if (request != null) { + PendingAsk rackAsk = application.getPendingAsk(schedulerKey, + node.getRMNode().getRackName()); + if (rackAsk.getCount() > 0) { // Don't allocate on this rack if the application doens't need containers - ResourceRequest offSwitchRequest = - application.getResourceRequest(schedulerKey, ResourceRequest.ANY); - if (offSwitchRequest.getNumContainers() <= 0) { + if (application.getOutstandingAsksCount(schedulerKey, + ResourceRequest.ANY) <= 0) { return 0; } - - int assignableContainers = - Math.min( - getMaxAllocatableContainers(application, schedulerKey, node, - NodeType.RACK_LOCAL), - request.getNumContainers()); + + int assignableContainers = + Math.min(getMaxAllocatableContainers(application, schedulerKey, node, + NodeType.RACK_LOCAL), rackAsk.getCount()); assignedContainers = - assignContainer(node, application, schedulerKey, - assignableContainers, request, NodeType.RACK_LOCAL); + assignContainer(node, application, schedulerKey, assignableContainers, + rackAsk.getPerAllocationResource(), NodeType.RACK_LOCAL); } return assignedContainers; } @@ -663,26 +652,26 @@ public class FifoScheduler extends private int assignOffSwitchContainers(FiCaSchedulerNode node, FifoAppAttempt application, SchedulerRequestKey schedulerKey) { int assignedContainers = 0; - ResourceRequest request = - application.getResourceRequest(schedulerKey, ResourceRequest.ANY); - if (request != null) { + PendingAsk offswitchAsk = application.getPendingAsk(schedulerKey, + ResourceRequest.ANY); + if (offswitchAsk.getCount() > 0) { assignedContainers = assignContainer(node, application, schedulerKey, - request.getNumContainers(), request, NodeType.OFF_SWITCH); + offswitchAsk.getCount(), + offswitchAsk.getPerAllocationResource(), NodeType.OFF_SWITCH); } return assignedContainers; } private int assignContainer(FiCaSchedulerNode node, FifoAppAttempt application, SchedulerRequestKey schedulerKey, int assignableContainers, - ResourceRequest request, NodeType type) { + Resource capability, NodeType type) { LOG.debug("assignContainers:" + " node=" + node.getRMNode().getNodeAddress() + " application=" + application.getApplicationId().getId() + " priority=" + schedulerKey.getPriority().getPriority() + " assignableContainers=" + assignableContainers + - " request=" + request + " type=" + type); - Resource capability = request.getCapability(); + " capability=" + capability + " type=" + type); // TODO: A buggy application with this zero would crash the scheduler. int availableContainers = @@ -708,7 +697,7 @@ public class FifoScheduler extends // Inform the application RMContainer rmContainer = application.allocate(type, node, schedulerKey, - request, container); + container); // Inform the node node.allocateContainer(rmContainer); http://git-wip-us.apache.org/repos/asf/hadoop/blob/2977bc6a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/LocalitySchedulingPlacementSet.java ---------------------------------------------------------------------- diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/LocalitySchedulingPlacementSet.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/LocalitySchedulingPlacementSet.java index 157518e..c32246d 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/LocalitySchedulingPlacementSet.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/placement/LocalitySchedulingPlacementSet.java @@ -19,12 +19,16 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.placement; import org.apache.commons.collections.IteratorUtils; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; import org.apache.hadoop.yarn.api.records.ResourceRequest; import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AppSchedulingInfo; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.SchedulingMode; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.PendingAsk; import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey; import java.util.ArrayList; @@ -37,9 +41,14 @@ import java.util.concurrent.locks.ReentrantReadWriteLock; public class LocalitySchedulingPlacementSet<N extends SchedulerNode> implements SchedulingPlacementSet<N> { + private static final Log LOG = + LogFactory.getLog(LocalitySchedulingPlacementSet.class); + private final Map<String, ResourceRequest> resourceRequestMap = new ConcurrentHashMap<>(); private AppSchedulingInfo appSchedulingInfo; + private volatile String primaryRequestedPartition = + RMNodeLabelsManager.NO_LABEL; private final ReentrantReadWriteLock.ReadLock readLock; private final ReentrantReadWriteLock.WriteLock writeLock; @@ -132,11 +141,14 @@ public class LocalitySchedulingPlacementSet<N extends SchedulerNode> resourceRequestMap.put(resourceName, request); if (resourceName.equals(ResourceRequest.ANY)) { + String partition = request.getNodeLabelExpression() == null ? + RMNodeLabelsManager.NO_LABEL : + request.getNodeLabelExpression(); + + this.primaryRequestedPartition = partition; + //update the applications requested labels set - appSchedulingInfo.addRequestedPartition( - request.getNodeLabelExpression() == null ? - RMNodeLabelsManager.NO_LABEL : - request.getNodeLabelExpression()); + appSchedulingInfo.addRequestedPartition(partition); updateResult = new ResourceRequestUpdateResult(lastRequest, request); } @@ -152,11 +164,43 @@ public class LocalitySchedulingPlacementSet<N extends SchedulerNode> return resourceRequestMap; } - @Override - public ResourceRequest getResourceRequest(String resourceName) { + private ResourceRequest getResourceRequest(String resourceName) { return resourceRequestMap.get(resourceName); } + @Override + public PendingAsk getPendingAsk(String resourceName) { + try { + readLock.lock(); + ResourceRequest request = getResourceRequest(resourceName); + if (null == request) { + return PendingAsk.ZERO; + } else{ + return new PendingAsk(request.getCapability(), + request.getNumContainers()); + } + } finally { + readLock.unlock(); + } + + } + + @Override + public int getOutstandingAsksCount(String resourceName) { + try { + readLock.lock(); + ResourceRequest request = getResourceRequest(resourceName); + if (null == request) { + return 0; + } else{ + return request.getNumContainers(); + } + } finally { + readLock.unlock(); + } + + } + private void decrementOutstanding(SchedulerRequestKey schedulerRequestKey, ResourceRequest offSwitchRequest) { int numOffSwitchContainers = offSwitchRequest.getNumContainers() - 1; @@ -282,21 +326,66 @@ public class LocalitySchedulingPlacementSet<N extends SchedulerNode> } @Override + public boolean canDelayTo(String resourceName) { + try { + readLock.lock(); + ResourceRequest request = getResourceRequest(resourceName); + return request == null || request.getRelaxLocality(); + } finally { + readLock.unlock(); + } + + } + + @Override + public boolean acceptNodePartition(String nodePartition, + SchedulingMode schedulingMode) { + // We will only look at node label = nodeLabelToLookAt according to + // schedulingMode and partition of node. + String nodePartitionToLookAt; + if (schedulingMode == SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY) { + nodePartitionToLookAt = nodePartition; + } else { + nodePartitionToLookAt = RMNodeLabelsManager.NO_LABEL; + } + + return primaryRequestedPartition.equals(nodePartitionToLookAt); + } + + @Override + public String getPrimaryRequestedNodePartition() { + return primaryRequestedPartition; + } + + @Override + public int getUniqueLocationAsks() { + return resourceRequestMap.size(); + } + + @Override + public void showRequests() { + for (ResourceRequest request : resourceRequestMap.values()) { + if (request.getNumContainers() > 0) { + LOG.debug("\tRequest=" + request); + } + } + } + + @Override public List<ResourceRequest> allocate(SchedulerRequestKey schedulerKey, - NodeType type, SchedulerNode node, ResourceRequest request) { + NodeType type, SchedulerNode node) { try { writeLock.lock(); List<ResourceRequest> resourceRequests = new ArrayList<>(); - if (null == request) { - if (type == NodeType.NODE_LOCAL) { - request = resourceRequestMap.get(node.getNodeName()); - } else if (type == NodeType.RACK_LOCAL) { - request = resourceRequestMap.get(node.getRackName()); - } else{ - request = resourceRequestMap.get(ResourceRequest.ANY); - } + ResourceRequest request; + if (type == NodeType.NODE_LOCAL) { + request = resourceRequestMap.get(node.getNodeName()); + } else if (type == NodeType.RACK_LOCAL) { + request = resourceRequestMap.get(node.getRackName()); + } else{ + request = resourceRequestMap.get(ResourceRequest.ANY); } if (type == NodeType.NODE_LOCAL) { @@ -312,4 +401,14 @@ public class LocalitySchedulingPlacementSet<N extends SchedulerNode> writeLock.unlock(); } } + + @Override + public Iterator<String> getAcceptedResouceNames() { + try { + readLock.lock(); + return resourceRequestMap.keySet().iterator(); + } finally { + readLock.unlock(); + } + } } --------------------------------------------------------------------- To unsubscribe, e-mail: common-commits-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-commits-h...@hadoop.apache.org