diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/conf/capacity-scheduler.xml hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/conf/capacity-scheduler.xml
index 47db01f..785ed04 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/conf/capacity-scheduler.xml
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/conf/capacity-scheduler.xml
@@ -111,9 +111,27 @@
40
Number of missed scheduling opportunities after which the CapacityScheduler
- attempts to schedule rack-local containers.
- Typically this should be set to number of nodes in the cluster, By default is setting
- approximately number of nodes in one rack which is 40.
+ attempts to schedule rack-local containers.
+ When setting this parameter, the size of the cluster should be taken into account.
+ We use 40 as the default value, which is approximately the number of nodes in one rack.
+
+
+
+
+ yarn.scheduler.capacity.rack-locality-additional-delay
+ -1
+
+ Number of additional missed scheduling opportunities over the node-locality-delay
+ ones, after which the CapacityScheduler attempts to schedule off-switch containers,
+ instead of rack-local ones.
+ Example: with node-locality-delay=40 and rack-locality-delay=20, the scheduler will
+ attempt rack-local assignments after 40 missed opportunities, and off-switch assignments
+ after 40+20=60 missed opportunities.
+ When setting this parameter, the size of the cluster should be taken into account.
+ We use -1 as the default value, which disables this feature. In this case, the number
+ of missed opportunities for assigning off-switch containers is calculated based on
+ the number of containers and unique locations specified in the resource request,
+ as well as the size of the cluster.
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java 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 91e29d5..f12f5fc 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
@@ -33,7 +33,6 @@
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.locks.ReentrantReadWriteLock;
-import com.google.common.collect.ConcurrentHashMultiset;
import org.apache.commons.lang.StringUtils;
import org.apache.commons.lang.time.DateUtils;
import org.apache.commons.lang.time.FastDateFormat;
@@ -73,14 +72,11 @@
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerUpdatesAcquiredEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeCleanContainerEvent;
-
-import org.apache.hadoop.yarn.server.resourcemanager.rmnode
- .RMNodeDecreaseContainerEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeDecreaseContainerEvent;
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.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;
@@ -89,6 +85,7 @@
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
+import com.google.common.collect.ConcurrentHashMultiset;
/**
* Represents an application attempt from the viewpoint of the scheduler.
@@ -1304,6 +1301,11 @@ protected void setAttemptRecovering(boolean isRecovering) {
return appSchedulingInfo.getSchedulingPlacementSet(schedulerRequestKey);
}
+ public Map getResourceRequests(
+ SchedulerRequestKey schedulerRequestKey) {
+ return appSchedulingInfo.getSchedulingPlacementSet(schedulerRequestKey)
+ .getResourceRequests();
+ }
public void incUnconfirmedRes(Resource res) {
unconfirmedAllocatedMem.addAndGet(res.getMemorySize());
diff --git 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 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 43ec390..2a22693 100644
--- 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
+++ 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
@@ -18,8 +18,18 @@
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.collect.ImmutableSet;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.StringTokenizer;
+
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience.Private;
@@ -51,17 +61,8 @@
import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
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.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.StringTokenizer;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.ImmutableSet;
public class CapacitySchedulerConfiguration extends ReservationSchedulerConfiguration {
@@ -198,6 +199,13 @@
public static final int DEFAULT_NODE_LOCALITY_DELAY = 40;
@Private
+ public static final String RACK_LOCALITY_ADDITIONAL_DELAY =
+ PREFIX + "rack-locality-additional-delay";
+
+ @Private
+ public static final int DEFAULT_RACK_LOCALITY_ADDITIONAL_DELAY = -1;
+
+ @Private
public static final String RACK_LOCALITY_FULL_RESET =
PREFIX + "rack-locality-full-reset";
@@ -829,6 +837,11 @@ public int getNodeLocalityDelay() {
return getInt(NODE_LOCALITY_DELAY, DEFAULT_NODE_LOCALITY_DELAY);
}
+ public int getRackLocalityAdditionalDelay() {
+ return getInt(RACK_LOCALITY_ADDITIONAL_DELAY,
+ DEFAULT_RACK_LOCALITY_ADDITIONAL_DELAY);
+ }
+
public boolean getRackLocalityFullReset() {
return getBoolean(RACK_LOCALITY_FULL_RESET,
DEFAULT_RACK_LOCALITY_FULL_RESET);
diff --git 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 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 1b20556..b993e14 100644
--- 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
+++ 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
@@ -19,7 +19,16 @@
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
import java.io.IOException;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
import java.util.concurrent.ConcurrentHashMap;
import org.apache.commons.lang.StringUtils;
@@ -43,26 +52,25 @@
import org.apache.hadoop.yarn.api.records.QueueState;
import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.exceptions.InvalidResourceRequestException;
import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
import org.apache.hadoop.yarn.security.AccessType;
-import org.apache.hadoop.yarn.server.resourcemanager.RMServerUtils;
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.rmcontainer.RMContainerState;
-
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractUsersManager;
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.SchedContainerChangeRequest;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplication;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.*;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.ActivityDiagnosticConstant;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt.AMState;
+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.activities.ActivitiesLogger;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.ActivityDiagnosticConstant;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.ActivityState;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicationAttempt.AMState;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.UsersManager.User;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.KillableContainer;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ContainerAllocationProposal;
@@ -95,6 +103,7 @@
private float maxAMResourcePerQueuePercent;
private volatile int nodeLocalityDelay;
+ private volatile int rackLocalityAdditionalDelay;
private volatile boolean rackLocalityFullReset;
Map applicationAttemptMap =
@@ -215,6 +224,7 @@ protected void setupQueueConfigs(Resource clusterResource)
}
nodeLocalityDelay = conf.getNodeLocalityDelay();
+ rackLocalityAdditionalDelay = conf.getRackLocalityAdditionalDelay();
rackLocalityFullReset = conf.getRackLocalityFullReset();
// re-init this since max allocation could have changed
@@ -271,9 +281,12 @@ protected void setupQueueConfigs(Resource clusterResource)
+ "numContainers = " + numContainers
+ " [= currentNumContainers ]" + "\n" + "state = " + getState()
+ " [= configuredState ]" + "\n" + "acls = " + aclsString
- + " [= configuredAcls ]" + "\n" + "nodeLocalityDelay = "
- + nodeLocalityDelay + "\n" + "labels=" + labelStrBuilder
- .toString() + "\n" + "reservationsContinueLooking = "
+ + " [= configuredAcls ]" + "\n"
+ + "nodeLocalityDelay = " + nodeLocalityDelay + "\n"
+ + "rackLocalityAdditionalDelay = "
+ + rackLocalityAdditionalDelay + "\n"
+ + "labels=" + labelStrBuilder.toString() + "\n"
+ + "reservationsContinueLooking = "
+ reservationsContinueLooking + "\n" + "preemptionDisabled = "
+ getPreemptionDisabled() + "\n" + "defaultAppPriorityPerQueue = "
+ defaultAppPriorityPerQueue + "\npriority = " + priority);
@@ -1347,6 +1360,11 @@ public int getNodeLocalityDelay() {
}
@Lock(NoLock.class)
+ public int getRackLocalityAdditionalDelay() {
+ return rackLocalityAdditionalDelay;
+ }
+
+ @Lock(NoLock.class)
public boolean getRackLocalityFullReset() {
return rackLocalityFullReset;
}
diff --git 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 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 8078bcd..a0c2249 100644
--- 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
+++ 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
@@ -33,27 +33,24 @@
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
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.RMContainerImpl;
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.activities.ActivityDiagnosticConstant;
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;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.activities.ActivityState;
-
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSAMContainerLaunchDiagnosticsConstants;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSAssignment;
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.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
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.scheduler.SchedulerRequestKey;
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
import org.apache.hadoop.yarn.util.resource.Resources;
@@ -278,6 +275,12 @@ private int getActualNodeLocalityDelay() {
.getCSLeafQueue().getNodeLocalityDelay());
}
+ private int getActualRackLocalityDelay() {
+ return Math.min(rmContext.getScheduler().getNumClusterNodes(),
+ application.getCSLeafQueue().getNodeLocalityDelay()
+ + application.getCSLeafQueue().getRackLocalityAdditionalDelay());
+ }
+
private boolean canAssign(SchedulerRequestKey schedulerKey,
FiCaSchedulerNode node, NodeType type, RMContainer reservedContainer) {
@@ -287,25 +290,34 @@ private boolean canAssign(SchedulerRequestKey schedulerKey,
return true;
}
+ // If we have only ANY requests for this schedulerKey, we should not
+ // delay its scheduling.
+ if (application.getResourceRequests(schedulerKey).size() == 1
+ && rmContext.getScheduler().getNumClusterNodes() > 0) {
+ return true;
+ }
+
// 'Delay' off-switch
long missedOpportunities =
application.getSchedulingOpportunities(schedulerKey);
- long requiredContainers = application.getOutstandingAsksCount(
- schedulerKey);
- float localityWaitFactor =
- getLocalityWaitFactor(schedulerKey, rmContext.getScheduler()
- .getNumClusterNodes());
- // Cap the delay by the number of nodes in the cluster. Under most
- // conditions this means we will consider each node in the cluster before
- // accepting an off-switch assignment.
- return (Math.min(rmContext.getScheduler().getNumClusterNodes(),
- (requiredContainers * localityWaitFactor)) < missedOpportunities);
+ // If rack locality additional delay parameter is enabled.
+ if (application.getCSLeafQueue().getRackLocalityAdditionalDelay() > -1) {
+ return missedOpportunities > getActualRackLocalityDelay();
+ } else {
+ long requiredContainers =
+ application.getOutstandingAsksCount(schedulerKey);
+ float localityWaitFactor = getLocalityWaitFactor(schedulerKey,
+ rmContext.getScheduler().getNumClusterNodes());
+ // Cap the delay by the number of nodes in the cluster.
+ return (Math.min(rmContext.getScheduler().getNumClusterNodes(),
+ (requiredContainers * localityWaitFactor)) < missedOpportunities);
+ }
}
// Check if we need containers on this rack
- if (application.getOutstandingAsksCount(schedulerKey, node.getRackName())
- <= 0) {
+ if (application.getOutstandingAsksCount(schedulerKey,
+ node.getRackName()) <= 0) {
return false;
}
diff --git hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java
index 1162b9f..1c8fa77 100644
--- hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java
+++ hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java
@@ -18,6 +18,7 @@
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
+import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.TestUtils.toSchedulerKey;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
@@ -41,7 +42,6 @@
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.CyclicBarrier;
-import com.google.common.collect.ImmutableMap;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.security.UserGroupInformation;
@@ -77,7 +77,6 @@
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueStateManager;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceLimits;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceUsage;
-import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.UsersManager.User;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.preemption.PreemptionManager;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ResourceCommitRequest;
@@ -91,6 +90,7 @@
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy.FifoOrderingPolicy;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy.OrderingPolicy;
import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
+import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey;
import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
import org.apache.hadoop.yarn.util.resource.DominantResourceCalculator;
import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
@@ -102,7 +102,7 @@
import org.mockito.Matchers;
import org.mockito.Mockito;
-import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.TestUtils.toSchedulerKey;
+import com.google.common.collect.ImmutableMap;
public class TestLeafQueue {
private final RecordFactory recordFactory =
@@ -2110,6 +2110,154 @@ public void testLocalityScheduling() throws Exception {
}
@Test
+ public void testRackLocalityDelayScheduling() throws Exception {
+
+ // Change parameter values for node locality and rack locality delay.
+ csConf.setInt(CapacitySchedulerConfiguration.NODE_LOCALITY_DELAY, 2);
+ csConf.setInt(
+ CapacitySchedulerConfiguration.RACK_LOCALITY_ADDITIONAL_DELAY, 1);
+ Map newQueues = new HashMap();
+ CSQueue newRoot = CapacitySchedulerQueueManager.parseQueue(csContext,
+ csConf, null, CapacitySchedulerConfiguration.ROOT, newQueues, queues,
+ TestUtils.spyHook);
+ queues = newQueues;
+ root.reinitialize(newRoot, cs.getClusterResource());
+
+ // Manipulate queue 'b'
+ LeafQueue a = stubLeafQueue((LeafQueue) queues.get(B));
+
+ // Check locality parameters.
+ assertEquals(2, a.getNodeLocalityDelay());
+ assertEquals(1, a.getRackLocalityAdditionalDelay());
+
+ // User
+ String user1 = "user_1";
+
+ // Submit applications
+ final ApplicationAttemptId appAttemptId1 =
+ TestUtils.getMockApplicationAttemptId(0, 0);
+ FiCaSchedulerApp app1 = new FiCaSchedulerApp(appAttemptId1, user1, a,
+ mock(ActiveUsersManager.class), spyRMContext);
+ a.submitApplicationAttempt(app1, user1);
+
+ // Setup some nodes and racks
+ String host1 = "127.0.0.1";
+ String host2 = "127.0.0.2";
+ String host3 = "127.0.0.3";
+ String host4 = "127.0.0.4";
+ String rack1 = "rack_1";
+ String rack2 = "rack_2";
+ String rack3 = "rack_3";
+ FiCaSchedulerNode node_2 = TestUtils.getMockNode(host3, rack2, 0, 8 * GB);
+ FiCaSchedulerNode node_3 = TestUtils.getMockNode(host4, rack3, 0, 8 * GB);
+
+ Map apps =
+ ImmutableMap.of(app1.getApplicationAttemptId(), app1);
+ Map nodes =
+ ImmutableMap.of(node_2.getNodeID(), node_2, node_3.getNodeID(), node_3);
+
+ final int numNodes = 5;
+ Resource clusterResource =
+ Resources.createResource(numNodes * (8 * GB), numNodes * 16);
+ when(spyRMContext.getScheduler().getNumClusterNodes()).thenReturn(numNodes);
+
+ // Setup resource-requests and submit
+ Priority priority = TestUtils.createMockPriority(1);
+ List app_0_requests_0 = new ArrayList();
+ app_0_requests_0.add(TestUtils.createResourceRequest(host1, 1 * GB, 1,
+ true, priority, recordFactory));
+ app_0_requests_0.add(TestUtils.createResourceRequest(rack1, 1 * GB, 1,
+ true, priority, recordFactory));
+ app_0_requests_0.add(TestUtils.createResourceRequest(host2, 1 * GB, 1,
+ true, priority, recordFactory));
+ app_0_requests_0.add(TestUtils.createResourceRequest(rack2, 1 * GB, 1,
+ true, priority, recordFactory));
+ // Adding one extra in the ANY.
+ app_0_requests_0.add(TestUtils.createResourceRequest(ResourceRequest.ANY,
+ 1 * GB, 3, true, priority, recordFactory));
+ app1.updateResourceRequests(app_0_requests_0);
+
+ // Start testing...
+ CSAssignment assignment = null;
+
+ SchedulerRequestKey schedulerKey = toSchedulerKey(priority);
+ assertEquals(3, app1.getOutstandingAsksCount(schedulerKey));
+
+ // No rack-local yet.
+ assignment = a.assignContainers(clusterResource, node_2,
+ new ResourceLimits(clusterResource),
+ SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
+ applyCSAssignment(clusterResource, assignment, a, nodes, apps);
+ verifyNoContainerAllocated(assignment);
+ assertEquals(1, app1.getSchedulingOpportunities(schedulerKey));
+ assertEquals(3, app1.getOutstandingAsksCount(schedulerKey));
+ assertEquals(NodeType.NODE_LOCAL, assignment.getType()); // None->NODE_LOCAL
+
+ // Still no rack-local.
+ assignment = a.assignContainers(clusterResource, node_2,
+ new ResourceLimits(clusterResource),
+ SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
+ applyCSAssignment(clusterResource, assignment, a, nodes, apps);
+ assertEquals(2, app1.getSchedulingOpportunities(schedulerKey));
+ assertEquals(3, app1.getOutstandingAsksCount(schedulerKey));
+ assertEquals(NodeType.NODE_LOCAL, assignment.getType()); // None->NODE_LOCAL
+
+ // Rack local now.
+ assignment = a.assignContainers(clusterResource, node_2,
+ new ResourceLimits(clusterResource),
+ SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
+ applyCSAssignment(clusterResource, assignment, a, nodes, apps);
+ assertEquals(0, app1.getSchedulingOpportunities(schedulerKey));
+ assertEquals(2, app1.getOutstandingAsksCount(schedulerKey));
+ assertEquals(NodeType.RACK_LOCAL, assignment.getType());
+
+ // No off-switch until 3 missed opportunities.
+ a.assignContainers(clusterResource, node_3,
+ new ResourceLimits(clusterResource),
+ SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
+ applyCSAssignment(clusterResource, assignment, a, nodes, apps);
+ a.assignContainers(clusterResource, node_3,
+ new ResourceLimits(clusterResource),
+ SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
+ applyCSAssignment(clusterResource, assignment, a, nodes, apps);
+ assignment = a.assignContainers(clusterResource, node_3,
+ new ResourceLimits(clusterResource),
+ SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
+ applyCSAssignment(clusterResource, assignment, a, nodes, apps);
+ assertEquals(3, app1.getSchedulingOpportunities(schedulerKey));
+ assertEquals(2, app1.getOutstandingAsksCount(schedulerKey));
+ assertEquals(NodeType.NODE_LOCAL, assignment.getType()); // None->NODE_LOCAL
+
+ // Now off-switch should succeed.
+ assignment = a.assignContainers(clusterResource, node_3,
+ new ResourceLimits(clusterResource),
+ SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
+ applyCSAssignment(clusterResource, assignment, a, nodes, apps);
+ assertEquals(4, app1.getSchedulingOpportunities(schedulerKey));
+ assertEquals(1, app1.getOutstandingAsksCount(schedulerKey));
+ assertEquals(NodeType.OFF_SWITCH, assignment.getType());
+
+ // Check capping by number of cluster nodes.
+ doReturn(10).when(a).getRackLocalityAdditionalDelay();
+ // Off-switch will happen at 6 missed opportunities now, since cluster size
+ // is 5.
+ assignment = a.assignContainers(clusterResource, node_3,
+ new ResourceLimits(clusterResource),
+ SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
+ applyCSAssignment(clusterResource, assignment, a, nodes, apps);
+ assertEquals(5, app1.getSchedulingOpportunities(schedulerKey));
+ assertEquals(1, app1.getOutstandingAsksCount(schedulerKey));
+ assertEquals(NodeType.NODE_LOCAL, assignment.getType()); // None->NODE_LOCAL
+ assignment = a.assignContainers(clusterResource, node_3,
+ new ResourceLimits(clusterResource),
+ SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
+ applyCSAssignment(clusterResource, assignment, a, nodes, apps);
+ assertEquals(6, app1.getSchedulingOpportunities(schedulerKey));
+ assertEquals(0, app1.getOutstandingAsksCount(schedulerKey));
+ assertEquals(NodeType.OFF_SWITCH, assignment.getType());
+ }
+
+ @Test
public void testApplicationPriorityScheduling() throws Exception {
// Manipulate queue 'a'
LeafQueue a = stubLeafQueue((LeafQueue)queues.get(A));
@@ -2414,16 +2562,18 @@ public void testActivateApplicationAfterQueueRefresh() throws Exception {
}
@Test (timeout = 30000)
- public void testNodeLocalityAfterQueueRefresh() throws Exception {
+ public void testLocalityDelaysAfterQueueRefresh() throws Exception {
// Manipulate queue 'e'
LeafQueue e = stubLeafQueue((LeafQueue)queues.get(E));
// before reinitialization
assertEquals(40, e.getNodeLocalityDelay());
+ assertEquals(-1, e.getRackLocalityAdditionalDelay());
- csConf.setInt(CapacitySchedulerConfiguration
- .NODE_LOCALITY_DELAY, 60);
+ csConf.setInt(CapacitySchedulerConfiguration.NODE_LOCALITY_DELAY, 60);
+ csConf.setInt(
+ CapacitySchedulerConfiguration.RACK_LOCALITY_ADDITIONAL_DELAY, 600);
Map newQueues = new HashMap();
CSQueue newRoot =
CapacitySchedulerQueueManager.parseQueue(csContext, csConf, null,
@@ -2435,6 +2585,7 @@ public void testNodeLocalityAfterQueueRefresh() throws Exception {
// after reinitialization
assertEquals(60, e.getNodeLocalityDelay());
+ assertEquals(600, e.getRackLocalityAdditionalDelay());
}
@Test (timeout = 30000)