|
@@ -594,6 +594,143 @@ public class TestFairScheduler extends FairSchedulerTestBase {
|
|
|
assertEquals(0, queue.getFairShare().getMemorySize());
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Test if we compute the maximum AM resource correctly.
|
|
|
+ *
|
|
|
+ * @throws IOException if scheduler reinitialization fails
|
|
|
+ */
|
|
|
+ @Test
|
|
|
+ public void testComputeMaxAMResource() throws IOException {
|
|
|
+ conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
|
|
|
+ PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
|
|
|
+ out.println("<?xml version=\"1.0\"?>");
|
|
|
+ out.println("<allocations>");
|
|
|
+ out.println("<queue name=\"queueFSZeroWithMax\">");
|
|
|
+ out.println("<weight>0</weight>");
|
|
|
+ out.println("<maxAMShare>0.5</maxAMShare>");
|
|
|
+ out.println("<maxResources>4096 mb 4 vcores</maxResources>");
|
|
|
+ out.println("</queue>");
|
|
|
+ out.println("<queue name=\"queueFSZeroWithAVL\">");
|
|
|
+ out.println("<weight>0.0</weight>");
|
|
|
+ out.println("<maxAMShare>0.5</maxAMShare>");
|
|
|
+ out.println("</queue>");
|
|
|
+ out.println("<queue name=\"queueFSNonZero\">");
|
|
|
+ out.println("<weight>1</weight>");
|
|
|
+ out.println("<maxAMShare>0.5</maxAMShare>");
|
|
|
+ out.println("</queue>");
|
|
|
+ out.println("<defaultQueueSchedulingPolicy>drf" +
|
|
|
+ "</defaultQueueSchedulingPolicy>");
|
|
|
+ out.println("</allocations>");
|
|
|
+ out.close();
|
|
|
+
|
|
|
+ scheduler.init(conf);
|
|
|
+ scheduler.start();
|
|
|
+ scheduler.reinitialize(conf, resourceManager.getRMContext());
|
|
|
+
|
|
|
+ long memCapacity = 20 * GB;
|
|
|
+ int cpuCapacity = 20;
|
|
|
+ RMNode node =
|
|
|
+ MockNodes.newNodeInfo(1, Resources.createResource(memCapacity,
|
|
|
+ cpuCapacity), 0, "127.0.0.1");
|
|
|
+ NodeAddedSchedulerEvent nodeEvent = new NodeAddedSchedulerEvent(node);
|
|
|
+ NodeUpdateSchedulerEvent updateEvent = new NodeUpdateSchedulerEvent(node);
|
|
|
+ scheduler.handle(nodeEvent);
|
|
|
+ scheduler.update();
|
|
|
+
|
|
|
+ Resource amResource = Resource.newInstance(1 * GB, 1);
|
|
|
+ int amPriority = RMAppAttemptImpl.AM_CONTAINER_PRIORITY.getPriority();
|
|
|
+
|
|
|
+ // queueFSZeroWithMax
|
|
|
+ FSLeafQueue queueFSZeroWithMax = scheduler.getQueueManager().
|
|
|
+ getLeafQueue("queueFSZeroWithMax", true);
|
|
|
+ ApplicationAttemptId attId1 = createAppAttemptId(1, 1);
|
|
|
+ createApplicationWithAMResource(attId1, "queueFSZeroWithMax", "user1",
|
|
|
+ amResource);
|
|
|
+ createSchedulingRequestExistingApplication(1 * GB, 1, amPriority, attId1);
|
|
|
+ scheduler.update();
|
|
|
+ scheduler.handle(updateEvent);
|
|
|
+
|
|
|
+ // queueFSZeroWithMax's weight is 0.0, so its fair share should be 0, we use
|
|
|
+ // the min(maxShare, available resource) to compute maxAMShare, in this
|
|
|
+ // case, we use maxShare, since it is smaller than available resource.
|
|
|
+ assertEquals("QueueFSZeroWithMax's fair share should be zero",
|
|
|
+ 0, queueFSZeroWithMax.getFairShare().getMemorySize());
|
|
|
+ assertEquals("QueueFSZeroWithMax's maximum AM resource should be "
|
|
|
+ + "maxShare * maxAMShare",
|
|
|
+ (long)(queueFSZeroWithMax.getMaxShare().getMemorySize() *
|
|
|
+ queueFSZeroWithMax.getMaxAMShare()),
|
|
|
+ queueFSZeroWithMax.getMetrics().getMaxAMShareMB());
|
|
|
+ assertEquals("QueueFSZeroWithMax's maximum AM resource should be "
|
|
|
+ + "maxShare * maxAMShare",
|
|
|
+ (long)(queueFSZeroWithMax.getMaxShare().getVirtualCores() *
|
|
|
+ queueFSZeroWithMax.getMaxAMShare()),
|
|
|
+ queueFSZeroWithMax.getMetrics().getMaxAMShareVCores());
|
|
|
+ assertEquals("QueueFSZeroWithMax's AM resource usage should be the same to "
|
|
|
+ + "AM resource request",
|
|
|
+ amResource.getMemorySize(),
|
|
|
+ queueFSZeroWithMax.getMetrics().getAMResourceUsageMB());
|
|
|
+
|
|
|
+ // queueFSZeroWithAVL
|
|
|
+ amResource = Resources.createResource(1 * GB, 1);
|
|
|
+ FSLeafQueue queueFSZeroWithAVL = scheduler.getQueueManager().
|
|
|
+ getLeafQueue("queueFSZeroWithAVL", true);
|
|
|
+ ApplicationAttemptId attId2 = createAppAttemptId(2, 1);
|
|
|
+ createApplicationWithAMResource(attId2, "queueFSZeroWithAVL", "user1",
|
|
|
+ amResource);
|
|
|
+ createSchedulingRequestExistingApplication(1 * GB, 1, amPriority, attId2);
|
|
|
+ scheduler.update();
|
|
|
+ scheduler.handle(updateEvent);
|
|
|
+
|
|
|
+ // queueFSZeroWithAVL's weight is 0.0, so its fair share is 0, and we use
|
|
|
+ // the min(maxShare, available resource) to compute maxAMShare, in this
|
|
|
+ // case, we use available resource since it is smaller than the
|
|
|
+ // default maxShare.
|
|
|
+ assertEquals("QueueFSZeroWithAVL's fair share should be zero",
|
|
|
+ 0, queueFSZeroWithAVL.getFairShare().getMemorySize());
|
|
|
+ assertEquals("QueueFSZeroWithAVL's maximum AM resource should be "
|
|
|
+ + " available resource * maxAMShare",
|
|
|
+ (long) ((memCapacity - amResource.getMemorySize()) *
|
|
|
+ queueFSZeroWithAVL.getMaxAMShare()),
|
|
|
+ queueFSZeroWithAVL.getMetrics().getMaxAMShareMB());
|
|
|
+ assertEquals("QueueFSZeroWithAVL's maximum AM resource should be "
|
|
|
+ + " available resource * maxAMShare",
|
|
|
+ (long) ((cpuCapacity - amResource.getVirtualCores()) *
|
|
|
+ queueFSZeroWithAVL.getMaxAMShare()),
|
|
|
+ queueFSZeroWithAVL.getMetrics().getMaxAMShareVCores());
|
|
|
+ assertEquals("QueueFSZeroWithMax's AM resource usage should be the same to "
|
|
|
+ + "AM resource request",
|
|
|
+ amResource.getMemorySize(),
|
|
|
+ queueFSZeroWithAVL.getMetrics().getAMResourceUsageMB());
|
|
|
+
|
|
|
+ // queueFSNonZero
|
|
|
+ amResource = Resources.createResource(1 * GB, 1);
|
|
|
+ FSLeafQueue queueFSNonZero = scheduler.getQueueManager().
|
|
|
+ getLeafQueue("queueFSNonZero", true);
|
|
|
+ ApplicationAttemptId attId3 = createAppAttemptId(3, 1);
|
|
|
+ createApplicationWithAMResource(attId3, "queueFSNonZero", "user1",
|
|
|
+ amResource);
|
|
|
+ createSchedulingRequestExistingApplication(1 * GB, 1, amPriority, attId3);
|
|
|
+ scheduler.update();
|
|
|
+ scheduler.handle(updateEvent);
|
|
|
+
|
|
|
+ // queueFSNonZero's weight is 1, so its fair share is not 0, and we use the
|
|
|
+ // fair share to compute maxAMShare
|
|
|
+ assertNotEquals("QueueFSNonZero's fair share shouldn't be zero",
|
|
|
+ 0, queueFSNonZero.getFairShare().getMemorySize());
|
|
|
+ assertEquals("QueueFSNonZero's maximum AM resource should be "
|
|
|
+ + " fair share * maxAMShare",
|
|
|
+ (long)(memCapacity * queueFSNonZero.getMaxAMShare()),
|
|
|
+ queueFSNonZero.getMetrics().getMaxAMShareMB());
|
|
|
+ assertEquals("QueueFSNonZero's maximum AM resource should be "
|
|
|
+ + " fair share * maxAMShare",
|
|
|
+ (long)(cpuCapacity * queueFSNonZero.getMaxAMShare()),
|
|
|
+ queueFSNonZero.getMetrics().getMaxAMShareVCores());
|
|
|
+ assertEquals("QueueFSNonZero's AM resource usage should be the same to "
|
|
|
+ + "AM resource request",
|
|
|
+ amResource.getMemorySize(),
|
|
|
+ queueFSNonZero.getMetrics().getAMResourceUsageMB());
|
|
|
+ }
|
|
|
+
|
|
|
@Test
|
|
|
public void testFairShareWithZeroWeightNoneZeroMinRes() throws IOException {
|
|
|
conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
|