|
@@ -22,6 +22,7 @@ import java.util.Iterator;
|
|
|
import java.util.Map;
|
|
|
import java.util.concurrent.ConcurrentHashMap;
|
|
|
import java.util.concurrent.ConcurrentMap;
|
|
|
+import java.util.concurrent.atomic.AtomicBoolean;
|
|
|
|
|
|
import com.google.common.annotations.VisibleForTesting;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
@@ -46,12 +47,14 @@ import org.slf4j.LoggerFactory;
|
|
|
*/
|
|
|
@SuppressWarnings({"unchecked", "rawtypes"})
|
|
|
public class TaskHeartbeatHandler extends AbstractService {
|
|
|
-
|
|
|
- private static class ReportTime {
|
|
|
+
|
|
|
+ static class ReportTime {
|
|
|
private long lastProgress;
|
|
|
-
|
|
|
+ private final AtomicBoolean reported;
|
|
|
+
|
|
|
public ReportTime(long time) {
|
|
|
setLastProgress(time);
|
|
|
+ reported = new AtomicBoolean(false);
|
|
|
}
|
|
|
|
|
|
public synchronized void setLastProgress(long time) {
|
|
@@ -61,6 +64,10 @@ public class TaskHeartbeatHandler extends AbstractService {
|
|
|
public synchronized long getLastProgress() {
|
|
|
return lastProgress;
|
|
|
}
|
|
|
+
|
|
|
+ public boolean isReported(){
|
|
|
+ return reported.get();
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
private static final Logger LOG =
|
|
@@ -72,6 +79,7 @@ public class TaskHeartbeatHandler extends AbstractService {
|
|
|
private volatile boolean stopped;
|
|
|
private long taskTimeOut;
|
|
|
private long unregisterTimeOut;
|
|
|
+ private long taskStuckTimeOut;
|
|
|
private int taskTimeOutCheckInterval = 30 * 1000; // 30 seconds.
|
|
|
|
|
|
private final EventHandler eventHandler;
|
|
@@ -98,6 +106,8 @@ public class TaskHeartbeatHandler extends AbstractService {
|
|
|
MRJobConfig.TASK_TIMEOUT, MRJobConfig.DEFAULT_TASK_TIMEOUT_MILLIS);
|
|
|
unregisterTimeOut = conf.getLong(MRJobConfig.TASK_EXIT_TIMEOUT,
|
|
|
MRJobConfig.TASK_EXIT_TIMEOUT_DEFAULT);
|
|
|
+ taskStuckTimeOut = conf.getLong(MRJobConfig.TASK_STUCK_TIMEOUT_MS,
|
|
|
+ MRJobConfig.DEFAULT_TASK_STUCK_TIMEOUT_MS);
|
|
|
|
|
|
// enforce task timeout is at least twice as long as task report interval
|
|
|
long taskProgressReportIntervalMillis = MRJobConfUtil.
|
|
@@ -135,6 +145,7 @@ public class TaskHeartbeatHandler extends AbstractService {
|
|
|
//TODO throw an exception if the task isn't registered.
|
|
|
ReportTime time = runningAttempts.get(attemptID);
|
|
|
if(time != null) {
|
|
|
+ time.reported.compareAndSet(false, true);
|
|
|
time.setLastProgress(clock.getTime());
|
|
|
}
|
|
|
}
|
|
@@ -179,13 +190,21 @@ public class TaskHeartbeatHandler extends AbstractService {
|
|
|
Map.Entry<TaskAttemptId, ReportTime> entry = iterator.next();
|
|
|
boolean taskTimedOut = (taskTimeOut > 0) &&
|
|
|
(currentTime > (entry.getValue().getLastProgress() + taskTimeOut));
|
|
|
+ // when container in NM not started in a long time,
|
|
|
+ // we think the taskAttempt is stuck
|
|
|
+ boolean taskStuck = (!entry.getValue().isReported()) &&
|
|
|
+ (currentTime >
|
|
|
+ (entry.getValue().getLastProgress() + taskStuckTimeOut));
|
|
|
|
|
|
- if(taskTimedOut) {
|
|
|
+ if(taskTimedOut || taskStuck) {
|
|
|
// task is lost, remove from the list and raise lost event
|
|
|
iterator.remove();
|
|
|
eventHandler.handle(new TaskAttemptDiagnosticsUpdateEvent(entry
|
|
|
.getKey(), "AttemptID:" + entry.getKey().toString()
|
|
|
- + " Timed out after " + taskTimeOut / 1000 + " secs"));
|
|
|
+ + " task timeout set: " + taskTimeOut / 1000 + "s,"
|
|
|
+ + " taskTimedOut: " + taskTimedOut + ";"
|
|
|
+ + " task stuck timeout set: " + taskStuckTimeOut / 1000 + "s,"
|
|
|
+ + " taskStuck: " + taskStuck));
|
|
|
eventHandler.handle(new TaskAttemptEvent(entry.getKey(),
|
|
|
TaskAttemptEventType.TA_TIMED_OUT));
|
|
|
}
|
|
@@ -205,6 +224,11 @@ public class TaskHeartbeatHandler extends AbstractService {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ @VisibleForTesting
|
|
|
+ ConcurrentMap getRunningAttempts(){
|
|
|
+ return runningAttempts;
|
|
|
+ }
|
|
|
+
|
|
|
@VisibleForTesting
|
|
|
public long getTaskTimeOut() {
|
|
|
return taskTimeOut;
|