|
@@ -104,7 +104,6 @@ class TaskInProgress {
|
|
|
|
|
|
// Map from taskId -> Task
|
|
// Map from taskId -> Task
|
|
private Map<String, Task> tasks = new TreeMap<String, Task>();
|
|
private Map<String, Task> tasks = new TreeMap<String, Task>();
|
|
- boolean savedTaskOutput = false;
|
|
|
|
|
|
|
|
private TreeSet<String> machinesWhereFailed = new TreeSet<String>();
|
|
private TreeSet<String> machinesWhereFailed = new TreeSet<String>();
|
|
private TreeSet<String> tasksReportedClosed = new TreeSet<String>();
|
|
private TreeSet<String> tasksReportedClosed = new TreeSet<String>();
|
|
@@ -185,6 +184,15 @@ class TaskInProgress {
|
|
return partition;
|
|
return partition;
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ public boolean isOnlyCommitPending() {
|
|
|
|
+ for (TaskStatus t : taskStatuses.values()) {
|
|
|
|
+ if (t.getRunState() == TaskStatus.State.COMMIT_PENDING) {
|
|
|
|
+ return true;
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ return false;
|
|
|
|
+ }
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* Initialization common to Map and Reduce
|
|
* Initialization common to Map and Reduce
|
|
*/
|
|
*/
|
|
@@ -218,6 +226,15 @@ class TaskInProgress {
|
|
return split != null;
|
|
return split != null;
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ /**
|
|
|
|
+ * Return the Task object associated with a taskId
|
|
|
|
+ * @param taskId
|
|
|
|
+ * @return
|
|
|
|
+ */
|
|
|
|
+ public Task getTaskObject(String taskId) {
|
|
|
|
+ return tasks.get(taskId);
|
|
|
|
+ }
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* Is this tip currently running any tasks?
|
|
* Is this tip currently running any tasks?
|
|
* @return true if any tasks are running
|
|
* @return true if any tasks are running
|
|
@@ -231,7 +248,7 @@ class TaskInProgress {
|
|
*
|
|
*
|
|
* @return <code>true</code> if the tip is complete, else <code>false</code>
|
|
* @return <code>true</code> if the tip is complete, else <code>false</code>
|
|
*/
|
|
*/
|
|
- public boolean isComplete() {
|
|
|
|
|
|
+ public synchronized boolean isComplete() {
|
|
return (completes > 0);
|
|
return (completes > 0);
|
|
}
|
|
}
|
|
|
|
|
|
@@ -350,7 +367,7 @@ class TaskInProgress {
|
|
* @param taskId id of the task
|
|
* @param taskId id of the task
|
|
* @param diagInfo diagnostic information for the task
|
|
* @param diagInfo diagnostic information for the task
|
|
*/
|
|
*/
|
|
- private void addDiagnosticInfo(String taskId, String diagInfo) {
|
|
|
|
|
|
+ public void addDiagnosticInfo(String taskId, String diagInfo) {
|
|
List<String> diagHistory = taskDiagnosticData.get(taskId);
|
|
List<String> diagHistory = taskDiagnosticData.get(taskId);
|
|
if (diagHistory == null) {
|
|
if (diagHistory == null) {
|
|
diagHistory = new ArrayList<String>();
|
|
diagHistory = new ArrayList<String>();
|
|
@@ -396,7 +413,8 @@ class TaskInProgress {
|
|
if (newState == TaskStatus.State.RUNNING &&
|
|
if (newState == TaskStatus.State.RUNNING &&
|
|
(oldState == TaskStatus.State.FAILED ||
|
|
(oldState == TaskStatus.State.FAILED ||
|
|
oldState == TaskStatus.State.KILLED ||
|
|
oldState == TaskStatus.State.KILLED ||
|
|
- oldState == TaskStatus.State.SUCCEEDED)) {
|
|
|
|
|
|
+ oldState == TaskStatus.State.SUCCEEDED ||
|
|
|
|
+ oldState == TaskStatus.State.COMMIT_PENDING)) {
|
|
return false;
|
|
return false;
|
|
}
|
|
}
|
|
|
|
|
|
@@ -419,10 +437,18 @@ class TaskInProgress {
|
|
//
|
|
//
|
|
// Note the failure and its location
|
|
// Note the failure and its location
|
|
//
|
|
//
|
|
- LOG.info("Task '" + taskid + "' has been lost.");
|
|
|
|
TaskStatus status = taskStatuses.get(taskid);
|
|
TaskStatus status = taskStatuses.get(taskid);
|
|
TaskStatus.State taskState = TaskStatus.State.FAILED;
|
|
TaskStatus.State taskState = TaskStatus.State.FAILED;
|
|
if (status != null) {
|
|
if (status != null) {
|
|
|
|
+ // Check if the user manually KILLED/FAILED this task-attempt...
|
|
|
|
+ Boolean shouldFail = tasksToKill.remove(taskid);
|
|
|
|
+ if (shouldFail != null) {
|
|
|
|
+ taskState = (shouldFail) ? TaskStatus.State.FAILED :
|
|
|
|
+ TaskStatus.State.KILLED;
|
|
|
|
+ status.setRunState(taskState);
|
|
|
|
+ addDiagnosticInfo(taskid, "Task has been " + taskState + " by the user" );
|
|
|
|
+ }
|
|
|
|
+
|
|
taskState = status.getRunState();
|
|
taskState = status.getRunState();
|
|
if (taskState != TaskStatus.State.FAILED &&
|
|
if (taskState != TaskStatus.State.FAILED &&
|
|
taskState != TaskStatus.State.KILLED) {
|
|
taskState != TaskStatus.State.KILLED) {
|
|
@@ -441,24 +467,18 @@ class TaskInProgress {
|
|
this.activeTasks.remove(taskid);
|
|
this.activeTasks.remove(taskid);
|
|
|
|
|
|
// Since we do not fail completed reduces (whose outputs go to hdfs), we
|
|
// Since we do not fail completed reduces (whose outputs go to hdfs), we
|
|
- // should note this failure only for completed maps; however if the job
|
|
|
|
- // is done, there is no need to manipulate completed maps
|
|
|
|
- if (this.completes > 0 && this.isMapTask() &&
|
|
|
|
|
|
+ // should note this failure only for completed maps, only if this taskid;
|
|
|
|
+ // completed this map. however if the job is done, there is no need to
|
|
|
|
+ // manipulate completed maps
|
|
|
|
+ if (this.isMapTask() && isComplete(taskid) &&
|
|
jobStatus.getRunState() != JobStatus.SUCCEEDED) {
|
|
jobStatus.getRunState() != JobStatus.SUCCEEDED) {
|
|
this.completes--;
|
|
this.completes--;
|
|
}
|
|
}
|
|
|
|
|
|
- // Discard task output
|
|
|
|
- Task t = tasks.get(taskid);
|
|
|
|
- try {
|
|
|
|
- t.discardTaskOutput();
|
|
|
|
- } catch (IOException ioe) {
|
|
|
|
- LOG.info("Failed to discard output of task '" + taskid + "' with " +
|
|
|
|
- StringUtils.stringifyException(ioe));
|
|
|
|
- }
|
|
|
|
|
|
|
|
if (taskState == TaskStatus.State.FAILED) {
|
|
if (taskState == TaskStatus.State.FAILED) {
|
|
numTaskFailures++;
|
|
numTaskFailures++;
|
|
|
|
+ machinesWhereFailed.add(trackerName);
|
|
} else {
|
|
} else {
|
|
numKilledTasks++;
|
|
numKilledTasks++;
|
|
}
|
|
}
|
|
@@ -467,7 +487,6 @@ class TaskInProgress {
|
|
LOG.info("TaskInProgress " + getTIPId() + " has failed " + numTaskFailures + " times.");
|
|
LOG.info("TaskInProgress " + getTIPId() + " has failed " + numTaskFailures + " times.");
|
|
kill();
|
|
kill();
|
|
}
|
|
}
|
|
- machinesWhereFailed.add(trackerName);
|
|
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -490,14 +509,6 @@ class TaskInProgress {
|
|
* taskid as {@link TaskStatus.State.KILLED}.
|
|
* taskid as {@link TaskStatus.State.KILLED}.
|
|
*/
|
|
*/
|
|
void alreadyCompletedTask(String taskid) {
|
|
void alreadyCompletedTask(String taskid) {
|
|
- Task t = tasks.get(taskid);
|
|
|
|
- try {
|
|
|
|
- t.discardTaskOutput();
|
|
|
|
- } catch (IOException ioe) {
|
|
|
|
- LOG.info("Failed to discard output of task '" + taskid + "' with " +
|
|
|
|
- StringUtils.stringifyException(ioe));
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
// 'KILL' the task
|
|
// 'KILL' the task
|
|
completedTask(taskid, TaskStatus.State.KILLED);
|
|
completedTask(taskid, TaskStatus.State.KILLED);
|
|
|
|
|
|
@@ -512,29 +523,11 @@ class TaskInProgress {
|
|
* Indicate that one of the taskids in this TaskInProgress
|
|
* Indicate that one of the taskids in this TaskInProgress
|
|
* has successfully completed!
|
|
* has successfully completed!
|
|
*/
|
|
*/
|
|
- public void completed(String taskid) throws IOException {
|
|
|
|
- //
|
|
|
|
- // Finalize the task's output
|
|
|
|
- //
|
|
|
|
- Task t = tasks.get(taskid);
|
|
|
|
- if (!savedTaskOutput) {
|
|
|
|
- t.saveTaskOutput();
|
|
|
|
- savedTaskOutput = true;
|
|
|
|
- } else {
|
|
|
|
- try {
|
|
|
|
- t.discardTaskOutput();
|
|
|
|
- } catch (IOException ioe) {
|
|
|
|
- LOG.info("Failed to discard 'already-saved' output of task: " +
|
|
|
|
- t.getTaskId() + " with: " +
|
|
|
|
- StringUtils.stringifyException(ioe));
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
|
|
+ public void completed(String taskid) {
|
|
//
|
|
//
|
|
// Record that this taskid is complete
|
|
// Record that this taskid is complete
|
|
//
|
|
//
|
|
completedTask(taskid, TaskStatus.State.SUCCEEDED);
|
|
completedTask(taskid, TaskStatus.State.SUCCEEDED);
|
|
-
|
|
|
|
|
|
|
|
//
|
|
//
|
|
// Now that the TIP is complete, the other speculative
|
|
// Now that the TIP is complete, the other speculative
|
|
@@ -545,7 +538,6 @@ class TaskInProgress {
|
|
this.completes++;
|
|
this.completes++;
|
|
recomputeProgress();
|
|
recomputeProgress();
|
|
|
|
|
|
- LOG.info("Task '" + taskid + "' has completed succesfully");
|
|
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -588,7 +580,8 @@ class TaskInProgress {
|
|
*/
|
|
*/
|
|
boolean killTask(String taskId, boolean shouldFail) {
|
|
boolean killTask(String taskId, boolean shouldFail) {
|
|
TaskStatus st = taskStatuses.get(taskId);
|
|
TaskStatus st = taskStatuses.get(taskId);
|
|
- if(st != null && st.getRunState() == TaskStatus.State.RUNNING
|
|
|
|
|
|
+ if(st != null && (st.getRunState() == TaskStatus.State.RUNNING
|
|
|
|
+ || st.getRunState() == TaskStatus.State.COMMIT_PENDING)
|
|
&& tasksToKill.put(taskId, shouldFail) == null ) {
|
|
&& tasksToKill.put(taskId, shouldFail) == null ) {
|
|
String logStr = "Request received to " + (shouldFail ? "fail" : "kill")
|
|
String logStr = "Request received to " + (shouldFail ? "fail" : "kill")
|
|
+ " task '" + taskId + "' by user";
|
|
+ " task '" + taskId + "' by user";
|
|
@@ -599,32 +592,6 @@ class TaskInProgress {
|
|
return false;
|
|
return false;
|
|
}
|
|
}
|
|
|
|
|
|
- /** Notification that a task with the given id has been killed */
|
|
|
|
- void taskKilled(String taskId, String trackerName, JobStatus jobStatus) {
|
|
|
|
- Boolean shouldFail = tasksToKill.remove(taskId);
|
|
|
|
- if(shouldFail != null && !shouldFail) {
|
|
|
|
- LOG.info("Task '" + taskId + "' has been killed");
|
|
|
|
- this.activeTasks.remove(taskId);
|
|
|
|
- taskStatuses.get(taskId).setRunState(TaskStatus.State.KILLED );
|
|
|
|
- addDiagnosticInfo(taskId, "Task has been killed" );
|
|
|
|
- // Discard task output
|
|
|
|
- Task t = tasks.get(taskId);
|
|
|
|
- try {
|
|
|
|
- t.discardTaskOutput();
|
|
|
|
- } catch (IOException ioe) {
|
|
|
|
- LOG.info("Failed to discard output of task '" + taskId + "' with " +
|
|
|
|
- StringUtils.stringifyException(ioe));
|
|
|
|
- }
|
|
|
|
- numKilledTasks++;
|
|
|
|
-
|
|
|
|
- }
|
|
|
|
- else {
|
|
|
|
- //set the task status as failed.
|
|
|
|
- taskStatuses.get(taskId).setRunState(TaskStatus.State.FAILED);
|
|
|
|
- incompleteSubTask(taskId, trackerName, jobStatus);
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
/**
|
|
/**
|
|
* This method is called whenever there's a status change
|
|
* This method is called whenever there's a status change
|
|
* for one of the TIP's sub-tasks. It recomputes the overall
|
|
* for one of the TIP's sub-tasks. It recomputes the overall
|
|
@@ -650,6 +617,12 @@ class TaskInProgress {
|
|
bestState = status.getStateString();
|
|
bestState = status.getStateString();
|
|
bestCounters = status.getCounters();
|
|
bestCounters = status.getCounters();
|
|
break;
|
|
break;
|
|
|
|
+ } else if (status.getRunState() == TaskStatus.State.COMMIT_PENDING) {
|
|
|
|
+ //for COMMIT_PENDING, we take the last state that we recorded
|
|
|
|
+ //when the task was RUNNING
|
|
|
|
+ bestProgress = this.progress;
|
|
|
|
+ bestState = this.state;
|
|
|
|
+ bestCounters = this.counters;
|
|
} else if (status.getRunState() == TaskStatus.State.RUNNING) {
|
|
} else if (status.getRunState() == TaskStatus.State.RUNNING) {
|
|
if (status.getProgress() >= bestProgress) {
|
|
if (status.getProgress() >= bestProgress) {
|
|
bestProgress = status.getProgress();
|
|
bestProgress = status.getProgress();
|
|
@@ -692,7 +665,7 @@ class TaskInProgress {
|
|
runSpeculative &&
|
|
runSpeculative &&
|
|
(averageProgress - progress >= SPECULATIVE_GAP) &&
|
|
(averageProgress - progress >= SPECULATIVE_GAP) &&
|
|
(System.currentTimeMillis() - startTime >= SPECULATIVE_LAG)
|
|
(System.currentTimeMillis() - startTime >= SPECULATIVE_LAG)
|
|
- && completes == 0) {
|
|
|
|
|
|
+ && completes == 0 && !isOnlyCommitPending()) {
|
|
return true;
|
|
return true;
|
|
}
|
|
}
|
|
return false;
|
|
return false;
|