|
@@ -34,6 +34,7 @@ import java.util.concurrent.CountDownLatch;
|
|
import java.util.concurrent.ExecutorService;
|
|
import java.util.concurrent.ExecutorService;
|
|
import java.util.concurrent.TimeUnit;
|
|
import java.util.concurrent.TimeUnit;
|
|
|
|
|
|
|
|
+import com.google.common.collect.ImmutableMap;
|
|
import org.apache.hadoop.util.concurrent.HadoopExecutors;
|
|
import org.apache.hadoop.util.concurrent.HadoopExecutors;
|
|
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
|
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
|
import org.apache.hadoop.yarn.api.records.Resource;
|
|
import org.apache.hadoop.yarn.api.records.Resource;
|
|
@@ -42,19 +43,26 @@ import org.apache.hadoop.yarn.server.resourcemanager.MockNodes;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
|
|
|
|
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetricsCustomResource;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
|
|
|
|
+import org.apache.hadoop.yarn.util.resource.ResourceUtils;
|
|
import org.apache.hadoop.yarn.util.resource.Resources;
|
|
import org.apache.hadoop.yarn.util.resource.Resources;
|
|
import org.junit.After;
|
|
import org.junit.After;
|
|
import org.junit.Before;
|
|
import org.junit.Before;
|
|
import org.junit.Test;
|
|
import org.junit.Test;
|
|
import org.mockito.Mockito;
|
|
import org.mockito.Mockito;
|
|
|
|
+import java.util.Map;
|
|
|
|
+import static org.junit.Assert.assertFalse;
|
|
|
|
+import static org.junit.Assert.assertNotNull;
|
|
|
|
|
|
public class TestFSLeafQueue extends FairSchedulerTestBase {
|
|
public class TestFSLeafQueue extends FairSchedulerTestBase {
|
|
private final static String ALLOC_FILE = new File(TEST_DIR,
|
|
private final static String ALLOC_FILE = new File(TEST_DIR,
|
|
TestFSLeafQueue.class.getName() + ".xml").getAbsolutePath();
|
|
TestFSLeafQueue.class.getName() + ".xml").getAbsolutePath();
|
|
private Resource maxResource = Resources.createResource(1024 * 8);
|
|
private Resource maxResource = Resources.createResource(1024 * 8);
|
|
|
|
+ private static final float MAX_AM_SHARE = 0.5f;
|
|
|
|
+ private static final String CUSTOM_RESOURCE = "test1";
|
|
|
|
|
|
@Before
|
|
@Before
|
|
public void setup() throws IOException {
|
|
public void setup() throws IOException {
|
|
@@ -105,6 +113,8 @@ public class TestFSLeafQueue extends FairSchedulerTestBase {
|
|
PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
|
|
PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
|
|
out.println("<?xml version=\"1.0\"?>");
|
|
out.println("<?xml version=\"1.0\"?>");
|
|
out.println("<allocations>");
|
|
out.println("<allocations>");
|
|
|
|
+ out.println("<queueMaxAMShareDefault>" + MAX_AM_SHARE +
|
|
|
|
+ "</queueMaxAMShareDefault>");
|
|
out.println("<queue name=\"queueA\"></queue>");
|
|
out.println("<queue name=\"queueA\"></queue>");
|
|
out.println("<queue name=\"queueB\"></queue>");
|
|
out.println("<queue name=\"queueB\"></queue>");
|
|
out.println("</allocations>");
|
|
out.println("</allocations>");
|
|
@@ -221,4 +231,128 @@ public class TestFSLeafQueue extends FairSchedulerTestBase {
|
|
assertTrue("Test failed with exception(s)" + exceptions,
|
|
assertTrue("Test failed with exception(s)" + exceptions,
|
|
exceptions.isEmpty());
|
|
exceptions.isEmpty());
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ @Test
|
|
|
|
+ public void testCanRunAppAMReturnsTrue() {
|
|
|
|
+ conf.set(YarnConfiguration.RESOURCE_TYPES, CUSTOM_RESOURCE);
|
|
|
|
+ ResourceUtils.resetResourceTypes(conf);
|
|
|
|
+
|
|
|
|
+ resourceManager = new MockRM(conf);
|
|
|
|
+ resourceManager.start();
|
|
|
|
+ scheduler = (FairScheduler) resourceManager.getResourceScheduler();
|
|
|
|
+
|
|
|
|
+ Resource maxShare = Resource.newInstance(1024 * 8, 4,
|
|
|
|
+ ImmutableMap.of(CUSTOM_RESOURCE, 10L));
|
|
|
|
+
|
|
|
|
+ // Add a node to increase available memory and vcores in scheduler's
|
|
|
|
+ // root queue metrics
|
|
|
|
+ addNodeToScheduler(Resource.newInstance(4096, 10,
|
|
|
|
+ ImmutableMap.of(CUSTOM_RESOURCE, 25L)));
|
|
|
|
+
|
|
|
|
+ FSLeafQueue queue = setupQueue(maxShare);
|
|
|
|
+
|
|
|
|
+ //Min(availableMemory, maxShareMemory (maxResourceOverridden))
|
|
|
|
+ // --> Min(4096, 8192) = 4096
|
|
|
|
+ //Min(availableVCores, maxShareVCores (maxResourceOverridden))
|
|
|
|
+ // --> Min(10, 4) = 4
|
|
|
|
+ //Min(available test1, maxShare test1 (maxResourceOverridden))
|
|
|
|
+ // --> Min(25, 10) = 10
|
|
|
|
+ //MaxAMResource: (4096 MB memory, 4 vcores, 10 test1) * MAX_AM_SHARE
|
|
|
|
+ // --> 2048 MB memory, 2 vcores, 5 test1
|
|
|
|
+ Resource expectedAMShare = Resource.newInstance(2048, 2,
|
|
|
|
+ ImmutableMap.of(CUSTOM_RESOURCE, 5L));
|
|
|
|
+
|
|
|
|
+ Resource appAMResource = Resource.newInstance(2048, 2,
|
|
|
|
+ ImmutableMap.of(CUSTOM_RESOURCE, 3L));
|
|
|
|
+
|
|
|
|
+ Map<String, Long> customResourceValues =
|
|
|
|
+ verifyQueueMetricsForCustomResources(queue);
|
|
|
|
+
|
|
|
|
+ boolean result = queue.canRunAppAM(appAMResource);
|
|
|
|
+ assertTrue("AM should have been allocated!", result);
|
|
|
|
+
|
|
|
|
+ verifyAMShare(queue, expectedAMShare, customResourceValues);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private FSLeafQueue setupQueue(Resource maxShare) {
|
|
|
|
+ String queueName = "root.queue1";
|
|
|
|
+ FSLeafQueue schedulable = new FSLeafQueue(queueName, scheduler, null);
|
|
|
|
+ schedulable.setMaxShare(new ConfigurableResource(maxShare));
|
|
|
|
+ schedulable.setMaxAMShare(MAX_AM_SHARE);
|
|
|
|
+ return schedulable;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @Test
|
|
|
|
+ public void testCanRunAppAMReturnsFalse() {
|
|
|
|
+ conf.set(YarnConfiguration.RESOURCE_TYPES, CUSTOM_RESOURCE);
|
|
|
|
+ ResourceUtils.resetResourceTypes(conf);
|
|
|
|
+
|
|
|
|
+ resourceManager = new MockRM(conf);
|
|
|
|
+ resourceManager.start();
|
|
|
|
+ scheduler = (FairScheduler) resourceManager.getResourceScheduler();
|
|
|
|
+
|
|
|
|
+ Resource maxShare = Resource.newInstance(1024 * 8, 4,
|
|
|
|
+ ImmutableMap.of(CUSTOM_RESOURCE, 10L));
|
|
|
|
+
|
|
|
|
+ // Add a node to increase available memory and vcores in scheduler's
|
|
|
|
+ // root queue metrics
|
|
|
|
+ addNodeToScheduler(Resource.newInstance(4096, 10,
|
|
|
|
+ ImmutableMap.of(CUSTOM_RESOURCE, 25L)));
|
|
|
|
+
|
|
|
|
+ FSLeafQueue queue = setupQueue(maxShare);
|
|
|
|
+
|
|
|
|
+ //Min(availableMemory, maxShareMemory (maxResourceOverridden))
|
|
|
|
+ // --> Min(4096, 8192) = 4096
|
|
|
|
+ //Min(availableVCores, maxShareVCores (maxResourceOverridden))
|
|
|
|
+ // --> Min(10, 4) = 4
|
|
|
|
+ //Min(available test1, maxShare test1 (maxResourceOverridden))
|
|
|
|
+ // --> Min(25, 10) = 10
|
|
|
|
+ //MaxAMResource: (4096 MB memory, 4 vcores, 10 test1) * MAX_AM_SHARE
|
|
|
|
+ // --> 2048 MB memory, 2 vcores, 5 test1
|
|
|
|
+ Resource expectedAMShare = Resource.newInstance(2048, 2,
|
|
|
|
+ ImmutableMap.of(CUSTOM_RESOURCE, 5L));
|
|
|
|
+
|
|
|
|
+ Resource appAMResource = Resource.newInstance(2048, 2,
|
|
|
|
+ ImmutableMap.of(CUSTOM_RESOURCE, 6L));
|
|
|
|
+
|
|
|
|
+ Map<String, Long> customResourceValues =
|
|
|
|
+ verifyQueueMetricsForCustomResources(queue);
|
|
|
|
+
|
|
|
|
+ boolean result = queue.canRunAppAM(appAMResource);
|
|
|
|
+ assertFalse("AM should not have been allocated!", result);
|
|
|
|
+
|
|
|
|
+ verifyAMShare(queue, expectedAMShare, customResourceValues);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private void addNodeToScheduler(Resource node1Resource) {
|
|
|
|
+ RMNode node1 = MockNodes.newNodeInfo(0, node1Resource, 1, "127.0.0.2");
|
|
|
|
+ scheduler.handle(new NodeAddedSchedulerEvent(node1));
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private void verifyAMShare(FSLeafQueue schedulable,
|
|
|
|
+ Resource expectedAMShare, Map<String, Long> customResourceValues) {
|
|
|
|
+ Resource actualAMShare = Resource.newInstance(
|
|
|
|
+ schedulable.getMetrics().getMaxAMShareMB(),
|
|
|
|
+ schedulable.getMetrics().getMaxAMShareVCores(), customResourceValues);
|
|
|
|
+ long customResourceValue =
|
|
|
|
+ actualAMShare.getResourceValue(CUSTOM_RESOURCE);
|
|
|
|
+
|
|
|
|
+ //make sure to verify custom resource value explicitly!
|
|
|
|
+ assertEquals(5L, customResourceValue);
|
|
|
|
+ assertEquals("AM share is not the expected!", expectedAMShare,
|
|
|
|
+ actualAMShare);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private Map<String, Long> verifyQueueMetricsForCustomResources(
|
|
|
|
+ FSLeafQueue schedulable) {
|
|
|
|
+ QueueMetricsCustomResource maxAMShareCustomResources =
|
|
|
|
+ schedulable.getMetrics().getCustomResources().getMaxAMShare();
|
|
|
|
+ Map<String, Long> customResourceValues = maxAMShareCustomResources
|
|
|
|
+ .getValues();
|
|
|
|
+ assertNotNull("Queue metrics for custom resources should not be null!",
|
|
|
|
+ maxAMShareCustomResources);
|
|
|
|
+ assertNotNull("Queue metrics for custom resources resource values " +
|
|
|
|
+ "should not be null!", customResourceValues);
|
|
|
|
+ return customResourceValues;
|
|
|
|
+ }
|
|
}
|
|
}
|