|
@@ -30,11 +30,15 @@ import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.mapreduce.MRJobConfig;
|
|
|
+import org.apache.hadoop.mapreduce.OutputCommitter;
|
|
|
import org.apache.hadoop.mapreduce.TypeConverter;
|
|
|
import org.apache.hadoop.mapreduce.v2.api.records.JobId;
|
|
|
import org.apache.hadoop.mapreduce.v2.app.client.ClientService;
|
|
|
+import org.apache.hadoop.mapreduce.v2.app.commit.CommitterEvent;
|
|
|
+import org.apache.hadoop.mapreduce.v2.app.commit.CommitterEventHandler;
|
|
|
import org.apache.hadoop.mapreduce.v2.app.job.JobStateInternal;
|
|
|
import org.apache.hadoop.mapreduce.v2.app.rm.ContainerAllocator;
|
|
|
+import org.apache.hadoop.mapreduce.v2.app.rm.RMHeartbeatHandler;
|
|
|
import org.apache.hadoop.mapreduce.v2.util.MRApps;
|
|
|
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
|
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
@@ -42,6 +46,7 @@ import org.apache.hadoop.yarn.YarnException;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
|
|
import org.apache.hadoop.yarn.api.records.ContainerId;
|
|
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
|
+import org.apache.hadoop.yarn.event.EventHandler;
|
|
|
import org.apache.hadoop.yarn.util.ConverterUtils;
|
|
|
import org.junit.Before;
|
|
|
import org.junit.BeforeClass;
|
|
@@ -80,7 +85,7 @@ public class TestMRAppMaster {
|
|
|
ContainerId containerId = ConverterUtils.toContainerId(containerIdStr);
|
|
|
MRAppMasterTest appMaster =
|
|
|
new MRAppMasterTest(applicationAttemptId, containerId, "host", -1, -1,
|
|
|
- System.currentTimeMillis());
|
|
|
+ System.currentTimeMillis(), MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS);
|
|
|
YarnConfiguration conf = new YarnConfiguration();
|
|
|
conf.set(MRJobConfig.MR_AM_STAGING_DIR, stagingDir);
|
|
|
MRAppMaster.initAndStartAppMaster(appMaster, conf, userName);
|
|
@@ -109,7 +114,8 @@ public class TestMRAppMaster {
|
|
|
ContainerId containerId = ConverterUtils.toContainerId(containerIdStr);
|
|
|
MRAppMaster appMaster =
|
|
|
new MRAppMasterTest(applicationAttemptId, containerId, "host", -1, -1,
|
|
|
- System.currentTimeMillis(), false);
|
|
|
+ System.currentTimeMillis(), MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS,
|
|
|
+ false, false);
|
|
|
boolean caught = false;
|
|
|
try {
|
|
|
MRAppMaster.initAndStartAppMaster(appMaster, conf, userName);
|
|
@@ -144,7 +150,8 @@ public class TestMRAppMaster {
|
|
|
ContainerId containerId = ConverterUtils.toContainerId(containerIdStr);
|
|
|
MRAppMaster appMaster =
|
|
|
new MRAppMasterTest(applicationAttemptId, containerId, "host", -1, -1,
|
|
|
- System.currentTimeMillis(), false);
|
|
|
+ System.currentTimeMillis(), MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS,
|
|
|
+ false, false);
|
|
|
boolean caught = false;
|
|
|
try {
|
|
|
MRAppMaster.initAndStartAppMaster(appMaster, conf, userName);
|
|
@@ -179,7 +186,8 @@ public class TestMRAppMaster {
|
|
|
ContainerId containerId = ConverterUtils.toContainerId(containerIdStr);
|
|
|
MRAppMaster appMaster =
|
|
|
new MRAppMasterTest(applicationAttemptId, containerId, "host", -1, -1,
|
|
|
- System.currentTimeMillis(), false);
|
|
|
+ System.currentTimeMillis(), MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS,
|
|
|
+ false, false);
|
|
|
boolean caught = false;
|
|
|
try {
|
|
|
MRAppMaster.initAndStartAppMaster(appMaster, conf, userName);
|
|
@@ -214,7 +222,8 @@ public class TestMRAppMaster {
|
|
|
ContainerId containerId = ConverterUtils.toContainerId(containerIdStr);
|
|
|
MRAppMaster appMaster =
|
|
|
new MRAppMasterTest(applicationAttemptId, containerId, "host", -1, -1,
|
|
|
- System.currentTimeMillis(), false);
|
|
|
+ System.currentTimeMillis(), MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS,
|
|
|
+ false, false);
|
|
|
boolean caught = false;
|
|
|
try {
|
|
|
MRAppMaster.initAndStartAppMaster(appMaster, conf, userName);
|
|
@@ -230,36 +239,73 @@ public class TestMRAppMaster {
|
|
|
assertEquals(JobStateInternal.ERROR, appMaster.forcedState);
|
|
|
appMaster.stop();
|
|
|
}
|
|
|
+
|
|
|
+ @Test (timeout = 30000)
|
|
|
+ public void testMRAppMasterMaxAppAttempts() throws IOException,
|
|
|
+ InterruptedException {
|
|
|
+ int[] maxAppAttemtps = new int[] { 1, 2, 3 };
|
|
|
+ Boolean[] expectedBools = new Boolean[]{ true, true, false };
|
|
|
+
|
|
|
+ String applicationAttemptIdStr = "appattempt_1317529182569_0004_000002";
|
|
|
+ String containerIdStr = "container_1317529182569_0004_000002_1";
|
|
|
+
|
|
|
+ String userName = "TestAppMasterUser";
|
|
|
+ ApplicationAttemptId applicationAttemptId = ConverterUtils
|
|
|
+ .toApplicationAttemptId(applicationAttemptIdStr);
|
|
|
+ ContainerId containerId = ConverterUtils.toContainerId(containerIdStr);
|
|
|
+ YarnConfiguration conf = new YarnConfiguration();
|
|
|
+ conf.set(MRJobConfig.MR_AM_STAGING_DIR, stagingDir);
|
|
|
+
|
|
|
+ File stagingDir =
|
|
|
+ new File(MRApps.getStagingAreaDir(conf, userName).toString());
|
|
|
+ stagingDir.mkdirs();
|
|
|
+ for (int i = 0; i < maxAppAttemtps.length; ++i) {
|
|
|
+ MRAppMasterTest appMaster =
|
|
|
+ new MRAppMasterTest(applicationAttemptId, containerId, "host", -1, -1,
|
|
|
+ System.currentTimeMillis(), maxAppAttemtps[i], false, true);
|
|
|
+ MRAppMaster.initAndStartAppMaster(appMaster, conf, userName);
|
|
|
+ assertEquals("isLastAMRetry is correctly computed.", expectedBools[i],
|
|
|
+ appMaster.isLastAMRetry());
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
}
|
|
|
|
|
|
class MRAppMasterTest extends MRAppMaster {
|
|
|
|
|
|
Path stagingDirPath;
|
|
|
private Configuration conf;
|
|
|
- private boolean overrideInitAndStart;
|
|
|
+ private boolean overrideInit;
|
|
|
+ private boolean overrideStart;
|
|
|
ContainerAllocator mockContainerAllocator;
|
|
|
+ CommitterEventHandler mockCommitterEventHandler;
|
|
|
+ RMHeartbeatHandler mockRMHeartbeatHandler;
|
|
|
|
|
|
public MRAppMasterTest(ApplicationAttemptId applicationAttemptId,
|
|
|
ContainerId containerId, String host, int port, int httpPort,
|
|
|
- long submitTime) {
|
|
|
- this(applicationAttemptId, containerId, host, port, httpPort, submitTime,
|
|
|
- true);
|
|
|
+ long submitTime, int maxAppAttempts) {
|
|
|
+ this(applicationAttemptId, containerId, host, port, httpPort,
|
|
|
+ submitTime, maxAppAttempts, true, true);
|
|
|
}
|
|
|
public MRAppMasterTest(ApplicationAttemptId applicationAttemptId,
|
|
|
ContainerId containerId, String host, int port, int httpPort,
|
|
|
- long submitTime, boolean overrideInitAndStart) {
|
|
|
- super(applicationAttemptId, containerId, host, port, httpPort, submitTime);
|
|
|
- this.overrideInitAndStart = overrideInitAndStart;
|
|
|
+ long submitTime, int maxAppAttempts, boolean overrideInit,
|
|
|
+ boolean overrideStart) {
|
|
|
+ super(applicationAttemptId, containerId, host, port, httpPort, submitTime,
|
|
|
+ maxAppAttempts);
|
|
|
+ this.overrideInit = overrideInit;
|
|
|
+ this.overrideStart = overrideStart;
|
|
|
mockContainerAllocator = mock(ContainerAllocator.class);
|
|
|
+ mockCommitterEventHandler = mock(CommitterEventHandler.class);
|
|
|
+ mockRMHeartbeatHandler = mock(RMHeartbeatHandler.class);
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
public void init(Configuration conf) {
|
|
|
- if (overrideInitAndStart) {
|
|
|
- this.conf = conf;
|
|
|
- } else {
|
|
|
+ if (!overrideInit) {
|
|
|
super.init(conf);
|
|
|
}
|
|
|
+ this.conf = conf;
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -277,9 +323,20 @@ class MRAppMasterTest extends MRAppMaster {
|
|
|
return mockContainerAllocator;
|
|
|
}
|
|
|
|
|
|
+ @Override
|
|
|
+ protected EventHandler<CommitterEvent> createCommitterEventHandler(
|
|
|
+ AppContext context, OutputCommitter committer) {
|
|
|
+ return mockCommitterEventHandler;
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ protected RMHeartbeatHandler getRMHeartbeatHandler() {
|
|
|
+ return mockRMHeartbeatHandler;
|
|
|
+ }
|
|
|
+
|
|
|
@Override
|
|
|
public void start() {
|
|
|
- if (overrideInitAndStart) {
|
|
|
+ if (overrideStart) {
|
|
|
try {
|
|
|
String user = UserGroupInformation.getCurrentUser().getShortUserName();
|
|
|
stagingDirPath = MRApps.getStagingAreaDir(conf, user);
|