|
@@ -25,16 +25,23 @@ import java.util.Set;
|
|
|
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.mapred.JobConf;
|
|
|
+import org.apache.hadoop.mapred.JobStatus;
|
|
|
import org.apache.hadoop.mapred.JobTracker.RetireJobInfo;
|
|
|
+import org.apache.hadoop.mapred.Counters;
|
|
|
import org.apache.hadoop.mapreduce.JobID;
|
|
|
import org.apache.hadoop.mapreduce.TaskID;
|
|
|
+import org.apache.hadoop.mapreduce.TaskType;
|
|
|
import org.apache.hadoop.mapreduce.server.jobtracker.TaskTracker;
|
|
|
import org.apache.hadoop.mapreduce.test.system.JTProtocol;
|
|
|
import org.apache.hadoop.mapreduce.test.system.JobInfo;
|
|
|
import org.apache.hadoop.mapreduce.test.system.TTInfo;
|
|
|
import org.apache.hadoop.mapreduce.test.system.TaskInfo;
|
|
|
+import org.apache.hadoop.mapreduce.ClusterMetrics;
|
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
|
import org.apache.hadoop.test.system.DaemonProtocol;
|
|
|
+import org.apache.hadoop.util.Shell.ShellCommandExecutor;
|
|
|
+import org.apache.hadoop.util.Shell;
|
|
|
+import org.apache.hadoop.util.StringUtils;
|
|
|
|
|
|
/**
|
|
|
* Aspect class which injects the code for {@link JobTracker} class.
|
|
@@ -43,6 +50,8 @@ import org.apache.hadoop.test.system.DaemonProtocol;
|
|
|
public privileged aspect JobTrackerAspect {
|
|
|
|
|
|
|
|
|
+ private static JobTracker tracker;
|
|
|
+
|
|
|
public Configuration JobTracker.getDaemonConf() throws IOException {
|
|
|
return conf;
|
|
|
}
|
|
@@ -207,6 +216,7 @@ public privileged aspect JobTrackerAspect {
|
|
|
tracker.LOG.warn("Unable to get the user information for the " +
|
|
|
"Jobtracker");
|
|
|
}
|
|
|
+ this.tracker = tracker;
|
|
|
tracker.setReady(true);
|
|
|
}
|
|
|
|
|
@@ -230,4 +240,104 @@ public privileged aspect JobTrackerAspect {
|
|
|
.isJobCleanupTask()), trackers);
|
|
|
return info;
|
|
|
}
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Get the job summary details from the jobtracker log files.
|
|
|
+ * @param jobId - job id
|
|
|
+ * @param filePattern - jobtracker log file pattern.
|
|
|
+ * @return String - Job summary details of given job id.
|
|
|
+ * @throws IOException if any I/O error occurs.
|
|
|
+ */
|
|
|
+ public String JobTracker.getJobSummaryFromLogs(JobID jobId,
|
|
|
+ String filePattern) throws IOException {
|
|
|
+ String pattern = "JobId=" + jobId.toString() + ",submitTime";
|
|
|
+ String[] cmd = new String[] {
|
|
|
+ "bash",
|
|
|
+ "-c",
|
|
|
+ "grep -i "
|
|
|
+ + pattern + " "
|
|
|
+ + filePattern + " "
|
|
|
+ + "| sed s/'JobSummary: '/'^'/g | cut -d'^' -f2"};
|
|
|
+ ShellCommandExecutor shexec = new ShellCommandExecutor(cmd);
|
|
|
+ shexec.execute();
|
|
|
+ return shexec.getOutput();
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Get the job summary information for given job id.
|
|
|
+ * @param jobId - job id.
|
|
|
+ * @return String - Job summary details as key value pair.
|
|
|
+ * @throws IOException if any I/O error occurs.
|
|
|
+ */
|
|
|
+ public String JobTracker.getJobSummaryInfo(JobID jobId) throws IOException {
|
|
|
+ StringBuffer jobSummary = new StringBuffer();
|
|
|
+ JobInProgress jip = jobs.
|
|
|
+ get(org.apache.hadoop.mapred.JobID.downgrade(jobId));
|
|
|
+ if (jip == null) {
|
|
|
+ LOG.warn("Job has not been found - " + jobId);
|
|
|
+ return null;
|
|
|
+ }
|
|
|
+ JobProfile profile = jip.getProfile();
|
|
|
+ JobStatus status = jip.getStatus();
|
|
|
+ final char[] charsToEscape = {StringUtils.COMMA, '=',
|
|
|
+ StringUtils.ESCAPE_CHAR};
|
|
|
+ String user = StringUtils.escapeString(profile.getUser(),
|
|
|
+ StringUtils.ESCAPE_CHAR, charsToEscape);
|
|
|
+ String queue = StringUtils.escapeString(profile.getQueueName(),
|
|
|
+ StringUtils.ESCAPE_CHAR, charsToEscape);
|
|
|
+ Counters jobCounters = jip.getJobCounters();
|
|
|
+ long mapSlotSeconds = (jobCounters.getCounter(
|
|
|
+ JobInProgress.Counter.SLOTS_MILLIS_MAPS) +
|
|
|
+ jobCounters.getCounter(JobInProgress.
|
|
|
+ Counter.FALLOW_SLOTS_MILLIS_MAPS)) / 1000;
|
|
|
+ long reduceSlotSeconds = (jobCounters.getCounter(
|
|
|
+ JobInProgress.Counter.SLOTS_MILLIS_REDUCES) +
|
|
|
+ jobCounters.getCounter(JobInProgress.
|
|
|
+ Counter.FALLOW_SLOTS_MILLIS_REDUCES)) / 1000;
|
|
|
+ jobSummary.append("jobId=");
|
|
|
+ jobSummary.append(jip.getJobID());
|
|
|
+ jobSummary.append(",");
|
|
|
+ jobSummary.append("startTime=");
|
|
|
+ jobSummary.append(jip.getStartTime());
|
|
|
+ jobSummary.append(",");
|
|
|
+ jobSummary.append("launchTime=");
|
|
|
+ jobSummary.append(jip.getLaunchTime());
|
|
|
+ jobSummary.append(",");
|
|
|
+ jobSummary.append("finishTime=");
|
|
|
+ jobSummary.append(jip.getFinishTime());
|
|
|
+ jobSummary.append(",");
|
|
|
+ jobSummary.append("numMaps=");
|
|
|
+ jobSummary.append(jip.getTasks(TaskType.MAP).length);
|
|
|
+ jobSummary.append(",");
|
|
|
+ jobSummary.append("numSlotsPerMap=");
|
|
|
+ jobSummary.append(jip.getNumSlotsPerMap() );
|
|
|
+ jobSummary.append(",");
|
|
|
+ jobSummary.append("numReduces=");
|
|
|
+ jobSummary.append(jip.getTasks(TaskType.REDUCE).length);
|
|
|
+ jobSummary.append(",");
|
|
|
+ jobSummary.append("numSlotsPerReduce=");
|
|
|
+ jobSummary.append(jip.getNumSlotsPerReduce());
|
|
|
+ jobSummary.append(",");
|
|
|
+ jobSummary.append("user=");
|
|
|
+ jobSummary.append(user);
|
|
|
+ jobSummary.append(",");
|
|
|
+ jobSummary.append("queue=");
|
|
|
+ jobSummary.append(queue);
|
|
|
+ jobSummary.append(",");
|
|
|
+ jobSummary.append("status=");
|
|
|
+ jobSummary.append(JobStatus.getJobRunState(status.getRunState()));
|
|
|
+ jobSummary.append(",");
|
|
|
+ jobSummary.append("mapSlotSeconds=");
|
|
|
+ jobSummary.append(mapSlotSeconds);
|
|
|
+ jobSummary.append(",");
|
|
|
+ jobSummary.append("reduceSlotsSeconds=");
|
|
|
+ jobSummary.append(reduceSlotSeconds);
|
|
|
+ jobSummary.append(",");
|
|
|
+ jobSummary.append("clusterMapCapacity=");
|
|
|
+ jobSummary.append(tracker.getClusterMetrics().getMapSlotCapacity());
|
|
|
+ jobSummary.append(",");
|
|
|
+ jobSummary.append("clusterReduceCapacity=");
|
|
|
+ jobSummary.append(tracker.getClusterMetrics().getReduceSlotCapacity());
|
|
|
+ return jobSummary.toString();
|
|
|
+ }
|
|
|
}
|