MAPREDUCE-3098. Fixed RM and MR AM to report YarnApplicationState and application's FinalStatus separately. Contributed by Hitesh Shah.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1177633 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Vinod Kumar Vavilapalli 2011-09-30 12:46:32 +00:00
parent 964d4a1666
commit 063e33a862
45 changed files with 864 additions and 580 deletions

View File

@ -324,6 +324,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

View File

@ -46,6 +46,7 @@
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 @@ protected float getApplicationProgress() {
protected void register() {
//Register
String host =
String host =
clientService.getBindAddress().getAddress().getHostAddress();
try {
RegisterApplicationMasterRequest request =
@ -155,7 +156,7 @@ protected void register() {
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 @@ protected void register() {
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 @@ protected void unregister() {
protected Resource getMinContainerCapability() {
return minContainerCapability;
}
protected Resource getMaxContainerCapability() {
return maxContainerCapability;
}

View File

@ -45,7 +45,7 @@
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 @@
public class TypeConverter {
private static RecordFactory recordFactory;
static {
recordFactory = RecordFactoryProvider.getRecordFactory(null);
}
@ -75,7 +75,7 @@ public static org.apache.hadoop.mapreduce.JobID fromYarn(ApplicationId appID) {
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 static TaskAttemptState toYarn(org.apache.hadoop.mapred.TaskStatus.State
}
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 static TaskCompletionEvent[] fromYarn(
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 static TaskAttemptId toYarn(
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 static JobStatus fromYarn(JobReport jobreport, String trackingUrl) {
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 static org.apache.hadoop.mapred.TIPStatus fromYarn(
}
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 static TaskReport fromYarn(org.apache.hadoop.mapreduce.v2.api.records.Tas
} 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<org.apache.hadoop.mapreduce.TaskAttemptID> runningAtts
List<org.apache.hadoop.mapreduce.TaskAttemptID> runningAtts
= new ArrayList<org.apache.hadoop.mapreduce.TaskAttemptID>();
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 @@ diagnostics, fromYarn(report.getTaskState()), report.getStartTime(), report.getF
}
return rep;
}
public static List<TaskReport> fromYarn(
List<org.apache.hadoop.mapreduce.v2.api.records.TaskReport> taskReports) {
List<TaskReport> reports = new ArrayList<TaskReport>();
@ -377,14 +377,14 @@ public static List<TaskReport> fromYarn(
}
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 static JobStatus.State fromYarn(ApplicationState state) {
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 static JobStatus fromYarn(ApplicationReport application,
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 static JobStatus[] fromYarnApps(List<ApplicationReport> applications,
List<JobStatus> jobStatuses = new ArrayList<JobStatus>();
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 static JobStatus[] fromYarnApps(List<ApplicationReport> applications,
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<org.apache.hadoop.yarn.api.records.QueueInfo> queues,
Configuration conf) {
@ -467,9 +467,9 @@ public static QueueAclsInfo[] fromYarnQueueUserAclsInfo(
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);
}

View File

@ -21,7 +21,7 @@
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 void testFromYarnApplicationReport() {
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";

View File

@ -61,7 +61,7 @@
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 ClientServiceDelegate(Configuration conf, ResourceMgrDelegate rm,
// 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<String, NotRunningJob> map = notRunningJobs.get(state);
@ -111,8 +111,8 @@ private NotRunningJob getNotRunningJob(ApplicationReport applicationReport,
map = new HashMap<String, NotRunningJob>();
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 @@ private MRClientProtocol getProxy() throws YarnRemoteException {
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 @@ private MRClientProtocol getProxy() throws YarnRemoteException {
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 @@ private MRClientProtocol getProxy() throws YarnRemoteException {
//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 @@ private MRClientProtocol getProxy() throws YarnRemoteException {
}
/** 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 MRClientProtocol run() {
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 MRClientProtocol run() {
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 org.apache.hadoop.mapreduce.Counters getJobCounters(JobID arg0) throws IO
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 TaskCompletionEvent[] getTaskCompletionEvents(JobID arg0, int arg1, int a
request.setJobId(jobID);
request.setFromEventId(arg1);
request.setMaxEvents(arg2);
List<org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptCompletionEvent> list =
List<org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptCompletionEvent> list =
((GetTaskAttemptCompletionEventsResponse) invoke(
"getTaskAttemptCompletionEvents", GetTaskAttemptCompletionEventsRequest.class, request)).
getCompletionEventList();
@ -332,12 +333,12 @@ public String[] getTaskDiagnostics(org.apache.hadoop.mapreduce.TaskAttemptID arg
}
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 JobStatus getJobStatus(JobID oldJobID) throws YarnRemoteException {
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<org.apache.hadoop.mapreduce.v2.api.records.TaskReport> taskReports =
((GetTaskReportsResponse) invoke("getTaskReports", GetTaskReportsRequest.class,
List<org.apache.hadoop.mapreduce.v2.api.records.TaskReport> 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 boolean killTask(TaskAttemptID taskAttemptID, boolean fail)
}
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 boolean killJob(JobID oldJobID)
return true;
}
}

View File

@ -22,8 +22,6 @@
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.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 @@ private ApplicationReport getUnknownApplicationReport() {
@Override
public FailTaskAttemptResponse failTaskAttempt(
FailTaskAttemptRequest request) throws YarnRemoteException {
FailTaskAttemptResponse resp =
FailTaskAttemptResponse resp =
recordFactory.newRecordInstance(FailTaskAttemptResponse.class);
return resp;
}
@ -104,7 +98,7 @@ public FailTaskAttemptResponse failTaskAttempt(
@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<String, CounterGroup>());
@ -115,7 +109,7 @@ public GetCountersResponse getCounters(GetCountersRequest request)
@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 GetJobReportResponse getJobReport(GetJobReportRequest request)
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 GetJobReportResponse getJobReport(GetJobReportRequest request)
public GetTaskAttemptCompletionEventsResponse getTaskAttemptCompletionEvents(
GetTaskAttemptCompletionEventsRequest request)
throws YarnRemoteException {
GetTaskAttemptCompletionEventsResponse resp =
GetTaskAttemptCompletionEventsResponse resp =
recordFactory.newRecordInstance(GetTaskAttemptCompletionEventsResponse.class);
resp.addAllCompletionEvents(new ArrayList<TaskAttemptCompletionEvent>());
return resp;
@ -161,7 +155,7 @@ public GetTaskAttemptReportResponse getTaskAttemptReport(
@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 GetTaskReportResponse getTaskReport(GetTaskReportRequest request)
@Override
public GetTaskReportsResponse getTaskReports(GetTaskReportsRequest request)
throws YarnRemoteException {
GetTaskReportsResponse resp =
GetTaskReportsResponse resp =
recordFactory.newRecordInstance(GetTaskReportsResponse.class);
resp.addAllTaskReports(new ArrayList<TaskReport>());
return resp;
@ -185,7 +179,7 @@ public GetTaskReportsResponse getTaskReports(GetTaskReportsRequest request)
@Override
public KillJobResponse killJob(KillJobRequest request)
throws YarnRemoteException {
KillJobResponse resp =
KillJobResponse resp =
recordFactory.newRecordInstance(KillJobResponse.class);
return resp;
}
@ -193,7 +187,7 @@ public KillJobResponse killJob(KillJobRequest request)
@Override
public KillTaskResponse killTask(KillTaskRequest request)
throws YarnRemoteException {
KillTaskResponse resp =
KillTaskResponse resp =
recordFactory.newRecordInstance(KillTaskResponse.class);
return resp;
}
@ -201,9 +195,9 @@ public KillTaskResponse killTask(KillTaskRequest request)
@Override
public KillTaskAttemptResponse killTaskAttempt(
KillTaskAttemptRequest request) throws YarnRemoteException {
KillTaskAttemptResponse resp =
KillTaskAttemptResponse resp =
recordFactory.newRecordInstance(KillTaskAttemptResponse.class);
return resp;
}
}

View File

@ -62,7 +62,6 @@
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.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 YARNRunner(Configuration conf) {
}
/**
* 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 YARNRunner(Configuration conf) {
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 YARNRunner(Configuration conf, ResourceMgrDelegate resMgrDelegate,
throw new RuntimeException("Error in instantiating YarnClient", ufe);
}
}
@Override
public void cancelDelegationToken(Token<DelegationTokenIdentifier> arg0)
throws IOException, InterruptedException {
@ -152,7 +152,7 @@ public TaskTrackerInfo[] getBlacklistedTrackers() throws IOException,
@Override
public ClusterMetrics getClusterMetrics() throws IOException,
InterruptedException {
InterruptedException {
return resMgrDelegate.getClusterMetrics();
}
@ -209,13 +209,13 @@ public String getStagingAreaDir() throws IOException, InterruptedException {
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 JobStatus submitJob(JobID jobId, String jobSubmitDir, Credentials ts)
}
// 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 ApplicationSubmissionContext createApplicationSubmissionContext(
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 ApplicationSubmissionContext createApplicationSubmissionContext(
// Setup LocalResources
Map<String, LocalResource> localResources =
new HashMap<String, LocalResource>();
Path jobConfPath = new Path(jobSubmitDir, MRJobConfig.JOB_CONF_FILE);
URL yarnUrlForJobSubmitDir = ConverterUtils
.getYarnUrlFromPath(defaultFileContext.getDefaultFileSystem()
.resolvePath(
@ -299,18 +299,18 @@ public ApplicationSubmissionContext createApplicationSubmissionContext(
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 ApplicationSubmissionContext createApplicationSubmissionContext(
// Setup the command to run the AM
Vector<CharSequence> vargs = new Vector<CharSequence>(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 ApplicationSubmissionContext createApplicationSubmissionContext(
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<String, String> environment = new HashMap<String, String>();
MRApps.setClasspath(environment);
// Parse distributed cache
MRApps.setupDistributedCache(jobConf, localResources);
@ -374,12 +374,12 @@ public ApplicationSubmissionContext createApplicationSubmissionContext(
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 void setJobPriority(JobID arg0, String arg1) throws IOException,
public long getProtocolVersion(String arg0, long arg1) throws IOException {
return resMgrDelegate.getProtocolVersion(arg0, arg1);
}
@Override
public long renewDelegationToken(Token<DelegationTokenIdentifier> arg0)
throws IOException, InterruptedException {
return resMgrDelegate.renewDelegationToken(arg0);
}
@Override
public Counters getJobCounters(JobID arg0) throws IOException,
InterruptedException {
@ -419,7 +419,7 @@ public JobStatus getJobStatus(JobID jobID) throws IOException,
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 void killJob(JobID arg0) throws IOException, InterruptedException {
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);

View File

@ -88,7 +88,8 @@
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 void testRedirect() throws Exception {
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 void testRedirect() throws Exception {
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 void testRedirect() throws Exception {
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 void testRedirect() throws Exception {
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 void start() {
public GetNewApplicationResponse getNewApplication(GetNewApplicationRequest request) throws YarnRemoteException {
return null;
}
@Override
public GetApplicationReportResponse getApplicationReport(
GetApplicationReportRequest request) throws YarnRemoteException {
@ -256,12 +257,14 @@ public GetApplicationReportResponse getApplicationReport(
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 GetCountersResponse getCounters(GetCountersRequest request) throws YarnRe
}
}
class AMService extends AbstractService
class AMService extends AbstractService
implements MRClientProtocol {
private InetSocketAddress bindAddress;
private Server server;
@ -347,7 +350,7 @@ class AMService extends AbstractService
public AMService() {
this(AMHOSTADDRESS);
}
public AMService(String hostAddress) {
super("AMService");
this.hostAddress = hostAddress;

View File

@ -32,8 +32,9 @@
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 @@ private GetJobReportResponse getJobReportResponse() {
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 @@ private ApplicationReport getApplicationReport() {
applicationReport.setFinishTime(0);
applicationReport.setTrackingUrl("N/A");
applicationReport.setDiagnostics("N/A");
applicationReport.setFinalApplicationStatus(FinalApplicationStatus.SUCCEEDED);
return applicationReport;
}

View File

@ -36,7 +36,6 @@
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;

View File

@ -64,10 +64,10 @@
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.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 ApplicationSubmissionContext answer(InvocationOnMock invocation)
}
).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 ApplicationSubmissionContext answer(InvocationOnMock invocation)
}
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 ClientServiceDelegate answer(InvocationOnMock invocation)
).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 void testJobSubmissionFailure() throws Exception {
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 void testJobSubmissionFailure() throws Exception {
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 void testResourceMgrDelegate() throws Exception {
.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 void testResourceMgrDelegate() throws Exception {
.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 void testResourceMgrDelegate() throws Exception {
thenReturn(queueInfoResponse);
delegate.getQueues();
verify(clientRMProtocol).getQueueInfo(any(GetQueueInfoRequest.class));
GetQueueUserAclsInfoResponse aclResponse = recordFactory.newRecordInstance(
GetQueueUserAclsInfoResponse.class);
when(clientRMProtocol.getQueueUserAcls(any(GetQueueUserAclsInfoRequest.class)))

View File

@ -22,15 +22,16 @@
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;
/**
* <p>The finalization request sent by the <code>ApplicationMaster</code> to
* <p>The finalization request sent by the <code>ApplicationMaster</code> to
* inform the <code>ResourceManager</code> about its completion.</p>
*
*
* <p>The final request includes details such:
* <ul>
* <li>
* {@link ApplicationAttemptId} being managed by the
* {@link ApplicationAttemptId} being managed by the
* <code>ApplicationMaster</code>
* </li>
* <li>Final state of the <code>ApplicationMaster</code></li>
@ -47,19 +48,19 @@
public interface FinishApplicationMasterRequest {
/**
* Get the <code>ApplicationAttemptId</code> being managed by the
* Get the <code>ApplicationAttemptId</code> being managed by the
* <code>ApplicationMaster</code>.
* @return <code>ApplicationAttemptId</code> being managed by the
* @return <code>ApplicationAttemptId</code> being managed by the
* <code>ApplicationMaster</code>
*/
@Public
@Stable
ApplicationAttemptId getApplicationAttemptId();
/**
* Set the <code>ApplicationAttemptId</code> being managed by the
* Set the <code>ApplicationAttemptId</code> being managed by the
* <code>ApplicationMaster</code>.
* @param applicationAttemptId <code>ApplicationAttemptId</code> being managed
* @param applicationAttemptId <code>ApplicationAttemptId</code> being managed
* by the <code>ApplicationMaster</code>
*/
@Public
@ -72,15 +73,15 @@ public interface FinishApplicationMasterRequest {
*/
@Public
@Stable
String getFinalState();
FinalApplicationStatus getFinalApplicationStatus();
/**
* Set <em>final state</em> of the <code>ApplicationMaster</code>
* @param finalState <em>final state</em> of the <code>ApplicationMaster</code>
* Set the <em>finish state</em> of the <code>ApplicationMaster</code>
* @param finishState <em>finish state</em> of the <code>ApplicationMaster</code>
*/
@Public
@Stable
void setFinalState(String finalState);
void setFinishApplicationStatus(FinalApplicationStatus finishState);
/**
* Get <em>diagnostic information</em> on application failure.
@ -89,7 +90,7 @@ public interface FinishApplicationMasterRequest {
@Public
@Stable
String getDiagnostics();
/**
* Set <em>diagnostic information</em> on application failure.
* @param diagnostics <em>diagnostic information</em> on application failure
@ -105,10 +106,10 @@ public interface FinishApplicationMasterRequest {
@Public
@Stable
String getTrackingUrl();
/**
* Set the <em>tracking URL</em>for the <code>ApplicationMaster</code>
* @param url <em>tracking URL</em>for the
* @param url <em>tracking URL</em>for the
* <code>ApplicationMaster</code>
*/
@Public

View File

@ -21,23 +21,24 @@
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<FinishApplicationMasterRequestProto> 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 FinishApplicationMasterRequestPBImpl(FinishApplicationMasterRequestProto
this.proto = proto;
viaProto = true;
}
public FinishApplicationMasterRequestProto getProto() {
mergeLocalToProto();
proto = viaProto ? proto : builder.build();
@ -61,7 +62,7 @@ private void mergeLocalToBuilder() {
}
private void mergeLocalToProto() {
if (viaProto)
if (viaProto)
maybeInitBuilder();
mergeLocalToBuilder();
proto = builder.build();
@ -74,8 +75,7 @@ private void maybeInitBuilder() {
}
viaProto = false;
}
@Override
public ApplicationAttemptId getApplicationAttemptId() {
FinishApplicationMasterRequestProtoOrBuilder p = viaProto ? proto : builder;
@ -92,7 +92,7 @@ public ApplicationAttemptId getApplicationAttemptId() {
@Override
public void setAppAttemptId(ApplicationAttemptId applicationAttemptId) {
maybeInitBuilder();
if (applicationAttemptId == null)
if (applicationAttemptId == null)
builder.clearApplicationAttemptId();
this.appAttemptId = applicationAttemptId;
}
@ -122,15 +122,22 @@ public void setTrackingUrl(String url) {
}
@Override
public String getFinalState() {
public FinalApplicationStatus getFinalApplicationStatus() {
FinishApplicationMasterRequestProtoOrBuilder p = viaProto ? proto : builder;
return p.getFinalState();
if (!p.hasFinalApplicationStatus()) {
return null;
}
return convertFromProtoFormat(p.getFinalApplicationStatus());
}
@Override
public void setFinalState(String state) {
public void setFinishApplicationStatus(FinalApplicationStatus finishState) {
maybeInitBuilder();
builder.setFinalState(state);
if (finishState == null) {
builder.clearFinalApplicationStatus();
return;
}
builder.setFinalApplicationStatus(convertToProtoFormat(finishState));
}
private ApplicationAttemptIdPBImpl convertFromProtoFormat(ApplicationAttemptIdProto p) {
@ -141,6 +148,13 @@ private ApplicationAttemptIdProto convertToProtoFormat(ApplicationAttemptId t) {
return ((ApplicationAttemptIdPBImpl)t).getProto();
}
private FinalApplicationStatus convertFromProtoFormat(FinalApplicationStatusProto s) {
return ProtoUtils.convertFromProtoFormat(s);
}
private FinalApplicationStatusProto convertToProtoFormat(FinalApplicationStatus s) {
return ProtoUtils.convertToProtoFormat(s);
}
}
}

View File

@ -23,24 +23,22 @@
import org.apache.hadoop.yarn.api.records.ProtoBase;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.impl.pb.ResourcePBImpl;
import org.apache.hadoop.yarn.proto.YarnProtos.QueueInfoProto;
import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProto;
import org.apache.hadoop.yarn.proto.YarnServiceProtos.RegisterApplicationMasterResponseProto;
import org.apache.hadoop.yarn.proto.YarnServiceProtos.RegisterApplicationMasterResponseProtoOrBuilder;
public class RegisterApplicationMasterResponsePBImpl
extends ProtoBase<RegisterApplicationMasterResponseProto>
public class RegisterApplicationMasterResponsePBImpl
extends ProtoBase<RegisterApplicationMasterResponseProto>
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 @@ public RegisterApplicationMasterResponsePBImpl(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 @@ public Resource getMaximumResourceCapability() {
if (!p.hasMaximumCapability()) {
return null;
}
this.maximumResourceCapability = convertFromProtoFormat(p.getMaximumCapability());
return this.maximumResourceCapability;
}
@ -109,7 +107,7 @@ public Resource getMinimumResourceCapability() {
if (!p.hasMinimumCapability()) {
return null;
}
this.minimumResourceCapability = convertFromProtoFormat(p.getMinimumCapability());
return this.minimumResourceCapability;
}
@ -140,4 +138,4 @@ private ResourceProto convertToProtoFormat(Resource resource) {
return ((ResourcePBImpl)resource).getProto();
}
}
}

View File

@ -22,38 +22,38 @@
import org.apache.hadoop.classification.InterfaceStability.Unstable;
/**
* <em>For internal use only...</em>
* <em>For internal use only...</em>
*/
@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);
}

View File

@ -36,13 +36,13 @@
* <li>Host on which the <code>ApplicationMaster</code>is running.</li>
* <li>RPC port of the <code>ApplicationMaster</code>.</li>
* <li>Tracking URL.</li>
* <li>{@link ApplicationState} of the application.</li>
* <li>{@link YarnApplicationState} of the application.</li>
* <li>Diagnostic information in case of errors.</li>
* <li>Start time of the application.</li>
* <li>Client token of the application (if security is enabled).</li>
* </ul>
* </p>
*
*
* @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 <em>host</em> on which the <code>ApplicationMaster</code>
* Get the <em>host</em> on which the <code>ApplicationMaster</code>
* is running.
* @return <em>host</em> on which the <code>ApplicationMaster</code>
* @return <em>host</em> on which the <code>ApplicationMaster</code>
* 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 <em>client token</em> for communicating with the
* Get the <em>client token</em> for communicating with the
* <code>ApplicationMaster</code>.
* @return <em>client token</em> for communicating with the
* @return <em>client token</em> for communicating with the
* <code>ApplicationMaster</code>
*/
@Public
@Stable
String getClientToken();
@Private
@Unstable
void setClientToken(String clientToken);
/**
* Get the <code>ApplicationState</code> of the application.
* @return <code>ApplicationState</code> of the application
* Get the <code>YarnApplicationState</code> of the application.
* @return <code>YarnApplicationState</code> of the application
*/
@Public
@Stable
ApplicationState getState();
YarnApplicationState getYarnApplicationState();
@Private
@Unstable
void setState(ApplicationState state);
void setYarnApplicationState(YarnApplicationState state);
/**
* Get the <em>diagnositic information</em> of the application in case of
* Get the <em>diagnositic information</em> of the application in case of
* errors.
* @return <em>diagnositic information</em> of the application in case
* @return <em>diagnositic information</em> 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 <em>start time</em> of the application.
* @return <em>start time</em> 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 <em>final finish status</em> of the application.
*/
@Public
@Stable
FinalApplicationStatus getFinalApplicationStatus();
@Private
@Unstable
void setFinalApplicationStatus(FinalApplicationStatus finishState);
}

View File

@ -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 <code>Application</code>.
*/
@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
}

View File

@ -22,26 +22,26 @@
import org.apache.hadoop.classification.InterfaceStability.Stable;
/**
* Ennumeration of various states of an <code>Application</code>.
* Ennumeration of various states of an <code>ApplicationMaster</code>.
*/
@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
}

View File

@ -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<ApplicationMasterProto> 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 ApplicationMasterPBImpl(ApplicationMasterProto proto) {
this.proto = proto;
viaProto = true;
}
public ApplicationMasterProto getProto() {
mergeLocalToProto();
proto = viaProto ? proto : builder.build();
viaProto = true;
@ -72,24 +69,24 @@ private void mergeLocalToBuilder() {
}
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 ApplicationState getState() {
}
@Override
public void setState(ApplicationState state) {
public void setState(YarnApplicationState state) {
maybeInitBuilder();
if (state == null) {
builder.clearState();
@ -124,7 +121,7 @@ public void setHost(String host) {
}
builder.setHost((host));
}
@Override
public ApplicationId getApplicationId() {
ApplicationMasterProtoOrBuilder p = viaProto ? proto : builder;
@ -135,7 +132,7 @@ public ApplicationId getApplicationId() {
return null;
}
applicationId = convertFromProtoFormat(p.getApplicationId());
return applicationId;
}
@ -145,7 +142,7 @@ public void setApplicationId(ApplicationId applicationId) {
if (applicationId == null)
builder.clearApplicationId();
this.applicationId = applicationId;
}
@Override
public int getRpcPort() {
@ -179,7 +176,7 @@ public ApplicationStatus getStatus() {
return null;
}
this.applicationStatus = convertFromProtoFormat(p.getStatus());
return this.applicationStatus;
}
@ -189,7 +186,7 @@ public void setStatus(ApplicationStatus status) {
if (status == null)
builder.clearStatus();
this.applicationStatus = status;
}
@Override
public String getClientToken() {
@ -209,7 +206,7 @@ public void setClientToken(String clientToken) {
}
builder.setClientToken((clientToken));
}
@Override
public int getAMFailCount() {
ApplicationMasterProtoOrBuilder p = viaProto ? proto : builder;
@ -250,11 +247,11 @@ public void setDiagnostics(String diagnostics) {
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 @@ private ApplicationStatusProto convertToProtoFormat(ApplicationStatus t) {
return ((ApplicationStatusPBImpl)t).getProto();
}
}
}

View File

@ -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<ApplicationReportProto>
public class ApplicationReportPBImpl extends ProtoBase<ApplicationReportProto>
implements ApplicationReport {
ApplicationReportProto proto = ApplicationReportProto.getDefaultInstance();
ApplicationReportProto.Builder builder = null;
@ -39,7 +41,7 @@ public class ApplicationReportPBImpl extends ProtoBase<ApplicationReportProto>
public ApplicationReportPBImpl() {
builder = ApplicationReportProto.newBuilder();
}
public ApplicationReportPBImpl(ApplicationReportProto proto) {
this.proto = proto;
viaProto = true;
@ -87,12 +89,12 @@ public String getQueue() {
}
@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 @@ public String getDiagnostics() {
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 @@ public void setQueue(String queue) {
}
@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 @@ public void setDiagnostics(String diagnostics) {
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 @@ private ApplicationIdProto convertToProtoFormat(ApplicationId t) {
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);
}
}

View File

@ -20,24 +20,26 @@
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 static ContainerStateProto convertToProtoFormat(ContainerState e) {
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 static LocalResourceTypeProto convertToProtoFormat(LocalResourceType e) {
public static LocalResourceType convertFromProtoFormat(LocalResourceTypeProto e) {
return LocalResourceType.valueOf(e.name());
}
/*
* LocalResourceVisibility
*/
@ -79,7 +92,7 @@ public static LocalResourceVisibilityProto convertToProtoFormat(LocalResourceVis
public static LocalResourceVisibility convertFromProtoFormat(LocalResourceVisibilityProto e) {
return LocalResourceVisibility.valueOf(e.name());
}
/*
* ByteBuffer
*/
@ -98,7 +111,7 @@ public static ByteString convertToProtoFormat(ByteBuffer byteBuffer) {
byteBuffer.position(oldPos);
return bs;
}
/*
* QueueState
*/
@ -109,7 +122,7 @@ public static QueueStateProto convertToProtoFormat(QueueState e) {
public static QueueState convertFromProtoFormat(QueueStateProto e) {
return QueueState.valueOf(e.name().replace(QUEUE_STATE_PREFIX, ""));
}
/*
* QueueACL
*/

View File

@ -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;

View File

@ -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;
}
//////////////////////////////////////////////////////

View File

@ -24,9 +24,10 @@
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 int compare(ContainerId c1,
}
}
public static class ResourceRequestComparator
public static class ResourceRequestComparator
implements java.util.Comparator<org.apache.hadoop.yarn.api.records.ResourceRequest> {
@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 static Container newContainer(RecordFactory recordFactory,
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 static ResourceRequest newResourceRequest(ResourceRequest r) {
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 static ApplicationReport newApplicationReport(
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);

View File

@ -33,7 +33,9 @@
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 static ApplicationAttemptId toApplicationAttemptId(
+ applicationAttmeptIdStr, n);
}
}
}

View File

@ -24,7 +24,8 @@
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<String> USERS = Iterators.cycle("dorothy", "tinman",
"scarecrow", "glinda", "nikko", "toto", "winkie", "zeke", "gulch");
static final Iterator<ApplicationState> STATES = Iterators.cycle(
ApplicationState.values());
static final Iterator<YarnApplicationState> STATES = Iterators.cycle(
YarnApplicationState.values());
static final Iterator<String> 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 static List<ApplicationReport> genApps(int n) {
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 String getDiagnostics() {
@Override
public void setDiagnostics(String diagnostics) {
// TODO Auto-generated method stub
}
@Override
public String getHost() {
@ -133,7 +135,7 @@ public String getHost() {
@Override
public void setHost(String host) {
// TODO Auto-generated method stub
}
@Override
public int getRpcPort() {
@ -143,7 +145,7 @@ public int getRpcPort() {
@Override
public void setRpcPort(int rpcPort) {
// TODO Auto-generated method stub
}
@Override
public String getClientToken() {
@ -153,9 +155,8 @@ public String getClientToken() {
@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 long getFinishTime() {
@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 static ApplicationAttemptId newAppAttemptID(ApplicationId appId, int i) {
return id;
}
public static ApplicationState newAppState() {
public static YarnApplicationState newAppState() {
synchronized(STATES) {
return STATES.next();
}
}
}

View File

@ -75,7 +75,7 @@ public class ApplicationMasterService extends AbstractService implements
new ConcurrentHashMap<ApplicationAttemptId, AMResponse>();
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 void start() {
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 RegisterApplicationMasterResponse registerApplicationMaster(
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 RegisterApplicationMasterResponse registerApplicationMaster(
.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 FinishApplicationMasterResponse finishApplicationMaster(
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 AllocateResponse allocate(AllocateRequest request)
List<ContainerId> 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());

View File

@ -20,6 +20,7 @@
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.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<RMAppEvent> {
@ -68,7 +69,7 @@ public interface RMApp extends EventHandler<RMAppEvent> {
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<RMAppEvent> {
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<RMAppEvent> {
/**
* {@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<RMAppEvent> {
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<RMAppEvent> {
* @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();
}

View File

@ -32,9 +32,10 @@
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<RMAppImpl,
RMAppState,
RMAppEventType,
RMAppEvent> stateMachineFactory
RMAppEvent> stateMachineFactory
= new StateMachineFactory<RMAppImpl,
RMAppState,
RMAppEventType,
@ -160,7 +161,7 @@ RMAppEventType.KILL, new KillAppAndAttemptTransition())
public RMAppImpl(ApplicationId applicationId, RMContext rmContext,
Configuration config, String name, String user, String queue,
ApplicationSubmissionContext submissionContext, String clientTokenStr,
ApplicationStore appStore,
ApplicationStore appStore,
YarnScheduler scheduler, ApplicationMasterService masterService) {
this.applicationId = applicationId;
@ -194,18 +195,23 @@ public ApplicationId getApplicationId() {
}
@Override
public String getAMFinalState() {
public FinalApplicationStatus getFinalApplicationStatus() {
this.readLock.lock();
try {
if (currentAttempt != null) {
return currentAttempt.getAMFinalState();
// finish state is obtained based on the state machine's current state
// as a fall-back in case the application has not been unregistered
// ( or if the app never unregistered itself )
// when the report is requested
if (currentAttempt != null
&& currentAttempt.getFinalApplicationStatus() != null) {
return currentAttempt.getFinalApplicationStatus();
}
return "UNKNOWN";
return createFinalApplicationStatus(this.stateMachine.getCurrentState());
} finally {
this.readLock.unlock();
}
}
@Override
public RMAppState getState() {
this.readLock.lock();
@ -273,25 +279,43 @@ public ApplicationStore getApplicationStore() {
return this.appStore;
}
private ApplicationState createApplicationState(RMAppState rmAppState) {
private YarnApplicationState createApplicationState(RMAppState rmAppState) {
switch(rmAppState) {
case NEW:
return ApplicationState.NEW;
return YarnApplicationState.NEW;
case SUBMITTED:
case ACCEPTED:
return ApplicationState.SUBMITTED;
return YarnApplicationState.SUBMITTED;
case RUNNING:
return ApplicationState.RUNNING;
return YarnApplicationState.RUNNING;
case FINISHED:
return ApplicationState.SUCCEEDED;
return YarnApplicationState.FINISHED;
case KILLED:
return ApplicationState.KILLED;
return YarnApplicationState.KILLED;
case FAILED:
return ApplicationState.FAILED;
return YarnApplicationState.FAILED;
}
throw new YarnException("Unknown state passed!");
}
private FinalApplicationStatus createFinalApplicationStatus(RMAppState state) {
switch(state) {
case NEW:
case SUBMITTED:
case ACCEPTED:
case RUNNING:
return FinalApplicationStatus.UNDEFINED;
// finished without a proper final state is the same as failed
case FINISHED:
case FAILED:
return FinalApplicationStatus.FAILED;
case KILLED:
return FinalApplicationStatus.KILLED;
}
throw new YarnException("Unknown state passed!");
}
@Override
public ApplicationReport createAndGetApplicationReport() {
this.readLock.lock();
@ -301,6 +325,7 @@ public ApplicationReport createAndGetApplicationReport() {
String trackingUrl = "N/A";
String host = "N/A";
int rpcPort = -1;
FinalApplicationStatus finishState = getFinalApplicationStatus();
if (this.currentAttempt != null) {
trackingUrl = this.currentAttempt.getTrackingUrl();
clientToken = this.currentAttempt.getClientToken();
@ -310,8 +335,8 @@ public ApplicationReport createAndGetApplicationReport() {
return BuilderUtils.newApplicationReport(this.applicationId, this.user,
this.queue, this.name, host, rpcPort, clientToken,
createApplicationState(this.stateMachine.getCurrentState()),
this.diagnostics.toString(), trackingUrl,
this.startTime, this.finishTime);
this.diagnostics.toString(), trackingUrl,
this.startTime, this.finishTime, finishState);
} finally {
this.readLock.unlock();
}
@ -432,7 +457,7 @@ private static class KillAppAndAttemptTransition extends AppKilledTransition {
@SuppressWarnings("unchecked")
@Override
public void transition(RMAppImpl app, RMAppEvent event) {
app.handler.handle(new RMAppAttemptEvent(app.currentAttempt.getAppAttemptId(),
app.handler.handle(new RMAppAttemptEvent(app.currentAttempt.getAppAttemptId(),
RMAppAttemptEventType.KILL));
super.transition(app, event);
}
@ -465,7 +490,7 @@ public void transition(RMAppImpl app, RMAppEvent event) {
}
app.finishTime = System.currentTimeMillis();
app.handler.handle(
new RMAppManagerEvent(app.applicationId,
new RMAppManagerEvent(app.applicationId,
RMAppManagerEventType.APP_COMPLETED));
};
}
@ -481,7 +506,7 @@ public AttemptFailedTransition(RMAppState initialState) {
@Override
public RMAppState transition(RMAppImpl app, RMAppEvent event) {
RMAppFailedAttemptEvent failedEvent = ((RMAppFailedAttemptEvent)event);
if (app.attempts.size() == app.maxRetries) {
String msg = "Application " + app.getApplicationId()
@ -495,7 +520,7 @@ public RMAppState transition(RMAppImpl app, RMAppEvent event) {
return RMAppState.FAILED;
}
app.createNewAttempt();
app.createNewAttempt();
return initialState;
}

View File

@ -22,6 +22,7 @@
import java.util.Set;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerStatus;
@ -32,8 +33,8 @@
/**
* Interface to an Application Attempt in the Resource Manager.
* A {@link RMApp} can have multiple app attempts based on
* {@link YarnConfiguration#RM_AM_MAX_RETRIES}. For specific
* A {@link RMApp} can have multiple app attempts based on
* {@link YarnConfiguration#RM_AM_MAX_RETRIES}. For specific
* implementation take a look at {@link RMAppAttemptImpl}.
*/
public interface RMAppAttempt extends EventHandler<RMAppAttemptEvent> {
@ -49,7 +50,7 @@ public interface RMAppAttempt extends EventHandler<RMAppAttemptEvent> {
* @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<RMAppAttemptEvent> {
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<RMAppAttemptEvent> {
Set<NodeId> 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<ContainerStatus> 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<ContainerStatus> getJustFinishedContainers();

View File

@ -31,6 +31,7 @@
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<NodeId> ranNodes =
private final Set<NodeId> ranNodes =
new HashSet<NodeId>();
private final List<ContainerStatus> justFinishedContainers =
private final List<ContainerStatus> justFinishedContainers =
new ArrayList<ContainerStatus>();
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<RMAppAttemptImpl,
@ -150,7 +153,7 @@ RMAppAttemptEventType.APP_ACCEPTED, new ScheduleTransition())
.addTransition(RMAppAttemptState.ALLOCATED,
RMAppAttemptState.ALLOCATED,
RMAppAttemptEventType.CONTAINER_ACQUIRED,
new ContainerAcquiredTransition())
new ContainerAcquiredTransition())
.addTransition(RMAppAttemptState.ALLOCATED, RMAppAttemptState.LAUNCHED,
RMAppAttemptEventType.LAUNCHED, new AMLaunchedTransition())
.addTransition(RMAppAttemptState.ALLOCATED, RMAppAttemptState.FAILED,
@ -266,12 +269,12 @@ public ApplicationAttemptId getAppAttemptId() {
public ApplicationSubmissionContext getSubmissionContext() {
return this.submissionContext;
}
@Override
public String getAMFinalState() {
public FinalApplicationStatus getFinalApplicationStatus() {
this.readLock.lock();
try {
return this.finalState;
return this.finalStatus;
} finally {
this.readLock.unlock();
}
@ -430,9 +433,9 @@ private static class BaseTransition implements
@Override
public void transition(RMAppAttemptImpl appAttempt,
RMAppAttemptEvent event) {
RMAppAttemptEvent event) {
}
}
private static final class AttemptStartedTransition extends BaseTransition {
@ -459,23 +462,23 @@ public void transition(RMAppAttemptImpl appAttempt,
RMAppAttemptEvent event) {
RMAppAttemptRejectedEvent rejectedEvent = (RMAppAttemptRejectedEvent) event;
// Save the diagnostic message
String message = rejectedEvent.getMessage();
appAttempt.setDiagnostics(message);
// Send the rejection event to app
appAttempt.eventHandler.handle(
new RMAppRejectedEvent(
rejectedEvent.getApplicationAttemptId().getApplicationId(),
rejectedEvent.getApplicationAttemptId().getApplicationId(),
message)
);
}
}
private static final List<ContainerId> EMPTY_CONTAINER_RELEASE_LIST =
private static final List<ContainerId> EMPTY_CONTAINER_RELEASE_LIST =
new ArrayList<ContainerId>();
private static final List<ResourceRequest> EMPTY_CONTAINER_REQUEST_LIST =
private static final List<ResourceRequest> EMPTY_CONTAINER_REQUEST_LIST =
new ArrayList<ResourceRequest>();
private static final class ScheduleTransition extends BaseTransition {
@ -540,23 +543,23 @@ public void transition(RMAppAttemptImpl appAttempt,
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 void transition(RMAppAttemptImpl appAttempt,
}
break;
}
appAttempt.eventHandler.handle(appEvent);
appAttempt.eventHandler.handle(new AppRemovedSchedulerEvent(appAttempt
.getAppAttemptId(), finalAttemptState));
@ -657,7 +660,7 @@ public void transition(RMAppAttemptImpl appAttempt,
RMAppAttemptContainerFinishedEvent finishEvent =
((RMAppAttemptContainerFinishedEvent)event);
// UnRegister from AMLivelinessMonitor
appAttempt.rmContext.getAMLivelinessMonitor().unregister(
appAttempt.getAppAttemptId());
@ -666,7 +669,7 @@ public void transition(RMAppAttemptImpl appAttempt,
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 void transition(RMAppAttemptImpl appAttempt,
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 RMAppAttemptState transition(RMAppAttemptImpl appAttempt,
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 RMAppAttemptState transition(RMAppAttemptImpl appAttempt,
// 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.");

View File

@ -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 String getTrackingUrl() {
return this.trackingUrl;
}
public String getFinalState() {
return this.finalState;
public FinalApplicationStatus getFinalApplicationStatus() {
return this.finalStatus;
}
public String getDiagnostics() {
return this.diagnostics;
}
}
}

View File

@ -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).

View File

@ -89,9 +89,8 @@ public void app() {
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()))).

View File

@ -30,7 +30,7 @@
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<ResourceRequest> requests = new ArrayList<ResourceRequest>();
private final List<ContainerId> releases = new ArrayList<ContainerId>();
MockAM(RMContext context, AMRMProtocol amRMProtocol,
MockAM(RMContext context, AMRMProtocol amRMProtocol,
ApplicationAttemptId attemptId) {
this.context = context;
this.amRMProtocol = amRMProtocol;
@ -85,7 +85,7 @@ public void registerAppAttempt() throws Exception {
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 AMResponse schedule() throws Exception {
return response;
}
public AMResponse allocate(
String host, int memory, int numContainers,
public AMResponse allocate(
String host, int memory, int numContainers,
List<ContainerId> releases) throws Exception {
List reqs = createReq(new String[]{host}, memory, 1, numContainers);
List<ResourceRequest> reqs = createReq(new String[]{host}, memory, 1, numContainers);
return allocate(reqs, releases);
}
public List<ResourceRequest> createReq(String[] hosts, int memory, int priority,
public List<ResourceRequest> createReq(String[] hosts, int memory, int priority,
int containers) throws Exception {
List<ResourceRequest> reqs = new ArrayList<ResourceRequest>();
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 ResourceRequest createResourceReq(String resource, int memory, int priori
}
public AMResponse allocate(
List<ResourceRequest> resourceRequest, List<ContainerId> releases)
List<ResourceRequest> resourceRequest, List<ContainerId> releases)
throws Exception {
AllocateRequest req = BuilderUtils.newAllocateRequest(attemptId,
++responseId, 0F, resourceRequest, releases);
@ -151,7 +151,7 @@ public void unregisterAppAttempt() throws Exception {
FinishApplicationMasterRequest req = Records.newRecord(FinishApplicationMasterRequest.class);
req.setAppAttemptId(attemptId);
req.setDiagnostics("");
req.setFinalState("");
req.setFinishApplicationStatus(FinalApplicationStatus.SUCCEEDED);
req.setTrackingUrl("");
amRMProtocol.finishApplicationMaster(req);
}

View File

@ -22,10 +22,11 @@
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 ApplicationStatus getStatus() {
}
@Override
public ApplicationState getState() {
public YarnApplicationState getState() {
throw new UnsupportedOperationException("Not supported yet.");
}
@ -119,7 +120,7 @@ public void setStatus(ApplicationStatus status) {
}
@Override
public void setState(ApplicationState state) {
public void setState(YarnApplicationState state) {
throw new UnsupportedOperationException("Not supported yet.");
}
@ -207,11 +208,11 @@ public ApplicationReport createAndGetApplicationReport() {
}
@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 StringBuilder getDiagnostics() {
public float getProgress() {
return (float)Math.random();
}
@Override
public FinalApplicationStatus getFinalApplicationStatus() {
return FinalApplicationStatus.UNDEFINED;
}
};
}
public static List<RMApp> newApplications(int n) {
List<RMApp> list = Lists.newArrayList();
for (int i = 0; i < n; ++i) {

View File

@ -32,7 +32,7 @@
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."

View File

@ -33,7 +33,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.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<ResourceRequest> ask, List<Container> release) throws IOException {
@ -222,7 +222,7 @@ public class TestAMRestart {
//
// @Override
// public void nodeUpdate(RMNode nodeInfo,
// Map<String, List<Container>> containers) {
// Map<String, List<Container>> 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;
// }
}
}

View File

@ -26,7 +26,7 @@
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());
// }
}

View File

@ -26,7 +26,7 @@
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 @@
/**
* 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<ASMEvent<ApplicationTrackerEventType>> {
// DummyApplicationTracker() {
// context.getDispatcher().register(ApplicationTrackerEventType.class, this);
@ -79,10 +79,10 @@ public class TestApplicationMasterExpiry {
// public void handle(ASMEvent<ApplicationTrackerEventType> event) {
// }
// }
//
//
// private AtomicInteger expiry = new AtomicInteger();
// private boolean expired = false;
//
//
// private class ApplicationEventTypeListener implements
// EventHandler<ApplicationEvent> {
// ApplicationEventTypeListener() {
@ -100,7 +100,7 @@ public class TestApplicationMasterExpiry {
// }
// }
// }
//
//
// private class DummySN implements EventHandler<ASMEvent<SNEventType>> {
// DummySN() {
// context.getDispatcher().register(SNEventType.class, this);
@ -109,7 +109,7 @@ public class TestApplicationMasterExpiry {
// public void handle(ASMEvent<SNEventType> event) {
// }
// }
//
//
// private class DummyLauncher implements EventHandler<ASMEvent<AMLauncherEventType>> {
// DummyLauncher() {
// context.getDispatcher().register(AMLauncherEventType.class, this);
@ -118,8 +118,8 @@ public class TestApplicationMasterExpiry {
// public void handle(ASMEvent<AMLauncherEventType> 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) {

View File

@ -28,7 +28,7 @@
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<String, List<Container>> 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

View File

@ -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 void handle(RMAppEvent event) {
}
@Override
public String getAMFinalState() {
return "UNKNOWN";
public FinalApplicationStatus getFinalApplicationStatus() {
return FinalApplicationStatus.UNDEFINED;
};
}

View File

@ -31,6 +31,7 @@
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 @@ private static void assertStartTimeSet(RMApp application) {
}
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 @@ private static void assertTimesAtFinish(RMApp application) {
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 @@ private static void assertAppAndAttemptKilled(RMApp application) {
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 @@ protected RMApp testCreateAppRunning() throws IOException {
application.handle(event);
assertStartTimeSet(application);
assertAppState(RMAppState.RUNNING, application);
assertFinalAppStatus(FinalApplicationStatus.UNDEFINED, application);
return application;
}
@ -273,6 +282,8 @@ protected RMApp testCreateAppFinished() throws IOException {
application.handle(event);
assertAppState(RMAppState.FINISHED, application);
assertTimesAtFinish(application);
// finished without a proper unregister implies failed
assertFinalAppStatus(FinalApplicationStatus.FAILED, application);
return application;
}

View File

@ -34,6 +34,7 @@
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.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 @@ private void testAppAttemptNewState() {
assertNull(applicationAttempt.getMasterContainer());
assertEquals(0.0, (double)applicationAttempt.getProgress(), 0.0001);
assertEquals(0, applicationAttempt.getRanNodes().size());
assertNull(applicationAttempt.getFinalApplicationStatus());
}
/**
@ -212,6 +216,7 @@ private void testAppAttemptSubmittedState() {
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 @@ private void testAppAttemptSubmittedToFailedState(String diagnostics) {
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 @@ private void testAppAttemptKilledState(Container amContainer,
assertEquals(amContainer, applicationAttempt.getMasterContainer());
assertEquals(0.0, (double)applicationAttempt.getProgress(), 0.0001);
assertEquals(0, applicationAttempt.getRanNodes().size());
assertNull(applicationAttempt.getFinalApplicationStatus());
}
/**
@ -259,6 +266,7 @@ private void testAppAttemptScheduledState() {
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 @@ private void testAppAttemptFailedState(Container container,
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 @@ private Container allocateApplicationAttempt() {
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 void testAllocatedToFailed() {
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);
}
}

View File

@ -25,7 +25,7 @@
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;