blob: dbe21d1c6cec5fe3d949595cc93d58dc988a03c0 [file] [log] [blame]
/**
* 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;
import static org.junit.Assert.assertEquals;
import java.io.IOException;
import java.util.List;
import junit.framework.Assert;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.net.NetworkTopology;
import org.apache.hadoop.yarn.api.records.Priority;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.event.AsyncDispatcher;
import org.apache.hadoop.yarn.server.resourcemanager.Application;
import org.apache.hadoop.yarn.server.resourcemanager.MockNodes;
import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
import org.apache.hadoop.yarn.server.resourcemanager.Task;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.Store;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.StoreFactory;
import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
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.NodeRemovedSchedulerEvent;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
public class TestCapacityScheduler {
private static final Log LOG = LogFactory.getLog(TestCapacityScheduler.class);
private final int GB = 1024;
private static final String A = CapacitySchedulerConfiguration.ROOT + ".a";
private static final String B = CapacitySchedulerConfiguration.ROOT + ".b";
private static final String A1 = A + ".a1";
private static final String A2 = A + ".a2";
private static final String B1 = B + ".b1";
private static final String B2 = B + ".b2";
private static final String B3 = B + ".b3";
private static int A_CAPACITY = 10;
private static int B_CAPACITY = 90;
private static int A1_CAPACITY = 30;
private static int A2_CAPACITY = 70;
private static int B1_CAPACITY = 50;
private static int B2_CAPACITY = 30;
private static int B3_CAPACITY = 20;
private ResourceManager resourceManager = null;
@Before
public void setUp() throws Exception {
Store store = StoreFactory.getStore(new Configuration());
resourceManager = new ResourceManager(store);
CapacitySchedulerConfiguration csConf =
new CapacitySchedulerConfiguration();
csConf.setClass(YarnConfiguration.RM_SCHEDULER,
CapacityScheduler.class, ResourceScheduler.class);
setupQueueConfiguration(csConf);
resourceManager.init(csConf);
((AsyncDispatcher)resourceManager.getRMContext().getDispatcher()).start();
}
@After
public void tearDown() throws Exception {
}
private org.apache.hadoop.yarn.server.resourcemanager.NodeManager
registerNode(String hostName, int containerManagerPort, int httpPort,
String rackName, int memory)
throws IOException {
return new org.apache.hadoop.yarn.server.resourcemanager.NodeManager(
hostName, containerManagerPort, httpPort, rackName, memory,
resourceManager.getResourceTrackerService(), resourceManager
.getRMContext());
}
// @Test
public void testCapacityScheduler() throws Exception {
LOG.info("--- START: testCapacityScheduler ---");
// Register node1
String host_0 = "host_0";
org.apache.hadoop.yarn.server.resourcemanager.NodeManager nm_0 =
registerNode(host_0, 1234, 2345, NetworkTopology.DEFAULT_RACK, 4 * GB);
nm_0.heartbeat();
// Register node2
String host_1 = "host_1";
org.apache.hadoop.yarn.server.resourcemanager.NodeManager nm_1 =
registerNode(host_1, 1234, 2345, NetworkTopology.DEFAULT_RACK, 2 * GB);
nm_1.heartbeat();
// ResourceRequest priorities
Priority priority_0 =
org.apache.hadoop.yarn.server.resourcemanager.resource.Priority.create(0);
Priority priority_1 =
org.apache.hadoop.yarn.server.resourcemanager.resource.Priority.create(1);
// Submit an application
Application application_0 = new Application("user_0", "a1", resourceManager);
application_0.submit();
application_0.addNodeManager(host_0, 1234, nm_0);
application_0.addNodeManager(host_1, 1234, nm_1);
Resource capability_0_0 = Resources.createResource(1 * GB);
application_0.addResourceRequestSpec(priority_1, capability_0_0);
Resource capability_0_1 = Resources.createResource(2 * GB);
application_0.addResourceRequestSpec(priority_0, capability_0_1);
Task task_0_0 = new Task(application_0, priority_1,
new String[] {host_0, host_1});
application_0.addTask(task_0_0);
// Submit another application
Application application_1 = new Application("user_1", "b2", resourceManager);
application_1.submit();
application_1.addNodeManager(host_0, 1234, nm_0);
application_1.addNodeManager(host_1, 1234, nm_1);
Resource capability_1_0 = Resources.createResource(3 * GB);
application_1.addResourceRequestSpec(priority_1, capability_1_0);
Resource capability_1_1 = Resources.createResource(2 * GB);
application_1.addResourceRequestSpec(priority_0, capability_1_1);
Task task_1_0 = new Task(application_1, priority_1,
new String[] {host_0, host_1});
application_1.addTask(task_1_0);
// Send resource requests to the scheduler
application_0.schedule();
application_1.schedule();
// Send a heartbeat to kick the tires on the Scheduler
LOG.info("Kick!");
nm_0.heartbeat(); // task_0_0 and task_1_0 allocated, used=4G
nm_1.heartbeat(); // nothing allocated
// Get allocations from the scheduler
application_0.schedule(); // task_0_0
checkApplicationResourceUsage(1 * GB, application_0);
application_1.schedule(); // task_1_0
checkApplicationResourceUsage(3 * GB, application_1);
nm_0.heartbeat();
nm_1.heartbeat();
checkNodeResourceUsage(4*GB, nm_0); // task_0_0 (1G) and task_1_0 (3G)
checkNodeResourceUsage(0*GB, nm_1); // no tasks, 2G available
LOG.info("Adding new tasks...");
Task task_1_1 = new Task(application_1, priority_0,
new String[] {RMNode.ANY});
application_1.addTask(task_1_1);
application_1.schedule();
Task task_0_1 = new Task(application_0, priority_0,
new String[] {host_0, host_1});
application_0.addTask(task_0_1);
application_0.schedule();
// Send a heartbeat to kick the tires on the Scheduler
LOG.info("Sending hb from " + nm_0.getHostName());
nm_0.heartbeat(); // nothing new, used=4G
LOG.info("Sending hb from " + nm_1.getHostName());
nm_1.heartbeat(); // task_0_3, used=2G
// Get allocations from the scheduler
LOG.info("Trying to allocate...");
application_0.schedule();
checkApplicationResourceUsage(1 * GB, application_0);
application_1.schedule();
checkApplicationResourceUsage(5 * GB, application_1);
nm_0.heartbeat();
nm_1.heartbeat();
checkNodeResourceUsage(4*GB, nm_0);
checkNodeResourceUsage(2*GB, nm_1);
LOG.info("--- END: testCapacityScheduler ---");
}
private void setupQueueConfiguration(CapacitySchedulerConfiguration conf) {
// Define top-level queues
conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {"a", "b"});
conf.setCapacity(CapacitySchedulerConfiguration.ROOT, 100);
conf.setCapacity(A, A_CAPACITY);
conf.setCapacity(B, B_CAPACITY);
// Define 2nd-level queues
conf.setQueues(A, new String[] {"a1", "a2"});
conf.setCapacity(A1, A1_CAPACITY);
conf.setUserLimitFactor(A1, 100.0f);
conf.setCapacity(A2, A2_CAPACITY);
conf.setUserLimitFactor(A2, 100.0f);
conf.setQueues(B, new String[] {"b1", "b2", "b3"});
conf.setCapacity(B1, B1_CAPACITY);
conf.setUserLimitFactor(B1, 100.0f);
conf.setCapacity(B2, B2_CAPACITY);
conf.setUserLimitFactor(B2, 100.0f);
conf.setCapacity(B3, B3_CAPACITY);
conf.setUserLimitFactor(B3, 100.0f);
LOG.info("Setup top-level queues a and b");
}
@Test
public void testRefreshQueues() throws Exception {
CapacityScheduler cs = new CapacityScheduler();
CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration();
setupQueueConfiguration(conf);
cs.reinitialize(conf, null, null);
checkQueueCapacities(cs, A_CAPACITY, B_CAPACITY);
conf.setCapacity(A, 80);
conf.setCapacity(B, 20);
cs.reinitialize(conf, null,null);
checkQueueCapacities(cs, 80, 20);
}
private void checkQueueCapacities(CapacityScheduler cs,
int capacityA, int capacityB) {
CSQueue rootQueue = cs.getRootQueue();
CSQueue queueA = findQueue(rootQueue, A);
CSQueue queueB = findQueue(rootQueue, B);
CSQueue queueA1 = findQueue(queueA, A1);
CSQueue queueA2 = findQueue(queueA, A2);
CSQueue queueB1 = findQueue(queueB, B1);
CSQueue queueB2 = findQueue(queueB, B2);
CSQueue queueB3 = findQueue(queueB, B3);
float capA = capacityA / 100.0f;
float capB = capacityB / 100.0f;
checkQueueCapacity(queueA, capA, capA, 1.0f, 1.0f);
checkQueueCapacity(queueB, capB, capB, 1.0f, 1.0f);
checkQueueCapacity(queueA1, A1_CAPACITY / 100.0f,
(A1_CAPACITY/100.0f) * capA, 1.0f, 1.0f);
checkQueueCapacity(queueA2, (float)A2_CAPACITY / 100.0f,
(A2_CAPACITY/100.0f) * capA, 1.0f, 1.0f);
checkQueueCapacity(queueB1, (float)B1_CAPACITY / 100.0f,
(B1_CAPACITY/100.0f) * capB, 1.0f, 1.0f);
checkQueueCapacity(queueB2, (float)B2_CAPACITY / 100.0f,
(B2_CAPACITY/100.0f) * capB, 1.0f, 1.0f);
checkQueueCapacity(queueB3, (float)B3_CAPACITY / 100.0f,
(B3_CAPACITY/100.0f) * capB, 1.0f, 1.0f);
}
private void checkQueueCapacity(CSQueue q, float expectedCapacity,
float expectedAbsCapacity, float expectedMaxCapacity,
float expectedAbsMaxCapacity) {
final float epsilon = 1e-5f;
assertEquals("capacity", expectedCapacity, q.getCapacity(), epsilon);
assertEquals("absolute capacity", expectedAbsCapacity,
q.getAbsoluteCapacity(), epsilon);
assertEquals("maximum capacity", expectedMaxCapacity,
q.getMaximumCapacity(), epsilon);
assertEquals("absolute maximum capacity", expectedAbsMaxCapacity,
q.getAbsoluteMaximumCapacity(), epsilon);
}
private CSQueue findQueue(CSQueue root, String queuePath) {
if (root.getQueuePath().equals(queuePath)) {
return root;
}
List<CSQueue> childQueues = root.getChildQueues();
if (childQueues != null) {
for (CSQueue q : childQueues) {
if (queuePath.startsWith(q.getQueuePath())) {
CSQueue result = findQueue(q, queuePath);
if (result != null) {
return result;
}
}
}
}
return null;
}
private void checkApplicationResourceUsage(int expected,
Application application) {
Assert.assertEquals(expected, application.getUsedResources().getMemory());
}
private void checkNodeResourceUsage(int expected,
org.apache.hadoop.yarn.server.resourcemanager.NodeManager node) {
Assert.assertEquals(expected, node.getUsed().getMemory());
node.checkResourceUsage();
}
/** Test that parseQueue throws an exception when two leaf queues have the
* same name
* @throws IOException
*/
@Test(expected=IOException.class)
public void testParseQueue() throws IOException {
CapacityScheduler cs = new CapacityScheduler();
CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration();
setupQueueConfiguration(conf);
conf.setQueues(CapacitySchedulerConfiguration.ROOT + ".a.a1", new String[] {"b1"} );
conf.setCapacity(CapacitySchedulerConfiguration.ROOT + ".a.a1.b1", 100);
conf.setUserLimitFactor(CapacitySchedulerConfiguration.ROOT + ".a.a1.b1", 100.0f);
cs.reinitialize(conf, null, null);
}
@Test
public void testReconnectedNode() throws Exception {
CapacitySchedulerConfiguration csConf =
new CapacitySchedulerConfiguration();
setupQueueConfiguration(csConf);
CapacityScheduler cs = new CapacityScheduler();
cs.reinitialize(csConf, null, null);
RMNode n1 = MockNodes.newNodeInfo(0, MockNodes.newResource(4 * GB), 1);
RMNode n2 = MockNodes.newNodeInfo(0, MockNodes.newResource(2 * GB), 2);
cs.handle(new NodeAddedSchedulerEvent(n1));
cs.handle(new NodeAddedSchedulerEvent(n2));
Assert.assertEquals(6 * GB, cs.getClusterResources().getMemory());
// reconnect n1 with downgraded memory
n1 = MockNodes.newNodeInfo(0, MockNodes.newResource(2 * GB), 1);
cs.handle(new NodeRemovedSchedulerEvent(n1));
cs.handle(new NodeAddedSchedulerEvent(n1));
Assert.assertEquals(4 * GB, cs.getClusterResources().getMemory());
}
}