|
@@ -43,6 +43,7 @@ import org.apache.hadoop.conf.Configurable;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
|
import org.apache.hadoop.fs.LocalDirAllocator;
|
|
|
+import org.apache.hadoop.fs.LocalFileSystem;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.fs.FileSystem.Statistics;
|
|
|
import org.apache.hadoop.io.BytesWritable;
|
|
@@ -64,6 +65,7 @@ import org.apache.hadoop.mapreduce.lib.reduce.WrappedReducer;
|
|
|
import org.apache.hadoop.mapreduce.task.ReduceContextImpl;
|
|
|
import org.apache.hadoop.yarn.util.ResourceCalculatorProcessTree;
|
|
|
import org.apache.hadoop.net.NetUtils;
|
|
|
+import org.apache.hadoop.util.ExitUtil;
|
|
|
import org.apache.hadoop.util.Progress;
|
|
|
import org.apache.hadoop.util.Progressable;
|
|
|
import org.apache.hadoop.util.ReflectionUtils;
|
|
@@ -730,11 +732,49 @@ abstract public class Task implements Writable, Configurable {
|
|
|
} else {
|
|
|
return split;
|
|
|
}
|
|
|
- }
|
|
|
- /**
|
|
|
- * The communication thread handles communication with the parent (Task Tracker).
|
|
|
- * It sends progress updates if progress has been made or if the task needs to
|
|
|
- * let the parent know that it's alive. It also pings the parent to see if it's alive.
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * exception thrown when the task exceeds some configured limits.
|
|
|
+ */
|
|
|
+ public class TaskLimitException extends IOException {
|
|
|
+ public TaskLimitException(String str) {
|
|
|
+ super(str);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * check the counters to see whether the task has exceeded any configured
|
|
|
+ * limits.
|
|
|
+ * @throws TaskLimitException
|
|
|
+ */
|
|
|
+ protected void checkTaskLimits() throws TaskLimitException {
|
|
|
+ // check the limit for writing to local file system
|
|
|
+ long limit = conf.getLong(MRJobConfig.TASK_LOCAL_WRITE_LIMIT_BYTES,
|
|
|
+ MRJobConfig.DEFAULT_TASK_LOCAL_WRITE_LIMIT_BYTES);
|
|
|
+ if (limit >= 0) {
|
|
|
+ Counters.Counter localWritesCounter = null;
|
|
|
+ try {
|
|
|
+ LocalFileSystem localFS = FileSystem.getLocal(conf);
|
|
|
+ localWritesCounter = counters.findCounter(localFS.getScheme(),
|
|
|
+ FileSystemCounter.BYTES_WRITTEN);
|
|
|
+ } catch (IOException e) {
|
|
|
+ LOG.warn("Could not get LocalFileSystem BYTES_WRITTEN counter");
|
|
|
+ }
|
|
|
+ if (localWritesCounter != null
|
|
|
+ && localWritesCounter.getCounter() > limit) {
|
|
|
+ throw new TaskLimitException("too much write to local file system." +
|
|
|
+ " current value is " + localWritesCounter.getCounter() +
|
|
|
+ " the limit is " + limit);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * The communication thread handles communication with the parent (Task
|
|
|
+ * Tracker). It sends progress updates if progress has been made or if
|
|
|
+ * the task needs to let the parent know that it's alive. It also pings
|
|
|
+ * the parent to see if it's alive.
|
|
|
*/
|
|
|
public void run() {
|
|
|
final int MAX_RETRIES = 3;
|
|
@@ -765,8 +805,9 @@ abstract public class Task implements Writable, Configurable {
|
|
|
if (sendProgress) {
|
|
|
// we need to send progress update
|
|
|
updateCounters();
|
|
|
+ checkTaskLimits();
|
|
|
taskStatus.statusUpdate(taskProgress.get(),
|
|
|
- taskProgress.toString(),
|
|
|
+ taskProgress.toString(),
|
|
|
counters);
|
|
|
amFeedback = umbilical.statusUpdate(taskId, taskStatus);
|
|
|
taskFound = amFeedback.getTaskFound();
|
|
@@ -797,10 +838,21 @@ abstract public class Task implements Writable, Configurable {
|
|
|
mustPreempt.get() + " given " + amFeedback.getPreemption() +
|
|
|
" for "+ taskId + " task status: " +taskStatus.getPhase());
|
|
|
}
|
|
|
- sendProgress = resetProgressFlag();
|
|
|
+ sendProgress = resetProgressFlag();
|
|
|
remainingRetries = MAX_RETRIES;
|
|
|
- }
|
|
|
- catch (Throwable t) {
|
|
|
+ } catch (TaskLimitException e) {
|
|
|
+ String errMsg = "Task exceeded the limits: " +
|
|
|
+ StringUtils.stringifyException(e);
|
|
|
+ LOG.fatal(errMsg);
|
|
|
+ try {
|
|
|
+ umbilical.fatalError(taskId, errMsg);
|
|
|
+ } catch (IOException ioe) {
|
|
|
+ LOG.fatal("Failed to update failure diagnosis", ioe);
|
|
|
+ }
|
|
|
+ LOG.fatal("Killing " + taskId);
|
|
|
+ resetDoneFlag();
|
|
|
+ ExitUtil.terminate(69);
|
|
|
+ } catch (Throwable t) {
|
|
|
LOG.info("Communication exception: " + StringUtils.stringifyException(t));
|
|
|
remainingRetries -=1;
|
|
|
if (remainingRetries == 0) {
|