|
@@ -24,86 +24,29 @@ import java.io.IOException;
|
|
|
import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
-import org.apache.hadoop.ha.ClientBaseWithFixes;
|
|
|
-import org.apache.hadoop.ha.HAServiceProtocol;
|
|
|
-import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
|
|
|
-import org.apache.hadoop.ha.HAServiceProtocol.StateChangeRequestInfo;
|
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
|
import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationRequest;
|
|
|
import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationResponse;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
|
|
-import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
|
|
|
-import org.apache.hadoop.yarn.conf.HAUtil;
|
|
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
|
import org.apache.hadoop.yarn.exceptions.ApplicationNotFoundException;
|
|
|
import org.apache.hadoop.yarn.exceptions.YarnException;
|
|
|
-import org.apache.hadoop.yarn.server.resourcemanager.recovery.ZKRMStateStore;
|
|
|
-import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
|
|
|
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
|
|
|
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.security.QueueACLsManager;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.security.RMDelegationTokenSecretManager;
|
|
|
import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
|
|
|
-import org.junit.After;
|
|
|
import org.junit.Assert;
|
|
|
-import org.junit.Before;
|
|
|
import org.junit.Test;
|
|
|
|
|
|
|
|
|
-public class TestKillApplicationWithRMHA extends ClientBaseWithFixes{
|
|
|
+public class TestKillApplicationWithRMHA extends RMHATestBase{
|
|
|
|
|
|
public static final Log LOG = LogFactory
|
|
|
.getLog(TestKillApplicationWithRMHA.class);
|
|
|
- private static final int ZK_TIMEOUT_MS = 5000;
|
|
|
- private static StateChangeRequestInfo requestInfo =
|
|
|
- new StateChangeRequestInfo(
|
|
|
- HAServiceProtocol.RequestSource.REQUEST_BY_USER);
|
|
|
- private Configuration configuration = new YarnConfiguration();
|
|
|
- static MockRM rm1 = null;
|
|
|
- static MockRM rm2 = null;
|
|
|
- Configuration confForRM1;
|
|
|
- Configuration confForRM2;
|
|
|
-
|
|
|
- @Before
|
|
|
- public void setup() throws Exception {
|
|
|
- configuration.setBoolean(YarnConfiguration.RM_HA_ENABLED, true);
|
|
|
- configuration.set(YarnConfiguration.RM_HA_IDS, "rm1,rm2");
|
|
|
- configuration.setBoolean(YarnConfiguration.RECOVERY_ENABLED, true);
|
|
|
- configuration.set(YarnConfiguration.RM_STORE,
|
|
|
- ZKRMStateStore.class.getName());
|
|
|
- configuration.set(YarnConfiguration.RM_ZK_ADDRESS, hostPort);
|
|
|
- configuration.setInt(YarnConfiguration.RM_ZK_TIMEOUT_MS, ZK_TIMEOUT_MS);
|
|
|
- configuration.setBoolean(YarnConfiguration.AUTO_FAILOVER_ENABLED, false);
|
|
|
- configuration.set(YarnConfiguration.RM_CLUSTER_ID, "test-yarn-cluster");
|
|
|
- int base = 100;
|
|
|
- for (String confKey : YarnConfiguration
|
|
|
- .getServiceAddressConfKeys(configuration)) {
|
|
|
- configuration.set(HAUtil.addSuffix(confKey, "rm1"), "0.0.0.0:"
|
|
|
- + (base + 20));
|
|
|
- configuration.set(HAUtil.addSuffix(confKey, "rm2"), "0.0.0.0:"
|
|
|
- + (base + 40));
|
|
|
- base = base * 2;
|
|
|
- }
|
|
|
- confForRM1 = new Configuration(configuration);
|
|
|
- confForRM1.set(YarnConfiguration.RM_HA_ID, "rm1");
|
|
|
- confForRM2 = new Configuration(configuration);
|
|
|
- confForRM2.set(YarnConfiguration.RM_HA_ID, "rm2");
|
|
|
- }
|
|
|
-
|
|
|
- @After
|
|
|
- public void teardown() {
|
|
|
- if (rm1 != null) {
|
|
|
- rm1.stop();
|
|
|
- }
|
|
|
- if (rm2 != null) {
|
|
|
- rm2.stop();
|
|
|
- }
|
|
|
- }
|
|
|
|
|
|
@Test (timeout = 20000)
|
|
|
public void testKillAppWhenFailoverHappensAtNewState()
|
|
@@ -221,18 +164,6 @@ public class TestKillApplicationWithRMHA extends ClientBaseWithFixes{
|
|
|
|
|
|
}
|
|
|
|
|
|
- private MockAM launchAM(RMApp app, MockRM rm, MockNM nm)
|
|
|
- throws Exception {
|
|
|
- RMAppAttempt attempt = app.getCurrentAppAttempt();
|
|
|
- nm.nodeHeartbeat(true);
|
|
|
- MockAM am = rm.sendAMLaunched(attempt.getAppAttemptId());
|
|
|
- am.registerAppAttempt();
|
|
|
- rm.waitForState(app.getApplicationId(), RMAppState.RUNNING);
|
|
|
- rm.waitForState(app.getCurrentAppAttempt().getAppAttemptId(),
|
|
|
- RMAppAttemptState.RUNNING);
|
|
|
- return am;
|
|
|
- }
|
|
|
-
|
|
|
private void failOverAndKillApp(ApplicationId appId,
|
|
|
ApplicationAttemptId appAttemptId, RMAppState initialRMAppState,
|
|
|
RMAppAttemptState initialRMAppAttemptState,
|
|
@@ -256,29 +187,6 @@ public class TestKillApplicationWithRMHA extends ClientBaseWithFixes{
|
|
|
killApplication(rm2, appId, null, initialRMAppState);
|
|
|
}
|
|
|
|
|
|
- private void startRMs() throws IOException {
|
|
|
- rm1 = new MockRM(confForRM1);
|
|
|
- rm2 = new MockRM(confForRM2);
|
|
|
- startRMs(rm1, confForRM1, rm2, confForRM2);
|
|
|
-
|
|
|
- }
|
|
|
-
|
|
|
- private void startRMsWithCustomizedRMAppManager() throws IOException {
|
|
|
- final Configuration conf1 = new Configuration(confForRM1);
|
|
|
-
|
|
|
- rm1 = new MockRM(conf1) {
|
|
|
- @Override
|
|
|
- protected RMAppManager createRMAppManager() {
|
|
|
- return new MyRMAppManager(this.rmContext, this.scheduler,
|
|
|
- this.masterService, this.applicationACLsManager, conf1);
|
|
|
- }
|
|
|
- };
|
|
|
-
|
|
|
- rm2 = new MockRM(confForRM2);
|
|
|
-
|
|
|
- startRMs(rm1, conf1, rm2, confForRM2);
|
|
|
- }
|
|
|
-
|
|
|
private void startRMsWithCustomizedClientRMService() throws IOException {
|
|
|
final Configuration conf1 = new Configuration(confForRM1);
|
|
|
|
|
@@ -296,39 +204,6 @@ public class TestKillApplicationWithRMHA extends ClientBaseWithFixes{
|
|
|
startRMs(rm1, conf1, rm2, confForRM2);
|
|
|
}
|
|
|
|
|
|
- private static class MyRMAppManager extends RMAppManager {
|
|
|
-
|
|
|
- private Configuration conf;
|
|
|
- private RMContext rmContext;
|
|
|
-
|
|
|
- public MyRMAppManager(RMContext context, YarnScheduler scheduler,
|
|
|
- ApplicationMasterService masterService,
|
|
|
- ApplicationACLsManager applicationACLsManager, Configuration conf) {
|
|
|
- super(context, scheduler, masterService, applicationACLsManager, conf);
|
|
|
- this.conf = conf;
|
|
|
- this.rmContext = context;
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- protected void submitApplication(
|
|
|
- ApplicationSubmissionContext submissionContext, long submitTime,
|
|
|
- String user, boolean isRecovered, RMState state) throws YarnException {
|
|
|
- //Do nothing, just add the application to RMContext
|
|
|
- RMAppImpl application =
|
|
|
- new RMAppImpl(submissionContext.getApplicationId(), this.rmContext,
|
|
|
- this.conf, submissionContext.getApplicationName(), user,
|
|
|
- submissionContext.getQueue(), submissionContext,
|
|
|
- this.rmContext.getScheduler(),
|
|
|
- this.rmContext.getApplicationMasterService(),
|
|
|
- submitTime, submissionContext.getApplicationType(),
|
|
|
- submissionContext.getApplicationTags());
|
|
|
- this.rmContext.getRMApps().put(submissionContext.getApplicationId(),
|
|
|
- application);
|
|
|
- //Do not send RMAppEventType.START event
|
|
|
- //so the state of Application will not reach to NEW_SAVING state.
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
private static class MyClientRMService extends ClientRMService {
|
|
|
|
|
|
private RMContext rmContext;
|
|
@@ -366,21 +241,6 @@ public class TestKillApplicationWithRMHA extends ClientBaseWithFixes{
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- private boolean isFinalState(RMAppState state) {
|
|
|
- return state.equals(RMAppState.FINISHING)
|
|
|
- || state.equals(RMAppState.FINISHED) || state.equals(RMAppState.FAILED)
|
|
|
- || state.equals(RMAppState.KILLED);
|
|
|
- }
|
|
|
-
|
|
|
- private void explicitFailover() throws IOException {
|
|
|
- rm1.adminService.transitionToStandby(requestInfo);
|
|
|
- rm2.adminService.transitionToActive(requestInfo);
|
|
|
- Assert.assertTrue(rm1.getRMContext().getHAServiceState()
|
|
|
- == HAServiceState.STANDBY);
|
|
|
- Assert.assertTrue(rm2.getRMContext().getHAServiceState()
|
|
|
- == HAServiceState.ACTIVE);
|
|
|
- }
|
|
|
-
|
|
|
private void killApplication(MockRM rm, ApplicationId appId,
|
|
|
ApplicationAttemptId appAttemptId, RMAppState rmAppState)
|
|
|
throws Exception {
|
|
@@ -396,21 +256,4 @@ public class TestKillApplicationWithRMHA extends ClientBaseWithFixes{
|
|
|
// no new attempt is created.
|
|
|
Assert.assertEquals(1, loadedApp0.getAppAttempts().size());
|
|
|
}
|
|
|
-
|
|
|
- private void startRMs(MockRM rm1, Configuration confForRM1, MockRM rm2,
|
|
|
- Configuration confForRM2) throws IOException {
|
|
|
- rm1.init(confForRM1);
|
|
|
- rm1.start();
|
|
|
- Assert.assertTrue(rm1.getRMContext().getHAServiceState()
|
|
|
- == HAServiceState.STANDBY);
|
|
|
-
|
|
|
- rm2.init(confForRM2);
|
|
|
- rm2.start();
|
|
|
- Assert.assertTrue(rm2.getRMContext().getHAServiceState()
|
|
|
- == HAServiceState.STANDBY);
|
|
|
-
|
|
|
- rm1.adminService.transitionToActive(requestInfo);
|
|
|
- Assert.assertTrue(rm1.getRMContext().getHAServiceState()
|
|
|
- == HAServiceState.ACTIVE);
|
|
|
- }
|
|
|
}
|