Browse Source

MAPREDUCE-2777. Merged back into 0.20-security-205, along with r1177389 to delete the now-empty test file TestTTMemoryReporting.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-0.20-security-205@1179465 13f79535-47bb-0310-9956-ffa450edef68
Matthew Foley 13 years ago
parent
commit
9b38198ddb

+ 9 - 96
src/core/org/apache/hadoop/util/LinuxMemoryCalculatorPlugin.java

@@ -18,115 +18,28 @@
 
 
 package org.apache.hadoop.util;
 package org.apache.hadoop.util;
 
 
-import java.io.BufferedReader;
-import java.io.FileNotFoundException;
-import java.io.FileReader;
-import java.io.IOException;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
 /**
 /**
  * Plugin to calculate virtual and physical memories on Linux systems.
  * Plugin to calculate virtual and physical memories on Linux systems.
+ * @deprecated Use {@link org.apache.hadoop.util.LinuxResourceCalculatorPlugin}
+ *             instead
  */
  */
+@Deprecated
 public class LinuxMemoryCalculatorPlugin extends MemoryCalculatorPlugin {
 public class LinuxMemoryCalculatorPlugin extends MemoryCalculatorPlugin {
-  private static final Log LOG =
-      LogFactory.getLog(LinuxMemoryCalculatorPlugin.class);
-
-  /**
-   * proc's meminfo virtual file has keys-values in the format
-   * "key:[ \t]*value[ \t]kB".
-   */
-  private static final String PROCFS_MEMFILE = "/proc/meminfo";
-  private static final Pattern PROCFS_MEMFILE_FORMAT =
-      Pattern.compile("^([a-zA-Z]*):[ \t]*([0-9]*)[ \t]kB");
-
-  // We just need the values for the keys MemTotal and SwapTotal
-  private static final String MEMTOTAL_STRING = "MemTotal";
-  private static final String SWAPTOTAL_STRING = "SwapTotal";
-
-  private long ramSize = 0;
-  private long swapSize = 0;
-
-  boolean readMemInfoFile = false;
-
-  private void readProcMemInfoFile() {
-
-    if (readMemInfoFile) {
-      return;
-    }
-
-    // Read "/proc/memInfo" file
-    BufferedReader in = null;
-    FileReader fReader = null;
-    try {
-      fReader = new FileReader(PROCFS_MEMFILE);
-      in = new BufferedReader(fReader);
-    } catch (FileNotFoundException f) {
-      // shouldn't happen....
-      return;
-    }
-
-    Matcher mat = null;
-
-    try {
-      String str = in.readLine();
-      while (str != null) {
-        mat = PROCFS_MEMFILE_FORMAT.matcher(str);
-        if (mat.find()) {
-          if (mat.group(1).equals(MEMTOTAL_STRING)) {
-            ramSize = Long.parseLong(mat.group(2));
-          } else if (mat.group(1).equals(SWAPTOTAL_STRING)) {
-            swapSize = Long.parseLong(mat.group(2));
-          }
-        }
-        str = in.readLine();
-      }
-    } catch (IOException io) {
-      LOG.warn("Error reading the stream " + io);
-    } finally {
-      // Close the streams
-      try {
-        fReader.close();
-        try {
-          in.close();
-        } catch (IOException i) {
-          LOG.warn("Error closing the stream " + in);
-        }
-      } catch (IOException i) {
-        LOG.warn("Error closing the stream " + fReader);
-      }
-    }
-
-    readMemInfoFile = true;
+  private LinuxResourceCalculatorPlugin resourceCalculatorPlugin;
+  // Use everything from LinuxResourceCalculatorPlugin
+  public LinuxMemoryCalculatorPlugin() {
+    resourceCalculatorPlugin = new LinuxResourceCalculatorPlugin();
   }
   }
 
 
   /** {@inheritDoc} */
   /** {@inheritDoc} */
   @Override
   @Override
   public long getPhysicalMemorySize() {
   public long getPhysicalMemorySize() {
-    readProcMemInfoFile();
-    return ramSize * 1024;
+    return resourceCalculatorPlugin.getPhysicalMemorySize();
   }
   }
 
 
   /** {@inheritDoc} */
   /** {@inheritDoc} */
   @Override
   @Override
   public long getVirtualMemorySize() {
   public long getVirtualMemorySize() {
-    readProcMemInfoFile();
-    return (ramSize + swapSize) * 1024;
-  }
-
-  /**
-   * Test the {@link LinuxMemoryCalculatorPlugin}
-   * 
-   * @param args
-   */
-  public static void main(String[] args) {
-    LinuxMemoryCalculatorPlugin plugin = new LinuxMemoryCalculatorPlugin();
-    System.out.println("Physical memory Size(bytes) : "
-        + plugin.getPhysicalMemorySize());
-    System.out.println("Total Virtual memory Size(bytes) : "
-        + plugin.getVirtualMemorySize());
+    return resourceCalculatorPlugin.getVirtualMemorySize();
   }
   }
 }
 }

+ 407 - 0
src/core/org/apache/hadoop/util/LinuxResourceCalculatorPlugin.java

