|
@@ -31,25 +31,32 @@ import java.util.TreeSet;
|
|
|
|
|
|
import org.apache.commons.logging.Log;
|
|
import org.apache.commons.logging.Log;
|
|
import org.apache.commons.logging.LogFactory;
|
|
import org.apache.commons.logging.LogFactory;
|
|
|
|
+
|
|
import org.apache.hadoop.mapred.JobInProgress.DataStatistics;
|
|
import org.apache.hadoop.mapred.JobInProgress.DataStatistics;
|
|
import org.apache.hadoop.mapred.SortedRanges.Range;
|
|
import org.apache.hadoop.mapred.SortedRanges.Range;
|
|
|
|
+
|
|
|
|
+import org.apache.hadoop.mapreduce.TaskCounter;
|
|
import org.apache.hadoop.mapreduce.TaskType;
|
|
import org.apache.hadoop.mapreduce.TaskType;
|
|
import org.apache.hadoop.mapreduce.jobhistory.JobHistory;
|
|
import org.apache.hadoop.mapreduce.jobhistory.JobHistory;
|
|
import org.apache.hadoop.mapreduce.jobhistory.TaskUpdatedEvent;
|
|
import org.apache.hadoop.mapreduce.jobhistory.TaskUpdatedEvent;
|
|
import org.apache.hadoop.mapreduce.split.JobSplit.TaskSplitMetaInfo;
|
|
import org.apache.hadoop.mapreduce.split.JobSplit.TaskSplitMetaInfo;
|
|
|
|
+
|
|
|
|
+import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig;
|
|
|
|
+
|
|
import org.apache.hadoop.net.Node;
|
|
import org.apache.hadoop.net.Node;
|
|
|
|
|
|
|
|
|
|
|
|
+
|
|
/*************************************************************
|
|
/*************************************************************
|
|
* TaskInProgress maintains all the info needed for a
|
|
* TaskInProgress maintains all the info needed for a
|
|
* Task in the lifetime of its owning Job. A given Task
|
|
* Task in the lifetime of its owning Job. A given Task
|
|
* might be speculatively executed or reexecuted, so we
|
|
* might be speculatively executed or reexecuted, so we
|
|
* need a level of indirection above the running-id itself.
|
|
* need a level of indirection above the running-id itself.
|
|
* <br>
|
|
* <br>
|
|
- * A given TaskInProgress contains multiple taskids,
|
|
|
|
|
|
+ * A given TaskInProgress contains multiple task attempt ids,
|
|
* 0 or more of which might be executing at any one time.
|
|
* 0 or more of which might be executing at any one time.
|
|
- * (That's what allows speculative execution.) A taskid
|
|
|
|
- * is now *never* recycled. A TIP allocates enough taskids
|
|
|
|
|
|
+ * (That's what allows speculative execution.) A task attempt id
|
|
|
|
+ * is now *never* recycled. A TIP allocates enough task attempt ids
|
|
* to account for all the speculation and failures it will
|
|
* to account for all the speculation and failures it will
|
|
* ever have to handle. Once those are up, the TIP is dead.
|
|
* ever have to handle. Once those are up, the TIP is dead.
|
|
* **************************************************************
|
|
* **************************************************************
|
|
@@ -60,6 +67,10 @@ class TaskInProgress {
|
|
static final long SPECULATIVE_LAG = 60 * 1000;
|
|
static final long SPECULATIVE_LAG = 60 * 1000;
|
|
private static final int NUM_ATTEMPTS_PER_RESTART = 1000;
|
|
private static final int NUM_ATTEMPTS_PER_RESTART = 1000;
|
|
|
|
|
|
|
|
+ private static final long MEMORY_SPLITS_RESOLUTION = 1024;
|
|
|
|
+
|
|
|
|
+ static final int DEFAULT_STATISTICS_INTERVALS = 12;
|
|
|
|
+
|
|
public static final Log LOG = LogFactory.getLog(TaskInProgress.class);
|
|
public static final Log LOG = LogFactory.getLog(TaskInProgress.class);
|
|
|
|
|
|
// Defines the TIP
|
|
// Defines the TIP
|
|
@@ -91,6 +102,10 @@ class TaskInProgress {
|
|
private volatile boolean skipping = false;
|
|
private volatile boolean skipping = false;
|
|
private boolean jobCleanup = false;
|
|
private boolean jobCleanup = false;
|
|
private boolean jobSetup = false;
|
|
private boolean jobSetup = false;
|
|
|
|
+
|
|
|
|
+ private static Enum CPU_COUNTER_KEY = TaskCounter.CPU_MILLISECONDS;
|
|
|
|
+ private static Enum VM_BYTES_KEY = TaskCounter.VIRTUAL_MEMORY_BYTES;
|
|
|
|
+ private static Enum PHYSICAL_BYTES_KEY = TaskCounter.PHYSICAL_MEMORY_BYTES;
|
|
|
|
|
|
// The 'next' usable taskid of this tip
|
|
// The 'next' usable taskid of this tip
|
|
int nextTaskId = 0;
|
|
int nextTaskId = 0;
|
|
@@ -109,12 +124,20 @@ class TaskInProgress {
|
|
private JobConf conf;
|
|
private JobConf conf;
|
|
private Map<TaskAttemptID,List<String>> taskDiagnosticData =
|
|
private Map<TaskAttemptID,List<String>> taskDiagnosticData =
|
|
new TreeMap<TaskAttemptID,List<String>>();
|
|
new TreeMap<TaskAttemptID,List<String>>();
|
|
|
|
+
|
|
/**
|
|
/**
|
|
- * Map from taskId -> TaskStatus
|
|
|
|
|
|
+ * Map from task attempt Id -> TaskStatus
|
|
*/
|
|
*/
|
|
TreeMap<TaskAttemptID,TaskStatus> taskStatuses =
|
|
TreeMap<TaskAttemptID,TaskStatus> taskStatuses =
|
|
new TreeMap<TaskAttemptID,TaskStatus>();
|
|
new TreeMap<TaskAttemptID,TaskStatus>();
|
|
|
|
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Map from task attempt Id -> splits block
|
|
|
|
+ */
|
|
|
|
+ private Map<TaskAttemptID, ProgressSplitsBlock> splitsBlocks
|
|
|
|
+ = new TreeMap<TaskAttemptID, ProgressSplitsBlock>();
|
|
|
|
+
|
|
// Map from taskId -> TaskTracker Id,
|
|
// Map from taskId -> TaskTracker Id,
|
|
// contains cleanup attempts and where they ran, if any
|
|
// contains cleanup attempts and where they ran, if any
|
|
private TreeMap<TaskAttemptID, String> cleanupTasks =
|
|
private TreeMap<TaskAttemptID, String> cleanupTasks =
|
|
@@ -183,6 +206,65 @@ class TaskInProgress {
|
|
}
|
|
}
|
|
this.user = job.getUser();
|
|
this.user = job.getUser();
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ synchronized ProgressSplitsBlock getSplits(TaskAttemptID statusAttemptID) {
|
|
|
|
+ ProgressSplitsBlock result = splitsBlocks.get(statusAttemptID);
|
|
|
|
+
|
|
|
|
+ if (result == null) {
|
|
|
|
+ result
|
|
|
|
+ = new ProgressSplitsBlock
|
|
|
|
+ (conf.getInt(JTConfig.JT_JOBHISTORY_TASKPROGRESS_NUMBER_SPLITS,
|
|
|
|
+ ProgressSplitsBlock.DEFAULT_NUMBER_PROGRESS_SPLITS));
|
|
|
|
+ splitsBlocks.put(statusAttemptID, result);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ return result;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private void updateProgressSplits(TaskStatus taskStatus) {
|
|
|
|
+ if (!taskStatus.getIncludeCounters()) {
|
|
|
|
+ return;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ double newProgress = taskStatus.getProgress();
|
|
|
|
+
|
|
|
|
+ Counters counters = taskStatus.getCounters();
|
|
|
|
+
|
|
|
|
+ TaskAttemptID statusAttemptID = taskStatus.getTaskID();
|
|
|
|
+ ProgressSplitsBlock splitsBlock = getSplits(statusAttemptID);
|
|
|
|
+
|
|
|
|
+ if (splitsBlock != null) {
|
|
|
|
+
|
|
|
|
+ long now = JobTracker.getClock().getTime();
|
|
|
|
+ Long start = getDispatchTime(statusAttemptID);
|
|
|
|
+
|
|
|
|
+ if (start != null && now - start <= Integer.MAX_VALUE) {
|
|
|
|
+ splitsBlock.progressWallclockTime.extend
|
|
|
|
+ (newProgress, (int)(now - start));
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ Counters.Counter cpuCounter = counters.findCounter(CPU_COUNTER_KEY);
|
|
|
|
+ if (cpuCounter != null
|
|
|
|
+ && cpuCounter.getCounter() <= Integer.MAX_VALUE) {
|
|
|
|
+ splitsBlock.progressCPUTime.extend
|
|
|
|
+ (newProgress, (int)(cpuCounter.getCounter()));
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ Counters.Counter virtualBytes = counters.findCounter(VM_BYTES_KEY);
|
|
|
|
+ if (virtualBytes != null) {
|
|
|
|
+ splitsBlock.progressVirtualMemoryKbytes.extend
|
|
|
|
+ (newProgress,
|
|
|
|
+ (int)(virtualBytes.getCounter() / (MEMORY_SPLITS_RESOLUTION)));
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ Counters.Counter physicalBytes = counters.findCounter(PHYSICAL_BYTES_KEY);
|
|
|
|
+ if (physicalBytes != null) {
|
|
|
|
+ splitsBlock.progressPhysicalMemoryKbytes.extend
|
|
|
|
+ (newProgress,
|
|
|
|
+ (int)(physicalBytes.getCounter() / (MEMORY_SPLITS_RESOLUTION)));
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
|
|
/**
|
|
/**
|
|
* Set the max number of attempts before we declare a TIP as "failed"
|
|
* Set the max number of attempts before we declare a TIP as "failed"
|
|
@@ -294,6 +376,7 @@ class TaskInProgress {
|
|
return execFinishTime;
|
|
return execFinishTime;
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* Set the exec finish time
|
|
* Set the exec finish time
|
|
*/
|
|
*/
|
|
@@ -582,23 +665,24 @@ class TaskInProgress {
|
|
* @return has the task changed its state noticeably?
|
|
* @return has the task changed its state noticeably?
|
|
*/
|
|
*/
|
|
synchronized boolean updateStatus(TaskStatus status) {
|
|
synchronized boolean updateStatus(TaskStatus status) {
|
|
- TaskAttemptID taskid = status.getTaskID();
|
|
|
|
- String tracker = status.getTaskTracker();
|
|
|
|
- String diagInfo = status.getDiagnosticInfo();
|
|
|
|
- TaskStatus oldStatus = taskStatuses.get(taskid);
|
|
|
|
- boolean changed = true;
|
|
|
|
- if (diagInfo != null && diagInfo.length() > 0) {
|
|
|
|
- LOG.info("Error from " + taskid + " on " + tracker + ": "+ diagInfo);
|
|
|
|
- addDiagnosticInfo(taskid, diagInfo);
|
|
|
|
- }
|
|
|
|
|
|
+ try {
|
|
|
|
+ TaskAttemptID taskid = status.getTaskID();
|
|
|
|
+ String tracker = status.getTaskTracker();
|
|
|
|
+ String diagInfo = status.getDiagnosticInfo();
|
|
|
|
+ TaskStatus oldStatus = taskStatuses.get(taskid);
|
|
|
|
+ boolean changed = true;
|
|
|
|
+ if (diagInfo != null && diagInfo.length() > 0) {
|
|
|
|
+ LOG.info("Error from " + taskid + " on " + tracker + ": "+ diagInfo);
|
|
|
|
+ addDiagnosticInfo(taskid, diagInfo);
|
|
|
|
+ }
|
|
|
|
|
|
- if(skipping) {
|
|
|
|
- failedRanges.updateState(status);
|
|
|
|
- }
|
|
|
|
|
|
+ if(skipping) {
|
|
|
|
+ failedRanges.updateState(status);
|
|
|
|
+ }
|
|
|
|
|
|
- if (oldStatus != null) {
|
|
|
|
- TaskStatus.State oldState = oldStatus.getRunState();
|
|
|
|
- TaskStatus.State newState = status.getRunState();
|
|
|
|
|
|
+ if (oldStatus != null) {
|
|
|
|
+ TaskStatus.State oldState = oldStatus.getRunState();
|
|
|
|
+ TaskStatus.State newState = status.getRunState();
|
|
|
|
|
|
// We should never receive a duplicate success/failure/killed
|
|
// We should never receive a duplicate success/failure/killed
|
|
// status update for the same taskid! This is a safety check,
|
|
// status update for the same taskid! This is a safety check,
|
|
@@ -617,60 +701,63 @@ class TaskInProgress {
|
|
return false;
|
|
return false;
|
|
}
|
|
}
|
|
|
|
|
|
- // The task is not allowed to move from completed back to running.
|
|
|
|
- // We have seen out of order status messagesmoving tasks from complete
|
|
|
|
- // to running. This is a spot fix, but it should be addressed more
|
|
|
|
- // globally.
|
|
|
|
- if ((newState == TaskStatus.State.RUNNING ||
|
|
|
|
- newState == TaskStatus.State.UNASSIGNED) &&
|
|
|
|
- (oldState == TaskStatus.State.FAILED ||
|
|
|
|
- oldState == TaskStatus.State.KILLED ||
|
|
|
|
- oldState == TaskStatus.State.FAILED_UNCLEAN ||
|
|
|
|
- oldState == TaskStatus.State.KILLED_UNCLEAN ||
|
|
|
|
- oldState == TaskStatus.State.SUCCEEDED ||
|
|
|
|
- oldState == TaskStatus.State.COMMIT_PENDING)) {
|
|
|
|
- return false;
|
|
|
|
- }
|
|
|
|
|
|
+ // The task is not allowed to move from completed back to running.
|
|
|
|
+ // We have seen out of order status messagesmoving tasks from complete
|
|
|
|
+ // to running. This is a spot fix, but it should be addressed more
|
|
|
|
+ // globally.
|
|
|
|
+ if ((newState == TaskStatus.State.RUNNING ||
|
|
|
|
+ newState == TaskStatus.State.UNASSIGNED) &&
|
|
|
|
+ (oldState == TaskStatus.State.FAILED ||
|
|
|
|
+ oldState == TaskStatus.State.KILLED ||
|
|
|
|
+ oldState == TaskStatus.State.FAILED_UNCLEAN ||
|
|
|
|
+ oldState == TaskStatus.State.KILLED_UNCLEAN ||
|
|
|
|
+ oldState == TaskStatus.State.SUCCEEDED ||
|
|
|
|
+ oldState == TaskStatus.State.COMMIT_PENDING)) {
|
|
|
|
+ return false;
|
|
|
|
+ }
|
|
|
|
|
|
- //Do not accept any status once the task is marked FAILED/KILLED
|
|
|
|
- //This is to handle the case of the JobTracker timing out a task
|
|
|
|
- //due to launch delay, but the TT comes back with any state or
|
|
|
|
- //TT got expired
|
|
|
|
- if (oldState == TaskStatus.State.FAILED ||
|
|
|
|
- oldState == TaskStatus.State.KILLED) {
|
|
|
|
- tasksToKill.put(taskid, true);
|
|
|
|
- return false;
|
|
|
|
- }
|
|
|
|
|
|
+ //Do not accept any status once the task is marked FAILED/KILLED
|
|
|
|
+ //This is to handle the case of the JobTracker timing out a task
|
|
|
|
+ //due to launch delay, but the TT comes back with any state or
|
|
|
|
+ //TT got expired
|
|
|
|
+ if (oldState == TaskStatus.State.FAILED ||
|
|
|
|
+ oldState == TaskStatus.State.KILLED) {
|
|
|
|
+ tasksToKill.put(taskid, true);
|
|
|
|
+ return false;
|
|
|
|
+ }
|
|
|
|
|
|
- changed = oldState != newState;
|
|
|
|
- }
|
|
|
|
- // if task is a cleanup attempt, do not replace the complete status,
|
|
|
|
- // update only specific fields.
|
|
|
|
- // For example, startTime should not be updated,
|
|
|
|
- // but finishTime has to be updated.
|
|
|
|
- if (!isCleanupAttempt(taskid)) {
|
|
|
|
- taskStatuses.put(taskid, status);
|
|
|
|
- //we don't want to include setup tasks in the task execution stats
|
|
|
|
- if (!isJobSetupTask() && ((isMapTask() && job.hasSpeculativeMaps()) ||
|
|
|
|
- (!isMapTask() && job.hasSpeculativeReduces()))) {
|
|
|
|
- long now = JobTracker.getClock().getTime();
|
|
|
|
- double oldProgRate = getOldProgressRate();
|
|
|
|
- double currProgRate = getCurrentProgressRate(now);
|
|
|
|
- job.updateStatistics(oldProgRate, currProgRate, isMapTask());
|
|
|
|
- //we need to store the current progress rate, so that we can
|
|
|
|
- //update statistics accurately the next time we invoke
|
|
|
|
- //updateStatistics
|
|
|
|
- setProgressRate(currProgRate);
|
|
|
|
|
|
+ changed = oldState != newState;
|
|
|
|
+ }
|
|
|
|
+ // if task is a cleanup attempt, do not replace the complete status,
|
|
|
|
+ // update only specific fields.
|
|
|
|
+ // For example, startTime should not be updated,
|
|
|
|
+ // but finishTime has to be updated.
|
|
|
|
+ if (!isCleanupAttempt(taskid)) {
|
|
|
|
+ taskStatuses.put(taskid, status);
|
|
|
|
+ //we don't want to include setup tasks in the task execution stats
|
|
|
|
+ if (!isJobSetupTask() && ((isMapTask() && job.hasSpeculativeMaps()) ||
|
|
|
|
+ (!isMapTask() && job.hasSpeculativeReduces()))) {
|
|
|
|
+ long now = JobTracker.getClock().getTime();
|
|
|
|
+ double oldProgRate = getOldProgressRate();
|
|
|
|
+ double currProgRate = getCurrentProgressRate(now);
|
|
|
|
+ job.updateStatistics(oldProgRate, currProgRate, isMapTask());
|
|
|
|
+ //we need to store the current progress rate, so that we can
|
|
|
|
+ //update statistics accurately the next time we invoke
|
|
|
|
+ //updateStatistics
|
|
|
|
+ setProgressRate(currProgRate);
|
|
|
|
+ }
|
|
|
|
+ } else {
|
|
|
|
+ taskStatuses.get(taskid).statusUpdate(status.getRunState(),
|
|
|
|
+ status.getProgress(), status.getStateString(), status.getPhase(),
|
|
|
|
+ status.getFinishTime());
|
|
}
|
|
}
|
|
- } else {
|
|
|
|
- taskStatuses.get(taskid).statusUpdate(status.getRunState(),
|
|
|
|
- status.getProgress(), status.getStateString(), status.getPhase(),
|
|
|
|
- status.getFinishTime());
|
|
|
|
- }
|
|
|
|
|
|
|
|
- // Recompute progress
|
|
|
|
- recomputeProgress();
|
|
|
|
- return changed;
|
|
|
|
|
|
+ // Recompute progress
|
|
|
|
+ recomputeProgress();
|
|
|
|
+ return changed;
|
|
|
|
+ } finally {
|
|
|
|
+ updateProgressSplits(status);
|
|
|
|
+ }
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|