|
@@ -84,6 +84,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptA
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptRemovedSchedulerEvent;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppRemovedSchedulerEvent;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.policy.FairOrderingPolicy;
|
|
|
import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
|
|
|
import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
|
|
|
import org.apache.hadoop.yarn.util.resource.Resources;
|
|
@@ -459,6 +460,41 @@ public class TestLeafQueue {
|
|
|
assertEquals(1, userMetrics.getAppsSubmitted());
|
|
|
}
|
|
|
|
|
|
+ @Test
|
|
|
+ public void testFairConfiguration() throws Exception {
|
|
|
+
|
|
|
+ CapacitySchedulerConfiguration testConf =
|
|
|
+ new CapacitySchedulerConfiguration();
|
|
|
+
|
|
|
+ String tproot = CapacitySchedulerConfiguration.ROOT + "." +
|
|
|
+ "testPolicyRoot" + System.currentTimeMillis();
|
|
|
+
|
|
|
+ OrderingPolicy<FiCaSchedulerApp> schedOrder =
|
|
|
+ testConf.<FiCaSchedulerApp>getOrderingPolicy(tproot);
|
|
|
+
|
|
|
+ //override default to fair
|
|
|
+ String policyType = CapacitySchedulerConfiguration.PREFIX + tproot +
|
|
|
+ "." + CapacitySchedulerConfiguration.ORDERING_POLICY;
|
|
|
+
|
|
|
+ testConf.set(policyType,
|
|
|
+ CapacitySchedulerConfiguration.FAIR_ORDERING_POLICY);
|
|
|
+ schedOrder =
|
|
|
+ testConf.<FiCaSchedulerApp>getOrderingPolicy(tproot);
|
|
|
+ FairOrderingPolicy fop = (FairOrderingPolicy<FiCaSchedulerApp>) schedOrder;
|
|
|
+ assertFalse(fop.getSizeBasedWeight());
|
|
|
+
|
|
|
+ //Now with sizeBasedWeight
|
|
|
+ String sbwConfig = CapacitySchedulerConfiguration.PREFIX + tproot +
|
|
|
+ "." + CapacitySchedulerConfiguration.ORDERING_POLICY + "." +
|
|
|
+ FairOrderingPolicy.ENABLE_SIZE_BASED_WEIGHT;
|
|
|
+ testConf.set(sbwConfig, "true");
|
|
|
+ schedOrder =
|
|
|
+ testConf.<FiCaSchedulerApp>getOrderingPolicy(tproot);
|
|
|
+ fop = (FairOrderingPolicy<FiCaSchedulerApp>) schedOrder;
|
|
|
+ assertTrue(fop.getSizeBasedWeight());
|
|
|
+
|
|
|
+ }
|
|
|
+
|
|
|
@Test
|
|
|
public void testSingleQueueWithOneUser() throws Exception {
|
|
|
|
|
@@ -2621,6 +2657,86 @@ public class TestLeafQueue {
|
|
|
|
|
|
}
|
|
|
|
|
|
+ @Test
|
|
|
+ public void testFairAssignment() throws Exception {
|
|
|
+
|
|
|
+ LeafQueue a = stubLeafQueue((LeafQueue)queues.get(A));
|
|
|
+
|
|
|
+ OrderingPolicy<FiCaSchedulerApp> schedulingOrder =
|
|
|
+ new FairOrderingPolicy<FiCaSchedulerApp>();
|
|
|
+
|
|
|
+ a.setOrderingPolicy(schedulingOrder);
|
|
|
+
|
|
|
+ String host_0_0 = "127.0.0.1";
|
|
|
+ String rack_0 = "rack_0";
|
|
|
+ FiCaSchedulerNode node_0_0 = TestUtils.getMockNode(host_0_0, rack_0, 0, 16*GB);
|
|
|
+
|
|
|
+ final int numNodes = 4;
|
|
|
+ Resource clusterResource = Resources.createResource(
|
|
|
+ numNodes * (16*GB), numNodes * 16);
|
|
|
+ when(csContext.getNumClusterNodes()).thenReturn(numNodes);
|
|
|
+
|
|
|
+ String user_0 = "user_0";
|
|
|
+
|
|
|
+ final ApplicationAttemptId appAttemptId_0 =
|
|
|
+ TestUtils.getMockApplicationAttemptId(0, 0);
|
|
|
+ FiCaSchedulerApp app_0 =
|
|
|
+ spy(new FiCaSchedulerApp(appAttemptId_0, user_0, a,
|
|
|
+ mock(ActiveUsersManager.class), spyRMContext));
|
|
|
+ a.submitApplicationAttempt(app_0, user_0);
|
|
|
+
|
|
|
+ final ApplicationAttemptId appAttemptId_1 =
|
|
|
+ TestUtils.getMockApplicationAttemptId(1, 0);
|
|
|
+ FiCaSchedulerApp app_1 =
|
|
|
+ spy(new FiCaSchedulerApp(appAttemptId_1, user_0, a,
|
|
|
+ mock(ActiveUsersManager.class), spyRMContext));
|
|
|
+ a.submitApplicationAttempt(app_1, user_0);
|
|
|
+
|
|
|
+ Priority priority = TestUtils.createMockPriority(1);
|
|
|
+ List<ResourceRequest> app_0_requests_0 = new ArrayList<ResourceRequest>();
|
|
|
+ List<ResourceRequest> app_1_requests_0 = new ArrayList<ResourceRequest>();
|
|
|
+
|
|
|
+ app_0_requests_0.clear();
|
|
|
+ app_0_requests_0.add(
|
|
|
+ TestUtils.createResourceRequest(ResourceRequest.ANY, 2*GB, 1,
|
|
|
+ true, priority, recordFactory));
|
|
|
+ app_0.updateResourceRequests(app_0_requests_0);
|
|
|
+
|
|
|
+ app_1_requests_0.clear();
|
|
|
+ app_1_requests_0.add(
|
|
|
+ TestUtils.createResourceRequest(ResourceRequest.ANY, 1*GB, 1,
|
|
|
+ true, priority, recordFactory));
|
|
|
+ app_1.updateResourceRequests(app_1_requests_0);
|
|
|
+
|
|
|
+ a.assignContainers(clusterResource, node_0_0, new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
|
|
+ Assert.assertEquals(2*GB, app_0.getCurrentConsumption().getMemory());
|
|
|
+ a.assignContainers(clusterResource, node_0_0, new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
|
|
+ Assert.assertEquals(1*GB, app_1.getCurrentConsumption().getMemory());
|
|
|
+
|
|
|
+ app_0_requests_0.clear();
|
|
|
+ app_0_requests_0.add(
|
|
|
+ TestUtils.createResourceRequest(ResourceRequest.ANY, 1*GB, 1,
|
|
|
+ true, priority, recordFactory));
|
|
|
+ app_0.updateResourceRequests(app_0_requests_0);
|
|
|
+
|
|
|
+ app_1_requests_0.clear();
|
|
|
+ app_1_requests_0.add(
|
|
|
+ TestUtils.createResourceRequest(ResourceRequest.ANY, 1*GB, 1,
|
|
|
+ true, priority, recordFactory));
|
|
|
+ app_1.updateResourceRequests(app_1_requests_0);
|
|
|
+
|
|
|
+ //Since it already has more resources, app_0 will not get
|
|
|
+ //assigned first, but app_1 will
|
|
|
+ a.assignContainers(clusterResource, node_0_0, new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
|
|
+ Assert.assertEquals(2*GB, app_0.getCurrentConsumption().getMemory());
|
|
|
+ Assert.assertEquals(2*GB, app_1.getCurrentConsumption().getMemory());
|
|
|
+
|
|
|
+ //and only then will app_0
|
|
|
+ a.assignContainers(clusterResource, node_0_0, new ResourceLimits(clusterResource), SchedulingMode.RESPECT_PARTITION_EXCLUSIVITY);
|
|
|
+ Assert.assertEquals(3*GB, app_0.getCurrentConsumption().getMemory());
|
|
|
+
|
|
|
+ }
|
|
|
+
|
|
|
private List<FiCaSchedulerApp> createListOfApps(int noOfApps, String user,
|
|
|
LeafQueue defaultQueue) {
|
|
|
List<FiCaSchedulerApp> appsLists = new ArrayList<FiCaSchedulerApp>();
|