|
@@ -24,6 +24,7 @@ import static org.mockito.Mockito.mock;
|
|
|
import static org.mockito.Mockito.when;
|
|
|
|
|
|
import java.io.IOException;
|
|
|
+import java.util.Collection;
|
|
|
import java.util.List;
|
|
|
import java.util.Set;
|
|
|
import java.util.TreeSet;
|
|
@@ -50,27 +51,38 @@ import org.apache.hadoop.yarn.api.records.Resource;
|
|
|
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
|
|
import org.apache.hadoop.yarn.api.records.YarnApplicationState;
|
|
|
import org.apache.hadoop.yarn.client.AMRMClient.ContainerRequest;
|
|
|
+import org.apache.hadoop.yarn.client.AMRMClient.StoredContainerRequest;
|
|
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
|
import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
|
|
|
import org.apache.hadoop.yarn.server.MiniYARNCluster;
|
|
|
import org.apache.hadoop.yarn.service.Service.STATE;
|
|
|
+import org.apache.hadoop.yarn.util.BuilderUtils;
|
|
|
import org.apache.hadoop.yarn.util.Records;
|
|
|
import org.junit.After;
|
|
|
+import org.junit.AfterClass;
|
|
|
import org.junit.Before;
|
|
|
+import org.junit.BeforeClass;
|
|
|
import org.junit.Test;
|
|
|
import org.mockito.invocation.InvocationOnMock;
|
|
|
import org.mockito.stubbing.Answer;
|
|
|
|
|
|
public class TestAMRMClient {
|
|
|
- Configuration conf = null;
|
|
|
- MiniYARNCluster yarnCluster = null;
|
|
|
- YarnClientImpl yarnClient = null;
|
|
|
- List<NodeReport> nodeReports = null;
|
|
|
- ApplicationAttemptId attemptId = null;
|
|
|
- int nodeCount = 3;
|
|
|
+ static Configuration conf = null;
|
|
|
+ static MiniYARNCluster yarnCluster = null;
|
|
|
+ static YarnClientImpl yarnClient = null;
|
|
|
+ static List<NodeReport> nodeReports = null;
|
|
|
+ static ApplicationAttemptId attemptId = null;
|
|
|
+ static int nodeCount = 3;
|
|
|
|
|
|
- @Before
|
|
|
- public void setup() throws YarnRemoteException, IOException {
|
|
|
+ static Resource capability;
|
|
|
+ static Priority priority;
|
|
|
+ static String node;
|
|
|
+ static String rack;
|
|
|
+ static String[] nodes;
|
|
|
+ static String[] racks;
|
|
|
+
|
|
|
+ @BeforeClass
|
|
|
+ public static void setup() throws Exception {
|
|
|
// start minicluster
|
|
|
conf = new YarnConfiguration();
|
|
|
yarnCluster = new MiniYARNCluster(TestAMRMClient.class.getName(), nodeCount, 1, 1);
|
|
@@ -84,7 +96,17 @@ public class TestAMRMClient {
|
|
|
|
|
|
// get node info
|
|
|
nodeReports = yarnClient.getNodeReports();
|
|
|
-
|
|
|
+
|
|
|
+ priority = BuilderUtils.newPriority(1);
|
|
|
+ capability = BuilderUtils.newResource(1024, 1);
|
|
|
+ node = nodeReports.get(0).getNodeId().getHost();
|
|
|
+ rack = nodeReports.get(0).getRackName();
|
|
|
+ nodes = new String[]{ node };
|
|
|
+ racks = new String[]{ rack };
|
|
|
+ }
|
|
|
+
|
|
|
+ @Before
|
|
|
+ public void startApp() throws Exception {
|
|
|
// submit new app
|
|
|
GetNewApplicationResponse newApp = yarnClient.getNewApplication();
|
|
|
ApplicationId appId = newApp.getApplicationId();
|
|
@@ -125,7 +147,12 @@ public class TestAMRMClient {
|
|
|
}
|
|
|
|
|
|
@After
|
|
|
- public void tearDown() {
|
|
|
+ public void cancelApp() {
|
|
|
+ attemptId = null;
|
|
|
+ }
|
|
|
+
|
|
|
+ @AfterClass
|
|
|
+ public static void tearDown() {
|
|
|
if (yarnClient != null && yarnClient.getServiceState() == STATE.STARTED) {
|
|
|
yarnClient.stop();
|
|
|
}
|
|
@@ -133,13 +160,235 @@ public class TestAMRMClient {
|
|
|
yarnCluster.stop();
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ @Test (timeout=60000)
|
|
|
+ public void testAMRMClientMatchingFit() throws YarnRemoteException, IOException {
|
|
|
+ AMRMClientImpl<StoredContainerRequest> amClient = null;
|
|
|
+ try {
|
|
|
+ // start am rm client
|
|
|
+ amClient = new AMRMClientImpl<StoredContainerRequest>(attemptId);
|
|
|
+ amClient.init(conf);
|
|
|
+ amClient.start();
|
|
|
+ amClient.registerApplicationMaster("Host", 10000, "");
|
|
|
+
|
|
|
+ Resource capability1 = BuilderUtils.newResource(1024, 2);
|
|
|
+ Resource capability2 = BuilderUtils.newResource(1024, 1);
|
|
|
+ Resource capability3 = BuilderUtils.newResource(1000, 2);
|
|
|
+ Resource capability4 = BuilderUtils.newResource(2000, 1);
|
|
|
+ Resource capability5 = BuilderUtils.newResource(1000, 3);
|
|
|
+ Resource capability6 = BuilderUtils.newResource(2000, 1);
|
|
|
+
|
|
|
+ StoredContainerRequest storedContainer1 =
|
|
|
+ new StoredContainerRequest(capability1, nodes, racks, priority);
|
|
|
+ StoredContainerRequest storedContainer2 =
|
|
|
+ new StoredContainerRequest(capability2, nodes, racks, priority);
|
|
|
+ StoredContainerRequest storedContainer3 =
|
|
|
+ new StoredContainerRequest(capability3, nodes, racks, priority);
|
|
|
+ StoredContainerRequest storedContainer4 =
|
|
|
+ new StoredContainerRequest(capability4, nodes, racks, priority);
|
|
|
+ StoredContainerRequest storedContainer5 =
|
|
|
+ new StoredContainerRequest(capability5, nodes, racks, priority);
|
|
|
+ StoredContainerRequest storedContainer6 =
|
|
|
+ new StoredContainerRequest(capability6, nodes, racks, priority);
|
|
|
+ amClient.addContainerRequest(storedContainer1);
|
|
|
+ amClient.addContainerRequest(storedContainer2);
|
|
|
+ amClient.addContainerRequest(storedContainer3);
|
|
|
+ amClient.addContainerRequest(storedContainer4);
|
|
|
+ amClient.addContainerRequest(storedContainer5);
|
|
|
+ amClient.addContainerRequest(storedContainer6);
|
|
|
+
|
|
|
+ // test matching of containers
|
|
|
+ List<? extends Collection<StoredContainerRequest>> matches;
|
|
|
+ StoredContainerRequest storedRequest;
|
|
|
+ // exact match
|
|
|
+ Resource testCapability1 = BuilderUtils.newResource(1024, 2);
|
|
|
+ matches = amClient.getMatchingRequests(priority, node, testCapability1);
|
|
|
+ verifyMatches(matches, 1);
|
|
|
+ storedRequest = matches.get(0).iterator().next();
|
|
|
+ assertTrue(storedContainer1 == storedRequest);
|
|
|
+ amClient.removeContainerRequest(storedContainer1);
|
|
|
+
|
|
|
+ // exact matching with order maintained
|
|
|
+ Resource testCapability2 = BuilderUtils.newResource(2000, 1);
|
|
|
+ matches = amClient.getMatchingRequests(priority, node, testCapability2);
|
|
|
+ verifyMatches(matches, 2);
|
|
|
+ // must be returned in the order they were made
|
|
|
+ int i = 0;
|
|
|
+ for(StoredContainerRequest storedRequest1 : matches.get(0)) {
|
|
|
+ if(i++ == 0) {
|
|
|
+ assertTrue(storedContainer4 == storedRequest1);
|
|
|
+ } else {
|
|
|
+ assertTrue(storedContainer6 == storedRequest1);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ amClient.removeContainerRequest(storedContainer6);
|
|
|
+
|
|
|
+ // matching with larger container. all requests returned
|
|
|
+ Resource testCapability3 = BuilderUtils.newResource(4000, 4);
|
|
|
+ matches = amClient.getMatchingRequests(priority, node, testCapability3);
|
|
|
+ assert(matches.size() == 4);
|
|
|
+
|
|
|
+ Resource testCapability4 = BuilderUtils.newResource(1024, 2);
|
|
|
+ matches = amClient.getMatchingRequests(priority, node, testCapability4);
|
|
|
+ assert(matches.size() == 2);
|
|
|
+ // verify non-fitting containers are not returned and fitting ones are
|
|
|
+ for(Collection<StoredContainerRequest> testSet : matches) {
|
|
|
+ assertTrue(testSet.size() == 1);
|
|
|
+ StoredContainerRequest testRequest = testSet.iterator().next();
|
|
|
+ assertTrue(testRequest != storedContainer4);
|
|
|
+ assertTrue(testRequest != storedContainer5);
|
|
|
+ assert(testRequest == storedContainer2 ||
|
|
|
+ testRequest == storedContainer3);
|
|
|
+ }
|
|
|
+
|
|
|
+ Resource testCapability5 = BuilderUtils.newResource(512, 4);
|
|
|
+ matches = amClient.getMatchingRequests(priority, node, testCapability5);
|
|
|
+ assert(matches.size() == 0);
|
|
|
+
|
|
|
+ amClient.unregisterApplicationMaster(FinalApplicationStatus.SUCCEEDED,
|
|
|
+ null, null);
|
|
|
+
|
|
|
+ } finally {
|
|
|
+ if (amClient != null && amClient.getServiceState() == STATE.STARTED) {
|
|
|
+ amClient.stop();
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private void verifyMatches(
|
|
|
+ List<? extends Collection<StoredContainerRequest>> matches,
|
|
|
+ int matchSize) {
|
|
|
+ assertTrue(matches.size() == 1);
|
|
|
+ assertTrue(matches.get(0).size() == matchSize);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test (timeout=60000)
|
|
|
+ public void testAMRMClientMatchStorage() throws YarnRemoteException, IOException {
|
|
|
+ AMRMClientImpl<StoredContainerRequest> amClient = null;
|
|
|
+ try {
|
|
|
+ // start am rm client
|
|
|
+ amClient = new AMRMClientImpl<StoredContainerRequest>(attemptId);
|
|
|
+ amClient.init(conf);
|
|
|
+ amClient.start();
|
|
|
+ amClient.registerApplicationMaster("Host", 10000, "");
|
|
|
+
|
|
|
+ Priority priority1 = Records.newRecord(Priority.class);
|
|
|
+ priority1.setPriority(2);
|
|
|
+
|
|
|
+ StoredContainerRequest storedContainer1 =
|
|
|
+ new StoredContainerRequest(capability, nodes, racks, priority);
|
|
|
+ StoredContainerRequest storedContainer2 =
|
|
|
+ new StoredContainerRequest(capability, nodes, racks, priority);
|
|
|
+ StoredContainerRequest storedContainer3 =
|
|
|
+ new StoredContainerRequest(capability, null, null, priority1);
|
|
|
+ amClient.addContainerRequest(storedContainer1);
|
|
|
+ amClient.addContainerRequest(storedContainer2);
|
|
|
+ amClient.addContainerRequest(storedContainer3);
|
|
|
+
|
|
|
+ // test addition and storage
|
|
|
+ int containersRequestedAny = amClient.remoteRequestsTable.get(priority)
|
|
|
+ .get(ResourceRequest.ANY).get(capability).remoteRequest.getNumContainers();
|
|
|
+ assertTrue(containersRequestedAny == 2);
|
|
|
+ containersRequestedAny = amClient.remoteRequestsTable.get(priority1)
|
|
|
+ .get(ResourceRequest.ANY).get(capability).remoteRequest.getNumContainers();
|
|
|
+ assertTrue(containersRequestedAny == 1);
|
|
|
+ List<? extends Collection<StoredContainerRequest>> matches =
|
|
|
+ amClient.getMatchingRequests(priority, node, capability);
|
|
|
+ verifyMatches(matches, 2);
|
|
|
+ matches = amClient.getMatchingRequests(priority, rack, capability);
|
|
|
+ verifyMatches(matches, 2);
|
|
|
+ matches =
|
|
|
+ amClient.getMatchingRequests(priority, ResourceRequest.ANY, capability);
|
|
|
+ verifyMatches(matches, 2);
|
|
|
+ matches = amClient.getMatchingRequests(priority1, rack, capability);
|
|
|
+ assertTrue(matches.isEmpty());
|
|
|
+ matches =
|
|
|
+ amClient.getMatchingRequests(priority1, ResourceRequest.ANY, capability);
|
|
|
+ verifyMatches(matches, 1);
|
|
|
+
|
|
|
+ // test removal
|
|
|
+ amClient.removeContainerRequest(storedContainer3);
|
|
|
+ matches = amClient.getMatchingRequests(priority, node, capability);
|
|
|
+ verifyMatches(matches, 2);
|
|
|
+ amClient.removeContainerRequest(storedContainer2);
|
|
|
+ matches = amClient.getMatchingRequests(priority, node, capability);
|
|
|
+ verifyMatches(matches, 1);
|
|
|
+ matches = amClient.getMatchingRequests(priority, rack, capability);
|
|
|
+ verifyMatches(matches, 1);
|
|
|
+
|
|
|
+ // test matching of containers
|
|
|
+ StoredContainerRequest storedRequest = matches.get(0).iterator().next();
|
|
|
+ assertTrue(storedContainer1 == storedRequest);
|
|
|
+ amClient.removeContainerRequest(storedContainer1);
|
|
|
+ matches =
|
|
|
+ amClient.getMatchingRequests(priority, ResourceRequest.ANY, capability);
|
|
|
+ assertTrue(matches.isEmpty());
|
|
|
+ matches =
|
|
|
+ amClient.getMatchingRequests(priority1, ResourceRequest.ANY, capability);
|
|
|
+ assertTrue(matches.isEmpty());
|
|
|
+ // 0 requests left. everything got cleaned up
|
|
|
+ assertTrue(amClient.remoteRequestsTable.isEmpty());
|
|
|
+
|
|
|
+ // go through an exemplary allocation, matching and release cycle
|
|
|
+ amClient.addContainerRequest(storedContainer1);
|
|
|
+ amClient.addContainerRequest(storedContainer3);
|
|
|
+ // RM should allocate container within 2 calls to allocate()
|
|
|
+ int allocatedContainerCount = 0;
|
|
|
+ int iterationsLeft = 2;
|
|
|
+ while (allocatedContainerCount < 2
|
|
|
+ && iterationsLeft-- > 0) {
|
|
|
+ AllocateResponse allocResponse = amClient.allocate(0.1f);
|
|
|
+ assertTrue(amClient.ask.size() == 0);
|
|
|
+ assertTrue(amClient.release.size() == 0);
|
|
|
+
|
|
|
+ assertTrue(nodeCount == amClient.getClusterNodeCount());
|
|
|
+ allocatedContainerCount += allocResponse.getAllocatedContainers().size();
|
|
|
+ for(Container container : allocResponse.getAllocatedContainers()) {
|
|
|
+ ContainerRequest expectedRequest =
|
|
|
+ container.getPriority().equals(storedContainer1.getPriority()) ?
|
|
|
+ storedContainer1 : storedContainer3;
|
|
|
+ matches = amClient.getMatchingRequests(container.getPriority(),
|
|
|
+ ResourceRequest.ANY,
|
|
|
+ container.getResource());
|
|
|
+ // test correct matched container is returned
|
|
|
+ verifyMatches(matches, 1);
|
|
|
+ ContainerRequest matchedRequest = matches.get(0).iterator().next();
|
|
|
+ assertTrue(matchedRequest == expectedRequest);
|
|
|
+
|
|
|
+ // assign this container, use it and release it
|
|
|
+ amClient.releaseAssignedContainer(container.getId());
|
|
|
+ }
|
|
|
+ if(allocatedContainerCount < containersRequestedAny) {
|
|
|
+ // sleep to let NM's heartbeat to RM and trigger allocations
|
|
|
+ sleep(1000);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ assertTrue(allocatedContainerCount == 2);
|
|
|
+ assertTrue(amClient.release.size() == 2);
|
|
|
+ assertTrue(amClient.ask.size() == 0);
|
|
|
+ AllocateResponse allocResponse = amClient.allocate(0.1f);
|
|
|
+ assertTrue(amClient.release.size() == 0);
|
|
|
+ assertTrue(amClient.ask.size() == 0);
|
|
|
+ assertTrue(allocResponse.getAllocatedContainers().size() == 0);
|
|
|
+
|
|
|
+
|
|
|
+ amClient.unregisterApplicationMaster(FinalApplicationStatus.SUCCEEDED,
|
|
|
+ null, null);
|
|
|
+
|
|
|
+ } finally {
|
|
|
+ if (amClient != null && amClient.getServiceState() == STATE.STARTED) {
|
|
|
+ amClient.stop();
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
|
|
|
@Test (timeout=60000)
|
|
|
public void testAMRMClient() throws YarnRemoteException, IOException {
|
|
|
- AMRMClientImpl amClient = null;
|
|
|
+ AMRMClientImpl<ContainerRequest> amClient = null;
|
|
|
try {
|
|
|
// start am rm client
|
|
|
- amClient = new AMRMClientImpl(attemptId);
|
|
|
+ amClient = new AMRMClientImpl<ContainerRequest>(attemptId);
|
|
|
amClient.init(conf);
|
|
|
amClient.start();
|
|
|
|
|
@@ -156,36 +405,27 @@ public class TestAMRMClient {
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
-
|
|
|
- private void testAllocation(final AMRMClientImpl amClient)
|
|
|
+
|
|
|
+ private void testAllocation(final AMRMClientImpl<ContainerRequest> amClient)
|
|
|
throws YarnRemoteException, IOException {
|
|
|
// setup container request
|
|
|
- final Resource capability = Records.newRecord(Resource.class);
|
|
|
- final Priority priority = Records.newRecord(Priority.class);
|
|
|
- priority.setPriority(0);
|
|
|
- capability.setMemory(1024);
|
|
|
- String node = nodeReports.get(0).getNodeId().getHost();
|
|
|
- String rack = nodeReports.get(0).getRackName();
|
|
|
- final String[] nodes = { node };
|
|
|
- final String[] racks = { rack };
|
|
|
|
|
|
assertTrue(amClient.ask.size() == 0);
|
|
|
assertTrue(amClient.release.size() == 0);
|
|
|
|
|
|
- amClient.addContainerRequest(new ContainerRequest(capability, nodes,
|
|
|
- racks, priority, 1));
|
|
|
- amClient.addContainerRequest(new ContainerRequest(capability, nodes,
|
|
|
- racks, priority, 3));
|
|
|
- amClient.removeContainerRequest(new ContainerRequest(capability, nodes,
|
|
|
- racks, priority, 2));
|
|
|
+ amClient.addContainerRequest(
|
|
|
+ new ContainerRequest(capability, nodes, racks, priority, 1));
|
|
|
+ amClient.addContainerRequest(
|
|
|
+ new ContainerRequest(capability, nodes, racks, priority, 3));
|
|
|
+ amClient.removeContainerRequest(
|
|
|
+ new ContainerRequest(capability, nodes, racks, priority, 2));
|
|
|
|
|
|
int containersRequestedNode = amClient.remoteRequestsTable.get(priority)
|
|
|
- .get(node).get(capability).getNumContainers();
|
|
|
+ .get(node).get(capability).remoteRequest.getNumContainers();
|
|
|
int containersRequestedRack = amClient.remoteRequestsTable.get(priority)
|
|
|
- .get(rack).get(capability).getNumContainers();
|
|
|
+ .get(rack).get(capability).remoteRequest.getNumContainers();
|
|
|
int containersRequestedAny = amClient.remoteRequestsTable.get(priority)
|
|
|
- .get(ResourceRequest.ANY).get(capability).getNumContainers();
|
|
|
+ .get(ResourceRequest.ANY).get(capability).remoteRequest.getNumContainers();
|
|
|
|
|
|
assertTrue(containersRequestedNode == 2);
|
|
|
assertTrue(containersRequestedRack == 2);
|
|
@@ -221,8 +461,8 @@ public class TestAMRMClient {
|
|
|
assertTrue(amClient.ask.size() == 0);
|
|
|
|
|
|
// need to tell the AMRMClient that we dont need these resources anymore
|
|
|
- amClient.removeContainerRequest(new ContainerRequest(capability, nodes,
|
|
|
- racks, priority, 2));
|
|
|
+ amClient.removeContainerRequest(
|
|
|
+ new ContainerRequest(capability, nodes, racks, priority, 2));
|
|
|
assertTrue(amClient.ask.size() == 3);
|
|
|
// send 0 container count request for resources that are no longer needed
|
|
|
ResourceRequest snoopRequest = amClient.ask.iterator().next();
|
|
@@ -241,8 +481,9 @@ public class TestAMRMClient {
|
|
|
new Answer<AllocateResponse>() {
|
|
|
public AllocateResponse answer(InvocationOnMock invocation)
|
|
|
throws Exception {
|
|
|
- amClient.removeContainerRequest(new ContainerRequest(capability,
|
|
|
- nodes, racks, priority, 2));
|
|
|
+ amClient.removeContainerRequest(
|
|
|
+ new ContainerRequest(capability, nodes,
|
|
|
+ racks, priority, 2));
|
|
|
throw new Exception();
|
|
|
}
|
|
|
});
|