You are viewing a plain text version of this content. The canonical link for it is here.
Posted to yarn-commits@hadoop.apache.org by bo...@apache.org on 2012/09/27 18:38:55 UTC

svn commit: r1391094 - in /hadoop/common/branches/branch-2/hadoop-yarn-project: ./ hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/conf/ hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ hadoop-yarn/hadoop-ya...

Author: bobby
Date: Thu Sep 27 16:38:54 2012
New Revision: 1391094

URL: http://svn.apache.org/viewvc?rev=1391094&view=rev
Log:
svn merge -c 1382072 FIXES: YARN-57. Allow process-tree based resource calculation et al. to be pluggable to support it on multiple platforms. Contributed by Radim Kolar.

Added:
    hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ResourceCalculatorProcessTree.java
      - copied unchanged from r1382072, hadoop/common/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ResourceCalculatorProcessTree.java
Modified:
    hadoop/common/branches/branch-2/hadoop-yarn-project/CHANGES.txt
    hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
    hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/LinuxResourceCalculatorPlugin.java
    hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ProcfsBasedProcessTree.java
    hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ResourceCalculatorPlugin.java
    hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestProcfsBasedProcessTree.java
    hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java
    hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/TestContainersMonitor.java

Modified: hadoop/common/branches/branch-2/hadoop-yarn-project/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-yarn-project/CHANGES.txt?rev=1391094&r1=1391093&r2=1391094&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-yarn-project/CHANGES.txt (original)
+++ hadoop/common/branches/branch-2/hadoop-yarn-project/CHANGES.txt Thu Sep 27 16:38:54 2012
@@ -87,6 +87,9 @@ Release 0.23.4 - UNRELEASED
     YARN-108. FSDownload can create cache directories with the wrong
     permissions (Jason Lowe via bobby)
 
+    YARN-57. Allow process-tree based resource calculation et al. to be
+    pluggable to support it on multiple platforms. (Radim Kolar via acmurthy)
+
   OPTIMIZATIONS
 
   BUG FIXES

Modified: hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java?rev=1391094&r1=1391093&r2=1391094&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java (original)
+++ hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java Thu Sep 27 16:38:54 2012
@@ -424,10 +424,13 @@ public class YarnConfiguration extends C
   public final static String NM_CONTAINER_MON_INTERVAL_MS =
     NM_PREFIX + "container-monitor.interval-ms";
   public final static int DEFAULT_NM_CONTAINER_MON_INTERVAL_MS = 3000;
-  
+
   /** Class that calculates containers current resource utilization.*/
   public static final String NM_CONTAINER_MON_RESOURCE_CALCULATOR =
     NM_PREFIX + "container-monitor.resource-calculator.class";
