|
@@ -60,7 +60,6 @@ import org.apache.hadoop.yarn.client.api.AMRMClient;
|
|
|
import org.apache.hadoop.yarn.client.api.NMTokenCache;
|
|
|
import org.apache.hadoop.yarn.client.api.YarnClient;
|
|
|
import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest;
|
|
|
-import org.apache.hadoop.yarn.client.api.AMRMClient.StoredContainerRequest;
|
|
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
|
import org.apache.hadoop.yarn.exceptions.YarnException;
|
|
|
import org.apache.hadoop.yarn.server.MiniYARNCluster;
|
|
@@ -93,6 +92,7 @@ public class TestAMRMClient {
|
|
|
public static void setup() throws Exception {
|
|
|
// start minicluster
|
|
|
conf = new YarnConfiguration();
|
|
|
+ conf.setInt(YarnConfiguration.RM_NM_HEARTBEAT_INTERVAL_MS, 100);
|
|
|
yarnCluster = new MiniYARNCluster(TestAMRMClient.class.getName(), nodeCount, 1, 1);
|
|
|
yarnCluster.init(conf);
|
|
|
yarnCluster.start();
|
|
@@ -169,10 +169,10 @@ public class TestAMRMClient {
|
|
|
|
|
|
@Test (timeout=60000)
|
|
|
public void testAMRMClientMatchingFit() throws YarnException, IOException {
|
|
|
- AMRMClient<StoredContainerRequest> amClient = null;
|
|
|
+ AMRMClient<ContainerRequest> amClient = null;
|
|
|
try {
|
|
|
// start am rm client
|
|
|
- amClient = AMRMClient.<StoredContainerRequest>createAMRMClient(attemptId);
|
|
|
+ amClient = AMRMClient.<ContainerRequest>createAMRMClient(attemptId);
|
|
|
amClient.init(conf);
|
|
|
amClient.start();
|
|
|
amClient.registerApplicationMaster("Host", 10000, "");
|
|
@@ -185,20 +185,20 @@ public class TestAMRMClient {
|
|
|
Resource capability6 = Resource.newInstance(2000, 1);
|
|
|
Resource capability7 = Resource.newInstance(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);
|
|
|
- StoredContainerRequest storedContainer7 =
|
|
|
- new StoredContainerRequest(capability7, nodes, racks, priority2, false);
|
|
|
+ ContainerRequest storedContainer1 =
|
|
|
+ new ContainerRequest(capability1, nodes, racks, priority);
|
|
|
+ ContainerRequest storedContainer2 =
|
|
|
+ new ContainerRequest(capability2, nodes, racks, priority);
|
|
|
+ ContainerRequest storedContainer3 =
|
|
|
+ new ContainerRequest(capability3, nodes, racks, priority);
|
|
|
+ ContainerRequest storedContainer4 =
|
|
|
+ new ContainerRequest(capability4, nodes, racks, priority);
|
|
|
+ ContainerRequest storedContainer5 =
|
|
|
+ new ContainerRequest(capability5, nodes, racks, priority);
|
|
|
+ ContainerRequest storedContainer6 =
|
|
|
+ new ContainerRequest(capability6, nodes, racks, priority);
|
|
|
+ ContainerRequest storedContainer7 =
|
|
|
+ new ContainerRequest(capability7, nodes, racks, priority2, false);
|
|
|
amClient.addContainerRequest(storedContainer1);
|
|
|
amClient.addContainerRequest(storedContainer2);
|
|
|
amClient.addContainerRequest(storedContainer3);
|
|
@@ -208,8 +208,8 @@ public class TestAMRMClient {
|
|
|
amClient.addContainerRequest(storedContainer7);
|
|
|
|
|
|
// test matching of containers
|
|
|
- List<? extends Collection<StoredContainerRequest>> matches;
|
|
|
- StoredContainerRequest storedRequest;
|
|
|
+ List<? extends Collection<ContainerRequest>> matches;
|
|
|
+ ContainerRequest storedRequest;
|
|
|
// exact match
|
|
|
Resource testCapability1 = Resource.newInstance(1024, 2);
|
|
|
matches = amClient.getMatchingRequests(priority, node, testCapability1);
|
|
@@ -224,7 +224,7 @@ public class TestAMRMClient {
|
|
|
verifyMatches(matches, 2);
|
|
|
// must be returned in the order they were made
|
|
|
int i = 0;
|
|
|
- for(StoredContainerRequest storedRequest1 : matches.get(0)) {
|
|
|
+ for(ContainerRequest storedRequest1 : matches.get(0)) {
|
|
|
if(i++ == 0) {
|
|
|
assertTrue(storedContainer4 == storedRequest1);
|
|
|
} else {
|
|
@@ -242,9 +242,9 @@ public class TestAMRMClient {
|
|
|
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) {
|
|
|
+ for(Collection<ContainerRequest> testSet : matches) {
|
|
|
assertTrue(testSet.size() == 1);
|
|
|
- StoredContainerRequest testRequest = testSet.iterator().next();
|
|
|
+ ContainerRequest testRequest = testSet.iterator().next();
|
|
|
assertTrue(testRequest != storedContainer4);
|
|
|
assertTrue(testRequest != storedContainer5);
|
|
|
assert(testRequest == storedContainer2 ||
|
|
@@ -275,7 +275,7 @@ public class TestAMRMClient {
|
|
|
}
|
|
|
|
|
|
private void verifyMatches(
|
|
|
- List<? extends Collection<StoredContainerRequest>> matches,
|
|
|
+ List<? extends Collection<ContainerRequest>> matches,
|
|
|
int matchSize) {
|
|
|
assertTrue(matches.size() == 1);
|
|
|
assertTrue(matches.get(0).size() == matchSize);
|
|
@@ -283,23 +283,23 @@ public class TestAMRMClient {
|
|
|
|
|
|
@Test (timeout=60000)
|
|
|
public void testAMRMClientMatchingFitInferredRack() throws YarnException, IOException {
|
|
|
- AMRMClientImpl<StoredContainerRequest> amClient = null;
|
|
|
+ AMRMClientImpl<ContainerRequest> amClient = null;
|
|
|
try {
|
|
|
// start am rm client
|
|
|
- amClient = new AMRMClientImpl<StoredContainerRequest>(attemptId);
|
|
|
+ amClient = new AMRMClientImpl<ContainerRequest>(attemptId);
|
|
|
amClient.init(conf);
|
|
|
amClient.start();
|
|
|
amClient.registerApplicationMaster("Host", 10000, "");
|
|
|
|
|
|
Resource capability = Resource.newInstance(1024, 2);
|
|
|
|
|
|
- StoredContainerRequest storedContainer1 =
|
|
|
- new StoredContainerRequest(capability, nodes, null, priority);
|
|
|
+ ContainerRequest storedContainer1 =
|
|
|
+ new ContainerRequest(capability, nodes, null, priority);
|
|
|
amClient.addContainerRequest(storedContainer1);
|
|
|
|
|
|
// verify matching with original node and inferred rack
|
|
|
- List<? extends Collection<StoredContainerRequest>> matches;
|
|
|
- StoredContainerRequest storedRequest;
|
|
|
+ List<? extends Collection<ContainerRequest>> matches;
|
|
|
+ ContainerRequest storedRequest;
|
|
|
// exact match node
|
|
|
matches = amClient.getMatchingRequests(priority, node, capability);
|
|
|
verifyMatches(matches, 1);
|
|
@@ -326,14 +326,14 @@ public class TestAMRMClient {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- @Test (timeout=60000)
|
|
|
+ @Test //(timeout=60000)
|
|
|
public void testAMRMClientMatchStorage() throws YarnException, IOException {
|
|
|
- AMRMClientImpl<StoredContainerRequest> amClient = null;
|
|
|
+ AMRMClientImpl<ContainerRequest> amClient = null;
|
|
|
try {
|
|
|
// start am rm client
|
|
|
amClient =
|
|
|
- (AMRMClientImpl<StoredContainerRequest>) AMRMClient
|
|
|
- .<StoredContainerRequest> createAMRMClient(attemptId);
|
|
|
+ (AMRMClientImpl<ContainerRequest>) AMRMClient
|
|
|
+ .<ContainerRequest> createAMRMClient(attemptId);
|
|
|
amClient.init(conf);
|
|
|
amClient.start();
|
|
|
amClient.registerApplicationMaster("Host", 10000, "");
|
|
@@ -341,12 +341,12 @@ public class TestAMRMClient {
|
|
|
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);
|
|
|
+ ContainerRequest storedContainer1 =
|
|
|
+ new ContainerRequest(capability, nodes, racks, priority);
|
|
|
+ ContainerRequest storedContainer2 =
|
|
|
+ new ContainerRequest(capability, nodes, racks, priority);
|
|
|
+ ContainerRequest storedContainer3 =
|
|
|
+ new ContainerRequest(capability, null, null, priority1);
|
|
|
amClient.addContainerRequest(storedContainer1);
|
|
|
amClient.addContainerRequest(storedContainer2);
|
|
|
amClient.addContainerRequest(storedContainer3);
|
|
@@ -358,7 +358,7 @@ public class TestAMRMClient {
|
|
|
containersRequestedAny = amClient.remoteRequestsTable.get(priority1)
|
|
|
.get(ResourceRequest.ANY).get(capability).remoteRequest.getNumContainers();
|
|
|
assertTrue(containersRequestedAny == 1);
|
|
|
- List<? extends Collection<StoredContainerRequest>> matches =
|
|
|
+ List<? extends Collection<ContainerRequest>> matches =
|
|
|
amClient.getMatchingRequests(priority, node, capability);
|
|
|
verifyMatches(matches, 2);
|
|
|
matches = amClient.getMatchingRequests(priority, rack, capability);
|
|
@@ -383,7 +383,7 @@ public class TestAMRMClient {
|
|
|
verifyMatches(matches, 1);
|
|
|
|
|
|
// test matching of containers
|
|
|
- StoredContainerRequest storedRequest = matches.get(0).iterator().next();
|
|
|
+ ContainerRequest storedRequest = matches.get(0).iterator().next();
|
|
|
assertTrue(storedContainer1 == storedRequest);
|
|
|
amClient.removeContainerRequest(storedContainer1);
|
|
|
matches =
|
|
@@ -400,7 +400,7 @@ public class TestAMRMClient {
|
|
|
amClient.addContainerRequest(storedContainer3);
|
|
|
// RM should allocate container within 2 calls to allocate()
|
|
|
int allocatedContainerCount = 0;
|
|
|
- int iterationsLeft = 2;
|
|
|
+ int iterationsLeft = 3;
|
|
|
while (allocatedContainerCount < 2
|
|
|
&& iterationsLeft-- > 0) {
|
|
|
AllocateResponse allocResponse = amClient.allocate(0.1f);
|
|
@@ -420,24 +420,23 @@ public class TestAMRMClient {
|
|
|
verifyMatches(matches, 1);
|
|
|
ContainerRequest matchedRequest = matches.get(0).iterator().next();
|
|
|
assertTrue(matchedRequest == expectedRequest);
|
|
|
-
|
|
|
+ amClient.removeContainerRequest(matchedRequest);
|
|
|
// 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);
|
|
|
+ sleep(100);
|
|
|
}
|
|
|
}
|
|
|
|
|
|
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);
|
|
|
-
|
|
|
+ // 0 requests left. everything got cleaned up
|
|
|
+ assertTrue(amClient.remoteRequestsTable.isEmpty());
|
|
|
|
|
|
amClient.unregisterApplicationMaster(FinalApplicationStatus.SUCCEEDED,
|
|
|
null, null);
|
|
@@ -480,11 +479,17 @@ public class TestAMRMClient {
|
|
|
assertTrue(amClient.release.size() == 0);
|
|
|
|
|
|
amClient.addContainerRequest(
|
|
|
- new ContainerRequest(capability, nodes, racks, priority, 1));
|
|
|
+ new ContainerRequest(capability, nodes, racks, priority));
|
|
|
amClient.addContainerRequest(
|
|
|
- new ContainerRequest(capability, nodes, racks, priority, 3));
|
|
|
+ new ContainerRequest(capability, nodes, racks, priority));
|
|
|
+ amClient.addContainerRequest(
|
|
|
+ new ContainerRequest(capability, nodes, racks, priority));
|
|
|
+ amClient.addContainerRequest(
|
|
|
+ new ContainerRequest(capability, nodes, racks, priority));
|
|
|
+ amClient.removeContainerRequest(
|
|
|
+ new ContainerRequest(capability, nodes, racks, priority));
|
|
|
amClient.removeContainerRequest(
|
|
|
- new ContainerRequest(capability, nodes, racks, priority, 2));
|
|
|
+ new ContainerRequest(capability, nodes, racks, priority));
|
|
|
|
|
|
int containersRequestedNode = amClient.remoteRequestsTable.get(priority)
|
|
|
.get(node).get(capability).remoteRequest.getNumContainers();
|
|
@@ -501,7 +506,7 @@ public class TestAMRMClient {
|
|
|
|
|
|
// RM should allocate container within 2 calls to allocate()
|
|
|
int allocatedContainerCount = 0;
|
|
|
- int iterationsLeft = 2;
|
|
|
+ int iterationsLeft = 3;
|
|
|
Set<ContainerId> releases = new TreeSet<ContainerId>();
|
|
|
|
|
|
NMTokenCache.clearCache();
|
|
@@ -532,7 +537,7 @@ public class TestAMRMClient {
|
|
|
|
|
|
if(allocatedContainerCount < containersRequestedAny) {
|
|
|
// sleep to let NM's heartbeat to RM and trigger allocations
|
|
|
- sleep(1000);
|
|
|
+ sleep(100);
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -546,7 +551,9 @@ public class TestAMRMClient {
|
|
|
|
|
|
// need to tell the AMRMClient that we dont need these resources anymore
|
|
|
amClient.removeContainerRequest(
|
|
|
- new ContainerRequest(capability, nodes, racks, priority, 2));
|
|
|
+ new ContainerRequest(capability, nodes, racks, priority));
|
|
|
+ amClient.removeContainerRequest(
|
|
|
+ new ContainerRequest(capability, nodes, racks, priority));
|
|
|
assertTrue(amClient.ask.size() == 3);
|
|
|
// send 0 container count request for resources that are no longer needed
|
|
|
ResourceRequest snoopRequest = amClient.ask.iterator().next();
|
|
@@ -554,7 +561,9 @@ public class TestAMRMClient {
|
|
|
|
|
|
// test RPC exception handling
|
|
|
amClient.addContainerRequest(new ContainerRequest(capability, nodes,
|
|
|
- racks, priority, 2));
|
|
|
+ racks, priority));
|
|
|
+ amClient.addContainerRequest(new ContainerRequest(capability, nodes,
|
|
|
+ racks, priority));
|
|
|
snoopRequest = amClient.ask.iterator().next();
|
|
|
assertTrue(snoopRequest.getNumContainers() == 2);
|
|
|
|
|
@@ -567,7 +576,9 @@ public class TestAMRMClient {
|
|
|
throws Exception {
|
|
|
amClient.removeContainerRequest(
|
|
|
new ContainerRequest(capability, nodes,
|
|
|
- racks, priority, 2));
|
|
|
+ racks, priority));
|
|
|
+ amClient.removeContainerRequest(
|
|
|
+ new ContainerRequest(capability, nodes, racks, priority));
|
|
|
throw new Exception();
|
|
|
}
|
|
|
});
|
|
@@ -585,7 +596,7 @@ public class TestAMRMClient {
|
|
|
// has not been lost
|
|
|
assertTrue(snoopRequest.getNumContainers() == 0);
|
|
|
|
|
|
- iterationsLeft = 2;
|
|
|
+ iterationsLeft = 3;
|
|
|
// do a few iterations to ensure RM is not going send new containers
|
|
|
while(!releases.isEmpty() || iterationsLeft-- > 0) {
|
|
|
// inform RM of rejection
|
|
@@ -604,7 +615,7 @@ public class TestAMRMClient {
|
|
|
}
|
|
|
if(iterationsLeft > 0) {
|
|
|
// sleep to make sure NM's heartbeat
|
|
|
- sleep(1000);
|
|
|
+ sleep(100);
|
|
|
}
|
|
|
}
|
|
|
assertTrue(amClient.ask.size() == 0);
|