|
@@ -28,7 +28,6 @@ import org.apache.commons.logging.LogFactory;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
|
|
|
import org.apache.hadoop.ipc.Server;
|
|
|
-import org.apache.hadoop.mapreduce.JobACL;
|
|
|
import org.apache.hadoop.mapreduce.MRJobConfig;
|
|
|
import org.apache.hadoop.mapreduce.TypeConverter;
|
|
|
import org.apache.hadoop.mapreduce.v2.api.MRClientProtocol;
|
|
@@ -79,8 +78,6 @@ import org.apache.hadoop.mapreduce.v2.app.job.event.TaskEventType;
|
|
|
import org.apache.hadoop.mapreduce.v2.app.security.authorize.MRAMPolicyProvider;
|
|
|
import org.apache.hadoop.mapreduce.v2.app.webapp.AMWebApp;
|
|
|
import org.apache.hadoop.net.NetUtils;
|
|
|
-import org.apache.hadoop.security.AccessControlException;
|
|
|
-import org.apache.hadoop.security.UserGroupInformation;
|
|
|
import org.apache.hadoop.security.authorize.PolicyProvider;
|
|
|
import org.apache.hadoop.service.AbstractService;
|
|
|
import org.apache.hadoop.yarn.factories.RecordFactory;
|
|
@@ -178,22 +175,16 @@ public class MRClientService extends AbstractService
|
|
|
return getBindAddress();
|
|
|
}
|
|
|
|
|
|
- private Job verifyAndGetJob(JobId jobID,
|
|
|
- JobACL accessType) throws IOException {
|
|
|
+ private Job verifyAndGetJob(JobId jobID,
|
|
|
+ boolean modifyAccess) throws IOException {
|
|
|
Job job = appContext.getJob(jobID);
|
|
|
- UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
|
|
|
- if (!job.checkAccess(ugi, accessType)) {
|
|
|
- throw new AccessControlException("User " + ugi.getShortUserName()
|
|
|
- + " cannot perform operation " + accessType.name() + " on "
|
|
|
- + jobID);
|
|
|
- }
|
|
|
return job;
|
|
|
}
|
|
|
|
|
|
private Task verifyAndGetTask(TaskId taskID,
|
|
|
- JobACL accessType) throws IOException {
|
|
|
+ boolean modifyAccess) throws IOException {
|
|
|
Task task = verifyAndGetJob(taskID.getJobId(),
|
|
|
- accessType).getTask(taskID);
|
|
|
+ modifyAccess).getTask(taskID);
|
|
|
if (task == null) {
|
|
|
throw new IOException("Unknown Task " + taskID);
|
|
|
}
|
|
@@ -201,9 +192,9 @@ public class MRClientService extends AbstractService
|
|
|
}
|
|
|
|
|
|
private TaskAttempt verifyAndGetAttempt(TaskAttemptId attemptID,
|
|
|
- JobACL accessType) throws IOException {
|
|
|
+ boolean modifyAccess) throws IOException {
|
|
|
TaskAttempt attempt = verifyAndGetTask(attemptID.getTaskId(),
|
|
|
- accessType).getAttempt(attemptID);
|
|
|
+ modifyAccess).getAttempt(attemptID);
|
|
|
if (attempt == null) {
|
|
|
throw new IOException("Unknown TaskAttempt " + attemptID);
|
|
|
}
|
|
@@ -214,7 +205,7 @@ public class MRClientService extends AbstractService
|
|
|
public GetCountersResponse getCounters(GetCountersRequest request)
|
|
|
throws IOException {
|
|
|
JobId jobId = request.getJobId();
|
|
|
- Job job = verifyAndGetJob(jobId, JobACL.VIEW_JOB);
|
|
|
+ Job job = verifyAndGetJob(jobId, false);
|
|
|
GetCountersResponse response =
|
|
|
recordFactory.newRecordInstance(GetCountersResponse.class);
|
|
|
response.setCounters(TypeConverter.toYarn(job.getAllCounters()));
|
|
@@ -225,7 +216,7 @@ public class MRClientService extends AbstractService
|
|
|
public GetJobReportResponse getJobReport(GetJobReportRequest request)
|
|
|
throws IOException {
|
|
|
JobId jobId = request.getJobId();
|
|
|
- Job job = verifyAndGetJob(jobId, JobACL.VIEW_JOB);
|
|
|
+ Job job = verifyAndGetJob(jobId, false);
|
|
|
GetJobReportResponse response =
|
|
|
recordFactory.newRecordInstance(GetJobReportResponse.class);
|
|
|
if (job != null) {
|
|
@@ -244,7 +235,7 @@ public class MRClientService extends AbstractService
|
|
|
GetTaskAttemptReportResponse response =
|
|
|
recordFactory.newRecordInstance(GetTaskAttemptReportResponse.class);
|
|
|
response.setTaskAttemptReport(
|
|
|
- verifyAndGetAttempt(taskAttemptId, JobACL.VIEW_JOB).getReport());
|
|
|
+ verifyAndGetAttempt(taskAttemptId, false).getReport());
|
|
|
return response;
|
|
|
}
|
|
|
|
|
@@ -254,8 +245,7 @@ public class MRClientService extends AbstractService
|
|
|
TaskId taskId = request.getTaskId();
|
|
|
GetTaskReportResponse response =
|
|
|
recordFactory.newRecordInstance(GetTaskReportResponse.class);
|
|
|
- response.setTaskReport(
|
|
|
- verifyAndGetTask(taskId, JobACL.VIEW_JOB).getReport());
|
|
|
+ response.setTaskReport(verifyAndGetTask(taskId, false).getReport());
|
|
|
return response;
|
|
|
}
|
|
|
|
|
@@ -266,7 +256,7 @@ public class MRClientService extends AbstractService
|
|
|
JobId jobId = request.getJobId();
|
|
|
int fromEventId = request.getFromEventId();
|
|
|
int maxEvents = request.getMaxEvents();
|
|
|
- Job job = verifyAndGetJob(jobId, JobACL.VIEW_JOB);
|
|
|
+ Job job = verifyAndGetJob(jobId, false);
|
|
|
|
|
|
GetTaskAttemptCompletionEventsResponse response =
|
|
|
recordFactory.newRecordInstance(GetTaskAttemptCompletionEventsResponse.class);
|
|
@@ -280,11 +270,9 @@ public class MRClientService extends AbstractService
|
|
|
public KillJobResponse killJob(KillJobRequest request)
|
|
|
throws IOException {
|
|
|
JobId jobId = request.getJobId();
|
|
|
- UserGroupInformation callerUGI = UserGroupInformation.getCurrentUser();
|
|
|
- String message = "Kill job " + jobId + " received from " + callerUGI
|
|
|
- + " at " + Server.getRemoteAddress();
|
|
|
+ String message = "Kill Job received from client " + jobId;
|
|
|
LOG.info(message);
|
|
|
- verifyAndGetJob(jobId, JobACL.MODIFY_JOB);
|
|
|
+ verifyAndGetJob(jobId, true);
|
|
|
appContext.getEventHandler().handle(
|
|
|
new JobDiagnosticsUpdateEvent(jobId, message));
|
|
|
appContext.getEventHandler().handle(
|
|
@@ -299,11 +287,9 @@ public class MRClientService extends AbstractService
|
|
|
public KillTaskResponse killTask(KillTaskRequest request)
|
|
|
throws IOException {
|
|
|
TaskId taskId = request.getTaskId();
|
|
|
- UserGroupInformation callerUGI = UserGroupInformation.getCurrentUser();
|
|
|
- String message = "Kill task " + taskId + " received from " + callerUGI
|
|
|
- + " at " + Server.getRemoteAddress();
|
|
|
+ String message = "Kill task received from client " + taskId;
|
|
|
LOG.info(message);
|
|
|
- verifyAndGetTask(taskId, JobACL.MODIFY_JOB);
|
|
|
+ verifyAndGetTask(taskId, true);
|
|
|
appContext.getEventHandler().handle(
|
|
|
new TaskEvent(taskId, TaskEventType.T_KILL));
|
|
|
KillTaskResponse response =
|
|
@@ -316,12 +302,9 @@ public class MRClientService extends AbstractService
|
|
|
public KillTaskAttemptResponse killTaskAttempt(
|
|
|
KillTaskAttemptRequest request) throws IOException {
|
|
|
TaskAttemptId taskAttemptId = request.getTaskAttemptId();
|
|
|
- UserGroupInformation callerUGI = UserGroupInformation.getCurrentUser();
|
|
|
- String message = "Kill task attempt " + taskAttemptId
|
|
|
- + " received from " + callerUGI + " at "
|
|
|
- + Server.getRemoteAddress();
|
|
|
+ String message = "Kill task attempt received from client " + taskAttemptId;
|
|
|
LOG.info(message);
|
|
|
- verifyAndGetAttempt(taskAttemptId, JobACL.MODIFY_JOB);
|
|
|
+ verifyAndGetAttempt(taskAttemptId, true);
|
|
|
appContext.getEventHandler().handle(
|
|
|
new TaskAttemptDiagnosticsUpdateEvent(taskAttemptId, message));
|
|
|
appContext.getEventHandler().handle(
|
|
@@ -339,8 +322,8 @@ public class MRClientService extends AbstractService
|
|
|
|
|
|
GetDiagnosticsResponse response =
|
|
|
recordFactory.newRecordInstance(GetDiagnosticsResponse.class);
|
|
|
- response.addAllDiagnostics(verifyAndGetAttempt(taskAttemptId,
|
|
|
- JobACL.VIEW_JOB).getDiagnostics());
|
|
|
+ response.addAllDiagnostics(
|
|
|
+ verifyAndGetAttempt(taskAttemptId, false).getDiagnostics());
|
|
|
return response;
|
|
|
}
|
|
|
|
|
@@ -349,12 +332,9 @@ public class MRClientService extends AbstractService
|
|
|
public FailTaskAttemptResponse failTaskAttempt(
|
|
|
FailTaskAttemptRequest request) throws IOException {
|
|
|
TaskAttemptId taskAttemptId = request.getTaskAttemptId();
|
|
|
- UserGroupInformation callerUGI = UserGroupInformation.getCurrentUser();
|
|
|
- String message = "Fail task attempt " + taskAttemptId
|
|
|
- + " received from " + callerUGI + " at "
|
|
|
- + Server.getRemoteAddress();
|
|
|
+ String message = "Fail task attempt received from client " + taskAttemptId;
|
|
|
LOG.info(message);
|
|
|
- verifyAndGetAttempt(taskAttemptId, JobACL.MODIFY_JOB);
|
|
|
+ verifyAndGetAttempt(taskAttemptId, true);
|
|
|
appContext.getEventHandler().handle(
|
|
|
new TaskAttemptDiagnosticsUpdateEvent(taskAttemptId, message));
|
|
|
appContext.getEventHandler().handle(
|
|
@@ -376,7 +356,7 @@ public class MRClientService extends AbstractService
|
|
|
GetTaskReportsResponse response =
|
|
|
recordFactory.newRecordInstance(GetTaskReportsResponse.class);
|
|
|
|
|
|
- Job job = verifyAndGetJob(jobId, JobACL.VIEW_JOB);
|
|
|
+ Job job = verifyAndGetJob(jobId, false);
|
|
|
Collection<Task> tasks = job.getTasks(taskType).values();
|
|
|
LOG.info("Getting task report for " + taskType + " " + jobId
|
|
|
+ ". Report-size will be " + tasks.size());
|