|
@@ -36,6 +36,7 @@ import java.util.concurrent.Executors;
|
|
|
import org.apache.hadoop.registry.client.api.RegistryOperations;
|
|
|
import org.apache.hadoop.registry.client.impl.FSRegistryOperationsService;
|
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
|
+import org.apache.hadoop.test.LambdaTestUtils;
|
|
|
import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
|
|
|
import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
|
|
|
import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest;
|
|
@@ -61,6 +62,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
|
import org.apache.hadoop.yarn.exceptions.ApplicationMasterNotRegisteredException;
|
|
|
import org.apache.hadoop.yarn.exceptions.InvalidApplicationMasterRequestException;
|
|
|
import org.apache.hadoop.yarn.exceptions.YarnException;
|
|
|
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
|
|
|
import org.apache.hadoop.yarn.server.MockResourceManagerFacade;
|
|
|
import org.apache.hadoop.yarn.server.federation.policies.manager.UniformBroadcastPolicyManager;
|
|
|
import org.apache.hadoop.yarn.server.federation.store.impl.MemoryFederationStateStore;
|
|
@@ -970,6 +972,101 @@ public class TestFederationInterceptor extends BaseAMRMProxyTest {
|
|
|
return preemptionMessage;
|
|
|
}
|
|
|
|
|
|
+ @Test
|
|
|
+ public void testSameContainerFromDiffRM() throws IOException, InterruptedException {
|
|
|
+
|
|
|
+ UserGroupInformation ugi =
|
|
|
+ interceptor.getUGIWithToken(interceptor.getAttemptId());
|
|
|
+
|
|
|
+ ugi.doAs((PrivilegedExceptionAction<Object>) () -> {
|
|
|
+
|
|
|
+ // 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;
|
|
|
+
|
|
|
+ Assert.assertEquals(0, interceptor.getUnmanagedAMPoolSize());
|
|
|
+
|
|
|
+ // Allocate the first batch of containers, with sc1 active
|
|
|
+ SubClusterId subClusterId1 = SubClusterId.newInstance("SC-1");
|
|
|
+ registerSubCluster(subClusterId1);
|
|
|
+
|
|
|
+ int numberOfContainers = 3;
|
|
|
+ List<Container> containers =
|
|
|
+ getContainersAndAssert(numberOfContainers, numberOfContainers);
|
|
|
+ Assert.assertNotNull(containers);
|
|
|
+ Assert.assertEquals(3, containers.size());
|
|
|
+
|
|
|
+ // with sc2 active
|
|
|
+ SubClusterId subClusterId2 = SubClusterId.newInstance("SC-2");
|
|
|
+ registerSubCluster(subClusterId2);
|
|
|
+
|
|
|
+ // 1.Container has been registered to SubCluster1, try to register the same Container
|
|
|
+ // to SubCluster2.
|
|
|
+ // Because SubCluster1 is in normal state at this time,
|
|
|
+ // So the SubCluster corresponding to Container should be SubCluster1
|
|
|
+ interceptor.cacheAllocatedContainersForSubClusterId(containers, subClusterId2);
|
|
|
+ Map<ContainerId, SubClusterId> cIdToSCMap = interceptor.getContainerIdToSubClusterIdMap();
|
|
|
+ for (SubClusterId subClusterId : cIdToSCMap.values()) {
|
|
|
+ Assert.assertNotNull(subClusterId);
|
|
|
+ Assert.assertEquals(subClusterId1, subClusterId);
|
|
|
+ }
|
|
|
+
|
|
|
+ // 2.Deregister SubCluster1, Register the same Containers to SubCluster2
|
|
|
+ // So the SubCluster corresponding to Container should be SubCluster2
|
|
|
+ deRegisterSubCluster(subClusterId1);
|
|
|
+ interceptor.cacheAllocatedContainersForSubClusterId(containers, subClusterId2);
|
|
|
+ Map<ContainerId, SubClusterId> cIdToSCMap2 = interceptor.getContainerIdToSubClusterIdMap();
|
|
|
+ for (SubClusterId subClusterId : cIdToSCMap2.values()) {
|
|
|
+ Assert.assertNotNull(subClusterId);
|
|
|
+ Assert.assertEquals(subClusterId2, subClusterId);
|
|
|
+ }
|
|
|
+
|
|
|
+ // 3.Deregister subClusterId2, Register the same Containers to SubCluster1
|
|
|
+ // Because both SubCluster1 and SubCluster2 are abnormal at this time,
|
|
|
+ // an exception will be thrown when registering the first Container.
|
|
|
+ deRegisterSubCluster(subClusterId2);
|
|
|
+ Container container1 = containers.get(0);
|
|
|
+ Assert.assertNotNull(container1);
|
|
|
+ String errMsg =
|
|
|
+ " Can't use any subCluster because an exception occurred" +
|
|
|
+ " ContainerId: " + container1.getId() +
|
|
|
+ " ApplicationId: " + interceptor.getAttemptId() +
|
|
|
+ " From RM: " + subClusterId1 + ". " +
|
|
|
+ " Previous Container was From subCluster: " + subClusterId2;
|
|
|
+
|
|
|
+ LambdaTestUtils.intercept(YarnRuntimeException.class, errMsg,
|
|
|
+ () -> interceptor.cacheAllocatedContainersForSubClusterId(containers, subClusterId1));
|
|
|
+
|
|
|
+ // 4. register SubCluster1, re-register the Container,
|
|
|
+ // and try to finish application
|
|
|
+ registerSubCluster(subClusterId1);
|
|
|
+ interceptor.cacheAllocatedContainersForSubClusterId(containers, subClusterId1);
|
|
|
+ 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;
|
|
|
+ });
|
|
|
+ }
|
|
|
+
|
|
|
@Test
|
|
|
public void testBatchFinishApplicationMaster() throws IOException, InterruptedException {
|
|
|
|