|
@@ -91,6 +91,7 @@ public class JobInProgress {
|
|
JobStatus status;
|
|
JobStatus status;
|
|
String jobFile = null;
|
|
String jobFile = null;
|
|
Path localJobFile = null;
|
|
Path localJobFile = null;
|
|
|
|
+ final QueueMetrics queueMetrics;
|
|
|
|
|
|
TaskInProgress maps[] = new TaskInProgress[0];
|
|
TaskInProgress maps[] = new TaskInProgress[0];
|
|
TaskInProgress reduces[] = new TaskInProgress[0];
|
|
TaskInProgress reduces[] = new TaskInProgress[0];
|
|
@@ -339,8 +340,9 @@ public class JobInProgress {
|
|
this.resourceEstimator = new ResourceEstimator(this);
|
|
this.resourceEstimator = new ResourceEstimator(this);
|
|
this.status = new JobStatus(jobid, 0.0f, 0.0f, JobStatus.PREP);
|
|
this.status = new JobStatus(jobid, 0.0f, 0.0f, JobStatus.PREP);
|
|
this.status.setUsername(conf.getUser());
|
|
this.status.setUsername(conf.getUser());
|
|
|
|
+ String queueName = conf.getQueueName();
|
|
this.profile = new JobProfile(conf.getUser(), jobid, "", "",
|
|
this.profile = new JobProfile(conf.getUser(), jobid, "", "",
|
|
- conf.getJobName(), conf.getQueueName());
|
|
|
|
|
|
+ conf.getJobName(), queueName);
|
|
this.memoryPerMap = conf.getMemoryForMapTask();
|
|
this.memoryPerMap = conf.getMemoryForMapTask();
|
|
this.memoryPerReduce = conf.getMemoryForReduceTask();
|
|
this.memoryPerReduce = conf.getMemoryForReduceTask();
|
|
this.maxTaskFailuresPerTracker = conf.getMaxTaskFailuresPerTracker();
|
|
this.maxTaskFailuresPerTracker = conf.getMaxTaskFailuresPerTracker();
|
|
@@ -377,6 +379,7 @@ public class JobInProgress {
|
|
this.status = new JobStatus(jobId, 0.0f, 0.0f, JobStatus.PREP);
|
|
this.status = new JobStatus(jobId, 0.0f, 0.0f, JobStatus.PREP);
|
|
this.status.setUsername(jobInfo.getUser().toString());
|
|
this.status.setUsername(jobInfo.getUser().toString());
|
|
this.jobtracker.getInstrumentation().addPrepJob(conf, jobId);
|
|
this.jobtracker.getInstrumentation().addPrepJob(conf, jobId);
|
|
|
|
+ // Add the queue-level metric below (after the profile has been initialized)
|
|
this.startTime = jobtracker.getClock().getTime();
|
|
this.startTime = jobtracker.getClock().getTime();
|
|
status.setStartTime(startTime);
|
|
status.setStartTime(startTime);
|
|
this.localFs = jobtracker.getLocalFileSystem();
|
|
this.localFs = jobtracker.getLocalFileSystem();
|
|
@@ -424,9 +427,9 @@ public class JobInProgress {
|
|
|
|
|
|
this.priority = conf.getJobPriority();
|
|
this.priority = conf.getJobPriority();
|
|
this.status.setJobPriority(this.priority);
|
|
this.status.setJobPriority(this.priority);
|
|
|
|
+ String queueName = conf.getQueueName();
|
|
this.profile = new JobProfile(user, jobId,
|
|
this.profile = new JobProfile(user, jobId,
|
|
- jobFile, url, conf.getJobName(),
|
|
|
|
- conf.getQueueName());
|
|
|
|
|
|
+ jobFile, url, conf.getJobName(), queueName);
|
|
|
|
|
|
Queue queue = this.jobtracker.getQueueManager().getQueue(queueName);
|
|
Queue queue = this.jobtracker.getQueueManager().getQueue(queueName);
|
|
if (queue == null) {
|
|
if (queue == null) {
|
|
@@ -487,7 +490,15 @@ public class JobInProgress {
|
|
FileSystem.closeAllForUGI(UserGroupInformation.getCurrentUser());
|
|
FileSystem.closeAllForUGI(UserGroupInformation.getCurrentUser());
|
|
}
|
|
}
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Get the QueueMetrics object associated with this job
|
|
|
|
+ * @return QueueMetrics
|
|
|
|
+ */
|
|
|
|
+ public QueueMetrics getQueueMetrics() {
|
|
|
|
+ return this.queueMetrics;
|
|
|
|
+ }
|
|
|
|
+
|
|
private void checkTaskLimits() throws IOException {
|
|
private void checkTaskLimits() throws IOException {
|
|
// if the number of tasks is larger than a configured value
|
|
// if the number of tasks is larger than a configured value
|
|
// then fail the job.
|
|
// then fail the job.
|
|
@@ -695,6 +706,8 @@ public class JobInProgress {
|
|
|
|
|
|
jobtracker.getInstrumentation().addWaitingMaps(getJobID(), numMapTasks);
|
|
jobtracker.getInstrumentation().addWaitingMaps(getJobID(), numMapTasks);
|
|
jobtracker.getInstrumentation().addWaitingReduces(getJobID(), numReduceTasks);
|
|
jobtracker.getInstrumentation().addWaitingReduces(getJobID(), numReduceTasks);
|
|
|
|
+ this.queueMetrics.addWaitingMaps(getJobID(), numMapTasks);
|
|
|
|
+ this.queueMetrics.addWaitingReduces(getJobID(), numReduceTasks);
|
|
|
|
|
|
maps = new TaskInProgress[numMapTasks];
|
|
maps = new TaskInProgress[numMapTasks];
|
|
for(int i=0; i < numMapTasks; ++i) {
|
|
for(int i=0; i < numMapTasks; ++i) {
|
|
@@ -1695,6 +1708,7 @@ public class JobInProgress {
|
|
if (tip.getActiveTasks().size() > 1)
|
|
if (tip.getActiveTasks().size() > 1)
|
|
speculativeMapTasks++;
|
|
speculativeMapTasks++;
|
|
metrics.launchMap(id);
|
|
metrics.launchMap(id);
|
|
|
|
+ this.queueMetrics.launchMap(id);
|
|
} else {
|
|
} else {
|
|
++runningReduceTasks;
|
|
++runningReduceTasks;
|
|
name = Values.REDUCE.name();
|
|
name = Values.REDUCE.name();
|
|
@@ -1702,6 +1716,7 @@ public class JobInProgress {
|
|
if (tip.getActiveTasks().size() > 1)
|
|
if (tip.getActiveTasks().size() > 1)
|
|
speculativeReduceTasks++;
|
|
speculativeReduceTasks++;
|
|
metrics.launchReduce(id);
|
|
metrics.launchReduce(id);
|
|
|
|
+ this.queueMetrics.launchReduce(id);
|
|
}
|
|
}
|
|
// Note that the logs are for the scheduled tasks only. Tasks that join on
|
|
// Note that the logs are for the scheduled tasks only. Tasks that join on
|
|
// restart has already their logs in place.
|
|
// restart has already their logs in place.
|
|
@@ -1852,9 +1867,11 @@ public class JobInProgress {
|
|
map.put(taskTracker, info);
|
|
map.put(taskTracker, info);
|
|
if (type == TaskType.MAP) {
|
|
if (type == TaskType.MAP) {
|
|
jobtracker.getInstrumentation().addReservedMapSlots(reservedSlots);
|
|
jobtracker.getInstrumentation().addReservedMapSlots(reservedSlots);
|
|
|
|
+ this.queueMetrics.addReservedMapSlots(reservedSlots);
|
|
}
|
|
}
|
|
else {
|
|
else {
|
|
jobtracker.getInstrumentation().addReservedReduceSlots(reservedSlots);
|
|
jobtracker.getInstrumentation().addReservedReduceSlots(reservedSlots);
|
|
|
|
+ this.queueMetrics.addReservedReduceSlots(reservedSlots);
|
|
}
|
|
}
|
|
jobtracker.incrementReservations(type, reservedSlots);
|
|
jobtracker.incrementReservations(type, reservedSlots);
|
|
}
|
|
}
|
|
@@ -1884,10 +1901,12 @@ public class JobInProgress {
|
|
map.remove(taskTracker);
|
|
map.remove(taskTracker);
|
|
if (type == TaskType.MAP) {
|
|
if (type == TaskType.MAP) {
|
|
jobtracker.getInstrumentation().decReservedMapSlots(info.getNumSlots());
|
|
jobtracker.getInstrumentation().decReservedMapSlots(info.getNumSlots());
|
|
|
|
+ this.queueMetrics.decReservedMapSlots(info.getNumSlots());
|
|
}
|
|
}
|
|
else {
|
|
else {
|
|
jobtracker.getInstrumentation().decReservedReduceSlots(
|
|
jobtracker.getInstrumentation().decReservedReduceSlots(
|
|
info.getNumSlots());
|
|
info.getNumSlots());
|
|
|
|
+ this.queueMetrics.decReservedReduceSlots(info.getNumSlots());
|
|
}
|
|
}
|
|
jobtracker.decrementReservations(type, info.getNumSlots());
|
|
jobtracker.decrementReservations(type, info.getNumSlots());
|
|
}
|
|
}
|
|
@@ -2596,6 +2615,7 @@ public class JobInProgress {
|
|
}
|
|
}
|
|
finishedMapTasks += 1;
|
|
finishedMapTasks += 1;
|
|
metrics.completeMap(taskid);
|
|
metrics.completeMap(taskid);
|
|
|
|
+ this.queueMetrics.completeMap(taskid);
|
|
// remove the completed map from the resp running caches
|
|
// remove the completed map from the resp running caches
|
|
retireMap(tip);
|
|
retireMap(tip);
|
|
if ((finishedMapTasks + failedMapTIPs) == (numMapTasks)) {
|
|
if ((finishedMapTasks + failedMapTIPs) == (numMapTasks)) {
|
|
@@ -2611,6 +2631,7 @@ public class JobInProgress {
|
|
}
|
|
}
|
|
finishedReduceTasks += 1;
|
|
finishedReduceTasks += 1;
|
|
metrics.completeReduce(taskid);
|
|
metrics.completeReduce(taskid);
|
|
|
|
+ this.queueMetrics.completeReduce(taskid);
|
|
// remove the completed reduces from the running reducers set
|
|
// remove the completed reduces from the running reducers set
|
|
retireReduce(tip);
|
|
retireReduce(tip);
|
|
if ((finishedReduceTasks + failedReduceTIPs) == (numReduceTasks)) {
|
|
if ((finishedReduceTasks + failedReduceTIPs) == (numReduceTasks)) {
|
|
@@ -2655,14 +2676,18 @@ public class JobInProgress {
|
|
//update the metrics
|
|
//update the metrics
|
|
if (oldState == JobStatus.PREP) {
|
|
if (oldState == JobStatus.PREP) {
|
|
this.jobtracker.getInstrumentation().decPrepJob(conf, jobId);
|
|
this.jobtracker.getInstrumentation().decPrepJob(conf, jobId);
|
|
|
|
+ this.queueMetrics.decPrepJob(conf, jobId);
|
|
} else if (oldState == JobStatus.RUNNING) {
|
|
} else if (oldState == JobStatus.RUNNING) {
|
|
this.jobtracker.getInstrumentation().decRunningJob(conf, jobId);
|
|
this.jobtracker.getInstrumentation().decRunningJob(conf, jobId);
|
|
|
|
+ this.queueMetrics.decRunningJob(conf, jobId);
|
|
}
|
|
}
|
|
|
|
|
|
if (newState == JobStatus.PREP) {
|
|
if (newState == JobStatus.PREP) {
|
|
this.jobtracker.getInstrumentation().addPrepJob(conf, jobId);
|
|
this.jobtracker.getInstrumentation().addPrepJob(conf, jobId);
|
|
|
|
+ this.queueMetrics.addPrepJob(conf, jobId);
|
|
} else if (newState == JobStatus.RUNNING) {
|
|
} else if (newState == JobStatus.RUNNING) {
|
|
this.jobtracker.getInstrumentation().addRunningJob(conf, jobId);
|
|
this.jobtracker.getInstrumentation().addRunningJob(conf, jobId);
|
|
|
|
+ this.queueMetrics.addRunningJob(conf, jobId);
|
|
}
|
|
}
|
|
|
|
|
|
}
|
|
}
|
|
@@ -2717,6 +2742,7 @@ public class JobInProgress {
|
|
garbageCollect();
|
|
garbageCollect();
|
|
|
|
|
|
metrics.completeJob(this.conf, this.status.getJobID());
|
|
metrics.completeJob(this.conf, this.status.getJobID());
|
|
|
|
+ this.queueMetrics.completeJob(this.conf, this.status.getJobID());
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
@@ -2757,9 +2783,11 @@ public class JobInProgress {
|
|
if (jobTerminationState == JobStatus.FAILED) {
|
|
if (jobTerminationState == JobStatus.FAILED) {
|
|
jobtracker.getInstrumentation().failedJob(
|
|
jobtracker.getInstrumentation().failedJob(
|
|
this.conf, this.status.getJobID());
|
|
this.conf, this.status.getJobID());
|
|
|
|
+ this.queueMetrics.failedJob(this.conf, this.status.getJobID());
|
|
} else {
|
|
} else {
|
|
jobtracker.getInstrumentation().killedJob(
|
|
jobtracker.getInstrumentation().killedJob(
|
|
this.conf, this.status.getJobID());
|
|
this.conf, this.status.getJobID());
|
|
|
|
+ this.queueMetrics.killedJob(this.conf, this.status.getJobID());
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
@@ -2910,9 +2938,11 @@ public class JobInProgress {
|
|
if (tip.isMapTask() && !metricsDone) {
|
|
if (tip.isMapTask() && !metricsDone) {
|
|
runningMapTasks -= 1;
|
|
runningMapTasks -= 1;
|
|
metrics.failedMap(taskid);
|
|
metrics.failedMap(taskid);
|
|
|
|
+ this.queueMetrics.failedMap(taskid);
|
|
} else if (!metricsDone) {
|
|
} else if (!metricsDone) {
|
|
runningReduceTasks -= 1;
|
|
runningReduceTasks -= 1;
|
|
metrics.failedReduce(taskid);
|
|
metrics.failedReduce(taskid);
|
|
|
|
+ this.queueMetrics.failedReduce(taskid);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
@@ -3155,6 +3185,8 @@ public class JobInProgress {
|
|
// Let the JobTracker know that a job is complete
|
|
// Let the JobTracker know that a job is complete
|
|
jobtracker.getInstrumentation().decWaitingMaps(getJobID(), pendingMaps());
|
|
jobtracker.getInstrumentation().decWaitingMaps(getJobID(), pendingMaps());
|
|
jobtracker.getInstrumentation().decWaitingReduces(getJobID(), pendingReduces());
|
|
jobtracker.getInstrumentation().decWaitingReduces(getJobID(), pendingReduces());
|
|
|
|
+ this.queueMetrics.decWaitingMaps(getJobID(), pendingMaps());
|
|
|
|
+ this.queueMetrics.decWaitingReduces(getJobID(), pendingReduces());
|
|
jobtracker.storeCompletedJob(this);
|
|
jobtracker.storeCompletedJob(this);
|
|
jobtracker.finalizeJob(this);
|
|
jobtracker.finalizeJob(this);
|
|
|
|
|