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/ResourceLimits.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ResourceLimits.java index c545e9e..721eb36 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ResourceLimits.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ResourceLimits.java @@ -38,6 +38,8 @@ // containers. private volatile Resource headroom; + private boolean allowPreempt = false; + public ResourceLimits(Resource limit) { this(limit, Resources.none()); } @@ -72,4 +74,11 @@ public void setAmountNeededUnreserve(Resource amountNeededUnreserve) { this.amountNeededUnreserve = amountNeededUnreserve; } + public boolean isAllowPreemption() { + return allowPreempt; + } + + public void setIsAllowPreemption(boolean allowPreempt) { + this.allowPreempt = allowPreempt; + } } 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/SchedulerNode.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java index 03edd40..5261c20 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java @@ -66,7 +66,7 @@ /* set of containers that are allocated containers */ - private final Map launchedContainers = + protected final Map launchedContainers = new HashMap(); private final RMNode rmNode; @@ -163,7 +163,7 @@ public synchronized void allocateContainer(RMContainer rmContainer) { + " available after allocation"); } - private synchronized void changeContainerResource(ContainerId containerId, + protected synchronized void changeContainerResource(ContainerId containerId, Resource deltaResource, boolean increase) { if (increase) { deductAvailableResource(deltaResource); @@ -228,7 +228,7 @@ public synchronized boolean isValidContainer(ContainerId containerId) { return false; } - private synchronized void updateResource(Container container) { + protected synchronized void updateResourceOfCompletedContainer(Container container) { addAvailableResource(container.getResource()); --numContainers; } @@ -247,7 +247,7 @@ public synchronized void releaseContainer(Container container) { /* remove the containers from the nodemanger */ if (null != launchedContainers.remove(container.getId())) { - updateResource(container); + updateResourceOfCompletedContainer(container); } LOG.info("Released container " + container.getId() + " of capacity " 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/AbstractCSQueue.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/AbstractCSQueue.java index acd7ae9..2726fd4 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/AbstractCSQueue.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/AbstractCSQueue.java @@ -20,6 +20,7 @@ import java.io.IOException; import java.util.HashMap; +import java.util.Iterator; import java.util.Map; import java.util.Set; @@ -43,10 +44,14 @@ import org.apache.hadoop.yarn.security.PrivilegedEntity.EntityType; import org.apache.hadoop.yarn.security.YarnAuthorizationProvider; import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager; +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.ResourceLimits; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceUsage; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode; import org.apache.hadoop.yarn.util.resource.ResourceCalculator; import org.apache.hadoop.yarn.util.resource.Resources; @@ -470,9 +475,15 @@ synchronized boolean canAssignToThisQueue(Resource clusterResource, Resource nowTotalUsed = queueUsage.getUsed(nodePartition); - // Set headroom for currentResourceLimits - currentResourceLimits.setHeadroom(Resources.subtract(currentLimitResource, - nowTotalUsed)); + // Set headroom for currentResourceLimits: + // When queue is a parent queue: Headroom = limit - used + killable + // When queue is a leaf queue: Headroom = limit - used (leaf queue cannot preempt itself) + Resource used = nowTotalUsed; + if (null != getChildQueues() || !getChildQueues().isEmpty()) { + used = Resources.subtract(used, getTotalKillableResource(nodePartition)); + } + currentResourceLimits.setHeadroom( + Resources.subtract(currentLimitResource, used)); if (Resources.greaterThanOrEqual(resourceCalculator, clusterResource, nowTotalUsed, currentLimitResource)) { @@ -609,4 +620,14 @@ public Priority getDefaultApplicationPriority() { // TODO add dummy implementation return null; } + + public Resource getTotalKillableResource(String partition) { + return csContext.getPreemptionManager().getKillableResource(queueName, + partition); + } + + public Iterator getKillableContainers(String partition) { + return csContext.getPreemptionManager().getKillableContainers(queueName, + partition); + } } 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/CSAssignment.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/CSAssignment.java index 68f6f12..5352e68 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/CSAssignment.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/CSAssignment.java @@ -26,6 +26,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp; import org.apache.hadoop.yarn.util.resource.Resources; +import java.util.List; + @Private @Unstable public class CSAssignment { @@ -42,6 +44,7 @@ private boolean fulfilledReservation; private final AssignmentInformation assignmentInformation; private boolean increaseAllocation; + private List toKillContainers; public CSAssignment(Resource resource, NodeType type) { this(resource, type, null, null, false, false); @@ -147,4 +150,12 @@ public boolean isIncreasedAllocation() { public void setIncreasedAllocation(boolean flag) { increaseAllocation = flag; } + + public void setToKillContainers(List toKillContainers) { + this.toKillContainers = toKillContainers; + } + + public List getToKillContainers() { + return this.getToKillContainers(); + } } \ No newline at end of file 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/CapacityScheduler.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/CapacityScheduler.java index 159c7a5..7184f6f 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/CapacityScheduler.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/CapacityScheduler.java @@ -105,6 +105,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerDynamicEditException; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerHealth; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.KillableContainer; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.AssignmentInformation; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.QueueEntitlement; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp; @@ -1617,13 +1618,29 @@ public void preemptContainer(ApplicationAttemptId aid, RMContainer cont) { } @Override - public void killContainer(RMContainer cont) { + public synchronized void killContainer(RMContainer cont) { if (LOG.isDebugEnabled()) { LOG.debug("KILL_CONTAINER: container" + cont.toString()); } - completedContainer(cont, SchedulerUtils.createPreemptedContainerStatus( - cont.getContainerId(), SchedulerUtils.PREEMPTED_CONTAINER), - RMContainerEventType.KILL); + if (!conf.getLazyPreemptionEnabled()) { + completedContainer(cont, SchedulerUtils + .createPreemptedContainerStatus(cont.getContainerId(), + SchedulerUtils.PREEMPTED_CONTAINER), RMContainerEventType.KILL); + } else { + FiCaSchedulerNode node = (FiCaSchedulerNode) getSchedulerNode( + cont.getAllocatedNode()); + node.markContainerCanBeKilled(cont.getContainerId()); + + // notify PreemptionManager + // Get the application for the finished container + FiCaSchedulerApp application = + getCurrentAttemptForContainer(cont.getContainerId()); + if (null != application) { + String leafQueueName = application.getCSLeafQueue().getQueueName(); + getPreemptionManager().addKillableContainer( + new KillableContainer(cont, node.getPartition(), leafQueueName)); + } + } } @Override 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/CapacitySchedulerConfiguration.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/CapacitySchedulerConfiguration.java index 1e62b44..f5addf3 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/CapacitySchedulerConfiguration.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/CapacitySchedulerConfiguration.java @@ -255,6 +255,12 @@ public static final String RESERVATION_ENFORCEMENT_WINDOW = "reservation-enforcement-window"; + @Private + public static final String LAZY_PREEMPTION_ENALBED = PREFIX + "lazy-preemption-enabled"; + + @Private + public static final boolean DEFAULT_LAZY_PREEMPTION_ENABLED = false; + public CapacitySchedulerConfiguration() { this(new Configuration()); } @@ -959,4 +965,8 @@ public Integer getDefaultApplicationPriorityConfPerQueue(String queue) { DEFAULT_CONFIGURATION_APPLICATION_PRIORITY); return defaultPriority; } + + public boolean getLazyPreemptionEnabled() { + return getBoolean(LAZY_PREEMPTION_ENALBED, DEFAULT_LAZY_PREEMPTION_ENABLED); + } } 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/CapacitySchedulerContext.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/CapacitySchedulerContext.java index 2a0dd0da..3b3e4e0 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/CapacitySchedulerContext.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/CapacitySchedulerContext.java @@ -21,9 +21,11 @@ import java.util.Comparator; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.server.resourcemanager.RMContext; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.PreemptionManager; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode; import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager; @@ -61,4 +63,8 @@ PartitionedQueueComparator getPartitionedQueueComparator(); FiCaSchedulerNode getNode(NodeId nodeId); + + FiCaSchedulerApp getApplicationAttempt(ApplicationAttemptId attemptId); + + PreemptionManager getPreemptionManager(); } 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/LeafQueue.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/LeafQueue.java index 5c3f4b9..08a7440 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/LeafQueue.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/LeafQueue.java @@ -62,6 +62,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt.AMState; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.KillableContainer; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy.OrderingPolicy; @@ -797,6 +798,37 @@ private void handleExcessReservedContainer(Resource clusterResource, assignment.setExcessReservation(null); } } + + private void killToPreemptContainers(Resource clusterResource, + FiCaSchedulerApp application, FiCaSchedulerNode node, + CSAssignment assignment) { + if (assignment.getToKillContainers() != null) { + StringBuilder sb = new StringBuilder("Killing containers: ["); + + for (RMContainer c : assignment.getToKillContainers()) { + completedContainer(clusterResource, application, node, c, SchedulerUtils + .createPreemptedContainerStatus(c.getContainerId(), + SchedulerUtils.PREEMPTED_CONTAINER), RMContainerEventType.KILL, + null, false); + sb.append("(container=" + c.getContainerId() + " resource=" + c + .getAllocatedResource() + ")"); + } + + sb.append("] for container=" + assignment.getAssignmentInformation() + .getFirstAllocatedOrReservedContainerId() + " resource=" + assignment + .getResource()); + LOG.info(sb.toString()); + + } + } + + private void setPreemptionAllowed(ResourceLimits limits, String nodePartition) { + // Set preemption-allowed: + // For leaf queue, only under-utilized queue is allowed to preempt resources from other queues + float usedCapacity = queueCapacities.getAbsoluteUsedCapacity(nodePartition); + float guaranteedCapacity = queueCapacities.getAbsoluteCapacity(nodePartition); + limits.setIsAllowPreemption(usedCapacity < guaranteedCapacity); + } @Override public synchronized CSAssignment assignContainers(Resource clusterResource, @@ -809,6 +841,8 @@ public synchronized CSAssignment assignContainers(Resource clusterResource, + " #applications=" + orderingPolicy.getNumSchedulableEntities()); } + setPreemptionAllowed(currentResourceLimits, node.getPartition()); + // Check for reserved resources RMContainer reservedContainer = node.getReservedContainer(); if (reservedContainer != null) { @@ -820,6 +854,7 @@ public synchronized CSAssignment assignContainers(Resource clusterResource, currentResourceLimits, schedulingMode, reservedContainer); handleExcessReservedContainer(clusterResource, assignment, node, application); + killToPreemptContainers(clusterResource, application, node, assignment); return assignment; } } @@ -881,6 +916,7 @@ public synchronized CSAssignment assignContainers(Resource clusterResource, handleExcessReservedContainer(clusterResource, assignment, node, application); + killToPreemptContainers(clusterResource, application, node, assignment); if (Resources.greaterThan(resourceCalculator, clusterResource, assigned, Resources.none())) { @@ -1237,6 +1273,10 @@ public void completedContainer(Resource clusterResource, rmContainer, null, event, this, sortQueues); } } + + // Notify PreemptionManager + csContext.getPreemptionManager().removeKillableContainer( + new KillableContainer(rmContainer, node.getPartition(), queueName)); } synchronized void allocateResource(Resource clusterResource, 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/ParentQueue.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/ParentQueue.java index badab72..0eb58b5 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/ParentQueue.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/ParentQueue.java @@ -18,18 +18,6 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.Comparator; -import java.util.HashMap; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.TreeSet; - import org.apache.commons.lang.StringUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -56,12 +44,25 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManager; 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.SchedulerApplicationAttempt; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedContainerChangeRequest; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode; import org.apache.hadoop.yarn.util.resource.Resources; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.TreeSet; + @Private @Evolving public class ParentQueue extends AbstractCSQueue { @@ -493,9 +494,22 @@ public synchronized CSAssignment assignContainers(Resource clusterResource, } private boolean canAssign(Resource clusterResource, FiCaSchedulerNode node) { - return (node.getReservedContainer() == null) && - Resources.greaterThanOrEqual(resourceCalculator, clusterResource, - node.getAvailableResource(), minimumAllocation); + if (node.getReservedContainer() == null) { + Resource totalUsable = node.getAvailableResource(); + if (this.getQueueCapacities().getUsedCapacity(node.getPartition()) + < 1.0f) { + // Only consider killable resources when I'm an under-satisified queue. + totalUsable = + Resources.add(totalUsable, node.getTotalKillableResources()); + } + + return Resources.greaterThanOrEqual(resourceCalculator, + clusterResource, + totalUsable, + minimumAllocation); + } + + return false; } private ResourceLimits getResourceLimitsOfChild(CSQueue child, @@ -800,4 +814,55 @@ public void detachContainer(Resource clusterResource, public synchronized int getNumApplications() { return numApplications; } + + @Override + synchronized void allocateResource(Resource clusterResource, + Resource resource, String nodePartition, + boolean changeContainerResource) { + super.allocateResource(clusterResource, resource, nodePartition, + changeContainerResource); + + // check if we need to kill (killable) containers if maximum resource violated. + if (getQueueCapacities().getAbsoluteMaximumCapacity(nodePartition) + < getQueueCapacities().getAbsoluteUsedCapacity(nodePartition)) { + killContainersToEnforceMaxQueueCapacity(nodePartition, clusterResource); + } + } + + private void killContainersToEnforceMaxQueueCapacity(String partition, + Resource clusterResource) { + Iterator killableContainerIter = getKillableContainers( + partition); + if (!killableContainerIter.hasNext()) { + return; + } + + Resource partitionResource = labelManager.getResourceByLabel(partition, + null); + Resource maxResource = Resources.multiply(partitionResource, + getQueueCapacities().getAbsoluteMaximumCapacity(partition)); + + while (Resources.greaterThan(resourceCalculator, partitionResource, + queueUsage.getUsed(partition), maxResource)) { + RMContainer toKillContainer = killableContainerIter.next(); + FiCaSchedulerApp attempt = csContext.getApplicationAttempt( + toKillContainer.getContainerId().getApplicationAttemptId()); + FiCaSchedulerNode node = csContext.getNode( + toKillContainer.getAllocatedNode()); + LeafQueue lq = attempt.getCSLeafQueue(); + if (null != attempt && null != node) { + lq.completedContainer(clusterResource, attempt, node, toKillContainer, + SchedulerUtils.createPreemptedContainerStatus(toKillContainer.getContainerId(), + SchedulerUtils.PREEMPTED_CONTAINER), RMContainerEventType.KILL, + null, false); + } + + LOG.info("Killed container=" + toKillContainer.getContainerId() + + " from queue=" + lq.getQueueName() + " to make queue=" + this + .getQueueName() + "'s max-capacity enforced"); + if (!killableContainerIter.hasNext()) { + break; + } + } + } } \ No newline at end of file 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 ee01bd1..584f5bc 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 @@ -108,6 +108,8 @@ protected CSAssignment getCSAssignmentFromAllocateResult( assignment.setFulfilledReservation(true); } } + + assignment.setToKillContainers(result.getToKillContainers()); } return assignment; 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/ContainerAllocation.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/ContainerAllocation.java index 1df9410..8f749f6 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/ContainerAllocation.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/ContainerAllocation.java @@ -19,11 +19,14 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.allocator; import org.apache.hadoop.yarn.api.records.Container; +import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType; import org.apache.hadoop.yarn.util.resource.Resources; +import java.util.List; + public class ContainerAllocation { /** * Skip the locality (e.g. node-local, rack-local, any), and look at other @@ -56,6 +59,7 @@ NodeType containerNodeType = NodeType.NODE_LOCAL; NodeType requestNodeType = NodeType.NODE_LOCAL; Container updatedContainer; + private List toKillContainers; public ContainerAllocation(RMContainer containerToBeUnreserved, Resource resourceToBeAllocated, AllocationState state) { @@ -86,4 +90,12 @@ public NodeType getContainerNodeType() { public Container getUpdatedContainer() { return updatedContainer; } + + public void setToKillContainers(List toKillContainers) { + this.toKillContainers = toKillContainers; + } + + public List getToKillContainers() { + return toKillContainers; + } } 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 820cccd..6ae833a 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 @@ -18,6 +18,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.allocator; +import org.apache.commons.collections.ArrayStack; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.yarn.api.records.Container; @@ -42,6 +43,10 @@ import org.apache.hadoop.yarn.util.resource.ResourceCalculator; import org.apache.hadoop.yarn.util.resource.Resources; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + /** * Allocate normal (new) containers, considers locality/label, etc. Using * delayed scheduling mechanism to get better locality allocation. @@ -460,6 +465,29 @@ private ContainerAllocation assignContainer(Resource clusterResource, boolean reservationsContinueLooking = application.getCSLeafQueue().getReservationContinueLooking(); + // Check if we need to kill some containers to allocate this one + List toKillContainers = null; + if (availableContainers == 0) { + Resource availableConsidersKillable = Resources.clone(available); + for (RMContainer killableContainer : node + .getKillableContainers().values()) { + if (null == toKillContainers) { + toKillContainers = new ArrayList<>(); + } + toKillContainers.add(killableContainer); + Resources.addTo(availableConsidersKillable, + killableContainer.getAllocatedResource()); + if (Resources.fitsIn(rc, + clusterResource, + capability, + availableConsidersKillable)) { + // Stop if we find enough spaces + availableContainers = 1; + break; + } + } + } + if (availableContainers > 0) { // Allocate... // We will only do continuous reservation when this is not allocated from @@ -499,6 +527,7 @@ private ContainerAllocation assignContainer(Resource clusterResource, new ContainerAllocation(unreservedContainer, request.getCapability(), AllocationState.ALLOCATED); result.containerNodeType = type; + result.setToKillContainers(toKillContainers); return result; } else { // if we are allowed to allocate but this node doesn't have space, reserve @@ -522,6 +551,7 @@ private ContainerAllocation assignContainer(Resource clusterResource, new ContainerAllocation(null, request.getCapability(), AllocationState.RESERVED); result.containerNodeType = type; + result.setToKillContainers(toKillContainers); return result; } // Skip the locality request @@ -613,8 +643,7 @@ private ContainerAllocation handleNewContainerAllocation( } ContainerAllocation doAllocation(ContainerAllocation allocationResult, - Resource clusterResource, FiCaSchedulerNode node, - SchedulingMode schedulingMode, Priority priority, + FiCaSchedulerNode node, Priority priority, RMContainer reservedContainer) { // Create the container if necessary Container container = @@ -678,9 +707,7 @@ private ContainerAllocation allocate(Resource clusterResource, if (AllocationState.ALLOCATED == result.state || AllocationState.RESERVED == result.state) { - result = - doAllocation(result, clusterResource, node, schedulingMode, priority, - reservedContainer); + result = doAllocation(result, node, priority, reservedContainer); } return result; 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/preemption/KillableContainer.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/preemption/KillableContainer.java new file mode 100644 index 0000000..675b0b4 --- /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/capacity/preemption/KillableContainer.java @@ -0,0 +1,45 @@ +/** + * 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.capacity.preemption; + +import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer; + +public class KillableContainer { + RMContainer container; + String partition; + String leafQueueName; + + public KillableContainer(RMContainer container, String partition, String leafQueueName) { + this.container = container; + this.partition = partition; + this.leafQueueName = leafQueueName; + } + + public RMContainer getRMContainer() { + return this.container; + } + + public String getNodePartition() { + return this.partition; + } + + public String getLeafQueueName() { + return this.leafQueueName; + } +} 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/preemption/PreemptionManager.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/preemption/PreemptionManager.java new file mode 100644 index 0000000..c33d8ae --- /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/capacity/preemption/PreemptionManager.java @@ -0,0 +1,196 @@ +/** + * 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.capacity.preemption; + +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueue; +import org.apache.hadoop.yarn.util.resource.Resources; + +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentSkipListMap; +import java.util.concurrent.locks.ReentrantReadWriteLock; + +public class PreemptionManager { + private ReentrantReadWriteLock.ReadLock readLock; + private ReentrantReadWriteLock.WriteLock writeLock; + private Map entities = new HashMap<>(); + private Set killableContainers = new HashSet<>(); + + private static class PreemptableEntity { + // Partition to killable resources and containers + Map totalKillableResources = new HashMap<>(); + Map> killableContainers = + new HashMap<>(); + PreemptableEntity parent; + + public PreemptableEntity(PreemptableEntity parent) { + this.parent = parent; + } + + public void addKillableContainer(KillableContainer container) { + String partition = container.getNodePartition(); + if (!totalKillableResources.containsKey(partition)) { + totalKillableResources.put(partition, Resources.createResource(0)); + killableContainers.put(partition, + new ConcurrentSkipListMap()); + } + + RMContainer c = container.getRMContainer(); + Resources.addTo(totalKillableResources.get(partition), + c.getAllocatedResource()); + killableContainers.get(partition).put(c.getContainerId(), c); + + if (null != parent) { + parent.addKillableContainer(container); + } + } + + public void removeKillableContainer(KillableContainer container) { + String partition = container.getNodePartition(); + RMContainer c = container.getRMContainer(); + + Resources.subtractFrom(totalKillableResources.get(partition), + c.getAllocatedResource()); + killableContainers.get(partition).remove(c.getContainerId()); + + if (null != partition) { + parent.removeKillableContainer(container); + } + } + } + + public PreemptionManager() { + ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); + readLock = lock.readLock(); + writeLock = lock.writeLock(); + } + + public void refreshQueues(CSQueue parent, CSQueue current) { + try { + writeLock.lock(); + PreemptableEntity parentEntity = null; + if (parent != null) { + parentEntity = entities.get(parent.getQueueName()); + } + + if (!entities.containsKey(current.getQueueName())) { + entities.put(current.getQueueName(), + new PreemptableEntity(parentEntity)); + } + + if (current.getChildQueues() != null) { + for (CSQueue child : current.getChildQueues()) { + refreshQueues(current, child); + } + } + } + finally { + writeLock.unlock(); + } + } + + public void addKillableContainer(KillableContainer container) { + try { + writeLock.lock(); + ContainerId cId = container.getRMContainer().getContainerId(); + if (!killableContainers.contains(cId)) { + PreemptableEntity entity = entities.get(container.leafQueueName); + if (null != entity) { + entity.addKillableContainer(container); + } + killableContainers.add(cId); + } + } + finally { + writeLock.unlock(); + ; + } + } + + public void removeKillableContainer(KillableContainer container) { + try { + writeLock.lock(); + ContainerId cId = container.getRMContainer().getContainerId(); + if (killableContainers.contains(cId)) { + PreemptableEntity entity = entities.get(container.leafQueueName); + if (null != entity) { + entity.removeKillableContainer(container); + } + } + } + finally { + writeLock.unlock(); + } + } + + public void moveKillableContainer(KillableContainer oldContainer, + KillableContainer newContainer) { + // TODO + } + + public void updateKillableContainerResource(KillableContainer container, + Resource oldResource, Resource newResource) { + // TODO + } + + public Iterator getKillableContainers(String queueName, + String partition) { + try { + readLock.lock(); + PreemptableEntity entity = entities.get(queueName); + if (entity != null) { + Map containers = + entity.killableContainers.get(partition); + if (containers != null) { + return containers.values().iterator(); + } + } + return Collections.emptyIterator(); + } + finally { + readLock.unlock(); + } + } + + public Resource getKillableResource(String queueName, String partition) { + try { + readLock.lock(); + PreemptableEntity entity = entities.get(queueName); + if (entity != null) { + Resource res = entity.totalKillableResources.get(partition); + if (res == null || res.equals(Resources.none())) { + return Resources.none(); + } + return Resources.clone(res); + } + return Resources.none(); + } + finally { + readLock.unlock(); + } + } +} 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/FiCaSchedulerNode.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/FiCaSchedulerNode.java index fe6db47..f180cf9 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/FiCaSchedulerNode.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/FiCaSchedulerNode.java @@ -18,22 +18,29 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica; - -import java.util.Set; - import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; +import org.apache.hadoop.yarn.api.records.Container; +import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.Priority; +import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode; +import org.apache.hadoop.yarn.util.resource.Resources; + +import java.util.HashMap; +import java.util.Map; +import java.util.Set; public class FiCaSchedulerNode extends SchedulerNode { private static final Log LOG = LogFactory.getLog(FiCaSchedulerNode.class); + private Map killableContainers = new HashMap<>(); + private Resource totalKillableResources = Resource.newInstance(0, 0); public FiCaSchedulerNode(RMNode node, boolean usePortForNodeName, Set nodeLabels) { @@ -92,7 +99,6 @@ public synchronized void reserveResource( @Override public synchronized void unreserveResource( SchedulerApplicationAttempt application) { - // adding NP checks as this can now be called for preemption if (getReservedContainer() != null && getReservedContainer().getContainer() != null @@ -115,4 +121,44 @@ public synchronized void unreserveResource( } setReservedContainer(null); } + + public synchronized void markContainerCanBeKilled(ContainerId containerId) { + RMContainer c = launchedContainers.get(containerId); + if (c != null && !killableContainers.containsKey(containerId)) { + killableContainers.put(containerId, c); + Resources.addTo(totalKillableResources, c.getAllocatedResource()); + } + } + + @Override + protected synchronized void updateResourceOfCompletedContainer( + Container container) { + super.updateResourceOfCompletedContainer(container); + if (killableContainers.containsKey(container.getId())) { + Resources.subtractFrom(totalKillableResources, container.getResource()); + killableContainers.remove(container.getId()); + } + } + + @Override + protected synchronized void changeContainerResource(ContainerId containerId, + Resource deltaResource, boolean increase) { + super.changeContainerResource(containerId, deltaResource, increase); + + if (killableContainers.containsKey(containerId)) { + if (increase) { + Resources.addTo(totalKillableResources, deltaResource); + } else { + Resources.subtractFrom(totalKillableResources, deltaResource); + } + } + } + + public synchronized Resource getTotalKillableResources() { + return totalKillableResources; + } + + public synchronized Map getKillableContainers() { + return killableContainers; + } }