diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index 6c7053b5338..39407e1da10 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -295,6 +295,9 @@ Release 0.23.0 - Unreleased
MAPREDUCE-3001. Added task-specific counters to AppMaster and JobHistory
web-UIs. (Robert Joseph Evans via vinodkv)
+ MAPREDUCE-3098. Fixed RM and MR AM to report YarnApplicationState and
+ application's FinalStatus separately. (Hitesh Shah via vinodkv)
+
OPTIMIZATIONS
MAPREDUCE-2026. Make JobTracker.getJobCounters() and
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMCommunicator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMCommunicator.java
index 15a7e3f6a5a..cee37f98147 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMCommunicator.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMCommunicator.java
@@ -46,6 +46,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest
import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
@@ -75,7 +76,7 @@ public abstract class RMCommunicator extends AbstractService {
private final RecordFactory recordFactory =
RecordFactoryProvider.getRecordFactory(null);
-
+
private final AppContext context;
private Job job;
@@ -146,7 +147,7 @@ public abstract class RMCommunicator extends AbstractService {
protected void register() {
//Register
- String host =
+ String host =
clientService.getBindAddress().getAddress().getHostAddress();
try {
RegisterApplicationMasterRequest request =
@@ -155,7 +156,7 @@ public abstract class RMCommunicator extends AbstractService {
request.setHost(host);
request.setRpcPort(clientService.getBindAddress().getPort());
request.setTrackingUrl(host + ":" + clientService.getHttpPort());
- RegisterApplicationMasterResponse response =
+ RegisterApplicationMasterResponse response =
scheduler.registerApplicationMaster(request);
minContainerCapability = response.getMinimumResourceCapability();
maxContainerCapability = response.getMaximumResourceCapability();
@@ -169,29 +170,29 @@ public abstract class RMCommunicator extends AbstractService {
protected void unregister() {
try {
- String finalState = "RUNNING";
+ FinalApplicationStatus finishState = FinalApplicationStatus.UNDEFINED;
if (job.getState() == JobState.SUCCEEDED) {
- finalState = "SUCCEEDED";
+ finishState = FinalApplicationStatus.SUCCEEDED;
} else if (job.getState() == JobState.KILLED) {
- finalState = "KILLED";
+ finishState = FinalApplicationStatus.KILLED;
} else if (job.getState() == JobState.FAILED
|| job.getState() == JobState.ERROR) {
- finalState = "FAILED";
+ finishState = FinalApplicationStatus.FAILED;
}
StringBuffer sb = new StringBuffer();
for (String s : job.getDiagnostics()) {
sb.append(s).append("\n");
}
LOG.info("Setting job diagnostics to " + sb.toString());
-
- String historyUrl = JobHistoryUtils.getHistoryUrl(getConfig(),
+
+ String historyUrl = JobHistoryUtils.getHistoryUrl(getConfig(),
context.getApplicationID());
LOG.info("History url is " + historyUrl);
FinishApplicationMasterRequest request =
recordFactory.newRecordInstance(FinishApplicationMasterRequest.class);
request.setAppAttemptId(this.applicationAttemptId);
- request.setFinalState(finalState.toString());
+ request.setFinishApplicationStatus(finishState);
request.setDiagnostics(sb.toString());
request.setTrackingUrl(historyUrl);
scheduler.finishApplicationMaster(request);
@@ -203,7 +204,7 @@ public abstract class RMCommunicator extends AbstractService {
protected Resource getMinContainerCapability() {
return minContainerCapability;
}
-
+
protected Resource getMaxContainerCapability() {
return maxContainerCapability;
}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/TypeConverter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/TypeConverter.java
index 1787757d1b6..be6e6d9f20e 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/TypeConverter.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/TypeConverter.java
@@ -45,7 +45,7 @@ import org.apache.hadoop.mapreduce.v2.util.MRApps;
import org.apache.hadoop.yarn.YarnException;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ApplicationReport;
-import org.apache.hadoop.yarn.api.records.ApplicationState;
+import org.apache.hadoop.yarn.api.records.YarnApplicationState;
import org.apache.hadoop.yarn.api.records.NodeReport;
import org.apache.hadoop.yarn.api.records.QueueACL;
import org.apache.hadoop.yarn.api.records.QueueState;
@@ -56,7 +56,7 @@ import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
public class TypeConverter {
private static RecordFactory recordFactory;
-
+
static {
recordFactory = RecordFactoryProvider.getRecordFactory(null);
}
@@ -75,7 +75,7 @@ public class TypeConverter {
public static JobId toYarn(org.apache.hadoop.mapreduce.JobID id) {
JobId jobId = recordFactory.newRecordInstance(JobId.class);
jobId.setId(id.getId()); //currently there is 1-1 mapping between appid and jobid
-
+
ApplicationId appId = recordFactory.newRecordInstance(ApplicationId.class);
appId.setId(id.getId());
appId.setClusterTimestamp(toClusterTimeStamp(id.getJtIdentifier()));
@@ -137,7 +137,7 @@ public class TypeConverter {
}
return TaskAttemptState.valueOf(state.toString());
}
-
+
public static Phase toYarn(org.apache.hadoop.mapred.TaskStatus.Phase phase) {
switch (phase) {
case STARTING:
@@ -161,7 +161,7 @@ public class TypeConverter {
TaskCompletionEvent[] oldEvents =
new TaskCompletionEvent[newEvents.length];
int i = 0;
- for (TaskAttemptCompletionEvent newEvent
+ for (TaskAttemptCompletionEvent newEvent
: newEvents) {
oldEvents[i++] = fromYarn(newEvent);
}
@@ -215,19 +215,19 @@ public class TypeConverter {
taskAttemptId.setId(id.getId());
return taskAttemptId;
}
-
+
public static org.apache.hadoop.mapreduce.Counters fromYarn(
Counters yCntrs) {
if (yCntrs == null) {
return null;
}
- org.apache.hadoop.mapreduce.Counters counters =
+ org.apache.hadoop.mapreduce.Counters counters =
new org.apache.hadoop.mapreduce.Counters();
for (CounterGroup yGrp : yCntrs.getAllCounterGroups().values()) {
counters.addGroup(yGrp.getName(), yGrp.getDisplayName());
for (Counter yCntr : yGrp.getAllCounters().values()) {
- org.apache.hadoop.mapreduce.Counter c =
- counters.findCounter(yGrp.getName(),
+ org.apache.hadoop.mapreduce.Counter c =
+ counters.findCounter(yGrp.getName(),
yCntr.getName());
c.setValue(yCntr.getValue());
}
@@ -292,16 +292,16 @@ public class TypeConverter {
jobStatus.setFailureInfo(jobreport.getDiagnostics());
return jobStatus;
}
-
+
public static org.apache.hadoop.mapreduce.QueueState fromYarn(
QueueState state) {
- org.apache.hadoop.mapreduce.QueueState qState =
+ org.apache.hadoop.mapreduce.QueueState qState =
org.apache.hadoop.mapreduce.QueueState.getState(
state.toString().toLowerCase());
return qState;
}
-
+
public static int fromYarn(JobState state) {
switch (state) {
case NEW:
@@ -339,7 +339,7 @@ public class TypeConverter {
}
throw new YarnException("Unrecognized task state: " + state);
}
-
+
public static TaskReport fromYarn(org.apache.hadoop.mapreduce.v2.api.records.TaskReport report) {
String[] diagnostics = null;
if (report.getDiagnosticsList() != null) {
@@ -351,14 +351,14 @@ public class TypeConverter {
} else {
diagnostics = new String[0];
}
-
- TaskReport rep = new TaskReport(fromYarn(report.getTaskId()),
+
+ TaskReport rep = new TaskReport(fromYarn(report.getTaskId()),
report.getProgress(), report.getTaskState().toString(),
diagnostics, fromYarn(report.getTaskState()), report.getStartTime(), report.getFinishTime(),
fromYarn(report.getCounters()));
- List runningAtts
+ List runningAtts
= new ArrayList();
- for (org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId id
+ for (org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId id
: report.getRunningAttemptsList()) {
runningAtts.add(fromYarn(id));
}
@@ -368,7 +368,7 @@ public class TypeConverter {
}
return rep;
}
-
+
public static List fromYarn(
List taskReports) {
List reports = new ArrayList();
@@ -377,14 +377,14 @@ public class TypeConverter {
}
return reports;
}
-
- public static JobStatus.State fromYarn(ApplicationState state) {
+
+ public static JobStatus.State fromYarn(YarnApplicationState state) {
switch (state) {
case SUBMITTED:
return State.PREP;
case RUNNING:
return State.RUNNING;
- case SUCCEEDED:
+ case FINISHED:
return State.SUCCEEDED;
case FAILED:
return State.FAILED;
@@ -396,7 +396,7 @@ public class TypeConverter {
private static final String TT_NAME_PREFIX = "tracker_";
public static TaskTrackerInfo fromYarn(NodeReport node) {
- TaskTrackerInfo taskTracker =
+ TaskTrackerInfo taskTracker =
new TaskTrackerInfo(TT_NAME_PREFIX + node.getNodeId().toString());
return taskTracker;
}
@@ -417,7 +417,7 @@ public class TypeConverter {
new JobStatus(
TypeConverter.fromYarn(application.getApplicationId()),
0.0f, 0.0f, 0.0f, 0.0f,
- TypeConverter.fromYarn(application.getState()),
+ TypeConverter.fromYarn(application.getYarnApplicationState()),
org.apache.hadoop.mapreduce.JobPriority.NORMAL,
application.getUser(), application.getName(),
application.getQueue(), jobFile, trackingUrl
@@ -433,7 +433,7 @@ public class TypeConverter {
List jobStatuses = new ArrayList();
for (ApplicationReport application : applications) {
// each applicationReport has its own jobFile
- org.apache.hadoop.mapreduce.JobID jobId =
+ org.apache.hadoop.mapreduce.JobID jobId =
TypeConverter.fromYarn(application.getApplicationId());
jobStatuses.add(TypeConverter.fromYarn(application,
MRApps.getJobFile(conf, application.getUser(), jobId)));
@@ -441,14 +441,14 @@ public class TypeConverter {
return jobStatuses.toArray(new JobStatus[jobStatuses.size()]);
}
-
- public static QueueInfo fromYarn(org.apache.hadoop.yarn.api.records.QueueInfo
+
+ public static QueueInfo fromYarn(org.apache.hadoop.yarn.api.records.QueueInfo
queueInfo, Configuration conf) {
return new QueueInfo(queueInfo.getQueueName(),queueInfo.toString(),
fromYarn(queueInfo.getQueueState()), TypeConverter.fromYarnApps(
queueInfo.getApplications(), conf));
}
-
+
public static QueueInfo[] fromYarnQueueInfo(
List queues,
Configuration conf) {
@@ -467,9 +467,9 @@ public class TypeConverter {
for (QueueACL qAcl : aclInfo.getUserAcls()) {
operations.add(qAcl.toString());
}
-
- QueueAclsInfo acl =
- new QueueAclsInfo(aclInfo.getQueueName(),
+
+ QueueAclsInfo acl =
+ new QueueAclsInfo(aclInfo.getQueueName(),
operations.toArray(new String[operations.size()]));
acls.add(acl);
}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapreduce/TestTypeConverter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapreduce/TestTypeConverter.java
index 1aeae987c80..43ca32020d1 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapreduce/TestTypeConverter.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapreduce/TestTypeConverter.java
@@ -21,7 +21,7 @@ import junit.framework.Assert;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.ApplicationState;
+import org.apache.hadoop.yarn.api.records.YarnApplicationState;
import org.apache.hadoop.yarn.api.records.ApplicationReport;
import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationIdPBImpl;
import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationReportPBImpl;
@@ -35,11 +35,11 @@ public class TestTypeConverter {
@Test
public void testFromYarn() throws Exception {
int appStartTime = 612354;
- ApplicationState state = ApplicationState.RUNNING;
+ YarnApplicationState state = YarnApplicationState.RUNNING;
ApplicationId applicationId = new ApplicationIdPBImpl();
ApplicationReportPBImpl applicationReport = new ApplicationReportPBImpl();
applicationReport.setApplicationId(applicationId);
- applicationReport.setState(state);
+ applicationReport.setYarnApplicationState(state);
applicationReport.setStartTime(appStartTime);
applicationReport.setUser("TestTypeConverter-user");
JobStatus jobStatus = TypeConverter.fromYarn(applicationReport, "dummy-jobfile");
@@ -56,7 +56,7 @@ public class TestTypeConverter {
ApplicationReport mockReport = mock(ApplicationReport.class);
when(mockReport.getTrackingUrl()).thenReturn("dummy-tracking-url");
when(mockReport.getApplicationId()).thenReturn(mockAppId);
- when(mockReport.getState()).thenReturn(ApplicationState.KILLED);
+ when(mockReport.getYarnApplicationState()).thenReturn(YarnApplicationState.KILLED);
when(mockReport.getUser()).thenReturn("dummy-user");
when(mockReport.getQueue()).thenReturn("dummy-queue");
String jobFile = "dummy-path/job.xml";
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ClientServiceDelegate.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ClientServiceDelegate.java
index e7177f54cd9..335d44b0826 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ClientServiceDelegate.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ClientServiceDelegate.java
@@ -61,7 +61,7 @@ import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.yarn.YarnException;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ApplicationReport;
-import org.apache.hadoop.yarn.api.records.ApplicationState;
+import org.apache.hadoop.yarn.api.records.YarnApplicationState;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
import org.apache.hadoop.yarn.factories.RecordFactory;
@@ -89,7 +89,7 @@ public class ClientServiceDelegate {
private static String UNKNOWN_USER = "Unknown User";
private String trackingUrl;
- public ClientServiceDelegate(Configuration conf, ResourceMgrDelegate rm,
+ public ClientServiceDelegate(Configuration conf, ResourceMgrDelegate rm,
JobID jobId, MRClientProtocol historyServerProxy) {
this.conf = new Configuration(conf); // Cloning for modifying.
// For faster redirects from AM to HS.
@@ -103,7 +103,7 @@ public class ClientServiceDelegate {
// Get the instance of the NotRunningJob corresponding to the specified
// user and state
- private NotRunningJob getNotRunningJob(ApplicationReport applicationReport,
+ private NotRunningJob getNotRunningJob(ApplicationReport applicationReport,
JobState state) {
synchronized (notRunningJobs) {
HashMap map = notRunningJobs.get(state);
@@ -111,8 +111,8 @@ public class ClientServiceDelegate {
map = new HashMap();
notRunningJobs.put(state, map);
}
- String user =
- (applicationReport == null) ?
+ String user =
+ (applicationReport == null) ?
UNKNOWN_USER : applicationReport.getUser();
NotRunningJob notRunningJob = map.get(user);
if (notRunningJob == null) {
@@ -135,7 +135,7 @@ public class ClientServiceDelegate {
trackingUrl = application.getTrackingUrl();
}
String serviceAddr = null;
- while (application == null || ApplicationState.RUNNING.equals(application.getState())) {
+ while (application == null || YarnApplicationState.RUNNING.equals(application.getYarnApplicationState())) {
if (application == null) {
LOG.info("Could not get Job info from RM for job " + jobId
+ ". Redirecting to job history server.");
@@ -145,8 +145,8 @@ public class ClientServiceDelegate {
if (application.getHost() == null || "".equals(application.getHost())) {
LOG.debug("AM not assigned to Job. Waiting to get the AM ...");
Thread.sleep(2000);
-
- LOG.debug("Application state is " + application.getState());
+
+ LOG.debug("Application state is " + application.getYarnApplicationState());
application = rm.getApplicationReport(appId);
continue;
}
@@ -168,7 +168,7 @@ public class ClientServiceDelegate {
//possibly the AM has crashed
//there may be some time before AM is restarted
//keep retrying by getting the address from RM
- LOG.info("Could not connect to " + serviceAddr +
+ LOG.info("Could not connect to " + serviceAddr +
". Waiting for getting the latest AM address...");
try {
Thread.sleep(2000);
@@ -189,35 +189,36 @@ public class ClientServiceDelegate {
}
/** we just want to return if its allocating, so that we don't
- * block on it. This is to be able to return job status
+ * block on it. This is to be able to return job status
* on an allocating Application.
*/
-
+
String user = application.getUser();
if (user == null) {
throw RPCUtil.getRemoteException("User is not set in the application report");
}
- if (application.getState() == ApplicationState.NEW ||
- application.getState() == ApplicationState.SUBMITTED) {
+ if (application.getYarnApplicationState() == YarnApplicationState.NEW ||
+ application.getYarnApplicationState() == YarnApplicationState.SUBMITTED) {
realProxy = null;
return getNotRunningJob(application, JobState.NEW);
}
-
- if (application.getState() == ApplicationState.FAILED) {
+
+ if (application.getYarnApplicationState() == YarnApplicationState.FAILED) {
realProxy = null;
return getNotRunningJob(application, JobState.FAILED);
}
-
- if (application.getState() == ApplicationState.KILLED) {
+
+ if (application.getYarnApplicationState() == YarnApplicationState.KILLED) {
realProxy = null;
return getNotRunningJob(application, JobState.KILLED);
}
-
- //History server can serve a job only if application
+
+ //History server can serve a job only if application
//succeeded.
- if (application.getState() == ApplicationState.SUCCEEDED) {
- LOG.info("Application state is completed. " +
- "Redirecting to job history server");
+ if (application.getYarnApplicationState() == YarnApplicationState.FINISHED) {
+ LOG.info("Application state is completed. FinalApplicationStatus="
+ + application.getFinalApplicationStatus().toString()
+ + ". Redirecting to job history server");
realProxy = checkAndGetHSProxy(application, JobState.SUCCEEDED);
}
return realProxy;
@@ -241,7 +242,7 @@ public class ClientServiceDelegate {
Configuration myConf = new Configuration(conf);
myConf.setClass(
YarnConfiguration.YARN_SECURITY_INFO,
- SchedulerSecurityInfo.class, SecurityInfo.class);
+ SchedulerSecurityInfo.class, SecurityInfo.class);
YarnRPC rpc = YarnRPC.create(myConf);
return (MRClientProtocol) rpc.getProxy(MRClientProtocol.class,
NetUtils.createSocketAddr(serviceAddr), myConf);
@@ -250,7 +251,7 @@ public class ClientServiceDelegate {
LOG.trace("Connected to ApplicationMaster at: " + serviceAddr);
}
- private synchronized Object invoke(String method, Class argClass,
+ private synchronized Object invoke(String method, Class argClass,
Object args) throws YarnRemoteException {
Method methodOb = null;
try {
@@ -289,10 +290,10 @@ public class ClientServiceDelegate {
org.apache.hadoop.mapreduce.v2.api.records.JobId jobID = TypeConverter.toYarn(arg0);
GetCountersRequest request = recordFactory.newRecordInstance(GetCountersRequest.class);
request.setJobId(jobID);
- Counters cnt = ((GetCountersResponse)
+ Counters cnt = ((GetCountersResponse)
invoke("getCounters", GetCountersRequest.class, request)).getCounters();
return TypeConverter.fromYarn(cnt);
-
+
}
public TaskCompletionEvent[] getTaskCompletionEvents(JobID arg0, int arg1, int arg2)
@@ -304,7 +305,7 @@ public class ClientServiceDelegate {
request.setJobId(jobID);
request.setFromEventId(arg1);
request.setMaxEvents(arg2);
- List list =
+ List list =
((GetTaskAttemptCompletionEventsResponse) invoke(
"getTaskAttemptCompletionEvents", GetTaskAttemptCompletionEventsRequest.class, request)).
getCompletionEventList();
@@ -332,12 +333,12 @@ public class ClientServiceDelegate {
}
public JobStatus getJobStatus(JobID oldJobID) throws YarnRemoteException {
- org.apache.hadoop.mapreduce.v2.api.records.JobId jobId =
+ org.apache.hadoop.mapreduce.v2.api.records.JobId jobId =
TypeConverter.toYarn(oldJobID);
- GetJobReportRequest request =
+ GetJobReportRequest request =
recordFactory.newRecordInstance(GetJobReportRequest.class);
request.setJobId(jobId);
- JobReport report = ((GetJobReportResponse) invoke("getJobReport",
+ JobReport report = ((GetJobReportResponse) invoke("getJobReport",
GetJobReportRequest.class, request)).getJobReport();
if (StringUtils.isEmpty(report.getJobFile())) {
String jobFile = MRApps.getJobFile(conf, report.getUser(), oldJobID);
@@ -351,24 +352,24 @@ public class ClientServiceDelegate {
public org.apache.hadoop.mapreduce.TaskReport[] getTaskReports(JobID oldJobID, TaskType taskType)
throws YarnRemoteException, YarnRemoteException {
- org.apache.hadoop.mapreduce.v2.api.records.JobId jobId =
+ org.apache.hadoop.mapreduce.v2.api.records.JobId jobId =
TypeConverter.toYarn(oldJobID);
- GetTaskReportsRequest request =
+ GetTaskReportsRequest request =
recordFactory.newRecordInstance(GetTaskReportsRequest.class);
request.setJobId(jobId);
request.setTaskType(TypeConverter.toYarn(taskType));
-
- List taskReports =
- ((GetTaskReportsResponse) invoke("getTaskReports", GetTaskReportsRequest.class,
+
+ List taskReports =
+ ((GetTaskReportsResponse) invoke("getTaskReports", GetTaskReportsRequest.class,
request)).getTaskReportList();
-
+
return TypeConverter.fromYarn
(taskReports).toArray(new org.apache.hadoop.mapreduce.TaskReport[0]);
}
public boolean killTask(TaskAttemptID taskAttemptID, boolean fail)
throws YarnRemoteException {
- org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId attemptID
+ org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId attemptID
= TypeConverter.toYarn(taskAttemptID);
if (fail) {
FailTaskAttemptRequest failRequest = recordFactory.newRecordInstance(FailTaskAttemptRequest.class);
@@ -381,10 +382,10 @@ public class ClientServiceDelegate {
}
return true;
}
-
+
public boolean killJob(JobID oldJobID)
throws YarnRemoteException {
- org.apache.hadoop.mapreduce.v2.api.records.JobId jobId
+ org.apache.hadoop.mapreduce.v2.api.records.JobId jobId
= TypeConverter.toYarn(oldJobID);
KillJobRequest killRequest = recordFactory.newRecordInstance(KillJobRequest.class);
killRequest.setJobId(jobId);
@@ -392,5 +393,5 @@ public class ClientServiceDelegate {
return true;
}
-
+
}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/NotRunningJob.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/NotRunningJob.java
index 17ad9f62aae..4b2d25676d0 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/NotRunningJob.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/NotRunningJob.java
@@ -22,8 +22,6 @@ import java.util.ArrayList;
import java.util.HashMap;
import org.apache.commons.lang.NotImplementedException;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.mapreduce.v2.api.MRClientProtocol;
import org.apache.hadoop.mapreduce.v2.api.protocolrecords.FailTaskAttemptRequest;
import org.apache.hadoop.mapreduce.v2.api.protocolrecords.FailTaskAttemptResponse;
@@ -55,40 +53,36 @@ import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptCompletionEvent;
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
import org.apache.hadoop.mapreduce.v2.api.records.TaskReport;
import org.apache.hadoop.mapreduce.v2.api.records.TaskState;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
import org.apache.hadoop.yarn.api.records.ApplicationReport;
+import org.apache.hadoop.yarn.api.records.YarnApplicationState;
import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
+import org.apache.hadoop.yarn.util.BuilderUtils;
public class NotRunningJob implements MRClientProtocol {
- private static final Log LOG = LogFactory.getLog(NotRunningJob.class);
-
- private RecordFactory recordFactory =
+ private RecordFactory recordFactory =
RecordFactoryProvider.getRecordFactory(null);
-
+
private final JobState jobState;
private final ApplicationReport applicationReport;
-
-
+
+
private ApplicationReport getUnknownApplicationReport() {
- ApplicationReport unknown =
- recordFactory.newRecordInstance(ApplicationReport.class);
- unknown.setUser("N/A");
- unknown.setHost("N/A");
- unknown.setName("N/A");
- unknown.setQueue("N/A");
- unknown.setStartTime(0);
- unknown.setFinishTime(0);
- unknown.setTrackingUrl("N/A");
- unknown.setDiagnostics("N/A");
- LOG.info("getUnknownApplicationReport");
- return unknown;
+ ApplicationId unknownAppId = recordFactory.newRecordInstance(ApplicationId.class);
+
+ // Setting AppState to NEW and finalStatus to UNDEFINED as they are never used
+ // for a non running job
+ return BuilderUtils.newApplicationReport(unknownAppId, "N/A", "N/A", "N/A", "N/A", 0, "",
+ YarnApplicationState.NEW, "N/A", "N/A", 0, 0, FinalApplicationStatus.UNDEFINED);
}
-
+
NotRunningJob(ApplicationReport applicationReport, JobState jobState) {
- this.applicationReport =
- (applicationReport == null) ?
+ this.applicationReport =
+ (applicationReport == null) ?
getUnknownApplicationReport() : applicationReport;
this.jobState = jobState;
}
@@ -96,7 +90,7 @@ public class NotRunningJob implements MRClientProtocol {
@Override
public FailTaskAttemptResponse failTaskAttempt(
FailTaskAttemptRequest request) throws YarnRemoteException {
- FailTaskAttemptResponse resp =
+ FailTaskAttemptResponse resp =
recordFactory.newRecordInstance(FailTaskAttemptResponse.class);
return resp;
}
@@ -104,7 +98,7 @@ public class NotRunningJob implements MRClientProtocol {
@Override
public GetCountersResponse getCounters(GetCountersRequest request)
throws YarnRemoteException {
- GetCountersResponse resp =
+ GetCountersResponse resp =
recordFactory.newRecordInstance(GetCountersResponse.class);
Counters counters = recordFactory.newRecordInstance(Counters.class);
counters.addAllCounterGroups(new HashMap());
@@ -115,7 +109,7 @@ public class NotRunningJob implements MRClientProtocol {
@Override
public GetDiagnosticsResponse getDiagnostics(GetDiagnosticsRequest request)
throws YarnRemoteException {
- GetDiagnosticsResponse resp =
+ GetDiagnosticsResponse resp =
recordFactory.newRecordInstance(GetDiagnosticsResponse.class);
resp.addDiagnostics("");
return resp;
@@ -135,7 +129,7 @@ public class NotRunningJob implements MRClientProtocol {
jobReport.setTrackingUrl(applicationReport.getTrackingUrl());
jobReport.setFinishTime(applicationReport.getFinishTime());
- GetJobReportResponse resp =
+ GetJobReportResponse resp =
recordFactory.newRecordInstance(GetJobReportResponse.class);
resp.setJobReport(jobReport);
return resp;
@@ -145,7 +139,7 @@ public class NotRunningJob implements MRClientProtocol {
public GetTaskAttemptCompletionEventsResponse getTaskAttemptCompletionEvents(
GetTaskAttemptCompletionEventsRequest request)
throws YarnRemoteException {
- GetTaskAttemptCompletionEventsResponse resp =
+ GetTaskAttemptCompletionEventsResponse resp =
recordFactory.newRecordInstance(GetTaskAttemptCompletionEventsResponse.class);
resp.addAllCompletionEvents(new ArrayList());
return resp;
@@ -161,7 +155,7 @@ public class NotRunningJob implements MRClientProtocol {
@Override
public GetTaskReportResponse getTaskReport(GetTaskReportRequest request)
throws YarnRemoteException {
- GetTaskReportResponse resp =
+ GetTaskReportResponse resp =
recordFactory.newRecordInstance(GetTaskReportResponse.class);
TaskReport report = recordFactory.newRecordInstance(TaskReport.class);
report.setTaskId(request.getTaskId());
@@ -176,7 +170,7 @@ public class NotRunningJob implements MRClientProtocol {
@Override
public GetTaskReportsResponse getTaskReports(GetTaskReportsRequest request)
throws YarnRemoteException {
- GetTaskReportsResponse resp =
+ GetTaskReportsResponse resp =
recordFactory.newRecordInstance(GetTaskReportsResponse.class);
resp.addAllTaskReports(new ArrayList());
return resp;
@@ -185,7 +179,7 @@ public class NotRunningJob implements MRClientProtocol {
@Override
public KillJobResponse killJob(KillJobRequest request)
throws YarnRemoteException {
- KillJobResponse resp =
+ KillJobResponse resp =
recordFactory.newRecordInstance(KillJobResponse.class);
return resp;
}
@@ -193,7 +187,7 @@ public class NotRunningJob implements MRClientProtocol {
@Override
public KillTaskResponse killTask(KillTaskRequest request)
throws YarnRemoteException {
- KillTaskResponse resp =
+ KillTaskResponse resp =
recordFactory.newRecordInstance(KillTaskResponse.class);
return resp;
}
@@ -201,9 +195,9 @@ public class NotRunningJob implements MRClientProtocol {
@Override
public KillTaskAttemptResponse killTaskAttempt(
KillTaskAttemptRequest request) throws YarnRemoteException {
- KillTaskAttemptResponse resp =
+ KillTaskAttemptResponse resp =
recordFactory.newRecordInstance(KillTaskAttemptResponse.class);
return resp;
}
-
+
}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/YARNRunner.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/YARNRunner.java
index a11968a16f9..aceb02378b2 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/YARNRunner.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/YARNRunner.java
@@ -62,7 +62,6 @@ import org.apache.hadoop.yarn.api.ApplicationConstants;
import org.apache.hadoop.yarn.api.ApplicationConstants.Environment;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ApplicationReport;
-import org.apache.hadoop.yarn.api.records.ApplicationState;
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
import org.apache.hadoop.yarn.api.records.LocalResource;
@@ -70,6 +69,7 @@ import org.apache.hadoop.yarn.api.records.LocalResourceType;
import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.URL;
+import org.apache.hadoop.yarn.api.records.YarnApplicationState;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
@@ -99,7 +99,7 @@ public class YARNRunner implements ClientProtocol {
}
/**
- * Similar to {@link #YARNRunner(Configuration)} but allowing injecting
+ * Similar to {@link #YARNRunner(Configuration)} but allowing injecting
* {@link ResourceMgrDelegate}. Enables mocking and testing.
* @param conf the configuration object for the client
* @param resMgrDelegate the resourcemanager client handle.
@@ -107,12 +107,12 @@ public class YARNRunner implements ClientProtocol {
public YARNRunner(Configuration conf, ResourceMgrDelegate resMgrDelegate) {
this(conf, resMgrDelegate, new ClientCache(conf, resMgrDelegate));
}
-
+
/**
- * Similar to {@link YARNRunner#YARNRunner(Configuration, ResourceMgrDelegate)}
+ * Similar to {@link YARNRunner#YARNRunner(Configuration, ResourceMgrDelegate)}
* but allowing injecting {@link ClientCache}. Enable mocking and testing.
* @param conf the configuration object
- * @param resMgrDelegate the resource manager delegate
+ * @param resMgrDelegate the resource manager delegate
* @param clientCache the client cache object.
*/
public YARNRunner(Configuration conf, ResourceMgrDelegate resMgrDelegate,
@@ -126,7 +126,7 @@ public class YARNRunner implements ClientProtocol {
throw new RuntimeException("Error in instantiating YarnClient", ufe);
}
}
-
+
@Override
public void cancelDelegationToken(Token arg0)
throws IOException, InterruptedException {
@@ -152,7 +152,7 @@ public class YARNRunner implements ClientProtocol {
@Override
public ClusterMetrics getClusterMetrics() throws IOException,
- InterruptedException {
+ InterruptedException {
return resMgrDelegate.getClusterMetrics();
}
@@ -209,13 +209,13 @@ public class YARNRunner implements ClientProtocol {
public String getSystemDir() throws IOException, InterruptedException {
return resMgrDelegate.getSystemDir();
}
-
+
@Override
public long getTaskTrackerExpiryInterval() throws IOException,
InterruptedException {
return resMgrDelegate.getTaskTrackerExpiryInterval();
}
-
+
@Override
public JobStatus submitJob(JobID jobId, String jobSubmitDir, Credentials ts)
throws IOException, InterruptedException {
@@ -230,20 +230,20 @@ public class YARNRunner implements ClientProtocol {
}
// Construct necessary information to start the MR AM
- ApplicationSubmissionContext appContext =
+ ApplicationSubmissionContext appContext =
createApplicationSubmissionContext(conf, jobSubmitDir, ts);
-
+
// Submit to ResourceManager
ApplicationId applicationId = resMgrDelegate.submitApplication(appContext);
-
+
ApplicationReport appMaster = resMgrDelegate
.getApplicationReport(applicationId);
- String diagnostics =
- (appMaster == null ?
+ String diagnostics =
+ (appMaster == null ?
"application report is null" : appMaster.getDiagnostics());
- if (appMaster == null || appMaster.getState() == ApplicationState.FAILED
- || appMaster.getState() == ApplicationState.KILLED) {
- throw new IOException("Failed to run job : " +
+ if (appMaster == null || appMaster.getYarnApplicationState() == YarnApplicationState.FAILED
+ || appMaster.getYarnApplicationState() == YarnApplicationState.KILLED) {
+ throw new IOException("Failed to run job : " +
diagnostics);
}
return clientCache.getClient(jobId).getJobStatus(jobId);
@@ -266,7 +266,7 @@ public class YARNRunner implements ClientProtocol {
Configuration jobConf,
String jobSubmitDir, Credentials ts) throws IOException {
ApplicationId applicationId = resMgrDelegate.getApplicationId();
-
+
// Setup resource requirements
Resource capability = recordFactory.newRecordInstance(Resource.class);
capability.setMemory(conf.getInt(MRJobConfig.MR_AM_VMEM_MB,
@@ -276,9 +276,9 @@ public class YARNRunner implements ClientProtocol {
// Setup LocalResources
Map localResources =
new HashMap();
-
+
Path jobConfPath = new Path(jobSubmitDir, MRJobConfig.JOB_CONF_FILE);
-
+
URL yarnUrlForJobSubmitDir = ConverterUtils
.getYarnUrlFromPath(defaultFileContext.getDefaultFileSystem()
.resolvePath(
@@ -299,18 +299,18 @@ public class YARNRunner implements ClientProtocol {
LOG.info("Job jar is not present. "
+ "Not adding any jar to the list of resources.");
}
-
+
// TODO gross hack
- for (String s : new String[] {
- MRJobConfig.JOB_SPLIT,
+ for (String s : new String[] {
+ MRJobConfig.JOB_SPLIT,
MRJobConfig.JOB_SPLIT_METAINFO,
MRJobConfig.APPLICATION_TOKENS_FILE }) {
localResources.put(
MRJobConfig.JOB_SUBMIT_DIR + "/" + s,
- createApplicationResource(defaultFileContext,
+ createApplicationResource(defaultFileContext,
new Path(jobSubmitDir, s)));
}
-
+
// Setup security tokens
ByteBuffer securityTokens = null;
if (UserGroupInformation.isSecurityEnabled()) {
@@ -322,20 +322,20 @@ public class YARNRunner implements ClientProtocol {
// Setup the command to run the AM
Vector vargs = new Vector(8);
vargs.add(Environment.JAVA_HOME.$() + "/bin/java");
-
+
long logSize = TaskLog.getTaskLogLength(new JobConf(conf));
vargs.add("-Dlog4j.configuration=container-log4j.properties");
vargs.add("-D" + MRJobConfig.TASK_LOG_DIR + "="
+ ApplicationConstants.LOG_DIR_EXPANSION_VAR);
vargs.add("-D" + MRJobConfig.TASK_LOG_SIZE + "=" + logSize);
-
+
vargs.add(conf.get(MRJobConfig.MR_AM_COMMAND_OPTS,
MRJobConfig.DEFAULT_MR_AM_COMMAND_OPTS));
vargs.add(MRJobConfig.APPLICATION_MASTER_CLASS);
- vargs.add("1>" + ApplicationConstants.LOG_DIR_EXPANSION_VAR +
+ vargs.add("1>" + ApplicationConstants.LOG_DIR_EXPANSION_VAR +
Path.SEPARATOR + ApplicationConstants.STDOUT);
- vargs.add("2>" + ApplicationConstants.LOG_DIR_EXPANSION_VAR +
+ vargs.add("2>" + ApplicationConstants.LOG_DIR_EXPANSION_VAR +
Path.SEPARATOR + ApplicationConstants.STDERR);
@@ -349,12 +349,12 @@ public class YARNRunner implements ClientProtocol {
LOG.info("Command to launch container for ApplicationMaster is : "
+ mergedCommand);
-
- // Setup the CLASSPATH in environment
+
+ // Setup the CLASSPATH in environment
// i.e. add { job jar, CWD, Hadoop jars} to classpath.
Map environment = new HashMap();
MRApps.setClasspath(environment);
-
+
// Parse distributed cache
MRApps.setupDistributedCache(jobConf, localResources);
@@ -374,12 +374,12 @@ public class YARNRunner implements ClientProtocol {
appContext.setUser( // User name
UserGroupInformation.getCurrentUser().getShortUserName());
appContext.setQueue( // Queue name
- jobConf.get(JobContext.QUEUE_NAME,
+ jobConf.get(JobContext.QUEUE_NAME,
YarnConfiguration.DEFAULT_QUEUE_NAME));
appContext.setApplicationName( // Job name
- jobConf.get(JobContext.JOB_NAME,
- YarnConfiguration.DEFAULT_APPLICATION_NAME));
- appContext.setAMContainerSpec(amContainer); // AM Container
+ jobConf.get(JobContext.JOB_NAME,
+ YarnConfiguration.DEFAULT_APPLICATION_NAME));
+ appContext.setAMContainerSpec(amContainer); // AM Container
return appContext;
}
@@ -394,14 +394,14 @@ public class YARNRunner implements ClientProtocol {
public long getProtocolVersion(String arg0, long arg1) throws IOException {
return resMgrDelegate.getProtocolVersion(arg0, arg1);
}
-
+
@Override
public long renewDelegationToken(Token arg0)
throws IOException, InterruptedException {
return resMgrDelegate.renewDelegationToken(arg0);
}
-
+
@Override
public Counters getJobCounters(JobID arg0) throws IOException,
InterruptedException {
@@ -419,7 +419,7 @@ public class YARNRunner implements ClientProtocol {
JobStatus status = clientCache.getClient(jobID).getJobStatus(jobID);
return status;
}
-
+
@Override
public TaskCompletionEvent[] getTaskCompletionEvents(JobID arg0, int arg1,
int arg2) throws IOException, InterruptedException {
@@ -446,8 +446,8 @@ public class YARNRunner implements ClientProtocol {
if (status.getState() != JobStatus.State.RUNNING) {
resMgrDelegate.killApplication(TypeConverter.toYarn(arg0).getAppId());
return;
- }
-
+ }
+
try {
/* send a kill to the AM */
clientCache.getClient(arg0).killJob(arg0);
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientRedirect.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientRedirect.java
index d90e7216941..5d839252eac 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientRedirect.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientRedirect.java
@@ -88,7 +88,8 @@ import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest;
import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationResponse;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ApplicationReport;
-import org.apache.hadoop.yarn.api.records.ApplicationState;
+import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
+import org.apache.hadoop.yarn.api.records.YarnApplicationState;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
import org.apache.hadoop.yarn.factories.RecordFactory;
@@ -107,17 +108,17 @@ public class TestClientRedirect {
private static final Log LOG = LogFactory.getLog(TestClientRedirect.class);
private static final String RMADDRESS = "0.0.0.0:8054";
private static final RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
-
+
private static final String AMHOSTADDRESS = "0.0.0.0:10020";
private static final String HSHOSTADDRESS = "0.0.0.0:10021";
- private volatile boolean amContact = false;
+ private volatile boolean amContact = false;
private volatile boolean hsContact = false;
private volatile boolean amRunning = false;
private volatile boolean amRestarting = false;
@Test
public void testRedirect() throws Exception {
-
+
Configuration conf = new YarnConfiguration();
conf.set(MRConfig.FRAMEWORK_NAME, MRConfig.YARN_FRAMEWORK_NAME);
conf.set(YarnConfiguration.RM_ADDRESS, RMADDRESS);
@@ -125,7 +126,7 @@ public class TestClientRedirect {
RMService rmService = new RMService("test");
rmService.init(conf);
rmService.start();
-
+
AMService amService = new AMService();
amService.init(conf);
amService.start(conf);
@@ -134,16 +135,16 @@ public class TestClientRedirect {
HistoryService historyService = new HistoryService();
historyService.init(conf);
historyService.start(conf);
-
+
LOG.info("services started");
Cluster cluster = new Cluster(conf);
org.apache.hadoop.mapreduce.JobID jobID =
new org.apache.hadoop.mapred.JobID("201103121733", 1);
- org.apache.hadoop.mapreduce.Counters counters =
+ org.apache.hadoop.mapreduce.Counters counters =
cluster.getJob(jobID).getCounters();
validateCounters(counters);
Assert.assertTrue(amContact);
-
+
LOG.info("Sleeping for 5 seconds before stop for" +
" the client socket to not get EOF immediately..");
Thread.sleep(5000);
@@ -155,17 +156,17 @@ public class TestClientRedirect {
LOG.info("Sleeping for 5 seconds after stop for" +
" the server to exit cleanly..");
Thread.sleep(5000);
-
+
amRestarting = true;
// Same client
//results are returned from fake (not started job)
counters = cluster.getJob(jobID).getCounters();
Assert.assertEquals(0, counters.countCounters());
Job job = cluster.getJob(jobID);
- org.apache.hadoop.mapreduce.TaskID taskId =
+ org.apache.hadoop.mapreduce.TaskID taskId =
new org.apache.hadoop.mapreduce.TaskID(jobID, TaskType.MAP, 0);
TaskAttemptID tId = new TaskAttemptID(taskId, 0);
-
+
//invoke all methods to check that no exception is thrown
job.killJob();
job.killTask(tId);
@@ -175,25 +176,25 @@ public class TestClientRedirect {
job.getTaskDiagnostics(tId);
job.getTaskReports(TaskType.MAP);
job.getTrackingURL();
-
+
amRestarting = false;
amService = new AMService();
amService.init(conf);
amService.start(conf);
amRunning = true;
amContact = false; //reset
-
+
counters = cluster.getJob(jobID).getCounters();
validateCounters(counters);
Assert.assertTrue(amContact);
-
+
amRunning = false;
// Same client
counters = cluster.getJob(jobID).getCounters();
validateCounters(counters);
Assert.assertTrue(hsContact);
-
+
rmService.stop();
historyService.stop();
}
@@ -248,7 +249,7 @@ public class TestClientRedirect {
public GetNewApplicationResponse getNewApplication(GetNewApplicationRequest request) throws YarnRemoteException {
return null;
}
-
+
@Override
public GetApplicationReportResponse getApplicationReport(
GetApplicationReportRequest request) throws YarnRemoteException {
@@ -256,12 +257,14 @@ public class TestClientRedirect {
ApplicationReport application = recordFactory
.newRecordInstance(ApplicationReport.class);
application.setApplicationId(applicationId);
+ application.setFinalApplicationStatus(FinalApplicationStatus.UNDEFINED);
if (amRunning) {
- application.setState(ApplicationState.RUNNING);
+ application.setYarnApplicationState(YarnApplicationState.RUNNING);
} else if (amRestarting) {
- application.setState(ApplicationState.SUBMITTED);
+ application.setYarnApplicationState(YarnApplicationState.SUBMITTED);
} else {
- application.setState(ApplicationState.SUCCEEDED);
+ application.setYarnApplicationState(YarnApplicationState.FINISHED);
+ application.setFinalApplicationStatus(FinalApplicationStatus.SUCCEEDED);
}
String[] split = AMHOSTADDRESS.split(":");
application.setHost(split[0]);
@@ -339,7 +342,7 @@ public class TestClientRedirect {
}
}
- class AMService extends AbstractService
+ class AMService extends AbstractService
implements MRClientProtocol {
private InetSocketAddress bindAddress;
private Server server;
@@ -347,7 +350,7 @@ public class TestClientRedirect {
public AMService() {
this(AMHOSTADDRESS);
}
-
+
public AMService(String hostAddress) {
super("AMService");
this.hostAddress = hostAddress;
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientServiceDelegate.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientServiceDelegate.java
index 78bfe79f304..d04d7e3d99c 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientServiceDelegate.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientServiceDelegate.java
@@ -32,8 +32,9 @@ import org.apache.hadoop.mapreduce.v2.api.protocolrecords.GetJobReportRequest;
import org.apache.hadoop.mapreduce.v2.api.protocolrecords.GetJobReportResponse;
import org.apache.hadoop.mapreduce.v2.api.records.JobReport;
import org.apache.hadoop.mapreduce.v2.api.records.JobState;
+import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
import org.apache.hadoop.yarn.api.records.ApplicationReport;
-import org.apache.hadoop.yarn.api.records.ApplicationState;
+import org.apache.hadoop.yarn.api.records.YarnApplicationState;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
import org.apache.hadoop.yarn.ipc.RPCUtil;
@@ -163,7 +164,7 @@ public class TestClientServiceDelegate {
private ApplicationReport getApplicationReport() {
ApplicationReport applicationReport = Records
.newRecord(ApplicationReport.class);
- applicationReport.setState(ApplicationState.SUCCEEDED);
+ applicationReport.setYarnApplicationState(YarnApplicationState.FINISHED);
applicationReport.setUser("root");
applicationReport.setHost("N/A");
applicationReport.setName("N/A");
@@ -172,6 +173,7 @@ public class TestClientServiceDelegate {
applicationReport.setFinishTime(0);
applicationReport.setTrackingUrl("N/A");
applicationReport.setDiagnostics("N/A");
+ applicationReport.setFinalApplicationStatus(FinalApplicationStatus.SUCCEEDED);
return applicationReport;
}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobsWithHistoryService.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobsWithHistoryService.java
index 49f0c174d9b..29c41404d5f 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobsWithHistoryService.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobsWithHistoryService.java
@@ -36,7 +36,6 @@ import org.apache.hadoop.mapreduce.Counters;
import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.TypeConverter;
import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.ApplicationState;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
import org.junit.Before;
import org.junit.After;
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestYARNRunner.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestYARNRunner.java
index 346ccd2f0da..bacf1648636 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestYARNRunner.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestYARNRunner.java
@@ -64,10 +64,10 @@ import org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoRequest;
import org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoResponse;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ApplicationReport;
-import org.apache.hadoop.yarn.api.records.ApplicationState;
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
import org.apache.hadoop.yarn.api.records.QueueInfo;
import org.apache.hadoop.yarn.api.records.YarnClusterMetrics;
+import org.apache.hadoop.yarn.api.records.YarnApplicationState;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
@@ -77,25 +77,25 @@ import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
/**
- * Test YarnRunner and make sure the client side plugin works
+ * Test YarnRunner and make sure the client side plugin works
* fine
*/
public class TestYARNRunner extends TestCase {
private static final Log LOG = LogFactory.getLog(TestYARNRunner.class);
private static final RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
-
+
private YARNRunner yarnRunner;
private ResourceMgrDelegate resourceMgrDelegate;
private YarnConfiguration conf;
private ClientCache clientCache;
private ApplicationId appId;
private JobID jobId;
- private File testWorkDir =
+ private File testWorkDir =
new File("target", TestYARNRunner.class.getName());
private ApplicationSubmissionContext submissionContext;
private ClientServiceDelegate clientDelegate;
private static final String failString = "Rejected job";
-
+
@Before
public void setUp() throws Exception {
resourceMgrDelegate = mock(ResourceMgrDelegate.class);
@@ -115,7 +115,7 @@ public class TestYARNRunner extends TestCase {
}
).when(yarnRunner).createApplicationSubmissionContext(any(Configuration.class),
any(String.class), any(Credentials.class));
-
+
appId = recordFactory.newRecordInstance(ApplicationId.class);
appId.setClusterTimestamp(System.currentTimeMillis());
appId.setId(1);
@@ -125,13 +125,13 @@ public class TestYARNRunner extends TestCase {
}
testWorkDir.mkdirs();
}
-
-
+
+
@Test
public void testJobKill() throws Exception {
clientDelegate = mock(ClientServiceDelegate.class);
- when(clientDelegate.getJobStatus(any(JobID.class))).thenReturn(new
- org.apache.hadoop.mapreduce.JobStatus(jobId, 0f, 0f, 0f, 0f,
+ when(clientDelegate.getJobStatus(any(JobID.class))).thenReturn(new
+ org.apache.hadoop.mapreduce.JobStatus(jobId, 0f, 0f, 0f, 0f,
State.PREP, JobPriority.HIGH, "tmp", "tmp", "tmp", "tmp"));
when(clientDelegate.killJob(any(JobID.class))).thenReturn(true);
doAnswer(
@@ -145,13 +145,13 @@ public class TestYARNRunner extends TestCase {
).when(clientCache).getClient(any(JobID.class));
yarnRunner.killJob(jobId);
verify(resourceMgrDelegate).killApplication(appId);
- when(clientDelegate.getJobStatus(any(JobID.class))).thenReturn(new
- org.apache.hadoop.mapreduce.JobStatus(jobId, 0f, 0f, 0f, 0f,
+ when(clientDelegate.getJobStatus(any(JobID.class))).thenReturn(new
+ org.apache.hadoop.mapreduce.JobStatus(jobId, 0f, 0f, 0f, 0f,
State.RUNNING, JobPriority.HIGH, "tmp", "tmp", "tmp", "tmp"));
yarnRunner.killJob(jobId);
verify(clientDelegate).killJob(jobId);
}
-
+
@Test
public void testJobSubmissionFailure() throws Exception {
when(resourceMgrDelegate.submitApplication(any(ApplicationSubmissionContext.class))).
@@ -159,7 +159,7 @@ public class TestYARNRunner extends TestCase {
ApplicationReport report = mock(ApplicationReport.class);
when(report.getApplicationId()).thenReturn(appId);
when(report.getDiagnostics()).thenReturn(failString);
- when(report.getState()).thenReturn(ApplicationState.FAILED);
+ when(report.getYarnApplicationState()).thenReturn(YarnApplicationState.FAILED);
when(resourceMgrDelegate.getApplicationReport(appId)).thenReturn(report);
Credentials credentials = new Credentials();
File jobxml = new File(testWorkDir, "job.xml");
@@ -167,13 +167,13 @@ public class TestYARNRunner extends TestCase {
conf.writeXml(out);
out.close();
try {
- yarnRunner.submitJob(jobId, testWorkDir.getAbsolutePath().toString(), credentials);
+ yarnRunner.submitJob(jobId, testWorkDir.getAbsolutePath().toString(), credentials);
} catch(IOException io) {
LOG.info("Logging exception:", io);
assertTrue(io.getLocalizedMessage().contains(failString));
}
}
-
+
@Test
public void testResourceMgrDelegate() throws Exception {
/* we not want a mock of resourcemgr deleagte */
@@ -184,19 +184,19 @@ public class TestYARNRunner extends TestCase {
.thenReturn(null);
delegate.killApplication(appId);
verify(clientRMProtocol).forceKillApplication(any(KillApplicationRequest.class));
-
+
/* make sure getalljobs calls get all applications */
when(clientRMProtocol.getAllApplications(any(GetAllApplicationsRequest.class))).
thenReturn(recordFactory.newRecordInstance(GetAllApplicationsResponse.class));
delegate.getAllJobs();
verify(clientRMProtocol).getAllApplications(any(GetAllApplicationsRequest.class));
-
+
/* make sure getapplication report is called */
when(clientRMProtocol.getApplicationReport(any(GetApplicationReportRequest.class)))
.thenReturn(recordFactory.newRecordInstance(GetApplicationReportResponse.class));
delegate.getApplicationReport(appId);
verify(clientRMProtocol).getApplicationReport(any(GetApplicationReportRequest.class));
-
+
/* make sure metrics is called */
GetClusterMetricsResponse clusterMetricsResponse = recordFactory.newRecordInstance
(GetClusterMetricsResponse.class);
@@ -206,7 +206,7 @@ public class TestYARNRunner extends TestCase {
.thenReturn(clusterMetricsResponse);
delegate.getClusterMetrics();
verify(clientRMProtocol).getClusterMetrics(any(GetClusterMetricsRequest.class));
-
+
when(clientRMProtocol.getClusterNodes(any(GetClusterNodesRequest.class))).
thenReturn(recordFactory.newRecordInstance(GetClusterNodesResponse.class));
delegate.getActiveTrackers();
@@ -227,7 +227,7 @@ public class TestYARNRunner extends TestCase {
thenReturn(queueInfoResponse);
delegate.getQueues();
verify(clientRMProtocol).getQueueInfo(any(GetQueueInfoRequest.class));
-
+
GetQueueUserAclsInfoResponse aclResponse = recordFactory.newRecordInstance(
GetQueueUserAclsInfoResponse.class);
when(clientRMProtocol.getQueueUserAcls(any(GetQueueUserAclsInfoRequest.class)))
diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/FinishApplicationMasterRequest.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/FinishApplicationMasterRequest.java
index 36d2af22f96..7a09e774523 100644
--- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/FinishApplicationMasterRequest.java
+++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/FinishApplicationMasterRequest.java
@@ -22,15 +22,16 @@ import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Stable;
import org.apache.hadoop.yarn.api.AMRMProtocol;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
/**
- * The finalization request sent by the ApplicationMaster
to
+ *
The finalization request sent by the ApplicationMaster
to
* inform the ResourceManager
about its completion.
- *
+ *
* The final request includes details such:
*
* -
- * {@link ApplicationAttemptId} being managed by the
+ * {@link ApplicationAttemptId} being managed by the
*
ApplicationMaster
*
* - Final state of the
ApplicationMaster
@@ -47,19 +48,19 @@ import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
public interface FinishApplicationMasterRequest {
/**
- * Get the ApplicationAttemptId
being managed by the
+ * Get the ApplicationAttemptId
being managed by the
* ApplicationMaster
.
- * @return ApplicationAttemptId
being managed by the
+ * @return ApplicationAttemptId
being managed by the
* ApplicationMaster
*/
@Public
@Stable
ApplicationAttemptId getApplicationAttemptId();
-
+
/**
- * Set the ApplicationAttemptId
being managed by the
+ * Set the ApplicationAttemptId
being managed by the
* ApplicationMaster
.
- * @param applicationAttemptId ApplicationAttemptId
being managed
+ * @param applicationAttemptId ApplicationAttemptId
being managed
* by the ApplicationMaster
*/
@Public
@@ -72,15 +73,15 @@ public interface FinishApplicationMasterRequest {
*/
@Public
@Stable
- String getFinalState();
-
+ FinalApplicationStatus getFinalApplicationStatus();
+
/**
- * Set final state of the ApplicationMaster
- * @param finalState final state of the ApplicationMaster
+ * Set the finish state of the ApplicationMaster
+ * @param finishState finish state of the ApplicationMaster
*/
@Public
@Stable
- void setFinalState(String finalState);
+ void setFinishApplicationStatus(FinalApplicationStatus finishState);
/**
* Get diagnostic information on application failure.
@@ -89,7 +90,7 @@ public interface FinishApplicationMasterRequest {
@Public
@Stable
String getDiagnostics();
-
+
/**
* Set diagnostic information on application failure.
* @param diagnostics diagnostic information on application failure
@@ -105,10 +106,10 @@ public interface FinishApplicationMasterRequest {
@Public
@Stable
String getTrackingUrl();
-
+
/**
* Set the tracking URLfor the ApplicationMaster
- * @param url tracking URLfor the
+ * @param url tracking URLfor the
* ApplicationMaster
*/
@Public
diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/FinishApplicationMasterRequestPBImpl.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/FinishApplicationMasterRequestPBImpl.java
index 7ec5abe1205..01b37388192 100644
--- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/FinishApplicationMasterRequestPBImpl.java
+++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/FinishApplicationMasterRequestPBImpl.java
@@ -21,23 +21,24 @@ package org.apache.hadoop.yarn.api.protocolrecords.impl.pb;
import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
import org.apache.hadoop.yarn.api.records.ProtoBase;
import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationAttemptIdPBImpl;
import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationAttemptIdProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.FinalApplicationStatusProto;
import org.apache.hadoop.yarn.proto.YarnServiceProtos.FinishApplicationMasterRequestProto;
import org.apache.hadoop.yarn.proto.YarnServiceProtos.FinishApplicationMasterRequestProtoOrBuilder;
-import org.apache.hadoop.yarn.proto.YarnServiceProtos.RegisterApplicationMasterRequestProtoOrBuilder;
+import org.apache.hadoop.yarn.util.ProtoUtils;
-
public class FinishApplicationMasterRequestPBImpl extends ProtoBase implements FinishApplicationMasterRequest {
FinishApplicationMasterRequestProto proto = FinishApplicationMasterRequestProto.getDefaultInstance();
FinishApplicationMasterRequestProto.Builder builder = null;
boolean viaProto = false;
-
+
private ApplicationAttemptId appAttemptId = null;
-
-
+
+
public FinishApplicationMasterRequestPBImpl() {
builder = FinishApplicationMasterRequestProto.newBuilder();
}
@@ -46,7 +47,7 @@ public class FinishApplicationMasterRequestPBImpl extends ProtoBase
+public class RegisterApplicationMasterResponsePBImpl
+extends ProtoBase
implements RegisterApplicationMasterResponse {
- RegisterApplicationMasterResponseProto proto =
+ RegisterApplicationMasterResponseProto proto =
RegisterApplicationMasterResponseProto.getDefaultInstance();
RegisterApplicationMasterResponseProto.Builder builder = null;
boolean viaProto = false;
-
+
private Resource minimumResourceCapability;
private Resource maximumResourceCapability;
-
+
public RegisterApplicationMasterResponsePBImpl() {
builder = RegisterApplicationMasterResponseProto.newBuilder();
}
@@ -49,16 +47,16 @@ implements RegisterApplicationMasterResponse {
this.proto = proto;
viaProto = true;
}
-
+
public RegisterApplicationMasterResponseProto getProto() {
mergeLocalToProto();
proto = viaProto ? proto : builder.build();
viaProto = true;
return proto;
}
-
+
private void mergeLocalToProto() {
- if (viaProto)
+ if (viaProto)
maybeInitBuilder();
mergeLocalToBuilder();
proto = builder.build();
@@ -94,7 +92,7 @@ implements RegisterApplicationMasterResponse {
if (!p.hasMaximumCapability()) {
return null;
}
-
+
this.maximumResourceCapability = convertFromProtoFormat(p.getMaximumCapability());
return this.maximumResourceCapability;
}
@@ -109,7 +107,7 @@ implements RegisterApplicationMasterResponse {
if (!p.hasMinimumCapability()) {
return null;
}
-
+
this.minimumResourceCapability = convertFromProtoFormat(p.getMinimumCapability());
return this.minimumResourceCapability;
}
@@ -140,4 +138,4 @@ implements RegisterApplicationMasterResponse {
return ((ResourcePBImpl)resource).getProto();
}
-}
+}
diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationMaster.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationMaster.java
index 3137009f48c..c3103cb16e4 100644
--- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationMaster.java
+++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationMaster.java
@@ -22,38 +22,38 @@ import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.classification.InterfaceStability.Unstable;
/**
- * For internal use only...
+ * For internal use only...
*/
@Private
@Unstable
public interface ApplicationMaster {
ApplicationId getApplicationId();
void setApplicationId(ApplicationId appId);
-
+
String getHost();
void setHost(String host);
-
+
int getRpcPort();
void setRpcPort(int rpcPort);
-
+
String getTrackingUrl();
void setTrackingUrl(String url);
-
+
ApplicationStatus getStatus();
void setStatus(ApplicationStatus status);
-
- ApplicationState getState();
- void setState(ApplicationState state);
-
+
+ YarnApplicationState getState();
+ void setState(YarnApplicationState state);
+
String getClientToken();
void setClientToken(String clientToken);
-
+
int getAMFailCount();
void setAMFailCount(int amFailCount);
-
+
int getContainerCount();
void setContainerCount(int containerCount);
-
+
String getDiagnostics();
void setDiagnostics(String diagnostics);
}
diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationReport.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationReport.java
index ffb920d5b90..254a87878e0 100644
--- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationReport.java
+++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationReport.java
@@ -36,13 +36,13 @@ import org.apache.hadoop.yarn.api.ClientRMProtocol;
* - Host on which the
ApplicationMaster
is running.
* - RPC port of the
ApplicationMaster
.
* - Tracking URL.
- * - {@link ApplicationState} of the application.
+ * - {@link YarnApplicationState} of the application.
* - Diagnostic information in case of errors.
* - Start time of the application.
* - Client token of the application (if security is enabled).
*
*
- *
+ *
* @see ClientRMProtocol#getApplicationReport(org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest)
*/
@Public
@@ -56,7 +56,7 @@ public interface ApplicationReport {
@Public
@Stable
ApplicationId getApplicationId();
-
+
@Private
@Unstable
void setApplicationId(ApplicationId applicationId);
@@ -68,7 +68,7 @@ public interface ApplicationReport {
@Public
@Stable
String getUser();
-
+
@Private
@Unstable
void setUser(String user);
@@ -80,7 +80,7 @@ public interface ApplicationReport {
@Public
@Stable
String getQueue();
-
+
@Private
@Unstable
void setQueue(String queue);
@@ -92,21 +92,21 @@ public interface ApplicationReport {
@Public
@Stable
String getName();
-
+
@Private
@Unstable
void setName(String name);
/**
- * Get the host on which the ApplicationMaster
+ * Get the host on which the ApplicationMaster
* is running.
- * @return host on which the ApplicationMaster
+ * @return host on which the ApplicationMaster
* is running
*/
@Public
@Stable
String getHost();
-
+
@Private
@Unstable
void setHost(String host);
@@ -118,47 +118,47 @@ public interface ApplicationReport {
@Public
@Stable
int getRpcPort();
-
+
@Private
@Unstable
void setRpcPort(int rpcPort);
/**
- * Get the client token for communicating with the
+ * Get the client token for communicating with the
* ApplicationMaster
.
- * @return client token for communicating with the
+ * @return client token for communicating with the
* ApplicationMaster
*/
@Public
@Stable
String getClientToken();
-
+
@Private
@Unstable
void setClientToken(String clientToken);
/**
- * Get the ApplicationState
of the application.
- * @return ApplicationState
of the application
+ * Get the YarnApplicationState
of the application.
+ * @return YarnApplicationState
of the application
*/
@Public
@Stable
- ApplicationState getState();
-
+ YarnApplicationState getYarnApplicationState();
+
@Private
@Unstable
- void setState(ApplicationState state);
+ void setYarnApplicationState(YarnApplicationState state);
/**
- * Get the diagnositic information of the application in case of
+ * Get the diagnositic information of the application in case of
* errors.
- * @return diagnositic information of the application in case
+ * @return diagnositic information of the application in case
* of errors
*/
@Public
@Stable
String getDiagnostics();
-
+
@Private
@Unstable
void setDiagnostics(String diagnostics);
@@ -170,11 +170,11 @@ public interface ApplicationReport {
@Public
@Stable
String getTrackingUrl();
-
+
@Private
@Unstable
void setTrackingUrl(String url);
-
+
/**
* Get the start time of the application.
* @return start time of the application
@@ -182,7 +182,7 @@ public interface ApplicationReport {
@Public
@Stable
long getStartTime();
-
+
@Private
@Unstable
void setStartTime(long startTime);
@@ -194,8 +194,21 @@ public interface ApplicationReport {
@Public
@Stable
long getFinishTime();
-
+
@Private
@Unstable
void setFinishTime(long finishTime);
+
+
+ /**
+ * Get the final finish status of the application.
+ */
+ @Public
+ @Stable
+ FinalApplicationStatus getFinalApplicationStatus();
+
+ @Private
+ @Unstable
+ void setFinalApplicationStatus(FinalApplicationStatus finishState);
+
}
diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/FinalApplicationStatus.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/FinalApplicationStatus.java
new file mode 100644
index 00000000000..170c25ae845
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/FinalApplicationStatus.java
@@ -0,0 +1,42 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.api.records;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Stable;
+
+/**
+ * Enumeration of various final states of an Application
.
+ */
+@Public
+@Stable
+public enum FinalApplicationStatus {
+
+ /** Undefined state when either the application has not yet finished */
+ UNDEFINED,
+
+ /** Application which finished successfully. */
+ SUCCEEDED,
+
+ /** Application which failed. */
+ FAILED,
+
+ /** Application which was terminated by a user or admin. */
+ KILLED
+}
diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationState.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/YarnApplicationState.java
similarity index 84%
rename from hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationState.java
rename to hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/YarnApplicationState.java
index 6fcdea5cac9..c45b62d54ac 100644
--- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationState.java
+++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/YarnApplicationState.java
@@ -22,26 +22,26 @@ import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Stable;
/**
- * Ennumeration of various states of an Application
.
+ * Ennumeration of various states of an ApplicationMaster
.
*/
@Public
@Stable
-public enum ApplicationState {
+public enum YarnApplicationState {
/** Application which was just created. */
- NEW,
-
+ NEW,
+
/** Application which has been submitted. */
- SUBMITTED,
-
+ SUBMITTED,
+
/** Application which is currently running. */
- RUNNING,
-
- /** Application which completed successfully. */
- SUCCEEDED,
-
+ RUNNING,
+
+ /** Application which finished successfully. */
+ FINISHED,
+
/** Application which failed. */
- FAILED,
-
+ FAILED,
+
/** Application which was terminated by a user or admin. */
KILLED
}
diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationMasterPBImpl.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationMasterPBImpl.java
index ee98eed5c47..123e178806b 100644
--- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationMasterPBImpl.java
+++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationMasterPBImpl.java
@@ -19,31 +19,28 @@
package org.apache.hadoop.yarn.api.records.impl.pb;
-import java.util.List;
-
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ApplicationMaster;
-import org.apache.hadoop.yarn.api.records.ApplicationState;
import org.apache.hadoop.yarn.api.records.ApplicationStatus;
import org.apache.hadoop.yarn.api.records.ProtoBase;
+import org.apache.hadoop.yarn.api.records.YarnApplicationState;
import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationIdProto;
import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationMasterProto;
import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationMasterProtoOrBuilder;
-import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationStateProto;
import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationStatusProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.YarnApplicationStateProto;
import org.apache.hadoop.yarn.util.ProtoUtils;
-
public class ApplicationMasterPBImpl extends ProtoBase implements ApplicationMaster {
ApplicationMasterProto proto = ApplicationMasterProto.getDefaultInstance();
ApplicationMasterProto.Builder builder = null;
boolean viaProto = false;
-
+
private ApplicationId applicationId = null;
private ApplicationStatus applicationStatus = null;
-
-
+
+
public ApplicationMasterPBImpl() {
builder = ApplicationMasterProto.newBuilder();
}
@@ -52,9 +49,9 @@ public class ApplicationMasterPBImpl extends ProtoBase i
this.proto = proto;
viaProto = true;
}
-
+
public ApplicationMasterProto getProto() {
-
+
mergeLocalToProto();
proto = viaProto ? proto : builder.build();
viaProto = true;
@@ -72,24 +69,24 @@ public class ApplicationMasterPBImpl extends ProtoBase i
}
private void mergeLocalToProto() {
- if (viaProto)
+ if (viaProto)
maybeInitBuilder();
mergeLocalToBuilder();
proto = builder.build();
-
+
viaProto = true;
}
-
+
private void maybeInitBuilder() {
if (viaProto || builder == null) {
builder = ApplicationMasterProto.newBuilder(proto);
}
viaProto = false;
}
-
-
+
+
@Override
- public ApplicationState getState() {
+ public YarnApplicationState getState() {
ApplicationMasterProtoOrBuilder p = viaProto ? proto : builder;
if (!p.hasState()) {
return null;
@@ -98,7 +95,7 @@ public class ApplicationMasterPBImpl extends ProtoBase i
}
@Override
- public void setState(ApplicationState state) {
+ public void setState(YarnApplicationState state) {
maybeInitBuilder();
if (state == null) {
builder.clearState();
@@ -124,7 +121,7 @@ public class ApplicationMasterPBImpl extends ProtoBase i
}
builder.setHost((host));
}
-
+
@Override
public ApplicationId getApplicationId() {
ApplicationMasterProtoOrBuilder p = viaProto ? proto : builder;
@@ -135,7 +132,7 @@ public class ApplicationMasterPBImpl extends ProtoBase i
return null;
}
applicationId = convertFromProtoFormat(p.getApplicationId());
-
+
return applicationId;
}
@@ -145,7 +142,7 @@ public class ApplicationMasterPBImpl extends ProtoBase i
if (applicationId == null)
builder.clearApplicationId();
this.applicationId = applicationId;
-
+
}
@Override
public int getRpcPort() {
@@ -179,7 +176,7 @@ public class ApplicationMasterPBImpl extends ProtoBase i
return null;
}
this.applicationStatus = convertFromProtoFormat(p.getStatus());
-
+
return this.applicationStatus;
}
@@ -189,7 +186,7 @@ public class ApplicationMasterPBImpl extends ProtoBase i
if (status == null)
builder.clearStatus();
this.applicationStatus = status;
-
+
}
@Override
public String getClientToken() {
@@ -209,7 +206,7 @@ public class ApplicationMasterPBImpl extends ProtoBase i
}
builder.setClientToken((clientToken));
}
-
+
@Override
public int getAMFailCount() {
ApplicationMasterProtoOrBuilder p = viaProto ? proto : builder;
@@ -250,11 +247,11 @@ public class ApplicationMasterPBImpl extends ProtoBase i
builder.setDiagnostics(diagnostics);
}
- private ApplicationStateProto convertToProtoFormat(ApplicationState e) {
+ private YarnApplicationStateProto convertToProtoFormat(YarnApplicationState e) {
return ProtoUtils.convertToProtoFormat(e);
}
- private ApplicationState convertFromProtoFormat(ApplicationStateProto e) {
+ private YarnApplicationState convertFromProtoFormat(YarnApplicationStateProto e) {
return ProtoUtils.convertFromProtoFormat(e);
}
@@ -274,4 +271,4 @@ public class ApplicationMasterPBImpl extends ProtoBase i
return ((ApplicationStatusPBImpl)t).getProto();
}
-}
+}
diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationReportPBImpl.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationReportPBImpl.java
index 2ea2ddbcdb2..bdb2f8d6a28 100644
--- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationReportPBImpl.java
+++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationReportPBImpl.java
@@ -18,17 +18,19 @@
package org.apache.hadoop.yarn.api.records.impl.pb;
+import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ApplicationReport;
-import org.apache.hadoop.yarn.api.records.ApplicationState;
+import org.apache.hadoop.yarn.api.records.YarnApplicationState;
import org.apache.hadoop.yarn.api.records.ProtoBase;
import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationIdProto;
import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationReportProto;
import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationReportProtoOrBuilder;
-import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationStateProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.FinalApplicationStatusProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.YarnApplicationStateProto;
import org.apache.hadoop.yarn.util.ProtoUtils;
-public class ApplicationReportPBImpl extends ProtoBase
+public class ApplicationReportPBImpl extends ProtoBase
implements ApplicationReport {
ApplicationReportProto proto = ApplicationReportProto.getDefaultInstance();
ApplicationReportProto.Builder builder = null;
@@ -39,7 +41,7 @@ implements ApplicationReport {
public ApplicationReportPBImpl() {
builder = ApplicationReportProto.newBuilder();
}
-
+
public ApplicationReportPBImpl(ApplicationReportProto proto) {
this.proto = proto;
viaProto = true;
@@ -87,12 +89,12 @@ implements ApplicationReport {
}
@Override
- public ApplicationState getState() {
+ public YarnApplicationState getYarnApplicationState() {
ApplicationReportProtoOrBuilder p = viaProto ? proto : builder;
- if (!p.hasState()) {
+ if (!p.hasYarnApplicationState()) {
return null;
}
- return convertFromProtoFormat(p.getState());
+ return convertFromProtoFormat(p.getYarnApplicationState());
}
@Override
@@ -138,6 +140,27 @@ implements ApplicationReport {
return p.getDiagnostics();
}
+ @Override
+ public long getStartTime() {
+ ApplicationReportProtoOrBuilder p = viaProto ? proto : builder;
+ return p.getStartTime();
+ }
+
+ @Override
+ public long getFinishTime() {
+ ApplicationReportProtoOrBuilder p = viaProto ? proto : builder;
+ return p.getFinishTime();
+ }
+
+ @Override
+ public FinalApplicationStatus getFinalApplicationStatus() {
+ ApplicationReportProtoOrBuilder p = viaProto ? proto : builder;
+ if (!p.hasFinalApplicationStatus()) {
+ return null;
+ }
+ return convertFromProtoFormat(p.getFinalApplicationStatus());
+ }
+
@Override
public void setApplicationId(ApplicationId applicationId) {
maybeInitBuilder();
@@ -177,13 +200,13 @@ implements ApplicationReport {
}
@Override
- public void setState(ApplicationState state) {
+ public void setYarnApplicationState(YarnApplicationState state) {
maybeInitBuilder();
if (state == null) {
- builder.clearState();
+ builder.clearYarnApplicationState();
return;
}
- builder.setState(convertToProtoFormat(state));
+ builder.setYarnApplicationState(convertToProtoFormat(state));
}
@Override
@@ -232,38 +255,36 @@ implements ApplicationReport {
builder.setDiagnostics(diagnostics);
}
- @Override
- public ApplicationReportProto getProto() {
- mergeLocalToProto();
- proto = viaProto ? proto : builder.build();
- viaProto = true;
- return proto;
- }
-
- @Override
- public long getStartTime() {
- ApplicationReportProtoOrBuilder p = viaProto ? proto : builder;
- return p.getStartTime();
- }
-
@Override
public void setStartTime(long startTime) {
maybeInitBuilder();
builder.setStartTime(startTime);
}
- @Override
- public long getFinishTime() {
- ApplicationReportProtoOrBuilder p = viaProto ? proto : builder;
- return p.getFinishTime();
- }
-
@Override
public void setFinishTime(long finishTime) {
maybeInitBuilder();
builder.setFinishTime(finishTime);
}
+ @Override
+ public void setFinalApplicationStatus(FinalApplicationStatus finishState) {
+ maybeInitBuilder();
+ if (finishState == null) {
+ builder.clearFinalApplicationStatus();
+ return;
+ }
+ builder.setFinalApplicationStatus(convertToProtoFormat(finishState));
+ }
+
+ @Override
+ public ApplicationReportProto getProto() {
+ mergeLocalToProto();
+ proto = viaProto ? proto : builder.build();
+ viaProto = true;
+ return proto;
+ }
+
private void mergeLocalToBuilder() {
if (this.applicationId != null
&& !((ApplicationIdPBImpl) this.applicationId).getProto().equals(
@@ -291,16 +312,25 @@ implements ApplicationReport {
return ((ApplicationIdPBImpl) t).getProto();
}
- private ApplicationState convertFromProtoFormat(ApplicationStateProto s) {
- return ProtoUtils.convertFromProtoFormat(s);
- }
-
- private ApplicationStateProto convertToProtoFormat(ApplicationState s) {
- return ProtoUtils.convertToProtoFormat(s);
- }
-
private ApplicationIdPBImpl convertFromProtoFormat(
ApplicationIdProto applicationId) {
return new ApplicationIdPBImpl(applicationId);
}
+
+ private YarnApplicationState convertFromProtoFormat(YarnApplicationStateProto s) {
+ return ProtoUtils.convertFromProtoFormat(s);
+ }
+
+ private YarnApplicationStateProto convertToProtoFormat(YarnApplicationState s) {
+ return ProtoUtils.convertToProtoFormat(s);
+ }
+
+ private FinalApplicationStatus convertFromProtoFormat(FinalApplicationStatusProto s) {
+ return ProtoUtils.convertFromProtoFormat(s);
+ }
+
+ private FinalApplicationStatusProto convertToProtoFormat(FinalApplicationStatus s) {
+ return ProtoUtils.convertToProtoFormat(s);
+ }
+
}
diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/ProtoUtils.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/ProtoUtils.java
index 1451847111b..927764b0630 100644
--- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/ProtoUtils.java
+++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/util/ProtoUtils.java
@@ -20,24 +20,26 @@ package org.apache.hadoop.yarn.util;
import java.nio.ByteBuffer;
-import org.apache.hadoop.yarn.api.records.ApplicationState;
+import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
import org.apache.hadoop.yarn.api.records.ContainerState;
import org.apache.hadoop.yarn.api.records.LocalResourceType;
import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
import org.apache.hadoop.yarn.api.records.QueueACL;
import org.apache.hadoop.yarn.api.records.QueueState;
-import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationStateProto;
+import org.apache.hadoop.yarn.api.records.YarnApplicationState;
import org.apache.hadoop.yarn.proto.YarnProtos.ContainerStateProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.FinalApplicationStatusProto;
import org.apache.hadoop.yarn.proto.YarnProtos.LocalResourceTypeProto;
import org.apache.hadoop.yarn.proto.YarnProtos.LocalResourceVisibilityProto;
import org.apache.hadoop.yarn.proto.YarnProtos.QueueACLProto;
import org.apache.hadoop.yarn.proto.YarnProtos.QueueStateProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.YarnApplicationStateProto;
import com.google.protobuf.ByteString;
public class ProtoUtils {
-
-
+
+
/*
* ContainerState
*/
@@ -48,18 +50,29 @@ public class ProtoUtils {
public static ContainerState convertFromProtoFormat(ContainerStateProto e) {
return ContainerState.valueOf(e.name().replace(CONTAINER_STATE_PREFIX, ""));
}
-
+
/*
- * ApplicationState
+ * YarnApplicationState
*/
- public static ApplicationStateProto convertToProtoFormat(ApplicationState e) {
- return ApplicationStateProto.valueOf(e.name());
+ public static YarnApplicationStateProto convertToProtoFormat(YarnApplicationState e) {
+ return YarnApplicationStateProto.valueOf(e.name());
}
- public static ApplicationState convertFromProtoFormat(ApplicationStateProto e) {
- return ApplicationState.valueOf(e.name());
+ public static YarnApplicationState convertFromProtoFormat(YarnApplicationStateProto e) {
+ return YarnApplicationState.valueOf(e.name());
}
-
+
+ /*
+ * FinalApplicationStatus
+ */
+ private static String FINAL_APPLICATION_STATUS_PREFIX = "APP_";
+ public static FinalApplicationStatusProto convertToProtoFormat(FinalApplicationStatus e) {
+ return FinalApplicationStatusProto.valueOf(FINAL_APPLICATION_STATUS_PREFIX + e.name());
+ }
+ public static FinalApplicationStatus convertFromProtoFormat(FinalApplicationStatusProto e) {
+ return FinalApplicationStatus.valueOf(e.name().replace(FINAL_APPLICATION_STATUS_PREFIX, ""));
+ }
+
/*
* LocalResourceType
*/
@@ -69,7 +82,7 @@ public class ProtoUtils {
public static LocalResourceType convertFromProtoFormat(LocalResourceTypeProto e) {
return LocalResourceType.valueOf(e.name());
}
-
+
/*
* LocalResourceVisibility
*/
@@ -79,7 +92,7 @@ public class ProtoUtils {
public static LocalResourceVisibility convertFromProtoFormat(LocalResourceVisibilityProto e) {
return LocalResourceVisibility.valueOf(e.name());
}
-
+
/*
* ByteBuffer
*/
@@ -98,7 +111,7 @@ public class ProtoUtils {
byteBuffer.position(oldPos);
return bs;
}
-
+
/*
* QueueState
*/
@@ -109,7 +122,7 @@ public class ProtoUtils {
public static QueueState convertFromProtoFormat(QueueStateProto e) {
return QueueState.valueOf(e.name().replace(QUEUE_STATE_PREFIX, ""));
}
-
+
/*
* QueueACL
*/
diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
index 704c7109964..14bfb11496f 100644
--- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
+++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
@@ -76,14 +76,20 @@ message ContainerProto {
optional ContainerStatusProto container_status = 8;
}
-enum ApplicationStateProto {
+enum YarnApplicationStateProto {
NEW = 1;
SUBMITTED = 2;
RUNNING = 3;
- RESTARTING = 4;
- SUCCEEDED = 5;
- FAILED = 6;
- KILLED = 7;
+ FINISHED = 4;
+ FAILED = 5;
+ KILLED = 6;
+}
+
+enum FinalApplicationStatusProto {
+ APP_UNDEFINED = 0;
+ APP_SUCCEEDED = 1;
+ APP_FAILED = 2;
+ APP_KILLED = 3;
}
message ApplicationStatusProto {
@@ -98,7 +104,7 @@ message ApplicationMasterProto {
optional int32 rpc_port = 3;
optional string trackingUrl = 4;
optional ApplicationStatusProto status = 5;
- optional ApplicationStateProto state = 6;
+ optional YarnApplicationStateProto state = 6;
optional string client_token = 7;
optional int32 containerCount = 8;
optional int32 amFailCount = 9;
@@ -107,7 +113,7 @@ message ApplicationMasterProto {
message URLProto {
optional string scheme = 1;
- optional string host = 2;
+ optional string host = 2;
optional int32 port = 3;
optional string file = 4;
}
@@ -140,12 +146,13 @@ message ApplicationReportProto {
optional int32 rpc_port = 6;
optional string client_token = 7;
optional ApplicationStatusProto status = 8;
- optional ApplicationStateProto state = 9;
+ optional YarnApplicationStateProto yarn_application_state = 9;
optional ContainerProto masterContainer = 10;
optional string trackingUrl = 11;
optional string diagnostics = 12 [default = "N/A"];
optional int64 startTime = 13;
optional int64 finishTime = 14;
+ optional FinalApplicationStatusProto final_application_status = 15;
}
message NodeIdProto {
@@ -195,7 +202,7 @@ message AMResponseProto {
message ApplicationSubmissionContextProto {
optional ApplicationIdProto application_id = 1;
optional string application_name = 2 [default = "N/A"];
- optional string user = 3;
+ optional string user = 3;
optional string queue = 4 [default = "default"];
optional PriorityProto priority = 5;
optional ContainerLaunchContextProto am_container_spec = 6;
diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
index 1a992ad578e..f477048ca75 100644
--- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
+++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
@@ -42,7 +42,7 @@ message FinishApplicationMasterRequestProto {
optional ApplicationAttemptIdProto application_attempt_id = 1;
optional string diagnostics = 2;
optional string tracking_url = 3;
- optional string final_state = 4;
+ optional FinalApplicationStatusProto final_application_status = 4;
}
message FinishApplicationMasterResponseProto {
@@ -115,7 +115,7 @@ message GetClusterNodesRequestProto {
}
message GetClusterNodesResponseProto {
- repeated NodeReportProto nodeReports = 1;
+ repeated NodeReportProto nodeReports = 1;
}
message GetQueueInfoRequestProto {
@@ -133,7 +133,7 @@ message GetQueueUserAclsInfoRequestProto {
}
message GetQueueUserAclsInfoResponseProto {
- repeated QueueUserACLInfoProto queueUserAcls = 1;
+ repeated QueueUserACLInfoProto queueUserAcls = 1;
}
//////////////////////////////////////////////////////
diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/BuilderUtils.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/BuilderUtils.java
index 7ec367292e1..9c485bf3ef1 100644
--- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/BuilderUtils.java
+++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/BuilderUtils.java
@@ -24,9 +24,10 @@ import java.util.List;
import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ApplicationReport;
-import org.apache.hadoop.yarn.api.records.ApplicationState;
+import org.apache.hadoop.yarn.api.records.YarnApplicationState;
import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.ContainerState;
@@ -69,12 +70,12 @@ public class BuilderUtils {
}
}
- public static class ResourceRequestComparator
+ public static class ResourceRequestComparator
implements java.util.Comparator {
@Override
public int compare(org.apache.hadoop.yarn.api.records.ResourceRequest r1,
org.apache.hadoop.yarn.api.records.ResourceRequest r2) {
-
+
// Compare priority, host and capability
int ret = r1.getPriority().compareTo(r2.getPriority());
if (ret == 0) {
@@ -198,12 +199,12 @@ public class BuilderUtils {
String nodeHttpAddress, Resource resource, Priority priority) {
ContainerId containerID =
newContainerId(recordFactory, appAttemptId, containerId);
- return newContainer(containerID, nodeId, nodeHttpAddress,
+ return newContainer(containerID, nodeId, nodeHttpAddress,
resource, priority);
}
public static Container newContainer(ContainerId containerId,
- NodeId nodeId, String nodeHttpAddress,
+ NodeId nodeId, String nodeHttpAddress,
Resource resource, Priority priority) {
Container container = recordFactory.newRecordInstance(Container.class);
container.setId(containerId);
@@ -242,8 +243,9 @@ public class BuilderUtils {
public static ApplicationReport newApplicationReport(
ApplicationId applicationId, String user, String queue, String name,
- String host, int rpcPort, String clientToken, ApplicationState state,
- String diagnostics, String url, long startTime, long finishTime) {
+ String host, int rpcPort, String clientToken, YarnApplicationState state,
+ String diagnostics, String url, long startTime, long finishTime,
+ FinalApplicationStatus finalStatus) {
ApplicationReport report = recordFactory
.newRecordInstance(ApplicationReport.class);
report.setApplicationId(applicationId);
@@ -253,20 +255,21 @@ public class BuilderUtils {
report.setHost(host);
report.setRpcPort(rpcPort);
report.setClientToken(clientToken);
- report.setState(state);
+ report.setYarnApplicationState(state);
report.setDiagnostics(diagnostics);
report.setTrackingUrl(url);
report.setStartTime(startTime);
report.setFinishTime(finishTime);
+ report.setFinalApplicationStatus(finalStatus);
return report;
}
-
+
public static Resource newResource(int memory) {
Resource resource = recordFactory.newRecordInstance(Resource.class);
resource.setMemory(memory);
return resource;
}
-
+
public static URL newURL(String scheme, String host, int port, String file) {
URL url = recordFactory.newRecordInstance(URL.class);
url.setScheme(scheme);
diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ConverterUtils.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ConverterUtils.java
index 6f5e9043192..296c9d75a12 100644
--- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ConverterUtils.java
+++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ConverterUtils.java
@@ -33,7 +33,9 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
import org.apache.hadoop.yarn.api.records.URL;
+import org.apache.hadoop.yarn.api.records.YarnApplicationState;
import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
@@ -181,4 +183,5 @@ public class ConverterUtils {
+ applicationAttmeptIdStr, n);
}
}
+
}
diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/MockApps.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/MockApps.java
index 7d233e2d9fc..ade32ffa286 100644
--- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/MockApps.java
+++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/MockApps.java
@@ -24,7 +24,8 @@ import java.util.List;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ApplicationReport;
-import org.apache.hadoop.yarn.api.records.ApplicationState;
+import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
+import org.apache.hadoop.yarn.api.records.YarnApplicationState;
import org.apache.hadoop.yarn.util.Records;
import com.google.common.collect.Iterators;
@@ -39,8 +40,8 @@ public class MockApps {
"I18nApp<☯>");
static final Iterator USERS = Iterators.cycle("dorothy", "tinman",
"scarecrow", "glinda", "nikko", "toto", "winkie", "zeke", "gulch");
- static final Iterator STATES = Iterators.cycle(
- ApplicationState.values());
+ static final Iterator STATES = Iterators.cycle(
+ YarnApplicationState.values());
static final Iterator QUEUES = Iterators.cycle("a.a1", "a.a2",
"b.b1", "b.b2", "b.b3", "c.c1.c11", "c.c1.c12", "c.c1.c13",
"c.c2", "c.c3", "c.c4");
@@ -74,46 +75,47 @@ public class MockApps {
public static ApplicationReport newApp(int i) {
final ApplicationId id = newAppID(i);
- final ApplicationState state = newAppState();
+ final YarnApplicationState state = newAppState();
final String user = newUserName();
final String name = newAppName();
final String queue = newQueue();
+ final FinalApplicationStatus finishState = FinalApplicationStatus.UNDEFINED;
return new ApplicationReport() {
@Override public ApplicationId getApplicationId() { return id; }
@Override public String getUser() { return user; }
@Override public String getName() { return name; }
- @Override public ApplicationState getState() { return state; }
+ @Override public YarnApplicationState getYarnApplicationState() { return state; }
@Override public String getQueue() { return queue; }
@Override public String getTrackingUrl() { return ""; }
- @Override
+ @Override public FinalApplicationStatus getFinalApplicationStatus() { return finishState; }
public void setApplicationId(ApplicationId applicationId) {
// TODO Auto-generated method stub
-
+
}
@Override
public void setTrackingUrl(String url) {
// TODO Auto-generated method stub
-
+
}
@Override
public void setName(String name) {
// TODO Auto-generated method stub
-
+
}
@Override
public void setQueue(String queue) {
// TODO Auto-generated method stub
-
+
}
@Override
- public void setState(ApplicationState state) {
+ public void setYarnApplicationState(YarnApplicationState state) {
// TODO Auto-generated method stub
-
+
}
@Override
public void setUser(String user) {
// TODO Auto-generated method stub
-
+
}
@Override
public String getDiagnostics() {
@@ -123,7 +125,7 @@ public class MockApps {
@Override
public void setDiagnostics(String diagnostics) {
// TODO Auto-generated method stub
-
+
}
@Override
public String getHost() {
@@ -133,7 +135,7 @@ public class MockApps {
@Override
public void setHost(String host) {
// TODO Auto-generated method stub
-
+
}
@Override
public int getRpcPort() {
@@ -143,7 +145,7 @@ public class MockApps {
@Override
public void setRpcPort(int rpcPort) {
// TODO Auto-generated method stub
-
+
}
@Override
public String getClientToken() {
@@ -153,9 +155,8 @@ public class MockApps {
@Override
public void setClientToken(String clientToken) {
// TODO Auto-generated method stub
-
+
}
-
@Override
public long getStartTime() {
// TODO Auto-generated method stub
@@ -175,7 +176,11 @@ public class MockApps {
@Override
public void setFinishTime(long finishTime) {
// TODO Auto-generated method stub
-
+
+ }
+ @Override
+ public void setFinalApplicationStatus(FinalApplicationStatus finishState) {
+ // TODO Auto-generated method stub
}
};
}
@@ -194,9 +199,10 @@ public class MockApps {
return id;
}
- public static ApplicationState newAppState() {
+ public static YarnApplicationState newAppState() {
synchronized(STATES) {
return STATES.next();
}
}
+
}
diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java
index 436ed23d6ea..8601545229e 100644
--- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java
+++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java
@@ -75,7 +75,7 @@ public class ApplicationMasterService extends AbstractService implements
new ConcurrentHashMap();
private final AMResponse reboot = recordFactory.newRecordInstance(AMResponse.class);
private final RMContext rmContext;
-
+
public ApplicationMasterService(RMContext rmContext,
ApplicationTokenSecretManager appTokenManager, YarnScheduler scheduler) {
super(ApplicationMasterService.class.getName());
@@ -105,12 +105,12 @@ public class ApplicationMasterService extends AbstractService implements
this.server =
rpc.getServer(AMRMProtocol.class, this, masterServiceAddress,
serverConf, this.appTokenManager,
- serverConf.getInt(YarnConfiguration.RM_SCHEDULER_CLIENT_THREAD_COUNT,
+ serverConf.getInt(YarnConfiguration.RM_SCHEDULER_CLIENT_THREAD_COUNT,
YarnConfiguration.DEFAULT_RM_SCHEDULER_CLIENT_THREAD_COUNT));
this.server.start();
super.start();
}
-
+
@Override
public RegisterApplicationMasterResponse registerApplicationMaster(
RegisterApplicationMasterRequest request) throws YarnRemoteException {
@@ -123,7 +123,7 @@ public class ApplicationMasterService extends AbstractService implements
String message = "Application doesn't exist in cache "
+ applicationAttemptId;
LOG.error(message);
- RMAuditLogger.logFailure(this.rmContext.getRMApps().get(appID).getUser(),
+ RMAuditLogger.logFailure(this.rmContext.getRMApps().get(appID).getUser(),
AuditConstants.REGISTER_AM, message, "ApplicationMasterService",
"Error in registering application master", appID,
applicationAttemptId);
@@ -141,7 +141,7 @@ public class ApplicationMasterService extends AbstractService implements
.getHost(), request.getRpcPort(), request.getTrackingUrl()));
RMAuditLogger.logSuccess(this.rmContext.getRMApps().get(appID).getUser(),
- AuditConstants.REGISTER_AM, "ApplicationMasterService", appID,
+ AuditConstants.REGISTER_AM, "ApplicationMasterService", appID,
applicationAttemptId);
// Pick up min/max resource from scheduler...
@@ -176,7 +176,7 @@ public class ApplicationMasterService extends AbstractService implements
rmContext.getDispatcher().getEventHandler().handle(
new RMAppAttemptUnregistrationEvent(applicationAttemptId, request
- .getTrackingUrl(), request.getFinalState(), request
+ .getTrackingUrl(), request.getFinalApplicationStatus(), request
.getDiagnostics()));
FinishApplicationMasterResponse response = recordFactory
@@ -225,7 +225,7 @@ public class ApplicationMasterService extends AbstractService implements
List release = request.getReleaseList();
// Send new requests to appAttempt.
- Allocation allocation =
+ Allocation allocation =
this.rScheduler.allocate(appAttemptId, ask, release);
RMApp app = this.rmContext.getRMApps().get(appAttemptId.getApplicationId());
diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMApp.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMApp.java
index 6e63e2248d6..0ea9202fcbf 100644
--- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMApp.java
+++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMApp.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.rmapp;
import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ApplicationReport;
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
@@ -28,8 +29,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.recovery.ApplicationsStore.
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
/**
- * The read interface to an Application in the ResourceManager. Take a
- * look at {@link RMAppImpl} for its implementation. This interface
+ * The read interface to an Application in the ResourceManager. Take a
+ * look at {@link RMAppImpl} for its implementation. This interface
* exposes methods to access various updates in application status/report.
*/
public interface RMApp extends EventHandler {
@@ -68,7 +69,7 @@ public interface RMApp extends EventHandler {
RMAppAttempt getRMAppAttempt(ApplicationAttemptId appAttemptId);
/**
- * Each Application is submitted to a queue decided by {@link
+ * Each Application is submitted to a queue decided by {@link
* ApplicationSubmissionContext#setQueue(String)}.
* This method returns the queue to which an application was submitted.
* @return the queue to which the application was submitted to.
@@ -76,7 +77,7 @@ public interface RMApp extends EventHandler {
String getQueue();
/**
- * The name of the application as set in {@link
+ * The name of the application as set in {@link
* ApplicationSubmissionContext#setApplicationName(String)}.
* @return the name of the application.
*/
@@ -85,7 +86,7 @@ public interface RMApp extends EventHandler {
/**
* {@link RMApp} can have multiple application attempts {@link RMAppAttempt}.
* This method returns the current {@link RMAppAttempt}.
- * @return the current {@link RMAppAttempt}
+ * @return the current {@link RMAppAttempt}
*/
RMAppAttempt getCurrentAppAttempt();
@@ -96,7 +97,7 @@ public interface RMApp extends EventHandler {
ApplicationReport createAndGetApplicationReport();
/**
- * Application level metadata is stored in {@link ApplicationStore} whicn
+ * Application level metadata is stored in {@link ApplicationStore} whicn
* can persist the information.
* @return the {@link ApplicationStore} for this {@link RMApp}.
*/
@@ -125,12 +126,12 @@ public interface RMApp extends EventHandler {
* @return the diagnostics information for the application master.
*/
StringBuilder getDiagnostics();
-
+
/**
- * The final state of the AM when unregistering as in
- * {@link FinishApplicationMasterRequest#setFinalState(String)}.
- * @return the final state of the AM as set in
- * {@link FinishApplicationMasterRequest#setFinalState(String)}.
+ * The final finish state of the AM when unregistering as in
+ * {@link FinishApplicationMasterRequest#setFinishApplicationStatus(FinalApplicationStatus)}.
+ * @return the final finish state of the AM as set in
+ * {@link FinishApplicationMasterRequest#setFinishApplicationStatus(FinalApplicationStatus)}.
*/
- String getAMFinalState();
+ FinalApplicationStatus getFinalApplicationStatus();
}
diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
index 94d04a8d127..8cdccfdddd4 100644
--- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
+++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
@@ -32,9 +32,10 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.yarn.YarnException;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ApplicationReport;
-import org.apache.hadoop.yarn.api.records.ApplicationState;
+import org.apache.hadoop.yarn.api.records.YarnApplicationState;
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
@@ -94,7 +95,7 @@ public class RMAppImpl implements RMApp {
private static final StateMachineFactory stateMachineFactory
+ RMAppEvent> stateMachineFactory
= new StateMachineFactory {
@@ -49,7 +50,7 @@ public interface RMAppAttempt extends EventHandler {
* @return the state {@link RMAppAttemptState} of this {@link RMAppAttempt}
*/
RMAppAttemptState getAppAttemptState();
-
+
/**
* The host on which the {@link RMAppAttempt} is running/ran on.
* @return the host on which the {@link RMAppAttempt} ran/is running on.
@@ -88,11 +89,12 @@ public interface RMAppAttempt extends EventHandler {
float getProgress();
/**
- * The final state set by the AM.
- * @return the final state that is set by the AM when unregistering itself.
+ * The final status set by the AM.
+ * @return the final status that is set by the AM when unregistering itself. Can return a null
+ * if the AM has not unregistered itself.
*/
- String getAMFinalState();
-
+ FinalApplicationStatus getFinalApplicationStatus();
+
/**
* Nodes on which the containers for this {@link RMAppAttempt} ran.
* @return the set of nodes that ran any containers from this {@link RMAppAttempt}
@@ -100,16 +102,16 @@ public interface RMAppAttempt extends EventHandler {
Set getRanNodes();
/**
- * Return a list of the last set of finished containers, resetting the
+ * Return a list of the last set of finished containers, resetting the
* finished containers to empty.
* @return the list of just finished containers, re setting the finished containers.
*/
List pullJustFinishedContainers();
/**
- * Return the list of last set of finished containers. This does not reset the
+ * Return the list of last set of finished containers. This does not reset the
* finished containers.
- * @return the list of just finished contianers, this does not reset the
+ * @return the list of just finished contianers, this does not reset the
* finished containers.
*/
List getJustFinishedContainers();
diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java
index 7f8ff82d6a1..81aae502015 100644
--- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java
+++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java
@@ -31,6 +31,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
import org.apache.hadoop.yarn.api.records.Container;
@@ -99,9 +100,9 @@ public class RMAppAttemptImpl implements RMAppAttempt {
private final ApplicationSubmissionContext submissionContext;
//nodes on while this attempt's containers ran
- private final Set ranNodes =
+ private final Set ranNodes =
new HashSet();
- private final List justFinishedContainers =
+ private final List justFinishedContainers =
new ArrayList();
private Container masterContainer;
@@ -109,7 +110,9 @@ public class RMAppAttemptImpl implements RMAppAttempt {
private String host = "N/A";
private int rpcPort;
private String trackingUrl = "N/A";
- private String finalState = "N/A";
+ // Set to null initially. Will eventually get set
+ // if an RMAppAttemptUnregistrationEvent occurs
+ private FinalApplicationStatus finalStatus = null;
private final StringBuilder diagnostics = new StringBuilder();
private static final StateMachineFactory EMPTY_CONTAINER_RELEASE_LIST =
+ private static final List EMPTY_CONTAINER_RELEASE_LIST =
new ArrayList();
- private static final List EMPTY_CONTAINER_REQUEST_LIST =
+ private static final List EMPTY_CONTAINER_REQUEST_LIST =
new ArrayList();
private static final class ScheduleTransition extends BaseTransition {
@@ -540,23 +543,23 @@ public class RMAppAttemptImpl implements RMAppAttempt {
switch (finalAttemptState) {
case FINISHED:
{
- appEvent =
+ appEvent =
new RMAppEvent(applicationId, RMAppEventType.ATTEMPT_FINISHED);
}
break;
case KILLED:
{
- appEvent =
- new RMAppFailedAttemptEvent(applicationId,
- RMAppEventType.ATTEMPT_KILLED,
+ appEvent =
+ new RMAppFailedAttemptEvent(applicationId,
+ RMAppEventType.ATTEMPT_KILLED,
"Application killed by user.");
}
break;
case FAILED:
{
- appEvent =
- new RMAppFailedAttemptEvent(applicationId,
- RMAppEventType.ATTEMPT_FAILED,
+ appEvent =
+ new RMAppFailedAttemptEvent(applicationId,
+ RMAppEventType.ATTEMPT_FAILED,
appAttempt.getDiagnostics());
}
break;
@@ -566,7 +569,7 @@ public class RMAppAttemptImpl implements RMAppAttempt {
}
break;
}
-
+
appAttempt.eventHandler.handle(appEvent);
appAttempt.eventHandler.handle(new AppRemovedSchedulerEvent(appAttempt
.getAppAttemptId(), finalAttemptState));
@@ -657,7 +660,7 @@ public class RMAppAttemptImpl implements RMAppAttempt {
RMAppAttemptContainerFinishedEvent finishEvent =
((RMAppAttemptContainerFinishedEvent)event);
-
+
// UnRegister from AMLivelinessMonitor
appAttempt.rmContext.getAMLivelinessMonitor().unregister(
appAttempt.getAppAttemptId());
@@ -666,7 +669,7 @@ public class RMAppAttemptImpl implements RMAppAttempt {
ContainerStatus status = finishEvent.getContainerStatus();
appAttempt.diagnostics.append("AM Container for " +
appAttempt.getAppAttemptId() + " exited with " +
- " exitCode: " + status.getExitStatus() +
+ " exitCode: " + status.getExitStatus() +
" due to: " + status.getDiagnostics() + "." +
"Failing this attempt.");
@@ -730,10 +733,9 @@ public class RMAppAttemptImpl implements RMAppAttempt {
RMAppAttemptUnregistrationEvent unregisterEvent
= (RMAppAttemptUnregistrationEvent) event;
- unregisterEvent.getFinalState();
appAttempt.diagnostics.append(unregisterEvent.getDiagnostics());
appAttempt.trackingUrl = unregisterEvent.getTrackingUrl();
- appAttempt.finalState = unregisterEvent.getFinalState();
+ appAttempt.finalStatus = unregisterEvent.getFinalApplicationStatus();
// Tell the app and the scheduler
super.transition(appAttempt, event);
@@ -761,7 +763,7 @@ public class RMAppAttemptImpl implements RMAppAttempt {
RMAppAttemptContainerFinishedEvent containerFinishedEvent
= (RMAppAttemptContainerFinishedEvent) event;
- ContainerStatus containerStatus =
+ ContainerStatus containerStatus =
containerFinishedEvent.getContainerStatus();
// Is this container the AmContainer? If the finished container is same as
@@ -771,7 +773,7 @@ public class RMAppAttemptImpl implements RMAppAttempt {
// Setup diagnostic message
appAttempt.diagnostics.append("AM Container for " +
appAttempt.getAppAttemptId() + " exited with " +
- " exitCode: " + containerStatus.getExitStatus() +
+ " exitCode: " + containerStatus.getExitStatus() +
" due to: " + containerStatus.getDiagnostics() + "." +
"Failing this attempt.");
diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/event/RMAppAttemptUnregistrationEvent.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/event/RMAppAttemptUnregistrationEvent.java
index 3511a03eedd..5ea461121ca 100644
--- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/event/RMAppAttemptUnregistrationEvent.java
+++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/event/RMAppAttemptUnregistrationEvent.java
@@ -19,20 +19,21 @@
package org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEventType;
public class RMAppAttemptUnregistrationEvent extends RMAppAttemptEvent {
private final String trackingUrl;
- private final String finalState;
+ private final FinalApplicationStatus finalStatus;
private final String diagnostics;
public RMAppAttemptUnregistrationEvent(ApplicationAttemptId appAttemptId,
- String trackingUrl, String finalState, String diagnostics) {
+ String trackingUrl, FinalApplicationStatus finalStatus, String diagnostics) {
super(appAttemptId, RMAppAttemptEventType.UNREGISTERED);
this.trackingUrl = trackingUrl;
- this.finalState = finalState;
+ this.finalStatus = finalStatus;
this.diagnostics = diagnostics;
}
@@ -40,12 +41,12 @@ public class RMAppAttemptUnregistrationEvent extends RMAppAttemptEvent {
return this.trackingUrl;
}
- public String getFinalState() {
- return this.finalState;
+ public FinalApplicationStatus getFinalApplicationStatus() {
+ return this.finalStatus;
}
public String getDiagnostics() {
return this.diagnostics;
}
-}
\ No newline at end of file
+}
diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppsBlock.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppsBlock.java
index d6415aeb811..41ed833a992 100644
--- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppsBlock.java
+++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppsBlock.java
@@ -50,6 +50,7 @@ class AppsBlock extends HtmlBlock {
th(".name", "Name").
th(".queue", "Queue").
th(".state", "State").
+ th(".finalstatus", "FinalStatus").
th(".progress", "Progress").
th(".ui", "Tracking UI").
th(".note", "Note")._()._().
@@ -70,8 +71,8 @@ class AppsBlock extends HtmlBlock {
td(app.getUser().toString()).
td(app.getName().toString()).
td(app.getQueue().toString()).
- td(app.getState() == RMAppState.FINISHED ? app.getAMFinalState() :
- app.getState().toString()).
+ td(app.getState().toString()).
+ td(app.getFinalApplicationStatus().toString()).
td().
br().$title(percent)._(). // for sorting
div(_PROGRESSBAR).
diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RmController.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RmController.java
index 698bc3c933a..2917dbe2525 100644
--- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RmController.java
+++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RmController.java
@@ -89,9 +89,8 @@ public class RmController extends Controller {
ResponseInfo info = info("Application Overview").
_("User:", app.getUser()).
_("Name:", app.getName()).
- _("State:", (app.getState() == RMAppState.FINISHED ?
- app.getAMFinalState() : app.getState().toString())
- ).
+ _("State:", app.getState().toString()).
+ _("FinalStatus:", app.getFinalApplicationStatus().toString()).
_("Started:", Times.format(app.getStartTime())).
_("Elapsed:", StringUtils.formatTime(
Times.elapsed(app.getStartTime(), app.getFinishTime()))).
diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockAM.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockAM.java
index 727cd1a2323..7abd7f88658 100644
--- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockAM.java
+++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockAM.java
@@ -30,7 +30,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest
import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
import org.apache.hadoop.yarn.api.records.AMResponse;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
-import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.Priority;
import org.apache.hadoop.yarn.api.records.Resource;
@@ -47,11 +47,11 @@ public class MockAM {
private final ApplicationAttemptId attemptId;
private final RMContext context;
private final AMRMProtocol amRMProtocol;
-
+
private final List requests = new ArrayList();
private final List releases = new ArrayList();
- MockAM(RMContext context, AMRMProtocol amRMProtocol,
+ MockAM(RMContext context, AMRMProtocol amRMProtocol,
ApplicationAttemptId attemptId) {
this.context = context;
this.amRMProtocol = amRMProtocol;
@@ -85,7 +85,7 @@ public class MockAM {
amRMProtocol.registerApplicationMaster(req);
}
- public void addRequests(String[] hosts, int memory, int priority,
+ public void addRequests(String[] hosts, int memory, int priority,
int containers) throws Exception {
requests.addAll(createReq(hosts, memory, priority, containers));
}
@@ -97,33 +97,33 @@ public class MockAM {
return response;
}
- public AMResponse allocate(
- String host, int memory, int numContainers,
+ public AMResponse allocate(
+ String host, int memory, int numContainers,
List releases) throws Exception {
- List reqs = createReq(new String[]{host}, memory, 1, numContainers);
+ List reqs = createReq(new String[]{host}, memory, 1, numContainers);
return allocate(reqs, releases);
}
- public List createReq(String[] hosts, int memory, int priority,
+ public List createReq(String[] hosts, int memory, int priority,
int containers) throws Exception {
List reqs = new ArrayList();
for (String host : hosts) {
- ResourceRequest hostReq = createResourceReq(host, memory, priority,
+ ResourceRequest hostReq = createResourceReq(host, memory, priority,
containers);
reqs.add(hostReq);
- ResourceRequest rackReq = createResourceReq("default-rack", memory,
+ ResourceRequest rackReq = createResourceReq("default-rack", memory,
priority, containers);
reqs.add(rackReq);
}
-
- ResourceRequest offRackReq = createResourceReq("*", memory, priority,
+
+ ResourceRequest offRackReq = createResourceReq("*", memory, priority,
containers);
reqs.add(offRackReq);
return reqs;
-
+
}
- public ResourceRequest createResourceReq(String resource, int memory, int priority,
+ public ResourceRequest createResourceReq(String resource, int memory, int priority,
int containers) throws Exception {
ResourceRequest req = Records.newRecord(ResourceRequest.class);
req.setHostName(resource);
@@ -138,7 +138,7 @@ public class MockAM {
}
public AMResponse allocate(
- List resourceRequest, List releases)
+ List resourceRequest, List releases)
throws Exception {
AllocateRequest req = BuilderUtils.newAllocateRequest(attemptId,
++responseId, 0F, resourceRequest, releases);
@@ -151,7 +151,7 @@ public class MockAM {
FinishApplicationMasterRequest req = Records.newRecord(FinishApplicationMasterRequest.class);
req.setAppAttemptId(attemptId);
req.setDiagnostics("");
- req.setFinalState("");
+ req.setFinishApplicationStatus(FinalApplicationStatus.SUCCEEDED);
req.setTrackingUrl("");
amRMProtocol.finishApplicationMaster(req);
}
diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/MockAsm.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/MockAsm.java
index 01dd078b772..bda4d46e4e9 100644
--- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/MockAsm.java
+++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/MockAsm.java
@@ -22,10 +22,11 @@ import java.util.List;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.yarn.MockApps;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ApplicationMaster;
import org.apache.hadoop.yarn.api.records.ApplicationReport;
-import org.apache.hadoop.yarn.api.records.ApplicationState;
+import org.apache.hadoop.yarn.api.records.YarnApplicationState;
import org.apache.hadoop.yarn.api.records.ApplicationStatus;
import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerId;
@@ -69,7 +70,7 @@ public abstract class MockAsm extends MockApps {
}
@Override
- public ApplicationState getState() {
+ public YarnApplicationState getState() {
throw new UnsupportedOperationException("Not supported yet.");
}
@@ -119,7 +120,7 @@ public abstract class MockAsm extends MockApps {
}
@Override
- public void setState(ApplicationState state) {
+ public void setState(YarnApplicationState state) {
throw new UnsupportedOperationException("Not supported yet.");
}
@@ -207,11 +208,11 @@ public abstract class MockAsm extends MockApps {
}
@Override
public void handle(RMAppEvent event) {
- throw new UnsupportedOperationException("Not supported yet.");
+ throw new UnsupportedOperationException("Not supported yet.");
}
@Override
- public String getAMFinalState() {
+ public FinalApplicationStatus getFinalApplicationStatus() {
throw new UnsupportedOperationException("Not supported yet.");
}
}
@@ -274,9 +275,14 @@ public abstract class MockAsm extends MockApps {
public float getProgress() {
return (float)Math.random();
}
+ @Override
+ public FinalApplicationStatus getFinalApplicationStatus() {
+ return FinalApplicationStatus.UNDEFINED;
+ }
+
};
}
-
+
public static List newApplications(int n) {
List list = Lists.newArrayList();
for (int i = 0; i < n; ++i) {
diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMLaunchFailure.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMLaunchFailure.java
index 85d84325358..2ee54311f9e 100644
--- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMLaunchFailure.java
+++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMLaunchFailure.java
@@ -32,7 +32,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ApplicationMaster;
import org.apache.hadoop.yarn.api.records.ApplicationReport;
-import org.apache.hadoop.yarn.api.records.ApplicationState;
+import org.apache.hadoop.yarn.api.records.YarnApplicationState;
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.Priority;
@@ -65,7 +65,7 @@ public class TestAMLaunchFailure {
// private static final RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
// ApplicationsManagerImpl asmImpl;
// YarnScheduler scheduler = new DummyYarnScheduler();
-// ApplicationTokenSecretManager applicationTokenSecretManager =
+// ApplicationTokenSecretManager applicationTokenSecretManager =
// new ApplicationTokenSecretManager();
// private ClientRMService clientService;
//
@@ -98,7 +98,7 @@ public class TestAMLaunchFailure {
// , ApplicationStore appStore)
// throws IOException {
// // TODO Auto-generated method stub
-//
+//
// }
//
// @Override
@@ -199,7 +199,7 @@ public class TestAMLaunchFailure {
// conf.setLong(YarnConfiguration.AM_EXPIRY_INTERVAL, 3000L);
// conf.setInt(RMConfig.AM_MAX_RETRIES, 1);
// asmImpl.init(conf);
-// asmImpl.start();
+// asmImpl.start();
// }
//
// @After
@@ -221,7 +221,7 @@ public class TestAMLaunchFailure {
// .newRecordInstance(SubmitApplicationRequest.class);
// request.setApplicationSubmissionContext(submissionContext);
// clientService.submitApplication(request);
-// AppAttempt application = context.getApplications().get(appID);
+// AppAttempt application = context.getApplications().get(appID);
//
// while (application.getState() != ApplicationState.FAILED) {
// LOG.info("Waiting for application to go to FAILED state."
diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java
index 61e38218570..e9062bc3dd8 100644
--- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java
+++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java
@@ -33,7 +33,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ApplicationMaster;
-import org.apache.hadoop.yarn.api.records.ApplicationState;
+import org.apache.hadoop.yarn.api.records.YarnApplicationState;
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerId;
@@ -75,7 +75,7 @@ public class TestAMRestart {
// private static final Log LOG = LogFactory.getLog(TestAMRestart.class);
// ApplicationsManagerImpl appImpl;
// RMContext asmContext = new RMContextImpl(new MemStore());
-// ApplicationTokenSecretManager appTokenSecretManager =
+// ApplicationTokenSecretManager appTokenSecretManager =
// new ApplicationTokenSecretManager();
// DummyResourceScheduler scheduler;
// private ClientRMService clientRMService;
@@ -90,7 +90,7 @@ public class TestAMRestart {
// int launcherLaunchCalled = 0;
// int launcherCleanupCalled = 0;
// private final static RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
-//
+//
// private class ExtApplicationsManagerImpl extends ApplicationsManagerImpl {
// public ExtApplicationsManagerImpl(
// ApplicationTokenSecretManager applicationTokenSecretManager,
@@ -115,7 +115,7 @@ public class TestAMRestart {
// LOG.info("DEBUG -- waiting for launch");
// synchronized(launchNotify) {
// while (launchNotify.get() == 0) {
-// try {
+// try {
// launchNotify.wait();
// } catch (InterruptedException e) {
// }
@@ -151,11 +151,11 @@ public class TestAMRestart {
// }
//
// private class DummyResourceScheduler implements ResourceScheduler {
-//
+//
// @Override
// public void removeNode(RMNode node) {
// }
-//
+//
// @Override
// public Allocation allocate(ApplicationId applicationId,
// List ask, List release) throws IOException {
@@ -222,7 +222,7 @@ public class TestAMRestart {
//
// @Override
// public void nodeUpdate(RMNode nodeInfo,
-// Map> containers) {
+// Map> containers) {
// }
//
// @Override
@@ -253,7 +253,7 @@ public class TestAMRestart {
// asmContext.getDispatcher().start();
// asmContext.getDispatcher().register(ApplicationTrackerEventType.class, scheduler);
// appImpl = new ExtApplicationsManagerImpl(appTokenSecretManager, scheduler, asmContext);
-//
+//
// conf.setLong(YarnConfiguration.AM_EXPIRY_INTERVAL, 1000L);
// conf.setInt(RMConfig.AM_MAX_RETRIES, maxFailures);
// appImpl.init(conf);
@@ -261,7 +261,7 @@ public class TestAMRestart {
//
// this.clientRMService = new ClientRMService(asmContext, appImpl
// .getAmLivelinessMonitor(), appImpl.getClientToAMSecretManager(),
-// scheduler);
+// scheduler);
// this.clientRMService.init(conf);
// }
//
@@ -269,7 +269,7 @@ public class TestAMRestart {
// public void tearDown() {
// }
//
-// private void waitForFailed(AppAttempt application, ApplicationState
+// private void waitForFailed(AppAttempt application, ApplicationState
// finalState) throws Exception {
// int count = 0;
// while(application.getState() != finalState && count < 10) {
@@ -292,7 +292,7 @@ public class TestAMRestart {
// .newRecordInstance(SubmitApplicationRequest.class);
// request.setApplicationSubmissionContext(subContext);
// clientRMService.submitApplication(request);
-// AppAttempt application = asmContext.getApplications().get(appID);
+// AppAttempt application = asmContext.getApplications().get(appID);
// synchronized (schedulerNotify) {
// while(schedulerNotify.get() == 0) {
// schedulerNotify.wait();
@@ -306,4 +306,4 @@ public class TestAMRestart {
// waitForFailed(application, ApplicationState.FAILED);
// stop = true;
// }
-}
\ No newline at end of file
+}
diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestASMStateMachine.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestASMStateMachine.java
index 18975084a2b..219a28fe193 100644
--- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestASMStateMachine.java
+++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestASMStateMachine.java
@@ -26,7 +26,7 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.ApplicationState;
+import org.apache.hadoop.yarn.api.records.YarnApplicationState;
import org.apache.hadoop.yarn.api.records.ApplicationStatus;
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
import org.apache.hadoop.yarn.event.EventHandler;
@@ -152,7 +152,7 @@ public class TestASMStateMachine {
// }
// }
//
-// private void waitForState( ApplicationState
+// private void waitForState( ApplicationState
// finalState, AppAttemptImpl masterInfo) throws Exception {
// int count = 0;
// while(masterInfo.getState() != finalState && count < 10) {
@@ -160,10 +160,10 @@ public class TestASMStateMachine {
// count++;
// }
// Assert.assertEquals(finalState, masterInfo.getState());
-// }
-//
-// /* Test the state machine.
-// *
+// }
+//
+// /* Test the state machine.
+// *
// */
// @Test
// public void testStateMachine() throws Exception {
@@ -211,6 +211,6 @@ public class TestASMStateMachine {
// /* check if expiry doesnt make it failed */
// handler.handle(new ApplicationEvent(ApplicationEventType.EXPIRE,
// masterInfo.getApplicationID()));
-// Assert.assertEquals(ApplicationState.COMPLETED, masterInfo.getState());
+// Assert.assertEquals(ApplicationState.COMPLETED, masterInfo.getState());
// }
}
diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestApplicationMasterExpiry.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestApplicationMasterExpiry.java
index ff4e798819e..00ab9545a02 100644
--- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestApplicationMasterExpiry.java
+++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestApplicationMasterExpiry.java
@@ -26,7 +26,7 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.ApplicationState;
+import org.apache.hadoop.yarn.api.records.YarnApplicationState;
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.event.EventHandler;
@@ -44,21 +44,21 @@ import org.junit.Test;
/**
* A test case that tests the expiry of the application master.
- * More tests can be added to this.
+ * More tests can be added to this.
*/
public class TestApplicationMasterExpiry {
// private static final Log LOG = LogFactory.getLog(TestApplicationMasterExpiry.class);
// private static RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
-//
+//
// private final RMContext context = new RMContextImpl(new MemStore());
// private AMLivelinessMonitor amLivelinessMonitor;
-//
+//
// @Before
// public void setUp() {
// new DummyApplicationTracker();
// new DummySN();
// new DummyLauncher();
-// new ApplicationEventTypeListener();
+// new ApplicationEventTypeListener();
// Configuration conf = new Configuration();
// context.getDispatcher().register(ApplicationEventType.class,
// new ResourceManager.ApplicationEventDispatcher(context));
@@ -70,7 +70,7 @@ public class TestApplicationMasterExpiry {
// amLivelinessMonitor.init(conf);
// amLivelinessMonitor.start();
// }
-//
+//
// private class DummyApplicationTracker implements EventHandler> {
// DummyApplicationTracker() {
// context.getDispatcher().register(ApplicationTrackerEventType.class, this);
@@ -79,10 +79,10 @@ public class TestApplicationMasterExpiry {
// public void handle(ASMEvent event) {
// }
// }
-//
+//
// private AtomicInteger expiry = new AtomicInteger();
// private boolean expired = false;
-//
+//
// private class ApplicationEventTypeListener implements
// EventHandler {
// ApplicationEventTypeListener() {
@@ -100,7 +100,7 @@ public class TestApplicationMasterExpiry {
// }
// }
// }
-//
+//
// private class DummySN implements EventHandler> {
// DummySN() {
// context.getDispatcher().register(SNEventType.class, this);
@@ -109,7 +109,7 @@ public class TestApplicationMasterExpiry {
// public void handle(ASMEvent event) {
// }
// }
-//
+//
// private class DummyLauncher implements EventHandler> {
// DummyLauncher() {
// context.getDispatcher().register(AMLauncherEventType.class, this);
@@ -118,8 +118,8 @@ public class TestApplicationMasterExpiry {
// public void handle(ASMEvent event) {
// }
// }
-//
-// private void waitForState(AppAttempt application, ApplicationState
+//
+// private void waitForState(AppAttempt application, ApplicationState
// finalState) throws Exception {
// int count = 0;
// while(application.getState() != finalState && count < 10) {
diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestSchedulerNegotiator.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestSchedulerNegotiator.java
index 459db70fbdb..3780617eda1 100644
--- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestSchedulerNegotiator.java
+++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestSchedulerNegotiator.java
@@ -28,7 +28,7 @@ import junit.framework.Assert;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ApplicationMaster;
-import org.apache.hadoop.yarn.api.records.ApplicationState;
+import org.apache.hadoop.yarn.api.records.YarnApplicationState;
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerId;
@@ -62,7 +62,7 @@ public class TestSchedulerNegotiator {
// private SchedulerNegotiator schedulerNegotiator;
// private DummyScheduler scheduler;
// private final int testNum = 99999;
-//
+//
// private final RMContext context = new RMContextImpl(new MemStore());
// AppAttemptImpl masterInfo;
// private EventHandler handler;
@@ -79,13 +79,13 @@ public class TestSchedulerNegotiator {
// containers.add(container);
// return new Allocation(containers, Resources.none());
// }
-//
-//
+//
+//
// @Override
// public void nodeUpdate(RMNode nodeInfo,
// Map> containers) {
// }
-//
+//
// @Override
// public void removeNode(RMNode node) {
// }
@@ -142,7 +142,7 @@ public class TestSchedulerNegotiator {
// return null;
// }
// }
-//
+//
// @Before
// public void setUp() {
// scheduler = new DummyScheduler();
@@ -153,12 +153,12 @@ public class TestSchedulerNegotiator {
// context.getDispatcher().init(conf);
// context.getDispatcher().start();
// }
-//
+//
// @After
// public void tearDown() {
// schedulerNegotiator.stop();
// }
-//
+//
// public void waitForState(ApplicationState state, AppAttemptImpl info) {
// int count = 0;
// while (info.getState() != state && count < 100) {
@@ -184,7 +184,7 @@ public class TestSchedulerNegotiator {
// submissionContext.setApplicationId(recordFactory.newRecordInstance(ApplicationId.class));
// submissionContext.getApplicationId().setClusterTimestamp(System.currentTimeMillis());
// submissionContext.getApplicationId().setId(1);
-//
+//
// masterInfo = new AppAttemptImpl(this.context, this.conf, "dummy",
// submissionContext, "dummyClientToken", StoreFactory
// .createVoidAppStore(), new AMLivelinessMonitor(context
diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/MockRMApp.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/MockRMApp.java
index 03229c34b48..81e10092c87 100644
--- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/MockRMApp.java
+++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/MockRMApp.java
@@ -19,6 +19,7 @@
package org.apache.hadoop.yarn.server.resourcemanager.rmapp;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ApplicationReport;
import org.apache.hadoop.yarn.MockApps;
@@ -167,8 +168,8 @@ public class MockRMApp implements RMApp {
}
@Override
- public String getAMFinalState() {
- return "UNKNOWN";
+ public FinalApplicationStatus getFinalApplicationStatus() {
+ return FinalApplicationStatus.UNDEFINED;
};
}
diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java
index 24408821e2d..a46673f36de 100644
--- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java
+++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.yarn.MockApps;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
+import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.event.AsyncDispatcher;
import org.apache.hadoop.yarn.event.EventHandler;
@@ -192,10 +193,15 @@ public class TestRMAppTransitions {
}
private static void assertAppState(RMAppState state, RMApp application) {
- Assert.assertEquals("application state should have been" + state,
+ Assert.assertEquals("application state should have been " + state,
state, application.getState());
}
+ private static void assertFinalAppStatus(FinalApplicationStatus status, RMApp application) {
+ Assert.assertEquals("Final application status should have been " + status,
+ status, application.getFinalApplicationStatus());
+ }
+
// test to make sure times are set when app finishes
private static void assertTimesAtFinish(RMApp application) {
assertStartTimeSet(application);
@@ -208,6 +214,7 @@ public class TestRMAppTransitions {
private static void assertKilled(RMApp application) {
assertTimesAtFinish(application);
assertAppState(RMAppState.KILLED, application);
+ assertFinalAppStatus(FinalApplicationStatus.KILLED, application);
StringBuilder diag = application.getDiagnostics();
Assert.assertEquals("application diagnostics is not correct",
"Application killed by user.", diag.toString());
@@ -224,6 +231,7 @@ public class TestRMAppTransitions {
private static void assertFailed(RMApp application, String regex) {
assertTimesAtFinish(application);
assertAppState(RMAppState.FAILED, application);
+ assertFinalAppStatus(FinalApplicationStatus.FAILED, application);
StringBuilder diag = application.getDiagnostics();
Assert.assertTrue("application diagnostics is not correct",
diag.toString().matches(regex));
@@ -261,6 +269,7 @@ public class TestRMAppTransitions {
application.handle(event);
assertStartTimeSet(application);
assertAppState(RMAppState.RUNNING, application);
+ assertFinalAppStatus(FinalApplicationStatus.UNDEFINED, application);
return application;
}
@@ -273,6 +282,8 @@ public class TestRMAppTransitions {
application.handle(event);
assertAppState(RMAppState.FINISHED, application);
assertTimesAtFinish(application);
+ // finished without a proper unregister implies failed
+ assertFinalAppStatus(FinalApplicationStatus.FAILED, application);
return application;
}
diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptTransitions.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptTransitions.java
index 03a4ba07441..7083197ad11 100644
--- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptTransitions.java
+++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptTransitions.java
@@ -34,6 +34,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
+import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.event.AsyncDispatcher;
import org.apache.hadoop.yarn.event.EventHandler;
@@ -52,7 +53,9 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppFailedAttemptEve
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppRejectedEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptContainerAllocatedEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptLaunchFailedEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptRegistrationEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptRejectedEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptUnregistrationEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.ContainerAllocationExpirer;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
@@ -199,6 +202,7 @@ public class TestRMAppAttemptTransitions {
assertNull(applicationAttempt.getMasterContainer());
assertEquals(0.0, (double)applicationAttempt.getProgress(), 0.0001);
assertEquals(0, applicationAttempt.getRanNodes().size());
+ assertNull(applicationAttempt.getFinalApplicationStatus());
}
/**
@@ -212,6 +216,7 @@ public class TestRMAppAttemptTransitions {
assertNull(applicationAttempt.getMasterContainer());
assertEquals(0.0, (double)applicationAttempt.getProgress(), 0.0001);
assertEquals(0, applicationAttempt.getRanNodes().size());
+ assertNull(applicationAttempt.getFinalApplicationStatus());
// Check events
verify(masterService).
@@ -230,6 +235,7 @@ public class TestRMAppAttemptTransitions {
assertNull(applicationAttempt.getMasterContainer());
assertEquals(0.0, (double)applicationAttempt.getProgress(), 0.0001);
assertEquals(0, applicationAttempt.getRanNodes().size());
+ assertNull(applicationAttempt.getFinalApplicationStatus());
// Check events
verify(application).handle(any(RMAppRejectedEvent.class));
@@ -247,6 +253,7 @@ public class TestRMAppAttemptTransitions {
assertEquals(amContainer, applicationAttempt.getMasterContainer());
assertEquals(0.0, (double)applicationAttempt.getProgress(), 0.0001);
assertEquals(0, applicationAttempt.getRanNodes().size());
+ assertNull(applicationAttempt.getFinalApplicationStatus());
}
/**
@@ -259,6 +266,7 @@ public class TestRMAppAttemptTransitions {
assertNull(applicationAttempt.getMasterContainer());
assertEquals(0.0, (double)applicationAttempt.getProgress(), 0.0001);
assertEquals(0, applicationAttempt.getRanNodes().size());
+ assertNull(applicationAttempt.getFinalApplicationStatus());
// Check events
verify(application).handle(any(RMAppEvent.class));
@@ -299,6 +307,49 @@ public class TestRMAppAttemptTransitions {
verify(application, times(2)).handle(any(RMAppFailedAttemptEvent.class));
}
+ /**
+ * {@link RMAppAttemptState#LAUNCH}
+ */
+ private void testAppAttemptLaunchedState(Container container) {
+ assertEquals(RMAppAttemptState.LAUNCHED,
+ applicationAttempt.getAppAttemptState());
+ assertEquals(container, applicationAttempt.getMasterContainer());
+
+ // TODO - need to add more checks relevant to this state
+ }
+
+ /**
+ * {@link RMAppAttemptState#RUNNING}
+ */
+ private void testAppAttemptRunningState(Container container,
+ String host, int rpcPort, String trackingUrl) {
+ assertEquals(RMAppAttemptState.RUNNING,
+ applicationAttempt.getAppAttemptState());
+ assertEquals(container, applicationAttempt.getMasterContainer());
+ assertEquals(host, applicationAttempt.getHost());
+ assertEquals(rpcPort, applicationAttempt.getRpcPort());
+ assertEquals(trackingUrl, applicationAttempt.getTrackingUrl());
+
+ // TODO - need to add more checks relevant to this state
+ }
+
+ /**
+ * {@link RMAppAttemptState#FINISHED}
+ */
+ private void testAppAttemptFinishedState(Container container,
+ FinalApplicationStatus finalStatus,
+ String trackingUrl,
+ String diagnostics) {
+ assertEquals(RMAppAttemptState.FINISHED,
+ applicationAttempt.getAppAttemptState());
+ assertEquals(diagnostics, applicationAttempt.getDiagnostics());
+ assertEquals(trackingUrl, applicationAttempt.getTrackingUrl());
+ assertEquals(0,applicationAttempt.getJustFinishedContainers().size());
+ assertEquals(container, applicationAttempt.getMasterContainer());
+ assertEquals(finalStatus, applicationAttempt.getFinalApplicationStatus());
+ }
+
+
private void submitApplicationAttempt() {
ApplicationAttemptId appAttemptId = applicationAttempt.getAppAttemptId();
applicationAttempt.handle(
@@ -339,6 +390,27 @@ public class TestRMAppAttemptTransitions {
return container;
}
+
+ private void launchApplicationAttempt(Container container) {
+ applicationAttempt.handle(
+ new RMAppAttemptEvent(applicationAttempt.getAppAttemptId(),
+ RMAppAttemptEventType.LAUNCHED));
+
+ testAppAttemptLaunchedState(container);
+ }
+
+ private void runApplicationAttempt(Container container,
+ String host,
+ int rpcPort,
+ String trackingUrl) {
+ applicationAttempt.handle(
+ new RMAppAttemptRegistrationEvent(
+ applicationAttempt.getAppAttemptId(),
+ host, rpcPort, trackingUrl));
+
+ testAppAttemptRunningState(container, host, rpcPort, trackingUrl);
+ }
+
@Test
public void testNewToKilled() {
@@ -400,4 +472,37 @@ public class TestRMAppAttemptTransitions {
testAppAttemptFailedState(amContainer, diagnostics);
}
+ @Test
+ public void testUnregisterToKilledFinish() {
+ Container amContainer = allocateApplicationAttempt();
+ launchApplicationAttempt(amContainer);
+ runApplicationAttempt(amContainer, "host", 9999, "oldtrackingurl");
+ String trackingUrl = "newtrackingurl";
+ String diagnostics = "Killed by user";
+ FinalApplicationStatus finalStatus = FinalApplicationStatus.KILLED;
+ applicationAttempt.handle(
+ new RMAppAttemptUnregistrationEvent(
+ applicationAttempt.getAppAttemptId(),
+ trackingUrl, finalStatus, diagnostics));
+ testAppAttemptFinishedState(amContainer, finalStatus,
+ trackingUrl, diagnostics);
+ }
+
+
+ @Test
+ public void testUnregisterToSuccessfulFinish() {
+ Container amContainer = allocateApplicationAttempt();
+ launchApplicationAttempt(amContainer);
+ runApplicationAttempt(amContainer, "host", 9999, "oldtrackingurl");
+ String trackingUrl = "mytrackingurl";
+ String diagnostics = "Successful";
+ FinalApplicationStatus finalStatus = FinalApplicationStatus.SUCCEEDED;
+ applicationAttempt.handle(
+ new RMAppAttemptUnregistrationEvent(
+ applicationAttempt.getAppAttemptId(),
+ trackingUrl, finalStatus, diagnostics));
+ testAppAttemptFinishedState(amContainer, finalStatus,
+ trackingUrl, diagnostics);
+ }
+
}
diff --git a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestQueueMetrics.java b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestQueueMetrics.java
index 0254c195dfa..fc326e9b74c 100644
--- a/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestQueueMetrics.java
+++ b/hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestQueueMetrics.java
@@ -25,7 +25,7 @@ import org.apache.hadoop.metrics2.MetricsRecordBuilder;
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
import static org.apache.hadoop.test.MetricsAsserts.*;
import static org.apache.hadoop.test.MockitoMaker.*;
-import org.apache.hadoop.yarn.api.records.ApplicationState;
+import org.apache.hadoop.yarn.api.records.YarnApplicationState;
import org.apache.hadoop.yarn.server.resourcemanager.resource.Resource;
import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;