|
@@ -203,7 +203,7 @@ public class TestFederationInterceptor extends BaseAMRMProxyTest {
|
|
|
List<Container> containers = new ArrayList<>(numberOfResourceRequests);
|
|
|
List<ResourceRequest> askList = new ArrayList<>(numberOfResourceRequests);
|
|
|
for (int id = 0; id < numberOfResourceRequests; id++) {
|
|
|
- askList.add(createResourceRequest("test-node-" + Integer.toString(id),
|
|
|
+ askList.add(createResourceRequest("test-node-" + id,
|
|
|
6000, 2, id % 5, 1));
|
|
|
}
|
|
|
|
|
@@ -217,7 +217,7 @@ public class TestFederationInterceptor extends BaseAMRMProxyTest {
|
|
|
|
|
|
containers.addAll(allocateResponse.getAllocatedContainers());
|
|
|
LOG.info("Number of allocated containers in the original request: "
|
|
|
- + Integer.toString(allocateResponse.getAllocatedContainers().size()));
|
|
|
+ + allocateResponse.getAllocatedContainers().size());
|
|
|
|
|
|
// Send max 10 heart beats to receive all the containers. If not, we will
|
|
|
// fail the test
|
|
@@ -236,10 +236,9 @@ public class TestFederationInterceptor extends BaseAMRMProxyTest {
|
|
|
interceptor.drainAllAsyncQueue(false);
|
|
|
|
|
|
containers.addAll(allocateResponse.getAllocatedContainers());
|
|
|
- LOG.info("Number of allocated containers in this request: "
|
|
|
- + Integer.toString(allocateResponse.getAllocatedContainers().size()));
|
|
|
- LOG.info("Total number of allocated containers: "
|
|
|
- + Integer.toString(containers.size()));
|
|
|
+ LOG.info("Number of allocated containers in this request: {}.",
|
|
|
+ allocateResponse.getAllocatedContainers().size());
|
|
|
+ LOG.info("Total number of allocated containers: {}.", containers.size());
|
|
|
Thread.sleep(10);
|
|
|
}
|
|
|
Assert.assertEquals(numberOfAllocationExcepted, containers.size());
|
|
@@ -250,7 +249,7 @@ public class TestFederationInterceptor extends BaseAMRMProxyTest {
|
|
|
throws Exception {
|
|
|
Assert.assertTrue(containers.size() > 0);
|
|
|
AllocateRequest allocateRequest = Records.newRecord(AllocateRequest.class);
|
|
|
- List<ContainerId> relList = new ArrayList<ContainerId>(containers.size());
|
|
|
+ List<ContainerId> relList = new ArrayList<>(containers.size());
|
|
|
for (Container container : containers) {
|
|
|
relList.add(container.getId());
|
|
|
}
|
|
@@ -267,8 +266,7 @@ public class TestFederationInterceptor extends BaseAMRMProxyTest {
|
|
|
// The release containers returned by the mock resource managers will be
|
|
|
// aggregated and returned back to us, and we can check if total request size
|
|
|
// and returned size are the same
|
|
|
- List<ContainerId> containersForReleasedContainerIds =
|
|
|
- new ArrayList<ContainerId>();
|
|
|
+ List<ContainerId> containersForReleasedContainerIds = new ArrayList<>();
|
|
|
List<ContainerId> newlyFinished = getCompletedContainerIds(
|
|
|
allocateResponse.getCompletedContainersStatuses());
|
|
|
containersForReleasedContainerIds.addAll(newlyFinished);
|
|
@@ -306,7 +304,7 @@ public class TestFederationInterceptor extends BaseAMRMProxyTest {
|
|
|
private void checkAMRMToken(Token amrmToken) {
|
|
|
if (amrmToken != null) {
|
|
|
// The token should be the one issued by home MockRM
|
|
|
- Assert.assertTrue(amrmToken.getKind().equals(Integer.toString(0)));
|
|
|
+ Assert.assertEquals(Integer.toString(0), amrmToken.getKind());
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -314,69 +312,66 @@ public class TestFederationInterceptor extends BaseAMRMProxyTest {
|
|
|
public void testMultipleSubClusters() throws Exception {
|
|
|
UserGroupInformation ugi =
|
|
|
interceptor.getUGIWithToken(interceptor.getAttemptId());
|
|
|
- ugi.doAs(new PrivilegedExceptionAction<Object>() {
|
|
|
- @Override
|
|
|
- public Object run() throws Exception {
|
|
|
- // Register the application
|
|
|
- RegisterApplicationMasterRequest registerReq =
|
|
|
- Records.newRecord(RegisterApplicationMasterRequest.class);
|
|
|
- registerReq.setHost(Integer.toString(testAppId));
|
|
|
- registerReq.setRpcPort(0);
|
|
|
- registerReq.setTrackingUrl("");
|
|
|
-
|
|
|
- RegisterApplicationMasterResponse registerResponse =
|
|
|
- interceptor.registerApplicationMaster(registerReq);
|
|
|
- Assert.assertNotNull(registerResponse);
|
|
|
- lastResponseId = 0;
|
|
|
+ ugi.doAs((PrivilegedExceptionAction<Object>) () -> {
|
|
|
+ // Register the application
|
|
|
+ RegisterApplicationMasterRequest registerReq =
|
|
|
+ Records.newRecord(RegisterApplicationMasterRequest.class);
|
|
|
+ registerReq.setHost(Integer.toString(testAppId));
|
|
|
+ registerReq.setRpcPort(0);
|
|
|
+ registerReq.setTrackingUrl("");
|
|
|
|
|
|
- Assert.assertEquals(0, interceptor.getUnmanagedAMPoolSize());
|
|
|
-
|
|
|
- // Allocate the first batch of containers, with sc1 and sc2 active
|
|
|
- registerSubCluster(SubClusterId.newInstance("SC-1"));
|
|
|
- registerSubCluster(SubClusterId.newInstance("SC-2"));
|
|
|
-
|
|
|
- int numberOfContainers = 3;
|
|
|
- List<Container> containers =
|
|
|
- getContainersAndAssert(numberOfContainers, numberOfContainers * 2);
|
|
|
- Assert.assertEquals(2, interceptor.getUnmanagedAMPoolSize());
|
|
|
-
|
|
|
- // Allocate the second batch of containers, with sc1 and sc3 active
|
|
|
- deRegisterSubCluster(SubClusterId.newInstance("SC-2"));
|
|
|
- registerSubCluster(SubClusterId.newInstance("SC-3"));
|
|
|
-
|
|
|
- numberOfContainers = 1;
|
|
|
- containers.addAll(
|
|
|
- getContainersAndAssert(numberOfContainers, numberOfContainers * 2));
|
|
|
- Assert.assertEquals(3, interceptor.getUnmanagedAMPoolSize());
|
|
|
-
|
|
|
- // Allocate the third batch of containers with only in home sub-cluster
|
|
|
- // active
|
|
|
- deRegisterSubCluster(SubClusterId.newInstance("SC-1"));
|
|
|
- deRegisterSubCluster(SubClusterId.newInstance("SC-3"));
|
|
|
- registerSubCluster(SubClusterId.newInstance(HOME_SC_ID));
|
|
|
-
|
|
|
- numberOfContainers = 2;
|
|
|
- containers.addAll(
|
|
|
- getContainersAndAssert(numberOfContainers, numberOfContainers * 1));
|
|
|
- Assert.assertEquals(3, interceptor.getUnmanagedAMPoolSize());
|
|
|
-
|
|
|
- // Release all containers
|
|
|
- releaseContainersAndAssert(containers);
|
|
|
-
|
|
|
- // Finish the application
|
|
|
- FinishApplicationMasterRequest finishReq =
|
|
|
- Records.newRecord(FinishApplicationMasterRequest.class);
|
|
|
- finishReq.setDiagnostics("");
|
|
|
- finishReq.setTrackingUrl("");
|
|
|
- finishReq.setFinalApplicationStatus(FinalApplicationStatus.SUCCEEDED);
|
|
|
-
|
|
|
- FinishApplicationMasterResponse finishResponse =
|
|
|
- interceptor.finishApplicationMaster(finishReq);
|
|
|
- Assert.assertNotNull(finishResponse);
|
|
|
- Assert.assertEquals(true, finishResponse.getIsUnregistered());
|
|
|
-
|
|
|
- return null;
|
|
|
- }
|
|
|
+ RegisterApplicationMasterResponse registerResponse =
|
|
|
+ interceptor.registerApplicationMaster(registerReq);
|
|
|
+ Assert.assertNotNull(registerResponse);
|
|
|
+ lastResponseId = 0;
|
|
|
+
|
|
|
+ Assert.assertEquals(0, interceptor.getUnmanagedAMPoolSize());
|
|
|
+
|
|
|
+ // Allocate the first batch of containers, with sc1 and sc2 active
|
|
|
+ registerSubCluster(SubClusterId.newInstance("SC-1"));
|
|
|
+ registerSubCluster(SubClusterId.newInstance("SC-2"));
|
|
|
+
|
|
|
+ int numberOfContainers = 3;
|
|
|
+ List<Container> containers =
|
|
|
+ getContainersAndAssert(numberOfContainers, numberOfContainers * 2);
|
|
|
+ Assert.assertEquals(2, interceptor.getUnmanagedAMPoolSize());
|
|
|
+
|
|
|
+ // Allocate the second batch of containers, with sc1 and sc3 active
|
|
|
+ deRegisterSubCluster(SubClusterId.newInstance("SC-2"));
|
|
|
+ registerSubCluster(SubClusterId.newInstance("SC-3"));
|
|
|
+
|
|
|
+ numberOfContainers = 1;
|
|
|
+ containers.addAll(
|
|
|
+ getContainersAndAssert(numberOfContainers, numberOfContainers * 2));
|
|
|
+ Assert.assertEquals(3, interceptor.getUnmanagedAMPoolSize());
|
|
|
+
|
|
|
+ // Allocate the third batch of containers with only in home sub-cluster
|
|
|
+ // active
|
|
|
+ deRegisterSubCluster(SubClusterId.newInstance("SC-1"));
|
|
|
+ deRegisterSubCluster(SubClusterId.newInstance("SC-3"));
|
|
|
+ registerSubCluster(SubClusterId.newInstance(HOME_SC_ID));
|
|
|
+
|
|
|
+ numberOfContainers = 2;
|
|
|
+ containers.addAll(
|
|
|
+ getContainersAndAssert(numberOfContainers, numberOfContainers));
|
|
|
+ Assert.assertEquals(3, interceptor.getUnmanagedAMPoolSize());
|
|
|
+
|
|
|
+ // Release all containers
|
|
|
+ releaseContainersAndAssert(containers);
|
|
|
+
|
|
|
+ // Finish the application
|
|
|
+ FinishApplicationMasterRequest finishReq =
|
|
|
+ Records.newRecord(FinishApplicationMasterRequest.class);
|
|
|
+ finishReq.setDiagnostics("");
|
|
|
+ finishReq.setTrackingUrl("");
|
|
|
+ finishReq.setFinalApplicationStatus(FinalApplicationStatus.SUCCEEDED);
|
|
|
+
|
|
|
+ FinishApplicationMasterResponse finishResponse =
|
|
|
+ interceptor.finishApplicationMaster(finishReq);
|
|
|
+ Assert.assertNotNull(finishResponse);
|
|
|
+ Assert.assertTrue(finishResponse.getIsUnregistered());
|
|
|
+
|
|
|
+ return null;
|
|
|
});
|
|
|
}
|
|
|
|
|
@@ -387,55 +382,52 @@ public class TestFederationInterceptor extends BaseAMRMProxyTest {
|
|
|
public void testReregister() throws Exception {
|
|
|
UserGroupInformation ugi =
|
|
|
interceptor.getUGIWithToken(interceptor.getAttemptId());
|
|
|
- ugi.doAs(new PrivilegedExceptionAction<Object>() {
|
|
|
- @Override
|
|
|
- public Object run() throws Exception {
|
|
|
-
|
|
|
- // Register the application
|
|
|
- RegisterApplicationMasterRequest registerReq =
|
|
|
- Records.newRecord(RegisterApplicationMasterRequest.class);
|
|
|
- registerReq.setHost(Integer.toString(testAppId));
|
|
|
- registerReq.setRpcPort(0);
|
|
|
- registerReq.setTrackingUrl("");
|
|
|
-
|
|
|
- RegisterApplicationMasterResponse registerResponse =
|
|
|
- interceptor.registerApplicationMaster(registerReq);
|
|
|
- Assert.assertNotNull(registerResponse);
|
|
|
- lastResponseId = 0;
|
|
|
+ ugi.doAs((PrivilegedExceptionAction<Object>) () -> {
|
|
|
+
|
|
|
+ // Register the application
|
|
|
+ RegisterApplicationMasterRequest registerReq =
|
|
|
+ Records.newRecord(RegisterApplicationMasterRequest.class);
|
|
|
+ registerReq.setHost(Integer.toString(testAppId));
|
|
|
+ registerReq.setRpcPort(0);
|
|
|
+ registerReq.setTrackingUrl("");
|
|
|
|
|
|
- Assert.assertEquals(0, interceptor.getUnmanagedAMPoolSize());
|
|
|
+ RegisterApplicationMasterResponse registerResponse =
|
|
|
+ interceptor.registerApplicationMaster(registerReq);
|
|
|
+ Assert.assertNotNull(registerResponse);
|
|
|
+ lastResponseId = 0;
|
|
|
|
|
|
- // Allocate the first batch of containers
|
|
|
- registerSubCluster(SubClusterId.newInstance("SC-1"));
|
|
|
- registerSubCluster(SubClusterId.newInstance(HOME_SC_ID));
|
|
|
+ Assert.assertEquals(0, interceptor.getUnmanagedAMPoolSize());
|
|
|
|
|
|
- interceptor.setShouldReRegisterNext();
|
|
|
+ // Allocate the first batch of containers
|
|
|
+ registerSubCluster(SubClusterId.newInstance("SC-1"));
|
|
|
+ registerSubCluster(SubClusterId.newInstance(HOME_SC_ID));
|
|
|
|
|
|
- int numberOfContainers = 3;
|
|
|
- List<Container> containers =
|
|
|
- getContainersAndAssert(numberOfContainers, numberOfContainers * 2);
|
|
|
- Assert.assertEquals(1, interceptor.getUnmanagedAMPoolSize());
|
|
|
+ interceptor.setShouldReRegisterNext();
|
|
|
|
|
|
- interceptor.setShouldReRegisterNext();
|
|
|
+ int numberOfContainers = 3;
|
|
|
+ List<Container> containers =
|
|
|
+ getContainersAndAssert(numberOfContainers, numberOfContainers * 2);
|
|
|
+ Assert.assertEquals(1, interceptor.getUnmanagedAMPoolSize());
|
|
|
|
|
|
- // Release all containers
|
|
|
- releaseContainersAndAssert(containers);
|
|
|
+ interceptor.setShouldReRegisterNext();
|
|
|
|
|
|
- interceptor.setShouldReRegisterNext();
|
|
|
+ // Release all containers
|
|
|
+ releaseContainersAndAssert(containers);
|
|
|
|
|
|
- // Finish the application
|
|
|
- FinishApplicationMasterRequest finishReq =
|
|
|
- Records.newRecord(FinishApplicationMasterRequest.class);
|
|
|
- finishReq.setDiagnostics("");
|
|
|
- finishReq.setTrackingUrl("");
|
|
|
- finishReq.setFinalApplicationStatus(FinalApplicationStatus.SUCCEEDED);
|
|
|
+ interceptor.setShouldReRegisterNext();
|
|
|
|
|
|
- FinishApplicationMasterResponse finishResponse =
|
|
|
- interceptor.finishApplicationMaster(finishReq);
|
|
|
- Assert.assertNotNull(finishResponse);
|
|
|
- Assert.assertTrue(finishResponse.getIsUnregistered());
|
|
|
- return null;
|
|
|
- }
|
|
|
+ // Finish the application
|
|
|
+ FinishApplicationMasterRequest finishReq =
|
|
|
+ Records.newRecord(FinishApplicationMasterRequest.class);
|
|
|
+ finishReq.setDiagnostics("");
|
|
|
+ finishReq.setTrackingUrl("");
|
|
|
+ finishReq.setFinalApplicationStatus(FinalApplicationStatus.SUCCEEDED);
|
|
|
+
|
|
|
+ FinishApplicationMasterResponse finishResponse =
|
|
|
+ interceptor.finishApplicationMaster(finishReq);
|
|
|
+ Assert.assertNotNull(finishResponse);
|
|
|
+ Assert.assertTrue(finishResponse.getIsUnregistered());
|
|
|
+ return null;
|
|
|
});
|
|
|
}
|
|
|
|
|
@@ -498,7 +490,7 @@ public class TestFederationInterceptor extends BaseAMRMProxyTest {
|
|
|
implements Callable<RegisterApplicationMasterResponse> {
|
|
|
@Override
|
|
|
public RegisterApplicationMasterResponse call() throws Exception {
|
|
|
- RegisterApplicationMasterResponse response = null;
|
|
|
+ RegisterApplicationMasterResponse response;
|
|
|
try {
|
|
|
// Use port number 1001 to let mock RM block in the register call
|
|
|
response = interceptor.registerApplicationMaster(
|
|
@@ -536,110 +528,107 @@ public class TestFederationInterceptor extends BaseAMRMProxyTest {
|
|
|
throws Exception {
|
|
|
UserGroupInformation ugi =
|
|
|
interceptor.getUGIWithToken(interceptor.getAttemptId());
|
|
|
- ugi.doAs(new PrivilegedExceptionAction<Object>() {
|
|
|
- @Override
|
|
|
- public Object run() throws Exception {
|
|
|
- interceptor = new TestableFederationInterceptor();
|
|
|
- interceptor.init(new AMRMProxyApplicationContextImpl(nmContext,
|
|
|
- getConf(), attemptId, "test-user", null, null, null, registryObj));
|
|
|
- interceptor.cleanupRegistry();
|
|
|
-
|
|
|
- // Register the application
|
|
|
- RegisterApplicationMasterRequest registerReq =
|
|
|
- Records.newRecord(RegisterApplicationMasterRequest.class);
|
|
|
- registerReq.setHost(Integer.toString(testAppId));
|
|
|
- registerReq.setRpcPort(testAppId);
|
|
|
- registerReq.setTrackingUrl("");
|
|
|
-
|
|
|
- RegisterApplicationMasterResponse registerResponse =
|
|
|
- interceptor.registerApplicationMaster(registerReq);
|
|
|
- Assert.assertNotNull(registerResponse);
|
|
|
- lastResponseId = 0;
|
|
|
+ ugi.doAs((PrivilegedExceptionAction<Object>) () -> {
|
|
|
+ interceptor = new TestableFederationInterceptor();
|
|
|
+ interceptor.init(new AMRMProxyApplicationContextImpl(nmContext,
|
|
|
+ getConf(), attemptId, "test-user", null, null, null, registryObj));
|
|
|
+ interceptor.cleanupRegistry();
|
|
|
|
|
|
- Assert.assertEquals(0, interceptor.getUnmanagedAMPoolSize());
|
|
|
+ // Register the application
|
|
|
+ RegisterApplicationMasterRequest registerReq =
|
|
|
+ Records.newRecord(RegisterApplicationMasterRequest.class);
|
|
|
+ registerReq.setHost(Integer.toString(testAppId));
|
|
|
+ registerReq.setRpcPort(testAppId);
|
|
|
+ registerReq.setTrackingUrl("");
|
|
|
|
|
|
- // Allocate one batch of containers
|
|
|
- registerSubCluster(SubClusterId.newInstance("SC-1"));
|
|
|
- registerSubCluster(SubClusterId.newInstance(HOME_SC_ID));
|
|
|
+ RegisterApplicationMasterResponse registerResponse =
|
|
|
+ interceptor.registerApplicationMaster(registerReq);
|
|
|
+ Assert.assertNotNull(registerResponse);
|
|
|
+ lastResponseId = 0;
|
|
|
|
|
|
- int numberOfContainers = 3;
|
|
|
- List<Container> containers =
|
|
|
- getContainersAndAssert(numberOfContainers, numberOfContainers * 2);
|
|
|
- Assert.assertEquals(1, interceptor.getUnmanagedAMPoolSize());
|
|
|
+ Assert.assertEquals(0, interceptor.getUnmanagedAMPoolSize());
|
|
|
|
|
|
- // Make sure all async hb threads are done
|
|
|
- interceptor.drainAllAsyncQueue(true);
|
|
|
+ // Allocate one batch of containers
|
|
|
+ registerSubCluster(SubClusterId.newInstance("SC-1"));
|
|
|
+ registerSubCluster(SubClusterId.newInstance(HOME_SC_ID));
|
|
|
|
|
|
- // Prepare for Federation Interceptor restart and recover
|
|
|
- Map<String, byte[]> recoveredDataMap =
|
|
|
- recoverDataMapForAppAttempt(nmStateStore, attemptId);
|
|
|
- String scEntry =
|
|
|
- FederationInterceptor.NMSS_SECONDARY_SC_PREFIX + "SC-1";
|
|
|
- if (registryObj == null) {
|
|
|
- Assert.assertTrue(recoveredDataMap.containsKey(scEntry));
|
|
|
- } else {
|
|
|
- // When AMRMPRoxy HA is enabled, NMSS should not have the UAM token,
|
|
|
- // it should be in Registry
|
|
|
- Assert.assertFalse(recoveredDataMap.containsKey(scEntry));
|
|
|
- }
|
|
|
-
|
|
|
- // Preserve the mock RM instances
|
|
|
- MockResourceManagerFacade homeRM = interceptor.getHomeRM();
|
|
|
- ConcurrentHashMap<String, MockResourceManagerFacade> secondaries =
|
|
|
- interceptor.getSecondaryRMs();
|
|
|
-
|
|
|
- // Create a new interceptor instance and recover
|
|
|
- interceptor = new TestableFederationInterceptor(homeRM, secondaries);
|
|
|
- interceptor.init(new AMRMProxyApplicationContextImpl(nmContext,
|
|
|
- getConf(), attemptId, "test-user", null, null, null, registryObj));
|
|
|
- interceptor.recover(recoveredDataMap);
|
|
|
-
|
|
|
- Assert.assertEquals(1, interceptor.getUnmanagedAMPoolSize());
|
|
|
- // SC1 should be initialized to be timed out
|
|
|
- Assert.assertEquals(1, interceptor.getTimedOutSCs(true).size());
|
|
|
-
|
|
|
- // The first allocate call expects a fail-over exception and re-register
|
|
|
- try {
|
|
|
- AllocateRequest allocateRequest =
|
|
|
- Records.newRecord(AllocateRequest.class);
|
|
|
- allocateRequest.setResponseId(lastResponseId);
|
|
|
- AllocateResponse allocateResponse =
|
|
|
- interceptor.allocate(allocateRequest);
|
|
|
- lastResponseId = allocateResponse.getResponseId();
|
|
|
- Assert.fail("Expecting an ApplicationMasterNotRegisteredException "
|
|
|
- + " after FederationInterceptor restarts and recovers");
|
|
|
- } catch (ApplicationMasterNotRegisteredException e) {
|
|
|
- }
|
|
|
- interceptor.registerApplicationMaster(registerReq);
|
|
|
- lastResponseId = 0;
|
|
|
+ int numberOfContainers = 3;
|
|
|
+ List<Container> containers =
|
|
|
+ getContainersAndAssert(numberOfContainers, numberOfContainers * 2);
|
|
|
+ Assert.assertEquals(1, interceptor.getUnmanagedAMPoolSize());
|
|
|
|
|
|
- // Release all containers
|
|
|
- releaseContainersAndAssert(containers);
|
|
|
-
|
|
|
- // Finish the application
|
|
|
- FinishApplicationMasterRequest finishReq =
|
|
|
- Records.newRecord(FinishApplicationMasterRequest.class);
|
|
|
- finishReq.setDiagnostics("");
|
|
|
- finishReq.setTrackingUrl("");
|
|
|
- finishReq.setFinalApplicationStatus(FinalApplicationStatus.SUCCEEDED);
|
|
|
-
|
|
|
- FinishApplicationMasterResponse finishResponse =
|
|
|
- interceptor.finishApplicationMaster(finishReq);
|
|
|
- Assert.assertNotNull(finishResponse);
|
|
|
- Assert.assertTrue(finishResponse.getIsUnregistered());
|
|
|
-
|
|
|
- // After the application succeeds, the registry/NMSS entry should be
|
|
|
- // cleaned up
|
|
|
- if (registryObj != null) {
|
|
|
- Assert.assertEquals(0,
|
|
|
- interceptor.getRegistryClient().getAllApplications().size());
|
|
|
- } else {
|
|
|
- recoveredDataMap =
|
|
|
- recoverDataMapForAppAttempt(nmStateStore, attemptId);
|
|
|
- Assert.assertFalse(recoveredDataMap.containsKey(scEntry));
|
|
|
- }
|
|
|
- return null;
|
|
|
+ // Make sure all async hb threads are done
|
|
|
+ interceptor.drainAllAsyncQueue(true);
|
|
|
+
|
|
|
+ // Prepare for Federation Interceptor restart and recover
|
|
|
+ Map<String, byte[]> recoveredDataMap =
|
|
|
+ recoverDataMapForAppAttempt(nmStateStore, attemptId);
|
|
|
+ String scEntry =
|
|
|
+ FederationInterceptor.NMSS_SECONDARY_SC_PREFIX + "SC-1";
|
|
|
+ if (registryObj == null) {
|
|
|
+ Assert.assertTrue(recoveredDataMap.containsKey(scEntry));
|
|
|
+ } else {
|
|
|
+ // When AMRMPRoxy HA is enabled, NMSS should not have the UAM token,
|
|
|
+ // it should be in Registry
|
|
|
+ Assert.assertFalse(recoveredDataMap.containsKey(scEntry));
|
|
|
}
|
|
|
+
|
|
|
+ // Preserve the mock RM instances
|
|
|
+ MockResourceManagerFacade homeRM = interceptor.getHomeRM();
|
|
|
+ ConcurrentHashMap<String, MockResourceManagerFacade> secondaries =
|
|
|
+ interceptor.getSecondaryRMs();
|
|
|
+
|
|
|
+ // Create a new interceptor instance and recover
|
|
|
+ interceptor = new TestableFederationInterceptor(homeRM, secondaries);
|
|
|
+ interceptor.init(new AMRMProxyApplicationContextImpl(nmContext,
|
|
|
+ getConf(), attemptId, "test-user", null, null, null, registryObj));
|
|
|
+ interceptor.recover(recoveredDataMap);
|
|
|
+
|
|
|
+ Assert.assertEquals(1, interceptor.getUnmanagedAMPoolSize());
|
|
|
+ // SC1 should be initialized to be timed out
|
|
|
+ Assert.assertEquals(1, interceptor.getTimedOutSCs(true).size());
|
|
|
+
|
|
|
+ // The first allocate call expects a fail-over exception and re-register
|
|
|
+ try {
|
|
|
+ AllocateRequest allocateRequest =
|
|
|
+ Records.newRecord(AllocateRequest.class);
|
|
|
+ allocateRequest.setResponseId(lastResponseId);
|
|
|
+ AllocateResponse allocateResponse =
|
|
|
+ interceptor.allocate(allocateRequest);
|
|
|
+ lastResponseId = allocateResponse.getResponseId();
|
|
|
+ Assert.fail("Expecting an ApplicationMasterNotRegisteredException "
|
|
|
+ + " after FederationInterceptor restarts and recovers");
|
|
|
+ } catch (ApplicationMasterNotRegisteredException e) {
|
|
|
+ }
|
|
|
+ interceptor.registerApplicationMaster(registerReq);
|
|
|
+ lastResponseId = 0;
|
|
|
+
|
|
|
+ // Release all containers
|
|
|
+ releaseContainersAndAssert(containers);
|
|
|
+
|
|
|
+ // Finish the application
|
|
|
+ FinishApplicationMasterRequest finishReq =
|
|
|
+ Records.newRecord(FinishApplicationMasterRequest.class);
|
|
|
+ finishReq.setDiagnostics("");
|
|
|
+ finishReq.setTrackingUrl("");
|
|
|
+ finishReq.setFinalApplicationStatus(FinalApplicationStatus.SUCCEEDED);
|
|
|
+
|
|
|
+ FinishApplicationMasterResponse finishResponse =
|
|
|
+ interceptor.finishApplicationMaster(finishReq);
|
|
|
+ Assert.assertNotNull(finishResponse);
|
|
|
+ Assert.assertTrue(finishResponse.getIsUnregistered());
|
|
|
+
|
|
|
+ // After the application succeeds, the registry/NMSS entry should be
|
|
|
+ // cleaned up
|
|
|
+ if (registryObj != null) {
|
|
|
+ Assert.assertEquals(0,
|
|
|
+ interceptor.getRegistryClient().getAllApplications().size());
|
|
|
+ } else {
|
|
|
+ recoveredDataMap =
|
|
|
+ recoverDataMapForAppAttempt(nmStateStore, attemptId);
|
|
|
+ Assert.assertFalse(recoveredDataMap.containsKey(scEntry));
|
|
|
+ }
|
|
|
+ return null;
|
|
|
});
|
|
|
}
|
|
|
|
|
@@ -774,53 +763,48 @@ public class TestFederationInterceptor extends BaseAMRMProxyTest {
|
|
|
public void testSubClusterTimeOut() throws Exception {
|
|
|
UserGroupInformation ugi =
|
|
|
interceptor.getUGIWithToken(interceptor.getAttemptId());
|
|
|
- ugi.doAs(new PrivilegedExceptionAction<Object>() {
|
|
|
- @Override
|
|
|
- public Object run() throws Exception {
|
|
|
- // Register the application first time
|
|
|
- RegisterApplicationMasterRequest registerReq =
|
|
|
- Records.newRecord(RegisterApplicationMasterRequest.class);
|
|
|
- registerReq.setHost(Integer.toString(testAppId));
|
|
|
- registerReq.setRpcPort(0);
|
|
|
- registerReq.setTrackingUrl("");
|
|
|
- RegisterApplicationMasterResponse registerResponse =
|
|
|
- interceptor.registerApplicationMaster(registerReq);
|
|
|
- Assert.assertNotNull(registerResponse);
|
|
|
- lastResponseId = 0;
|
|
|
+ ugi.doAs((PrivilegedExceptionAction<Object>) () -> {
|
|
|
+ // Register the application first time
|
|
|
+ RegisterApplicationMasterRequest registerReq =
|
|
|
+ Records.newRecord(RegisterApplicationMasterRequest.class);
|
|
|
+ registerReq.setHost(Integer.toString(testAppId));
|
|
|
+ registerReq.setRpcPort(0);
|
|
|
+ registerReq.setTrackingUrl("");
|
|
|
+ RegisterApplicationMasterResponse registerResponse =
|
|
|
+ interceptor.registerApplicationMaster(registerReq);
|
|
|
+ Assert.assertNotNull(registerResponse);
|
|
|
+ lastResponseId = 0;
|
|
|
|
|
|
- registerSubCluster(SubClusterId.newInstance("SC-1"));
|
|
|
+ registerSubCluster(SubClusterId.newInstance("SC-1"));
|
|
|
|
|
|
- getContainersAndAssert(1, 1);
|
|
|
+ getContainersAndAssert(1, 1);
|
|
|
|
|
|
- AllocateResponse allocateResponse =
|
|
|
- interceptor.generateBaseAllocationResponse();
|
|
|
- Assert.assertEquals(2, allocateResponse.getNumClusterNodes());
|
|
|
- Assert.assertEquals(0, interceptor.getTimedOutSCs(true).size());
|
|
|
+ AllocateResponse allocateResponse = interceptor.generateBaseAllocationResponse();
|
|
|
+ Assert.assertEquals(2, allocateResponse.getNumClusterNodes());
|
|
|
+ Assert.assertEquals(0, interceptor.getTimedOutSCs(true).size());
|
|
|
|
|
|
- // Let all SC timeout (home and SC-1), without an allocate from AM
|
|
|
- Thread.sleep(800);
|
|
|
+ // Let all SC timeout (home and SC-1), without an allocate from AM
|
|
|
+ Thread.sleep(800);
|
|
|
|
|
|
- // Should not be considered timeout, because there's no recent AM
|
|
|
- // heartbeat
|
|
|
- allocateResponse = interceptor.generateBaseAllocationResponse();
|
|
|
- Assert.assertEquals(2, allocateResponse.getNumClusterNodes());
|
|
|
- Assert.assertEquals(0, interceptor.getTimedOutSCs(true).size());
|
|
|
+ // Should not be considered timeout, because there's no recent AM
|
|
|
+ // heartbeat
|
|
|
+ allocateResponse = interceptor.generateBaseAllocationResponse();
|
|
|
+ Assert.assertEquals(2, allocateResponse.getNumClusterNodes());
|
|
|
+ Assert.assertEquals(0, interceptor.getTimedOutSCs(true).size());
|
|
|
|
|
|
- // Generate a duplicate heartbeat from AM, so that it won't really
|
|
|
- // trigger a heartbeat to all SC
|
|
|
- AllocateRequest allocateRequest =
|
|
|
- Records.newRecord(AllocateRequest.class);
|
|
|
- // Set to lastResponseId - 1 so that it will be considered a duplicate
|
|
|
- // heartbeat and thus not forwarded to all SCs
|
|
|
- allocateRequest.setResponseId(lastResponseId - 1);
|
|
|
- interceptor.allocate(allocateRequest);
|
|
|
-
|
|
|
- // Should be considered timeout
|
|
|
- allocateResponse = interceptor.generateBaseAllocationResponse();
|
|
|
- Assert.assertEquals(0, allocateResponse.getNumClusterNodes());
|
|
|
- Assert.assertEquals(2, interceptor.getTimedOutSCs(true).size());
|
|
|
- return null;
|
|
|
- }
|
|
|
+ // Generate a duplicate heartbeat from AM, so that it won't really
|
|
|
+ // trigger a heartbeat to all SC
|
|
|
+ AllocateRequest allocateRequest = Records.newRecord(AllocateRequest.class);
|
|
|
+ // Set to lastResponseId - 1 so that it will be considered a duplicate
|
|
|
+ // heartbeat and thus not forwarded to all SCs
|
|
|
+ allocateRequest.setResponseId(lastResponseId - 1);
|
|
|
+ interceptor.allocate(allocateRequest);
|
|
|
+
|
|
|
+ // Should be considered timeout
|
|
|
+ allocateResponse = interceptor.generateBaseAllocationResponse();
|
|
|
+ Assert.assertEquals(0, allocateResponse.getNumClusterNodes());
|
|
|
+ Assert.assertEquals(2, interceptor.getTimedOutSCs(true).size());
|
|
|
+ return null;
|
|
|
});
|
|
|
}
|
|
|
|
|
@@ -834,87 +818,81 @@ public class TestFederationInterceptor extends BaseAMRMProxyTest {
|
|
|
|
|
|
UserGroupInformation ugi =
|
|
|
interceptor.getUGIWithToken(interceptor.getAttemptId());
|
|
|
- ugi.doAs(new PrivilegedExceptionAction<Object>() {
|
|
|
- @Override
|
|
|
- public Object run() throws Exception {
|
|
|
- // Register the application
|
|
|
- RegisterApplicationMasterResponse registerResponse =
|
|
|
- interceptor.registerApplicationMaster(registerReq);
|
|
|
- Assert.assertNotNull(registerResponse);
|
|
|
- lastResponseId = 0;
|
|
|
+ ugi.doAs((PrivilegedExceptionAction<Object>) () -> {
|
|
|
+ // Register the application
|
|
|
+ RegisterApplicationMasterResponse registerResponse =
|
|
|
+ interceptor.registerApplicationMaster(registerReq);
|
|
|
+ Assert.assertNotNull(registerResponse);
|
|
|
+ lastResponseId = 0;
|
|
|
|
|
|
- Assert.assertEquals(0, interceptor.getUnmanagedAMPoolSize());
|
|
|
+ Assert.assertEquals(0, interceptor.getUnmanagedAMPoolSize());
|
|
|
|
|
|
- // Allocate one batch of containers
|
|
|
- registerSubCluster(SubClusterId.newInstance("SC-1"));
|
|
|
- registerSubCluster(SubClusterId.newInstance(HOME_SC_ID));
|
|
|
+ // Allocate one batch of containers
|
|
|
+ registerSubCluster(SubClusterId.newInstance("SC-1"));
|
|
|
+ registerSubCluster(SubClusterId.newInstance(HOME_SC_ID));
|
|
|
|
|
|
- int numberOfContainers = 3;
|
|
|
- List<Container> containers =
|
|
|
- getContainersAndAssert(numberOfContainers, numberOfContainers * 2);
|
|
|
- for (Container c : containers) {
|
|
|
- LOG.info("Allocated container " + c.getId());
|
|
|
- }
|
|
|
- Assert.assertEquals(1, interceptor.getUnmanagedAMPoolSize());
|
|
|
+ int numberOfContainers = 3;
|
|
|
+ List<Container> containers =
|
|
|
+ getContainersAndAssert(numberOfContainers, numberOfContainers * 2);
|
|
|
+ for (Container c : containers) {
|
|
|
+ LOG.info("Allocated container {}.", c.getId());
|
|
|
+ }
|
|
|
+ Assert.assertEquals(1, interceptor.getUnmanagedAMPoolSize());
|
|
|
|
|
|
- // Make sure all async hb threads are done
|
|
|
- interceptor.drainAllAsyncQueue(true);
|
|
|
+ // Make sure all async hb threads are done
|
|
|
+ interceptor.drainAllAsyncQueue(true);
|
|
|
|
|
|
- // Preserve the mock RM instances for secondaries
|
|
|
- ConcurrentHashMap<String, MockResourceManagerFacade> secondaries =
|
|
|
- interceptor.getSecondaryRMs();
|
|
|
+ // Preserve the mock RM instances for secondaries
|
|
|
+ ConcurrentHashMap<String, MockResourceManagerFacade> secondaries =
|
|
|
+ interceptor.getSecondaryRMs();
|
|
|
|
|
|
- // Increase the attemptId and create a new interceptor instance for it
|
|
|
- attemptId = ApplicationAttemptId.newInstance(
|
|
|
- attemptId.getApplicationId(), attemptId.getAttemptId() + 1);
|
|
|
+ // Increase the attemptId and create a new interceptor instance for it
|
|
|
+ attemptId = ApplicationAttemptId.newInstance(
|
|
|
+ attemptId.getApplicationId(), attemptId.getAttemptId() + 1);
|
|
|
|
|
|
- interceptor = new TestableFederationInterceptor(null, secondaries);
|
|
|
- interceptor.init(new AMRMProxyApplicationContextImpl(nmContext,
|
|
|
- getConf(), attemptId, "test-user", null, null, null, registry));
|
|
|
- return null;
|
|
|
- }
|
|
|
+ interceptor = new TestableFederationInterceptor(null, secondaries);
|
|
|
+ interceptor.init(new AMRMProxyApplicationContextImpl(nmContext,
|
|
|
+ getConf(), attemptId, "test-user", null, null, null, registry));
|
|
|
+ return null;
|
|
|
});
|
|
|
|
|
|
// Update the ugi with new attemptId
|
|
|
ugi = interceptor.getUGIWithToken(interceptor.getAttemptId());
|
|
|
- ugi.doAs(new PrivilegedExceptionAction<Object>() {
|
|
|
- @Override
|
|
|
- public Object run() throws Exception {
|
|
|
- RegisterApplicationMasterResponse registerResponse =
|
|
|
- interceptor.registerApplicationMaster(registerReq);
|
|
|
- lastResponseId = 0;
|
|
|
+ ugi.doAs((PrivilegedExceptionAction<Object>) () -> {
|
|
|
+ RegisterApplicationMasterResponse registerResponse =
|
|
|
+ interceptor.registerApplicationMaster(registerReq);
|
|
|
+ lastResponseId = 0;
|
|
|
+
|
|
|
+ int numberOfContainers = 3;
|
|
|
+ // Should re-attach secondaries and get the three running containers
|
|
|
+ Assert.assertEquals(1, interceptor.getUnmanagedAMPoolSize());
|
|
|
+ // SC1 should be initialized to be timed out
|
|
|
+ Assert.assertEquals(1, interceptor.getTimedOutSCs(true).size());
|
|
|
+ Assert.assertEquals(numberOfContainers,
|
|
|
+ registerResponse.getContainersFromPreviousAttempts().size());
|
|
|
|
|
|
- int numberOfContainers = 3;
|
|
|
- // Should re-attach secondaries and get the three running containers
|
|
|
- Assert.assertEquals(1, interceptor.getUnmanagedAMPoolSize());
|
|
|
- // SC1 should be initialized to be timed out
|
|
|
- Assert.assertEquals(1, interceptor.getTimedOutSCs(true).size());
|
|
|
- Assert.assertEquals(numberOfContainers,
|
|
|
- registerResponse.getContainersFromPreviousAttempts().size());
|
|
|
-
|
|
|
- // Release all containers
|
|
|
- releaseContainersAndAssert(
|
|
|
- registerResponse.getContainersFromPreviousAttempts());
|
|
|
-
|
|
|
- // Finish the application
|
|
|
- FinishApplicationMasterRequest finishReq =
|
|
|
- Records.newRecord(FinishApplicationMasterRequest.class);
|
|
|
- finishReq.setDiagnostics("");
|
|
|
- finishReq.setTrackingUrl("");
|
|
|
- finishReq.setFinalApplicationStatus(FinalApplicationStatus.SUCCEEDED);
|
|
|
-
|
|
|
- FinishApplicationMasterResponse finishResponse =
|
|
|
- interceptor.finishApplicationMaster(finishReq);
|
|
|
- Assert.assertNotNull(finishResponse);
|
|
|
- Assert.assertTrue(finishResponse.getIsUnregistered());
|
|
|
-
|
|
|
- // After the application succeeds, the registry entry should be deleted
|
|
|
- if (interceptor.getRegistryClient() != null) {
|
|
|
- Assert.assertEquals(0,
|
|
|
- interceptor.getRegistryClient().getAllApplications().size());
|
|
|
- }
|
|
|
- return null;
|
|
|
+ // Release all containers
|
|
|
+ releaseContainersAndAssert(
|
|
|
+ registerResponse.getContainersFromPreviousAttempts());
|
|
|
+
|
|
|
+ // Finish the application
|
|
|
+ FinishApplicationMasterRequest finishReq =
|
|
|
+ Records.newRecord(FinishApplicationMasterRequest.class);
|
|
|
+ finishReq.setDiagnostics("");
|
|
|
+ finishReq.setTrackingUrl("");
|
|
|
+ finishReq.setFinalApplicationStatus(FinalApplicationStatus.SUCCEEDED);
|
|
|
+
|
|
|
+ FinishApplicationMasterResponse finishResponse =
|
|
|
+ interceptor.finishApplicationMaster(finishReq);
|
|
|
+ Assert.assertNotNull(finishResponse);
|
|
|
+ Assert.assertTrue(finishResponse.getIsUnregistered());
|
|
|
+
|
|
|
+ // After the application succeeds, the registry entry should be deleted
|
|
|
+ if (interceptor.getRegistryClient() != null) {
|
|
|
+ Assert.assertEquals(0,
|
|
|
+ interceptor.getRegistryClient().getAllApplications().size());
|
|
|
}
|
|
|
+ return null;
|
|
|
});
|
|
|
}
|
|
|
|
|
@@ -924,22 +902,17 @@ public class TestFederationInterceptor extends BaseAMRMProxyTest {
|
|
|
ContainerStatus cStatus = Records.newRecord(ContainerStatus.class);
|
|
|
cStatus.setContainerId(cid);
|
|
|
Container container =
|
|
|
- Container.newInstance(cid, null, null, null, null, null);
|
|
|
-
|
|
|
+ Container.newInstance(cid, null, null, null, null, null);
|
|
|
|
|
|
AllocateResponse homeResponse = Records.newRecord(AllocateResponse.class);
|
|
|
homeResponse.setAllocatedContainers(Collections.singletonList(container));
|
|
|
- homeResponse.setCompletedContainersStatuses(
|
|
|
- Collections.singletonList(cStatus));
|
|
|
- homeResponse.setUpdatedNodes(
|
|
|
- Collections.singletonList(Records.newRecord(NodeReport.class)));
|
|
|
- homeResponse.setNMTokens(
|
|
|
- Collections.singletonList(Records.newRecord(NMToken.class)));
|
|
|
- homeResponse.setUpdatedContainers(
|
|
|
- Collections.singletonList(
|
|
|
- Records.newRecord(UpdatedContainer.class)));
|
|
|
- homeResponse.setUpdateErrors(Collections
|
|
|
- .singletonList(Records.newRecord(UpdateContainerError.class)));
|
|
|
+ homeResponse.setCompletedContainersStatuses(Collections.singletonList(cStatus));
|
|
|
+ homeResponse.setUpdatedNodes(Collections.singletonList(Records.newRecord(NodeReport.class)));
|
|
|
+ homeResponse.setNMTokens(Collections.singletonList(Records.newRecord(NMToken.class)));
|
|
|
+ homeResponse.setUpdatedContainers(Collections.singletonList(
|
|
|
+ Records.newRecord(UpdatedContainer.class)));
|
|
|
+ homeResponse.setUpdateErrors(Collections.singletonList(
|
|
|
+ Records.newRecord(UpdateContainerError.class)));
|
|
|
homeResponse.setAvailableResources(Records.newRecord(Resource.class));
|
|
|
homeResponse.setPreemptionMessage(createDummyPreemptionMessage(
|
|
|
ContainerId.newContainerId(attemptId, 0)));
|
|
@@ -947,15 +920,12 @@ public class TestFederationInterceptor extends BaseAMRMProxyTest {
|
|
|
AllocateResponse response = Records.newRecord(AllocateResponse.class);
|
|
|
response.setAllocatedContainers(Collections.singletonList(container));
|
|
|
response.setCompletedContainersStatuses(Collections.singletonList(cStatus));
|
|
|
- response.setUpdatedNodes(
|
|
|
- Collections.singletonList(Records.newRecord(NodeReport.class)));
|
|
|
- response.setNMTokens(
|
|
|
- Collections.singletonList(Records.newRecord(NMToken.class)));
|
|
|
- response.setUpdatedContainers(
|
|
|
- Collections.singletonList(
|
|
|
- Records.newRecord(UpdatedContainer.class)));
|
|
|
- response.setUpdateErrors(Collections
|
|
|
- .singletonList(Records.newRecord(UpdateContainerError.class)));
|
|
|
+ response.setUpdatedNodes(Collections.singletonList(Records.newRecord(NodeReport.class)));
|
|
|
+ response.setNMTokens(Collections.singletonList(Records.newRecord(NMToken.class)));
|
|
|
+ response.setUpdatedContainers(Collections.singletonList(
|
|
|
+ Records.newRecord(UpdatedContainer.class)));
|
|
|
+ response.setUpdateErrors(Collections.singletonList(
|
|
|
+ Records.newRecord(UpdateContainerError.class)));
|
|
|
response.setAvailableResources(Records.newRecord(Resource.class));
|
|
|
response.setPreemptionMessage(createDummyPreemptionMessage(
|
|
|
ContainerId.newContainerId(attemptId, 1)));
|
|
@@ -964,14 +934,10 @@ public class TestFederationInterceptor extends BaseAMRMProxyTest {
|
|
|
response, SubClusterId.newInstance("SC-1"));
|
|
|
|
|
|
Assert.assertEquals(2,
|
|
|
- homeResponse.getPreemptionMessage().getContract()
|
|
|
- .getContainers().size());
|
|
|
- Assert.assertEquals(2,
|
|
|
- homeResponse.getAllocatedContainers().size());
|
|
|
- Assert.assertEquals(2,
|
|
|
- homeResponse.getUpdatedNodes().size());
|
|
|
- Assert.assertEquals(2,
|
|
|
- homeResponse.getCompletedContainersStatuses().size());
|
|
|
+ homeResponse.getPreemptionMessage().getContract().getContainers().size());
|
|
|
+ Assert.assertEquals(2, homeResponse.getAllocatedContainers().size());
|
|
|
+ Assert.assertEquals(2, homeResponse.getUpdatedNodes().size());
|
|
|
+ Assert.assertEquals(2, homeResponse.getCompletedContainersStatuses().size());
|
|
|
}
|
|
|
|
|
|
private PreemptionMessage createDummyPreemptionMessage(
|