|
@@ -19,14 +19,13 @@
|
|
|
package org.apache.hadoop.mapred;
|
|
|
|
|
|
import java.io.IOException;
|
|
|
-import java.net.UnknownHostException;
|
|
|
+import java.lang.reflect.Method;
|
|
|
import java.security.PrivilegedAction;
|
|
|
import java.util.List;
|
|
|
|
|
|
import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
-import org.apache.hadoop.fs.CommonConfigurationKeys;
|
|
|
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
|
|
|
import org.apache.hadoop.io.Text;
|
|
|
import org.apache.hadoop.mapreduce.JobID;
|
|
@@ -37,16 +36,21 @@ import org.apache.hadoop.mapreduce.TypeConverter;
|
|
|
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.GetCountersRequest;
|
|
|
+import org.apache.hadoop.mapreduce.v2.api.protocolrecords.GetCountersResponse;
|
|
|
import org.apache.hadoop.mapreduce.v2.api.protocolrecords.GetDiagnosticsRequest;
|
|
|
+import org.apache.hadoop.mapreduce.v2.api.protocolrecords.GetDiagnosticsResponse;
|
|
|
import org.apache.hadoop.mapreduce.v2.api.protocolrecords.GetJobReportRequest;
|
|
|
+import org.apache.hadoop.mapreduce.v2.api.protocolrecords.GetJobReportResponse;
|
|
|
import org.apache.hadoop.mapreduce.v2.api.protocolrecords.GetTaskAttemptCompletionEventsRequest;
|
|
|
+import org.apache.hadoop.mapreduce.v2.api.protocolrecords.GetTaskAttemptCompletionEventsResponse;
|
|
|
import org.apache.hadoop.mapreduce.v2.api.protocolrecords.GetTaskReportsRequest;
|
|
|
+import org.apache.hadoop.mapreduce.v2.api.protocolrecords.GetTaskReportsResponse;
|
|
|
import org.apache.hadoop.mapreduce.v2.api.protocolrecords.KillJobRequest;
|
|
|
import org.apache.hadoop.mapreduce.v2.api.protocolrecords.KillTaskAttemptRequest;
|
|
|
+import org.apache.hadoop.mapreduce.v2.api.records.Counters;
|
|
|
import org.apache.hadoop.mapreduce.v2.api.records.JobReport;
|
|
|
import org.apache.hadoop.mapreduce.v2.api.records.JobState;
|
|
|
import org.apache.hadoop.mapreduce.v2.jobhistory.JHConfig;
|
|
|
-import org.apache.hadoop.mapreduce.v2.jobhistory.JobHistoryUtils;
|
|
|
import org.apache.hadoop.net.NetUtils;
|
|
|
import org.apache.hadoop.security.SecurityInfo;
|
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
@@ -65,48 +69,47 @@ import org.apache.hadoop.yarn.security.ApplicationTokenIdentifier;
|
|
|
import org.apache.hadoop.yarn.security.SchedulerSecurityInfo;
|
|
|
import org.apache.hadoop.yarn.security.client.ClientRMSecurityInfo;
|
|
|
|
|
|
-public class ClientServiceDelegate {
|
|
|
+class ClientServiceDelegate {
|
|
|
private static final Log LOG = LogFactory.getLog(ClientServiceDelegate.class);
|
|
|
+ private static final NotRunningJob NOTSTARTEDJOB =
|
|
|
+ new NotRunningJob(JobState.NEW);
|
|
|
+
|
|
|
+ private static final NotRunningJob FAILEDJOB =
|
|
|
+ new NotRunningJob(JobState.FAILED);
|
|
|
+
|
|
|
+ private static final NotRunningJob KILLEDJOB =
|
|
|
+ new NotRunningJob(JobState.KILLED);
|
|
|
|
|
|
- private Configuration conf;
|
|
|
- private ApplicationId currentAppId;
|
|
|
- private ApplicationState currentAppState = ApplicationState.NEW;
|
|
|
+ private final Configuration conf;
|
|
|
+ private final JobID jobId;
|
|
|
+ private final ApplicationId appId;
|
|
|
private final ResourceMgrDelegate rm;
|
|
|
+ private final MRClientProtocol historyServerProxy;
|
|
|
+ private boolean forceRefresh;
|
|
|
private MRClientProtocol realProxy = null;
|
|
|
- private String serviceAddr = "";
|
|
|
- private String serviceHttpAddr = "";
|
|
|
private RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
|
|
|
|
|
|
- ClientServiceDelegate(Configuration conf, ResourceMgrDelegate rm) {
|
|
|
+ ClientServiceDelegate(Configuration conf, ResourceMgrDelegate rm,
|
|
|
+ JobID jobId, MRClientProtocol historyServerProxy) {
|
|
|
this.conf = new Configuration(conf); // Cloning for modifying.
|
|
|
// For faster redirects from AM to HS.
|
|
|
this.conf.setInt(
|
|
|
CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY, 3);
|
|
|
this.rm = rm;
|
|
|
+ this.jobId = jobId;
|
|
|
+ this.historyServerProxy = historyServerProxy;
|
|
|
+ this.appId = TypeConverter.toYarn(jobId).getAppId();
|
|
|
}
|
|
|
|
|
|
- private MRClientProtocol getProxy(JobID jobId) throws YarnRemoteException {
|
|
|
- return getProxy(TypeConverter.toYarn(jobId).getAppId(), false);
|
|
|
- }
|
|
|
-
|
|
|
- private MRClientProtocol getRefreshedProxy(JobID jobId) throws YarnRemoteException {
|
|
|
- return getProxy(TypeConverter.toYarn(jobId).getAppId(), true);
|
|
|
- }
|
|
|
-
|
|
|
- private MRClientProtocol getProxy(ApplicationId appId,
|
|
|
- boolean forceRefresh) throws YarnRemoteException {
|
|
|
- if (!appId.equals(currentAppId) || forceRefresh || realProxy == null) {
|
|
|
- currentAppId = appId;
|
|
|
- refreshProxy();
|
|
|
+ private MRClientProtocol getProxy() throws YarnRemoteException {
|
|
|
+ if (!forceRefresh && realProxy != null) {
|
|
|
+ return realProxy;
|
|
|
}
|
|
|
- return realProxy;
|
|
|
- }
|
|
|
-
|
|
|
- private void refreshProxy() throws YarnRemoteException {
|
|
|
- //TODO RM NPEs for unknown jobs. History may still be aware.
|
|
|
+ //TODO RM NPEs for unknown jobs. History may still be aware.
|
|
|
// Possibly allow nulls through the PB tunnel, otherwise deal with an exception
|
|
|
// and redirect to the history server.
|
|
|
- ApplicationReport application = rm.getApplicationReport(currentAppId);
|
|
|
+ ApplicationReport application = rm.getApplicationReport(appId);
|
|
|
+ String serviceAddr = null;
|
|
|
while (ApplicationState.RUNNING.equals(application.getState())) {
|
|
|
try {
|
|
|
if (application.getHost() == null || "".equals(application.getHost())) {
|
|
@@ -114,12 +117,10 @@ public class ClientServiceDelegate {
|
|
|
Thread.sleep(2000);
|
|
|
|
|
|
LOG.debug("Application state is " + application.getState());
|
|
|
- application = rm.getApplicationReport(currentAppId);
|
|
|
+ application = rm.getApplicationReport(appId);
|
|
|
continue;
|
|
|
}
|
|
|
serviceAddr = application.getHost() + ":" + application.getRpcPort();
|
|
|
- serviceHttpAddr = application.getTrackingUrl();
|
|
|
- currentAppState = application.getState();
|
|
|
if (UserGroupInformation.isSecurityEnabled()) {
|
|
|
String clientTokenEncoded = application.getClientToken();
|
|
|
Token<ApplicationTokenIdentifier> clientToken =
|
|
@@ -129,9 +130,10 @@ public class ClientServiceDelegate {
|
|
|
+ application.getRpcPort()));
|
|
|
UserGroupInformation.getCurrentUser().addToken(clientToken);
|
|
|
}
|
|
|
+ LOG.info("Tracking Url of JOB is " + application.getTrackingUrl());
|
|
|
LOG.info("Connecting to " + serviceAddr);
|
|
|
instantiateAMProxy(serviceAddr);
|
|
|
- return;
|
|
|
+ return realProxy;
|
|
|
} catch (Exception e) {
|
|
|
//possibly
|
|
|
//possibly the AM has crashed
|
|
@@ -143,41 +145,39 @@ public class ClientServiceDelegate {
|
|
|
Thread.sleep(2000);
|
|
|
} catch (InterruptedException e1) {
|
|
|
}
|
|
|
- application = rm.getApplicationReport(currentAppId);
|
|
|
+ application = rm.getApplicationReport(appId);
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- currentAppState = application.getState();
|
|
|
/** we just want to return if its allocating, so that we dont
|
|
|
* block on it. This is to be able to return job status
|
|
|
* on a allocating Application.
|
|
|
*/
|
|
|
|
|
|
- if (currentAppState == ApplicationState.NEW) {
|
|
|
+ if (application.getState() == ApplicationState.NEW ||
|
|
|
+ application.getState() == ApplicationState.SUBMITTED) {
|
|
|
+ realProxy = null;
|
|
|
+ return NOTSTARTEDJOB;
|
|
|
+ }
|
|
|
+
|
|
|
+ if (application.getState() == ApplicationState.FAILED) {
|
|
|
realProxy = null;
|
|
|
- return;
|
|
|
+ return FAILEDJOB;
|
|
|
}
|
|
|
|
|
|
- if (currentAppState == ApplicationState.SUCCEEDED
|
|
|
- || currentAppState == ApplicationState.FAILED
|
|
|
- || currentAppState == ApplicationState.KILLED) {
|
|
|
- serviceAddr = conf.get(JHConfig.HS_BIND_ADDRESS,
|
|
|
- JHConfig.DEFAULT_HS_BIND_ADDRESS);
|
|
|
+ if (application.getState() == ApplicationState.KILLED) {
|
|
|
+ realProxy = null;
|
|
|
+ return KILLEDJOB;
|
|
|
+ }
|
|
|
+
|
|
|
+ //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 " + serviceAddr);
|
|
|
- try {
|
|
|
- serviceHttpAddr = JobHistoryUtils.getHistoryUrl(conf, currentAppId);
|
|
|
- } catch (UnknownHostException e) {
|
|
|
- LOG.warn("Unable to get history url", e);
|
|
|
- serviceHttpAddr = "UNKNOWN";
|
|
|
- }
|
|
|
- try {
|
|
|
- instantiateHistoryProxy(serviceAddr);
|
|
|
- return;
|
|
|
- } catch (IOException e) {
|
|
|
- throw new YarnException(e);
|
|
|
- }
|
|
|
+ realProxy = historyServerProxy;
|
|
|
}
|
|
|
+ return realProxy;
|
|
|
}
|
|
|
|
|
|
private void instantiateAMProxy(final String serviceAddr) throws IOException {
|
|
@@ -198,129 +198,70 @@ public class ClientServiceDelegate {
|
|
|
LOG.trace("Connected to ApplicationMaster at: " + serviceAddr);
|
|
|
}
|
|
|
|
|
|
- private void instantiateHistoryProxy(final String serviceAddr)
|
|
|
- throws IOException {
|
|
|
- LOG.trace("Connecting to HistoryServer at: " + serviceAddr);
|
|
|
- Configuration myConf = new Configuration(conf);
|
|
|
- //TODO This should ideally be using it's own class (instead of ClientRMSecurityInfo)
|
|
|
- myConf.setClass(YarnConfiguration.YARN_SECURITY_INFO,
|
|
|
- ClientRMSecurityInfo.class, SecurityInfo.class);
|
|
|
- YarnRPC rpc = YarnRPC.create(myConf);
|
|
|
- realProxy = (MRClientProtocol) rpc.getProxy(MRClientProtocol.class,
|
|
|
- NetUtils.createSocketAddr(serviceAddr), myConf);
|
|
|
- LOG.trace("Connected to HistoryServer at: " + serviceAddr);
|
|
|
- }
|
|
|
-
|
|
|
- public org.apache.hadoop.mapreduce.Counters getJobCounters(JobID arg0) throws IOException,
|
|
|
- InterruptedException {
|
|
|
- org.apache.hadoop.mapreduce.v2.api.records.JobId jobID = TypeConverter.toYarn(arg0);
|
|
|
+ private synchronized Object invoke(String method, Class argClass,
|
|
|
+ Object args) throws YarnRemoteException {
|
|
|
+ Method methodOb = null;
|
|
|
try {
|
|
|
- GetCountersRequest request = recordFactory.newRecordInstance(GetCountersRequest.class);
|
|
|
- request.setJobId(jobID);
|
|
|
- MRClientProtocol protocol = getProxy(arg0);
|
|
|
- if (protocol == null) {
|
|
|
- /* no AM to connect to, fake counters */
|
|
|
- return new org.apache.hadoop.mapreduce.Counters();
|
|
|
- }
|
|
|
- return TypeConverter.fromYarn(protocol.getCounters(request).getCounters());
|
|
|
- } catch(YarnRemoteException yre) {//thrown by remote server, no need to redirect
|
|
|
- LOG.warn(RPCUtil.toString(yre));
|
|
|
- throw yre;
|
|
|
- } catch(Exception e) {
|
|
|
- LOG.debug("Failing to contact application master", e);
|
|
|
+ methodOb = MRClientProtocol.class.getMethod(method, argClass);
|
|
|
+ } catch (SecurityException e) {
|
|
|
+ throw new YarnException(e);
|
|
|
+ } catch (NoSuchMethodException e) {
|
|
|
+ throw new YarnException("Method name mismatch", e);
|
|
|
+ }
|
|
|
+ while (true) {
|
|
|
try {
|
|
|
- GetCountersRequest request = recordFactory.newRecordInstance(GetCountersRequest.class);
|
|
|
- request.setJobId(jobID);
|
|
|
- MRClientProtocol protocol = getRefreshedProxy(arg0);
|
|
|
- if (protocol == null) {
|
|
|
- /* no History to connect to, fake counters */
|
|
|
- return new org.apache.hadoop.mapreduce.Counters();
|
|
|
- }
|
|
|
- return TypeConverter.fromYarn(protocol.getCounters(request).getCounters());
|
|
|
- } catch(YarnRemoteException yre) {
|
|
|
+ return methodOb.invoke(getProxy(), args);
|
|
|
+ } catch (YarnRemoteException yre) {
|
|
|
+ LOG.warn("Exception thrown by remote end.");
|
|
|
LOG.warn(RPCUtil.toString(yre));
|
|
|
throw yre;
|
|
|
+ } catch (Exception e) {
|
|
|
+ LOG.info("Failed to contact AM for job " + jobId + " Will retry..");
|
|
|
+ LOG.debug("Failing to contact application master", e);
|
|
|
+ forceRefresh = true;
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- public String getJobHistoryDir() throws IOException, InterruptedException {
|
|
|
- return JobHistoryUtils.getConfiguredHistoryServerDoneDirPrefix(conf);
|
|
|
- }
|
|
|
-
|
|
|
- public TaskCompletionEvent[] getTaskCompletionEvents(JobID arg0, int arg1,
|
|
|
- int arg2) throws IOException, InterruptedException {
|
|
|
+ org.apache.hadoop.mapreduce.Counters getJobCounters(JobID arg0) throws IOException,
|
|
|
+ InterruptedException {
|
|
|
org.apache.hadoop.mapreduce.v2.api.records.JobId jobID = TypeConverter.toYarn(arg0);
|
|
|
- List<org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptCompletionEvent> list = null;
|
|
|
- GetTaskAttemptCompletionEventsRequest request = recordFactory.newRecordInstance(GetTaskAttemptCompletionEventsRequest.class);
|
|
|
- MRClientProtocol protocol;
|
|
|
- try {
|
|
|
+ GetCountersRequest request = recordFactory.newRecordInstance(GetCountersRequest.class);
|
|
|
request.setJobId(jobID);
|
|
|
- request.setFromEventId(arg1);
|
|
|
- request.setMaxEvents(arg2);
|
|
|
- protocol = getProxy(arg0);
|
|
|
- /** This is hack to get around the issue of faking jobstatus while the AM
|
|
|
- * is coming up.
|
|
|
- */
|
|
|
- if (protocol == null) {
|
|
|
- return new TaskCompletionEvent[0];
|
|
|
- }
|
|
|
- list = getProxy(arg0).getTaskAttemptCompletionEvents(request).getCompletionEventList();
|
|
|
- } catch(YarnRemoteException yre) {//thrown by remote server, no need to redirect
|
|
|
- LOG.warn(RPCUtil.toString(yre));
|
|
|
- throw yre;
|
|
|
- } catch(Exception e) {
|
|
|
- LOG.debug("Failed to contact application master ", e);
|
|
|
- try {
|
|
|
- request.setJobId(jobID);
|
|
|
- request.setFromEventId(arg1);
|
|
|
- request.setMaxEvents(arg2);
|
|
|
- protocol = getRefreshedProxy(arg0);
|
|
|
- if (protocol == null) {
|
|
|
- return new TaskCompletionEvent[0];
|
|
|
- }
|
|
|
- list = protocol.getTaskAttemptCompletionEvents(request).getCompletionEventList();
|
|
|
- } catch(YarnRemoteException yre) {
|
|
|
- LOG.warn(RPCUtil.toString(yre));
|
|
|
- throw yre;
|
|
|
- }
|
|
|
- }
|
|
|
- return TypeConverter.fromYarn(
|
|
|
- list.toArray(new org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptCompletionEvent[0]));
|
|
|
+ Counters cnt = ((GetCountersResponse)
|
|
|
+ invoke("getCounters", GetCountersRequest.class, request)).getCounters();
|
|
|
+ return TypeConverter.fromYarn(cnt);
|
|
|
+
|
|
|
}
|
|
|
|
|
|
- public String[] getTaskDiagnostics(org.apache.hadoop.mapreduce.TaskAttemptID
|
|
|
- arg0)
|
|
|
- throws IOException,
|
|
|
- InterruptedException {
|
|
|
+ TaskCompletionEvent[] getTaskCompletionEvents(JobID arg0, int arg1, int arg2)
|
|
|
+ throws IOException, InterruptedException {
|
|
|
+ org.apache.hadoop.mapreduce.v2.api.records.JobId jobID = TypeConverter
|
|
|
+ .toYarn(arg0);
|
|
|
+ GetTaskAttemptCompletionEventsRequest request = recordFactory
|
|
|
+ .newRecordInstance(GetTaskAttemptCompletionEventsRequest.class);
|
|
|
+ request.setJobId(jobID);
|
|
|
+ request.setFromEventId(arg1);
|
|
|
+ request.setMaxEvents(arg2);
|
|
|
+ List<org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptCompletionEvent> list =
|
|
|
+ ((GetTaskAttemptCompletionEventsResponse) invoke(
|
|
|
+ "getTaskAttemptCompletionEvents", GetTaskAttemptCompletionEventsRequest.class, request)).
|
|
|
+ getCompletionEventList();
|
|
|
+ return TypeConverter
|
|
|
+ .fromYarn(list
|
|
|
+ .toArray(new org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptCompletionEvent[0]));
|
|
|
+ }
|
|
|
|
|
|
- List<String> list = null;
|
|
|
- org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId attemptID = TypeConverter.toYarn(arg0);
|
|
|
- GetDiagnosticsRequest request = recordFactory.newRecordInstance(GetDiagnosticsRequest.class);
|
|
|
- MRClientProtocol protocol;
|
|
|
- try {
|
|
|
- request.setTaskAttemptId(attemptID);
|
|
|
- protocol = getProxy(arg0.getJobID());
|
|
|
- if (protocol == null) {
|
|
|
- return new String[0];
|
|
|
- }
|
|
|
- list = getProxy(arg0.getJobID()).getDiagnostics(request).getDiagnosticsList();
|
|
|
- } catch(YarnRemoteException yre) {//thrown by remote server, no need to redirect
|
|
|
- LOG.warn(RPCUtil.toString(yre));
|
|
|
- throw yre;
|
|
|
- } catch(Exception e) {
|
|
|
- LOG.debug("Failed to contact application master ", e);
|
|
|
- try {
|
|
|
- protocol = getRefreshedProxy(arg0.getJobID());
|
|
|
- if (protocol == null) {
|
|
|
- return new String[0];
|
|
|
- }
|
|
|
- list = protocol.getDiagnostics(request).getDiagnosticsList();
|
|
|
- } catch(YarnRemoteException yre) {
|
|
|
- LOG.warn(RPCUtil.toString(yre));
|
|
|
- throw yre;
|
|
|
- }
|
|
|
- }
|
|
|
+ String[] getTaskDiagnostics(org.apache.hadoop.mapreduce.TaskAttemptID arg0)
|
|
|
+ throws IOException, InterruptedException {
|
|
|
+
|
|
|
+ org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId attemptID = TypeConverter
|
|
|
+ .toYarn(arg0);
|
|
|
+ GetDiagnosticsRequest request = recordFactory
|
|
|
+ .newRecordInstance(GetDiagnosticsRequest.class);
|
|
|
+ request.setTaskAttemptId(attemptID);
|
|
|
+ List<String> list = ((GetDiagnosticsResponse) invoke("getDiagnostics",
|
|
|
+ GetDiagnosticsRequest.class, request)).getDiagnosticsList();
|
|
|
String[] result = new String[list.size()];
|
|
|
int i = 0;
|
|
|
for (String c : list) {
|
|
@@ -329,180 +270,59 @@ public class ClientServiceDelegate {
|
|
|
return result;
|
|
|
}
|
|
|
|
|
|
- private JobStatus createFakeJobReport(ApplicationState state,
|
|
|
- org.apache.hadoop.mapreduce.v2.api.records.JobId jobId, String jobFile) {
|
|
|
- JobReport jobreport = recordFactory.newRecordInstance(JobReport.class);
|
|
|
- jobreport.setCleanupProgress(0);
|
|
|
- jobreport.setFinishTime(0);
|
|
|
- jobreport.setJobId(jobId);
|
|
|
- jobreport.setMapProgress(0);
|
|
|
- /** fix this, the start time should be fixed */
|
|
|
- jobreport.setStartTime(0);
|
|
|
- jobreport.setReduceProgress(0);
|
|
|
- jobreport.setSetupProgress(0);
|
|
|
-
|
|
|
- if (currentAppState == ApplicationState.NEW) {
|
|
|
- /* the protocol wasnt instantiated because the applicaton wasnt launched
|
|
|
- * return a fake report.
|
|
|
- */
|
|
|
- jobreport.setJobState(JobState.NEW);
|
|
|
- } else if (currentAppState == ApplicationState.SUBMITTED) {
|
|
|
- jobreport.setJobState(JobState.NEW);
|
|
|
- } else if (currentAppState == ApplicationState.KILLED) {
|
|
|
- jobreport.setJobState(JobState.KILLED);
|
|
|
- } else if (currentAppState == ApplicationState.FAILED) {
|
|
|
- jobreport.setJobState(JobState.FAILED);
|
|
|
- }
|
|
|
- return TypeConverter.fromYarn(jobreport, jobFile, serviceHttpAddr);
|
|
|
- }
|
|
|
-
|
|
|
- public JobStatus getJobStatus(JobID oldJobID) throws YarnRemoteException,
|
|
|
- YarnRemoteException {
|
|
|
+ JobStatus getJobStatus(JobID oldJobID) throws YarnRemoteException,
|
|
|
+ YarnRemoteException {
|
|
|
org.apache.hadoop.mapreduce.v2.api.records.JobId jobId =
|
|
|
TypeConverter.toYarn(oldJobID);
|
|
|
String stagingDir = conf.get("yarn.apps.stagingDir");
|
|
|
String jobFile = stagingDir + "/" + jobId.toString();
|
|
|
- JobReport report = null;
|
|
|
MRClientProtocol protocol;
|
|
|
GetJobReportRequest request = recordFactory.newRecordInstance(GetJobReportRequest.class);
|
|
|
- try {
|
|
|
- request.setJobId(jobId);
|
|
|
- protocol = getProxy(oldJobID);
|
|
|
-
|
|
|
- if (protocol == null) {
|
|
|
- return createFakeJobReport(currentAppState, jobId, jobFile);
|
|
|
- }
|
|
|
- report = getProxy(oldJobID).getJobReport(request).getJobReport();
|
|
|
- } catch(YarnRemoteException yre) {//thrown by remote server, no need to redirect
|
|
|
- LOG.warn(RPCUtil.toString(yre));
|
|
|
- throw yre;
|
|
|
- } catch (Exception e) {
|
|
|
- try {
|
|
|
- request.setJobId(jobId);
|
|
|
- protocol = getRefreshedProxy(oldJobID);
|
|
|
- /* this is possible if an application that was running is killed */
|
|
|
- if (protocol == null) {
|
|
|
- return createFakeJobReport(currentAppState, jobId, jobFile);
|
|
|
- }
|
|
|
- report = protocol.getJobReport(request).getJobReport();
|
|
|
- } catch(YarnRemoteException yre) {
|
|
|
- LOG.warn(RPCUtil.toString(yre));
|
|
|
- throw yre;
|
|
|
- }
|
|
|
- }
|
|
|
- return TypeConverter.fromYarn(report, jobFile, serviceHttpAddr);
|
|
|
+ request.setJobId(jobId);
|
|
|
+ JobReport report = ((GetJobReportResponse) invoke("getJobReport",
|
|
|
+ GetJobReportRequest.class, request)).getJobReport();
|
|
|
+ //TODO: add tracking url in JobReport
|
|
|
+ return TypeConverter.fromYarn(report, jobFile, "");
|
|
|
}
|
|
|
|
|
|
- public org.apache.hadoop.mapreduce.TaskReport[] getTaskReports(JobID jobID, TaskType taskType)
|
|
|
- throws YarnRemoteException, YarnRemoteException {
|
|
|
- List<org.apache.hadoop.mapreduce.v2.api.records.TaskReport> taskReports = null;
|
|
|
+ org.apache.hadoop.mapreduce.TaskReport[] getTaskReports(JobID jobID, TaskType taskType)
|
|
|
+ throws YarnRemoteException, YarnRemoteException {
|
|
|
org.apache.hadoop.mapreduce.v2.api.records.JobId nJobID = TypeConverter.toYarn(jobID);
|
|
|
GetTaskReportsRequest request = recordFactory.newRecordInstance(GetTaskReportsRequest.class);
|
|
|
- MRClientProtocol protocol = null;
|
|
|
- try {
|
|
|
- request.setJobId(nJobID);
|
|
|
- request.setTaskType(TypeConverter.toYarn(taskType));
|
|
|
- protocol = getProxy(jobID);
|
|
|
- if (protocol == null) {
|
|
|
- return new org.apache.hadoop.mapreduce.TaskReport[0];
|
|
|
- }
|
|
|
- taskReports = getProxy(jobID).getTaskReports(request).getTaskReportList();
|
|
|
- } catch(YarnRemoteException yre) {//thrown by remote server, no need to redirect
|
|
|
- LOG.warn(RPCUtil.toString(yre));
|
|
|
- throw yre;
|
|
|
- } catch(Exception e) {
|
|
|
- LOG.debug("Failed to contact application master ", e);
|
|
|
- try {
|
|
|
- request.setJobId(nJobID);
|
|
|
- request.setTaskType(TypeConverter.toYarn(taskType));
|
|
|
- protocol = getRefreshedProxy(jobID);
|
|
|
- if (protocol == null) {
|
|
|
- return new org.apache.hadoop.mapreduce.TaskReport[0];
|
|
|
- }
|
|
|
- taskReports = protocol.getTaskReports(request).getTaskReportList();
|
|
|
- } catch(YarnRemoteException yre) {
|
|
|
- LOG.warn(RPCUtil.toString(yre));
|
|
|
- throw yre;
|
|
|
- }
|
|
|
- }
|
|
|
+
|
|
|
+ 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 {
|
|
|
+ boolean killTask(TaskAttemptID taskAttemptID, boolean fail)
|
|
|
+ throws YarnRemoteException {
|
|
|
org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId attemptID
|
|
|
- = TypeConverter.toYarn(taskAttemptID);
|
|
|
- KillTaskAttemptRequest killRequest = recordFactory.newRecordInstance(KillTaskAttemptRequest.class);
|
|
|
- FailTaskAttemptRequest failRequest = recordFactory.newRecordInstance(FailTaskAttemptRequest.class);
|
|
|
- MRClientProtocol protocol = getProxy(taskAttemptID.getJobID());
|
|
|
- if (protocol == null) {
|
|
|
- return false;
|
|
|
- }
|
|
|
- try {
|
|
|
- if (fail) {
|
|
|
- failRequest.setTaskAttemptId(attemptID);
|
|
|
- getProxy(taskAttemptID.getJobID()).failTaskAttempt(failRequest);
|
|
|
- } else {
|
|
|
- killRequest.setTaskAttemptId(attemptID);
|
|
|
- getProxy(taskAttemptID.getJobID()).killTaskAttempt(killRequest);
|
|
|
- }
|
|
|
- } catch(YarnRemoteException yre) {//thrown by remote server, no need to redirect
|
|
|
- LOG.warn(RPCUtil.toString(yre));
|
|
|
- throw yre;
|
|
|
- } catch(Exception e) {
|
|
|
- LOG.debug("Failed to contact application master ", e);
|
|
|
- MRClientProtocol proxy = getRefreshedProxy(taskAttemptID.getJobID());
|
|
|
- if (proxy == null) {
|
|
|
- return false;
|
|
|
- }
|
|
|
- try {
|
|
|
- if (fail) {
|
|
|
- failRequest.setTaskAttemptId(attemptID);
|
|
|
- proxy.failTaskAttempt(failRequest);
|
|
|
- } else {
|
|
|
- killRequest.setTaskAttemptId(attemptID);
|
|
|
- proxy.killTaskAttempt(killRequest);
|
|
|
- }
|
|
|
- } catch(YarnRemoteException yre) {
|
|
|
- LOG.warn(RPCUtil.toString(yre));
|
|
|
- throw yre;
|
|
|
- }
|
|
|
+ = TypeConverter.toYarn(taskAttemptID);
|
|
|
+ if (fail) {
|
|
|
+ FailTaskAttemptRequest failRequest = recordFactory.newRecordInstance(FailTaskAttemptRequest.class);
|
|
|
+ failRequest.setTaskAttemptId(attemptID);
|
|
|
+ invoke("failTaskAttempt", FailTaskAttemptRequest.class, failRequest);
|
|
|
+ } else {
|
|
|
+ KillTaskAttemptRequest killRequest = recordFactory.newRecordInstance(KillTaskAttemptRequest.class);
|
|
|
+ killRequest.setTaskAttemptId(attemptID);
|
|
|
+ invoke("killTaskAttempt", KillTaskAttemptRequest.class, killRequest);
|
|
|
}
|
|
|
return true;
|
|
|
}
|
|
|
|
|
|
- public boolean killJob(JobID oldJobID)
|
|
|
- throws YarnRemoteException {
|
|
|
+ boolean killJob(JobID oldJobID)
|
|
|
+ throws YarnRemoteException {
|
|
|
org.apache.hadoop.mapreduce.v2.api.records.JobId jobId
|
|
|
= TypeConverter.toYarn(oldJobID);
|
|
|
KillJobRequest killRequest = recordFactory.newRecordInstance(KillJobRequest.class);
|
|
|
- MRClientProtocol protocol = getProxy(oldJobID);
|
|
|
- if (protocol == null) {
|
|
|
- return false;
|
|
|
- }
|
|
|
- try {
|
|
|
- killRequest.setJobId(jobId);
|
|
|
- protocol.killJob(killRequest);
|
|
|
- return true;
|
|
|
- } catch(YarnRemoteException yre) {//thrown by remote server, no need to redirect
|
|
|
- LOG.warn(RPCUtil.toString(yre));
|
|
|
- throw yre;
|
|
|
- } catch(Exception e) {
|
|
|
- // Not really requied - if this is always the history context.
|
|
|
- LOG.debug("Failed to contact application master ", e);
|
|
|
- MRClientProtocol proxy = getRefreshedProxy(oldJobID);
|
|
|
- if (proxy == null) {
|
|
|
- return false;
|
|
|
- }
|
|
|
- try {
|
|
|
- killRequest.setJobId(jobId);
|
|
|
- protocol.killJob(killRequest);
|
|
|
- return true;
|
|
|
- } catch(YarnRemoteException yre) {
|
|
|
- LOG.warn(RPCUtil.toString(yre));
|
|
|
- throw yre;
|
|
|
- }
|
|
|
- }
|
|
|
+ killRequest.setJobId(jobId);
|
|
|
+ invoke("killJob", KillJobRequest.class, killRequest);
|
|
|
+ return true;
|
|
|
}
|
|
|
+
|
|
|
+
|
|
|
}
|