@@ -0,0 +1,407 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.util;
+
+import java.io.BufferedReader;
+import java.io.FileNotFoundException;
+import java.io.FileReader;
+import java.io.IOException;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+/**
+ * Plugin to calculate resource information on Linux systems.
+ */
+public class LinuxResourceCalculatorPlugin extends ResourceCalculatorPlugin {
+  private static final Log LOG =
+      LogFactory.getLog(LinuxResourceCalculatorPlugin.class);
+
+  /**
+   * proc's meminfo virtual file has keys-values in the format
+   * "key:[ \t]*value[ \t]kB".
+   */
+  private static final String PROCFS_MEMFILE = "/proc/meminfo";
+  private static final Pattern PROCFS_MEMFILE_FORMAT =
+      Pattern.compile("^([a-zA-Z]*):[ \t]*([0-9]*)[ \t]kB");
+
+  // We need the values for the following keys in meminfo
+  private static final String MEMTOTAL_STRING = "MemTotal";
+  private static final String SWAPTOTAL_STRING = "SwapTotal";
+  private static final String MEMFREE_STRING = "MemFree";
+  private static final String SWAPFREE_STRING = "SwapFree";
+  private static final String INACTIVE_STRING = "Inactive";
+  private static final int UNAVAILABLE = -1;
+
+  /**
+   * Patterns for parsing /proc/cpuinfo
+   */
+  private static final String PROCFS_CPUINFO = "/proc/cpuinfo";
+  private static final Pattern PROCESSOR_FORMAT =
+      Pattern.compile("^processor[ \t]:[ \t]*([0-9]*)");
+  private static final Pattern FREQUENCY_FORMAT =
+      Pattern.compile("^cpu MHz[ \t]*:[ \t]*([0-9.]*)");
+
+  /**
+   * Pattern for parsing /proc/stat
+   */
+  private static final String PROCFS_STAT = "/proc/stat";
+  private static final Pattern CPU_TIME_FORMAT =
+    Pattern.compile("^cpu[ \t]*([0-9]*)" +
+    		            "[ \t]*([0-9]*)[ \t]*([0-9]*)[ \t].*");
+  
+  private String procfsMemFile;
+  private String procfsCpuFile;
+  private String procfsStatFile;
+  long jiffyLengthInMillis;
+
+  private long ramSize = 0;
+  private long swapSize = 0;
+  private long ramSizeFree = 0;  // free ram space on the machine (kB)
+  private long swapSizeFree = 0; // free swap space on the machine (kB)
+  private long inactiveSize = 0; // inactive cache memory (kB)
+  private int numProcessors = 0; // number of processors on the system
+  private long cpuFrequency = 0L; // CPU frequency on the system (kHz)
+  private long cumulativeCpuTime = 0L; // CPU used time since system is on (ms)
+  private long lastCumulativeCpuTime = 0L; // CPU used time read last time (ms)
+  // Unix timestamp while reading the CPU time (ms)
+  private float cpuUsage = UNAVAILABLE;
+  private long sampleTime = UNAVAILABLE;
+  private long lastSampleTime = UNAVAILABLE;
+  private ProcfsBasedProcessTree pTree = null;
+
+  boolean readMemInfoFile = false;
+  boolean readCpuInfoFile = false;
+  
+  /**
+   * Get current time
+   * @return Unix time stamp in millisecond
+   */
+  long getCurrentTime() {
+    return System.currentTimeMillis();
+  }
+  
+  public LinuxResourceCalculatorPlugin() {
+    procfsMemFile = PROCFS_MEMFILE;
+    procfsCpuFile = PROCFS_CPUINFO;
+    procfsStatFile = PROCFS_STAT;
+    jiffyLengthInMillis = ProcfsBasedProcessTree.JIFFY_LENGTH_IN_MILLIS;
+    String pid = System.getenv().get("JVM_PID");
+    pTree = new ProcfsBasedProcessTree(pid);
+  }
+  
+  /**
+   * Constructor which allows assigning the /proc/ directories. This will be
+   * used only in unit tests
+   * @param procfsMemFile fake file for /proc/meminfo
+   * @param procfsCpuFile fake file for /proc/cpuinfo
+   * @param procfsStatFile fake file for /proc/stat
+   * @param jiffyLengthInMillis fake jiffy length value
+   */
+  public LinuxResourceCalculatorPlugin(String procfsMemFile,
+                                       String procfsCpuFile,
+                                       String procfsStatFile,
+                                       long jiffyLengthInMillis) {
+    this.procfsMemFile = procfsMemFile;
+    this.procfsCpuFile = procfsCpuFile;
+    this.procfsStatFile = procfsStatFile;
+    this.jiffyLengthInMillis = jiffyLengthInMillis;
+    String pid = System.getenv().get("JVM_PID");
+    pTree = new ProcfsBasedProcessTree(pid);
+  }
+
+  /**
+   * Read /proc/meminfo, parse and compute memory information only once
+   */
+  private void readProcMemInfoFile() {
+    readProcMemInfoFile(false);
+  }
+
+  /**
+   * Read /proc/meminfo, parse and compute memory information
+   * @param readAgain if false, read only on the first time
+   */
+  private void readProcMemInfoFile(boolean readAgain) {
+
+    if (readMemInfoFile && !readAgain) {
+      return;
+    }
+
+    // Read "/proc/memInfo" file
+    BufferedReader in = null;
+    FileReader fReader = null;
+    try {
+      fReader = new FileReader(procfsMemFile);
+      in = new BufferedReader(fReader);
+    } catch (FileNotFoundException f) {
+      // shouldn't happen....
+      return;
+    }
+
+    Matcher mat = null;
+
+    try {
+      String str = in.readLine();
+      while (str != null) {
+        mat = PROCFS_MEMFILE_FORMAT.matcher(str);
+        if (mat.find()) {
+          if (mat.group(1).equals(MEMTOTAL_STRING)) {
+            ramSize = Long.parseLong(mat.group(2));
+          } else if (mat.group(1).equals(SWAPTOTAL_STRING)) {
+            swapSize = Long.parseLong(mat.group(2));
+          } else if (mat.group(1).equals(MEMFREE_STRING)) {
+            ramSizeFree = Long.parseLong(mat.group(2));
+          } else if (mat.group(1).equals(SWAPFREE_STRING)) {
+            swapSizeFree = Long.parseLong(mat.group(2));
+          } else if (mat.group(1).equals(INACTIVE_STRING)) {
+            inactiveSize = Long.parseLong(mat.group(2));
+          }
+        }
+        str = in.readLine();
+      }
+    } catch (IOException io) {
+      LOG.warn("Error reading the stream " + io);
+    } finally {
+      // Close the streams
+      try {
+        fReader.close();
+        try {
+          in.close();
+        } catch (IOException i) {
+          LOG.warn("Error closing the stream " + in);
+        }
+      } catch (IOException i) {
+        LOG.warn("Error closing the stream " + fReader);
+      }
+    }
+
+    readMemInfoFile = true;
+  }
+
+  /**
+   * Read /proc/cpuinfo, parse and calculate CPU information
+   */
+  private void readProcCpuInfoFile() {
+    // This directory needs to be read only once
+    if (readCpuInfoFile) {
+      return;
+    }
+    // Read "/proc/cpuinfo" file
+    BufferedReader in = null;
+    FileReader fReader = null;
+    try {
+      fReader = new FileReader(procfsCpuFile);
+      in = new BufferedReader(fReader);
+    } catch (FileNotFoundException f) {
+      // shouldn't happen....
+      return;
+    }
+    Matcher mat = null;
+    try {
+      numProcessors = 0;
+      String str = in.readLine();
+      while (str != null) {
+        mat = PROCESSOR_FORMAT.matcher(str);
+        if (mat.find()) {
+          numProcessors++;
+        }
+        mat = FREQUENCY_FORMAT.matcher(str);
+        if (mat.find()) {
+          cpuFrequency = (long)(Double.parseDouble(mat.group(1)) * 1000); // kHz
+        }
+        str = in.readLine();
+      }
+    } catch (IOException io) {
+      LOG.warn("Error reading the stream " + io);
+    } finally {
+      // Close the streams
+      try {
+        fReader.close();
+        try {
+          in.close();
+        } catch (IOException i) {
+          LOG.warn("Error closing the stream " + in);
+        }
+      } catch (IOException i) {
+        LOG.warn("Error closing the stream " + fReader);
+      }
+    }
+    readCpuInfoFile = true;
+  }
+
+  /**
+   * Read /proc/stat file, parse and calculate cumulative CPU
+   */
+  private void readProcStatFile() {
+    // Read "/proc/stat" file
+    BufferedReader in = null;
+    FileReader fReader = null;
+    try {
+      fReader = new FileReader(procfsStatFile);
+      in = new BufferedReader(fReader);
+    } catch (FileNotFoundException f) {
+      // shouldn't happen....
+      return;
+    }
+
+    Matcher mat = null;
+    try {
+      String str = in.readLine();
+      while (str != null) {
+        mat = CPU_TIME_FORMAT.matcher(str);
+        if (mat.find()) {
+          long uTime = Long.parseLong(mat.group(1));
+          long nTime = Long.parseLong(mat.group(2));
+          long sTime = Long.parseLong(mat.group(3));
+          cumulativeCpuTime = uTime + nTime + sTime; // milliseconds
+          break;
+        }
+        str = in.readLine();
+      }
+      cumulativeCpuTime *= jiffyLengthInMillis;
+    } catch (IOException io) {
+      LOG.warn("Error reading the stream " + io);
+    } finally {
+      // Close the streams
+      try {
+        fReader.close();
+        try {
+          in.close();
+        } catch (IOException i) {
+          LOG.warn("Error closing the stream " + in);
+        }
+      } catch (IOException i) {
+        LOG.warn("Error closing the stream " + fReader);
+      }
+    }
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public long getPhysicalMemorySize() {
+    readProcMemInfoFile();
+    return ramSize * 1024;
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public long getVirtualMemorySize() {
+    readProcMemInfoFile();
+    return (ramSize + swapSize) * 1024;
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public long getAvailablePhysicalMemorySize() {
+    readProcMemInfoFile(true);
+    return (ramSizeFree + inactiveSize) * 1024;
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public long getAvailableVirtualMemorySize() {
+    readProcMemInfoFile(true);
+    return (ramSizeFree + swapSizeFree + inactiveSize) * 1024;
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public int getNumProcessors() {
+    readProcCpuInfoFile();
+    return numProcessors;
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public long getCpuFrequency() {
+    readProcCpuInfoFile();
+    return cpuFrequency;
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public long getCumulativeCpuTime() {
+    readProcStatFile();
+    return cumulativeCpuTime;
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public float getCpuUsage() {
+    readProcStatFile();
+    sampleTime = getCurrentTime();
+    if (lastSampleTime == UNAVAILABLE ||
+        lastSampleTime > sampleTime) {
+      // lastSampleTime > sampleTime may happen when the system time is changed
+      lastSampleTime = sampleTime;
+      lastCumulativeCpuTime = cumulativeCpuTime;
+      return cpuUsage;
+    }
+    // When lastSampleTime is sufficiently old, update cpuUsage.
+    // Also take a sample of the current time and cumulative CPU time for the
+    // use of the next calculation.
+    final long MINIMUM_UPDATE_INTERVAL = 10 * jiffyLengthInMillis;
+    if (sampleTime > lastSampleTime + MINIMUM_UPDATE_INTERVAL) {
+	    cpuUsage = (float)(cumulativeCpuTime - lastCumulativeCpuTime) * 100F /
+	               ((float)(sampleTime - lastSampleTime) * getNumProcessors());
+	    lastSampleTime = sampleTime;
+      lastCumulativeCpuTime = cumulativeCpuTime;
+    }
+    return cpuUsage;
+  }
+
+  /**
+   * Test the {@link LinuxResourceCalculatorPlugin}
+   *
+   * @param args
+   */
+  public static void main(String[] args) {
+    LinuxResourceCalculatorPlugin plugin = new LinuxResourceCalculatorPlugin();
+    System.out.println("Physical memory Size (bytes) : "
+        + plugin.getPhysicalMemorySize());
+    System.out.println("Total Virtual memory Size (bytes) : "
+        + plugin.getVirtualMemorySize());
+    System.out.println("Available Physical memory Size (bytes) : "
+        + plugin.getAvailablePhysicalMemorySize());
+    System.out.println("Total Available Virtual memory Size (bytes) : "
+        + plugin.getAvailableVirtualMemorySize());
+    System.out.println("Number of Processors : " + plugin.getNumProcessors());
+    System.out.println("CPU frequency (kHz) : " + plugin.getCpuFrequency());
+    System.out.println("Cumulative CPU time (ms) : " +
+            plugin.getCumulativeCpuTime());
+    try {
+      // Sleep so we can compute the CPU usage
+      Thread.sleep(500L);
+    } catch (InterruptedException e) {
+      // do nothing
+    }
+    System.out.println("CPU usage % : " + plugin.getCpuUsage());
+  }
+
+  @Override
+  public ProcResourceValues getProcResourceValues() {
+    pTree = pTree.getProcessTree();
+    long cpuTime = pTree.getCumulativeCpuTime();
+    long pMem = pTree.getCumulativeRssmem();
+    long vMem = pTree.getCumulativeVmem();
+    return new ProcResourceValues(cpuTime, pMem, vMem);
+  }
+}

+ 4 - 0
src/core/org/apache/hadoop/util/MemoryCalculatorPlugin.java

@@ -24,7 +24,11 @@ import org.apache.hadoop.conf.Configured;
 /**
 /**
  * Plugin to calculate virtual and physical memories on the system.
  * Plugin to calculate virtual and physical memories on the system.
  * 
  * 
+ * @deprecated Use
+ *             {@link org.apache.hadoop.util.ResourceCalculatorPlugin}
+ *             instead
  */
  */
+@Deprecated
 public abstract class MemoryCalculatorPlugin extends Configured {
 public abstract class MemoryCalculatorPlugin extends Configured {
 
 
   /**
   /**

+ 140 - 14
src/core/org/apache/hadoop/util/ProcfsBasedProcessTree.java

@@ -33,7 +33,7 @@ import java.util.LinkedList;
 
 
 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.fs.FileUtil;
+import org.apache.hadoop.util.Shell.ShellCommandExecutor;
 
 
 /**
 /**
  * A Proc file-system based ProcessTree. Works only on Linux.
  * A Proc file-system based ProcessTree. Works only on Linux.
@@ -44,17 +44,49 @@ public class ProcfsBasedProcessTree extends ProcessTree {
       .getLog(ProcfsBasedProcessTree.class);
       .getLog(ProcfsBasedProcessTree.class);
 
 
   private static final String PROCFS = "/proc/";
   private static final String PROCFS = "/proc/";
-  private static final Pattern PROCFS_STAT_FILE_FORMAT = Pattern
-      .compile("^([0-9-]+)\\s([^\\s]+)\\s[^\\s]\\s([0-9-]+)\\s([0-9-]+)\\s([0-9-]+)\\s([0-9-]+\\s){16}([0-9]+)(\\s[0-9-]+){16}");
+  private static final Pattern PROCFS_STAT_FILE_FORMAT = Pattern .compile(
+    "^([0-9-]+)\\s([^\\s]+)\\s[^\\s]\\s([0-9-]+)\\s([0-9-]+)\\s([0-9-]+)\\s" +
+    "([0-9-]+\\s){7}([0-9]+)\\s([0-9]+)\\s([0-9-]+\\s){7}([0-9]+)\\s([0-9]+)" +
+    "(\\s[0-9-]+){15}");
 
 
   static final String PROCFS_STAT_FILE = "stat";
   static final String PROCFS_STAT_FILE = "stat";
   static final String PROCFS_CMDLINE_FILE = "cmdline";
   static final String PROCFS_CMDLINE_FILE = "cmdline";
+  public static final long PAGE_SIZE;
+  static {
+    ShellCommandExecutor shellExecutor =
+            new ShellCommandExecutor(new String[]{"getconf",  "PAGESIZE"});
+    long pageSize = -1;
+    try {
+      shellExecutor.execute();
+      pageSize = Long.parseLong(shellExecutor.getOutput().replace("\n", ""));
+    } catch (IOException e) {
+      LOG.error(StringUtils.stringifyException(e));
+    } finally {
+      PAGE_SIZE = pageSize;
+    }
+  }
+  public static final long JIFFY_LENGTH_IN_MILLIS; // in millisecond
+  static {
+    ShellCommandExecutor shellExecutor =
+            new ShellCommandExecutor(new String[]{"getconf",  "CLK_TCK"});
+    long jiffiesPerSecond = -1;
+    try {
+      shellExecutor.execute();
+      jiffiesPerSecond = Long.parseLong(shellExecutor.getOutput().replace("\n", ""));
+    } catch (IOException e) {
+      LOG.error(StringUtils.stringifyException(e));
+    } finally {
+      JIFFY_LENGTH_IN_MILLIS = jiffiesPerSecond != -1 ?
+                     Math.round(1000D / jiffiesPerSecond) : -1;
+    }
+  }
 
 
   // to enable testing, using this variable which can be configured
   // to enable testing, using this variable which can be configured
   // to a test directory.
   // to a test directory.
   private String procfsDir;
   private String procfsDir;
   
   
   private Integer pid = -1;
   private Integer pid = -1;
+  private Long cpuTime = 0L;
 
 
   private Map<Integer, ProcessInfo> processTree = new HashMap<Integer, ProcessInfo>();
   private Map<Integer, ProcessInfo> processTree = new HashMap<Integer, ProcessInfo>();
 
 
@@ -149,11 +181,12 @@ public class ProcfsBasedProcessTree extends ProcessTree {
         pInfoQueue.addAll(pInfo.getChildren());
         pInfoQueue.addAll(pInfo.getChildren());
       }
       }
 
 
-      // update age values.
+      // update age values and compute the number of jiffies since last update
       for (Map.Entry<Integer, ProcessInfo> procs : processTree.entrySet()) {
       for (Map.Entry<Integer, ProcessInfo> procs : processTree.entrySet()) {
         ProcessInfo oldInfo = oldProcs.get(procs.getKey());
         ProcessInfo oldInfo = oldProcs.get(procs.getKey());
-        if (oldInfo != null) {
-          if (procs.getValue() != null) {
+        if (procs.getValue() != null) {
+          procs.getValue().updateJiffy(oldInfo);
+          if (oldInfo != null) {
             procs.getValue().updateAge(oldInfo);  
             procs.getValue().updateAge(oldInfo);  
           }
           }
         }
         }
@@ -196,7 +229,7 @@ public class ProcfsBasedProcessTree extends ProcessTree {
   }
   }
 
 
   private static final String PROCESSTREE_DUMP_FORMAT =
   private static final String PROCESSTREE_DUMP_FORMAT =
-      "\t|- %d %d %d %d %s %d %s\n";
+      "\t|- %d %d %d %d %s %d %d %d %d %s\n";
 
 
   /**
   /**
    * Get a dump of the process-tree.
    * Get a dump of the process-tree.
@@ -208,12 +241,14 @@ public class ProcfsBasedProcessTree extends ProcessTree {
     StringBuilder ret = new StringBuilder();
     StringBuilder ret = new StringBuilder();
     // The header.
     // The header.
     ret.append(String.format("\t|- PID PPID PGRPID SESSID CMD_NAME "
     ret.append(String.format("\t|- PID PPID PGRPID SESSID CMD_NAME "
-        + "VMEM_USAGE(BYTES) FULL_CMD_LINE\n"));
+        + "USER_MODE_TIME(MILLIS) SYSTEM_TIME(MILLIS) VMEM_USAGE(BYTES) "
+        + "RSSMEM_USAGE(PAGES) FULL_CMD_LINE\n"));
     for (ProcessInfo p : processTree.values()) {
     for (ProcessInfo p : processTree.values()) {
       if (p != null) {
       if (p != null) {
         ret.append(String.format(PROCESSTREE_DUMP_FORMAT, p.getPid(), p
         ret.append(String.format(PROCESSTREE_DUMP_FORMAT, p.getPid(), p
             .getPpid(), p.getPgrpId(), p.getSessionId(), p.getName(), p
             .getPpid(), p.getPgrpId(), p.getSessionId(), p.getName(), p
-            .getVmem(), p.getCmdLine(procfsDir)));
+            .getUtime(), p.getStime(), p.getVmem(), p.getRssmemPage(), p
+            .getCmdLine(procfsDir)));
       }
       }
     }
     }
     return ret.toString();
     return ret.toString();
@@ -230,6 +265,18 @@ public class ProcfsBasedProcessTree extends ProcessTree {
     return getCumulativeVmem(0);
     return getCumulativeVmem(0);
   }
   }
 
 
+  /**
+   * Get the cumulative resident set size (rss) memory used by all the processes
+   * in the process-tree.
+   *
+   * @return cumulative rss memory used by the process-tree in bytes. return 0
+   *         if it cannot be calculated
+   */
+  public long getCumulativeRssmem() {
+    // include all processes.. all processes will be older than 0.
+    return getCumulativeRssmem(0);
+  }
+
   /**
   /**
    * Get the cumulative virtual memory used by all the processes in the
    * Get the cumulative virtual memory used by all the processes in the
    * process-tree that are older than the passed in age.
    * process-tree that are older than the passed in age.
@@ -249,6 +296,50 @@ public class ProcfsBasedProcessTree extends ProcessTree {
     return total;
     return total;
   }
   }
 
 
+  /**
+   * Get the cumulative resident set size (rss) memory used by all the processes
+   * in the process-tree that are older than the passed in age.
+   *
+   * @param olderThanAge processes above this age are included in the
+   *                      memory addition
+   * @return cumulative rss memory used by the process-tree in bytes,
+   *          for processes older than this age. return 0 if it cannot be
+   *          calculated
+   */
+  public long getCumulativeRssmem(int olderThanAge) {
+    if (PAGE_SIZE < 0) {
+      return 0;
+    }
+    long totalPages = 0;
+    for (ProcessInfo p : processTree.values()) {
+      if ((p != null) && (p.getAge() > olderThanAge)) {
+        totalPages += p.getRssmemPage();
+      }
+    }
+    return totalPages * PAGE_SIZE; // convert # pages to byte
+  }
+
+  /**
+   * Get the CPU time in millisecond used by all the processes in the
+   * process-tree since the process-tree created
+   *
+   * @return cumulative CPU time in millisecond since the process-tree created
+   *         return 0 if it cannot be calculated
+   */
+  public long getCumulativeCpuTime() {
+    if (JIFFY_LENGTH_IN_MILLIS < 0) {
+      return 0;
+    }
+    long incJiffies = 0;
+    for (ProcessInfo p : processTree.values()) {
+      if (p != null) {
+        incJiffies += p.dtime;
+      }
+    }
+    cpuTime += incJiffies * JIFFY_LENGTH_IN_MILLIS;
+    return cpuTime;
+  }
+
   private static Integer getValidPID(String pid) {
   private static Integer getValidPID(String pid) {
     Integer retPid = -1;
     Integer retPid = -1;
     try {
     try {
@@ -318,10 +409,11 @@ public class ProcfsBasedProcessTree extends ProcessTree {
       Matcher m = PROCFS_STAT_FILE_FORMAT.matcher(str);
       Matcher m = PROCFS_STAT_FILE_FORMAT.matcher(str);
       boolean mat = m.find();
       boolean mat = m.find();
       if (mat) {
       if (mat) {
-        // Set ( name ) ( ppid ) ( pgrpId ) (session ) (vsize )
-        pinfo.updateProcessInfo(m.group(2), Integer.parseInt(m.group(3)), Integer
-            .parseInt(m.group(4)), Integer.parseInt(m.group(5)), Long
-            .parseLong(m.group(7)));
+        // Set (name) (ppid) (pgrpId) (session) (utime) (stime) (vsize) (rss)
+         pinfo.updateProcessInfo(m.group(2), Integer.parseInt(m.group(3)),
+                 Integer.parseInt(m.group(4)), Integer.parseInt(m.group(5)),
+                 Long.parseLong(m.group(7)), Long.parseLong(m.group(8)),
+                 Long.parseLong(m.group(10)), Long.parseLong(m.group(11)));
       }
       }
     } catch (IOException io) {
     } catch (IOException io) {
       LOG.warn("Error reading the stream " + io);
       LOG.warn("Error reading the stream " + io);
@@ -368,8 +460,18 @@ public class ProcfsBasedProcessTree extends ProcessTree {
     private Integer ppid; // parent process-id
     private Integer ppid; // parent process-id
     private Integer sessionId; // session-id
     private Integer sessionId; // session-id
     private Long vmem; // virtual memory usage
     private Long vmem; // virtual memory usage
+    private Long rssmemPage; // rss memory usage in # of pages
+    private Long utime = 0L; // # of jiffies in user mode
+    private Long stime = 0L; // # of jiffies in kernel mode
     // how many times has this process been seen alive
     // how many times has this process been seen alive
     private int age; 
     private int age; 
+
+    // # of jiffies used since last update:
+    private Long dtime = 0L;
+    // dtime = (utime + stime) - (utimeOld + stimeOld)
+    // We need this to compute the cumulative CPU time
+    // because the subprocess may finish earlier than root process
+
     private List<ProcessInfo> children = new ArrayList<ProcessInfo>(); // list of children
     private List<ProcessInfo> children = new ArrayList<ProcessInfo>(); // list of children
 
 
     public ProcessInfo(int pid) {
     public ProcessInfo(int pid) {
@@ -402,17 +504,41 @@ public class ProcfsBasedProcessTree extends ProcessTree {
       return vmem;
       return vmem;
     }
     }
 
 
+    public Long getUtime() {
+      return utime;
+    }
+
+    public Long getStime() {
+      return stime;
+    }
+
+    public Long getDtime() {
+      return dtime;
+    }
+
+    public Long getRssmemPage() { // get rss # of pages
+      return rssmemPage;
+    }
+
     public int getAge() {
     public int getAge() {
       return age;
       return age;
     }
     }
     
     
     public void updateProcessInfo(String name, Integer ppid, Integer pgrpId,
     public void updateProcessInfo(String name, Integer ppid, Integer pgrpId,
-        Integer sessionId, Long vmem) {
+        Integer sessionId, Long utime, Long stime, Long vmem, Long rssmem) {
       this.name = name;
       this.name = name;
       this.ppid = ppid;
       this.ppid = ppid;
       this.pgrpId = pgrpId;
       this.pgrpId = pgrpId;
       this.sessionId = sessionId;
       this.sessionId = sessionId;
+      this.utime = utime;
+      this.stime = stime;
       this.vmem = vmem;
       this.vmem = vmem;
+      this.rssmemPage = rssmem;
+    }
+
+    public void updateJiffy(ProcessInfo oldInfo) {
+      this.dtime = (oldInfo == null ? this.utime + this.stime
+              : (this.utime + this.stime) - (oldInfo.utime + oldInfo.stime));
     }
     }
 
 
     public void updateAge(ProcessInfo oldInfo) {
     public void updateAge(ProcessInfo oldInfo) {

+ 159 - 0
src/core/org/apache/hadoop/util/ResourceCalculatorPlugin.java

@@ -0,0 +1,159 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.util;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.util.ReflectionUtils;
+
+/**
+ * Plugin to calculate resource information on the system.
+ * 
+ */
+public abstract class ResourceCalculatorPlugin extends Configured {
+
+  /**
+   * Obtain the total size of the virtual memory present in the system.
+   * 
+   * @return virtual memory size in bytes.
+   */
+  public abstract long getVirtualMemorySize();
+
+  /**
+   * Obtain the total size of the physical memory present in the system.
+   * 
+   * @return physical memory size bytes.
+   */
+  public abstract long getPhysicalMemorySize();
+
+  /**
+   * Obtain the total size of the available virtual memory present
+   * in the system.
+   *
+   * @return available virtual memory size in bytes.
+   */
+  public abstract long getAvailableVirtualMemorySize();
+
+  /**
+   * Obtain the total size of the available physical memory present
+   * in the system.
+   *
+   * @return available physical memory size bytes.
+   */
+  public abstract long getAvailablePhysicalMemorySize();
+
+  /**
+   * Obtain the total number of processors present on the system.
+   *
+   * @return number of processors
+   */
+  public abstract int getNumProcessors();
+
+  /**
+   * Obtain the CPU frequency of on the system.
+   *
+   * @return CPU frequency in kHz
+   */
+  public abstract long getCpuFrequency();
+
+  /**
+   * Obtain the cumulative CPU time since the system is on.
+   *
+   * @return cumulative CPU time in milliseconds
+   */
+  public abstract long getCumulativeCpuTime();
+
+  /**
+   * Obtain the CPU usage % of the machine. Return -1 if it is unavailable
+   *
+   * @return CPU usage in %
+   */
+  public abstract float getCpuUsage();
+
+  /**
+   * Obtain resource status used by current process tree.
+   */
+  public abstract ProcResourceValues getProcResourceValues();
+
+  public static class ProcResourceValues {
+    private final long cumulativeCpuTime;
+    private final long physicalMemorySize;
+    private final long virtualMemorySize;
+    public ProcResourceValues(long cumulativeCpuTime, long physicalMemorySize,
+                              long virtualMemorySize) {
+      this.cumulativeCpuTime = cumulativeCpuTime;
+      this.physicalMemorySize = physicalMemorySize;
+      this.virtualMemorySize = virtualMemorySize;
+    }
+    /**
+     * Obtain the physical memory size used by current process tree.
+     * @return physical memory size in bytes.
+     */
+    public long getPhysicalMemorySize() {
+      return physicalMemorySize;
+    }
+
+    /**
+     * Obtain the virtual memory size used by a current process tree.
+     * @return virtual memory size in bytes.
+     */
+    public long getVirtualMemorySize() {
+      return virtualMemorySize;
+    }
+
+    /**
+     * Obtain the cumulative CPU time used by a current process tree.
+     * @return cumulative CPU time in milliseconds
+     */
+    public long getCumulativeCpuTime() {
+      return cumulativeCpuTime;
+    }
+  }
+
+  /**
+   * Get the ResourceCalculatorPlugin from the class name and configure it. If
+   * class name is null, this method will try and return a memory calculator
+   * plugin available for this system.
+   * 
+   * @param clazz class-name
+   * @param conf configure the plugin with this.
+   * @return ResourceCalculatorPlugin
+   */
+  public static ResourceCalculatorPlugin getResourceCalculatorPlugin(
+      Class<? extends ResourceCalculatorPlugin> clazz, Configuration conf) {
+
+    if (clazz != null) {
+      return ReflectionUtils.newInstance(clazz, conf);
+    }
+
+    // No class given, try a os specific class
+    try {
+      String osName = System.getProperty("os.name");
+      if (osName.startsWith("Linux")) {
+        return new LinuxResourceCalculatorPlugin();
+      }
+    } catch (SecurityException se) {
+      // Failed to get Operating System name.
+      return null;
+    }
+
+    // Not supported on this system.
+    return null;
+  }
+}

+ 3 - 3
src/mapred/mapred-default.xml

@@ -186,14 +186,14 @@
 -->
 -->
 
 
 <property>
 <property>
-  <name>mapred.tasktracker.memory_calculator_plugin</name>
+  <name>mapred.tasktracker.resourcecalculatorplugin</name>
   <value></value>
   <value></value>
   <description>
   <description>
-   Name of the class whose instance will be used to query memory information
+   Name of the class whose instance will be used to query resource information
    on the tasktracker.
    on the tasktracker.
    
    
    The class must be an instance of 
    The class must be an instance of 
-   org.apache.hadoop.util.MemoryCalculatorPlugin. If the value is null, the
+   org.apache.hadoop.util.ResourceCalculatorPlugin. If the value is null, the
    tasktracker attempts to use a class appropriate to the platform. 
    tasktracker attempts to use a class appropriate to the platform. 
    Currently, the only platform supported is Linux.
    Currently, the only platform supported is Linux.
   </description>
   </description>

+ 3 - 1
src/mapred/org/apache/hadoop/mapred/InterTrackerProtocol.java

@@ -69,8 +69,10 @@ interface InterTrackerProtocol extends VersionedProtocol {
    * Version 26: Added numRequiredSlots to TaskStatus for MAPREDUCE-516
    * Version 26: Added numRequiredSlots to TaskStatus for MAPREDUCE-516
    * Version 27: Adding node health status to TaskStatus for MAPREDUCE-211
    * Version 27: Adding node health status to TaskStatus for MAPREDUCE-211
    * Version 28: Adding user name to the serialized Task for use by TT.
    * Version 28: Adding user name to the serialized Task for use by TT.
+   * Version 29: Adding available memory and CPU usage information on TT to
+   *             TaskTrackerStatus for MAPREDUCE-1218
    */
    */
-  public static final long versionID = 28L;
+  public static final long versionID = 29L;
   
   
   public final static int TRACKERS_OK = 0;
   public final static int TRACKERS_OK = 0;
   public final static int UNKNOWN_TASKTRACKER = 1;
   public final static int UNKNOWN_TASKTRACKER = 1;

+ 2 - 2
src/mapred/org/apache/hadoop/mapred/JobTracker.java

@@ -4468,9 +4468,9 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
     }
     }
   }
   }
   
   
-  static final String MAPRED_CLUSTER_MAP_MEMORY_MB_PROPERTY =
+  public static final String MAPRED_CLUSTER_MAP_MEMORY_MB_PROPERTY =
       "mapred.cluster.map.memory.mb";
       "mapred.cluster.map.memory.mb";
-  static final String MAPRED_CLUSTER_REDUCE_MEMORY_MB_PROPERTY =
+  public static final String MAPRED_CLUSTER_REDUCE_MEMORY_MB_PROPERTY =
       "mapred.cluster.reduce.memory.mb";
       "mapred.cluster.reduce.memory.mb";
 
 
   static final String MAPRED_CLUSTER_MAX_MAP_MEMORY_MB_PROPERTY =
   static final String MAPRED_CLUSTER_MAX_MAP_MEMORY_MB_PROPERTY =

+ 55 - 2
src/mapred/org/apache/hadoop/mapred/Task.java

@@ -50,10 +50,11 @@ import org.apache.hadoop.io.serializer.SerializationFactory;
 import org.apache.hadoop.mapred.IFile.Writer;
 import org.apache.hadoop.mapred.IFile.Writer;
 import org.apache.hadoop.mapreduce.JobStatus;
 import org.apache.hadoop.mapreduce.JobStatus;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.util.Progress;
 import org.apache.hadoop.util.Progress;
 import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.util.ResourceCalculatorPlugin;
+import org.apache.hadoop.util.ResourceCalculatorPlugin.ProcResourceValues;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataInputStream;
 
 
@@ -85,7 +86,11 @@ abstract public class Task implements Writable, Configurable {
     REDUCE_SKIPPED_GROUPS,
     REDUCE_SKIPPED_GROUPS,
     REDUCE_SKIPPED_RECORDS,
     REDUCE_SKIPPED_RECORDS,
     SPILLED_RECORDS,
     SPILLED_RECORDS,
-    SPLIT_RAW_BYTES
+    SPLIT_RAW_BYTES,
+    CPU_MILLISECONDS,
+    PHYSICAL_MEMORY_BYTES,
+    VIRTUAL_MEMORY_BYTES,
+    COMMITTED_HEAP_BYTES
   }
   }
   
   
   /**
   /**
@@ -143,6 +148,9 @@ abstract public class Task implements Writable, Configurable {
   private Iterator<Long> currentRecIndexIterator = 
   private Iterator<Long> currentRecIndexIterator = 
     skipRanges.skipRangeIterator();
     skipRanges.skipRangeIterator();
   
   
+  private ResourceCalculatorPlugin resourceCalculator = null;
+  private long initCpuCumulativeTime = 0;
+
   protected JobConf conf;
   protected JobConf conf;
   protected MapOutputFile mapOutputFile = new MapOutputFile();
   protected MapOutputFile mapOutputFile = new MapOutputFile();
   protected LocalDirAllocator lDirAlloc;
   protected LocalDirAllocator lDirAlloc;
@@ -518,6 +526,16 @@ abstract public class Task implements Writable, Configurable {
       }
       }
     }
     }
     committer.setupTask(taskContext);
     committer.setupTask(taskContext);
+    Class<? extends ResourceCalculatorPlugin> clazz =
+        conf.getClass(TaskTracker.TT_RESOURCE_CALCULATOR_PLUGIN,
+            null, ResourceCalculatorPlugin.class);
+    resourceCalculator = ResourceCalculatorPlugin
+            .getResourceCalculatorPlugin(clazz, conf);
+    LOG.info(" Using ResourceCalculatorPlugin : " + resourceCalculator);
+    if (resourceCalculator != null) {
+      initCpuCumulativeTime =
+        resourceCalculator.getProcResourceValues().getCumulativeCpuTime();
+    }
   }
   }
   
   
   protected class TaskReporter 
   protected class TaskReporter 
@@ -698,6 +716,7 @@ abstract public class Task implements Writable, Configurable {
       }
       }
     }
     }
     public void stopCommunicationThread() throws InterruptedException {
     public void stopCommunicationThread() throws InterruptedException {
+      // Updating resources specified in ResourceCalculatorPlugin
       if (pingThread != null) {
       if (pingThread != null) {
         synchronized(lock) {
         synchronized(lock) {
           while(!done) {
           while(!done) {
@@ -776,6 +795,27 @@ abstract public class Task implements Writable, Configurable {
   private Map<String, FileSystemStatisticUpdater> statisticUpdaters =
   private Map<String, FileSystemStatisticUpdater> statisticUpdaters =
      new HashMap<String, FileSystemStatisticUpdater>();
      new HashMap<String, FileSystemStatisticUpdater>();
   
   
+  /**
+   * Update resource information counters
+   */
+   void updateResourceCounters() {
+     // Update generic resource counters
+     updateHeapUsageCounter();
+     
+     if (resourceCalculator == null) {
+       return;
+     }
+     ProcResourceValues res = resourceCalculator.getProcResourceValues();
+     long cpuTime = res.getCumulativeCpuTime();
+     long pMem = res.getPhysicalMemorySize();
+     long vMem = res.getVirtualMemorySize();
+     // Remove the CPU time consumed previously by JVM reuse
+     cpuTime -= initCpuCumulativeTime;
+     counters.findCounter(Counter.CPU_MILLISECONDS).setValue(cpuTime);
+     counters.findCounter(Counter.PHYSICAL_MEMORY_BYTES).setValue(pMem);
+     counters.findCounter(Counter.VIRTUAL_MEMORY_BYTES).setValue(vMem);
+   }
+  
   private synchronized void updateCounters() {
   private synchronized void updateCounters() {
     for(Statistics stat: FileSystem.getAllStatistics()) {
     for(Statistics stat: FileSystem.getAllStatistics()) {
       String uriScheme = stat.getScheme();
       String uriScheme = stat.getScheme();
@@ -786,6 +826,19 @@ abstract public class Task implements Writable, Configurable {
       }
       }
       updater.updateCounters();      
       updater.updateCounters();      
     }
     }
+    // TODO Should CPU related counters be update only once i.e in the end
+    updateResourceCounters();
+  }
+
+  /**
+   * Updates the {@link TaskCounter#COMMITTED_HEAP_BYTES} counter to reflect the
+   * current total committed heap space usage of this JVM.
+   */
+  @SuppressWarnings("deprecation")
+  private void updateHeapUsageCounter() {
+    long currentHeapUsage = Runtime.getRuntime().totalMemory();
+    counters.findCounter(Counter.COMMITTED_HEAP_BYTES)
+            .setValue(currentHeapUsage);
   }
   }
 
 
   public void done(TaskUmbilicalProtocol umbilical,
   public void done(TaskUmbilicalProtocol umbilical,

+ 115 - 13
src/mapred/org/apache/hadoop/mapred/TaskTracker.java

@@ -92,6 +92,7 @@ import org.apache.hadoop.security.authorize.PolicyProvider;
 import org.apache.hadoop.security.authorize.ServiceAuthorizationManager;
 import org.apache.hadoop.security.authorize.ServiceAuthorizationManager;
 import org.apache.hadoop.util.DiskChecker;
 import org.apache.hadoop.util.DiskChecker;
 import org.apache.hadoop.util.MemoryCalculatorPlugin;
 import org.apache.hadoop.util.MemoryCalculatorPlugin;
+import org.apache.hadoop.util.ResourceCalculatorPlugin;
 import org.apache.hadoop.util.ProcfsBasedProcessTree;
 import org.apache.hadoop.util.ProcfsBasedProcessTree;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.security.token.TokenIdentifier;
@@ -347,11 +348,14 @@ public class TaskTracker implements MRConstants, TaskUmbilicalProtocol,
   private long mapSlotMemorySizeOnTT = JobConf.DISABLED_MEMORY_LIMIT;
   private long mapSlotMemorySizeOnTT = JobConf.DISABLED_MEMORY_LIMIT;
   private long reduceSlotSizeMemoryOnTT = JobConf.DISABLED_MEMORY_LIMIT;
   private long reduceSlotSizeMemoryOnTT = JobConf.DISABLED_MEMORY_LIMIT;
   private long totalMemoryAllottedForTasks = JobConf.DISABLED_MEMORY_LIMIT;
   private long totalMemoryAllottedForTasks = JobConf.DISABLED_MEMORY_LIMIT;
+  private ResourceCalculatorPlugin resourceCalculatorPlugin = null;
 
 
   private UserLogManager userLogManager;
   private UserLogManager userLogManager;
 
 
   static final String MAPRED_TASKTRACKER_MEMORY_CALCULATOR_PLUGIN_PROPERTY =
   static final String MAPRED_TASKTRACKER_MEMORY_CALCULATOR_PLUGIN_PROPERTY =
       "mapred.tasktracker.memory_calculator_plugin";
       "mapred.tasktracker.memory_calculator_plugin";
+  public static final String TT_RESOURCE_CALCULATOR_PLUGIN = 
+      "mapreduce.tasktracker.resourcecalculatorplugin";
 
 
   /**
   /**
    * the minimum interval between jobtracker polls
    * the minimum interval between jobtracker polls
@@ -802,6 +806,12 @@ public class TaskTracker implements MRConstants, TaskUmbilicalProtocol,
                              taskTrackerName);
                              taskTrackerName);
     mapEventsFetcher.start();
     mapEventsFetcher.start();
 
 
+    Class<? extends ResourceCalculatorPlugin> clazz =
+        fConf.getClass(TT_RESOURCE_CALCULATOR_PLUGIN,
+                       null, ResourceCalculatorPlugin.class);
+    resourceCalculatorPlugin = 
+      ResourceCalculatorPlugin.getResourceCalculatorPlugin(clazz, fConf);
+    LOG.info(" Using ResourceCalculatorPlugin : " + resourceCalculatorPlugin);
     initializeMemoryManagement();
     initializeMemoryManagement();
 
 
     getUserLogManager().clearOldUserLogs(fConf);
     getUserLogManager().clearOldUserLogs(fConf);
@@ -1739,6 +1749,12 @@ public class TaskTracker implements MRConstants, TaskUmbilicalProtocol,
       long freeDiskSpace = getFreeSpace();
       long freeDiskSpace = getFreeSpace();
       long totVmem = getTotalVirtualMemoryOnTT();
       long totVmem = getTotalVirtualMemoryOnTT();
       long totPmem = getTotalPhysicalMemoryOnTT();
       long totPmem = getTotalPhysicalMemoryOnTT();
+      long availableVmem = getAvailableVirtualMemoryOnTT();
+      long availablePmem = getAvailablePhysicalMemoryOnTT();
+      long cumuCpuTime = getCumulativeCpuTimeOnTT();
+      long cpuFreq = getCpuFrequencyOnTT();
+      int numCpu = getNumProcessorsOnTT();
+      float cpuUsage = getCpuUsageOnTT();
 
 
       status.getResourceStatus().setAvailableSpace(freeDiskSpace);
       status.getResourceStatus().setAvailableSpace(freeDiskSpace);
       status.getResourceStatus().setTotalVirtualMemory(totVmem);
       status.getResourceStatus().setTotalVirtualMemory(totVmem);
@@ -1747,6 +1763,12 @@ public class TaskTracker implements MRConstants, TaskUmbilicalProtocol,
           mapSlotMemorySizeOnTT);
           mapSlotMemorySizeOnTT);
       status.getResourceStatus().setReduceSlotMemorySizeOnTT(
       status.getResourceStatus().setReduceSlotMemorySizeOnTT(
           reduceSlotSizeMemoryOnTT);
           reduceSlotSizeMemoryOnTT);
+      status.getResourceStatus().setAvailableVirtualMemory(availableVmem); 
+      status.getResourceStatus().setAvailablePhysicalMemory(availablePmem);
+      status.getResourceStatus().setCumulativeCpuTime(cumuCpuTime);
+      status.getResourceStatus().setCpuFrequency(cpuFreq);
+      status.getResourceStatus().setNumProcessors(numCpu);
+      status.getResourceStatus().setCpuUsage(cpuUsage);
     }
     }
     //add node health information
     //add node health information
     
     
@@ -1819,6 +1841,80 @@ public class TaskTracker implements MRConstants, TaskUmbilicalProtocol,
     return totalPhysicalMemoryOnTT;
     return totalPhysicalMemoryOnTT;
   }
   }
 
 
+  /**
+   * Return the free virtual memory available on this TaskTracker.
+   * @return total size of free virtual memory.
+   */
+  long getAvailableVirtualMemoryOnTT() {
+    long availableVirtualMemoryOnTT = TaskTrackerStatus.UNAVAILABLE;
+    if (resourceCalculatorPlugin != null) {
+      availableVirtualMemoryOnTT =
+              resourceCalculatorPlugin.getAvailableVirtualMemorySize();
+    }
+    return availableVirtualMemoryOnTT;
+  }
+
+  /**
+   * Return the free physical memory available on this TaskTracker.
+   * @return total size of free physical memory in bytes
+   */
+  long getAvailablePhysicalMemoryOnTT() {
+    long availablePhysicalMemoryOnTT = TaskTrackerStatus.UNAVAILABLE;
+    if (resourceCalculatorPlugin != null) {
+      availablePhysicalMemoryOnTT =
+              resourceCalculatorPlugin.getAvailablePhysicalMemorySize();
+    }
+    return availablePhysicalMemoryOnTT;
+  }
+
+  /**
+   * Return the cumulative CPU used time on this TaskTracker since system is on
+   * @return cumulative CPU used time in millisecond
+   */
+  long getCumulativeCpuTimeOnTT() {
+    long cumulativeCpuTime = TaskTrackerStatus.UNAVAILABLE;
+    if (resourceCalculatorPlugin != null) {
+      cumulativeCpuTime = resourceCalculatorPlugin.getCumulativeCpuTime();
+    }
+    return cumulativeCpuTime;
+  }
+
+  /**
+   * Return the number of Processors on this TaskTracker
+   * @return number of processors
+   */
+  int getNumProcessorsOnTT() {
+    int numProcessors = TaskTrackerStatus.UNAVAILABLE;
+    if (resourceCalculatorPlugin != null) {
+      numProcessors = resourceCalculatorPlugin.getNumProcessors();
+    }
+    return numProcessors;
+  }
+
+  /**
+   * Return the CPU frequency of this TaskTracker
+   * @return CPU frequency in kHz
+   */
+  long getCpuFrequencyOnTT() {
+    long cpuFrequency = TaskTrackerStatus.UNAVAILABLE;
+    if (resourceCalculatorPlugin != null) {
+      cpuFrequency = resourceCalculatorPlugin.getCpuFrequency();
+    }
+    return cpuFrequency;
+  }
+
+  /**
+   * Return the CPU usage in % of this TaskTracker
+   * @return CPU usage in %
+   */
+  float getCpuUsageOnTT() {
+    float cpuUsage = TaskTrackerStatus.UNAVAILABLE;
+    if (resourceCalculatorPlugin != null) {
+      cpuUsage = resourceCalculatorPlugin.getCpuUsage();
+    }
+    return cpuUsage;
+  }
+  
   long getTotalMemoryAllottedForTasksOnTT() {
   long getTotalMemoryAllottedForTasksOnTT() {
     return totalMemoryAllottedForTasks;
     return totalMemoryAllottedForTasks;
   }
   }
@@ -3974,25 +4070,31 @@ public class TaskTracker implements MRConstants, TaskUmbilicalProtocol,
           JobConf.UPPER_LIMIT_ON_TASK_VMEM_PROPERTY));
           JobConf.UPPER_LIMIT_ON_TASK_VMEM_PROPERTY));
     }
     }
 
 
-    Class<? extends MemoryCalculatorPlugin> clazz =
-        fConf.getClass(MAPRED_TASKTRACKER_MEMORY_CALCULATOR_PLUGIN_PROPERTY,
-            null, MemoryCalculatorPlugin.class);
-    MemoryCalculatorPlugin memoryCalculatorPlugin =
-        MemoryCalculatorPlugin
-            .getMemoryCalculatorPlugin(clazz, fConf);
-    LOG.info(" Using MemoryCalculatorPlugin : " + memoryCalculatorPlugin);
-
-    if (memoryCalculatorPlugin != null) {
-      totalVirtualMemoryOnTT = memoryCalculatorPlugin.getVirtualMemorySize();
+    // Use TT_RESOURCE_CALCULATOR_PLUGIN if it is configured.
+    Class<? extends MemoryCalculatorPlugin> clazz = 
+      fConf.getClass(MAPRED_TASKTRACKER_MEMORY_CALCULATOR_PLUGIN_PROPERTY, 
+                     null, MemoryCalculatorPlugin.class); 
+    MemoryCalculatorPlugin memoryCalculatorPlugin = 
+      (clazz == null 
+       ? null 
+       : MemoryCalculatorPlugin.getMemoryCalculatorPlugin(clazz, fConf)); 
+    if (memoryCalculatorPlugin != null || resourceCalculatorPlugin != null) {
+      totalVirtualMemoryOnTT = 
+        (memoryCalculatorPlugin == null 
+         ? resourceCalculatorPlugin.getVirtualMemorySize() 
+         : memoryCalculatorPlugin.getVirtualMemorySize());
       if (totalVirtualMemoryOnTT <= 0) {
       if (totalVirtualMemoryOnTT <= 0) {
         LOG.warn("TaskTracker's totalVmem could not be calculated. "
         LOG.warn("TaskTracker's totalVmem could not be calculated. "
-            + "Setting it to " + JobConf.DISABLED_MEMORY_LIMIT);
+                 + "Setting it to " + JobConf.DISABLED_MEMORY_LIMIT);
         totalVirtualMemoryOnTT = JobConf.DISABLED_MEMORY_LIMIT;
         totalVirtualMemoryOnTT = JobConf.DISABLED_MEMORY_LIMIT;
       }
       }
-      totalPhysicalMemoryOnTT = memoryCalculatorPlugin.getPhysicalMemorySize();
+      totalPhysicalMemoryOnTT = 
+        (memoryCalculatorPlugin == null 
+         ? resourceCalculatorPlugin.getPhysicalMemorySize() 
+         : memoryCalculatorPlugin.getPhysicalMemorySize());
       if (totalPhysicalMemoryOnTT <= 0) {
       if (totalPhysicalMemoryOnTT <= 0) {
         LOG.warn("TaskTracker's totalPmem could not be calculated. "
         LOG.warn("TaskTracker's totalPmem could not be calculated. "
-            + "Setting it to " + JobConf.DISABLED_MEMORY_LIMIT);
+                 + "Setting it to " + JobConf.DISABLED_MEMORY_LIMIT);
         totalPhysicalMemoryOnTT = JobConf.DISABLED_MEMORY_LIMIT;
         totalPhysicalMemoryOnTT = JobConf.DISABLED_MEMORY_LIMIT;
       }
       }
     }
     }

+ 4227 - 0
src/mapred/org/apache/hadoop/mapred/TaskTracker.java.orig

@@ -0,0 +1,4227 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.mapred;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.io.RandomAccessFile;
+import java.net.InetSocketAddress;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.security.PrivilegedExceptionAction;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.Vector;
+import java.util.Map.Entry;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.regex.Pattern;
+
+import javax.crypto.SecretKey;
+import javax.servlet.ServletContext;
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.filecache.TaskDistributedCacheManager;
+import org.apache.hadoop.filecache.TrackerDistributedCacheManager;
+import org.apache.hadoop.mapreduce.server.tasktracker.*;
+import org.apache.hadoop.mapreduce.server.tasktracker.userlogs.*;
+import org.apache.hadoop.fs.DF;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.LocalDirAllocator;
+import org.apache.hadoop.fs.LocalFileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.http.HttpServer;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.SecureIOUtils;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.ipc.Server;
+import org.apache.hadoop.mapred.CleanupQueue.PathDeletionContext;
+import org.apache.hadoop.mapred.TaskLog.LogFileDetail;
+import org.apache.hadoop.mapred.TaskLog.LogName;
+import org.apache.hadoop.mapred.TaskStatus.Phase;
+import org.apache.hadoop.mapred.TaskTrackerStatus.TaskTrackerHealthStatus;
+import org.apache.hadoop.mapred.pipes.Submitter;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.hadoop.mapreduce.security.SecureShuffleUtils;
+import org.apache.hadoop.mapreduce.security.token.JobTokenIdentifier;
+import org.apache.hadoop.mapreduce.security.token.JobTokenSecretManager;
+import org.apache.hadoop.net.DNS;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authorize.PolicyProvider;
+import org.apache.hadoop.security.authorize.ServiceAuthorizationManager;
+import org.apache.hadoop.util.DiskChecker;
+import org.apache.hadoop.util.MemoryCalculatorPlugin;
+import org.apache.hadoop.util.ProcfsBasedProcessTree;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.util.VersionInfo;
+import org.apache.hadoop.util.DiskChecker.DiskErrorException;
+import org.apache.hadoop.util.Shell.ShellCommandExecutor;
+import org.apache.hadoop.mapreduce.security.TokenCache;
+import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
+import org.apache.hadoop.metrics2.util.MBeans;
+import org.apache.hadoop.security.Credentials;
+
+/*******************************************************
+ * TaskTracker is a process that starts and tracks MR Tasks
+ * in a networked environment.  It contacts the JobTracker
+ * for Task assignments and reporting results.
+ *
+ *******************************************************/
+public class TaskTracker implements MRConstants, TaskUmbilicalProtocol,
+    Runnable, TaskTrackerMXBean {
+  
+  /**
+   * @deprecated
+   */
+  @Deprecated
+  static final String MAPRED_TASKTRACKER_VMEM_RESERVED_PROPERTY =
+    "mapred.tasktracker.vmem.reserved";
+  /**
+   * @deprecated
+   */
+  @Deprecated
+  static final String MAPRED_TASKTRACKER_PMEM_RESERVED_PROPERTY =
+     "mapred.tasktracker.pmem.reserved";
+
+  static final String CONF_VERSION_KEY = "mapreduce.tasktracker.conf.version";
+  static final String CONF_VERSION_DEFAULT = "default";
+
+  static final long WAIT_FOR_DONE = 3 * 1000;
+  private int httpPort;
+
+  static enum State {NORMAL, STALE, INTERRUPTED, DENIED}
+
+  static{
+    Configuration.addDefaultResource("mapred-default.xml");
+    Configuration.addDefaultResource("mapred-site.xml");
+  }
+
+  public static final Log LOG =
+    LogFactory.getLog(TaskTracker.class);
+
+  public static final String MR_CLIENTTRACE_FORMAT =
+        "src: %s" +     // src IP
+        ", dest: %s" +  // dst IP
+        ", bytes: %s" + // byte count
+        ", op: %s" +    // operation
+        ", cliID: %s" +  // task id
+        ", duration: %s"; // duration
+  public static final Log ClientTraceLog =
+    LogFactory.getLog(TaskTracker.class.getName() + ".clienttrace");
+
+  //Job ACLs file is created by TaskController under userlogs/$jobid directory
+  //for each job at job localization time. This will be used by TaskLogServlet 
+  //for authorizing viewing of task logs of that job
+  static String jobACLsFile = "job-acls.xml";
+
+  volatile boolean running = true;
+
+  /**
+   * Manages TT local storage directories.
+   */
+  static class LocalStorage {
+    private List<String> localDirs;
+    private int numFailures;
+
+    public LocalStorage(String[] dirs) {
+      localDirs = new ArrayList<String>();
+      localDirs.addAll(Arrays.asList(dirs));
+    }
+
+    /**
+     * @return the current valid directories 
+     */
+    synchronized String[] getDirs() {
+      return localDirs.toArray(new String[localDirs.size()]);
+    }
+
+    /**
+     * @return the current valid dirs as comma separated string
+     */
+    synchronized String getDirsString() {
+      return StringUtils.join(",", localDirs);
+    }
+
+    /**
+     * @return the number of valid local directories
+     */
+    synchronized int numDirs() {
+      return localDirs.size();
+    }
+
+    /**
+     * @return the number of directory failures
+     */
+    synchronized int numFailures() {
+      return numFailures;
+    }
+
+    /**
+     * Check the current set of local directories, updating the list
+     * of valid directories if necessary.
+     * @throws DiskErrorException if no directories are writable
+     */
+    synchronized void checkDirs() throws DiskErrorException {
+      for (String dir : localDirs) {
+        try {
+          DiskChecker.checkDir(new File(dir));
+        } catch (DiskErrorException de) {
+          LOG.warn("TaskTracker local dir " + dir + " error " + 
+              de.getMessage() + ", removing from local dirs");
+          localDirs.remove(dir);
+          numFailures++;
+        }
+      }
+      if (localDirs.isEmpty()) {
+        throw new DiskErrorException(
+            "No mapred local directories are writable");
+      }
+    }
+  }
+
+  private LocalStorage localStorage;
+  private long lastCheckDirsTime;
+  private int lastNumFailures;
+  private LocalDirAllocator localDirAllocator;
+  String taskTrackerName;
+  String localHostname;
+  InetSocketAddress jobTrackAddr;
+    
+  InetSocketAddress taskReportAddress;
+
+  Server taskReportServer = null;
+  InterTrackerProtocol jobClient;
+  
+  private TrackerDistributedCacheManager distributedCacheManager;
+  static int FILE_CACHE_SIZE = 2000;
+    
+  // last heartbeat response recieved
+  short heartbeatResponseId = -1;
+  
+  static final String TASK_CLEANUP_SUFFIX = ".cleanup";
+
+  /*
+   * This is the last 'status' report sent by this tracker to the JobTracker.
+   * 
+   * If the rpc call succeeds, this 'status' is cleared-out by this tracker;
+   * indicating that a 'fresh' status report be generated; in the event the
+   * rpc calls fails for whatever reason, the previous status report is sent
+   * again.
+   */
+  TaskTrackerStatus status = null;
+  
+  // The system-directory on HDFS where job files are stored 
+  Path systemDirectory = null;
+  
+  // The filesystem where job files are stored
+  FileSystem systemFS = null;
+  private FileSystem localFs = null;
+  private final HttpServer server;
+    
+  volatile boolean shuttingDown = false;
+    
+  Map<TaskAttemptID, TaskInProgress> tasks = new HashMap<TaskAttemptID, TaskInProgress>();
+  /**
+   * Map from taskId -> TaskInProgress.
+   */
+  Map<TaskAttemptID, TaskInProgress> runningTasks = null;
+  Map<JobID, RunningJob> runningJobs = new TreeMap<JobID, RunningJob>();
+  private final JobTokenSecretManager jobTokenSecretManager
+    = new JobTokenSecretManager();
+
+  JobTokenSecretManager getJobTokenSecretManager() {
+    return jobTokenSecretManager;
+  }
+
+  RunningJob getRunningJob(JobID jobId) {
+    return runningJobs.get(jobId);
+  }
+
+  volatile int mapTotal = 0;
+  volatile int reduceTotal = 0;
+  boolean justStarted = true;
+  boolean justInited = true;
+  // Mark reduce tasks that are shuffling to rollback their events index
+  Set<TaskAttemptID> shouldReset = new HashSet<TaskAttemptID>();
+    
+  //dir -> DF
+  Map<String, DF> localDirsDf = new HashMap<String, DF>();
+  long minSpaceStart = 0;
+  //must have this much space free to start new tasks
+  boolean acceptNewTasks = true;
+  long minSpaceKill = 0;
+  //if we run under this limit, kill one task
+  //and make sure we never receive any new jobs
+  //until all the old tasks have been cleaned up.
+  //this is if a machine is so full it's only good
+  //for serving map output to the other nodes
+
+  static Random r = new Random();
+  public static final String SUBDIR = "taskTracker";
+  static final String DISTCACHEDIR = "distcache";
+  static final String JOBCACHE = "jobcache";
+  static final String OUTPUT = "output";
+  static final String JARSDIR = "jars";
+  static final String LOCAL_SPLIT_FILE = "split.info";
+  static final String JOBFILE = "job.xml";
+  static final String TT_PRIVATE_DIR = "ttprivate";
+  public static final String TT_LOG_TMP_DIR = "tt_log_tmp";
+  static final String JVM_EXTRA_ENV_FILE = "jvm.extra.env";
+
+  static final String JOB_LOCAL_DIR = "job.local.dir";
+  static final String JOB_TOKEN_FILE="jobToken"; //localized file
+
+  private JobConf fConf;
+  private JobConf originalConf;
+  private Localizer localizer;
+  private int maxMapSlots;
+  private int maxReduceSlots;
+  private int failures;
+  final long mapRetainSize;
+  final long reduceRetainSize;
+
+  private ACLsManager aclsManager;
+  
+  // Performance-related config knob to send an out-of-band heartbeat
+  // on task completion
+  static final String TT_OUTOFBAND_HEARBEAT =
+    "mapreduce.tasktracker.outofband.heartbeat";
+  private volatile boolean oobHeartbeatOnTaskCompletion;
+  
+  // Track number of completed tasks to send an out-of-band heartbeat
+  private IntWritable finishedCount = new IntWritable(0);
+  
+  private MapEventsFetcherThread mapEventsFetcher;
+  final int workerThreads;
+  CleanupQueue directoryCleanupThread;
+  private volatile JvmManager jvmManager;
+  
+  private TaskMemoryManagerThread taskMemoryManager;
+  private boolean taskMemoryManagerEnabled = true;
+  private long totalVirtualMemoryOnTT = JobConf.DISABLED_MEMORY_LIMIT;
+  private long totalPhysicalMemoryOnTT = JobConf.DISABLED_MEMORY_LIMIT;
+  private long mapSlotMemorySizeOnTT = JobConf.DISABLED_MEMORY_LIMIT;
+  private long reduceSlotSizeMemoryOnTT = JobConf.DISABLED_MEMORY_LIMIT;
+  private long totalMemoryAllottedForTasks = JobConf.DISABLED_MEMORY_LIMIT;
+
+  private UserLogManager userLogManager;
+
+  static final String MAPRED_TASKTRACKER_MEMORY_CALCULATOR_PLUGIN_PROPERTY =
+      "mapred.tasktracker.memory_calculator_plugin";
+
+  /**
+   * the minimum interval between jobtracker polls
+   */
+  private volatile int heartbeatInterval = HEARTBEAT_INTERVAL_MIN;
+  /**
+   * Number of maptask completion events locations to poll for at one time
+   */  
+  private int probe_sample_size = 500;
+
+  private IndexCache indexCache;
+
+  /**
+  * Handle to the specific instance of the {@link TaskController} class
+  */
+  private TaskController taskController;
+  
+  /**
+   * Handle to the specific instance of the {@link NodeHealthCheckerService}
+   */
+  private NodeHealthCheckerService healthChecker;
+  
+  /**
+   * Configuration property for disk health check interval in milli seconds.
+   * Currently, configuring this to a value smaller than the heartbeat interval
+   * is equivalent to setting this to heartbeat interval value.
+   */
+  static final String DISK_HEALTH_CHECK_INTERVAL_PROPERTY =
+      "mapred.disk.healthChecker.interval";
+  /**
+   * How often TaskTracker needs to check the health of its disks.
+   * Default value is {@link MRConstants#DEFAULT_DISK_HEALTH_CHECK_INTERVAL}
+   */
+  private long diskHealthCheckInterval;
+
+  /*
+   * A list of commitTaskActions for whom commit response has been received 
+   */
+  private List<TaskAttemptID> commitResponses = 
+            Collections.synchronizedList(new ArrayList<TaskAttemptID>());
+
+  private ShuffleServerInstrumentation shuffleServerMetrics;
+
+  private TaskTrackerInstrumentation myInstrumentation = null;
+
+  public TaskTrackerInstrumentation getTaskTrackerInstrumentation() {
+    return myInstrumentation;
+  }
+  
+  /**
+   * A list of tips that should be cleaned up.
+   */
+  private BlockingQueue<TaskTrackerAction> tasksToCleanup = 
+    new LinkedBlockingQueue<TaskTrackerAction>();
+    
+  /**
+   * A daemon-thread that pulls tips off the list of things to cleanup.
+   */
+  private Thread taskCleanupThread = 
+    new Thread(new Runnable() {
+        public void run() {
+          while (true) {
+            try {
+              TaskTrackerAction action = tasksToCleanup.take();
+              checkJobStatusAndWait(action);
+              if (action instanceof KillJobAction) {
+                purgeJob((KillJobAction) action);
+              } else if (action instanceof KillTaskAction) {
+                processKillTaskAction((KillTaskAction) action);
+              } else {
+                LOG.error("Non-delete action given to cleanup thread: "
+                          + action);
+              }
+            } catch (Throwable except) {
+              LOG.warn(StringUtils.stringifyException(except));
+            }
+          }
+        }
+      }, "taskCleanup");
+
+  void processKillTaskAction(KillTaskAction killAction) throws IOException {
+    TaskInProgress tip;
+    synchronized (TaskTracker.this) {
+      tip = tasks.get(killAction.getTaskID());
+    }
+    LOG.info("Received KillTaskAction for task: " + killAction.getTaskID());
+    purgeTask(tip, false);
+  }
+  
+  private void checkJobStatusAndWait(TaskTrackerAction action) 
+  throws InterruptedException {
+    JobID jobId = null;
+    if (action instanceof KillJobAction) {
+      jobId = ((KillJobAction)action).getJobID();
+    } else if (action instanceof KillTaskAction) {
+      jobId = ((KillTaskAction)action).getTaskID().getJobID();
+    } else {
+      return;
+    }
+    RunningJob rjob = null;
+    synchronized (runningJobs) {
+      rjob = runningJobs.get(jobId);
+    }
+    if (rjob != null) {
+      synchronized (rjob) {
+        while (rjob.localizing) {
+          rjob.wait();
+        }
+      }
+    }
+  }
+
+  public TaskController getTaskController() {
+    return taskController;
+  }
+  
+  // Currently this is used only by tests
+  void setTaskController(TaskController t) {
+    taskController = t;
+  }
+  
+  private RunningJob addTaskToJob(JobID jobId, 
+                                  TaskInProgress tip) {
+    synchronized (runningJobs) {
+      RunningJob rJob = null;
+      if (!runningJobs.containsKey(jobId)) {
+        rJob = new RunningJob(jobId);
+        rJob.tasks = new HashSet<TaskInProgress>();
+        runningJobs.put(jobId, rJob);
+      } else {
+        rJob = runningJobs.get(jobId);
+      }
+      synchronized (rJob) {
+        rJob.tasks.add(tip);
+      }
+      return rJob;
+    }
+  }
+
+  private void removeTaskFromJob(JobID jobId, TaskInProgress tip) {
+    synchronized (runningJobs) {
+      RunningJob rjob = runningJobs.get(jobId);
+      if (rjob == null) {
+        LOG.warn("Unknown job " + jobId + " being deleted.");
+      } else {
+        synchronized (rjob) {
+          rjob.tasks.remove(tip);
+        }
+      }
+    }
+  }
+
+  UserLogManager getUserLogManager() {
+    return this.userLogManager;
+  }
+
+  void setUserLogManager(UserLogManager u) {
+    this.userLogManager = u;
+  }
+
+  public static String getUserDir(String user) {
+    return TaskTracker.SUBDIR + Path.SEPARATOR + user;
+  } 
+
+  Localizer getLocalizer() {
+    return localizer;
+  }
+
+  void setLocalizer(Localizer l) {
+    localizer = l;
+  }
+
+  public static String getPrivateDistributedCacheDir(String user) {
+    return getUserDir(user) + Path.SEPARATOR + TaskTracker.DISTCACHEDIR;
+  }
+  
+  public static String getPublicDistributedCacheDir() {
+    return TaskTracker.SUBDIR + Path.SEPARATOR + TaskTracker.DISTCACHEDIR;
+  }
+
+  public static String getJobCacheSubdir(String user) {
+    return getUserDir(user) + Path.SEPARATOR + TaskTracker.JOBCACHE;
+  }
+
+  public static String getLocalJobDir(String user, String jobid) {
+    return getJobCacheSubdir(user) + Path.SEPARATOR + jobid;
+  }
+
+  static String getLocalJobConfFile(String user, String jobid) {
+    return getLocalJobDir(user, jobid) + Path.SEPARATOR + TaskTracker.JOBFILE;
+  }
+  
+  static String getPrivateDirJobConfFile(String user, String jobid) {
+    return TT_PRIVATE_DIR + Path.SEPARATOR + getLocalJobConfFile(user, jobid);
+  }
+
+  static String getTaskConfFile(String user, String jobid, String taskid,
+      boolean isCleanupAttempt) {
+    return getLocalTaskDir(user, jobid, taskid, isCleanupAttempt)
+    + Path.SEPARATOR + TaskTracker.JOBFILE;
+  }
+  
+  static String getPrivateDirTaskScriptLocation(String user, String jobid, 
+     String taskid) {
+    return TT_PRIVATE_DIR + Path.SEPARATOR + 
+           getLocalTaskDir(user, jobid, taskid);
+  }
+
+  static String getJobJarsDir(String user, String jobid) {
+    return getLocalJobDir(user, jobid) + Path.SEPARATOR + TaskTracker.JARSDIR;
+  }
+
+  public static String getJobJarFile(String user, String jobid) {
+    return getJobJarsDir(user, jobid) + Path.SEPARATOR + "job.jar";
+  }
+  
+  static String getJobWorkDir(String user, String jobid) {
+    return getLocalJobDir(user, jobid) + Path.SEPARATOR + MRConstants.WORKDIR;
+  }
+
+  static String getLocalSplitFile(String user, String jobid, String taskid) {
+    return TaskTracker.getLocalTaskDir(user, jobid, taskid) + Path.SEPARATOR
+    + TaskTracker.LOCAL_SPLIT_FILE;
+  }
+
+  static String getIntermediateOutputDir(String user, String jobid,
+      String taskid) {
+    return getLocalTaskDir(user, jobid, taskid) + Path.SEPARATOR
+    + TaskTracker.OUTPUT;
+  }
+
+  public static String getLocalTaskDir(String user, String jobid, 
+      String taskid) {
+    return getLocalTaskDir(user, jobid, taskid, false);
+  }
+  
+  public static String getLocalTaskDir(String user, String jobid, String taskid,
+      boolean isCleanupAttempt) {
+    String taskDir = getLocalJobDir(user, jobid) + Path.SEPARATOR + taskid;
+    if (isCleanupAttempt) {
+      taskDir = taskDir + TASK_CLEANUP_SUFFIX;
+    }
+    return taskDir;
+  }
+  
+  static String getTaskWorkDir(String user, String jobid, String taskid,
+      boolean isCleanupAttempt) {
+    String dir = getLocalTaskDir(user, jobid, taskid, isCleanupAttempt);
+    return dir + Path.SEPARATOR + MRConstants.WORKDIR;
+  }
+
+  static String getLocalJobTokenFile(String user, String jobid) {
+    return getLocalJobDir(user, jobid) + Path.SEPARATOR + TaskTracker.JOB_TOKEN_FILE;
+  }
+  
+  static String getPrivateDirJobTokenFile(String user, String jobid) {
+    return TT_PRIVATE_DIR + Path.SEPARATOR + 
+           getLocalJobTokenFile(user, jobid); 
+  }
+  
+  static String getPrivateDirForJob(String user, String jobid) {
+    return TT_PRIVATE_DIR + Path.SEPARATOR + getLocalJobDir(user, jobid) ;
+  }
+
+  private FileSystem getFS(final Path filePath, JobID jobId,
+      final Configuration conf) throws IOException, InterruptedException {
+    RunningJob rJob = runningJobs.get(jobId);
+    FileSystem userFs = 
+      rJob.ugi.doAs(new PrivilegedExceptionAction<FileSystem>() {
+        public FileSystem run() throws IOException {
+          return filePath.getFileSystem(conf);
+      }});
+    return userFs;
+  }
+  
+  String getPid(TaskAttemptID tid) {
+    TaskInProgress tip = tasks.get(tid);
+    if (tip != null) {
+      return jvmManager.getPid(tip.getTaskRunner());
+    }
+    return null;
+  }
+  
+  public long getProtocolVersion(String protocol, 
+                                 long clientVersion) throws IOException {
+    if (protocol.equals(TaskUmbilicalProtocol.class.getName())) {
+      return TaskUmbilicalProtocol.versionID;
+    } else {
+      throw new IOException("Unknown protocol for task tracker: " +
+                            protocol);
+    }
+  }
+
+  /**
+   * Delete all of the user directories.
+   * @param conf the TT configuration
+   * @throws IOException
+   */
+  private void deleteUserDirectories(Configuration conf) throws IOException {
+    for(String root: localStorage.getDirs()) {
+      for(FileStatus status: localFs.listStatus(new Path(root, SUBDIR))) {
+        String owner = status.getOwner();
+        String path = status.getPath().getName();
+        if (path.equals(owner)) {
+          taskController.deleteAsUser(owner, "");
+        }
+      }
+    }
+  }
+
+  public static final String TT_USER_NAME = "mapreduce.tasktracker.kerberos.principal";
+  public static final String TT_KEYTAB_FILE =
+    "mapreduce.tasktracker.keytab.file";  
+  /**
+   * Do the real constructor work here.  It's in a separate method
+   * so we can call it again and "recycle" the object after calling
+   * close().
+   */
+  synchronized void initialize() throws IOException, InterruptedException {
+    this.fConf = new JobConf(originalConf);
+    
+    LOG.info("Starting tasktracker with owner as "
+        + getMROwner().getShortUserName());
+
+    localFs = FileSystem.getLocal(fConf);
+    if (fConf.get("slave.host.name") != null) {
+      this.localHostname = fConf.get("slave.host.name");
+    }
+    if (localHostname == null) {
+      this.localHostname =
+      DNS.getDefaultHost
+      (fConf.get("mapred.tasktracker.dns.interface","default"),
+       fConf.get("mapred.tasktracker.dns.nameserver","default"));
+    }
+
+    final String dirs = localStorage.getDirsString();
+    fConf.setStrings(JobConf.MAPRED_LOCAL_DIR_PROPERTY, dirs);
+    LOG.info("Good mapred local directories are: " + dirs);
+    taskController.setConf(fConf);
+    // Setup task controller so that deletion of user dirs happens properly
+    taskController.setup(localDirAllocator, localStorage);
+    server.setAttribute("conf", fConf);
+
+    deleteUserDirectories(fConf);
+
+    // NB: deleteLocalFiles uses the configured local dirs, but does not 
+    // fail if a local directory has failed. 
+    fConf.deleteLocalFiles(SUBDIR);
+    final FsPermission ttdir = FsPermission.createImmutable((short) 0755);
+    for (String s : localStorage.getDirs()) {
+      localFs.mkdirs(new Path(s, SUBDIR), ttdir);
+    }
+    fConf.deleteLocalFiles(TT_PRIVATE_DIR);
+    final FsPermission priv = FsPermission.createImmutable((short) 0700);
+    for (String s : localStorage.getDirs()) {
+      localFs.mkdirs(new Path(s, TT_PRIVATE_DIR), priv);
+    }
+    fConf.deleteLocalFiles(TT_LOG_TMP_DIR);
+    final FsPermission pub = FsPermission.createImmutable((short) 0755);
+    for (String s : localStorage.getDirs()) {
+      localFs.mkdirs(new Path(s, TT_LOG_TMP_DIR), pub);
+    }
+    // Create userlogs directory under all good mapred-local-dirs
+    for (String s : localStorage.getDirs()) {
+      Path userLogsDir = new Path(s, TaskLog.USERLOGS_DIR_NAME);
+      if (!localFs.exists(userLogsDir)) {
+        localFs.mkdirs(userLogsDir, pub);
+      }
+    }
+    // Clear out state tables
+    this.tasks.clear();
+    this.runningTasks = new LinkedHashMap<TaskAttemptID, TaskInProgress>();
+    this.runningJobs = new TreeMap<JobID, RunningJob>();
+    this.mapTotal = 0;
+    this.reduceTotal = 0;
+    this.acceptNewTasks = true;
+    this.status = null;
+
+    this.minSpaceStart = this.fConf.getLong("mapred.local.dir.minspacestart", 0L);
+    this.minSpaceKill = this.fConf.getLong("mapred.local.dir.minspacekill", 0L);
+    //tweak the probe sample size (make it a function of numCopiers)
+    probe_sample_size = this.fConf.getInt("mapred.tasktracker.events.batchsize", 500);
+
+    createInstrumentation();
+
+    // bind address
+    String address = 
+      NetUtils.getServerAddress(fConf,
+                                "mapred.task.tracker.report.bindAddress", 
+                                "mapred.task.tracker.report.port", 
+                                "mapred.task.tracker.report.address");
+    InetSocketAddress socAddr = NetUtils.createSocketAddr(address);
+    String bindAddress = socAddr.getHostName();
+    int tmpPort = socAddr.getPort();
+    
+    this.jvmManager = new JvmManager(this);
+
+    // Set service-level authorization security policy
+    if (this.fConf.getBoolean(
+          ServiceAuthorizationManager.SERVICE_AUTHORIZATION_CONFIG, false)) {
+      PolicyProvider policyProvider = 
+        (PolicyProvider)(ReflectionUtils.newInstance(
+            this.fConf.getClass(PolicyProvider.POLICY_PROVIDER_CONFIG, 
+                MapReducePolicyProvider.class, PolicyProvider.class), 
+            this.fConf));
+      ServiceAuthorizationManager.refresh(fConf, policyProvider);
+    }
+    
+    // RPC initialization
+    int max = maxMapSlots > maxReduceSlots ? 
+                       maxMapSlots : maxReduceSlots;
+    //set the num handlers to max*2 since canCommit may wait for the duration
+    //of a heartbeat RPC
+    this.taskReportServer = RPC.getServer(this, bindAddress,
+        tmpPort, 2 * max, false, this.fConf, this.jobTokenSecretManager);
+    this.taskReportServer.start();
+
+    // get the assigned address
+    this.taskReportAddress = taskReportServer.getListenerAddress();
+    this.fConf.set("mapred.task.tracker.report.address",
+        taskReportAddress.getHostName() + ":" + taskReportAddress.getPort());
+    LOG.info("TaskTracker up at: " + this.taskReportAddress);
+
+    this.taskTrackerName = "tracker_" + localHostname + ":" + taskReportAddress;
+    LOG.info("Starting tracker " + taskTrackerName);
+
+    // Initialize DistributedCache
+    this.distributedCacheManager = new TrackerDistributedCacheManager(
+        this.fConf, taskController);
+    this.distributedCacheManager.startCleanupThread();
+    
+    this.jobClient = (InterTrackerProtocol) 
+    UserGroupInformation.getLoginUser().doAs(
+        new PrivilegedExceptionAction<Object>() {
+      public Object run() throws IOException {
+        return RPC.waitForProxy(InterTrackerProtocol.class,
+            InterTrackerProtocol.versionID,
+            jobTrackAddr, fConf);
+      }
+    });
+    this.justInited = true;
+    this.running = true;    
+    // start the thread that will fetch map task completion events
+    this.mapEventsFetcher = new MapEventsFetcherThread();
+    mapEventsFetcher.setDaemon(true);
+    mapEventsFetcher.setName(
+                             "Map-events fetcher for all reduce tasks " + "on " + 
+                             taskTrackerName);
+    mapEventsFetcher.start();
+
+    initializeMemoryManagement();
+
+    getUserLogManager().clearOldUserLogs(fConf);
+
+    setIndexCache(new IndexCache(this.fConf));
+
+    mapLauncher = new TaskLauncher(TaskType.MAP, maxMapSlots);
+    reduceLauncher = new TaskLauncher(TaskType.REDUCE, maxReduceSlots);
+    mapLauncher.start();
+    reduceLauncher.start();
+
+    // create a localizer instance
+    setLocalizer(new Localizer(localFs, localStorage.getDirs()));
+
+    //Start up node health checker service.
+    if (shouldStartHealthMonitor(this.fConf)) {
+      startHealthMonitor(this.fConf);
+    }
+    
+    oobHeartbeatOnTaskCompletion = 
+      fConf.getBoolean(TT_OUTOFBAND_HEARBEAT, false);
+  }
+
+  private void createInstrumentation() {
+    Class<? extends TaskTrackerInstrumentation> metricsInst =
+        getInstrumentationClass(fConf);
+    LOG.debug("instrumentation class="+ metricsInst);
+    if (metricsInst == null) {
+      myInstrumentation = TaskTrackerInstrumentation.create(this);
+      return;
+    }
+    try {
+      java.lang.reflect.Constructor<? extends TaskTrackerInstrumentation> c =
+        metricsInst.getConstructor(new Class<?>[] {TaskTracker.class} );
+      this.myInstrumentation = c.newInstance(this);
+    } catch(Exception e) {
+      //Reflection can throw lots of exceptions -- handle them all by
+      //falling back on the default.
+      LOG.error("failed to initialize taskTracker metrics", e);
+      this.myInstrumentation = TaskTrackerInstrumentation.create(this);
+    }
+
+  }
+
+  UserGroupInformation getMROwner() {
+    return aclsManager.getMROwner();
+  }
+
+  /**
+   * Are ACLs for authorization checks enabled on the TT ?
+   */
+  boolean areACLsEnabled() {
+    return fConf.getBoolean(JobConf.MR_ACLS_ENABLED, false);
+  }
+
+  static Class<? extends TaskTrackerInstrumentation> getInstrumentationClass(
+    Configuration conf) {
+    return conf.getClass("mapred.tasktracker.instrumentation", null,
+                         TaskTrackerInstrumentation.class);
+  }
+
+  static void setInstrumentationClass(
+    Configuration conf, Class<? extends TaskTrackerInstrumentation> t) {
+    conf.setClass("mapred.tasktracker.instrumentation",
+        t, TaskTrackerInstrumentation.class);
+  }
+  
+  /** 
+   * Removes all contents of temporary storage.  Called upon 
+   * startup, to remove any leftovers from previous run.
+   */
+  public void cleanupStorage() throws IOException {
+    this.fConf.deleteLocalFiles(SUBDIR);
+    this.fConf.deleteLocalFiles(TT_PRIVATE_DIR);
+    this.fConf.deleteLocalFiles(TT_LOG_TMP_DIR);
+  }
+
+  // Object on wait which MapEventsFetcherThread is going to wait.
+  private Object waitingOn = new Object();
+
+  private class MapEventsFetcherThread extends Thread {
+
+    private List <FetchStatus> reducesInShuffle() {
+      List <FetchStatus> fList = new ArrayList<FetchStatus>();
+      for (Map.Entry <JobID, RunningJob> item : runningJobs.entrySet()) {
+        RunningJob rjob = item.getValue();
+        if (!rjob.localized) {
+          continue;
+        }
+        JobID jobId = item.getKey();
+        FetchStatus f;
+        synchronized (rjob) {
+          f = rjob.getFetchStatus();
+          for (TaskInProgress tip : rjob.tasks) {
+            Task task = tip.getTask();
+            if (!task.isMapTask()) {
+              if (((ReduceTask)task).getPhase() == 
+                  TaskStatus.Phase.SHUFFLE) {
+                if (rjob.getFetchStatus() == null) {
+                  //this is a new job; we start fetching its map events
+                  f = new FetchStatus(jobId, 
+                                      ((ReduceTask)task).getNumMaps());
+                  rjob.setFetchStatus(f);
+                }
+                f = rjob.getFetchStatus();
+                fList.add(f);
+                break; //no need to check any more tasks belonging to this
+              }
+            }
+          }
+        }
+      }
+      //at this point, we have information about for which of
+      //the running jobs do we need to query the jobtracker for map 
+      //outputs (actually map events).
+      return fList;
+    }
+      
+    @Override
+    public void run() {
+      LOG.info("Starting thread: " + this.getName());
+        
+      while (running) {
+        try {
+          List <FetchStatus> fList = null;
+          synchronized (runningJobs) {
+            while (((fList = reducesInShuffle()).size()) == 0) {
+              try {
+                runningJobs.wait();
+              } catch (InterruptedException e) {
+                LOG.info("Shutting down: " + this.getName());
+                return;
+              }
+            }
+          }
+          // now fetch all the map task events for all the reduce tasks
+          // possibly belonging to different jobs
+          boolean fetchAgain = false; //flag signifying whether we want to fetch
+                                      //immediately again.
+          for (FetchStatus f : fList) {
+            long currentTime = System.currentTimeMillis();
+            try {
+              //the method below will return true when we have not 
+              //fetched all available events yet
+              if (f.fetchMapCompletionEvents(currentTime)) {
+                fetchAgain = true;
+              }
+            } catch (Exception e) {
+              LOG.warn(
+                       "Ignoring exception that fetch for map completion" +
+                       " events threw for " + f.jobId + " threw: " +
+                       StringUtils.stringifyException(e)); 
+            }
+            if (!running) {
+              break;
+            }
+          }
+          synchronized (waitingOn) {
+            try {
+              if (!fetchAgain) {
+                waitingOn.wait(heartbeatInterval);
+              }
+            } catch (InterruptedException ie) {
+              LOG.info("Shutting down: " + this.getName());
+              return;
+            }
+          }
+        } catch (Exception e) {
+          LOG.info("Ignoring exception "  + e.getMessage());
+        }
+      }
+    } 
+  }
+
+  private class FetchStatus {
+    /** The next event ID that we will start querying the JobTracker from*/
+    private IntWritable fromEventId;
+    /** This is the cache of map events for a given job */ 
+    private List<TaskCompletionEvent> allMapEvents;
+    /** What jobid this fetchstatus object is for*/
+    private JobID jobId;
+    private long lastFetchTime;
+    private boolean fetchAgain;
+     
+    public FetchStatus(JobID jobId, int numMaps) {
+      this.fromEventId = new IntWritable(0);
+      this.jobId = jobId;
+      this.allMapEvents = new ArrayList<TaskCompletionEvent>(numMaps);
+    }
+      
+    /**
+     * Reset the events obtained so far.
+     */
+    public void reset() {
+      // Note that the sync is first on fromEventId and then on allMapEvents
+      synchronized (fromEventId) {
+        synchronized (allMapEvents) {
+          fromEventId.set(0); // set the new index for TCE
+          allMapEvents.clear();
+        }
+      }
+    }
+    
+    public TaskCompletionEvent[] getMapEvents(int fromId, int max) {
+        
+      TaskCompletionEvent[] mapEvents = 
+        TaskCompletionEvent.EMPTY_ARRAY;
+      boolean notifyFetcher = false; 
+      synchronized (allMapEvents) {
+        if (allMapEvents.size() > fromId) {
+          int actualMax = Math.min(max, (allMapEvents.size() - fromId));
+          List <TaskCompletionEvent> eventSublist = 
+            allMapEvents.subList(fromId, actualMax + fromId);
+          mapEvents = eventSublist.toArray(mapEvents);
+        } else {
+          // Notify Fetcher thread. 
+          notifyFetcher = true;
+        }
+      }
+      if (notifyFetcher) {
+        synchronized (waitingOn) {
+          waitingOn.notify();
+        }
+      }
+      return mapEvents;
+    }
+      
+    public boolean fetchMapCompletionEvents(long currTime) throws IOException {
+      if (!fetchAgain && (currTime - lastFetchTime) < heartbeatInterval) {
+        return false;
+      }
+      int currFromEventId = 0;
+      synchronized (fromEventId) {
+        currFromEventId = fromEventId.get();
+        List <TaskCompletionEvent> recentMapEvents = 
+          queryJobTracker(fromEventId, jobId, jobClient);
+        synchronized (allMapEvents) {
+          allMapEvents.addAll(recentMapEvents);
+        }
+        lastFetchTime = currTime;
+        if (fromEventId.get() - currFromEventId >= probe_sample_size) {
+          //return true when we have fetched the full payload, indicating
+          //that we should fetch again immediately (there might be more to
+          //fetch
+          fetchAgain = true;
+          return true;
+        }
+      }
+      fetchAgain = false;
+      return false;
+    }
+  }
+
+  private static LocalDirAllocator lDirAlloc = 
+                              new LocalDirAllocator("mapred.local.dir");
+
+  // intialize the job directory
+  RunningJob localizeJob(TaskInProgress tip) 
+  throws IOException, InterruptedException {
+    Task t = tip.getTask();
+    JobID jobId = t.getJobID();
+    RunningJob rjob = addTaskToJob(jobId, tip);
+    InetSocketAddress ttAddr = getTaskTrackerReportAddress();
+    try {
+      synchronized (rjob) {
+        if (!rjob.localized) {
+          while (rjob.localizing) {
+            rjob.wait();
+          }
+          if (!rjob.localized) {
+            //this thread is localizing the job
+            rjob.localizing = true;
+          }
+        }
+      }
+      if (!rjob.localized) {
+        Path localJobConfPath = initializeJob(t, rjob, ttAddr);
+        JobConf localJobConf = new JobConf(localJobConfPath);
+        //to be doubly sure, overwrite the user in the config with the one the TT 
+        //thinks it is
+        localJobConf.setUser(t.getUser());
+        //also reset the #tasks per jvm
+        resetNumTasksPerJvm(localJobConf);
+        //set the base jobconf path in rjob; all tasks will use
+        //this as the base path when they run
+        synchronized (rjob) {
+          rjob.localizedJobConf = localJobConfPath;
+          rjob.jobConf = localJobConf;  
+          rjob.keepJobFiles = ((localJobConf.getKeepTaskFilesPattern() != null) ||
+              localJobConf.getKeepFailedTaskFiles());
+
+          rjob.localized = true;
+        }
+      } 
+    } finally {
+      synchronized (rjob) {
+        if (rjob.localizing) {
+          rjob.localizing = false;
+          rjob.notifyAll();
+        }
+      }
+    }
+    synchronized (runningJobs) {
+      runningJobs.notify(); //notify the fetcher thread
+    }
+    return rjob;
+  }
+
+  /**
+   * Localize the job on this tasktracker. Specifically
+   * <ul>
+   * <li>Cleanup and create job directories on all disks</li>
+   * <li>Download the credentials file</li>
+   * <li>Download the job config file job.xml from the FS</li>
+   * <li>Invokes the {@link TaskController} to do the rest of the job 
+   * initialization</li>
+   * </ul>
+   *
+   * @param t task whose job has to be localized on this TT
+   * @param rjob the {@link RunningJob}
+   * @param ttAddr the tasktracker's RPC address
+   * @return the path to the job configuration to be used for all the tasks
+   *         of this job as a starting point.
+   * @throws IOException
+   */
+  Path initializeJob(final Task t, final RunningJob rjob, 
+      final InetSocketAddress ttAddr)
+  throws IOException, InterruptedException {
+    final JobID jobId = t.getJobID();
+
+    final Path jobFile = new Path(t.getJobFile());
+    final String userName = t.getUser();
+    final Configuration conf = getJobConf();
+
+    // save local copy of JobToken file
+    final String localJobTokenFile = localizeJobTokenFile(t.getUser(), jobId);
+    synchronized (rjob) {
+      rjob.ugi = UserGroupInformation.createRemoteUser(t.getUser());
+
+      Credentials ts = TokenCache.loadTokens(localJobTokenFile, conf);
+      Token<JobTokenIdentifier> jt = TokenCache.getJobToken(ts);
+      if (jt != null) { //could be null in the case of some unit tests
+        getJobTokenSecretManager().addTokenForJob(jobId.toString(), jt);
+      }
+      for (Token<? extends TokenIdentifier> token : ts.getAllTokens()) {
+        rjob.ugi.addToken(token);
+      }
+    }
+
+    FileSystem userFs = getFS(jobFile, jobId, conf);
+
+    // Download the job.xml for this job from the system FS
+    final Path localJobFile =
+      localizeJobConfFile(new Path(t.getJobFile()), userName, userFs, jobId);
+
+    /**
+      * Now initialize the job via task-controller to do the rest of the
+      * job-init. Do this within a doAs since the public distributed cache 
+      * is also set up here.
+      * To support potential authenticated HDFS accesses, we need the tokens
+      */
+    rjob.ugi.doAs(new PrivilegedExceptionAction<Object>() {
+      public Object run() throws IOException, InterruptedException {
+        try {
+          final JobConf localJobConf = new JobConf(localJobFile);
+          // Setup the public distributed cache
+          TaskDistributedCacheManager taskDistributedCacheManager =
+            getTrackerDistributedCacheManager()
+           .newTaskDistributedCacheManager(jobId, localJobConf);
+          rjob.distCacheMgr = taskDistributedCacheManager;
+          taskDistributedCacheManager.setupCache(localJobConf,
+            TaskTracker.getPublicDistributedCacheDir(),
+            TaskTracker.getPrivateDistributedCacheDir(userName));
+
+          // Set some config values
+          localJobConf.set(JobConf.MAPRED_LOCAL_DIR_PROPERTY,
+              getJobConf().get(JobConf.MAPRED_LOCAL_DIR_PROPERTY));
+          if (conf.get("slave.host.name") != null) {
+            localJobConf.set("slave.host.name", conf.get("slave.host.name"));
+          }
+          resetNumTasksPerJvm(localJobConf);
+          localJobConf.setUser(t.getUser());
+
+          // write back the config (this config will have the updates that the
+          // distributed cache manager makes as well)
+          JobLocalizer.writeLocalJobFile(localJobFile, localJobConf);
+          taskController.initializeJob(t.getUser(), jobId.toString(), 
+              new Path(localJobTokenFile), localJobFile, TaskTracker.this,
+              ttAddr);
+        } catch (IOException e) {
+          LOG.warn("Exception while localization " + 
+              StringUtils.stringifyException(e));
+          throw e;
+        } catch (InterruptedException ie) {
+          LOG.warn("Exception while localization " + 
+              StringUtils.stringifyException(ie));
+          throw ie;
+        }
+        return null;
+      }
+    });
+    //search for the conf that the initializeJob created
+    //need to look up certain configs from this conf, like
+    //the distributed cache, profiling, etc. ones
+    Path initializedConf = lDirAlloc.getLocalPathToRead(getLocalJobConfFile(
+           userName, jobId.toString()), getJobConf());
+    return initializedConf;
+  }
+  
+  /** If certain configs are enabled, the jvm-reuse should be disabled
+   * @param localJobConf
+   */
+  static void resetNumTasksPerJvm(JobConf localJobConf) {
+    boolean debugEnabled = false;
+    if (localJobConf.getNumTasksToExecutePerJvm() == 1) {
+      return;
+    }
+    if (localJobConf.getMapDebugScript() != null || 
+        localJobConf.getReduceDebugScript() != null) {
+      debugEnabled = true;
+    }
+    String keepPattern = localJobConf.getKeepTaskFilesPattern();
+    
+    if (debugEnabled || localJobConf.getProfileEnabled() ||
+        keepPattern != null || localJobConf.getKeepFailedTaskFiles()) {
+      //disable jvm reuse
+      localJobConf.setNumTasksToExecutePerJvm(1);
+    }
+  }
+
+  // Remove the log dir from the tasklog cleanup thread
+  void saveLogDir(JobID jobId, JobConf localJobConf)
+      throws IOException {
+    // remove it from tasklog cleanup thread first,
+    // it might be added there because of tasktracker reinit or restart
+    JobStartedEvent jse = new JobStartedEvent(jobId);
+    getUserLogManager().addLogEvent(jse);
+  }
+
+  
+  /**
+   * Download the job configuration file from the FS.
+   *
+   * @param jobFile the original location of the configuration file
+   * @param user the user in question
+   * @param userFs the FileSystem created on behalf of the user
+   * @param jobId jobid in question
+   * @return the local file system path of the downloaded file.
+   * @throws IOException
+   */
+  private Path localizeJobConfFile(Path jobFile, String user, 
+      FileSystem userFs, JobID jobId)
+  throws IOException {
+    // Get sizes of JobFile and JarFile
+    // sizes are -1 if they are not present.
+    FileStatus status = null;
+    long jobFileSize = -1;
+    try {
+      status = userFs.getFileStatus(jobFile);
+      jobFileSize = status.getLen();
+    } catch(FileNotFoundException fe) {
+      jobFileSize = -1;
+    }
+    Path localJobFile =
+      lDirAlloc.getLocalPathForWrite(getPrivateDirJobConfFile(user,
+          jobId.toString()), jobFileSize, fConf);
+
+    // Download job.xml
+    userFs.copyToLocalFile(jobFile, localJobFile);
+    return localJobFile;
+  }
+
+  protected void launchTaskForJob(TaskInProgress tip, JobConf jobConf,
+                                RunningJob rjob) throws IOException {
+    synchronized (tip) {
+      jobConf.set(JobConf.MAPRED_LOCAL_DIR_PROPERTY,
+                  localStorage.getDirsString());
+      tip.setJobConf(jobConf);
+      tip.setUGI(rjob.ugi);
+      tip.launchTask(rjob);
+    }
+  }
+    
+  public synchronized void shutdown() throws IOException, InterruptedException {
+    shuttingDown = true;
+    close();
+    if (this.server != null) {
+      try {
+        LOG.info("Shutting down StatusHttpServer");
+        this.server.stop();
+      } catch (Exception e) {
+        LOG.warn("Exception shutting down TaskTracker", e);
+      }
+    }
+  }
+  /**
+   * Close down the TaskTracker and all its components.  We must also shutdown
+   * any running tasks or threads, and cleanup disk space.  A new TaskTracker
+   * within the same process space might be restarted, so everything must be
+   * clean.
+   * @throws InterruptedException 
+   */
+  public synchronized void close() throws IOException, InterruptedException {
+    //
+    // Kill running tasks.  Do this in a 2nd vector, called 'tasksToClose',
+    // because calling jobHasFinished() may result in an edit to 'tasks'.
+    //
+    TreeMap<TaskAttemptID, TaskInProgress> tasksToClose =
+      new TreeMap<TaskAttemptID, TaskInProgress>();
+    tasksToClose.putAll(tasks);
+    for (TaskInProgress tip : tasksToClose.values()) {
+      tip.jobHasFinished(false);
+    }
+    
+    this.running = false;
+
+    // Clear local storage
+    cleanupStorage();
+        
+    // Shutdown the fetcher thread
+    this.mapEventsFetcher.interrupt();
+    
+    //stop the launchers
+    this.mapLauncher.interrupt();
+    this.reduceLauncher.interrupt();
+
+    this.distributedCacheManager.stopCleanupThread();
+    jvmManager.stop();
+    
+    // shutdown RPC connections
+    RPC.stopProxy(jobClient);
+
+    // wait for the fetcher thread to exit
+    for (boolean done = false; !done; ) {
+      try {
+        this.mapEventsFetcher.join();
+        done = true;
+      } catch (InterruptedException e) {
+      }
+    }
+    
+    if (taskReportServer != null) {
+      taskReportServer.stop();
+      taskReportServer = null;
+    }
+    if (healthChecker != null) {
+      //stop node health checker service
+      healthChecker.stop();
+      healthChecker = null;
+    }
+  }
+
+  /**
+   * For testing
+   */
+  TaskTracker() {
+    server = null;
+    workerThreads = 0;
+    mapRetainSize = TaskLogsTruncater.DEFAULT_RETAIN_SIZE;
+    reduceRetainSize = TaskLogsTruncater.DEFAULT_RETAIN_SIZE;
+  }
+
+  void setConf(JobConf conf) {
+    fConf = conf;
+  }
+
+  /**
+   * Start with the local machine name, and the default JobTracker
+   */
+  public TaskTracker(JobConf conf) throws IOException, InterruptedException {
+    originalConf = conf;
+    FILE_CACHE_SIZE = conf.getInt("mapred.tasktracker.file.cache.size", 2000);
+    maxMapSlots = conf.getInt(
+                  "mapred.tasktracker.map.tasks.maximum", 2);
+    maxReduceSlots = conf.getInt(
+                  "mapred.tasktracker.reduce.tasks.maximum", 2);
+    diskHealthCheckInterval = conf.getLong(DISK_HEALTH_CHECK_INTERVAL_PROPERTY,
+                                           DEFAULT_DISK_HEALTH_CHECK_INTERVAL);
+    UserGroupInformation.setConfiguration(originalConf);
+    aclsManager = new ACLsManager(conf, new JobACLsManager(conf), null);
+    this.jobTrackAddr = JobTracker.getAddress(conf);
+    String infoAddr = 
+      NetUtils.getServerAddress(conf,
+                                "tasktracker.http.bindAddress", 
+                                "tasktracker.http.port",
+                                "mapred.task.tracker.http.address");
+    InetSocketAddress infoSocAddr = NetUtils.createSocketAddr(infoAddr);
+    String httpBindAddress = infoSocAddr.getHostName();
+    int httpPort = infoSocAddr.getPort();
+    this.server = new HttpServer("task", httpBindAddress, httpPort,
+        httpPort == 0, conf, aclsManager.getAdminsAcl());
+    workerThreads = conf.getInt("tasktracker.http.threads", 40);
+    server.setThreads(1, workerThreads);
+    // let the jsp pages get to the task tracker, config, and other relevant
+    // objects
+    FileSystem local = FileSystem.getLocal(conf);
+    this.localDirAllocator = new LocalDirAllocator("mapred.local.dir");
+    Class<? extends TaskController> taskControllerClass = 
+      conf.getClass("mapred.task.tracker.task-controller", 
+                     DefaultTaskController.class, TaskController.class);
+
+    fConf = new JobConf(conf);
+    localStorage = new LocalStorage(fConf.getLocalDirs());
+    localStorage.checkDirs();
+    taskController = 
+      (TaskController) ReflectionUtils.newInstance(taskControllerClass, fConf);
+    taskController.setup(localDirAllocator, localStorage);
+    lastNumFailures = localStorage.numFailures();
+
+    // create user log manager
+    setUserLogManager(new UserLogManager(conf, taskController));
+    SecurityUtil.login(originalConf, TT_KEYTAB_FILE, TT_USER_NAME);
+
+    initialize();
+    this.shuffleServerMetrics = ShuffleServerInstrumentation.create(this);
+    server.setAttribute("task.tracker", this);
+    server.setAttribute("local.file.system", local);
+
+    server.setAttribute("log", LOG);
+    server.setAttribute("localDirAllocator", localDirAllocator);
+    server.setAttribute("shuffleServerMetrics", shuffleServerMetrics);
+
+    String exceptionStackRegex =
+      conf.get("mapreduce.reduce.shuffle.catch.exception.stack.regex");
+    String exceptionMsgRegex =
+      conf.get("mapreduce.reduce.shuffle.catch.exception.message.regex");
+
+    server.setAttribute("exceptionStackRegex", exceptionStackRegex);
+    server.setAttribute("exceptionMsgRegex", exceptionMsgRegex);
+
+    server.addInternalServlet("mapOutput", "/mapOutput", MapOutputServlet.class);
+    server.addServlet("taskLog", "/tasklog", TaskLogServlet.class);
+    server.start();
+    this.httpPort = server.getPort();
+    checkJettyPort(httpPort);
+    LOG.info("FILE_CACHE_SIZE for mapOutputServlet set to : " + FILE_CACHE_SIZE);
+    mapRetainSize = conf.getLong(TaskLogsTruncater.MAP_USERLOG_RETAIN_SIZE, 
+        TaskLogsTruncater.DEFAULT_RETAIN_SIZE);
+    reduceRetainSize = conf.getLong(TaskLogsTruncater.REDUCE_USERLOG_RETAIN_SIZE,
+        TaskLogsTruncater.DEFAULT_RETAIN_SIZE);
+  }
+
+  private void checkJettyPort(int port) throws IOException { 
+    //See HADOOP-4744
+    if (port < 0) {
+      shuttingDown = true;
+      throw new IOException("Jetty problem. Jetty didn't bind to a " +
+      		"valid port");
+    }
+  }
+  
+  private void startCleanupThreads() throws IOException {
+    taskCleanupThread.setDaemon(true);
+    taskCleanupThread.start();
+    directoryCleanupThread = CleanupQueue.getInstance();
+  }
+
+  // only used by tests
+  void setCleanupThread(CleanupQueue c) {
+    directoryCleanupThread = c;
+  }
+  
+  CleanupQueue getCleanupThread() {
+    return directoryCleanupThread;
+  }
+
+  /**
+   * The connection to the JobTracker, used by the TaskRunner 
+   * for locating remote files.
+   */
+  public InterTrackerProtocol getJobClient() {
+    return jobClient;
+  }
+        
+  /** Return the port at which the tasktracker bound to */
+  public synchronized InetSocketAddress getTaskTrackerReportAddress() {
+    return taskReportAddress;
+  }
+    
+  /** Queries the job tracker for a set of outputs ready to be copied
+   * @param fromEventId the first event ID we want to start from, this is
+   * modified by the call to this method
+   * @param jobClient the job tracker
+   * @return a set of locations to copy outputs from
+   * @throws IOException
+   */  
+  private List<TaskCompletionEvent> queryJobTracker(IntWritable fromEventId,
+                                                    JobID jobId,
+                                                    InterTrackerProtocol jobClient)
+    throws IOException {
+
+    TaskCompletionEvent t[] = jobClient.getTaskCompletionEvents(
+                                                                jobId,
+                                                                fromEventId.get(),
+                                                                probe_sample_size);
+    //we are interested in map task completion events only. So store
+    //only those
+    List <TaskCompletionEvent> recentMapEvents = 
+      new ArrayList<TaskCompletionEvent>();
+    for (int i = 0; i < t.length; i++) {
+      if (t[i].isMap) {
+        recentMapEvents.add(t[i]);
+      }
+    }
+    fromEventId.set(fromEventId.get() + t.length);
+    return recentMapEvents;
+  }
+
+  /**
+   * Main service loop.  Will stay in this loop forever.
+   */
+  State offerService() throws Exception {
+    long lastHeartbeat = 0;
+
+    while (running && !shuttingDown) {
+      try {
+        long now = System.currentTimeMillis();
+
+        long waitTime = heartbeatInterval - (now - lastHeartbeat);
+        if (waitTime > 0) {
+          // sleeps for the wait time or 
+          // until there are empty slots to schedule tasks
+          synchronized (finishedCount) {
+            if (finishedCount.get() == 0) {
+              finishedCount.wait(waitTime);
+            }
+            finishedCount.set(0);
+          }
+        }
+
+        // If the TaskTracker is just starting up:
+        // 1. Verify the buildVersion
+        // 2. Get the system directory & filesystem
+        if(justInited) {
+          String jobTrackerBV = jobClient.getBuildVersion();
+          if(!VersionInfo.getBuildVersion().equals(jobTrackerBV)) {
+            String msg = "Shutting down. Incompatible buildVersion." +
+            "\nJobTracker's: " + jobTrackerBV + 
+            "\nTaskTracker's: "+ VersionInfo.getBuildVersion();
+            LOG.error(msg);
+            try {
+              jobClient.reportTaskTrackerError(taskTrackerName, null, msg);
+            } catch(Exception e ) {
+              LOG.info("Problem reporting to jobtracker: " + e);
+            }
+            return State.DENIED;
+          }
+          
+          String dir = jobClient.getSystemDir();
+          if (dir == null) {
+            throw new IOException("Failed to get system directory");
+          }
+          systemDirectory = new Path(dir);
+          systemFS = systemDirectory.getFileSystem(fConf);
+        }
+
+        now = System.currentTimeMillis();
+        if (now > (lastCheckDirsTime + diskHealthCheckInterval)) {
+          localStorage.checkDirs();
+          lastCheckDirsTime = now;
+          int numFailures = localStorage.numFailures();
+          // Re-init the task tracker if there were any new failures
+          if (numFailures > lastNumFailures) {
+            lastNumFailures = numFailures;
+            return State.STALE;
+          }
+        }
+
+        // Send the heartbeat and process the jobtracker's directives
+        HeartbeatResponse heartbeatResponse = transmitHeartBeat(now);
+
+        // Note the time when the heartbeat returned, use this to decide when to send the
+        // next heartbeat   
+        lastHeartbeat = System.currentTimeMillis();
+        
+        // Check if the map-event list needs purging
+        Set<JobID> jobs = heartbeatResponse.getRecoveredJobs();
+        if (jobs.size() > 0) {
+          synchronized (this) {
+            // purge the local map events list
+            for (JobID job : jobs) {
+              RunningJob rjob;
+              synchronized (runningJobs) {
+                rjob = runningJobs.get(job);          
+                if (rjob != null) {
+                  synchronized (rjob) {
+                    FetchStatus f = rjob.getFetchStatus();
+                    if (f != null) {
+                      f.reset();
+                    }
+                  }
+                }
+              }
+            }
+
+            // Mark the reducers in shuffle for rollback
+            synchronized (shouldReset) {
+              for (Map.Entry<TaskAttemptID, TaskInProgress> entry 
+                   : runningTasks.entrySet()) {
+                if (entry.getValue().getStatus().getPhase() == Phase.SHUFFLE) {
+                  this.shouldReset.add(entry.getKey());
+                }
+              }
+            }
+          }
+        }
+        
+        TaskTrackerAction[] actions = heartbeatResponse.getActions();
+        if(LOG.isDebugEnabled()) {
+          LOG.debug("Got heartbeatResponse from JobTracker with responseId: " + 
+                    heartbeatResponse.getResponseId() + " and " + 
+                    ((actions != null) ? actions.length : 0) + " actions");
+        }
+        if (reinitTaskTracker(actions)) {
+          return State.STALE;
+        }
+            
+        // resetting heartbeat interval from the response.
+        heartbeatInterval = heartbeatResponse.getHeartbeatInterval();
+        justStarted = false;
+        justInited = false;
+        if (actions != null){ 
+          for(TaskTrackerAction action: actions) {
+            if (action instanceof LaunchTaskAction) {
+              addToTaskQueue((LaunchTaskAction)action);
+            } else if (action instanceof CommitTaskAction) {
+              CommitTaskAction commitAction = (CommitTaskAction)action;
+              if (!commitResponses.contains(commitAction.getTaskID())) {
+                LOG.info("Received commit task action for " + 
+                          commitAction.getTaskID());
+                commitResponses.add(commitAction.getTaskID());
+              }
+            } else {
+              tasksToCleanup.put(action);
+            }
+          }
+        }
+        markUnresponsiveTasks();
+        killOverflowingTasks();
+            
+        //we've cleaned up, resume normal operation
+        if (!acceptNewTasks && isIdle()) {
+          acceptNewTasks=true;
+        }
+        //The check below may not be required every iteration but we are 
+        //erring on the side of caution here. We have seen many cases where
+        //the call to jetty's getLocalPort() returns different values at 
+        //different times. Being a real paranoid here.
+        checkJettyPort(server.getPort());
+      } catch (InterruptedException ie) {
+        LOG.info("Interrupted. Closing down.");
+        return State.INTERRUPTED;
+      } catch (DiskErrorException de) {
+        String msg = "Exiting task tracker for disk error:\n" +
+          StringUtils.stringifyException(de);
+        LOG.error(msg);
+        synchronized (this) {
+          jobClient.reportTaskTrackerError(taskTrackerName, 
+                                           "DiskErrorException", msg);
+        }
+        return State.STALE;
+      } catch (RemoteException re) {
+        String reClass = re.getClassName();
+        if (DisallowedTaskTrackerException.class.getName().equals(reClass)) {
+          LOG.info("Tasktracker disallowed by JobTracker.");
+          return State.DENIED;
+        }
+      } catch (Exception except) {
+        String msg = "Caught exception: " + 
+          StringUtils.stringifyException(except);
+        LOG.error(msg);
+      }
+    }
+
+    return State.NORMAL;
+  }
+
+  private long previousUpdate = 0;
+
+  void setIndexCache(IndexCache cache) {
+    this.indexCache = cache;
+  }
+
+  /**
+   * Build and transmit the heart beat to the JobTracker
+   * @param now current time
+   * @return false if the tracker was unknown
+   * @throws IOException
+   */
+  HeartbeatResponse transmitHeartBeat(long now) throws IOException {
+    // Send Counters in the status once every COUNTER_UPDATE_INTERVAL
+    boolean sendCounters;
+    if (now > (previousUpdate + COUNTER_UPDATE_INTERVAL)) {
+      sendCounters = true;
+      previousUpdate = now;
+    }
+    else {
+      sendCounters = false;
+    }
+
+    // 
+    // Check if the last heartbeat got through... 
+    // if so then build the heartbeat information for the JobTracker;
+    // else resend the previous status information.
+    //
+    if (status == null) {
+      synchronized (this) {
+        status = new TaskTrackerStatus(taskTrackerName, localHostname, 
+                                       httpPort, 
+                                       cloneAndResetRunningTaskStatuses(
+                                         sendCounters), 
+                                       failures, 
+                                       maxMapSlots,
+                                       maxReduceSlots); 
+      }
+    } else {
+      LOG.info("Resending 'status' to '" + jobTrackAddr.getHostName() +
+               "' with reponseId '" + heartbeatResponseId);
+    }
+      
+    //
+    // Check if we should ask for a new Task
+    //
+    boolean askForNewTask;
+    long localMinSpaceStart;
+    synchronized (this) {
+      askForNewTask = 
+        ((status.countOccupiedMapSlots() < maxMapSlots || 
+          status.countOccupiedReduceSlots() < maxReduceSlots) && 
+         acceptNewTasks); 
+      localMinSpaceStart = minSpaceStart;
+    }
+    if (askForNewTask) {
+      askForNewTask = enoughFreeSpace(localMinSpaceStart);
+      long freeDiskSpace = getFreeSpace();
+      long totVmem = getTotalVirtualMemoryOnTT();
+      long totPmem = getTotalPhysicalMemoryOnTT();
+
+      status.getResourceStatus().setAvailableSpace(freeDiskSpace);
+      status.getResourceStatus().setTotalVirtualMemory(totVmem);
+      status.getResourceStatus().setTotalPhysicalMemory(totPmem);
+      status.getResourceStatus().setMapSlotMemorySizeOnTT(
+          mapSlotMemorySizeOnTT);
+      status.getResourceStatus().setReduceSlotMemorySizeOnTT(
+          reduceSlotSizeMemoryOnTT);
+    }
+    //add node health information
+    
+    TaskTrackerHealthStatus healthStatus = status.getHealthStatus();
+    synchronized (this) {
+      if (healthChecker != null) {
+        healthChecker.setHealthStatus(healthStatus);
+      } else {
+        healthStatus.setNodeHealthy(true);
+        healthStatus.setLastReported(0L);
+        healthStatus.setHealthReport("");
+      }
+    }
+    //
+    // Xmit the heartbeat
+    //
+    HeartbeatResponse heartbeatResponse = jobClient.heartbeat(status, 
+                                                              justStarted,
+                                                              justInited,
+                                                              askForNewTask, 
+                                                              heartbeatResponseId);
+      
+    //
+    // The heartbeat got through successfully!
+    //
+    heartbeatResponseId = heartbeatResponse.getResponseId();
+      
+    synchronized (this) {
+      for (TaskStatus taskStatus : status.getTaskReports()) {
+        if (taskStatus.getRunState() != TaskStatus.State.RUNNING &&
+            taskStatus.getRunState() != TaskStatus.State.UNASSIGNED &&
+            taskStatus.getRunState() != TaskStatus.State.COMMIT_PENDING &&
+            !taskStatus.inTaskCleanupPhase()) {
+          if (taskStatus.getIsMap()) {
+            mapTotal--;
+          } else {
+            reduceTotal--;
+          }
+          myInstrumentation.completeTask(taskStatus.getTaskID());
+          runningTasks.remove(taskStatus.getTaskID());
+        }
+      }
+      
+      // Clear transient status information which should only
+      // be sent once to the JobTracker
+      for (TaskInProgress tip: runningTasks.values()) {
+        tip.getStatus().clearStatus();
+      }
+    }
+
+    // Force a rebuild of 'status' on the next iteration
+    status = null;                                
+
+    return heartbeatResponse;
+  }
+
+  /**
+   * Return the total virtual memory available on this TaskTracker.
+   * @return total size of virtual memory.
+   */
+  long getTotalVirtualMemoryOnTT() {
+    return totalVirtualMemoryOnTT;
+  }
+
+  /**
+   * Return the total physical memory available on this TaskTracker.
+   * @return total size of physical memory.
+   */
+  long getTotalPhysicalMemoryOnTT() {
+    return totalPhysicalMemoryOnTT;
+  }
+
+  long getTotalMemoryAllottedForTasksOnTT() {
+    return totalMemoryAllottedForTasks;
+  }
+
+  long getRetainSize(org.apache.hadoop.mapreduce.TaskAttemptID tid) {
+    return tid.isMap() ? mapRetainSize : reduceRetainSize;
+  }
+  
+  /**
+   * Check if the jobtracker directed a 'reset' of the tasktracker.
+   * 
+   * @param actions the directives of the jobtracker for the tasktracker.
+   * @return <code>true</code> if tasktracker is to be reset, 
+   *         <code>false</code> otherwise.
+   */
+  private boolean reinitTaskTracker(TaskTrackerAction[] actions) {
+    if (actions != null) {
+      for (TaskTrackerAction action : actions) {
+        if (action.getActionId() == 
+            TaskTrackerAction.ActionType.REINIT_TRACKER) {
+          LOG.info("Recieved ReinitTrackerAction from JobTracker");
+          return true;
+        }
+      }
+    }
+    return false;
+  }
+    
+  /**
+   * Kill any tasks that have not reported progress in the last X seconds.
+   */
+  private synchronized void markUnresponsiveTasks() throws IOException {
+    long now = System.currentTimeMillis();
+    for (TaskInProgress tip: runningTasks.values()) {
+      if (tip.getRunState() == TaskStatus.State.RUNNING ||
+          tip.getRunState() == TaskStatus.State.COMMIT_PENDING ||
+          tip.isCleaningup()) {
+        // Check the per-job timeout interval for tasks;
+        // an interval of '0' implies it is never timed-out
+        long jobTaskTimeout = tip.getTaskTimeout();
+        if (jobTaskTimeout == 0) {
+          continue;
+        }
+          
+        // Check if the task has not reported progress for a 
+        // time-period greater than the configured time-out
+        long timeSinceLastReport = now - tip.getLastProgressReport();
+        if (timeSinceLastReport > jobTaskTimeout && !tip.wasKilled) {
+          String msg = 
+            "Task " + tip.getTask().getTaskID() + " failed to report status for " 
+            + (timeSinceLastReport / 1000) + " seconds. Killing!";
+          LOG.info(tip.getTask().getTaskID() + ": " + msg);
+          ReflectionUtils.logThreadInfo(LOG, "lost task", 30);
+          tip.reportDiagnosticInfo(msg);
+          myInstrumentation.timedoutTask(tip.getTask().getTaskID());
+          purgeTask(tip, true);
+        }
+      }
+    }
+  }
+
+  /**
+   * The task tracker is done with this job, so we need to clean up.
+   * @param action The action with the job
+   * @throws IOException
+   */
+  synchronized void purgeJob(KillJobAction action) throws IOException {
+    JobID jobId = action.getJobID();
+    LOG.info("Received 'KillJobAction' for job: " + jobId);
+    RunningJob rjob = null;
+    synchronized (runningJobs) {
+      rjob = runningJobs.get(jobId);
+    }
+      
+    if (rjob == null) {
+      LOG.warn("Unknown job " + jobId + " being deleted.");
+    } else {
+      synchronized (rjob) {
+        // decrement the reference counts for the items this job references
+        rjob.distCacheMgr.release();
+        // Add this tips of this job to queue of tasks to be purged 
+        for (TaskInProgress tip : rjob.tasks) {
+          tip.jobHasFinished(false);
+          Task t = tip.getTask();
+          if (t.isMapTask()) {
+            indexCache.removeMap(tip.getTask().getTaskID().toString());
+          }
+        }
+        // Delete the job directory for this  
+        // task if the job is done/failed
+        if (!rjob.keepJobFiles) {
+          removeJobFiles(rjob.ugi.getShortUserName(), rjob.getJobID());
+        }
+        // add job to user log manager
+        long now = System.currentTimeMillis();
+        JobCompletedEvent jca = new JobCompletedEvent(rjob
+            .getJobID(), now, UserLogCleaner.getUserlogRetainHours(rjob
+            .getJobConf()));
+        getUserLogManager().addLogEvent(jca);
+
+        // Remove this job 
+        rjob.tasks.clear();
+        // Close all FileSystems for this job
+        try {
+          FileSystem.closeAllForUGI(rjob.getUGI());
+        } catch (IOException ie) {
+          LOG.warn("Ignoring exception " + StringUtils.stringifyException(ie) + 
+              " while closing FileSystem for " + rjob.getUGI());
+        }
+      }
+    }
+
+    synchronized(runningJobs) {
+      runningJobs.remove(jobId);
+    }
+    getJobTokenSecretManager().removeTokenForJob(jobId.toString());  
+  }      
+    
+  /**
+   * This job's files are no longer needed on this TT, remove them.
+   *
+   * @param rjob
+   * @throws IOException
+   */
+  void removeJobFiles(String user, JobID jobId) throws IOException {
+    String userDir = getUserDir(user);
+    String jobDir = getLocalJobDir(user, jobId.toString());
+    PathDeletionContext jobCleanup = 
+      new TaskController.DeletionContext(getTaskController(), false, user, 
+                                         jobDir.substring(userDir.length()));
+    directoryCleanupThread.addToQueue(jobCleanup);
+    
+    for (String str : localStorage.getDirs()) {
+      Path ttPrivateJobDir = FileSystem.getLocal(fConf).makeQualified(
+        new Path(str, TaskTracker.getPrivateDirForJob(user, jobId.toString())));
+      PathDeletionContext ttPrivateJobCleanup =
+        new CleanupQueue.PathDeletionContext(ttPrivateJobDir, fConf);
+      directoryCleanupThread.addToQueue(ttPrivateJobCleanup);
+    }
+  }
+
+  /**
+   * Remove the tip and update all relevant state.
+   * 
+   * @param tip {@link TaskInProgress} to be removed.
+   * @param wasFailure did the task fail or was it killed?
+   */
+  private void purgeTask(TaskInProgress tip, boolean wasFailure) 
+  throws IOException {
+    if (tip != null) {
+      LOG.info("About to purge task: " + tip.getTask().getTaskID());
+        
+      // Remove the task from running jobs, 
+      // removing the job if it's the last task
+      removeTaskFromJob(tip.getTask().getJobID(), tip);
+      tip.jobHasFinished(wasFailure);
+      if (tip.getTask().isMapTask()) {
+        indexCache.removeMap(tip.getTask().getTaskID().toString());
+      }
+    }
+  }
+
+  /** Check if we're dangerously low on disk space
+   * If so, kill jobs to free up space and make sure
+   * we don't accept any new tasks
+   * Try killing the reduce jobs first, since I believe they
+   * use up most space
+   * Then pick the one with least progress
+   */
+  private void killOverflowingTasks() throws IOException {
+    long localMinSpaceKill;
+    synchronized(this){
+      localMinSpaceKill = minSpaceKill;  
+    }
+    if (!enoughFreeSpace(localMinSpaceKill)) {
+      acceptNewTasks=false; 
+      //we give up! do not accept new tasks until
+      //all the ones running have finished and they're all cleared up
+      synchronized (this) {
+        TaskInProgress killMe = findTaskToKill(null);
+
+        if (killMe!=null) {
+          String msg = "Tasktracker running out of space." +
+            " Killing task.";
+          LOG.info(killMe.getTask().getTaskID() + ": " + msg);
+          killMe.reportDiagnosticInfo(msg);
+          purgeTask(killMe, false);
+        }
+      }
+    }
+  }
+
+  /**
+   * Pick a task to kill to free up memory/disk-space 
+   * @param tasksToExclude tasks that are to be excluded while trying to find a
+   *          task to kill. If null, all runningTasks will be searched.
+   * @return the task to kill or null, if one wasn't found
+   */
+  synchronized TaskInProgress findTaskToKill(List<TaskAttemptID> tasksToExclude) {
+    TaskInProgress killMe = null;
+    for (Iterator it = runningTasks.values().iterator(); it.hasNext();) {
+      TaskInProgress tip = (TaskInProgress) it.next();
+
+      if (tasksToExclude != null
+          && tasksToExclude.contains(tip.getTask().getTaskID())) {
+        // exclude this task
+        continue;
+      }
+
+      if ((tip.getRunState() == TaskStatus.State.RUNNING ||
+           tip.getRunState() == TaskStatus.State.COMMIT_PENDING) &&
+          !tip.wasKilled) {
+                
+        if (killMe == null) {
+          killMe = tip;
+
+        } else if (!tip.getTask().isMapTask()) {
+          //reduce task, give priority
+          if (killMe.getTask().isMapTask() || 
+              (tip.getTask().getProgress().get() < 
+               killMe.getTask().getProgress().get())) {
+
+            killMe = tip;
+          }
+
+        } else if (killMe.getTask().isMapTask() &&
+                   tip.getTask().getProgress().get() < 
+                   killMe.getTask().getProgress().get()) {
+          //map task, only add if the progress is lower
+
+          killMe = tip;
+        }
+      }
+    }
+    return killMe;
+  }
+
+  /**
+   * Check if any of the local directories has enough
+   * free space  (more than minSpace)
+   * 
+   * If not, do not try to get a new task assigned 
+   * @return
+   * @throws IOException 
+   */
+  private boolean enoughFreeSpace(long minSpace) throws IOException {
+    if (minSpace == 0) {
+      return true;
+    }
+    return minSpace < getFreeSpace();
+  }
+  
+  private long getFreeSpace() throws IOException {
+    long biggestSeenSoFar = 0;
+    String[] localDirs = localStorage.getDirs();
+    for (int i = 0; i < localDirs.length; i++) {
+      DF df = null;
+      if (localDirsDf.containsKey(localDirs[i])) {
+        df = localDirsDf.get(localDirs[i]);
+      } else {
+        df = new DF(new File(localDirs[i]), fConf);
+        localDirsDf.put(localDirs[i], df);
+      }
+
+      long availOnThisVol = df.getAvailable();
+      if (availOnThisVol > biggestSeenSoFar) {
+        biggestSeenSoFar = availOnThisVol;
+      }
+    }
+    
+    //Should ultimately hold back the space we expect running tasks to use but 
+    //that estimate isn't currently being passed down to the TaskTrackers    
+    return biggestSeenSoFar;
+  }
+    
+  private TaskLauncher mapLauncher;
+  private TaskLauncher reduceLauncher;
+  public JvmManager getJvmManagerInstance() {
+    return jvmManager;
+  }
+
+  // called from unit test  
+  void setJvmManagerInstance(JvmManager jvmManager) {
+    this.jvmManager = jvmManager;
+  }
+
+  private void addToTaskQueue(LaunchTaskAction action) {
+    if (action.getTask().isMapTask()) {
+      mapLauncher.addToTaskQueue(action);
+    } else {
+      reduceLauncher.addToTaskQueue(action);
+    }
+  }
+  
+  class TaskLauncher extends Thread {
+    private IntWritable numFreeSlots;
+    private final int maxSlots;
+    private List<TaskInProgress> tasksToLaunch;
+
+    public TaskLauncher(TaskType taskType, int numSlots) {
+      this.maxSlots = numSlots;
+      this.numFreeSlots = new IntWritable(numSlots);
+      this.tasksToLaunch = new LinkedList<TaskInProgress>();
+      setDaemon(true);
+      setName("TaskLauncher for " + taskType + " tasks");
+    }
+
+    public void addToTaskQueue(LaunchTaskAction action) {
+      synchronized (tasksToLaunch) {
+        TaskInProgress tip = registerTask(action, this);
+        tasksToLaunch.add(tip);
+        tasksToLaunch.notifyAll();
+      }
+    }
+    
+    public void cleanTaskQueue() {
+      tasksToLaunch.clear();
+    }
+    
+    public void addFreeSlots(int numSlots) {
+      synchronized (numFreeSlots) {
+        numFreeSlots.set(numFreeSlots.get() + numSlots);
+        assert (numFreeSlots.get() <= maxSlots);
+        LOG.info("addFreeSlot : current free slots : " + numFreeSlots.get());
+        numFreeSlots.notifyAll();
+      }
+    }
+    
+    void notifySlots() {
+      synchronized (numFreeSlots) {
+        numFreeSlots.notifyAll();
+      }
+    }
+
+    int getNumWaitingTasksToLaunch() {
+      synchronized (tasksToLaunch) {
+        return tasksToLaunch.size();
+      }
+    }
+
+    public void run() {
+      while (!Thread.interrupted()) {
+        try {
+          TaskInProgress tip;
+          Task task;
+          synchronized (tasksToLaunch) {
+            while (tasksToLaunch.isEmpty()) {
+              tasksToLaunch.wait();
+            }
+            //get the TIP
+            tip = tasksToLaunch.remove(0);
+            task = tip.getTask();
+            LOG.info("Trying to launch : " + tip.getTask().getTaskID() + 
+                     " which needs " + task.getNumSlotsRequired() + " slots");
+          }
+          //wait for free slots to run
+          synchronized (numFreeSlots) {
+            boolean canLaunch = true;
+            while (numFreeSlots.get() < task.getNumSlotsRequired()) {
+              //Make sure that there is no kill task action for this task!
+              //We are not locking tip here, because it would reverse the
+              //locking order!
+              //Also, Lock for the tip is not required here! because :
+              // 1. runState of TaskStatus is volatile
+              // 2. Any notification is not missed because notification is
+              // synchronized on numFreeSlots. So, while we are doing the check,
+              // if the tip is half way through the kill(), we don't miss
+              // notification for the following wait().
+              if (!tip.canBeLaunched()) {
+                //got killed externally while still in the launcher queue
+                LOG.info("Not blocking slots for " + task.getTaskID()
+                    + " as it got killed externally. Task's state is "
+                    + tip.getRunState());
+                canLaunch = false;
+                break;
+              }
+              LOG.info("TaskLauncher : Waiting for " + task.getNumSlotsRequired() + 
+                       " to launch " + task.getTaskID() + ", currently we have " + 
+                       numFreeSlots.get() + " free slots");
+              numFreeSlots.wait();
+            }
+            if (!canLaunch) {
+              continue;
+            }
+            LOG.info("In TaskLauncher, current free slots : " + numFreeSlots.get()+
+                     " and trying to launch "+tip.getTask().getTaskID() + 
+                     " which needs " + task.getNumSlotsRequired() + " slots");
+            numFreeSlots.set(numFreeSlots.get() - task.getNumSlotsRequired());
+            assert (numFreeSlots.get() >= 0);
+          }
+          synchronized (tip) {
+            //to make sure that there is no kill task action for this
+            if (!tip.canBeLaunched()) {
+              //got killed externally while still in the launcher queue
+              LOG.info("Not launching task " + task.getTaskID() + " as it got"
+                + " killed externally. Task's state is " + tip.getRunState());
+              addFreeSlots(task.getNumSlotsRequired());
+              continue;
+            }
+            tip.slotTaken = true;
+          }
+          //got a free slot. launch the task
+          startNewTask(tip);
+        } catch (InterruptedException e) { 
+          return; // ALL DONE
+        } catch (Throwable th) {
+          LOG.error("TaskLauncher error " + 
+              StringUtils.stringifyException(th));
+        }
+      }
+    }
+  }
+  private TaskInProgress registerTask(LaunchTaskAction action, 
+      TaskLauncher launcher) {
+    Task t = action.getTask();
+    LOG.info("LaunchTaskAction (registerTask): " + t.getTaskID() +
+             " task's state:" + t.getState());
+    TaskInProgress tip = new TaskInProgress(t, this.fConf, launcher);
+    synchronized (this) {
+      tasks.put(t.getTaskID(), tip);
+      runningTasks.put(t.getTaskID(), tip);
+      boolean isMap = t.isMapTask();
+      if (isMap) {
+        mapTotal++;
+      } else {
+        reduceTotal++;
+      }
+    }
+    return tip;
+  }
+
+  /**
+   * Start a new task.
+   * All exceptions are handled locally, so that we don't mess up the
+   * task tracker.
+   * @throws InterruptedException 
+   */
+  void startNewTask(final TaskInProgress tip) throws InterruptedException {
+    Thread launchThread = new Thread(new Runnable() {
+      @Override
+      public void run() {
+        try {
+          RunningJob rjob = localizeJob(tip);
+          tip.getTask().setJobFile(rjob.getLocalizedJobConf().toString());
+          // Localization is done. Neither rjob.jobConf nor rjob.ugi can be null
+          launchTaskForJob(tip, new JobConf(rjob.getJobConf()), rjob); 
+        } catch (Throwable e) {
+          String msg = ("Error initializing " + tip.getTask().getTaskID() + 
+                        ":\n" + StringUtils.stringifyException(e));
+          LOG.warn(msg);
+          tip.reportDiagnosticInfo(msg);
+          try {
+            tip.kill(true);
+            tip.cleanup(true);
+          } catch (IOException ie2) {
+            LOG.info("Error cleaning up " + tip.getTask().getTaskID(), ie2);
+          } catch (InterruptedException ie2) {
+            LOG.info("Error cleaning up " + tip.getTask().getTaskID(), ie2);
+          }
+          if (e instanceof Error) {
+            LOG.error("TaskLauncher error " + 
+                StringUtils.stringifyException(e));
+          }
+        }
+      }
+    });
+    launchThread.start();
+  }
+
+  void addToMemoryManager(TaskAttemptID attemptId, boolean isMap,
+                          JobConf conf) {
+    if (isTaskMemoryManagerEnabled()) {
+      taskMemoryManager.addTask(attemptId, 
+          isMap ? conf
+              .getMemoryForMapTask() * 1024 * 1024L : conf
+              .getMemoryForReduceTask() * 1024 * 1024L);
+    }
+  }
+
+  void removeFromMemoryManager(TaskAttemptID attemptId) {
+    // Remove the entry from taskMemoryManagerThread's data structures.
+    if (isTaskMemoryManagerEnabled()) {
+      taskMemoryManager.removeTask(attemptId);
+    }
+  }
+
+  /** 
+   * Notify the tasktracker to send an out-of-band heartbeat.
+   */
+  private void notifyTTAboutTaskCompletion() {
+    if (oobHeartbeatOnTaskCompletion) {
+      synchronized (finishedCount) {
+        int value = finishedCount.get();
+        finishedCount.set(value+1);
+        finishedCount.notify();
+      }
+    }
+  }
+  
+  /**
+   * The server retry loop.  
+   * This while-loop attempts to connect to the JobTracker.  It only 
+   * loops when the old TaskTracker has gone bad (its state is
+   * stale somehow) and we need to reinitialize everything.
+   */
+  public void run() {
+    try {
+      getUserLogManager().start();
+      startCleanupThreads();
+      boolean denied = false;
+      while (running && !shuttingDown && !denied) {
+        boolean staleState = false;
+        try {
+          // This while-loop attempts reconnects if we get network errors
+          while (running && !staleState && !shuttingDown && !denied) {
+            try {
+              State osState = offerService();
+              if (osState == State.STALE) {
+                staleState = true;
+              } else if (osState == State.DENIED) {
+                denied = true;
+              }
+            } catch (Exception ex) {
+              if (!shuttingDown) {
+                LOG.info("Lost connection to JobTracker [" +
+                         jobTrackAddr + "].  Retrying...", ex);
+                try {
+                  Thread.sleep(5000);
+                } catch (InterruptedException ie) {
+                }
+              }
+            }
+          }
+        } finally {
+          close();
+        }
+        if (shuttingDown) { return; }
+        LOG.warn("Reinitializing local state");
+        initialize();
+      }
+      if (denied) {
+        shutdown();
+      }
+    } catch (IOException iex) {
+      LOG.error("Got fatal exception while reinitializing TaskTracker: " +
+                StringUtils.stringifyException(iex));
+      return;
+    }
+    catch (InterruptedException i) {
+      LOG.error("Got interrupted while reinitializing TaskTracker: " +
+          i.getMessage());
+      return;
+    }
+  }
+    
+  ///////////////////////////////////////////////////////
+  // TaskInProgress maintains all the info for a Task that
+  // lives at this TaskTracker.  It maintains the Task object,
+  // its TaskStatus, and the TaskRunner.
+  ///////////////////////////////////////////////////////
+  class TaskInProgress {
+    Task task;
+    long lastProgressReport;
+    StringBuffer diagnosticInfo = new StringBuffer();
+    private TaskRunner runner;
+    volatile boolean done = false;
+    volatile boolean wasKilled = false;
+    private JobConf ttConf;
+    private JobConf localJobConf;
+    private boolean keepFailedTaskFiles;
+    private boolean alwaysKeepTaskFiles;
+    private TaskStatus taskStatus; 
+    private long taskTimeout;
+    private String debugCommand;
+    private volatile boolean slotTaken = false;
+    private TaskLauncher launcher;
+
+    // The ugi of the user who is running the job. This contains all the tokens
+    // too which will be populated during job-localization
+    private UserGroupInformation ugi;
+
+    UserGroupInformation getUGI() {
+      return ugi;
+    }
+
+    void setUGI(UserGroupInformation userUGI) {
+      ugi = userUGI;
+    }
+
+    /**
+     */
+    public TaskInProgress(Task task, JobConf conf) {
+      this(task, conf, null);
+    }
+    
+    public TaskInProgress(Task task, JobConf conf, TaskLauncher launcher) {
+      this.task = task;
+      this.launcher = launcher;
+      this.lastProgressReport = System.currentTimeMillis();
+      this.ttConf = conf;
+      localJobConf = null;
+      taskStatus = TaskStatus.createTaskStatus(task.isMapTask(), task.getTaskID(), 
+                                               0.0f, 
+                                               task.getNumSlotsRequired(),
+                                               task.getState(),
+                                               diagnosticInfo.toString(), 
+                                               "initializing",  
+                                               getName(), 
+                                               task.isTaskCleanupTask() ? 
+                                                 TaskStatus.Phase.CLEANUP :  
+                                               task.isMapTask()? TaskStatus.Phase.MAP:
+                                               TaskStatus.Phase.SHUFFLE,
+                                               task.getCounters()); 
+      taskTimeout = (10 * 60 * 1000);
+    }
+        
+    void localizeTask(Task task) throws IOException{
+
+      // Do the task-type specific localization
+//TODO: are these calls really required
+      task.localizeConfiguration(localJobConf);
+      
+      task.setConf(localJobConf);
+    }
+        
+    /**
+     */
+    public Task getTask() {
+      return task;
+    }
+    
+    TaskRunner getTaskRunner() {
+      return runner;
+    }
+
+    void setTaskRunner(TaskRunner rnr) {
+      this.runner = rnr;
+    }
+
+    public synchronized void setJobConf(JobConf lconf){
+      this.localJobConf = lconf;
+      keepFailedTaskFiles = localJobConf.getKeepFailedTaskFiles();
+      taskTimeout = localJobConf.getLong("mapred.task.timeout", 
+                                         10 * 60 * 1000);
+      if (task.isMapTask()) {
+        debugCommand = localJobConf.getMapDebugScript();
+      } else {
+        debugCommand = localJobConf.getReduceDebugScript();
+      }
+      String keepPattern = localJobConf.getKeepTaskFilesPattern();
+      if (keepPattern != null) {
+        alwaysKeepTaskFiles = 
+          Pattern.matches(keepPattern, task.getTaskID().toString());
+      } else {
+        alwaysKeepTaskFiles = false;
+      }
+    }
+        
+    public synchronized JobConf getJobConf() {
+      return localJobConf;
+    }
+        
+    /**
+     */
+    public synchronized TaskStatus getStatus() {
+      taskStatus.setDiagnosticInfo(diagnosticInfo.toString());
+      if (diagnosticInfo.length() > 0) {
+        diagnosticInfo = new StringBuffer();
+      }
+      
+      return taskStatus;
+    }
+
+    /**
+     * Kick off the task execution
+     */
+    public synchronized void launchTask(RunningJob rjob) throws IOException {
+      if (this.taskStatus.getRunState() == TaskStatus.State.UNASSIGNED ||
+          this.taskStatus.getRunState() == TaskStatus.State.FAILED_UNCLEAN ||
+          this.taskStatus.getRunState() == TaskStatus.State.KILLED_UNCLEAN) {
+        localizeTask(task);
+        if (this.taskStatus.getRunState() == TaskStatus.State.UNASSIGNED) {
+          this.taskStatus.setRunState(TaskStatus.State.RUNNING);
+        }
+        setTaskRunner(task.createRunner(TaskTracker.this, this, rjob));
+        this.runner.start();
+        long now = System.currentTimeMillis();
+        this.taskStatus.setStartTime(now);
+        this.lastProgressReport = now;
+      } else {
+        LOG.info("Not launching task: " + task.getTaskID() + 
+            " since it's state is " + this.taskStatus.getRunState());
+      }
+    }
+
+    boolean isCleaningup() {
+   	  return this.taskStatus.inTaskCleanupPhase();
+    }
+    
+    // checks if state has been changed for the task to be launched
+    boolean canBeLaunched() {
+      return (getRunState() == TaskStatus.State.UNASSIGNED ||
+          getRunState() == TaskStatus.State.FAILED_UNCLEAN ||
+          getRunState() == TaskStatus.State.KILLED_UNCLEAN);
+    }
+
+    /**
+     * The task is reporting its progress
+     */
+    public synchronized void reportProgress(TaskStatus taskStatus) 
+    {
+      LOG.info(task.getTaskID() + " " + taskStatus.getProgress() + 
+          "% " + taskStatus.getStateString());
+      // task will report its state as
+      // COMMIT_PENDING when it is waiting for commit response and 
+      // when it is committing.
+      // cleanup attempt will report its state as FAILED_UNCLEAN/KILLED_UNCLEAN
+      if (this.done || 
+          (this.taskStatus.getRunState() != TaskStatus.State.RUNNING &&
+          this.taskStatus.getRunState() != TaskStatus.State.COMMIT_PENDING &&
+          !isCleaningup()) ||
+          ((this.taskStatus.getRunState() == TaskStatus.State.COMMIT_PENDING ||
+           this.taskStatus.getRunState() == TaskStatus.State.FAILED_UNCLEAN ||
+           this.taskStatus.getRunState() == TaskStatus.State.KILLED_UNCLEAN) &&
+           (taskStatus.getRunState() == TaskStatus.State.RUNNING ||
+            taskStatus.getRunState() == TaskStatus.State.UNASSIGNED))) {
+        //make sure we ignore progress messages after a task has 
+        //invoked TaskUmbilicalProtocol.done() or if the task has been
+        //KILLED/FAILED/FAILED_UNCLEAN/KILLED_UNCLEAN
+        //Also ignore progress update if the state change is from 
+        //COMMIT_PENDING/FAILED_UNCLEAN/KILLED_UNCLEA to RUNNING or UNASSIGNED
+        LOG.info(task.getTaskID() + " Ignoring status-update since " +
+                 ((this.done) ? "task is 'done'" : 
+                                ("runState: " + this.taskStatus.getRunState()))
+                 ); 
+        return;
+      }
+      
+      /** check for counter limits and fail the task in case limits are exceeded **/
+      Counters taskCounters = taskStatus.getCounters();
+      if (taskCounters.size() > Counters.MAX_COUNTER_LIMIT ||
+          taskCounters.getGroupNames().size() > Counters.MAX_GROUP_LIMIT) {
+        LOG.warn("Killing task " + task.getTaskID() + ": " +
+        		"Exceeded limit on counters.");
+        try { 
+          reportDiagnosticInfo("Error: Exceeded counter limits - " +
+          		"Counters=" + taskCounters.size() + " Limit=" 
+              + Counters.MAX_COUNTER_LIMIT  + ". " + 
+              "Groups=" + taskCounters.getGroupNames().size() + " Limit=" +
+              Counters.MAX_GROUP_LIMIT);
+          kill(true);
+        } catch (IOException e) {
+          LOG.error("Error killing task " + task.getTaskID(), e);
+        } catch (InterruptedException e) {
+          LOG.error("Error killing task " + task.getTaskID(), e);
+        }
+      }
+      
+      this.taskStatus.statusUpdate(taskStatus);
+      this.lastProgressReport = System.currentTimeMillis();
+    }
+
+    /**
+     */
+    public long getLastProgressReport() {
+      return lastProgressReport;
+    }
+
+    /**
+     */
+    public TaskStatus.State getRunState() {
+      return taskStatus.getRunState();
+    }
+
+    /**
+     * The task's configured timeout.
+     * 
+     * @return the task's configured timeout.
+     */
+    public long getTaskTimeout() {
+      return taskTimeout;
+    }
+        
+    /**
+     * The task has reported some diagnostic info about its status
+     */
+    public synchronized void reportDiagnosticInfo(String info) {
+      this.diagnosticInfo.append(info);
+    }
+    
+    public synchronized void reportNextRecordRange(SortedRanges.Range range) {
+      this.taskStatus.setNextRecordRange(range);
+    }
+
+    /**
+     * The task is reporting that it's done running
+     */
+    public synchronized void reportDone() {
+      if (isCleaningup()) {
+        if (this.taskStatus.getRunState() == TaskStatus.State.FAILED_UNCLEAN) {
+          this.taskStatus.setRunState(TaskStatus.State.FAILED);
+        } else if (this.taskStatus.getRunState() == 
+                   TaskStatus.State.KILLED_UNCLEAN) {
+          this.taskStatus.setRunState(TaskStatus.State.KILLED);
+        }
+      } else {
+        this.taskStatus.setRunState(TaskStatus.State.SUCCEEDED);
+      }
+      this.taskStatus.setProgress(1.0f);
+      this.taskStatus.setFinishTime(System.currentTimeMillis());
+      this.done = true;
+      jvmManager.taskFinished(runner);
+      runner.signalDone();
+      LOG.info("Task " + task.getTaskID() + " is done.");
+      LOG.info("reported output size for " + task.getTaskID() +  "  was " + taskStatus.getOutputSize());
+
+    }
+    
+    public boolean wasKilled() {
+      return wasKilled;
+    }
+
+    /**
+     * A task is reporting in as 'done'.
+     * 
+     * We need to notify the tasktracker to send an out-of-band heartbeat.
+     * If isn't <code>commitPending</code>, we need to finalize the task
+     * and release the slot it's occupied.
+     * 
+     * @param commitPending is the task-commit pending?
+     */
+    void reportTaskFinished(boolean commitPending) {
+      if (!commitPending) {
+        taskFinished();
+        releaseSlot();
+      }
+      notifyTTAboutTaskCompletion();
+    }
+
+    /* State changes:
+     * RUNNING/COMMIT_PENDING -> FAILED_UNCLEAN/FAILED/KILLED_UNCLEAN/KILLED
+     * FAILED_UNCLEAN -> FAILED
+     * KILLED_UNCLEAN -> KILLED 
+     */
+    private void setTaskFailState(boolean wasFailure) {
+      // go FAILED_UNCLEAN -> FAILED and KILLED_UNCLEAN -> KILLED always
+      if (taskStatus.getRunState() == TaskStatus.State.FAILED_UNCLEAN) {
+        taskStatus.setRunState(TaskStatus.State.FAILED);
+      } else if (taskStatus.getRunState() == 
+                 TaskStatus.State.KILLED_UNCLEAN) {
+        taskStatus.setRunState(TaskStatus.State.KILLED);
+      } else if (task.isMapOrReduce() && 
+                 taskStatus.getPhase() != TaskStatus.Phase.CLEANUP) {
+        if (wasFailure) {
+          taskStatus.setRunState(TaskStatus.State.FAILED_UNCLEAN);
+        } else {
+          taskStatus.setRunState(TaskStatus.State.KILLED_UNCLEAN);
+        }
+      } else {
+        if (wasFailure) {
+          taskStatus.setRunState(TaskStatus.State.FAILED);
+        } else {
+          taskStatus.setRunState(TaskStatus.State.KILLED);
+        }
+      }
+    }
+    
+    /**
+     * The task has actually finished running.
+     */
+    public void taskFinished() {
+      long start = System.currentTimeMillis();
+
+      //
+      // Wait until task reports as done.  If it hasn't reported in,
+      // wait for a second and try again.
+      //
+      while (!done && (System.currentTimeMillis() - start < WAIT_FOR_DONE)) {
+        try {
+          Thread.sleep(1000);
+        } catch (InterruptedException ie) {
+        }
+      }
+
+      //
+      // Change state to success or failure, depending on whether
+      // task was 'done' before terminating
+      //
+      boolean needCleanup = false;
+      synchronized (this) {
+        // Remove the task from MemoryManager, if the task SUCCEEDED or FAILED.
+        // KILLED tasks are removed in method kill(), because Kill 
+        // would result in launching a cleanup attempt before 
+        // TaskRunner returns; if remove happens here, it would remove
+        // wrong task from memory manager.
+        if (done || !wasKilled) {
+          removeFromMemoryManager(task.getTaskID());
+        }
+        if (!done) {
+          if (!wasKilled) {
+            failures += 1;
+            setTaskFailState(true);
+            // call the script here for the failed tasks.
+            if (debugCommand != null) {
+              String taskStdout ="";
+              String taskStderr ="";
+              String taskSyslog ="";
+              String jobConf = task.getJobFile();
+              try {
+                Map<LogName, LogFileDetail> allFilesDetails = TaskLog
+                    .getAllLogsFileDetails(task.getTaskID(), task
+                        .isTaskCleanupTask());
+                // get task's stdout file
+                taskStdout =
+                    TaskLog.getRealTaskLogFilePath(
+                        allFilesDetails.get(LogName.STDOUT).location,
+                        LogName.STDOUT);
+                // get task's stderr file
+                taskStderr =
+                    TaskLog.getRealTaskLogFilePath(
+                        allFilesDetails.get(LogName.STDERR).location,
+                        LogName.STDERR);
+                // get task's syslog file
+                taskSyslog =
+                    TaskLog.getRealTaskLogFilePath(
+                        allFilesDetails.get(LogName.SYSLOG).location,
+                        LogName.SYSLOG);
+              } catch(IOException e){
+                LOG.warn("Exception finding task's stdout/err/syslog files");
+              }
+              File workDir = null;
+              try {
+                workDir =
+                    new File(lDirAlloc.getLocalPathToRead(
+                        TaskTracker.getLocalTaskDir(task.getUser(), task
+                            .getJobID().toString(), task.getTaskID()
+                            .toString(), task.isTaskCleanupTask())
+                            + Path.SEPARATOR + MRConstants.WORKDIR,
+                        localJobConf).toString());
+              } catch (IOException e) {
+                LOG.warn("Working Directory of the task " + task.getTaskID() +
+                                " doesnt exist. Caught exception " +
+                          StringUtils.stringifyException(e));
+              }
+              // Build the command  
+              File stdout = TaskLog.getTaskLogFile(task.getTaskID(), task
+                  .isTaskCleanupTask(), TaskLog.LogName.DEBUGOUT);
+              // add pipes program as argument if it exists.
+              String program ="";
+              String executable = Submitter.getExecutable(localJobConf);
+              if ( executable != null) {
+            	try {
+            	  program = new URI(executable).getFragment();
+            	} catch (URISyntaxException ur) {
+            	  LOG.warn("Problem in the URI fragment for pipes executable");
+            	}	  
+              }
+              String [] debug = debugCommand.split(" ");
+              Vector<String> vargs = new Vector<String>();
+              for (String component : debug) {
+                vargs.add(component);
+              }
+              vargs.add(taskStdout);
+              vargs.add(taskStderr);
+              vargs.add(taskSyslog);
+              vargs.add(jobConf);
+              vargs.add(program);
+              try {
+                List<String>  wrappedCommand = TaskLog.captureDebugOut
+                                                          (vargs, stdout);
+                // run the script.
+                try {
+                  runScript(wrappedCommand, workDir);
+                } catch (IOException ioe) {
+                  LOG.warn("runScript failed with: " + StringUtils.
+                                                      stringifyException(ioe));
+                }
+              } catch(IOException e) {
+                LOG.warn("Error in preparing wrapped debug command");
+              }
+
+              // add all lines of debug out to diagnostics
+              try {
+                int num = localJobConf.getInt("mapred.debug.out.lines", -1);
+                addDiagnostics(FileUtil.makeShellPath(stdout),num,"DEBUG OUT");
+              } catch(IOException ioe) {
+                LOG.warn("Exception in add diagnostics!");
+              }
+
+              // Debug-command is run. Do the post-debug-script-exit debug-logs
+              // processing. Truncate the logs.
+              JvmFinishedEvent jvmFinished = new JvmFinishedEvent(new JVMInfo(
+                  TaskLog.getAttemptDir(task.getTaskID(), task
+                      .isTaskCleanupTask()), Arrays.asList(task)));
+              getUserLogManager().addLogEvent(jvmFinished);
+            }
+          }
+          taskStatus.setProgress(0.0f);
+        }
+        this.taskStatus.setFinishTime(System.currentTimeMillis());
+        needCleanup = (taskStatus.getRunState() == TaskStatus.State.FAILED || 
+                taskStatus.getRunState() == TaskStatus.State.FAILED_UNCLEAN ||
+                taskStatus.getRunState() == TaskStatus.State.KILLED_UNCLEAN || 
+                taskStatus.getRunState() == TaskStatus.State.KILLED);
+      }
+
+      //
+      // If the task has failed, or if the task was killAndCleanup()'ed,
+      // we should clean up right away.  We only wait to cleanup
+      // if the task succeeded, and its results might be useful
+      // later on to downstream job processing.
+      //
+      if (needCleanup) {
+        removeTaskFromJob(task.getJobID(), this);
+      }
+      try {
+        cleanup(needCleanup);
+      } catch (IOException ie) {
+      }
+
+    }
+    
+
+    /**
+     * Runs the script given in args
+     * @param args script name followed by its argumnets
+     * @param dir current working directory.
+     * @throws IOException
+     */
+    public void runScript(List<String> args, File dir) throws IOException {
+      ShellCommandExecutor shexec = 
+              new ShellCommandExecutor(args.toArray(new String[0]), dir);
+      shexec.execute();
+      int exitCode = shexec.getExitCode();
+      if (exitCode != 0) {
+        throw new IOException("Task debug script exit with nonzero status of " 
+                              + exitCode + ".");
+      }
+    }
+
+    /**
+     * Add last 'num' lines of the given file to the diagnostics.
+     * if num =-1, all the lines of file are added to the diagnostics.
+     * @param file The file from which to collect diagnostics.
+     * @param num The number of lines to be sent to diagnostics.
+     * @param tag The tag is printed before the diagnostics are printed. 
+     */
+    public void addDiagnostics(String file, int num, String tag) {
+      RandomAccessFile rafile = null;
+      try {
+        rafile = new RandomAccessFile(file,"r");
+        int no_lines =0;
+        String line = null;
+        StringBuffer tail = new StringBuffer();
+        tail.append("\n-------------------- "+tag+"---------------------\n");
+        String[] lines = null;
+        if (num >0) {
+          lines = new String[num];
+        }
+        while ((line = rafile.readLine()) != null) {
+          no_lines++;
+          if (num >0) {
+            if (no_lines <= num) {
+              lines[no_lines-1] = line;
+            }
+            else { // shift them up
+              for (int i=0; i<num-1; ++i) {
+                lines[i] = lines[i+1];
+              }
+              lines[num-1] = line;
+            }
+          }
+          else if (num == -1) {
+            tail.append(line); 
+            tail.append("\n");
+          }
+        }
+        int n = no_lines > num ?num:no_lines;
+        if (num >0) {
+          for (int i=0;i<n;i++) {
+            tail.append(lines[i]);
+            tail.append("\n");
+          }
+        }
+        if(n!=0)
+          reportDiagnosticInfo(tail.toString());
+      } catch (FileNotFoundException fnfe){
+        LOG.warn("File "+file+ " not found");
+      } catch (IOException ioe){
+        LOG.warn("Error reading file "+file);
+      } finally {
+         try {
+           if (rafile != null) {
+             rafile.close();
+           }
+         } catch (IOException ioe) {
+           LOG.warn("Error closing file "+file);
+         }
+      }
+    }
+    
+    /**
+     * We no longer need anything from this task, as the job has
+     * finished.  If the task is still running, kill it and clean up.
+     * 
+     * @param wasFailure did the task fail, as opposed to was it killed by
+     *                   the framework
+     */
+    public void jobHasFinished(boolean wasFailure) throws IOException {
+      // Kill the task if it is still running
+      synchronized(this){
+        if (getRunState() == TaskStatus.State.RUNNING ||
+            getRunState() == TaskStatus.State.UNASSIGNED ||
+            getRunState() == TaskStatus.State.COMMIT_PENDING ||
+            isCleaningup()) {
+          try {
+            kill(wasFailure);
+          } catch (InterruptedException e) {
+            throw new IOException("Interrupted while killing " +
+                getTask().getTaskID(), e);
+          }
+        }
+      }
+      
+      // Cleanup on the finished task
+      cleanup(true);
+    }
+
+    /**
+     * Something went wrong and the task must be killed.
+     * @param wasFailure was it a failure (versus a kill request)?
+     * @throws InterruptedException 
+     */
+    public synchronized void kill(boolean wasFailure
+                                  ) throws IOException, InterruptedException {
+      if (taskStatus.getRunState() == TaskStatus.State.RUNNING ||
+          taskStatus.getRunState() == TaskStatus.State.COMMIT_PENDING ||
+          isCleaningup()) {
+        wasKilled = true;
+        if (wasFailure) {
+          failures += 1;
+        }
+        // runner could be null if task-cleanup attempt is not localized yet
+        if (runner != null) {
+          runner.kill();
+        }
+        setTaskFailState(wasFailure);
+      } else if (taskStatus.getRunState() == TaskStatus.State.UNASSIGNED) {
+        if (wasFailure) {
+          failures += 1;
+          taskStatus.setRunState(TaskStatus.State.FAILED);
+        } else {
+          taskStatus.setRunState(TaskStatus.State.KILLED);
+        }
+      }
+      taskStatus.setFinishTime(System.currentTimeMillis());
+      removeFromMemoryManager(task.getTaskID());
+      releaseSlot();
+      notifyTTAboutTaskCompletion();
+    }
+    
+    private synchronized void releaseSlot() {
+      if (slotTaken) {
+        if (launcher != null) {
+          launcher.addFreeSlots(task.getNumSlotsRequired());
+        }
+        slotTaken = false;
+      } else {
+        // wake up the launcher. it may be waiting to block slots for this task.
+        if (launcher != null) {
+          launcher.notifySlots();
+        }
+      }
+    }
+
+    /**
+     * The map output has been lost.
+     */
+    private synchronized void mapOutputLost(String failure
+                                           ) throws IOException {
+      if (taskStatus.getRunState() == TaskStatus.State.COMMIT_PENDING || 
+          taskStatus.getRunState() == TaskStatus.State.SUCCEEDED) {
+        // change status to failure
+        LOG.info("Reporting output lost:"+task.getTaskID());
+        taskStatus.setRunState(TaskStatus.State.FAILED);
+        taskStatus.setProgress(0.0f);
+        reportDiagnosticInfo("Map output lost, rescheduling: " + 
+                             failure);
+        runningTasks.put(task.getTaskID(), this);
+        mapTotal++;
+      } else {
+        LOG.warn("Output already reported lost:"+task.getTaskID());
+      }
+    }
+
+    /**
+     * We no longer need anything from this task.  Either the 
+     * controlling job is all done and the files have been copied
+     * away, or the task failed and we don't need the remains.
+     * Any calls to cleanup should not lock the tip first.
+     * cleanup does the right thing- updates tasks in Tasktracker
+     * by locking tasktracker first and then locks the tip.
+     * 
+     * if needCleanup is true, the whole task directory is cleaned up.
+     * otherwise the current working directory of the task 
+     * i.e. &lt;taskid&gt;/work is cleaned up.
+     */
+    void cleanup(boolean needCleanup) throws IOException {
+      TaskAttemptID taskId = task.getTaskID();
+      LOG.debug("Cleaning up " + taskId);
+
+
+      synchronized (TaskTracker.this) {
+        if (needCleanup) {
+          // see if tasks data structure is holding this tip.
+          // tasks could hold the tip for cleanup attempt, if cleanup attempt 
+          // got launched before this method.
+          if (tasks.get(taskId) == this) {
+            tasks.remove(taskId);
+          }
+        }
+        synchronized (this){
+          if (alwaysKeepTaskFiles ||
+              (taskStatus.getRunState() == TaskStatus.State.FAILED && 
+               keepFailedTaskFiles)) {
+            return;
+          }
+        }
+      }
+      synchronized (this) {
+        // localJobConf could be null if localization has not happened
+        // then no cleanup will be required.
+        if (localJobConf == null) {
+          return;
+        }
+        try {
+          removeTaskFiles(needCleanup);
+        } catch (Throwable ie) {
+          LOG.info("Error cleaning up task runner: "
+              + StringUtils.stringifyException(ie));
+        }
+      }
+    }
+
+    /**
+     * Some or all of the files from this task are no longer required. Remove
+     * them via CleanupQueue.
+     * 
+     * @param removeOutputs remove outputs as well as output
+     * @param taskId
+     * @throws IOException 
+     */
+    void removeTaskFiles(boolean removeOutputs) throws IOException {
+      if (localJobConf.getNumTasksToExecutePerJvm() == 1) {
+        String user = ugi.getShortUserName();
+        int userDirLen = TaskTracker.getUserDir(user).length();
+        String jobId = task.getJobID().toString();
+        String taskId = task.getTaskID().toString();
+        boolean cleanup = task.isTaskCleanupTask();
+        String taskDir;
+        if (!removeOutputs) {
+          taskDir = TaskTracker.getTaskWorkDir(user, jobId, taskId, cleanup);
+        } else {
+          taskDir = TaskTracker.getLocalTaskDir(user, jobId, taskId, cleanup);
+        }
+        PathDeletionContext item =
+          new TaskController.DeletionContext(taskController, false, user,
+                                             taskDir.substring(userDirLen));          
+        directoryCleanupThread.addToQueue(item);
+      }
+    }
+        
+    @Override
+    public boolean equals(Object obj) {
+      return (obj instanceof TaskInProgress) &&
+        task.getTaskID().equals
+        (((TaskInProgress) obj).getTask().getTaskID());
+    }
+        
+    @Override
+    public int hashCode() {
+      return task.getTaskID().hashCode();
+    }
+  }
+  
+  private void validateJVM(TaskInProgress tip, JvmContext jvmContext, TaskAttemptID taskid) throws IOException {
+    if (jvmContext == null) {
+      LOG.warn("Null jvmContext. Cannot verify Jvm. validateJvm throwing exception");
+      throw new IOException("JvmValidate Failed. JvmContext is null - cannot validate JVM");
+    }
+    if (!jvmManager.validateTipToJvm(tip, jvmContext.jvmId)) {
+      throw new IOException("JvmValidate Failed. Ignoring request from task: " + taskid + ", with JvmId: " + jvmContext.jvmId);
+    }
+  }
+  
+  private void authorizeJVM(org.apache.hadoop.mapreduce.JobID jobId) 
+  throws IOException {
+    String currentJobId = 
+      UserGroupInformation.getCurrentUser().getUserName();
+    if (!currentJobId.equals(jobId.toString())) {
+      throw new IOException ("JVM with " + currentJobId + 
+          " is not authorized for " + jobId);
+    }
+  }
+
+    
+  // ///////////////////////////////////////////////////////////////
+  // TaskUmbilicalProtocol
+  /////////////////////////////////////////////////////////////////
+
+  /**
+   * Called upon startup by the child process, to fetch Task data.
+   */
+  public synchronized JvmTask getTask(JvmContext context) 
+  throws IOException {
+    authorizeJVM(context.jvmId.getJobId());
+    JVMId jvmId = context.jvmId;
+    LOG.debug("JVM with ID : " + jvmId + " asked for a task");
+    // save pid of task JVM sent by child
+    jvmManager.setPidToJvm(jvmId, context.pid);
+    if (!jvmManager.isJvmKnown(jvmId)) {
+      LOG.info("Killing unknown JVM " + jvmId);
+      return new JvmTask(null, true);
+    }
+    RunningJob rjob = runningJobs.get(jvmId.getJobId());
+    if (rjob == null) { //kill the JVM since the job is dead
+      LOG.info("Killing JVM " + jvmId + " since job " + jvmId.getJobId() +
+               " is dead");
+      try {
+        jvmManager.killJvm(jvmId);
+      } catch (InterruptedException e) {
+        LOG.warn("Failed to kill " + jvmId, e);
+      }
+      return new JvmTask(null, true);
+    }
+    TaskInProgress tip = jvmManager.getTaskForJvm(jvmId);
+    if (tip == null) {
+      return new JvmTask(null, false);
+    }
+    if (tasks.get(tip.getTask().getTaskID()) != null) { //is task still present
+      LOG.info("JVM with ID: " + jvmId + " given task: " + 
+          tip.getTask().getTaskID());
+      return new JvmTask(tip.getTask(), false);
+    } else {
+      LOG.info("Killing JVM with ID: " + jvmId + " since scheduled task: " + 
+          tip.getTask().getTaskID() + " is " + tip.taskStatus.getRunState());
+      return new JvmTask(null, true);
+    }
+  }
+
+  /**
+   * Called periodically to report Task progress, from 0.0 to 1.0.
+   */
+  public synchronized boolean statusUpdate(TaskAttemptID taskid, 
+                                              TaskStatus taskStatus, 
+                                              JvmContext jvmContext) 
+  throws IOException {
+    authorizeJVM(taskid.getJobID());
+    TaskInProgress tip = tasks.get(taskid);
+    if (tip != null) {
+      try {
+        validateJVM(tip, jvmContext, taskid);
+      } catch (IOException ie) {
+        LOG.warn("Failed validating JVM", ie);
+        return false;
+      }
+      tip.reportProgress(taskStatus);
+      return true;
+    } else {
+      LOG.warn("Progress from unknown child task: "+taskid);
+      return false;
+    }
+  }
+
+  /**
+   * Called when the task dies before completion, and we want to report back
+   * diagnostic info
+   */
+  public synchronized void reportDiagnosticInfo(TaskAttemptID taskid,
+      String info, JvmContext jvmContext) throws IOException {
+    authorizeJVM(taskid.getJobID());
+    TaskInProgress tip = tasks.get(taskid);
+    if (tip != null) {
+      validateJVM(tip, jvmContext, taskid);
+      tip.reportDiagnosticInfo(info);
+    } else {
+      LOG.warn("Error from unknown child task: "+taskid+". Ignored.");
+    }
+  }
+  /**
+   * Meant to be used internally
+   * @param taskid
+   * @param info
+   * @throws IOException
+   */
+  synchronized void reportDiagnosticInfoInternal(TaskAttemptID taskid, 
+      String info) throws IOException {
+    TaskInProgress tip = tasks.get(taskid);
+    if (tip != null) {
+      tip.reportDiagnosticInfo(info);
+    } else {
+      LOG.warn("Error from unknown child task: "+taskid+". Ignored.");
+    }
+  }
+  
+  public synchronized void reportNextRecordRange(TaskAttemptID taskid, 
+      SortedRanges.Range range, JvmContext jvmContext) throws IOException {
+    authorizeJVM(taskid.getJobID());
+    TaskInProgress tip = tasks.get(taskid);
+    if (tip != null) {
+      validateJVM(tip, jvmContext, taskid);
+      tip.reportNextRecordRange(range);
+    } else {
+      LOG.warn("reportNextRecordRange from unknown child task: "+taskid+". " +
+      		"Ignored.");
+    }
+  }
+
+  /** Child checking to see if we're alive. Normally does nothing. */
+  public synchronized boolean ping(TaskAttemptID taskid, JvmContext jvmContext)
+      throws IOException {
+    authorizeJVM(taskid.getJobID());
+    TaskInProgress tip = tasks.get(taskid);
+    if (tip != null) {
+      validateJVM(tip, jvmContext, taskid);
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  /**
+   * Task is reporting that it is in commit_pending
+   * and it is waiting for the commit Response
+   */
+  public synchronized void commitPending(TaskAttemptID taskid,
+                                         TaskStatus taskStatus,
+                                         JvmContext jvmContext) 
+  throws IOException {
+    authorizeJVM(taskid.getJobID());
+    LOG.info("Task " + taskid + " is in commit-pending," +"" +
+             " task state:" +taskStatus.getRunState());
+    // validateJVM is done in statusUpdate
+    if (!statusUpdate(taskid, taskStatus, jvmContext)) {
+      throw new IOException("Task not found for taskid: " + taskid);
+    }
+    reportTaskFinished(taskid, true);
+  }
+  
+  /**
+   * Child checking whether it can commit 
+   */
+  public synchronized boolean canCommit(TaskAttemptID taskid,
+      JvmContext jvmContext) throws IOException {
+    authorizeJVM(taskid.getJobID());
+    TaskInProgress tip = tasks.get(taskid);
+    validateJVM(tip, jvmContext, taskid);
+    return commitResponses.contains(taskid); // don't remove it now
+  }
+  
+  /**
+   * The task is done.
+   */
+  public synchronized void done(TaskAttemptID taskid, JvmContext jvmContext) 
+  throws IOException {
+    authorizeJVM(taskid.getJobID());
+    TaskInProgress tip = tasks.get(taskid);
+    if (tip != null) {
+      validateJVM(tip, jvmContext, taskid);
+      commitResponses.remove(taskid);
+      tip.reportDone();
+    } else {
+      LOG.warn("Unknown child task done: "+taskid+". Ignored.");
+    }
+  }
+
+
+  /** 
+   * A reduce-task failed to shuffle the map-outputs. Kill the task.
+   */  
+  public synchronized void shuffleError(TaskAttemptID taskId, String message, JvmContext jvmContext) 
+  throws IOException { 
+    authorizeJVM(taskId.getJobID());
+    TaskInProgress tip = runningTasks.get(taskId);
+    if (tip != null) {
+      validateJVM(tip, jvmContext, taskId);
+      LOG.fatal("Task: " + taskId + " - Killed due to Shuffle Failure: "
+          + message);
+      tip.reportDiagnosticInfo("Shuffle Error: " + message);
+      purgeTask(tip, true);
+    } else {
+      LOG.warn("Unknown child task shuffleError: " + taskId + ". Ignored.");
+    }
+  }
+
+  /** 
+   * A child task had a local filesystem error. Kill the task.
+   */  
+  public synchronized void fsError(TaskAttemptID taskId, String message,
+      JvmContext jvmContext) throws IOException {
+    authorizeJVM(taskId.getJobID());
+    TaskInProgress tip = runningTasks.get(taskId);
+    if (tip != null) {
+      validateJVM(tip, jvmContext, taskId);
+      LOG.fatal("Task: " + taskId + " - Killed due to FSError: " + message);
+      tip.reportDiagnosticInfo("FSError: " + message);
+      purgeTask(tip, true);
+    } else {
+      LOG.warn("Unknown child task fsError: "+taskId+". Ignored.");
+    }
+  }
+  /**
+   * Meant to be used internally
+   * @param taskId
+   * @param message
+   * @throws IOException
+   */
+  synchronized void fsErrorInternal(TaskAttemptID taskId, String message) 
+  throws IOException {
+    LOG.fatal("Task: " + taskId + " - Killed due to FSError: " + message);
+    TaskInProgress tip = runningTasks.get(taskId);
+    tip.reportDiagnosticInfo("FSError: " + message);
+    purgeTask(tip, true);
+  }
+
+  /** 
+   * A child task had a fatal error. Kill the task.
+   */  
+  public synchronized void fatalError(TaskAttemptID taskId, String msg,
+      JvmContext jvmContext) throws IOException {
+    authorizeJVM(taskId.getJobID());
+    TaskInProgress tip = runningTasks.get(taskId);
+    if (tip != null) {
+      validateJVM(tip, jvmContext, taskId);
+      LOG.fatal("Task: " + taskId + " - Killed : " + msg);
+      tip.reportDiagnosticInfo("Error: " + msg);
+      purgeTask(tip, true);
+    } else {
+      LOG.warn("Unknown child task fatalError: "+taskId+". Ignored.");
+    }
+  }
+
+  public synchronized MapTaskCompletionEventsUpdate getMapCompletionEvents(
+      JobID jobId, int fromEventId, int maxLocs, TaskAttemptID id,
+      JvmContext jvmContext) throws IOException {
+    TaskInProgress tip = runningTasks.get(id);
+    if (tip == null) {
+      throw new IOException("Unknown task; " + id
+          + ". Ignoring getMapCompletionEvents Request");
+    }
+    validateJVM(tip, jvmContext, id);
+    authorizeJVM(jobId);
+    TaskCompletionEvent[]mapEvents = TaskCompletionEvent.EMPTY_ARRAY;
+    synchronized (shouldReset) {
+      if (shouldReset.remove(id)) {
+        return new MapTaskCompletionEventsUpdate(mapEvents, true);
+      }
+    }
+    RunningJob rjob;
+    synchronized (runningJobs) {
+      rjob = runningJobs.get(jobId);          
+      if (rjob != null) {
+        synchronized (rjob) {
+          FetchStatus f = rjob.getFetchStatus();
+          if (f != null) {
+            mapEvents = f.getMapEvents(fromEventId, maxLocs);
+          }
+        }
+      }
+    }
+    return new MapTaskCompletionEventsUpdate(mapEvents, false);
+  }
+    
+  /////////////////////////////////////////////////////
+  //  Called by TaskTracker thread after task process ends
+  /////////////////////////////////////////////////////
+  /**
+   * The task is no longer running.  It may not have completed successfully
+   */
+  void reportTaskFinished(TaskAttemptID taskid, boolean commitPending) {
+    TaskInProgress tip;
+    synchronized (this) {
+      tip = tasks.get(taskid);
+    }
+    if (tip != null) {
+      tip.reportTaskFinished(commitPending);
+    } else {
+      LOG.warn("Unknown child task finished: "+taskid+". Ignored.");
+    }
+  }
+  
+
+  /**
+   * A completed map task's output has been lost.
+   */
+  public synchronized void mapOutputLost(TaskAttemptID taskid,
+                                         String errorMsg) throws IOException {
+    TaskInProgress tip = tasks.get(taskid);
+    if (tip != null) {
+      tip.mapOutputLost(errorMsg);
+    } else {
+      LOG.warn("Unknown child with bad map output: "+taskid+". Ignored.");
+    }
+  }
+    
+  /**
+   *  The datastructure for initializing a job
+   */
+  static class RunningJob{
+    private JobID jobid; 
+    private JobConf jobConf;
+    private Path localizedJobConf;
+    // keep this for later use
+    volatile Set<TaskInProgress> tasks;
+    //the 'localizing' and 'localized' fields have the following
+    //state transitions (first entry is for 'localizing')
+    //{false,false} -> {true,false} -> {false,true}
+    volatile boolean localized;
+    boolean localizing;
+    boolean keepJobFiles;
+    UserGroupInformation ugi;
+    FetchStatus f;
+    TaskDistributedCacheManager distCacheMgr;
+    
+    RunningJob(JobID jobid) {
+      this.jobid = jobid;
+      localized = false;
+      localizing = false;
+      tasks = new HashSet<TaskInProgress>();
+      keepJobFiles = false;
+    }
+      
+    JobID getJobID() {
+      return jobid;
+    }
+      
+    UserGroupInformation getUGI() {
+      return ugi;
+    }
+
+    void setFetchStatus(FetchStatus f) {
+      this.f = f;
+    }
+      
+    FetchStatus getFetchStatus() {
+      return f;
+    }
+
+    JobConf getJobConf() {
+      return jobConf;
+    }
+
+    Path getLocalizedJobConf() {
+      return localizedJobConf;
+    }
+  }
+
+  /**
+   * Get the name for this task tracker.
+   * @return the string like "tracker_mymachine:50010"
+   */
+  String getName() {
+    return taskTrackerName;
+  }
+    
+  private synchronized List<TaskStatus> cloneAndResetRunningTaskStatuses(
+                                          boolean sendCounters) {
+    List<TaskStatus> result = new ArrayList<TaskStatus>(runningTasks.size());
+    for(TaskInProgress tip: runningTasks.values()) {
+      TaskStatus status = tip.getStatus();
+      status.setIncludeCounters(sendCounters);
+      // send counters for finished or failed tasks and commit pending tasks
+      if (status.getRunState() != TaskStatus.State.RUNNING) {
+        status.setIncludeCounters(true);
+      }
+      result.add((TaskStatus)status.clone());
+      status.clearStatus();
+    }
+    return result;
+  }
+  /**
+   * Get the list of tasks that will be reported back to the 
+   * job tracker in the next heartbeat cycle.
+   * @return a copy of the list of TaskStatus objects
+   */
+  synchronized List<TaskStatus> getRunningTaskStatuses() {
+    List<TaskStatus> result = new ArrayList<TaskStatus>(runningTasks.size());
+    for(TaskInProgress tip: runningTasks.values()) {
+      result.add(tip.getStatus());
+    }
+    return result;
+  }
+
+  /**
+   * Get the list of stored tasks on this task tracker.
+   * @return
+   */
+  synchronized List<TaskStatus> getNonRunningTasks() {
+    List<TaskStatus> result = new ArrayList<TaskStatus>(tasks.size());
+    for(Map.Entry<TaskAttemptID, TaskInProgress> task: tasks.entrySet()) {
+      if (!runningTasks.containsKey(task.getKey())) {
+        result.add(task.getValue().getStatus());
+      }
+    }
+    return result;
+  }
+
+
+  /**
+   * Get the list of tasks from running jobs on this task tracker.
+   * @return a copy of the list of TaskStatus objects
+   */
+  synchronized List<TaskStatus> getTasksFromRunningJobs() {
+    List<TaskStatus> result = new ArrayList<TaskStatus>(tasks.size());
+    for (Map.Entry <JobID, RunningJob> item : runningJobs.entrySet()) {
+      RunningJob rjob = item.getValue();
+      synchronized (rjob) {
+        for (TaskInProgress tip : rjob.tasks) {
+          result.add(tip.getStatus());
+        }
+      }
+    }
+    return result;
+  }
+  
+  /**
+   * Get the default job conf for this tracker.
+   */
+  JobConf getJobConf() {
+    return fConf;
+  }
+    
+  /**
+   * Is this task tracker idle?
+   * @return has this task tracker finished and cleaned up all of its tasks?
+   */
+  public synchronized boolean isIdle() {
+    return tasks.isEmpty() && tasksToCleanup.isEmpty();
+  }
+    
+  /**
+   * Start the TaskTracker, point toward the indicated JobTracker
+   */
+  public static void main(String argv[]) throws Exception {
+    StringUtils.startupShutdownMessage(TaskTracker.class, argv, LOG);
+    if (argv.length != 0) {
+      System.out.println("usage: TaskTracker");
+      System.exit(-1);
+    }
+    try {
+      JobConf conf=new JobConf();
+      // enable the server to track time spent waiting on locks
+      ReflectionUtils.setContentionTracing
+        (conf.getBoolean("tasktracker.contention.tracking", false));
+      DefaultMetricsSystem.initialize("TaskTracker");
+      TaskTracker tt = new TaskTracker(conf);
+      MBeans.register("TaskTracker", "TaskTrackerInfo", tt);
+      tt.run();
+    } catch (Throwable e) {
+      LOG.error("Can not start task tracker because "+
+                StringUtils.stringifyException(e));
+      System.exit(-1);
+    }
+  }
+  
+  static class LRUCache<K, V> {
+    private int cacheSize;
+    private LinkedHashMap<K, V> map;
+	
+    public LRUCache(int cacheSize) {
+      this.cacheSize = cacheSize;
+      this.map = new LinkedHashMap<K, V>(cacheSize, 0.75f, true) {
+          protected boolean removeEldestEntry(Map.Entry<K, V> eldest) {
+	    return size() > LRUCache.this.cacheSize;
+	  }
+      };
+    }
+	
+    public synchronized V get(K key) {
+      return map.get(key);
+    }
+	
+    public synchronized void put(K key, V value) {
+      map.put(key, value);
+    }
+	
+    public synchronized int size() {
+      return map.size();
+    }
+	
+    public Iterator<Entry<K, V>> getIterator() {
+      return new LinkedList<Entry<K, V>>(map.entrySet()).iterator();
+    }
+   
+    public synchronized void clear() {
+      map.clear();
+    }
+  }
+
+  /**
+   * This class is used in TaskTracker's Jetty to serve the map outputs
+   * to other nodes.
+   */
+  public static class MapOutputServlet extends HttpServlet {
+    private static final long serialVersionUID = 1L;
+    private static final int MAX_BYTES_TO_READ = 64 * 1024;
+    
+    private static LRUCache<String, Path> fileCache = new LRUCache<String, Path>(FILE_CACHE_SIZE);
+    private static LRUCache<String, Path> fileIndexCache = new LRUCache<String, Path>(FILE_CACHE_SIZE);
+    
+    @Override
+    public void doGet(HttpServletRequest request, 
+                      HttpServletResponse response
+                      ) throws ServletException, IOException {
+      String mapId = request.getParameter("map");
+      String reduceId = request.getParameter("reduce");
+      String jobId = request.getParameter("job");
+
+      if (jobId == null) {
+        throw new IOException("job parameter is required");
+      }
+
+      if (mapId == null || reduceId == null) {
+        throw new IOException("map and reduce parameters are required");
+      }
+      ServletContext context = getServletContext();
+      int reduce = Integer.parseInt(reduceId);
+      byte[] buffer = new byte[MAX_BYTES_TO_READ];
+      // true iff IOException was caused by attempt to access input
+      boolean isInputException = true;
+      OutputStream outStream = null;
+      FileInputStream mapOutputIn = null;
+ 
+      long totalRead = 0;
+      ShuffleServerInstrumentation shuffleMetrics =
+        (ShuffleServerInstrumentation) context.getAttribute("shuffleServerMetrics");
+      TaskTracker tracker = 
+        (TaskTracker) context.getAttribute("task.tracker");
+      String exceptionStackRegex =
+        (String) context.getAttribute("exceptionStackRegex");
+      String exceptionMsgRegex =
+        (String) context.getAttribute("exceptionMsgRegex");
+
+      verifyRequest(request, response, tracker, jobId);
+
+      long startTime = 0;
+      try {
+        shuffleMetrics.serverHandlerBusy();
+        if(ClientTraceLog.isInfoEnabled())
+          startTime = System.nanoTime();
+        outStream = response.getOutputStream();
+        JobConf conf = (JobConf) context.getAttribute("conf");
+        LocalDirAllocator lDirAlloc = 
+          (LocalDirAllocator)context.getAttribute("localDirAllocator");
+        FileSystem rfs = ((LocalFileSystem)
+            context.getAttribute("local.file.system")).getRaw();
+
+      String userName = null;
+      String runAsUserName = null;
+      synchronized (tracker.runningJobs) {
+        RunningJob rjob = tracker.runningJobs.get(JobID.forName(jobId));
+        if (rjob == null) {
+          throw new IOException("Unknown job " + jobId + "!!");
+        }
+        userName = rjob.jobConf.getUser();
+        runAsUserName = tracker.getTaskController().getRunAsUser(rjob.jobConf);
+      }
+      // Index file
+      String intermediateOutputDir = TaskTracker.getIntermediateOutputDir(userName, jobId, mapId);
+      String indexKey = intermediateOutputDir + "/file.out.index";
+      Path indexFileName = fileIndexCache.get(indexKey);
+      if (indexFileName == null) {
+        indexFileName = lDirAlloc.getLocalPathToRead(indexKey, conf);
+        fileIndexCache.put(indexKey, indexFileName);
+      }
+
+      // Map-output file
+      String fileKey = intermediateOutputDir + "/file.out";
+      Path mapOutputFileName = fileCache.get(fileKey);
+      if (mapOutputFileName == null) {
+        mapOutputFileName = lDirAlloc.getLocalPathToRead(fileKey, conf);
+        fileCache.put(fileKey, mapOutputFileName);
+      }
+       
+
+        /**
+         * Read the index file to get the information about where
+         * the map-output for the given reducer is available. 
+         */
+        IndexRecord info = 
+          tracker.indexCache.getIndexInformation(mapId, reduce,indexFileName, 
+              runAsUserName);
+          
+        //set the custom "from-map-task" http header to the map task from which
+        //the map output data is being transferred
+        response.setHeader(FROM_MAP_TASK, mapId);
+        
+        //set the custom "Raw-Map-Output-Length" http header to 
+        //the raw (decompressed) length
+        response.setHeader(RAW_MAP_OUTPUT_LENGTH,
+            Long.toString(info.rawLength));
+
+        //set the custom "Map-Output-Length" http header to 
+        //the actual number of bytes being transferred
+        response.setHeader(MAP_OUTPUT_LENGTH,
+            Long.toString(info.partLength));
+
+        //set the custom "for-reduce-task" http header to the reduce task number
+        //for which this map output is being transferred
+        response.setHeader(FOR_REDUCE_TASK, Integer.toString(reduce));
+        
+        //use the same buffersize as used for reading the data from disk
+        response.setBufferSize(MAX_BYTES_TO_READ);
+        
+        /**
+         * Read the data from the sigle map-output file and
+         * send it to the reducer.
+         */
+        //open the map-output file
+        mapOutputIn = SecureIOUtils.openForRead(
+            new File(mapOutputFileName.toUri().getPath()), runAsUserName);
+
+        //seek to the correct offset for the reduce
+        mapOutputIn.skip(info.startOffset);
+        long rem = info.partLength;
+        int len =
+          mapOutputIn.read(buffer, 0, (int)Math.min(rem, MAX_BYTES_TO_READ));
+        while (rem > 0 && len >= 0) {
+          rem -= len;
+          try {
+            shuffleMetrics.outputBytes(len);
+            outStream.write(buffer, 0, len);
+            outStream.flush();
+          } catch (IOException ie) {
+            isInputException = false;
+            throw ie;
+          }
+          totalRead += len;
+          len =
+            mapOutputIn.read(buffer, 0, (int)Math.min(rem, MAX_BYTES_TO_READ));
+        }
+        
+        if (LOG.isDebugEnabled()) {
+          LOG.info("Sent out " + totalRead + " bytes for reduce: " + reduce + 
+                 " from map: " + mapId + " given " + info.partLength + "/" + 
+                 info.rawLength);
+        }
+
+      } catch (IOException ie) {
+        Log log = (Log) context.getAttribute("log");
+        String errorMsg = ("getMapOutput(" + mapId + "," + reduceId + 
+                           ") failed :\n"+
+                           StringUtils.stringifyException(ie));
+        log.warn(errorMsg);
+        checkException(ie, exceptionMsgRegex, exceptionStackRegex, shuffleMetrics);
+        if (isInputException) {
+          tracker.mapOutputLost(TaskAttemptID.forName(mapId), errorMsg);
+        }
+        response.sendError(HttpServletResponse.SC_GONE, errorMsg);
+        shuffleMetrics.failedOutput();
+        throw ie;
+      } finally {
+        if (null != mapOutputIn) {
+          mapOutputIn.close();
+        }
+        final long endTime = ClientTraceLog.isInfoEnabled() ? System.nanoTime() : 0;
+        shuffleMetrics.serverHandlerFree();
+        if (ClientTraceLog.isInfoEnabled()) {
+          ClientTraceLog.info(String.format(MR_CLIENTTRACE_FORMAT,
+                request.getLocalAddr() + ":" + request.getLocalPort(),
+                request.getRemoteAddr() + ":" + request.getRemotePort(),
+                totalRead, "MAPRED_SHUFFLE", mapId, endTime-startTime));
+        }
+      }
+      outStream.close();
+      shuffleMetrics.successOutput();
+    }
+    
+    protected void checkException(IOException ie, String exceptionMsgRegex,
+        String exceptionStackRegex, ShuffleServerInstrumentation shuffleMetrics) {
+      // parse exception to see if it looks like a regular expression you
+      // configure. If both msgRegex and StackRegex set then make sure both
+      // match, otherwise only the one set has to match.
+      if (exceptionMsgRegex != null) {
+        String msg = ie.getMessage();
+        if (msg == null || !msg.matches(exceptionMsgRegex)) {
+          return;
+        }
+      }
+      if (exceptionStackRegex != null
+          && !checkStackException(ie, exceptionStackRegex)) {
+        return;
+      }
+      shuffleMetrics.exceptionsCaught();
+    }
+
+    private boolean checkStackException(IOException ie,
+        String exceptionStackRegex) {
+      StackTraceElement[] stack = ie.getStackTrace();
+
+      for (StackTraceElement elem : stack) {
+        String stacktrace = elem.toString();
+        if (stacktrace.matches(exceptionStackRegex)) {
+          return true;
+        }
+      }
+      return false;
+    }
+
+
+    /**
+     * verify that request has correct HASH for the url
+     * and also add a field to reply header with hash of the HASH
+     * @param request
+     * @param response
+     * @param jt the job token
+     * @throws IOException
+     */
+    private void verifyRequest(HttpServletRequest request, 
+        HttpServletResponse response, TaskTracker tracker, String jobId) 
+    throws IOException {
+      SecretKey tokenSecret = tracker.getJobTokenSecretManager()
+          .retrieveTokenSecret(jobId);
+      // string to encrypt
+      String enc_str = SecureShuffleUtils.buildMsgFrom(request);
+      
+      // hash from the fetcher
+      String urlHashStr = request.getHeader(SecureShuffleUtils.HTTP_HEADER_URL_HASH);
+      if(urlHashStr == null) {
+        response.sendError(HttpServletResponse.SC_UNAUTHORIZED);
+        throw new IOException("fetcher cannot be authenticated " + 
+            request.getRemoteHost());
+      }
+      int len = urlHashStr.length();
+      LOG.debug("verifying request. enc_str="+enc_str+"; hash=..."+
+          urlHashStr.substring(len-len/2, len-1)); // half of the hash for debug
+
+      // verify - throws exception
+      try {
+        SecureShuffleUtils.verifyReply(urlHashStr, enc_str, tokenSecret);
+      } catch (IOException ioe) {
+        response.sendError(HttpServletResponse.SC_UNAUTHORIZED);
+        throw ioe;
+      }
+      
+      // verification passed - encode the reply
+      String reply = SecureShuffleUtils.generateHash(urlHashStr.getBytes(), tokenSecret);
+      response.addHeader(SecureShuffleUtils.HTTP_HEADER_REPLY_URL_HASH, reply);
+      
+      len = reply.length();
+      LOG.debug("Fetcher request verfied. enc_str="+enc_str+";reply="
+          +reply.substring(len-len/2, len-1));
+    }
+  }
+  
+
+  // get the full paths of the directory in all the local disks.
+  Path[] getLocalFiles(JobConf conf, String subdir) throws IOException{
+    String[] localDirs = conf.getLocalDirs();
+    Path[] paths = new Path[localDirs.length];
+    FileSystem localFs = FileSystem.getLocal(conf);
+    boolean subdirNeeded = (subdir != null) && (subdir.length() > 0);
+    for (int i = 0; i < localDirs.length; i++) {
+      paths[i] = (subdirNeeded) ? new Path(localDirs[i], subdir)
+                                : new Path(localDirs[i]);
+      paths[i] = paths[i].makeQualified(localFs);
+    }
+    return paths;
+  }
+
+  FileSystem getLocalFileSystem(){
+    return localFs;
+  }
+
+  // only used by tests
+  void setLocalFileSystem(FileSystem fs){
+    localFs = fs;
+  }
+
+  int getMaxCurrentMapTasks() {
+    return maxMapSlots;
+  }
+  
+  int getMaxCurrentReduceTasks() {
+    return maxReduceSlots;
+  }
+
+  //called from unit test
+  synchronized void setMaxMapSlots(int mapSlots) {
+    maxMapSlots = mapSlots;
+  }
+
+  //called from unit test
+  synchronized void setMaxReduceSlots(int reduceSlots) {
+    maxReduceSlots = reduceSlots;
+  }
+
+  /**
+   * Is the TaskMemoryManager Enabled on this system?
+   * @return true if enabled, false otherwise.
+   */
+  public boolean isTaskMemoryManagerEnabled() {
+    return taskMemoryManagerEnabled;
+  }
+  
+  public TaskMemoryManagerThread getTaskMemoryManager() {
+    return taskMemoryManager;
+  }
+
+  /**
+   * Normalize the negative values in configuration
+   * 
+   * @param val
+   * @return normalized val
+   */
+  private long normalizeMemoryConfigValue(long val) {
+    if (val < 0) {
+      val = JobConf.DISABLED_MEMORY_LIMIT;
+    }
+    return val;
+  }
+
+  /**
+   * Memory-related setup
+   */
+  private void initializeMemoryManagement() {
+
+    //handling @deprecated
+    if (fConf.get(MAPRED_TASKTRACKER_VMEM_RESERVED_PROPERTY) != null) {
+      LOG.warn(
+        JobConf.deprecatedString(
+          MAPRED_TASKTRACKER_VMEM_RESERVED_PROPERTY));
+    }
+
+    //handling @deprecated
+    if (fConf.get(MAPRED_TASKTRACKER_PMEM_RESERVED_PROPERTY) != null) {
+      LOG.warn(
+        JobConf.deprecatedString(
+          MAPRED_TASKTRACKER_PMEM_RESERVED_PROPERTY));
+    }
+
+    //handling @deprecated
+    if (fConf.get(JobConf.MAPRED_TASK_DEFAULT_MAXVMEM_PROPERTY) != null) {
+      LOG.warn(
+        JobConf.deprecatedString(
+          JobConf.MAPRED_TASK_DEFAULT_MAXVMEM_PROPERTY));
+    }
+
+    //handling @deprecated
+    if (fConf.get(JobConf.UPPER_LIMIT_ON_TASK_VMEM_PROPERTY) != null) {
+      LOG.warn(
+        JobConf.deprecatedString(
+          JobConf.UPPER_LIMIT_ON_TASK_VMEM_PROPERTY));
+    }
+
+    Class<? extends MemoryCalculatorPlugin> clazz =
+        fConf.getClass(MAPRED_TASKTRACKER_MEMORY_CALCULATOR_PLUGIN_PROPERTY,
+            null, MemoryCalculatorPlugin.class);
+    MemoryCalculatorPlugin memoryCalculatorPlugin =
+        MemoryCalculatorPlugin
+            .getMemoryCalculatorPlugin(clazz, fConf);
+    LOG.info(" Using MemoryCalculatorPlugin : " + memoryCalculatorPlugin);
+
+    if (memoryCalculatorPlugin != null) {
+      totalVirtualMemoryOnTT = memoryCalculatorPlugin.getVirtualMemorySize();
+      if (totalVirtualMemoryOnTT <= 0) {
+        LOG.warn("TaskTracker's totalVmem could not be calculated. "
+            + "Setting it to " + JobConf.DISABLED_MEMORY_LIMIT);
+        totalVirtualMemoryOnTT = JobConf.DISABLED_MEMORY_LIMIT;
+      }
+      totalPhysicalMemoryOnTT = memoryCalculatorPlugin.getPhysicalMemorySize();
+      if (totalPhysicalMemoryOnTT <= 0) {
+        LOG.warn("TaskTracker's totalPmem could not be calculated. "
+            + "Setting it to " + JobConf.DISABLED_MEMORY_LIMIT);
+        totalPhysicalMemoryOnTT = JobConf.DISABLED_MEMORY_LIMIT;
+      }
+    }
+
+    mapSlotMemorySizeOnTT =
+        fConf.getLong(
+            JobTracker.MAPRED_CLUSTER_MAP_MEMORY_MB_PROPERTY,
+            JobConf.DISABLED_MEMORY_LIMIT);
+    reduceSlotSizeMemoryOnTT =
+        fConf.getLong(
+            JobTracker.MAPRED_CLUSTER_REDUCE_MEMORY_MB_PROPERTY,
+            JobConf.DISABLED_MEMORY_LIMIT);
+    totalMemoryAllottedForTasks =
+        maxMapSlots * mapSlotMemorySizeOnTT + maxReduceSlots
+            * reduceSlotSizeMemoryOnTT;
+    if (totalMemoryAllottedForTasks < 0) {
+      //adding check for the old keys which might be used by the administrator
+      //while configuration of the memory monitoring on TT
+      long memoryAllotedForSlot = fConf.normalizeMemoryConfigValue(
+          fConf.getLong(JobConf.MAPRED_TASK_DEFAULT_MAXVMEM_PROPERTY, 
+              JobConf.DISABLED_MEMORY_LIMIT));
+      long limitVmPerTask = fConf.normalizeMemoryConfigValue(
+          fConf.getLong(JobConf.UPPER_LIMIT_ON_TASK_VMEM_PROPERTY, 
+              JobConf.DISABLED_MEMORY_LIMIT));
+      if(memoryAllotedForSlot == JobConf.DISABLED_MEMORY_LIMIT) {
+        totalMemoryAllottedForTasks = JobConf.DISABLED_MEMORY_LIMIT; 
+      } else {
+        if(memoryAllotedForSlot > limitVmPerTask) {
+          LOG.info("DefaultMaxVmPerTask is mis-configured. " +
+          		"It shouldn't be greater than task limits");
+          totalMemoryAllottedForTasks = JobConf.DISABLED_MEMORY_LIMIT;
+        } else {
+          totalMemoryAllottedForTasks = (maxMapSlots + 
+              maxReduceSlots) *  (memoryAllotedForSlot/(1024 * 1024));
+        }
+      }
+    }
+    if (totalMemoryAllottedForTasks > totalPhysicalMemoryOnTT) {
+      LOG.info("totalMemoryAllottedForTasks > totalPhysicalMemoryOnTT."
+          + " Thrashing might happen.");
+    } else if (totalMemoryAllottedForTasks > totalVirtualMemoryOnTT) {
+      LOG.info("totalMemoryAllottedForTasks > totalVirtualMemoryOnTT."
+          + " Thrashing might happen.");
+    }
+
+    // start the taskMemoryManager thread only if enabled
+    setTaskMemoryManagerEnabledFlag();
+    if (isTaskMemoryManagerEnabled()) {
+      taskMemoryManager = new TaskMemoryManagerThread(this);
+      taskMemoryManager.setDaemon(true);
+      taskMemoryManager.start();
+    }
+  }
+
+  void setTaskMemoryManagerEnabledFlag() {
+    if (!ProcfsBasedProcessTree.isAvailable()) {
+      LOG.info("ProcessTree implementation is missing on this system. "
+          + "TaskMemoryManager is disabled.");
+      taskMemoryManagerEnabled = false;
+      return;
+    }
+
+    if (totalMemoryAllottedForTasks == JobConf.DISABLED_MEMORY_LIMIT) {
+      taskMemoryManagerEnabled = false;
+      LOG.warn("TaskTracker's totalMemoryAllottedForTasks is -1."
+          + " TaskMemoryManager is disabled.");
+      return;
+    }
+
+    taskMemoryManagerEnabled = true;
+  }
+
+  /**
+   * Clean-up the task that TaskMemoryMangerThread requests to do so.
+   * @param tid
+   * @param wasFailure mark the task as failed or killed. 'failed' if true,
+   *          'killed' otherwise
+   * @param diagnosticMsg
+   */
+  synchronized void cleanUpOverMemoryTask(TaskAttemptID tid, boolean wasFailure,
+      String diagnosticMsg) {
+    TaskInProgress tip = runningTasks.get(tid);
+    if (tip != null) {
+      tip.reportDiagnosticInfo(diagnosticMsg);
+      try {
+        purgeTask(tip, wasFailure); // Marking it as failed/killed.
+      } catch (IOException ioe) {
+        LOG.warn("Couldn't purge the task of " + tid + ". Error : " + ioe);
+      }
+    }
+  }
+  
+  /**
+   * Wrapper method used by TaskTracker to check if {@link  NodeHealthCheckerService}
+   * can be started
+   * @param conf configuration used to check if service can be started
+   * @return true if service can be started
+   */
+  private boolean shouldStartHealthMonitor(Configuration conf) {
+    return NodeHealthCheckerService.shouldRun(conf);
+  }
+  
+  /**
+   * Wrapper method used to start {@link NodeHealthCheckerService} for 
+   * Task Tracker
+   * @param conf Configuration used by the service.
+   */
+  private void startHealthMonitor(Configuration conf) {
+    healthChecker = new NodeHealthCheckerService(conf);
+    healthChecker.start();
+  }
+  
+  TrackerDistributedCacheManager getTrackerDistributedCacheManager() {
+    return distributedCacheManager;
+  }
+
+    /**
+     * Download the job-token file from the FS and save on local fs.
+     * @param user
+     * @param jobId
+     * @return the local file system path of the downloaded file.
+     * @throws IOException
+     */
+  private String localizeJobTokenFile(String user, JobID jobId)
+        throws IOException {
+      // check if the tokenJob file is there..
+      Path skPath = new Path(systemDirectory, 
+          jobId.toString()+"/"+TokenCache.JOB_TOKEN_HDFS_FILE);
+      
+      FileStatus status = null;
+      long jobTokenSize = -1;
+      status = systemFS.getFileStatus(skPath); //throws FileNotFoundException
+      jobTokenSize = status.getLen();
+      
+      Path localJobTokenFile =
+          lDirAlloc.getLocalPathForWrite(getPrivateDirJobTokenFile(user, 
+              jobId.toString()), jobTokenSize, fConf);
+    
+      String localJobTokenFileStr = localJobTokenFile.toUri().getPath();
+      if(LOG.isDebugEnabled())
+        LOG.debug("localizingJobTokenFile from sd="+skPath.toUri().getPath() + 
+            " to " + localJobTokenFileStr);
+      
+      // Download job_token
+      systemFS.copyToLocalFile(skPath, localJobTokenFile);      
+      return localJobTokenFileStr;
+    }
+
+    JobACLsManager getJobACLsManager() {
+      return aclsManager.getJobACLsManager();
+    }
+    
+    ACLsManager getACLsManager() {
+      return aclsManager;
+    }
+
+  // Begin MXBean implementation
+  @Override
+  public String getHostname() {
+    return localHostname;
+  }
+
+  @Override
+  public String getVersion() {
+    return VersionInfo.getVersion() +", r"+ VersionInfo.getRevision();
+  }
+
+  @Override
+  public String getConfigVersion() {
+    return originalConf.get(CONF_VERSION_KEY, CONF_VERSION_DEFAULT);
+  }
+
+  @Override
+  public String getJobTrackerUrl() {
+    return originalConf.get("mapred.job.tracker");
+  }
+
+  @Override
+  public int getRpcPort() {
+    return taskReportAddress.getPort();
+  }
+
+  @Override
+  public int getHttpPort() {
+    return httpPort;
+  }
+
+  @Override
+  public boolean isHealthy() {
+    boolean healthy = true;
+    TaskTrackerHealthStatus hs = new TaskTrackerHealthStatus();
+    if (healthChecker != null) {
+      healthChecker.setHealthStatus(hs);
+      healthy = hs.isNodeHealthy();
+    }    
+    return healthy;
+  }
+
+  @Override
+  public String getTasksInfoJson() {
+    return getTasksInfo().toJson();
+  }
+
+  InfoMap getTasksInfo() {
+    InfoMap map = new InfoMap();
+    int failed = 0;
+    int commitPending = 0;
+    for (TaskStatus st : getNonRunningTasks()) {
+      if (st.getRunState() == TaskStatus.State.FAILED ||
+          st.getRunState() == TaskStatus.State.FAILED_UNCLEAN) {
+        ++failed;
+      } else if (st.getRunState() == TaskStatus.State.COMMIT_PENDING) {
+        ++commitPending;
+      }
+    }
+    map.put("running", runningTasks.size());
+    map.put("failed", failed);
+    map.put("commit_pending", commitPending);
+    return map;
+  }
+  // End MXBean implemenation
+
+  @Override
+  public void 
+  updatePrivateDistributedCacheSizes(org.apache.hadoop.mapreduce.JobID jobId,
+                                     long[] sizes
+                                     ) throws IOException {
+    authorizeJVM(jobId);
+    distributedCacheManager.setArchiveSizes(jobId, sizes);
+  }
+
+}

+ 147 - 0
src/mapred/org/apache/hadoop/mapred/TaskTrackerStatus.java

@@ -55,6 +55,7 @@ public class TaskTrackerStatus implements Writable {
   private int maxReduceTasks;
   private int maxReduceTasks;
   private TaskTrackerHealthStatus healthStatus;
   private TaskTrackerHealthStatus healthStatus;
    
    
+  public static final int UNAVAILABLE = -1;
   /**
   /**
    * Class representing a collection of resources on this tasktracker.
    * Class representing a collection of resources on this tasktracker.
    */
    */
@@ -66,6 +67,13 @@ public class TaskTrackerStatus implements Writable {
     private long reduceSlotMemorySizeOnTT;
     private long reduceSlotMemorySizeOnTT;
     private long availableSpace;
     private long availableSpace;
     
     
+    private long availableVirtualMemory = UNAVAILABLE; // in byte
+    private long availablePhysicalMemory = UNAVAILABLE; // in byte
+    private int numProcessors = UNAVAILABLE;
+    private long cumulativeCpuTime = UNAVAILABLE; // in millisecond
+    private long cpuFrequency = UNAVAILABLE; // in kHz
+    private float cpuUsage = UNAVAILABLE; // in %
+
     ResourceStatus() {
     ResourceStatus() {
       totalVirtualMemory = JobConf.DISABLED_MEMORY_LIMIT;
       totalVirtualMemory = JobConf.DISABLED_MEMORY_LIMIT;
       totalPhysicalMemory = JobConf.DISABLED_MEMORY_LIMIT;
       totalPhysicalMemory = JobConf.DISABLED_MEMORY_LIMIT;
@@ -172,21 +180,160 @@ public class TaskTrackerStatus implements Writable {
     long getAvailableSpace() {
     long getAvailableSpace() {
       return availableSpace;
       return availableSpace;
     }
     }
+
+    /**
+     * Set the amount of available virtual memory on the tasktracker.
+     * If the input is not a valid number, it will be set to UNAVAILABLE
+     *
+     * @param vmem amount of available virtual memory on the tasktracker
+     *                    in bytes.
+     */
+    void setAvailableVirtualMemory(long availableMem) {
+      availableVirtualMemory = availableMem > 0 ?
+                               availableMem : UNAVAILABLE;
+    }
+
+    /**
+     * Get the amount of available virtual memory on the tasktracker.
+     * Will return UNAVAILABLE if it cannot be obtained
+     *
+     * @return the amount of available virtual memory on the tasktracker
+     *             in bytes.
+     */
+    long getAvailableVirtualMemory() {
+      return availableVirtualMemory;
+    }
+
+    /**
+     * Set the amount of available physical memory on the tasktracker.
+     * If the input is not a valid number, it will be set to UNAVAILABLE
+     *
+     * @param availableRAM amount of available physical memory on the
+     *                     tasktracker in bytes.
+     */
+    void setAvailablePhysicalMemory(long availableRAM) {
+      availablePhysicalMemory = availableRAM > 0 ?
+                                availableRAM : UNAVAILABLE;
+    }
+
+    /**
+     * Get the amount of available physical memory on the tasktracker.
+     * Will return UNAVAILABLE if it cannot be obtained
+     *
+     * @return amount of available physical memory on the tasktracker in bytes.
+     */
+    long getAvailablePhysicalMemory() {
+      return availablePhysicalMemory;
+    }
+
+    /**
+     * Set the CPU frequency of this TaskTracker
+     * If the input is not a valid number, it will be set to UNAVAILABLE
+     *
+     * @param cpuFrequency CPU frequency in kHz
+     */
+    public void setCpuFrequency(long cpuFrequency) {
+      this.cpuFrequency = cpuFrequency > 0 ?
+                          cpuFrequency : UNAVAILABLE;
+    }
+
+    /**
+     * Get the CPU frequency of this TaskTracker
+     * Will return UNAVAILABLE if it cannot be obtained
+     *
+     * @return CPU frequency in kHz
+     */
+    public long getCpuFrequency() {
+      return cpuFrequency;
+    }
+
+    /**
+     * Set the number of processors on this TaskTracker
+     * If the input is not a valid number, it will be set to UNAVAILABLE
+     *
+     * @param numProcessors number of processors
+     */
+    public void setNumProcessors(int numProcessors) {
+      this.numProcessors = numProcessors > 0 ?
+                           numProcessors : UNAVAILABLE;
+    }
+
+    /**
+     * Get the number of processors on this TaskTracker
+     * Will return UNAVAILABLE if it cannot be obtained
+     *
+     * @return number of processors
+     */
+    public int getNumProcessors() {
+      return numProcessors;
+    }
+
+    /**
+     * Set the cumulative CPU time on this TaskTracker since it is up
+     * It can be set to UNAVAILABLE if it is currently unavailable.
+     *
+     * @param cumulativeCpuTime Used CPU time in millisecond
+     */
+    public void setCumulativeCpuTime(long cumulativeCpuTime) {
+      this.cumulativeCpuTime = cumulativeCpuTime > 0 ?
+                               cumulativeCpuTime : UNAVAILABLE;
+    }
+
+    /**
+     * Get the cumulative CPU time on this TaskTracker since it is up
+     * Will return UNAVAILABLE if it cannot be obtained
+     *
+     * @return used CPU time in milliseconds
+     */
+    public long getCumulativeCpuTime() {
+      return cumulativeCpuTime;
+    }
+    
+    /**
+     * Set the CPU usage on this TaskTracker
+     * 
+     * @param cpuUsage CPU usage in %
+     */
+    public void setCpuUsage(float cpuUsage) {
+      this.cpuUsage = cpuUsage;
+    }
+
+    /**
+     * Get the CPU usage on this TaskTracker
+     * Will return UNAVAILABLE if it cannot be obtained
+     *
+     * @return CPU usage in %
+     */
+    public float getCpuUsage() {
+      return cpuUsage;
+    }
     
     
     public void write(DataOutput out) throws IOException {
     public void write(DataOutput out) throws IOException {
       WritableUtils.writeVLong(out, totalVirtualMemory);
       WritableUtils.writeVLong(out, totalVirtualMemory);
       WritableUtils.writeVLong(out, totalPhysicalMemory);
       WritableUtils.writeVLong(out, totalPhysicalMemory);
+      WritableUtils.writeVLong(out, availableVirtualMemory);
+      WritableUtils.writeVLong(out, availablePhysicalMemory);
       WritableUtils.writeVLong(out, mapSlotMemorySizeOnTT);
       WritableUtils.writeVLong(out, mapSlotMemorySizeOnTT);
       WritableUtils.writeVLong(out, reduceSlotMemorySizeOnTT);
       WritableUtils.writeVLong(out, reduceSlotMemorySizeOnTT);
       WritableUtils.writeVLong(out, availableSpace);
       WritableUtils.writeVLong(out, availableSpace);
+      WritableUtils.writeVLong(out, cumulativeCpuTime);
+      WritableUtils.writeVLong(out, cpuFrequency);
+      WritableUtils.writeVInt(out, numProcessors);
+      out.writeFloat(getCpuUsage());
     }
     }
     
     
     public void readFields(DataInput in) throws IOException {
     public void readFields(DataInput in) throws IOException {
       totalVirtualMemory = WritableUtils.readVLong(in);
       totalVirtualMemory = WritableUtils.readVLong(in);
       totalPhysicalMemory = WritableUtils.readVLong(in);
       totalPhysicalMemory = WritableUtils.readVLong(in);
+      availableVirtualMemory = WritableUtils.readVLong(in);
+      availablePhysicalMemory = WritableUtils.readVLong(in);
       mapSlotMemorySizeOnTT = WritableUtils.readVLong(in);
       mapSlotMemorySizeOnTT = WritableUtils.readVLong(in);
       reduceSlotMemorySizeOnTT = WritableUtils.readVLong(in);
       reduceSlotMemorySizeOnTT = WritableUtils.readVLong(in);
       availableSpace = WritableUtils.readVLong(in);
       availableSpace = WritableUtils.readVLong(in);
+      cumulativeCpuTime = WritableUtils.readVLong(in);
+      cpuFrequency = WritableUtils.readVLong(in);
+      numProcessors = WritableUtils.readVInt(in);
+      setCpuUsage(in.readFloat());
     }
     }
   }
   }
   
   

+ 4 - 0
src/mapred/org/apache/hadoop/mapred/Task_Counter.properties

@@ -17,4 +17,8 @@ REDUCE_OUTPUT_RECORDS.name=    Reduce output records
 REDUCE_SKIPPED_RECORDS.name=   Reduce skipped records
 REDUCE_SKIPPED_RECORDS.name=   Reduce skipped records
 REDUCE_SKIPPED_GROUPS.name=    Reduce skipped groups
 REDUCE_SKIPPED_GROUPS.name=    Reduce skipped groups
 SPILLED_RECORDS.name=          Spilled Records
 SPILLED_RECORDS.name=          Spilled Records
+COMMITTED_HEAP_BYTES.name=     Total committed heap usage (bytes)
+CPU_MILLISECONDS.name=         CPU time spent (ms)
+PHYSICAL_MEMORY_BYTES.name=    Physical memory (bytes) snapshot
+VIRTUAL_MEMORY_BYTES.name=     Virtual memory (bytes) snapshot
 
 

+ 8 - 0
src/mapred/org/apache/hadoop/mapreduce/Counter.java

@@ -103,6 +103,14 @@ public class Counter implements Writable {
     return value;
     return value;
   }
   }
     
     
+  /**
+   * Set this counter by the given value
+   * @param value the value to set
+   */
+  public synchronized void setValue(long value) {
+    this.value = value;
+  }
+
   /**
   /**
    * Increment this counter by the given value
    * Increment this counter by the given value
    * @param incr the value to increase this counter by
    * @param incr the value to increase this counter by

+ 30 - 0
src/test/org/apache/hadoop/mapred/TestCounters.java

@@ -20,6 +20,9 @@ package org.apache.hadoop.mapred;
 import junit.framework.TestCase;
 import junit.framework.TestCase;
 import java.io.IOException;
 import java.io.IOException;
 import java.text.ParseException;
 import java.text.ParseException;
+import java.util.Random;
+
+import org.apache.hadoop.mapred.Counters.Counter;
 
 
 /**
 /**
  * TestCounters checks the sanity and recoverability of {@code Counters}
  * TestCounters checks the sanity and recoverability of {@code Counters}
@@ -90,6 +93,33 @@ public class TestCounters extends TestCase {
     }
     }
   }
   }
   
   
+  /**
+   * Verify counter value works
+   */
+  public void testCounterValue() {
+    final int NUMBER_TESTS = 100;
+    final int NUMBER_INC = 10;
+    final Random rand = new Random();
+    for (int i = 0; i < NUMBER_TESTS; i++) {
+      long initValue = rand.nextInt();
+      long expectedValue = initValue;
+      Counter counter = new Counter("foo", "bar", expectedValue);
+      assertEquals("Counter value is not initialized correctly",
+                   expectedValue, counter.getValue());
+      for (int j = 0; j < NUMBER_INC; j++) {
+        int incValue = rand.nextInt();
+        counter.increment(incValue);
+        expectedValue += incValue;
+        assertEquals("Counter value is not incremented correctly",
+                     expectedValue, counter.getValue());
+      }
+      expectedValue = rand.nextInt();
+      counter.setValue(expectedValue);
+      assertEquals("Counter value is not set correctly",
+                   expectedValue, counter.getValue());
+    }
+  }
+  
   public static void main(String[] args) throws IOException {
   public static void main(String[] args) throws IOException {
     new TestCounters().testCounters();
     new TestCounters().testCounters();
   }
   }

+ 253 - 3
src/test/org/apache/hadoop/mapred/TestJobCounters.java

@@ -23,12 +23,12 @@ import java.io.FileWriter;
 import java.io.Writer;
 import java.io.Writer;
 import java.io.BufferedWriter;
 import java.io.BufferedWriter;
 import java.io.IOException;
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
 import java.util.StringTokenizer;
 import java.util.StringTokenizer;
 
 
 import junit.framework.TestCase;
 import junit.framework.TestCase;
-import junit.extensions.TestSetup;
-import junit.framework.Test;
-import junit.framework.TestSuite;
 
 
 import static org.apache.hadoop.mapred.Task.Counter.SPILLED_RECORDS;
 import static org.apache.hadoop.mapred.Task.Counter.SPILLED_RECORDS;
 import static org.apache.hadoop.mapred.Task.Counter.MAP_INPUT_RECORDS;
 import static org.apache.hadoop.mapred.Task.Counter.MAP_INPUT_RECORDS;
@@ -36,12 +36,17 @@ import static org.apache.hadoop.mapred.Task.Counter.MAP_OUTPUT_RECORDS;
 import static org.apache.hadoop.mapred.Task.Counter.MAP_INPUT_BYTES;
 import static org.apache.hadoop.mapred.Task.Counter.MAP_INPUT_BYTES;
 import static org.apache.hadoop.mapred.Task.Counter.MAP_OUTPUT_BYTES;
 import static org.apache.hadoop.mapred.Task.Counter.MAP_OUTPUT_BYTES;
 import static org.apache.hadoop.mapred.Task.Counter.MAP_OUTPUT_MATERIALIZED_BYTES;
 import static org.apache.hadoop.mapred.Task.Counter.MAP_OUTPUT_MATERIALIZED_BYTES;
+import static org.apache.hadoop.mapred.Task.Counter.COMMITTED_HEAP_BYTES;
 
 
+import org.apache.commons.lang.RandomStringUtils;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.hadoop.mapreduce.Reducer;
 import org.apache.hadoop.mapreduce.Reducer;
@@ -450,4 +455,249 @@ public class TestJobCounters extends TestCase {
       }
       }
     }
     }
   }
   }
+  
+  /** 
+   * Increases the JVM's heap usage to the specified target value.
+   */
+  static class MemoryLoader {
+    private static final int DEFAULT_UNIT_LOAD_SIZE = 10 * 1024 * 1024; // 10mb
+    
+    // the target value to reach
+    private long targetValue;
+    // a list to hold the load objects
+    private List<String> loadObjects = new ArrayList<String>();
+    
+    MemoryLoader(long targetValue) {
+      this.targetValue = targetValue;
+    }
+    
+    /**
+     * Loads the memory to the target value.
+     */
+    void load() {
+      while (Runtime.getRuntime().totalMemory() < targetValue) {
+        System.out.println("Loading memory with " + DEFAULT_UNIT_LOAD_SIZE 
+                           + " characters. Current usage : " 
+                           + Runtime.getRuntime().totalMemory());
+        // load some objects in the memory
+        loadObjects.add(RandomStringUtils.random(DEFAULT_UNIT_LOAD_SIZE));
+
+        // sleep for 100ms
+        try {
+          Thread.sleep(100);
+        } catch (InterruptedException ie) {}
+      }
+    }
+  }
+
+  /**
+   * A mapper that increases the JVM's heap usage to a target value configured 
+   * via {@link MemoryLoaderMapper#TARGET_VALUE} using a {@link MemoryLoader}.
+   */
+  @SuppressWarnings({"deprecation", "unchecked"})
+  static class MemoryLoaderMapper 
+  extends MapReduceBase 
+  implements org.apache.hadoop.mapred.Mapper<WritableComparable, Writable, 
+                    WritableComparable, Writable> {
+    static final String TARGET_VALUE = "map.memory-loader.target-value";
+    
+    private static MemoryLoader loader = null;
+    
+    public void map(WritableComparable key, Writable val, 
+                    OutputCollector<WritableComparable, Writable> output,
+                    Reporter reporter)
+    throws IOException {
+      assertNotNull("Mapper not configured!", loader);
+      
+      // load the memory
+      loader.load();
+      
+      // work as identity mapper
+      output.collect(key, val);
+    }
+
+    public void configure(JobConf conf) {
+      loader = new MemoryLoader(conf.getLong(TARGET_VALUE, -1));
+    }
+  }
+
+  /** 
+   * A reducer that increases the JVM's heap usage to a target value configured 
+   * via {@link MemoryLoaderReducer#TARGET_VALUE} using a {@link MemoryLoader}.
+   */
+  @SuppressWarnings({"deprecation", "unchecked"})
+  static class MemoryLoaderReducer extends MapReduceBase 
+  implements org.apache.hadoop.mapred.Reducer<WritableComparable, Writable, 
+                     WritableComparable, Writable> {
+    static final String TARGET_VALUE = "reduce.memory-loader.target-value";
+    private static MemoryLoader loader = null;
+    
+    public void reduce(WritableComparable key, Iterator<Writable> val, 
+                       OutputCollector<WritableComparable, Writable> output,
+                       Reporter reporter)
+    throws IOException {
+      assertNotNull("Reducer not configured!", loader);
+      
+      // load the memory
+      loader.load();
+      
+      // work as identity reducer
+      output.collect(key, key);
+    }
+
+    public void configure(JobConf conf) {
+      loader = new MemoryLoader(conf.getLong(TARGET_VALUE, -1));
+    }
+  }
+
+  @SuppressWarnings("deprecation")
+  private long getTaskCounterUsage (JobClient client, JobID id, int numReports,
+                                    int taskId, boolean isMap) 
+  throws Exception {
+    TaskReport[] reports = null;
+    if (isMap) {
+      reports = client.getMapTaskReports(id);
+    } else {
+      reports = client.getReduceTaskReports(id);
+    }
+    
+    assertNotNull("No reports found for " + (isMap? "map" : "reduce") + " tasks" 
+                  + "' in job " + id, reports);
+    // make sure that the total number of reports match the expected
+    assertEquals("Mismatch in task id", numReports, reports.length);
+    
+    Counters counters = reports[taskId].getCounters();
+    
+    return counters.getCounter(COMMITTED_HEAP_BYTES);
+  }
+
+  // set up heap options, target value for memory loader and the output 
+  // directory before running the job
+  @SuppressWarnings("deprecation")
+  private static RunningJob runHeapUsageTestJob(JobConf conf, Path testRootDir,
+                              String heapOptions, long targetMapValue,
+                              long targetReduceValue, FileSystem fs, 
+                              JobClient client, Path inDir) 
+  throws IOException {
+    // define a job
+    JobConf jobConf = new JobConf(conf);
+    
+    // configure the jobs
+    jobConf.setNumMapTasks(1);
+    jobConf.setNumReduceTasks(1);
+    jobConf.setMapperClass(MemoryLoaderMapper.class);
+    jobConf.setReducerClass(MemoryLoaderReducer.class);
+    jobConf.setInputFormat(TextInputFormat.class);
+    jobConf.setOutputKeyClass(LongWritable.class);
+    jobConf.setOutputValueClass(Text.class);
+    jobConf.setMaxMapAttempts(1);
+    jobConf.setMaxReduceAttempts(1);
+    jobConf.set(JobConf.MAPRED_TASK_JAVA_OPTS, heapOptions);
+    
+    // set the targets
+    jobConf.setLong(MemoryLoaderMapper.TARGET_VALUE, targetMapValue);
+    jobConf.setLong(MemoryLoaderReducer.TARGET_VALUE, targetReduceValue);
+    
+    // set the input directory for the job
+    FileInputFormat.setInputPaths(jobConf, inDir);
+    
+    // define job output folder
+    Path outDir = new Path(testRootDir, "out");
+    fs.delete(outDir, true);
+    FileOutputFormat.setOutputPath(jobConf, outDir);
+    
+    // run the job
+    RunningJob job = client.submitJob(jobConf);
+    job.waitForCompletion();
+    JobID jobID = job.getID();
+    assertTrue("Job " + jobID + " failed!", job.isSuccessful());
+    
+    return job;
+  }
+
+  /**
+   * Tests {@link TaskCounter}'s {@link TaskCounter.COMMITTED_HEAP_BYTES}. 
+   * The test consists of running a low-memory job which consumes less heap 
+   * memory and then running a high-memory job which consumes more heap memory, 
+   * and then ensuring that COMMITTED_HEAP_BYTES of low-memory job is smaller 
+   * than that of the high-memory job.
+   * @throws IOException
+   */
+  @SuppressWarnings("deprecation")
+  public void testHeapUsageCounter() throws Exception {
+    JobConf conf = new JobConf();
+    // create a local filesystem handle
+    FileSystem fileSystem = FileSystem.getLocal(conf);
+    
+    // define test root directories
+    File rootDir =
+      new File(System.getProperty("test.build.data", "/tmp"));
+    File testRootDir = new File(rootDir, "testHeapUsageCounter");
+    // cleanup the test root directory
+    Path testRootDirPath = new Path(testRootDir.toString());
+    fileSystem.delete(testRootDirPath, true);
+    // set the current working directory
+    fileSystem.setWorkingDirectory(testRootDirPath);
+    
+    fileSystem.deleteOnExit(testRootDirPath);
+    
+    // create a mini cluster using the local file system
+    MiniMRCluster mrCluster = 
+      new MiniMRCluster(1, fileSystem.getUri().toString(), 1);
+    
+    try {
+      conf = mrCluster.createJobConf();
+      JobClient jobClient = new JobClient(conf);
+
+      // define job input
+      File file = new File(testRootDir, "in");
+      Path inDir = new Path(file.toString());
+      // create input data
+      createWordsFile(file);
+
+      // configure and run a low memory job which will run without loading the
+      // jvm's heap
+      RunningJob lowMemJob = 
+        runHeapUsageTestJob(conf, testRootDirPath, "-Xms32m -Xmx1G", 
+                            0, 0, fileSystem, jobClient, inDir);
+      JobID lowMemJobID = lowMemJob.getID();
+      long lowMemJobMapHeapUsage = getTaskCounterUsage(jobClient, lowMemJobID, 
+                                                       1, 0, true);
+      System.out.println("Job1 (low memory job) map task heap usage: " 
+                         + lowMemJobMapHeapUsage);
+      long lowMemJobReduceHeapUsage =
+        getTaskCounterUsage(jobClient, lowMemJobID, 1, 0, false);
+      System.out.println("Job1 (low memory job) reduce task heap usage: " 
+                         + lowMemJobReduceHeapUsage);
+
+      // configure and run a high memory job which will load the jvm's heap
+      RunningJob highMemJob = 
+        runHeapUsageTestJob(conf, testRootDirPath, "-Xms32m -Xmx1G", 
+                            lowMemJobMapHeapUsage + 256*1024*1024, 
+                            lowMemJobReduceHeapUsage + 256*1024*1024,
+                            fileSystem, jobClient, inDir);
+      JobID highMemJobID = highMemJob.getID();
+
+      long highMemJobMapHeapUsage = getTaskCounterUsage(jobClient, highMemJobID,
+                                                        1, 0, true);
+      System.out.println("Job2 (high memory job) map task heap usage: " 
+                         + highMemJobMapHeapUsage);
+      long highMemJobReduceHeapUsage =
+        getTaskCounterUsage(jobClient, highMemJobID, 1, 0, false);
+      System.out.println("Job2 (high memory job) reduce task heap usage: " 
+                         + highMemJobReduceHeapUsage);
+
+      assertTrue("Incorrect map heap usage reported by the map task", 
+                 lowMemJobMapHeapUsage < highMemJobMapHeapUsage);
+
+      assertTrue("Incorrect reduce heap usage reported by the reduce task", 
+                 lowMemJobReduceHeapUsage < highMemJobReduceHeapUsage);
+    } finally {
+      // shutdown the mr cluster
+      mrCluster.shutdown();
+      try {
+        fileSystem.delete(testRootDirPath, true);
+      } catch (IOException ioe) {} 
+    }
+  }
 }
 }

+ 0 - 230
src/test/org/apache/hadoop/mapred/TestTTMemoryReporting.java

@@ -1,230 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.mapred;
-
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.examples.SleepJob;
-import org.apache.hadoop.util.LinuxMemoryCalculatorPlugin;
-import org.apache.hadoop.util.MemoryCalculatorPlugin;
-import org.apache.hadoop.util.ToolRunner;
-import org.apache.hadoop.mapreduce.server.jobtracker.TaskTracker;
-
-import junit.framework.TestCase;
-
-/**
- * This test class tests the functionality related to configuring, reporting
- * and computing memory related parameters in a Map/Reduce cluster.
- * 
- * Each test sets up a {@link MiniMRCluster} with a locally defined 
- * {@link org.apache.hadoop.mapred.TaskScheduler}. This scheduler validates 
- * the memory related configuration is correctly computed and reported from 
- * the tasktracker in 
- * {@link org.apache.hadoop.mapred.TaskScheduler#assignTasks(TaskTrackerStatus)}.
- */
-public class TestTTMemoryReporting extends TestCase {
-
-  static final Log LOG = LogFactory.getLog(TestTTMemoryReporting.class);
-  
-  private MiniMRCluster miniMRCluster;
-
-  /**
-   * Fake scheduler to test the proper reporting of memory values by TT
-   */
-  public static class FakeTaskScheduler extends JobQueueTaskScheduler {
-    
-    private boolean hasPassed = true;
-    private String message;
-    
-    public FakeTaskScheduler() {
-      super();
-    }
-    
-    public boolean hasTestPassed() {
-      return hasPassed;
-    }
-    
-    public String getFailureMessage() {
-      return message;
-    }
-    
-    @Override
-    public List<Task> assignTasks(TaskTracker taskTracker)
-        throws IOException {
-      TaskTrackerStatus status = taskTracker.getStatus();
-      
-      long totalVirtualMemoryOnTT =
-          getConf().getLong("totalVmemOnTT", JobConf.DISABLED_MEMORY_LIMIT);
-      long totalPhysicalMemoryOnTT =
-          getConf().getLong("totalPmemOnTT", JobConf.DISABLED_MEMORY_LIMIT);
-      long mapSlotMemorySize =
-          getConf().getLong("mapSlotMemorySize", JobConf.DISABLED_MEMORY_LIMIT);
-      long reduceSlotMemorySize =
-          getConf()
-              .getLong("reduceSlotMemorySize", JobConf.DISABLED_MEMORY_LIMIT);
-
-      long reportedTotalVirtualMemoryOnTT =
-          status.getResourceStatus().getTotalVirtualMemory();
-      long reportedTotalPhysicalMemoryOnTT =
-          status.getResourceStatus().getTotalPhysicalMemory();
-      long reportedMapSlotMemorySize =
-          status.getResourceStatus().getMapSlotMemorySizeOnTT();
-      long reportedReduceSlotMemorySize =
-          status.getResourceStatus().getReduceSlotMemorySizeOnTT();
-
-      message =
-          "expected memory values : (totalVirtualMemoryOnTT, totalPhysicalMemoryOnTT, "
-              + "mapSlotMemSize, reduceSlotMemorySize) = ("
-              + totalVirtualMemoryOnTT + ", " + totalPhysicalMemoryOnTT + ","
-              + mapSlotMemorySize + "," + reduceSlotMemorySize + ")";
-      message +=
-          "\nreported memory values : (totalVirtualMemoryOnTT, totalPhysicalMemoryOnTT, "
-              + "reportedMapSlotMemorySize, reportedReduceSlotMemorySize) = ("
-              + reportedTotalVirtualMemoryOnTT
-              + ", "
-              + reportedTotalPhysicalMemoryOnTT
-              + ","
-              + reportedMapSlotMemorySize
-              + ","
-              + reportedReduceSlotMemorySize
-              + ")";
-      LOG.info(message);
-      if (totalVirtualMemoryOnTT != reportedTotalVirtualMemoryOnTT
-          || totalPhysicalMemoryOnTT != reportedTotalPhysicalMemoryOnTT
-          || mapSlotMemorySize != reportedMapSlotMemorySize
-          || reduceSlotMemorySize != reportedReduceSlotMemorySize) {
-        hasPassed = false;
-      }
-      return super.assignTasks(taskTracker);
-    }
-  }
-
-  /**
-   * Test that verifies default values are configured and reported correctly.
-   * 
-   * @throws Exception
-   */
-  public void testDefaultMemoryValues()
-      throws Exception {
-    JobConf conf = new JobConf();
-    try {
-      // Memory values are disabled by default.
-      conf.setClass(
-          org.apache.hadoop.mapred.TaskTracker.MAPRED_TASKTRACKER_MEMORY_CALCULATOR_PLUGIN_PROPERTY,
-          DummyMemoryCalculatorPlugin.class, MemoryCalculatorPlugin.class);
-      setUpCluster(conf);
-      runSleepJob(miniMRCluster.createJobConf());
-      verifyTestResults();
-    } finally {
-      tearDownCluster();
-    }
-  }
-
-  /**
-   * Test that verifies that configured values are reported correctly.
-   * 
-   * @throws Exception
-   */
-  public void testConfiguredMemoryValues()
-      throws Exception {
-    JobConf conf = new JobConf();
-    conf.setLong("totalVmemOnTT", 4 * 1024 * 1024 * 1024L);
-    conf.setLong("totalPmemOnTT", 2 * 1024 * 1024 * 1024L);
-    conf.setLong("mapSlotMemorySize", 1 * 512L);
-    conf.setLong("reduceSlotMemorySize", 1 * 1024L);
-
-    conf.setClass(
-        org.apache.hadoop.mapred.TaskTracker.MAPRED_TASKTRACKER_MEMORY_CALCULATOR_PLUGIN_PROPERTY,
-        DummyMemoryCalculatorPlugin.class, MemoryCalculatorPlugin.class);
-    conf.setLong(DummyMemoryCalculatorPlugin.MAXVMEM_TESTING_PROPERTY,
-        4 * 1024 * 1024 * 1024L);
-    conf.setLong(DummyMemoryCalculatorPlugin.MAXPMEM_TESTING_PROPERTY,
-        2 * 1024 * 1024 * 1024L);
-    conf.setLong(JobTracker.MAPRED_CLUSTER_MAP_MEMORY_MB_PROPERTY,
-        512L);
-    conf.setLong(
-        JobTracker.MAPRED_CLUSTER_REDUCE_MEMORY_MB_PROPERTY, 1024L);
-    
-    try {
-      setUpCluster(conf);
-      JobConf jobConf = miniMRCluster.createJobConf();
-      jobConf.setMemoryForMapTask(1 * 1024L);
-      jobConf.setMemoryForReduceTask(2 * 1024L);
-      runSleepJob(jobConf);
-      verifyTestResults();
-    } finally {
-      tearDownCluster();
-    }
-  }
-
-  /**
-   * Test that verifies that total memory values are calculated and reported
-   * correctly.
-   * 
-   * @throws Exception
-   */
-  public void testMemoryValuesOnLinux()
-      throws Exception {
-    if (!System.getProperty("os.name").startsWith("Linux")) {
-      return;
-    }
-
-    JobConf conf = new JobConf();
-    LinuxMemoryCalculatorPlugin plugin = new LinuxMemoryCalculatorPlugin();
-    conf.setLong("totalVmemOnTT", plugin.getVirtualMemorySize());
-    conf.setLong("totalPmemOnTT", plugin.getPhysicalMemorySize());
-
-    try {
-      setUpCluster(conf);
-      runSleepJob(miniMRCluster.createJobConf());
-      verifyTestResults();
-    } finally {
-      tearDownCluster();
-    }
-  }
-
-  private void setUpCluster(JobConf conf)
-                                throws Exception {
-    conf.setClass("mapred.jobtracker.taskScheduler",
-        TestTTMemoryReporting.FakeTaskScheduler.class, TaskScheduler.class);
-    conf.set("mapred.job.tracker.handler.count", "1");
-    miniMRCluster = new MiniMRCluster(1, "file:///", 3, null, null, conf);
-  }
-  
-  private void runSleepJob(JobConf conf) throws Exception {
-    String[] args = { "-m", "1", "-r", "1",
-                      "-mt", "10", "-rt", "10" };
-    ToolRunner.run(conf, new SleepJob(), args);
-  }
-
-  private void verifyTestResults() {
-    FakeTaskScheduler scheduler = 
-      (FakeTaskScheduler)miniMRCluster.getJobTrackerRunner().
-                              getJobTracker().getTaskScheduler();
-    assertTrue(scheduler.getFailureMessage(), scheduler.hasTestPassed());
-  }
-  
-  private void tearDownCluster() {
-    if (miniMRCluster != null) {
-      miniMRCluster.shutdown();
-    }
-  }
-}

+ 366 - 0
src/test/org/apache/hadoop/mapred/TestTTResourceReporting.java

@@ -0,0 +1,366 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.mapred;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.examples.SleepJob;
+import org.apache.hadoop.mapred.Counters;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.JobTracker;
+import org.apache.hadoop.mapred.Task;
+import org.apache.hadoop.mapred.TaskStatus;
+import org.apache.hadoop.mapreduce.server.jobtracker.TaskTracker;
+import org.apache.hadoop.mapred.TaskTrackerStatus;
+import org.apache.hadoop.mapred.Task.Counter;
+import org.apache.hadoop.util.DummyResourceCalculatorPlugin;
+import org.apache.hadoop.util.LinuxResourceCalculatorPlugin;
+import org.apache.hadoop.util.ResourceCalculatorPlugin;
+import org.apache.hadoop.util.ToolRunner;
+
+import junit.framework.TestCase;
+import org.junit.Test;
+import org.junit.After;
+
+/**
+ * This test class tests the functionality related to configuring, reporting
+ * and computing memory related parameters in a Map/Reduce cluster.
+ * 
+ * Each test sets up a {@link MiniMRCluster} with a locally defined 
+ * {@link org.apache.hadoop.mapred.TaskScheduler}. This scheduler validates 
+ * the memory related configuration is correctly computed and reported from 
+ * the tasktracker in 
+ * {@link org.apache.hadoop.mapred.TaskScheduler#assignTasks(TaskTrackerStatus)}.
+ */
+public class TestTTResourceReporting extends TestCase {
+
+  static final Log LOG = LogFactory.getLog(TestTTResourceReporting.class);
+  
+  private MiniMRCluster miniMRCluster;
+
+  /**
+   * Fake scheduler to test the proper reporting of memory values by TT
+   */
+  public static class FakeTaskScheduler extends JobQueueTaskScheduler {
+    
+    private boolean hasPassed = true;
+    private boolean hasDynamicValuePassed = true;
+    private String message;
+    
+    public FakeTaskScheduler() {
+      super();
+    }
+    
+    public boolean hasTestPassed() {
+      return hasPassed;
+    }
+
+    public boolean hasDynamicTestPassed() {
+      return hasDynamicValuePassed;
+    }
+    
+    public String getFailureMessage() {
+      return message;
+    }
+    
+    @Override
+    public List<Task> assignTasks(TaskTracker taskTracker)
+        throws IOException {
+      TaskTrackerStatus status = taskTracker.getStatus();
+      long totalVirtualMemoryOnTT =
+          getConf().getLong("totalVmemOnTT", -1);
+      long totalPhysicalMemoryOnTT =
+          getConf().getLong("totalPmemOnTT", -1);
+      long mapSlotMemorySize =
+          getConf().getLong("mapSlotMemorySize", -1);
+      long reduceSlotMemorySize =
+          getConf()
+              .getLong("reduceSlotMemorySize", -1);
+      long availableVirtualMemoryOnTT =
+          getConf().getLong("availableVmemOnTT", -1);
+      long availablePhysicalMemoryOnTT =
+          getConf().getLong("availablePmemOnTT", -1);
+      long cumulativeCpuTime =
+          getConf().getLong("cumulativeCpuTime", -1);
+      long cpuFrequency =
+          getConf().getLong("cpuFrequency", -1);
+      int numProcessors =
+          getConf().getInt("numProcessors", -1);
+      float cpuUsage =
+          getConf().getFloat("cpuUsage", -1);
+
+      long reportedTotalVirtualMemoryOnTT =
+          status.getResourceStatus().getTotalVirtualMemory();
+      long reportedTotalPhysicalMemoryOnTT =
+          status.getResourceStatus().getTotalPhysicalMemory();
+      long reportedMapSlotMemorySize =
+          status.getResourceStatus().getMapSlotMemorySizeOnTT();
+      long reportedReduceSlotMemorySize =
+          status.getResourceStatus().getReduceSlotMemorySizeOnTT();
+      long reportedAvailableVirtualMemoryOnTT =
+          status.getResourceStatus().getAvailableVirtualMemory();
+      long reportedAvailablePhysicalMemoryOnTT =
+          status.getResourceStatus().getAvailablePhysicalMemory();
+      long reportedCumulativeCpuTime =
+          status.getResourceStatus().getCumulativeCpuTime();
+      long reportedCpuFrequency = status.getResourceStatus().getCpuFrequency();
+      int reportedNumProcessors = status.getResourceStatus().getNumProcessors();
+      float reportedCpuUsage = status.getResourceStatus().getCpuUsage();
+
+      message =
+          "expected memory values : "
+              + "(totalVirtualMemoryOnTT, totalPhysicalMemoryOnTT, "
+              + "availableVirtualMemoryOnTT, availablePhysicalMemoryOnTT, "
+              + "mapSlotMemSize, reduceSlotMemorySize, cumulativeCpuTime, "
+              + "cpuFrequency, numProcessors, cpuUsage) = ("
+              + totalVirtualMemoryOnTT + ", "
+              + totalPhysicalMemoryOnTT + ","
+              + availableVirtualMemoryOnTT + ", "
+              + availablePhysicalMemoryOnTT + ","
+              + mapSlotMemorySize + ","
+              + reduceSlotMemorySize + ","
+              + cumulativeCpuTime + ","
+              + cpuFrequency + ","
+              + numProcessors + ","
+              + cpuUsage
+              +")";
+      message +=
+          "\nreported memory values : "
+              + "(totalVirtualMemoryOnTT, totalPhysicalMemoryOnTT, "
+              + "availableVirtualMemoryOnTT, availablePhysicalMemoryOnTT, "
+              + "reportedMapSlotMemorySize, reportedReduceSlotMemorySize, "
+              + "reportedCumulativeCpuTime, reportedCpuFrequency, "
+              + "reportedNumProcessors, cpuUsage) = ("
+              + reportedTotalVirtualMemoryOnTT + ", "
+              + reportedTotalPhysicalMemoryOnTT + ","
+              + reportedAvailableVirtualMemoryOnTT + ", "
+              + reportedAvailablePhysicalMemoryOnTT + ","
+              + reportedMapSlotMemorySize + ","
+              + reportedReduceSlotMemorySize + ","
+              + reportedCumulativeCpuTime + ","
+              + reportedCpuFrequency + ","
+              + reportedNumProcessors + ","
+              + reportedCpuUsage
+               + ")";
+      LOG.info(message);
+      hasDynamicValuePassed = true;
+      // Check task resource status in task reports
+      for (TaskStatus taskStatus : status.getTaskReports()) {
+        Counters counters = taskStatus.getCounters();
+        // This should be zero because the initial CPU time is subtracted.
+        long procCumulativeCpuTime = 0;
+        long procVirtualMemorySize =
+          getConf().getLong("procVirtualMemorySize", -1);
+        long procPhysicalMemorySize =
+          getConf().getLong("procPhysicalMemorySize", -1);
+        long reportedProcCumulativeCpuTime =
+          counters.findCounter(Task.Counter.CPU_MILLISECONDS).getValue();
+        long reportedProcVirtualMemorySize =
+          counters.findCounter(Task.Counter.VIRTUAL_MEMORY_BYTES).getValue();
+        long reportedProcPhysicalMemorySize =
+          counters.findCounter(Task.Counter.PHYSICAL_MEMORY_BYTES).getValue();
+        String procMessage =
+          "expected values : "
+              + "(procCumulativeCpuTime, procVirtualMemorySize,"
+              + " procPhysicalMemorySize) = ("
+              + procCumulativeCpuTime + ", "
+              + procVirtualMemorySize + ", "
+              + procPhysicalMemorySize + ")";
+        procMessage +=
+          "\nreported values : "
+              + "(procCumulativeCpuTime, procVirtualMemorySize,"
+              + " procPhysicalMemorySize) = ("
+              + reportedProcCumulativeCpuTime + ", "
+              + reportedProcVirtualMemorySize + ", "
+              + reportedProcPhysicalMemorySize + ")";
+        LOG.info(procMessage);
+        message += "\n" + procMessage;
+        if (procCumulativeCpuTime != reportedProcCumulativeCpuTime ||
+            procVirtualMemorySize != reportedProcVirtualMemorySize ||
+            procPhysicalMemorySize != reportedProcPhysicalMemorySize) {
+          hasDynamicValuePassed = false;
+        }
+      }
+      hasPassed = true;
+      if (totalVirtualMemoryOnTT != reportedTotalVirtualMemoryOnTT
+          || totalPhysicalMemoryOnTT != reportedTotalPhysicalMemoryOnTT
+          || mapSlotMemorySize != reportedMapSlotMemorySize
+          || reduceSlotMemorySize != reportedReduceSlotMemorySize
+          || numProcessors != reportedNumProcessors) {
+        hasPassed = false;
+      }
+      // These values changes every moment on the node so it can only be
+      // tested by DummyMemoryCalculatorPlugin. Need to check them separately
+      if (availableVirtualMemoryOnTT != reportedAvailableVirtualMemoryOnTT
+          || availablePhysicalMemoryOnTT != reportedAvailablePhysicalMemoryOnTT
+          || cumulativeCpuTime != reportedCumulativeCpuTime
+          || cpuFrequency != reportedCpuFrequency
+          || cpuUsage != reportedCpuUsage) {
+        hasDynamicValuePassed = false;
+      }
+      return super.assignTasks(taskTracker);
+    }
+  }
+
+  /**
+   * Test that verifies default values are configured and reported correctly.
+   * 
+   * @throws Exception
+   */
+  public void testDefaultResourceValues()
+      throws Exception {
+    JobConf conf = new JobConf();
+    try {
+      // Memory values are disabled by default.
+      conf.setClass(
+          org.apache.hadoop.mapred.TaskTracker.TT_RESOURCE_CALCULATOR_PLUGIN,       
+          DummyResourceCalculatorPlugin.class, ResourceCalculatorPlugin.class);
+      setUpCluster(conf);
+      JobConf jobConf = miniMRCluster.createJobConf();
+      jobConf.setClass(
+          org.apache.hadoop.mapred.TaskTracker.TT_RESOURCE_CALCULATOR_PLUGIN,
+          DummyResourceCalculatorPlugin.class, ResourceCalculatorPlugin.class);
+      runSleepJob(jobConf);
+      verifyTestResults();
+    } finally {
+      tearDownCluster();
+    }
+  }
+
+  /**
+   * Test that verifies that configured values are reported correctly.
+   * 
+   * @throws Exception
+   */
+  public void testConfiguredResourceValues()
+      throws Exception {
+    JobConf conf = new JobConf();
+    conf.setLong("totalVmemOnTT", 4 * 1024 * 1024 * 1024L);
+    conf.setLong("totalPmemOnTT", 2 * 1024 * 1024 * 1024L);
+    conf.setLong("mapSlotMemorySize", 1 * 512L);
+    conf.setLong("reduceSlotMemorySize", 1 * 1024L);
+    conf.setLong("availableVmemOnTT", 4 * 1024 * 1024 * 1024L);
+    conf.setLong("availablePmemOnTT", 2 * 1024 * 1024 * 1024L);
+    conf.setLong("cumulativeCpuTime", 10000L);
+    conf.setLong("cpuFrequency", 2000000L);
+    conf.setInt("numProcessors", 8);
+    conf.setFloat("cpuUsage", 15.5F);
+    conf.setLong("procCumulativeCpuTime", 1000L);
+    conf.setLong("procVirtualMemorySize", 2 * 1024 * 1024 * 1024L);
+    conf.setLong("procPhysicalMemorySize", 1024 * 1024 * 1024L);
+
+    conf.setClass(
+        org.apache.hadoop.mapred.TaskTracker.TT_RESOURCE_CALCULATOR_PLUGIN,       
+        DummyResourceCalculatorPlugin.class, ResourceCalculatorPlugin.class);
+    conf.setLong(DummyResourceCalculatorPlugin.MAXVMEM_TESTING_PROPERTY,
+        4 * 1024 * 1024 * 1024L);
+    conf.setLong(DummyResourceCalculatorPlugin.MAXPMEM_TESTING_PROPERTY,
+        2 * 1024 * 1024 * 1024L);
+    conf.setLong(JobTracker.MAPRED_CLUSTER_MAP_MEMORY_MB_PROPERTY, 512L);
+    conf.setLong(JobTracker.MAPRED_CLUSTER_REDUCE_MEMORY_MB_PROPERTY, 1024L);
+    conf.setLong(DummyResourceCalculatorPlugin.CUMULATIVE_CPU_TIME, 10000L);
+    conf.setLong(DummyResourceCalculatorPlugin.CPU_FREQUENCY, 2000000L);
+    conf.setInt(DummyResourceCalculatorPlugin.NUM_PROCESSORS, 8);
+    conf.setFloat(DummyResourceCalculatorPlugin.CPU_USAGE, 15.5F);
+    try {
+      setUpCluster(conf);
+      JobConf jobConf = miniMRCluster.createJobConf();
+      jobConf.setMemoryForMapTask(1 * 1024L);
+      jobConf.setMemoryForReduceTask(2 * 1024L);
+      jobConf.setClass(
+        org.apache.hadoop.mapred.TaskTracker.TT_RESOURCE_CALCULATOR_PLUGIN,
+        DummyResourceCalculatorPlugin.class, ResourceCalculatorPlugin.class);
+      jobConf.setLong(DummyResourceCalculatorPlugin.PROC_CUMULATIVE_CPU_TIME, 1000L);
+      jobConf.setLong(DummyResourceCalculatorPlugin.PROC_VMEM_TESTING_PROPERTY,
+                      2 * 1024 * 1024 * 1024L);
+      jobConf.setLong(DummyResourceCalculatorPlugin.PROC_PMEM_TESTING_PROPERTY,
+                      1024 * 1024 * 1024L);
+      runSleepJob(jobConf);
+      verifyTestResults();
+    } finally {
+      tearDownCluster();
+    }
+  }
+
+  /**
+   * Test that verifies that total memory values are calculated and reported
+   * correctly.
+   * 
+   * @throws Exception
+   */
+  public void testResourceValuesOnLinux()
+      throws Exception {
+    if (!System.getProperty("os.name").startsWith("Linux")) {
+      return;
+    }
+
+    JobConf conf = new JobConf();
+    LinuxResourceCalculatorPlugin plugin = new LinuxResourceCalculatorPlugin();
+    // In this case, we only check these four fields because they are static
+    conf.setLong("totalVmemOnTT", plugin.getVirtualMemorySize());
+    conf.setLong("totalPmemOnTT", plugin.getPhysicalMemorySize());
+    conf.setLong("numProcessors", plugin.getNumProcessors());
+
+    try {
+      setUpCluster(conf);
+      runSleepJob(miniMRCluster.createJobConf());
+      verifyTestResults(true);
+    } finally {
+      tearDownCluster();
+    }
+  }
+
+  private void setUpCluster(JobConf conf)
+                                throws Exception {
+    conf.setClass("mapred.jobtracker.taskScheduler",
+        TestTTResourceReporting.FakeTaskScheduler.class, TaskScheduler.class);
+    conf.set("mapred.job.tracker.handler.count", "1");
+    miniMRCluster = new MiniMRCluster(1, "file:///", 3, null, null, conf);
+  }
+  
+  private void runSleepJob(JobConf conf) throws Exception {
+    String[] args = { "-m", "1", "-r", "1",
+                      "-mt", "10", "-rt", "10" };
+    ToolRunner.run(conf, new SleepJob(), args);
+  }
+
+  private void verifyTestResults() {
+    verifyTestResults(false);
+  }
+
+  private void verifyTestResults(boolean excludeDynamic) {
+    FakeTaskScheduler scheduler = 
+      (FakeTaskScheduler)miniMRCluster.getJobTrackerRunner().
+                              getJobTracker().getTaskScheduler();
+    assertTrue(scheduler.getFailureMessage(), scheduler.hasTestPassed());
+    if (!excludeDynamic) {
+      assertTrue(scheduler.getFailureMessage(),
+                 scheduler.hasDynamicTestPassed());
+    }
+  }
+  
+  //TODO make it after
+  private void tearDownCluster() {
+    if (miniMRCluster != null) {
+      miniMRCluster.shutdown();
+    }
+  }
+}

+ 114 - 0
src/test/org/apache/hadoop/util/DummyResourceCalculatorPlugin.java

@@ -0,0 +1,114 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.util;
+
+/**
+ * Plugin class to test resource information reported by TT. Use
+ * configuration items {@link #MAXVMEM_TESTING_PROPERTY} and
+ * {@link #MAXPMEM_TESTING_PROPERTY} to tell TT the total vmem and the total
+ * pmem. Use configuration items {@link #NUM_PROCESSORS},
+ * {@link #CPU_FREQUENCY}, {@link #CUMULATIVE_CPU_TIME} and {@link #CPU_USAGE}
+ * to tell TT the CPU information.
+ */
+public class DummyResourceCalculatorPlugin extends ResourceCalculatorPlugin {
+
+  /** max vmem on the TT */
+  public static final String MAXVMEM_TESTING_PROPERTY =
+      "mapred.tasktracker.maxvmem.testing";
+  /** max pmem on the TT */
+  public static final String MAXPMEM_TESTING_PROPERTY =
+      "mapred.tasktracker.maxpmem.testing";
+  /** number of processors for testing */
+  public static final String NUM_PROCESSORS =
+      "mapred.tasktracker.numprocessors.testing";
+  /** CPU frequency for testing */
+  public static final String CPU_FREQUENCY =
+      "mapred.tasktracker.cpufrequency.testing";
+  /** cumulative CPU usage time for testing */
+  public static final String CUMULATIVE_CPU_TIME =
+      "mapred.tasktracker.cumulativecputime.testing";
+  /** CPU usage percentage for testing */
+  public static final String CPU_USAGE =
+      "mapred.tasktracker.cpuusage.testing";
+  /** process cumulative CPU usage time for testing */
+  public static final String PROC_CUMULATIVE_CPU_TIME =
+      "mapred.tasktracker.proccumulativecputime.testing";
+  /** process pmem for testing*/
+  public static final String PROC_PMEM_TESTING_PROPERTY =
+      "mapred.tasktracker.procpmem.testing";
+  /** process vmem for testing*/
+  public static final String PROC_VMEM_TESTING_PROPERTY =
+      "mapred.tasktracker.procvmem.testing";
+
+  /** {@inheritDoc} */
+  @Override
+  public long getVirtualMemorySize() {
+    return getConf().getLong(MAXVMEM_TESTING_PROPERTY, -1);
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public long getPhysicalMemorySize() {
+    return getConf().getLong(MAXPMEM_TESTING_PROPERTY, -1);
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public long getAvailableVirtualMemorySize() {
+    return getConf().getLong(MAXVMEM_TESTING_PROPERTY, -1);
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public long getAvailablePhysicalMemorySize() {
+    return getConf().getLong(MAXPMEM_TESTING_PROPERTY, -1);
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public int getNumProcessors() {
+    return getConf().getInt(NUM_PROCESSORS, -1);
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public long getCpuFrequency() {
+    return getConf().getLong(CPU_FREQUENCY, -1);
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public long getCumulativeCpuTime() {
+    return getConf().getLong(CUMULATIVE_CPU_TIME, -1);
+  }
+
+  /** {@inheritDoc} */
+  @Override
+  public float getCpuUsage() {
+    return getConf().getFloat(CPU_USAGE, -1);
+  }
+
+  @Override
+  public ProcResourceValues getProcResourceValues() {
+    long cpuTime = getConf().getLong(PROC_CUMULATIVE_CPU_TIME, -1);
+    long pMem = getConf().getLong(PROC_PMEM_TESTING_PROPERTY, -1);
+    long vMem = getConf().getLong(PROC_VMEM_TESTING_PROPERTY, -1);
+    return new ProcResourceValues(cpuTime, pMem, vMem);
+  }
+}

+ 234 - 0
src/test/org/apache/hadoop/util/TestLinuxResourceCalculatorPlugin.java

@@ -0,0 +1,234 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.util;
+
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.util.Random;
+
+import junit.framework.TestCase;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.TaskTrackerStatus;
+import org.junit.Test;
+
+/**
+ * A JUnit test to test {@link LinuxResourceCalculatorPlugin}
+ * Create the fake /proc/ information and verify the parsing and calculation
+ */
+public class TestLinuxResourceCalculatorPlugin extends TestCase {
+  /**
+   * LinuxResourceCalculatorPlugin with a fake timer
+   */
+  static class FakeLinuxResourceCalculatorPlugin extends
+      LinuxResourceCalculatorPlugin {
+    
+	  long currentTime = 0;
+	  public FakeLinuxResourceCalculatorPlugin(String procfsMemFile,
+			                                       String procfsCpuFile,
+			                                       String procfsStatFile,
+			                                       long jiffyLengthInMillis) {
+	    super(procfsMemFile, procfsCpuFile, procfsStatFile, jiffyLengthInMillis);
+	  }
+	  @Override
+	  long getCurrentTime() {
+	    return currentTime;
+	  }
+	  public void advanceTime(long adv) {
+	    currentTime += adv * jiffyLengthInMillis;
+	  }
+  }
+  private static final FakeLinuxResourceCalculatorPlugin plugin;
+  private static String TEST_ROOT_DIR = new Path(System.getProperty(
+         "test.build.data", "/tmp")).toString().replace(' ', '+');
+  private static final String FAKE_MEMFILE;
+  private static final String FAKE_CPUFILE;
+  private static final String FAKE_STATFILE;
+  private static final long FAKE_JIFFY_LENGTH = 10L;
+  static {
+    int randomNum = (new Random()).nextInt(1000000000);
+    FAKE_MEMFILE = TEST_ROOT_DIR + File.separator + "MEMINFO_" + randomNum;
+    FAKE_CPUFILE = TEST_ROOT_DIR + File.separator + "CPUINFO_" + randomNum;
+    FAKE_STATFILE = TEST_ROOT_DIR + File.separator + "STATINFO_" + randomNum;
+    plugin = new FakeLinuxResourceCalculatorPlugin(FAKE_MEMFILE, FAKE_CPUFILE,
+                                                   FAKE_STATFILE,
+                                                   FAKE_JIFFY_LENGTH);
+  }
+  static final String MEMINFO_FORMAT = 
+	  "MemTotal:      %d kB\n" +
+	  "MemFree:         %d kB\n" +
+	  "Buffers:        138244 kB\n" +
+	  "Cached:         947780 kB\n" +
+	  "SwapCached:     142880 kB\n" +
+	  "Active:        3229888 kB\n" +
+	  "Inactive:       %d kB\n" +
+	  "SwapTotal:     %d kB\n" +
+	  "SwapFree:      %d kB\n" +
+	  "Dirty:          122012 kB\n" +
+	  "Writeback:           0 kB\n" +
+	  "AnonPages:     2710792 kB\n" +
+	  "Mapped:          24740 kB\n" +
+	  "Slab:           132528 kB\n" +
+	  "SReclaimable:   105096 kB\n" +
+	  "SUnreclaim:      27432 kB\n" +
+	  "PageTables:      11448 kB\n" +
+	  "NFS_Unstable:        0 kB\n" +
+	  "Bounce:              0 kB\n" +
+	  "CommitLimit:   4125904 kB\n" +
+	  "Committed_AS:  4143556 kB\n" +
+	  "VmallocTotal: 34359738367 kB\n" +
+	  "VmallocUsed:      1632 kB\n" +
+	  "VmallocChunk: 34359736375 kB\n" +
+	  "HugePages_Total:     0\n" +
+	  "HugePages_Free:      0\n" +
+	  "HugePages_Rsvd:      0\n" +
+	  "Hugepagesize:     2048 kB";
+  
+  static final String CPUINFO_FORMAT =
+    "processor : %s\n" +
+    "vendor_id : AuthenticAMD\n" +
+    "cpu family  : 15\n" +
+    "model   : 33\n" +
+    "model name  : Dual Core AMD Opteron(tm) Processor 280\n" +
+    "stepping  : 2\n" +
+    "cpu MHz   : %f\n" +
+    "cache size  : 1024 KB\n" +
+    "physical id : 0\n" +
+    "siblings  : 2\n" +
+    "core id   : 0\n" +
+    "cpu cores : 2\n" +
+    "fpu   : yes\n" +
+    "fpu_exception : yes\n" +
+    "cpuid level : 1\n" +
+    "wp    : yes\n" +
+    "flags   : fpu vme de pse tsc msr pae mce cx8 apic sep mtrr pge mca cmov " +
+    "pat pse36 clflush mmx fxsr sse sse2 ht syscall nx mmxext fxsr_opt lm " +
+    "3dnowext 3dnow pni lahf_lm cmp_legacy\n" +
+    "bogomips  : 4792.41\n" +
+    "TLB size  : 1024 4K pages\n" +
+    "clflush size  : 64\n" +
+    "cache_alignment : 64\n" +
+    "address sizes : 40 bits physical, 48 bits virtual\n" +
+    "power management: ts fid vid ttp";
+  
+  static final String STAT_FILE_FORMAT = 
+    "cpu  %d %d %d 1646495089 831319 48713 164346 0\n" +
+    "cpu0 15096055 30805 3823005 411456015 206027 13 14269 0\n" +
+    "cpu1 14760561 89890 6432036 408707910 456857 48074 130857 0\n" +
+    "cpu2 12761169 20842 3758639 413976772 98028 411 10288 0\n" +
+    "cpu3 12355207 47322 5789691 412354390 70406 213 8931 0\n" +
+    "intr 114648668 20010764 2 0 945665 2 0 0 0 0 0 0 0 4 0 0 0 0 0 0\n" +
+    "ctxt 242017731764\n" +
+    "btime 1257808753\n" +
+    "processes 26414943\n" +
+    "procs_running 1\n" +
+    "procs_blocked 0\n";
+  
+  /**
+   * Test parsing /proc/stat and /proc/cpuinfo
+   * @throws IOException
+   */
+  public void testParsingProcStatAndCpuFile() throws IOException {
+    // Write fake /proc/cpuinfo file.
+    long numProcessors = 8;
+    long cpuFrequencyKHz = 2392781;
+    String fileContent = "";
+    for (int i = 0; i < numProcessors; i++) {
+      fileContent += String.format(CPUINFO_FORMAT, i, cpuFrequencyKHz / 1000D) +
+                     "\n";
+    }
+    File tempFile = new File(FAKE_CPUFILE);
+    tempFile.deleteOnExit();
+    FileWriter fWriter = new FileWriter(FAKE_CPUFILE);
+    fWriter.write(fileContent);
+    fWriter.close();
+    assertEquals(plugin.getNumProcessors(), numProcessors);
+    assertEquals(plugin.getCpuFrequency(), cpuFrequencyKHz);
+    
+    // Write fake /proc/stat file.
+    long uTime = 54972994;
+    long nTime = 188860;
+    long sTime = 19803373;
+    tempFile = new File(FAKE_STATFILE);
+    tempFile.deleteOnExit();
+    updateStatFile(uTime, nTime, sTime);
+    assertEquals(plugin.getCumulativeCpuTime(),
+                 FAKE_JIFFY_LENGTH * (uTime + nTime + sTime));
+    assertEquals(plugin.getCpuUsage(), (float)(TaskTrackerStatus.UNAVAILABLE));
+    
+    // Advance the time and sample again to test the CPU usage calculation
+    uTime += 100L;
+    plugin.advanceTime(200L);
+    updateStatFile(uTime, nTime, sTime);
+    assertEquals(plugin.getCumulativeCpuTime(),
+                 FAKE_JIFFY_LENGTH * (uTime + nTime + sTime));
+    assertEquals(plugin.getCpuUsage(), 6.25F);
+    
+    // Advance the time and sample again. This time, we call getCpuUsage() only.
+    uTime += 600L;
+    plugin.advanceTime(300L);
+    updateStatFile(uTime, nTime, sTime);
+    assertEquals(plugin.getCpuUsage(), 25F);
+    
+    // Advance very short period of time (one jiffy length).
+    // In this case, CPU usage should not be updated.
+    uTime += 1L;
+    plugin.advanceTime(1L);
+    updateStatFile(uTime, nTime, sTime);
+    assertEquals(plugin.getCumulativeCpuTime(),
+                 FAKE_JIFFY_LENGTH * (uTime + nTime + sTime));
+    assertEquals(plugin.getCpuUsage(), 25F); // CPU usage is not updated.
+  }
+  
+  /**
+   * Write information to fake /proc/stat file
+   */
+  private void updateStatFile(long uTime, long nTime, long sTime)
+    throws IOException {
+    FileWriter fWriter = new FileWriter(FAKE_STATFILE);
+    fWriter.write(String.format(STAT_FILE_FORMAT, uTime, nTime, sTime));
+    fWriter.close();
+  }
+  
+  /**
+   * Test parsing /proc/meminfo
+   * @throws IOException
+   */
+  public void testParsingProcMemFile() throws IOException {
+    long memTotal = 4058864L;
+    long memFree = 99632L;
+    long inactive = 567732L;
+    long swapTotal = 2096472L;
+    long swapFree = 1818480L;
+    File tempFile = new File(FAKE_MEMFILE);
+    tempFile.deleteOnExit();
+    FileWriter fWriter = new FileWriter(FAKE_MEMFILE);
+    fWriter.write(String.format(MEMINFO_FORMAT,
+      memTotal, memFree, inactive, swapTotal, swapFree));
+    
+    fWriter.close();
+    assertEquals(plugin.getAvailablePhysicalMemorySize(),
+                 1024L * (memFree + inactive));
+    assertEquals(plugin.getAvailableVirtualMemorySize(),
+                 1024L * (memFree + inactive + swapFree));
+    assertEquals(plugin.getPhysicalMemorySize(), 1024L * memTotal);
+    assertEquals(plugin.getVirtualMemorySize(), 1024L * (memTotal + swapTotal));
+  }
+}

+ 114 - 60
src/test/org/apache/hadoop/util/TestProcfsBasedProcessTree.java

@@ -22,6 +22,7 @@ import java.io.BufferedWriter;
 import java.io.File;
 import java.io.File;
 import java.io.FileWriter;
 import java.io.FileWriter;
 import java.io.IOException;
 import java.io.IOException;
+import java.util.Arrays;
 import java.util.Random;
 import java.util.Random;
 import java.util.Vector;
 import java.util.Vector;
 import java.util.regex.Matcher;
 import java.util.regex.Matcher;
@@ -182,12 +183,12 @@ public class TestProcfsBasedProcessTree extends TestCase {
 
 
     LOG.info("Process-tree dump follows: \n" + processTreeDump);
     LOG.info("Process-tree dump follows: \n" + processTreeDump);
     assertTrue("Process-tree dump doesn't start with a proper header",
     assertTrue("Process-tree dump doesn't start with a proper header",
-        processTreeDump.startsWith("\t|- PID PPID PGRPID SESSID CMD_NAME "
-            + "VMEM_USAGE(BYTES) FULL_CMD_LINE\n"));
+        processTreeDump.startsWith("\t|- PID PPID PGRPID SESSID CMD_NAME " +
+        "USER_MODE_TIME(MILLIS) SYSTEM_TIME(MILLIS) VMEM_USAGE(BYTES) " +
+        "RSSMEM_USAGE(PAGES) FULL_CMD_LINE\n"));
     for (int i = N; i >= 0; i--) {
     for (int i = N; i >= 0; i--) {
-      String cmdLineDump =
-          "\\|- [0-9]+ [0-9]+ [0-9]+ [0-9]+ \\(sh\\) [0-9]+ sh " + shellScript
-              + " " + i;
+      String cmdLineDump = "\\|- [0-9]+ [0-9]+ [0-9]+ [0-9]+ \\(sh\\)" +
+          " [0-9]+ [0-9]+ [0-9]+ [0-9]+ sh " + shellScript + " " + i;
       Pattern pat = Pattern.compile(cmdLineDump);
       Pattern pat = Pattern.compile(cmdLineDump);
       Matcher mat = pat.matcher(processTreeDump);
       Matcher mat = pat.matcher(processTreeDump);
       assertTrue("Process-tree dump doesn't contain the cmdLineDump of " + i
       assertTrue("Process-tree dump doesn't contain the cmdLineDump of " + i
@@ -221,7 +222,10 @@ public class TestProcfsBasedProcessTree extends TestCase {
     String ppid;
     String ppid;
     String pgrpId;
     String pgrpId;
     String session;
     String session;
-    String vmem;
+    String vmem = "0";
+    String rssmemPage = "0";
+    String utime = "0";
+    String stime = "0";
     
     
     public ProcessStatInfo(String[] statEntries) {
     public ProcessStatInfo(String[] statEntries) {
       pid = statEntries[0];
       pid = statEntries[0];
@@ -230,27 +234,35 @@ public class TestProcfsBasedProcessTree extends TestCase {
       pgrpId = statEntries[3];
       pgrpId = statEntries[3];
       session = statEntries[4];
       session = statEntries[4];
       vmem = statEntries[5];
       vmem = statEntries[5];
+      if (statEntries.length > 6) {
+        rssmemPage = statEntries[6];
+      }
+      if (statEntries.length > 7) {
+        utime = statEntries[7];
+        stime = statEntries[8];
+      }
     }
     }
     
     
     // construct a line that mimics the procfs stat file.
     // construct a line that mimics the procfs stat file.
     // all unused numerical entries are set to 0.
     // all unused numerical entries are set to 0.
     public String getStatLine() {
     public String getStatLine() {
       return String.format("%s (%s) S %s %s %s 0 0 0" +
       return String.format("%s (%s) S %s %s %s 0 0 0" +
-                      " 0 0 0 0 0 0 0 0 0 0 0 0 0 %s 0 0 0" +
+                      " 0 0 0 0 %s %s 0 0 0 0 0 0 0 %s %s 0 0" +
                       " 0 0 0 0 0 0 0 0" +
                       " 0 0 0 0 0 0 0 0" +
                       " 0 0 0 0 0", 
                       " 0 0 0 0 0", 
-                      pid, name, ppid, pgrpId, session, vmem);
+                      pid, name, ppid, pgrpId, session,
+                      utime, stime, vmem, rssmemPage);
     }
     }
   }
   }
   
   
   /**
   /**
    * A basic test that creates a few process directories and writes
    * A basic test that creates a few process directories and writes
-   * stat files. Verifies that the virtual memory is correctly  
+   * stat files. Verifies that the cpu time and memory is correctly
    * computed.
    * computed.
    * @throws IOException if there was a problem setting up the
    * @throws IOException if there was a problem setting up the
    *                      fake procfs directories or files.
    *                      fake procfs directories or files.
    */
    */
-  public void testVirtualMemoryForProcessTree() throws IOException {
+  public void testCpuAndMemoryForProcessTree() throws IOException {
 
 
     // test processes
     // test processes
     String[] pids = { "100", "200", "300", "400" };
     String[] pids = { "100", "200", "300", "400" };
@@ -265,13 +277,13 @@ public class TestProcfsBasedProcessTree extends TestCase {
       // assuming processes 100, 200, 300 are in tree and 400 is not.
       // assuming processes 100, 200, 300 are in tree and 400 is not.
       ProcessStatInfo[] procInfos = new ProcessStatInfo[4];
       ProcessStatInfo[] procInfos = new ProcessStatInfo[4];
       procInfos[0] = new ProcessStatInfo(new String[] 
       procInfos[0] = new ProcessStatInfo(new String[] 
-                                  {"100", "proc1", "1", "100", "100", "100000"});
+          {"100", "proc1", "1", "100", "100", "100000", "100", "1000", "200"});
       procInfos[1] = new ProcessStatInfo(new String[] 
       procInfos[1] = new ProcessStatInfo(new String[] 
-                                  {"200", "proc2", "100", "100", "100", "200000"});
+          {"200", "proc2", "100", "100", "100", "200000", "200", "2000", "400"});
       procInfos[2] = new ProcessStatInfo(new String[] 
       procInfos[2] = new ProcessStatInfo(new String[] 
-                                  {"300", "proc3", "200", "100", "100", "300000"});
+          {"300", "proc3", "200", "100", "100", "300000", "300", "3000", "600"});
       procInfos[3] = new ProcessStatInfo(new String[] 
       procInfos[3] = new ProcessStatInfo(new String[] 
-                                  {"400", "proc4", "1", "400", "400", "400000"});
+          {"400", "proc4", "1", "400", "400", "400000", "400", "4000", "800"});
       
       
       writeStatFiles(procfsRootDir, pids, procInfos);
       writeStatFiles(procfsRootDir, pids, procInfos);
       
       
@@ -282,8 +294,36 @@ public class TestProcfsBasedProcessTree extends TestCase {
       processTree.getProcessTree();
       processTree.getProcessTree();
       
       
       // verify cumulative memory
       // verify cumulative memory
-      assertEquals("Cumulative memory does not match", 
-              Long.parseLong("600000"), processTree.getCumulativeVmem());
+      assertEquals("Cumulative virtual memory does not match", 600000L,
+                   processTree.getCumulativeVmem());
+
+      // verify rss memory
+      long cumuRssMem = ProcfsBasedProcessTree.PAGE_SIZE > 0 ?
+                        600L * ProcfsBasedProcessTree.PAGE_SIZE : 0L;
+      assertEquals("Cumulative rss memory does not match",
+                   cumuRssMem, processTree.getCumulativeRssmem());
+
+      // verify cumulative cpu time
+      long cumuCpuTime = ProcfsBasedProcessTree.JIFFY_LENGTH_IN_MILLIS > 0 ?
+             7200L * ProcfsBasedProcessTree.JIFFY_LENGTH_IN_MILLIS : 0L;
+      assertEquals("Cumulative cpu time does not match",
+                   cumuCpuTime, processTree.getCumulativeCpuTime());
+
+      // test the cpu time again to see if it cumulates
+      procInfos[0] = new ProcessStatInfo(new String[]
+          {"100", "proc1", "1", "100", "100", "100000", "100", "2000", "300"});
+      procInfos[1] = new ProcessStatInfo(new String[]
+          {"200", "proc2", "100", "100", "100", "200000", "200", "3000", "500"});
+      writeStatFiles(procfsRootDir, pids, procInfos);
+
+      // build the process tree.
+      processTree.getProcessTree();
+
+      // verify cumulative cpu time again
+      cumuCpuTime = ProcfsBasedProcessTree.JIFFY_LENGTH_IN_MILLIS > 0 ?
+             9400L * ProcfsBasedProcessTree.JIFFY_LENGTH_IN_MILLIS : 0L;
+      assertEquals("Cumulative cpu time does not match",
+                   cumuCpuTime, processTree.getCumulativeCpuTime());
     } finally {
     } finally {
       FileUtil.fullyDelete(procfsRootDir);
       FileUtil.fullyDelete(procfsRootDir);
     }
     }
@@ -295,7 +335,7 @@ public class TestProcfsBasedProcessTree extends TestCase {
    * @throws IOException if there was a problem setting up the
    * @throws IOException if there was a problem setting up the
    *                      fake procfs directories or files.
    *                      fake procfs directories or files.
    */
    */
-  public void testVMemForOlderProcesses() throws IOException {
+  public void testMemForOlderProcesses() throws IOException {
     // initial list of processes
     // initial list of processes
     String[] pids = { "100", "200", "300", "400" };
     String[] pids = { "100", "200", "300", "400" };
     // create the fake procfs root directory. 
     // create the fake procfs root directory. 
@@ -309,13 +349,13 @@ public class TestProcfsBasedProcessTree extends TestCase {
       // assuming 100, 200 and 400 are in tree, 300 is not.
       // assuming 100, 200 and 400 are in tree, 300 is not.
       ProcessStatInfo[] procInfos = new ProcessStatInfo[4];
       ProcessStatInfo[] procInfos = new ProcessStatInfo[4];
       procInfos[0] = new ProcessStatInfo(new String[] 
       procInfos[0] = new ProcessStatInfo(new String[] 
-                                  {"100", "proc1", "1", "100", "100", "100000"});
+                        {"100", "proc1", "1", "100", "100", "100000", "100"});
       procInfos[1] = new ProcessStatInfo(new String[] 
       procInfos[1] = new ProcessStatInfo(new String[] 
-                                  {"200", "proc2", "100", "100", "100", "200000"});
+                        {"200", "proc2", "100", "100", "100", "200000", "200"});
       procInfos[2] = new ProcessStatInfo(new String[] 
       procInfos[2] = new ProcessStatInfo(new String[] 
-                                  {"300", "proc3", "1", "300", "300", "300000"});
+                        {"300", "proc3", "1", "300", "300", "300000", "300"});
       procInfos[3] = new ProcessStatInfo(new String[] 
       procInfos[3] = new ProcessStatInfo(new String[] 
-                                  {"400", "proc4", "100", "100", "100", "400000"});
+                        {"400", "proc4", "100", "100", "100", "400000", "400"});
       
       
       writeStatFiles(procfsRootDir, pids, procInfos);
       writeStatFiles(procfsRootDir, pids, procInfos);
       
       
@@ -326,51 +366,69 @@ public class TestProcfsBasedProcessTree extends TestCase {
       processTree.getProcessTree();
       processTree.getProcessTree();
       
       
       // verify cumulative memory
       // verify cumulative memory
-      assertEquals("Cumulative memory does not match", 
-              Long.parseLong("700000"), processTree.getCumulativeVmem());
-      
+      assertEquals("Cumulative memory does not match",
+                   700000L, processTree.getCumulativeVmem());
+
       // write one more process as child of 100.
       // write one more process as child of 100.
       String[] newPids = { "500" };
       String[] newPids = { "500" };
       setupPidDirs(procfsRootDir, newPids);
       setupPidDirs(procfsRootDir, newPids);
       
       
       ProcessStatInfo[] newProcInfos = new ProcessStatInfo[1];
       ProcessStatInfo[] newProcInfos = new ProcessStatInfo[1];
       newProcInfos[0] = new ProcessStatInfo(new String[]
       newProcInfos[0] = new ProcessStatInfo(new String[]
-                             {"500", "proc5", "100", "100", "100", "500000"});
+                      {"500", "proc5", "100", "100", "100", "500000", "500"});
       writeStatFiles(procfsRootDir, newPids, newProcInfos);
       writeStatFiles(procfsRootDir, newPids, newProcInfos);
       
       
-      // check vmem includes the new process.
+      // check memory includes the new process.
       processTree.getProcessTree();
       processTree.getProcessTree();
-      assertEquals("Cumulative memory does not include new process",
-              Long.parseLong("1200000"), processTree.getCumulativeVmem());
+      assertEquals("Cumulative vmem does not include new process",
+                   1200000L, processTree.getCumulativeVmem());
+      long cumuRssMem = ProcfsBasedProcessTree.PAGE_SIZE > 0 ?
+                        1200L * ProcfsBasedProcessTree.PAGE_SIZE : 0L;
+      assertEquals("Cumulative rssmem does not include new process",
+                   cumuRssMem, processTree.getCumulativeRssmem());
       
       
       // however processes older than 1 iteration will retain the older value
       // however processes older than 1 iteration will retain the older value
-      assertEquals("Cumulative memory shouldn't have included new process",
-              Long.parseLong("700000"), processTree.getCumulativeVmem(1));
-      
+      assertEquals("Cumulative vmem shouldn't have included new process",
+                   700000L, processTree.getCumulativeVmem(1));
+      cumuRssMem = ProcfsBasedProcessTree.PAGE_SIZE > 0 ?
+                   700L * ProcfsBasedProcessTree.PAGE_SIZE : 0L;
+      assertEquals("Cumulative rssmem shouldn't have included new process",
+                   cumuRssMem, processTree.getCumulativeRssmem(1));
+
       // one more process
       // one more process
       newPids = new String[]{ "600" };
       newPids = new String[]{ "600" };
       setupPidDirs(procfsRootDir, newPids);
       setupPidDirs(procfsRootDir, newPids);
       
       
       newProcInfos = new ProcessStatInfo[1];
       newProcInfos = new ProcessStatInfo[1];
       newProcInfos[0] = new ProcessStatInfo(new String[]
       newProcInfos[0] = new ProcessStatInfo(new String[]
-                                     {"600", "proc6", "100", "100", "100", "600000"});
+                      {"600", "proc6", "100", "100", "100", "600000", "600"});
       writeStatFiles(procfsRootDir, newPids, newProcInfos);
       writeStatFiles(procfsRootDir, newPids, newProcInfos);
 
 
       // refresh process tree
       // refresh process tree
       processTree.getProcessTree();
       processTree.getProcessTree();
       
       
       // processes older than 2 iterations should be same as before.
       // processes older than 2 iterations should be same as before.
-      assertEquals("Cumulative memory shouldn't have included new processes",
-          Long.parseLong("700000"), processTree.getCumulativeVmem(2));
-      
+      assertEquals("Cumulative vmem shouldn't have included new processes",
+                   700000L, processTree.getCumulativeVmem(2));
+      cumuRssMem = ProcfsBasedProcessTree.PAGE_SIZE > 0 ?
+                   700L * ProcfsBasedProcessTree.PAGE_SIZE : 0L;
+      assertEquals("Cumulative rssmem shouldn't have included new processes",
+                   cumuRssMem, processTree.getCumulativeRssmem(2));
+
       // processes older than 1 iteration should not include new process,
       // processes older than 1 iteration should not include new process,
       // but include process 500
       // but include process 500
-      assertEquals("Cumulative memory shouldn't have included new processes",
-          Long.parseLong("1200000"), processTree.getCumulativeVmem(1));
-      
+      assertEquals("Cumulative vmem shouldn't have included new processes",
+                   1200000L, processTree.getCumulativeVmem(1));
+      cumuRssMem = ProcfsBasedProcessTree.PAGE_SIZE > 0 ?
+                   1200L * ProcfsBasedProcessTree.PAGE_SIZE : 0L;
+      assertEquals("Cumulative rssmem shouldn't have included new processes",
+                   cumuRssMem, processTree.getCumulativeRssmem(1));
+
       // no processes older than 3 iterations, this should be 0
       // no processes older than 3 iterations, this should be 0
       assertEquals("Getting non-zero vmem for processes older than 3 iterations",
       assertEquals("Getting non-zero vmem for processes older than 3 iterations",
                     0L, processTree.getCumulativeVmem(3));
                     0L, processTree.getCumulativeVmem(3));
+      assertEquals("Getting non-zero rssmem for processes older than 3 iterations",
+                    0L, processTree.getCumulativeRssmem(3));
     } finally {
     } finally {
       FileUtil.fullyDelete(procfsRootDir);
       FileUtil.fullyDelete(procfsRootDir);
     }
     }
@@ -395,24 +453,18 @@ public class TestProcfsBasedProcessTree extends TestCase {
       int numProcesses = pids.length;
       int numProcesses = pids.length;
       // Processes 200, 300, 400 and 500 are descendants of 100. 600 is not.
       // Processes 200, 300, 400 and 500 are descendants of 100. 600 is not.
       ProcessStatInfo[] procInfos = new ProcessStatInfo[numProcesses];
       ProcessStatInfo[] procInfos = new ProcessStatInfo[numProcesses];
-      procInfos[0] =
-          new ProcessStatInfo(new String[] { "100", "proc1", "1", "100",
-              "100", "100000" });
-      procInfos[1] =
-          new ProcessStatInfo(new String[] { "200", "proc2", "100", "100",
-              "100", "200000" });
-      procInfos[2] =
-          new ProcessStatInfo(new String[] { "300", "proc3", "200", "100",
-              "100", "300000" });
-      procInfos[3] =
-          new ProcessStatInfo(new String[] { "400", "proc4", "200", "100",
-              "100", "400000" });
-      procInfos[4] =
-          new ProcessStatInfo(new String[] { "500", "proc5", "400", "100",
-              "100", "400000" });
-      procInfos[5] =
-          new ProcessStatInfo(new String[] { "600", "proc6", "1", "1", "1",
-              "400000" });
+      procInfos[0] = new ProcessStatInfo(new String[] {
+          "100", "proc1", "1", "100", "100", "100000", "100", "1000", "200"});
+      procInfos[1] = new ProcessStatInfo(new String[] {
+          "200", "proc2", "100", "100", "100", "200000", "200", "2000", "400"});
+      procInfos[2] = new ProcessStatInfo(new String[] {
+          "300", "proc3", "200", "100", "100", "300000", "300", "3000", "600"});
+      procInfos[3] = new ProcessStatInfo(new String[] {
+          "400", "proc4", "200", "100", "100", "400000", "400", "4000", "800"});
+      procInfos[4] = new ProcessStatInfo(new String[] {
+          "500", "proc5", "400", "100", "100", "400000", "400", "4000", "800"});
+      procInfos[5] = new ProcessStatInfo(new String[] {
+          "600", "proc6", "1", "1", "1", "400000", "400", "4000", "800"});
 
 
       String[] cmdLines = new String[numProcesses];
       String[] cmdLines = new String[numProcesses];
       cmdLines[0] = "proc1 arg1 arg2";
       cmdLines[0] = "proc1 arg1 arg2";
@@ -435,15 +487,17 @@ public class TestProcfsBasedProcessTree extends TestCase {
 
 
       LOG.info("Process-tree dump follows: \n" + processTreeDump);
       LOG.info("Process-tree dump follows: \n" + processTreeDump);
       assertTrue("Process-tree dump doesn't start with a proper header",
       assertTrue("Process-tree dump doesn't start with a proper header",
-          processTreeDump.startsWith("\t|- PID PPID PGRPID SESSID CMD_NAME "
-              + "VMEM_USAGE(BYTES) FULL_CMD_LINE\n"));
+          processTreeDump.startsWith("\t|- PID PPID PGRPID SESSID CMD_NAME " +
+          "USER_MODE_TIME(MILLIS) SYSTEM_TIME(MILLIS) VMEM_USAGE(BYTES) " +
+          "RSSMEM_USAGE(PAGES) FULL_CMD_LINE\n"));
       for (int i = 0; i < 5; i++) {
       for (int i = 0; i < 5; i++) {
         ProcessStatInfo p = procInfos[i];
         ProcessStatInfo p = procInfos[i];
         assertTrue(
         assertTrue(
             "Process-tree dump doesn't contain the cmdLineDump of process "
             "Process-tree dump doesn't contain the cmdLineDump of process "
                 + p.pid, processTreeDump.contains("\t|- " + p.pid + " "
                 + p.pid, processTreeDump.contains("\t|- " + p.pid + " "
                 + p.ppid + " " + p.pgrpId + " " + p.session + " (" + p.name
                 + p.ppid + " " + p.pgrpId + " " + p.session + " (" + p.name
-                + ") " + p.vmem + " " + cmdLines[i]));
+                + ") " + p.utime + " " + p.stime + " " + p.vmem + " "
+                + p.rssmemPage + " " + cmdLines[i]));
       }
       }
 
 
       // 600 should not be in the dump
       // 600 should not be in the dump
@@ -452,7 +506,7 @@ public class TestProcfsBasedProcessTree extends TestCase {
           "Process-tree dump shouldn't contain the cmdLineDump of process "
           "Process-tree dump shouldn't contain the cmdLineDump of process "
               + p.pid, processTreeDump.contains("\t|- " + p.pid + " " + p.ppid
               + p.pid, processTreeDump.contains("\t|- " + p.pid + " " + p.ppid
               + " " + p.pgrpId + " " + p.session + " (" + p.name + ") "
               + " " + p.pgrpId + " " + p.session + " (" + p.name + ") "
-              + p.vmem + " " + cmdLines[5]));
+              + p.utime + " " + p.stime + " " + p.vmem + " " + cmdLines[5]));
     } finally {
     } finally {
       FileUtil.fullyDelete(procfsRootDir);
       FileUtil.fullyDelete(procfsRootDir);
     }
     }