+  /** Class that calculates process tree resource utilization.*/
+  public static final String NM_CONTAINER_MON_PROCESS_TREE =
+    NM_PREFIX + "container-monitor.process-tree.class";
 
   /**
    * Enable/Disable disks' health checker. Default is true.

Modified: hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/LinuxResourceCalculatorPlugin.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/LinuxResourceCalculatorPlugin.java?rev=1391094&r1=1391093&r2=1391094&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/LinuxResourceCalculatorPlugin.java (original)
+++ hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/LinuxResourceCalculatorPlugin.java Thu Sep 27 16:38:54 2012
@@ -72,7 +72,7 @@ public class LinuxResourceCalculatorPlug
   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;
@@ -91,11 +91,11 @@ public class LinuxResourceCalculatorPlug
   private float cpuUsage = UNAVAILABLE;
   private long sampleTime = UNAVAILABLE;
   private long lastSampleTime = UNAVAILABLE;
-  private ProcfsBasedProcessTree pTree = null;
+  private ResourceCalculatorProcessTree pTree = null;
 
   boolean readMemInfoFile = false;
   boolean readCpuInfoFile = false;
-  
+
   /**
    * Get current time
    * @return Unix time stamp in millisecond
@@ -103,7 +103,7 @@ public class LinuxResourceCalculatorPlug
   long getCurrentTime() {
     return System.currentTimeMillis();
   }
-  
+
   public LinuxResourceCalculatorPlugin() {
     procfsMemFile = PROCFS_MEMFILE;
     procfsCpuFile = PROCFS_CPUINFO;
@@ -112,7 +112,7 @@ public class LinuxResourceCalculatorPlug
     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

Modified: hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ProcfsBasedProcessTree.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ProcfsBasedProcessTree.java?rev=1391094&r1=1391093&r2=1391094&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ProcfsBasedProcessTree.java (original)
+++ hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ProcfsBasedProcessTree.java Thu Sep 27 16:38:54 2012
@@ -44,7 +44,7 @@ import org.apache.hadoop.util.StringUtil
  */
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
-public class ProcfsBasedProcessTree {
+public class ProcfsBasedProcessTree extends ResourceCalculatorProcessTree {
 
   static final Log LOG = LogFactory
       .getLog(ProcfsBasedProcessTree.class);
@@ -91,44 +91,36 @@ public class ProcfsBasedProcessTree {
   // to enable testing, using this variable which can be configured
   // to a test directory.
   private String procfsDir;
-  
+
   static private String deadPid = "-1";
   private String pid = deadPid;
   static private Pattern numberPattern = Pattern.compile("[1-9][0-9]*");
   private Long cpuTime = 0L;
-  private boolean setsidUsed = false;
 
   protected Map<String, ProcessInfo> processTree =
     new HashMap<String, ProcessInfo>();
 
   public ProcfsBasedProcessTree(String pid) {
-    this(pid, false);
-  }
-
-  public ProcfsBasedProcessTree(String pid, boolean setsidUsed) {
-    this(pid, setsidUsed, PROCFS);
+    this(pid, PROCFS);
   }
 
   /**
    * Build a new process tree rooted at the pid.
-   * 
+   *
    * This method is provided mainly for testing purposes, where
    * the root of the proc file system can be adjusted.
-   * 
+   *
    * @param pid root of the process tree
-   * @param setsidUsed true, if setsid was used for the root pid
-   * @param procfsDir the root of a proc file system - only used for testing. 
+   * @param procfsDir the root of a proc file system - only used for testing.
    */
-  public ProcfsBasedProcessTree(String pid, boolean setsidUsed,
-      String procfsDir) {
+  public ProcfsBasedProcessTree(String pid, String procfsDir) {
     this.pid = getValidPID(pid);
-    this.setsidUsed = setsidUsed;
     this.procfsDir = procfsDir;
   }
 
   /**
    * Checks if the ProcfsBasedProcessTree is available on this system.
-   * 
+   *
    * @return true if ProcfsBasedProcessTree is available. False otherwise.
    */
   public static boolean isAvailable() {
@@ -149,18 +141,19 @@ public class ProcfsBasedProcessTree {
   /**
    * Get the process-tree with latest state. If the root-process is not alive,
    * an empty tree will be returned.
-   * 
+   *
    * @return the process-tree with latest state.
    */
-  public ProcfsBasedProcessTree getProcessTree() {
+  @Override
+  public ResourceCalculatorProcessTree getProcessTree() {
     if (!pid.equals(deadPid)) {
       // Get the list of processes
       List<String> processList = getProcessList();
 
       Map<String, ProcessInfo> allProcessInfo = new HashMap<String, ProcessInfo>();
-      
+
       // cache the processTree to get the age for processes
-      Map<String, ProcessInfo> oldProcs = 
+      Map<String, ProcessInfo> oldProcs =
               new HashMap<String, ProcessInfo>(processTree);
       processTree.clear();
 
@@ -178,7 +171,7 @@ public class ProcfsBasedProcessTree {
       }
 
       if (me == null) {
-        return this; 
+        return this;
       }
 
       // Add each process to its parent.
@@ -210,7 +203,7 @@ public class ProcfsBasedProcessTree {
         if (procs.getValue() != null) {
           procs.getValue().updateJiffy(oldInfo);
           if (oldInfo != null) {
-            procs.getValue().updateAge(oldInfo);  
+            procs.getValue().updateAge(oldInfo);
           }
         }
       }
@@ -226,6 +219,7 @@ public class ProcfsBasedProcessTree {
   /** Verify that the given process id is same as its process group id.
    * @return true if the process id matches else return false.
    */
+  @Override
   public boolean checkPidPgrpidForMatch() {
     return checkPidPgrpidForMatch(pid, PROCFS);
   }
@@ -252,10 +246,11 @@ public class ProcfsBasedProcessTree {
 
   /**
    * Get a dump of the process-tree.
-   * 
+   *
    * @return a string concatenating the dump of information of all the processes
    *         in the process-tree
    */
+  @Override
   public String getProcessTreeDump() {
     StringBuilder ret = new StringBuilder();
     // The header.
@@ -275,36 +270,14 @@ public class ProcfsBasedProcessTree {
 
   /**
    * Get the cumulative virtual memory used by all the processes in the
-   * process-tree.
-   * 
-   * @return cumulative virtual memory used by the process-tree in bytes.
-   */
-  public long getCumulativeVmem() {
-    // include all processes.. all processes will be older than 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
    * process-tree that are older than the passed in age.
-   * 
+   *
    * @param olderThanAge processes above this age are included in the
    *                      memory addition
    * @return cumulative virtual memory used by the process-tree in bytes,
    *          for processes older than this age.
    */
+  @Override
   public long getCumulativeVmem(int olderThanAge) {
     long total = 0;
     for (ProcessInfo p : processTree.values()) {
@@ -314,7 +287,7 @@ public class ProcfsBasedProcessTree {
     }
     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.
@@ -325,6 +298,7 @@ public class ProcfsBasedProcessTree {
    *          for processes older than this age. return 0 if it cannot be
    *          calculated
    */
+  @Override
   public long getCumulativeRssmem(int olderThanAge) {
     if (PAGE_SIZE < 0) {
       return 0;
@@ -345,6 +319,7 @@ public class ProcfsBasedProcessTree {
    * @return cumulative CPU time in millisecond since the process-tree created
    *         return 0 if it cannot be calculated
    */
+  @Override
   public long getCumulativeCpuTime() {
     if (JIFFY_LENGTH_IN_MILLIS < 0) {
       return 0;
@@ -352,7 +327,7 @@ public class ProcfsBasedProcessTree {
     long incJiffies = 0;
     for (ProcessInfo p : processTree.values()) {
       if (p != null) {
-        incJiffies += p.dtime;
+        incJiffies += p.getDtime();
       }
     }
     cpuTime += incJiffies * JIFFY_LENGTH_IN_MILLIS;
@@ -391,14 +366,14 @@ public class ProcfsBasedProcessTree {
    * Construct the ProcessInfo using the process' PID and procfs rooted at the
    * specified directory and return the same. It is provided mainly to assist
    * testing purposes.
-   * 
+   *
    * Returns null on failing to read from procfs,
    *
    * @param pinfo ProcessInfo that needs to be updated
    * @param procfsDir root of the proc file system
    * @return updated ProcessInfo, null on errors.
    */
-  private static ProcessInfo constructProcessInfo(ProcessInfo pinfo, 
+  private static ProcessInfo constructProcessInfo(ProcessInfo pinfo,
                                                     String procfsDir) {
     ProcessInfo ret = null;
     // Read "procfsDir/<pid>/stat" file - typically /proc/<pid>/stat
@@ -454,6 +429,7 @@ public class ProcfsBasedProcessTree {
    * Returns a string printing PIDs of process present in the
    * ProcfsBasedProcessTree. Output format : [pid pid ..]
    */
+  @Override
   public String toString() {
     StringBuffer pTree = new StringBuffer("[ ");
     for (String p : processTree.keySet()) {
@@ -464,9 +440,9 @@ public class ProcfsBasedProcessTree {
   }
 
   /**
-   * 
+   *
    * Class containing information of a process.
-   * 
+   *
    */
   private static class ProcessInfo {
     private String pid; // process-id
@@ -480,7 +456,7 @@ public class ProcfsBasedProcessTree {
     private final BigInteger MAX_LONG = BigInteger.valueOf(Long.MAX_VALUE);
     private BigInteger stime = new BigInteger("0"); // # of jiffies in kernel mode
     // 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;
@@ -539,13 +515,6 @@ public class ProcfsBasedProcessTree {
     public int getAge() {
       return age;
     }
-    
-    public boolean isParent(ProcessInfo p) {
-      if (pid.equals(p.getPpid())) {
-        return true;
-      }
-      return false;
-    }
 
     public void updateProcessInfo(String name, String ppid, Integer pgrpId,
         Integer sessionId, Long utime, BigInteger stime, Long vmem, Long rssmem) {
@@ -558,7 +527,7 @@ public class ProcfsBasedProcessTree {
       this.vmem = vmem;
       this.rssmemPage = rssmem;
     }
-    
+
     public void updateJiffy(ProcessInfo oldInfo) {
       if (oldInfo == null) {
         BigInteger sum = this.stime.add(BigInteger.valueOf(this.utime));
@@ -578,7 +547,7 @@ public class ProcfsBasedProcessTree {
     public void updateAge(ProcessInfo oldInfo) {
       this.age = oldInfo.age + 1;
     }
-    
+
     public boolean addChild(ProcessInfo p) {
       return children.add(p);
     }

Modified: hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ResourceCalculatorPlugin.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ResourceCalculatorPlugin.java?rev=1391094&r1=1391093&r2=1391094&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ResourceCalculatorPlugin.java (original)
+++ hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ResourceCalculatorPlugin.java Thu Sep 27 16:38:54 2012
@@ -26,7 +26,7 @@ import org.apache.hadoop.util.Reflection
 
 /**
  * Plugin to calculate resource information on the system.
- * 
+ *
  */
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
@@ -34,14 +34,14 @@ public abstract class ResourceCalculator
 
   /**
    * 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();
@@ -133,13 +133,14 @@ public abstract class ResourceCalculator
   }
 
   /**
-   * Get the ResourceCalculatorPlugin from the class name and configure it. If
+   * Create 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 clazz ResourceCalculator plugin class-name
    * @param conf configure the plugin with this.
-   * @return ResourceCalculatorPlugin
+   * @return ResourceCalculatorPlugin or null if ResourceCalculatorPlugin is not
+   * 		 available for current system
    */
   public static ResourceCalculatorPlugin getResourceCalculatorPlugin(
       Class<? extends ResourceCalculatorPlugin> clazz, Configuration conf) {

Modified: hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestProcfsBasedProcessTree.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestProcfsBasedProcessTree.java?rev=1391094&r1=1391093&r2=1391094&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestProcfsBasedProcessTree.java (original)
+++ hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestProcfsBasedProcessTree.java Thu Sep 27 16:38:54 2012
@@ -161,7 +161,7 @@ public class TestProcfsBasedProcessTree 
     String pid = getRogueTaskPID();
     LOG.info("Root process pid: " + pid);
     ProcfsBasedProcessTree p = createProcessTree(pid);
-    p = p.getProcessTree(); // initialize
+    p.getProcessTree(); // initialize
     LOG.info("ProcessTree: " + p.toString());
 
     File leaf = new File(lowestDescendant);
@@ -174,7 +174,7 @@ public class TestProcfsBasedProcessTree 
       }
     }
 
-    p = p.getProcessTree(); // reconstruct
+    p.getProcessTree(); // reconstruct
     LOG.info("ProcessTree: " + p.toString());
 
     // Get the process-tree dump
@@ -213,7 +213,7 @@ public class TestProcfsBasedProcessTree 
     }
 
     // ProcessTree is gone now. Any further calls should be sane.
-    p = p.getProcessTree();
+    p.getProcessTree();
     Assert.assertFalse("ProcessTree must have been gone", isAlive(pid));
     Assert.assertTrue("Cumulative vmem for the gone-process is "
         + p.getCumulativeVmem() + " . It should be zero.", p
@@ -222,13 +222,11 @@ public class TestProcfsBasedProcessTree 
   }
 
   protected ProcfsBasedProcessTree createProcessTree(String pid) {
-    return new ProcfsBasedProcessTree(pid,
-                               isSetsidAvailable());
+    return new ProcfsBasedProcessTree(pid);
   }
 
-  protected ProcfsBasedProcessTree createProcessTree(String pid,
-      boolean setsidUsed, String procfsRootDir) {
-    return new ProcfsBasedProcessTree(pid, setsidUsed, procfsRootDir);
+  protected ProcfsBasedProcessTree createProcessTree(String pid, String procfsRootDir) {
+    return new ProcfsBasedProcessTree(pid, procfsRootDir);
   }
 
   protected void destroyProcessTree(String pid) throws IOException {
@@ -237,7 +235,7 @@ public class TestProcfsBasedProcessTree 
 
   /**
    * Get PID from a pid-file.
-   * 
+   *
    * @param pidFileName
    *          Name of the pid-file.
    * @return the PID string read from the pid-file. Returns null if the
@@ -279,11 +277,11 @@ public class TestProcfsBasedProcessTree 
     }
     return pid;
   }
-  
+
   public static class ProcessStatInfo {
-    // sample stat in a single line : 3910 (gpm) S 1 3910 3910 0 -1 4194624 
-    // 83 0 0 0 0 0 0 0 16 0 1 0 7852 2408448 88 4294967295 134512640 
-    // 134590050 3220521392 3220520036 10975138 0 0 4096 134234626 
+    // sample stat in a single line : 3910 (gpm) S 1 3910 3910 0 -1 4194624
+    // 83 0 0 0 0 0 0 0 16 0 1 0 7852 2408448 88 4294967295 134512640
+    // 134590050 3220521392 3220520036 10975138 0 0 4096 134234626
     // 4294967295 0 0 17 1 0 0
     String pid;
     String name;
@@ -294,7 +292,7 @@ public class TestProcfsBasedProcessTree 
     String rssmemPage = "0";
     String utime = "0";
     String stime = "0";
-    
+
     public ProcessStatInfo(String[] statEntries) {
       pid = statEntries[0];
       name = statEntries[1];
@@ -310,19 +308,19 @@ public class TestProcfsBasedProcessTree 
         stime = statEntries[8];
       }
     }
-    
+
     // construct a line that mimics the procfs stat file.
     // all unused numerical entries are set to 0.
     public String getStatLine() {
       return String.format("%s (%s) S %s %s %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",
                       pid, name, ppid, pgrpId, session,
                       utime, stime, vmem, rssmemPage);
     }
   }
-  
+
   /**
    * A basic test that creates a few process directories and writes
    * stat files. Verifies that the cpu time and memory is correctly
@@ -335,33 +333,33 @@ public class TestProcfsBasedProcessTree 
 
     // test processes
     String[] pids = { "100", "200", "300", "400" };
-    // create the fake procfs root directory. 
+    // create the fake procfs root directory.
     File procfsRootDir = new File(TEST_ROOT_DIR, "proc");
 
     try {
       setupProcfsRootDir(procfsRootDir);
       setupPidDirs(procfsRootDir, pids);
-      
+
       // create stat objects.
       // assuming processes 100, 200, 300 are in tree and 400 is not.
       ProcessStatInfo[] procInfos = new ProcessStatInfo[4];
-      procInfos[0] = new ProcessStatInfo(new String[] 
+      procInfos[0] = new ProcessStatInfo(new String[]
           {"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", "2000", "400"});
-      procInfos[2] = new ProcessStatInfo(new String[] 
+      procInfos[2] = new ProcessStatInfo(new String[]
           {"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", "4000", "800"});
-      
+
       writeStatFiles(procfsRootDir, pids, procInfos);
-      
+
       // crank up the process tree class.
       ProcfsBasedProcessTree processTree =
-          createProcessTree("100", true, procfsRootDir.getAbsolutePath());
+          createProcessTree("100", procfsRootDir.getAbsolutePath());
       // build the process tree.
       processTree.getProcessTree();
-      
+
       // verify cumulative memory
       Assert.assertEquals("Cumulative virtual memory does not match", 600000L,
                    processTree.getCumulativeVmem());
@@ -397,7 +395,7 @@ public class TestProcfsBasedProcessTree 
       FileUtil.fullyDelete(procfsRootDir);
     }
   }
-  
+
   /**
    * Tests that cumulative memory is computed only for
    * processes older than a given age.
@@ -408,33 +406,33 @@ public class TestProcfsBasedProcessTree 
   public void testMemForOlderProcesses() throws IOException {
     // initial list of processes
     String[] pids = { "100", "200", "300", "400" };
-    // create the fake procfs root directory. 
+    // create the fake procfs root directory.
     File procfsRootDir = new File(TEST_ROOT_DIR, "proc");
 
     try {
       setupProcfsRootDir(procfsRootDir);
       setupPidDirs(procfsRootDir, pids);
-      
+
       // create stat objects.
       // assuming 100, 200 and 400 are in tree, 300 is not.
       ProcessStatInfo[] procInfos = new ProcessStatInfo[4];
-      procInfos[0] = new ProcessStatInfo(new String[] 
+      procInfos[0] = new ProcessStatInfo(new String[]
                         {"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"});
-      procInfos[2] = new ProcessStatInfo(new String[] 
+      procInfos[2] = new ProcessStatInfo(new String[]
                         {"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"});
-      
+
       writeStatFiles(procfsRootDir, pids, procInfos);
-      
+
       // crank up the process tree class.
       ProcfsBasedProcessTree processTree =
-          createProcessTree("100", true, procfsRootDir.getAbsolutePath());
+          createProcessTree("100", procfsRootDir.getAbsolutePath());
       // build the process tree.
       processTree.getProcessTree();
-      
+
       // verify cumulative memory
       Assert.assertEquals("Cumulative memory does not match",
                    700000L, processTree.getCumulativeVmem());
@@ -442,12 +440,12 @@ public class TestProcfsBasedProcessTree 
       // write one more process as child of 100.
       String[] newPids = { "500" };
       setupPidDirs(procfsRootDir, newPids);
-      
+
       ProcessStatInfo[] newProcInfos = new ProcessStatInfo[1];
       newProcInfos[0] = new ProcessStatInfo(new String[]
                       {"500", "proc5", "100", "100", "100", "500000", "500"});
       writeStatFiles(procfsRootDir, newPids, newProcInfos);
-      
+
       // check memory includes the new process.
       processTree.getProcessTree();
       Assert.assertEquals("Cumulative vmem does not include new process",
@@ -456,7 +454,7 @@ public class TestProcfsBasedProcessTree 
                         1200L * ProcfsBasedProcessTree.PAGE_SIZE : 0L;
       Assert.assertEquals("Cumulative rssmem does not include new process",
                    cumuRssMem, processTree.getCumulativeRssmem());
-      
+
       // however processes older than 1 iteration will retain the older value
       Assert.assertEquals("Cumulative vmem shouldn't have included new process",
                    700000L, processTree.getCumulativeVmem(1));
@@ -468,7 +466,7 @@ public class TestProcfsBasedProcessTree 
       // one more process
       newPids = new String[]{ "600" };
       setupPidDirs(procfsRootDir, newPids);
-      
+
       newProcInfos = new ProcessStatInfo[1];
       newProcInfos[0] = new ProcessStatInfo(new String[]
                       {"600", "proc6", "100", "100", "100", "600000", "600"});
@@ -476,7 +474,7 @@ public class TestProcfsBasedProcessTree 
 
       // refresh process tree
       processTree.getProcessTree();
-      
+
       // processes older than 2 iterations should be same as before.
       Assert.assertEquals("Cumulative vmem shouldn't have included new processes",
                    700000L, processTree.getCumulativeVmem(2));
@@ -515,15 +513,14 @@ public class TestProcfsBasedProcessTree 
   public void testDestroyProcessTree() throws IOException {
     // test process
     String pid = "100";
-    // create the fake procfs root directory. 
+    // create the fake procfs root directory.
     File procfsRootDir = new File(TEST_ROOT_DIR, "proc");
 
     try {
       setupProcfsRootDir(procfsRootDir);
-      
+
       // crank up the process tree class.
-      ProcfsBasedProcessTree processTree =
-          createProcessTree(pid, true, procfsRootDir.getAbsolutePath());
+      createProcessTree(pid, procfsRootDir.getAbsolutePath());
 
       // Let us not create stat file for pid 100.
       Assert.assertTrue(ProcfsBasedProcessTree.checkPidPgrpidForMatch(
@@ -532,10 +529,10 @@ public class TestProcfsBasedProcessTree 
       FileUtil.fullyDelete(procfsRootDir);
     }
   }
-  
+
   /**
    * Test the correctness of process-tree dump.
-   * 
+   *
    * @throws IOException
    */
   @Test
@@ -578,7 +575,7 @@ public class TestProcfsBasedProcessTree 
       writeCmdLineFiles(procfsRootDir, pids, cmdLines);
 
       ProcfsBasedProcessTree processTree = createProcessTree(
-          "100", true, procfsRootDir.getAbsolutePath());
+          "100", procfsRootDir.getAbsolutePath());
       // build the process tree.
       processTree.getProcessTree();
 
@@ -675,11 +672,11 @@ public class TestProcfsBasedProcessTree 
    * @param procfsRootDir root directory to create.
    * @throws IOException if could not delete the procfs root directory
    */
-  public static void setupProcfsRootDir(File procfsRootDir) 
-                                        throws IOException { 
+  public static void setupProcfsRootDir(File procfsRootDir)
+                                        throws IOException {
     // cleanup any existing process root dir.
     if (procfsRootDir.exists()) {
-      Assert.assertTrue(FileUtil.fullyDelete(procfsRootDir));  
+      Assert.assertTrue(FileUtil.fullyDelete(procfsRootDir));
     }
 
     // create afresh
@@ -692,7 +689,7 @@ public class TestProcfsBasedProcessTree 
    * @param pids the PID directories to create.
    * @throws IOException If PID dirs could not be created
    */
-  public static void setupPidDirs(File procfsRootDir, String[] pids) 
+  public static void setupPidDirs(File procfsRootDir, String[] pids)
                       throws IOException {
     for (String pid : pids) {
       File pidDir = new File(procfsRootDir, pid);
@@ -705,7 +702,7 @@ public class TestProcfsBasedProcessTree 
       }
     }
   }
-  
+
   /**
    * Write stat files under the specified pid directories with data
    * setup in the corresponding ProcessStatInfo objects
@@ -715,7 +712,7 @@ public class TestProcfsBasedProcessTree 
    *              written to the stat files.
    * @throws IOException if stat files could not be written
    */
-  public static void writeStatFiles(File procfsRootDir, String[] pids, 
+  public static void writeStatFiles(File procfsRootDir, String[] pids,
                               ProcessStatInfo[] procs) throws IOException {
     for (int i=0; i<pids.length; i++) {
       File statFile =
@@ -726,7 +723,7 @@ public class TestProcfsBasedProcessTree 
         FileWriter fw = new FileWriter(statFile);
         bw = new BufferedWriter(fw);
         bw.write(procs[i].getStatLine());
-        LOG.info("wrote stat file for " + pids[i] + 
+        LOG.info("wrote stat file for " + pids[i] +
                   " with contents: " + procs[i].getStatLine());
       } finally {
         // not handling exception - will throw an error and fail the test.

Modified: hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java?rev=1391094&r1=1391093&r2=1391094&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java (original)
+++ hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/ContainersMonitorImpl.java Thu Sep 27 16:38:54 2012
@@ -37,7 +37,7 @@ import org.apache.hadoop.yarn.server.nod
 import org.apache.hadoop.yarn.server.nodemanager.Context;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerKillEvent;
 import org.apache.hadoop.yarn.service.AbstractService;
-import org.apache.hadoop.yarn.util.ProcfsBasedProcessTree;
+import org.apache.hadoop.yarn.util.ResourceCalculatorProcessTree;
 import org.apache.hadoop.yarn.util.ResourceCalculatorPlugin;
 
 import com.google.common.base.Preconditions;
@@ -60,6 +60,8 @@ public class ContainersMonitorImpl exten
   private final Dispatcher eventDispatcher;
   private final Context context;
   private ResourceCalculatorPlugin resourceCalculatorPlugin;
+  private Configuration conf;
+  private Class<? extends ResourceCalculatorProcessTree> processTreeClass;
 
   private long maxVmemAllottedForContainers = DISABLED_MEMORY_LIMIT;
   private long maxPmemAllottedForContainers = DISABLED_MEMORY_LIMIT;
@@ -96,6 +98,11 @@ public class ContainersMonitorImpl exten
         ResourceCalculatorPlugin.getResourceCalculatorPlugin(clazz, conf);
     LOG.info(" Using ResourceCalculatorPlugin : "
         + this.resourceCalculatorPlugin);
+    processTreeClass = conf.getClass(YarnConfiguration.NM_CONTAINER_MON_PROCESS_TREE, null,
+            ResourceCalculatorProcessTree.class);
+    this.conf = conf;
+    LOG.info(" Using ResourceCalculatorProcessTree : "
+        + this.processTreeClass);
 
     long totalPhysicalMemoryOnNM = DISABLED_MEMORY_LIMIT;
     if (this.resourceCalculatorPlugin != null) {
@@ -140,7 +147,7 @@ public class ContainersMonitorImpl exten
 
   /**
    * Is the total physical memory check enabled?
-   * 
+   *
    * @return true if total physical memory check is enabled.
    */
   boolean isPhysicalMemoryCheckEnabled() {
@@ -149,7 +156,7 @@ public class ContainersMonitorImpl exten
 
   /**
    * Is the total virtual memory check enabled?
-   * 
+   *
    * @return true if total virtual memory check is enabled.
    */
   boolean isVirtualMemoryCheckEnabled() {
@@ -157,12 +164,16 @@ public class ContainersMonitorImpl exten
   }
 
   private boolean isEnabled() {
-    if (!ProcfsBasedProcessTree.isAvailable()) {
-      LOG.info("ProcessTree implementation is missing on this system. "
-          + this.getClass().getName() + " is disabled.");
-      return false;
+    if (resourceCalculatorPlugin == null) {
+            LOG.info("ResourceCalculatorPlugin is unavailable on this system. "
+                + this.getClass().getName() + " is disabled.");
+            return false;
+    }
+    if (ResourceCalculatorProcessTree.getResourceCalculatorProcessTree("0", processTreeClass, conf) == null) {
+        LOG.info("ResourceCalculatorProcessTree is unavailable on this system. "
+                + this.getClass().getName() + " is disabled.");
+            return false;
     }
-
     if (!(isPhysicalMemoryCheckEnabled() || isVirtualMemoryCheckEnabled())) {
       LOG.info("Neither virutal-memory nor physical-memory monitoring is " +
           "needed. Not running the monitor-thread");
@@ -196,12 +207,12 @@ public class ContainersMonitorImpl exten
   private static class ProcessTreeInfo {
     private ContainerId containerId;
     private String pid;
-    private ProcfsBasedProcessTree pTree;
+    private ResourceCalculatorProcessTree pTree;
     private long vmemLimit;
     private long pmemLimit;
 
     public ProcessTreeInfo(ContainerId containerId, String pid,
-        ProcfsBasedProcessTree pTree, long vmemLimit, long pmemLimit) {
+        ResourceCalculatorProcessTree pTree, long vmemLimit, long pmemLimit) {
       this.containerId = containerId;
       this.pid = pid;
       this.pTree = pTree;
@@ -221,11 +232,11 @@ public class ContainersMonitorImpl exten
       this.pid = pid;
     }
 
-    public ProcfsBasedProcessTree getProcessTree() {
+    public ResourceCalculatorProcessTree getProcessTree() {
       return this.pTree;
     }
 
-    public void setProcessTree(ProcfsBasedProcessTree pTree) {
+    public void setProcessTree(ResourceCalculatorProcessTree pTree) {
       this.pTree = pTree;
     }
 
@@ -245,20 +256,20 @@ public class ContainersMonitorImpl exten
   /**
    * Check whether a container's process tree's current memory usage is over
    * limit.
-   * 
+   *
    * When a java process exec's a program, it could momentarily account for
    * double the size of it's memory, because the JVM does a fork()+exec()
    * which at fork time creates a copy of the parent's memory. If the
    * monitoring thread detects the memory used by the container tree at the
    * same instance, it could assume it is over limit and kill the tree, for no
    * fault of the process itself.
-   * 
+   *
    * We counter this problem by employing a heuristic check: - if a process
    * tree exceeds the memory limit by more than twice, it is killed
    * immediately - if a process tree has processes older than the monitoring
    * interval exceeding the memory limit by even 1 time, it is killed. Else it
    * is given the benefit of doubt to lie around for one more iteration.
-   * 
+   *
    * @param containerId
    *          Container Id for the container tree
    * @param currentMemUsage
@@ -295,7 +306,7 @@ public class ContainersMonitorImpl exten
   }
 
   // method provided just for easy testing purposes
-  boolean isProcessTreeOverLimit(ProcfsBasedProcessTree pTree,
+  boolean isProcessTreeOverLimit(ResourceCalculatorProcessTree pTree,
       String containerId, long limit) {
     long currentMemUsage = pTree.getCumulativeVmem();
     // as processes begin with an age 1, we want to see if there are processes
@@ -370,9 +381,8 @@ public class ContainersMonitorImpl exten
                 LOG.debug("Tracking ProcessTree " + pId
                     + " for the first time");
 
-                ProcfsBasedProcessTree pt =
-                    new ProcfsBasedProcessTree(pId,
-                        ContainerExecutor.isSetsidAvailable);
+                ResourceCalculatorProcessTree pt =
+                    ResourceCalculatorProcessTree.getResourceCalculatorProcessTree(pId, processTreeClass, conf);
                 ptInfo.setPid(pId);
                 ptInfo.setProcessTree(pt);
               }
@@ -385,7 +395,7 @@ public class ContainersMonitorImpl exten
 
             LOG.debug("Constructing ProcessTree for : PID = " + pId
                 + " ContainerId = " + containerId);
-            ProcfsBasedProcessTree pTree = ptInfo.getProcessTree();
+            ResourceCalculatorProcessTree pTree = ptInfo.getProcessTree();
             pTree = pTree.getProcessTree(); // get the updated process-tree
             ptInfo.setProcessTree(pTree); // update ptInfo with proces-tree of
                                           // updated state
@@ -471,7 +481,7 @@ public class ContainersMonitorImpl exten
     private String formatErrorMessage(String memTypeExceeded,
         long currentVmemUsage, long vmemLimit,
         long currentPmemUsage, long pmemLimit,
-        String pId, ContainerId containerId, ProcfsBasedProcessTree pTree) {
+        String pId, ContainerId containerId, ResourceCalculatorProcessTree pTree) {
       return
         String.format("Container [pid=%s,containerID=%s] is running beyond %s memory limits. ",
             pId, containerId, memTypeExceeded) +

Modified: hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/TestContainersMonitor.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/TestContainersMonitor.java?rev=1391094&r1=1391093&r2=1391094&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/TestContainersMonitor.java (original)
+++ hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/TestContainersMonitor.java Thu Sep 27 16:38:54 2012
@@ -82,7 +82,7 @@ public class TestContainersMonitor exten
 
   /**
    * Test to verify the check for whether a process tree is over limit or not.
-   * 
+   *
    * @throws IOException
    *           if there was a problem setting up the fake procfs directories or
    *           files.
@@ -132,7 +132,7 @@ public class TestContainersMonitor exten
       // tree rooted at 100 is over limit immediately, as it is
       // twice over the mem limit.
       ProcfsBasedProcessTree pTree = new ProcfsBasedProcessTree(
-                                          "100", true,
+                                          "100",
                                           procfsRootDir.getAbsolutePath());
       pTree.getProcessTree();
       assertTrue("tree rooted at 100 should be over limit " +
@@ -140,7 +140,7 @@ public class TestContainersMonitor exten
                   test.isProcessTreeOverLimit(pTree, "dummyId", limit));
 
       // the tree rooted at 200 is initially below limit.
-      pTree = new ProcfsBasedProcessTree("200", true,
+      pTree = new ProcfsBasedProcessTree("200",
                                           procfsRootDir.getAbsolutePath());
       pTree.getProcessTree();
       assertFalse("tree rooted at 200 shouldn't be over limit " +
@@ -154,7 +154,7 @@ public class TestContainersMonitor exten
                   test.isProcessTreeOverLimit(pTree, "dummyId", limit));
 
       // the tree rooted at 600 is never over limit.
-      pTree = new ProcfsBasedProcessTree("600", true,
+      pTree = new ProcfsBasedProcessTree("600",
                                             procfsRootDir.getAbsolutePath());
       pTree.getProcessTree();
       assertFalse("tree rooted at 600 should never be over limit.",
@@ -198,7 +198,7 @@ public class TestContainersMonitor exten
         recordFactory.newRecordInstance(ApplicationId.class);
     appId.setClusterTimestamp(0);
     appId.setId(0);
-    ApplicationAttemptId appAttemptId = 
+    ApplicationAttemptId appAttemptId =
         recordFactory.newRecordInstance(ApplicationAttemptId.class);
     appAttemptId.setApplicationId(appId);
     appAttemptId.setAttemptId(1);
@@ -220,7 +220,7 @@ public class TestContainersMonitor exten
     rsrc_alpha.setType(LocalResourceType.FILE);
     rsrc_alpha.setTimestamp(scriptFile.lastModified());
     String destinationFile = "dest_file";
-    Map<String, LocalResource> localResources = 
+    Map<String, LocalResource> localResources =
         new HashMap<String, LocalResource>();
     localResources.put(destinationFile, rsrc_alpha);
     containerLaunchContext.setLocalResources(localResources);