浏览代码

YARN-1775. Enhanced ProcfsBasedProcessTree to optionally add the ability to use smaps for obtaining used memory information. Contributed by Rajesh Balamohan.
svn merge --ignore-ancestry -c 1580087 ../../trunk/


git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2.4@1580089 13f79535-47bb-0310-9956-ffa450edef68

Vinod Kumar Vavilapalli 11 年之前
父节点
当前提交
ea7d07fd20

+ 4 - 0
hadoop-yarn-project/CHANGES.txt

@@ -122,6 +122,10 @@ Release 2.4.0 - UNRELEASED
     YARN-1690. Made DistributedShell send timeline entities+events. (Mayank Bansal
     via zjshen)
 
+    YARN-1775. Enhanced ProcfsBasedProcessTree to optionally add the ability to
+    use smaps for obtaining used memory information. (Rajesh Balamohan via
+    vinodkv)
+
   IMPROVEMENTS
 
     YARN-1007. Enhance History Reader interface for Containers. (Mayank Bansal via

+ 4 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java

@@ -718,6 +718,10 @@ public class YarnConfiguration extends Configuration {
   /** Class that calculates process tree resource utilization.*/
   public static final String NM_CONTAINER_MON_PROCESS_TREE =
     NM_PREFIX + "container-monitor.process-tree.class";
+  public static final String PROCFS_USE_SMAPS_BASED_RSS_ENABLED = NM_PREFIX +
+      ".container-monitor.procfs-tree.smaps-based-rss.enabled";
+  public static final boolean DEFAULT_PROCFS_USE_SMAPS_BASED_RSS_ENABLED =
+      false;
   
   /** Prefix for all node manager disk health checker configs. */
   private static final String NM_DISK_HEALTH_CHECK_PREFIX =

+ 357 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ProcfsBasedProcessTree.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.yarn.util;
 
 import java.io.BufferedReader;
 import java.io.File;
+import java.io.FileInputStream;
 import java.io.FileNotFoundException;
 import java.io.FileReader;
 import java.io.IOException;
@@ -32,13 +33,16 @@ import java.util.Map;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
+import org.apache.commons.io.IOUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.util.Shell.ShellCommandExecutor;
 import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
 
 /**
  * A Proc file-system based ProcessTree. Works only on Linux.
@@ -61,7 +65,46 @@ public class ProcfsBasedProcessTree extends ResourceCalculatorProcessTree {
   public static final String PROCFS_CMDLINE_FILE = "cmdline";
   public static final long PAGE_SIZE;
   public static final long JIFFY_LENGTH_IN_MILLIS; // in millisecond
-  
+
+  enum MemInfo {
+    SIZE("Size"), RSS("Rss"), PSS("Pss"), SHARED_CLEAN("Shared_Clean"),
+    SHARED_DIRTY("Shared_Dirty"), PRIVATE_CLEAN("Private_Clean"),
+    PRIVATE_DIRTY("Private_Dirty"), REFERENCED("Referenced"), ANONYMOUS(
+        "Anonymous"), ANON_HUGE_PAGES("AnonHugePages"), SWAP("swap"),
+    KERNEL_PAGE_SIZE("kernelPageSize"), MMU_PAGE_SIZE("mmuPageSize"), INVALID(
+        "invalid");
+
+    private String name;
+
+    private MemInfo(String name) {
+      this.name = name;
+    }
+
+    public static MemInfo getMemInfoByName(String name) {
+      for (MemInfo info : MemInfo.values()) {
+        if (info.name.trim().equalsIgnoreCase(name.trim())) {
+          return info;
+        }
+      }
+      return INVALID;
+    }
+  }
+
+  public static final String SMAPS = "smaps";
+  public static final int KB_TO_BYTES = 1024;
+  private static final String KB = "kB";
+  private static final String READ_ONLY_WITH_SHARED_PERMISSION = "r--s";
+  private static final String READ_EXECUTE_WITH_SHARED_PERMISSION = "r-xs";
+  private static final Pattern ADDRESS_PATTERN = Pattern
+    .compile("([[a-f]|(0-9)]*)-([[a-f]|(0-9)]*)(\\s)*([rxwps\\-]*)");
+  private static final Pattern MEM_INFO_PATTERN = Pattern
+    .compile("(^[A-Z].*):[\\s ]*(.*)");
+
+  private boolean smapsEnabled;
+
+  protected Map<String, ProcessTreeSmapMemInfo> processSMAPTree =
+      new HashMap<String, ProcessTreeSmapMemInfo>();
+
   static {
     long jiffiesPerSecond = -1;
     long pageSize = -1;
@@ -103,6 +146,16 @@ public class ProcfsBasedProcessTree extends ResourceCalculatorProcessTree {
     this(pid, PROCFS);
   }
 
+  @Override
+  public void setConf(Configuration conf) {
+    super.setConf(conf);
+    if (conf != null) {
+      smapsEnabled =
+          conf.getBoolean(YarnConfiguration.PROCFS_USE_SMAPS_BASED_RSS_ENABLED,
+            YarnConfiguration.DEFAULT_PROCFS_USE_SMAPS_BASED_RSS_ENABLED);
+    }
+  }
+
   /**
    * Build a new process tree rooted at the pid.
    *
@@ -210,6 +263,18 @@ public class ProcfsBasedProcessTree extends ResourceCalculatorProcessTree {
         // Log.debug the ProcfsBasedProcessTree
         LOG.debug(this.toString());
       }
+      if (smapsEnabled) {
+        //Update smaps info
+        processSMAPTree.clear();
+        for (ProcessInfo p : processTree.values()) {
+          if (p != null) {
+            // Get information for each process
+            ProcessTreeSmapMemInfo memInfo = new ProcessTreeSmapMemInfo(p.getPid());
+            constructProcessSMAPInfo(memInfo, procfsDir);
+            processSMAPTree.put(p.getPid(), memInfo);
+          }
+        }
+      }
     }
   }
 
@@ -300,6 +365,9 @@ public class ProcfsBasedProcessTree extends ResourceCalculatorProcessTree {
     if (PAGE_SIZE < 0) {
       return 0;
     }
+    if (smapsEnabled) {
+      return getSmapBasedCumulativeRssmem(olderThanAge);
+    }
     long totalPages = 0;
     for (ProcessInfo p : processTree.values()) {
       if ((p != null) && (p.getAge() > olderThanAge)) {
@@ -309,6 +377,53 @@ public class ProcfsBasedProcessTree extends ResourceCalculatorProcessTree {
     return totalPages * PAGE_SIZE; // convert # pages to byte
   }
 
+  /**
+   * 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. RSS is
+   * calculated based on SMAP information. Skip mappings with "r--s", "r-xs"
+   * permissions to get real RSS usage of the process.
+   *
+   * @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
+   */
+  private long getSmapBasedCumulativeRssmem(int olderThanAge) {
+    long total = 0;
+    for (ProcessInfo p : processTree.values()) {
+      if ((p != null) && (p.getAge() > olderThanAge)) {
+        ProcessTreeSmapMemInfo procMemInfo = processSMAPTree.get(p.getPid());
+        if (procMemInfo != null) {
+          for (ProcessSmapMemoryInfo info : procMemInfo.getMemoryInfoList()) {
+            // Do not account for r--s or r-xs mappings
+            if (info.getPermission().trim()
+              .equalsIgnoreCase(READ_ONLY_WITH_SHARED_PERMISSION)
+                || info.getPermission().trim()
+                  .equalsIgnoreCase(READ_EXECUTE_WITH_SHARED_PERMISSION)) {
+              continue;
+            }
+            total +=
+                Math.min(info.sharedDirty, info.pss) + info.privateDirty
+                    + info.privateClean;
+            if (LOG.isDebugEnabled()) {
+              LOG.debug(" total(" + olderThanAge + "): PID : " + p.getPid()
+                  + ", SharedDirty : " + info.sharedDirty + ", PSS : "
+                  + info.pss + ", Private_Dirty : " + info.privateDirty
+                  + ", Private_Clean : " + info.privateClean + ", total : "
+                  + (total * KB_TO_BYTES));
+            }
+          }
+        }
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(procMemInfo.toString());
+        }
+      }
+    }
+    total = (total * KB_TO_BYTES); // convert to bytes
+    LOG.info("SmapBasedCumulativeRssmem (bytes) : " + total);
+    return total; // size
+  }
+
   /**
    * Get the CPU time in millisecond used by all the processes in the
    * process-tree since the process-tree created
@@ -601,4 +716,245 @@ public class ProcfsBasedProcessTree extends ResourceCalculatorProcessTree {
       return ret;
     }
   }
+
+  /**
+   * Update memory related information
+   *
+   * @param pInfo
+   * @param procfsDir
+   */
+  private static void constructProcessSMAPInfo(ProcessTreeSmapMemInfo pInfo,
+      String procfsDir) {
+    BufferedReader in = null;
+    FileReader fReader = null;
+    try {
+      File pidDir = new File(procfsDir, pInfo.getPid());
+      File file = new File(pidDir, SMAPS);
+      if (!file.exists()) {
+        return;
+      }
+      fReader = new FileReader(file);
+      in = new BufferedReader(fReader);
+      ProcessSmapMemoryInfo memoryMappingInfo = null;
+      List<String> lines = IOUtils.readLines(new FileInputStream(file));
+      for (String line : lines) {
+        line = line.trim();
+        try {
+          Matcher address = ADDRESS_PATTERN.matcher(line);
+          if (address.find()) {
+            memoryMappingInfo = new ProcessSmapMemoryInfo(line);
+            memoryMappingInfo.setPermission(address.group(4));
+            pInfo.getMemoryInfoList().add(memoryMappingInfo);
+            continue;
+          }
+          Matcher memInfo = MEM_INFO_PATTERN.matcher(line);
+          if (memInfo.find()) {
+            String key = memInfo.group(1).trim();
+            String value = memInfo.group(2).replace(KB, "").trim();
+            if (LOG.isDebugEnabled()) {
+              LOG.debug("MemInfo : " + key + " : Value  : " + value);
+            }
+            memoryMappingInfo.setMemInfo(key, value);
+          }
+        } catch (Throwable t) {
+          LOG
+            .warn("Error parsing smaps line : " + line + "; " + t.getMessage());
+        }
+      }
+    } catch (FileNotFoundException f) {
+      LOG.error(f.getMessage());
+    } catch (IOException e) {
+      LOG.error(e.getMessage());
+    } catch (Throwable t) {
+      LOG.error(t.getMessage());
+    } finally {
+      IOUtils.closeQuietly(in);
+    }
+  }
+
+  /**
+   * Placeholder for process's SMAPS information
+   */
+  static class ProcessTreeSmapMemInfo {
+    private String pid;
+    private List<ProcessSmapMemoryInfo> memoryInfoList;
+
+    public ProcessTreeSmapMemInfo(String pid) {
+      this.pid = pid;
+      this.memoryInfoList = new LinkedList<ProcessSmapMemoryInfo>();
+    }
+
+    public List<ProcessSmapMemoryInfo> getMemoryInfoList() {
+      return memoryInfoList;
+    }
+
+    public String getPid() {
+      return pid;
+    }
+
+    public String toString() {
+      StringBuilder sb = new StringBuilder();
+      for (ProcessSmapMemoryInfo info : memoryInfoList) {
+        sb.append("\n");
+        sb.append(info.toString());
+      }
+      return sb.toString();
+    }
+  }
+
+  /**
+   * <pre>
+   * Private Pages : Pages that were mapped only by the process
+   * Shared Pages : Pages that were shared with other processes
+   *
+   * Clean Pages : Pages that have not been modified since they were mapped
+   * Dirty Pages : Pages that have been modified since they were mapped
+   *
+   * Private RSS = Private Clean Pages + Private Dirty Pages
+   * Shared RSS = Shared Clean Pages + Shared Dirty Pages
+   * RSS = Private RSS + Shared RSS
+   * PSS = The count of all pages mapped uniquely by the process, 
+   *  plus a fraction of each shared page, said fraction to be 
+   *  proportional to the number of processes which have mapped the page.
+   * 
+   * </pre>
+   */
+  static class ProcessSmapMemoryInfo {
+    private int size;
+    private int rss;
+    private int pss;
+    private int sharedClean;
+    private int sharedDirty;
+    private int privateClean;
+    private int privateDirty;
+    private int referenced;
+    private String regionName;
+    private String permission;
+
+    public ProcessSmapMemoryInfo(String name) {
+      this.regionName = name;
+    }
+
+    public String getName() {
+      return regionName;
+    }
+
+    public void setPermission(String permission) {
+      this.permission = permission;
+    }
+
+    public String getPermission() {
+      return permission;
+    }
+
+    public int getSize() {
+      return size;
+    }
+
+    public int getRss() {
+      return rss;
+    }
+
+    public int getPss() {
+      return pss;
+    }
+
+    public int getSharedClean() {
+      return sharedClean;
+    }
+
+    public int getSharedDirty() {
+      return sharedDirty;
+    }
+
+    public int getPrivateClean() {
+      return privateClean;
+    }
+
+    public int getPrivateDirty() {
+      return privateDirty;
+    }
+
+    public int getReferenced() {
+      return referenced;
+    }
+
+    public void setMemInfo(String key, String value) {
+      MemInfo info = MemInfo.getMemInfoByName(key);
+      int val = 0;
+      try {
+        val = Integer.parseInt(value.trim());
+      } catch (NumberFormatException ne) {
+        LOG.error("Error in parsing : " + info + " : value" + value.trim());
+        return;
+      }
+      if (info == null) {
+        return;
+      }
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("setMemInfo : memInfo : " + info);
+      }
+      switch (info) {
+      case SIZE:
+        size = val;
+        break;
+      case RSS:
+        rss = val;
+        break;
+      case PSS:
+        pss = val;
+        break;
+      case SHARED_CLEAN:
+        sharedClean = val;
+        break;
+      case SHARED_DIRTY:
+        sharedDirty = val;
+        break;
+      case PRIVATE_CLEAN:
+        privateClean = val;
+        break;
+      case PRIVATE_DIRTY:
+        privateDirty = val;
+        break;
+      case REFERENCED:
+        referenced = val;
+        break;
+      default:
+        break;
+      }
+    }
+
+    public String toString() {
+      StringBuilder sb = new StringBuilder();
+      sb.append("\t").append(this.getName()).append("\n");
+      sb.append("\t").append(MemInfo.SIZE.name + ":" + this.getSize())
+        .append(" kB\n");
+      sb.append("\t").append(MemInfo.PSS.name + ":" + this.getPss())
+        .append(" kB\n");
+      sb.append("\t").append(MemInfo.RSS.name + ":" + this.getRss())
+        .append(" kB\n");
+      sb.append("\t")
+        .append(MemInfo.SHARED_CLEAN.name + ":" + this.getSharedClean())
+        .append(" kB\n");
+      sb.append("\t")
+        .append(MemInfo.SHARED_DIRTY.name + ":" + this.getSharedDirty())
+        .append(" kB\n");
+      sb.append("\t")
+        .append(MemInfo.PRIVATE_CLEAN.name + ":" + this.getPrivateClean())
+        .append(" kB\n");
+      sb.append("\t")
+        .append(MemInfo.PRIVATE_DIRTY.name + ":" + this.getPrivateDirty())
+        .append(" kB\n");
+      sb.append("\t")
+        .append(MemInfo.REFERENCED.name + ":" + this.getReferenced())
+        .append(" kB\n");
+      sb.append("\t")
+        .append(MemInfo.PRIVATE_DIRTY.name + ":" + this.getPrivateDirty())
+        .append(" kB\n");
+      sb.append("\t")
+        .append(MemInfo.PRIVATE_DIRTY.name + ":" + this.getPrivateDirty())
+        .append(" kB\n");
+      return sb.toString();
+    }
+  }
 }

+ 13 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml

@@ -1206,4 +1206,17 @@
     <name>yarn.client.application-client-protocol.poll-interval-ms</name>
     <value>200</value>
   </property>
+
+  <property>
+    <description>RSS usage of a process computed via 
+    /proc/pid/stat is not very accurate as it includes shared pages of a
+    process. /proc/pid/smaps provides useful information like
+    Private_Dirty, Private_Clean, Shared_Dirty, Shared_Clean which can be used
+    for computing more accurate RSS. When this flag is enabled, RSS is computed
+    as Min(Shared_Dirty, Pss) + Private_Clean + Private_Dirty. It excludes
+    read-only shared mappings in RSS computation.  
+    </description>
+    <name>yarn.nodemanager.container-monitor.procfs-tree.smaps-based-rss.enabled</name>
+    <value>false</value>
+  </property>
 </configuration>

+ 402 - 215
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestProcfsBasedProcessTree.java

@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.yarn.util;
 
+import static org.apache.hadoop.yarn.util.ProcfsBasedProcessTree.KB_TO_BYTES;
 import static org.junit.Assert.fail;
 
 import java.io.BufferedReader;
@@ -27,6 +28,7 @@ import java.io.FileNotFoundException;
 import java.io.FileReader;
 import java.io.FileWriter;
 import java.io.IOException;
+import java.util.List;
 import java.util.Random;
 import java.util.Vector;
 import java.util.regex.Matcher;
@@ -34,14 +36,18 @@ import java.util.regex.Pattern;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.util.Shell.ExitCodeException;
 import org.apache.hadoop.util.Shell.ShellCommandExecutor;
-import org.apache.hadoop.yarn.util.ProcfsBasedProcessTree;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.util.ProcfsBasedProcessTree.MemInfo;
+import org.apache.hadoop.yarn.util.ProcfsBasedProcessTree.ProcessSmapMemoryInfo;
+import org.apache.hadoop.yarn.util.ProcfsBasedProcessTree.ProcessTreeSmapMemInfo;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
@@ -52,9 +58,9 @@ import org.junit.Test;
 public class TestProcfsBasedProcessTree {
 
   private static final Log LOG = LogFactory
-      .getLog(TestProcfsBasedProcessTree.class);
+    .getLog(TestProcfsBasedProcessTree.class);
   protected static File TEST_ROOT_DIR = new File("target",
-      TestProcfsBasedProcessTree.class.getName() + "-localDir");
+    TestProcfsBasedProcessTree.class.getName() + "-localDir");
 
   private ShellCommandExecutor shexec = null;
   private String pidFile, lowestDescendant;
@@ -66,19 +72,19 @@ public class TestProcfsBasedProcessTree {
     public void run() {
       try {
         Vector<String> args = new Vector<String>();
-        if(isSetsidAvailable()) {
+        if (isSetsidAvailable()) {
           args.add("setsid");
         }
         args.add("bash");
         args.add("-c");
-        args.add(" echo $$ > " + pidFile + "; sh " +
-                          shellScript + " " + N + ";") ;
+        args.add(" echo $$ > " + pidFile + "; sh " + shellScript + " " + N
+            + ";");
         shexec = new ShellCommandExecutor(args.toArray(new String[0]));
         shexec.execute();
       } catch (ExitCodeException ee) {
-        LOG.info("Shell Command exit with a non-zero exit code. This is" +
-                 " expected as we are killing the subprocesses of the" +
-                 " task intentionally. " + ee);
+        LOG.info("Shell Command exit with a non-zero exit code. This is"
+            + " expected as we are killing the subprocesses of the"
+            + " task intentionally. " + ee);
       } catch (IOException ioe) {
         LOG.info("Error executing shell command " + ioe);
       } finally {
@@ -104,15 +110,15 @@ public class TestProcfsBasedProcessTree {
   @Before
   public void setup() throws IOException {
     FileContext.getLocalFSFileContext().delete(
-        new Path(TEST_ROOT_DIR.getAbsolutePath()), true);
+      new Path(TEST_ROOT_DIR.getAbsolutePath()), true);
   }
 
-  @Test (timeout = 30000)
+  @Test(timeout = 30000)
   public void testProcessTree() throws Exception {
 
     if (!Shell.LINUX) {
       System.out
-          .println("ProcfsBasedProcessTree is not available on this system. Not testing");
+        .println("ProcfsBasedProcessTree is not available on this system. Not testing");
       return;
 
     }
@@ -139,24 +145,16 @@ public class TestProcfsBasedProcessTree {
     tempFile.deleteOnExit();
     pidFile = TEST_ROOT_DIR + File.separator + tempFile.getName();
 
-    lowestDescendant = TEST_ROOT_DIR + File.separator + "lowestDescendantPidFile";
+    lowestDescendant =
+        TEST_ROOT_DIR + File.separator + "lowestDescendantPidFile";
 
     // write to shell-script
     try {
       FileWriter fWriter = new FileWriter(shellScript);
-      fWriter.write(
-          "# rogue task\n" +
-          "sleep 1\n" +
-          "echo hello\n" +
-          "if [ $1 -ne 0 ]\n" +
-          "then\n" +
-          " sh " + shellScript + " $(($1-1))\n" +
-          "else\n" +
-          " echo $$ > " + lowestDescendant + "\n" +
-          " while true\n do\n" +
-          "  sleep 5\n" +
-          " done\n" +
-          "fi");
+      fWriter.write("# rogue task\n" + "sleep 1\n" + "echo hello\n"
+          + "if [ $1 -ne 0 ]\n" + "then\n" + " sh " + shellScript
+          + " $(($1-1))\n" + "else\n" + " echo $$ > " + lowestDescendant + "\n"
+          + " while true\n do\n" + "  sleep 5\n" + " done\n" + "fi");
       fWriter.close();
     } catch (IOException ioe) {
       LOG.info("Error: " + ioe);
@@ -172,7 +170,7 @@ public class TestProcfsBasedProcessTree {
     LOG.info("ProcessTree: " + p.toString());
 
     File leaf = new File(lowestDescendant);
-    //wait till lowest descendant process of Rougue Task starts execution
+    // wait till lowest descendant process of Rougue Task starts execution
     while (!leaf.exists()) {
       try {
         Thread.sleep(500);
@@ -208,16 +206,17 @@ public class TestProcfsBasedProcessTree {
 
     LOG.info("Process-tree dump follows: \n" + processTreeDump);
     Assert.assertTrue("Process-tree dump doesn't start with a proper header",
-        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"));
+      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--) {
-      String cmdLineDump = "\\|- [0-9]+ [0-9]+ [0-9]+ [0-9]+ \\(sh\\)" +
-          " [0-9]+ [0-9]+ [0-9]+ [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);
       Matcher mat = pat.matcher(processTreeDump);
-      Assert.assertTrue("Process-tree dump doesn't contain the cmdLineDump of " + i
-          + "th process!", mat.find());
+      Assert.assertTrue("Process-tree dump doesn't contain the cmdLineDump of "
+          + i + "th process!", mat.find());
     }
 
     // Not able to join thread sometimes when forking with large N.
@@ -231,9 +230,9 @@ public class TestProcfsBasedProcessTree {
     // ProcessTree is gone now. Any further calls should be sane.
     p.updateProcessTree();
     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
-        .getCumulativeVmem() == 0);
+    Assert.assertTrue(
+      "Cumulative vmem for the gone-process is " + p.getCumulativeVmem()
+          + " . It should be zero.", p.getCumulativeVmem() == 0);
     Assert.assertTrue(p.toString().equals("[ ]"));
   }
 
@@ -241,7 +240,8 @@ public class TestProcfsBasedProcessTree {
     return new ProcfsBasedProcessTree(pid);
   }
 
-  protected ProcfsBasedProcessTree createProcessTree(String pid, String procfsRootDir) {
+  protected ProcfsBasedProcessTree createProcessTree(String pid,
+      String procfsRootDir) {
     return new ProcfsBasedProcessTree(pid, procfsRootDir);
   }
 
@@ -251,7 +251,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
@@ -328,23 +328,67 @@ public class TestProcfsBasedProcessTree {
     // 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",
-                      pid, name, ppid, pgrpId, session,
-                      utime, stime, vmem, rssmemPage);
+      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", pid, name, ppid, pgrpId, session, utime, stime, vmem,
+        rssmemPage);
+    }
+  }
+
+  public ProcessSmapMemoryInfo constructMemoryMappingInfo(String address,
+      String[] entries) {
+    ProcessSmapMemoryInfo info = new ProcessSmapMemoryInfo(address);
+    info.setMemInfo(MemInfo.SIZE.name(), entries[0]);
+    info.setMemInfo(MemInfo.RSS.name(), entries[1]);
+    info.setMemInfo(MemInfo.PSS.name(), entries[2]);
+    info.setMemInfo(MemInfo.SHARED_CLEAN.name(), entries[3]);
+    info.setMemInfo(MemInfo.SHARED_DIRTY.name(), entries[4]);
+    info.setMemInfo(MemInfo.PRIVATE_CLEAN.name(), entries[5]);
+    info.setMemInfo(MemInfo.PRIVATE_DIRTY.name(), entries[6]);
+    info.setMemInfo(MemInfo.REFERENCED.name(), entries[7]);
+    info.setMemInfo(MemInfo.ANONYMOUS.name(), entries[8]);
+    info.setMemInfo(MemInfo.ANON_HUGE_PAGES.name(), entries[9]);
+    info.setMemInfo(MemInfo.SWAP.name(), entries[10]);
+    info.setMemInfo(MemInfo.KERNEL_PAGE_SIZE.name(), entries[11]);
+    info.setMemInfo(MemInfo.MMU_PAGE_SIZE.name(), entries[12]);
+    return info;
+  }
+
+  public void createMemoryMappingInfo(ProcessTreeSmapMemInfo[] procMemInfo) {
+    for (int i = 0; i < procMemInfo.length; i++) {
+      // Construct 4 memory mappings per process.
+      // As per min(Shared_Dirty, Pss) + Private_Clean + Private_Dirty
+      // and not including r--s, r-xs, we should get 100 KB per process
+      List<ProcessSmapMemoryInfo> memoryMappingList =
+          procMemInfo[i].getMemoryInfoList();
+      memoryMappingList.add(constructMemoryMappingInfo(
+        "7f56c177c000-7f56c177d000 "
+            + "rw-p 00010000 08:02 40371558                   "
+            + "/grid/0/jdk1.7.0_25/jre/lib/amd64/libnio.so",
+        new String[] { "4", "4", "25", "4", "25", "15", "10", "4", "0", "0",
+            "0", "4", "4" }));
+      memoryMappingList.add(constructMemoryMappingInfo(
+        "7fb09382e000-7fb09382f000 r--s 00003000 " + "08:02 25953545",
+        new String[] { "4", "4", "25", "4", "0", "15", "10", "4", "0", "0",
+            "0", "4", "4" }));
+      memoryMappingList.add(constructMemoryMappingInfo(
+        "7e8790000-7e8b80000 r-xs 00000000 00:00 0", new String[] { "4", "4",
+            "25", "4", "0", "15", "10", "4", "0", "0", "0", "4", "4" }));
+      memoryMappingList.add(constructMemoryMappingInfo(
+        "7da677000-7e0dcf000 rw-p 00000000 00:00 0", new String[] { "4", "4",
+            "25", "4", "50", "15", "10", "4", "0", "0", "0", "4", "4" }));
     }
   }
 
   /**
-   * A basic test that creates a few process directories and writes
-   * stat files. Verifies that the cpu time and memory is correctly
-   * computed.
-   * @throws IOException if there was a problem setting up the
-   *                      fake procfs directories or files.
+   * A basic test that creates a few process directories and writes stat files.
+   * Verifies that the cpu time and memory is correctly computed.
+   *
+   * @throws IOException
+   *           if there was a problem setting up the fake procfs directories or
+   *           files.
    */
-  @Test (timeout = 30000)
+  @Test(timeout = 30000)
   public void testCpuAndMemoryForProcessTree() throws IOException {
 
     // test processes
@@ -359,67 +403,108 @@ public class TestProcfsBasedProcessTree {
       // 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[]
-          {"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", "1", "400", "400", "400000", "400", "4000", "800"});
-
-      writeStatFiles(procfsRootDir, pids, procInfos);
+      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", "1", "400", "400",
+              "400000", "400", "4000", "800" });
+
+      ProcessTreeSmapMemInfo[] memInfo = new ProcessTreeSmapMemInfo[4];
+      memInfo[0] = new ProcessTreeSmapMemInfo("100");
+      memInfo[1] = new ProcessTreeSmapMemInfo("200");
+      memInfo[2] = new ProcessTreeSmapMemInfo("300");
+      memInfo[3] = new ProcessTreeSmapMemInfo("400");
+      createMemoryMappingInfo(memInfo);
+      writeStatFiles(procfsRootDir, pids, procInfos, memInfo);
 
       // crank up the process tree class.
+      Configuration conf = new Configuration();
       ProcfsBasedProcessTree processTree =
           createProcessTree("100", procfsRootDir.getAbsolutePath());
+      processTree.setConf(conf);
       // build the process tree.
       processTree.updateProcessTree();
 
       // verify cumulative memory
       Assert.assertEquals("Cumulative virtual memory does not match", 600000L,
-                   processTree.getCumulativeVmem());
+        processTree.getCumulativeVmem());
 
       // verify rss memory
-      long cumuRssMem = ProcfsBasedProcessTree.PAGE_SIZE > 0 ?
-                        600L * ProcfsBasedProcessTree.PAGE_SIZE : 0L;
-      Assert.assertEquals("Cumulative rss memory does not match",
-                   cumuRssMem, processTree.getCumulativeRssmem());
+      long cumuRssMem =
+          ProcfsBasedProcessTree.PAGE_SIZE > 0
+              ? 600L * ProcfsBasedProcessTree.PAGE_SIZE : 0L;
+      Assert.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;
-      Assert.assertEquals("Cumulative cpu time does not match",
-                   cumuCpuTime, processTree.getCumulativeCpuTime());
+      long cumuCpuTime =
+          ProcfsBasedProcessTree.JIFFY_LENGTH_IN_MILLIS > 0
+              ? 7200L * ProcfsBasedProcessTree.JIFFY_LENGTH_IN_MILLIS : 0L;
+      Assert.assertEquals("Cumulative cpu time does not match", cumuCpuTime,
+        processTree.getCumulativeCpuTime());
+      // Check by enabling smaps
+      setSmapsInProceTree(processTree, true);
+      // RSS=Min(shared_dirty,PSS)+PrivateClean+PrivateDirty (exclude r-xs,
+      // r--s)
+      Assert.assertEquals("Cumulative rss memory does not match",
+        (100 * KB_TO_BYTES * 3), processTree.getCumulativeRssmem());
 
       // 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);
+      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, memInfo);
 
       // build the process tree.
       processTree.updateProcessTree();
 
       // verify cumulative cpu time again
-      cumuCpuTime = ProcfsBasedProcessTree.JIFFY_LENGTH_IN_MILLIS > 0 ?
-             9400L * ProcfsBasedProcessTree.JIFFY_LENGTH_IN_MILLIS : 0L;
-      Assert.assertEquals("Cumulative cpu time does not match",
-                   cumuCpuTime, processTree.getCumulativeCpuTime());
+      cumuCpuTime =
+          ProcfsBasedProcessTree.JIFFY_LENGTH_IN_MILLIS > 0
+              ? 9400L * ProcfsBasedProcessTree.JIFFY_LENGTH_IN_MILLIS : 0L;
+      Assert.assertEquals("Cumulative cpu time does not match", cumuCpuTime,
+        processTree.getCumulativeCpuTime());
     } finally {
       FileUtil.fullyDelete(procfsRootDir);
     }
   }
 
+  private void setSmapsInProceTree(ProcfsBasedProcessTree processTree,
+      boolean enableFlag) {
+    Configuration conf = processTree.getConf();
+    if (conf == null) {
+      conf = new Configuration();
+    }
+    conf.setBoolean(YarnConfiguration.PROCFS_USE_SMAPS_BASED_RSS_ENABLED, enableFlag);
+    processTree.setConf(conf);
+    processTree.updateProcessTree();
+  }
+
   /**
-   * Tests that cumulative memory is computed only for
-   * processes older than a given age.
-   * @throws IOException if there was a problem setting up the
-   *                      fake procfs directories or files.
+   * Tests that cumulative memory is computed only for processes older than a
+   * given age.
+   *
+   * @throws IOException
+   *           if there was a problem setting up the fake procfs directories or
+   *           files.
    */
-  @Test (timeout = 30000)
+  @Test(timeout = 30000)
   public void testMemForOlderProcesses() throws IOException {
+    testMemForOlderProcesses(false);
+    testMemForOlderProcesses(true);
+  }
+
+  private void testMemForOlderProcesses(boolean smapEnabled) throws IOException {
     // initial list of processes
     String[] pids = { "100", "200", "300", "400" };
     // create the fake procfs root directory.
@@ -432,87 +517,141 @@ public class TestProcfsBasedProcessTree {
       // 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[]
-                        {"100", "proc1", "1", "100", "100", "100000", "100"});
-      procInfos[1] = new ProcessStatInfo(new String[]
-                        {"200", "proc2", "100", "100", "100", "200000", "200"});
-      procInfos[2] = new ProcessStatInfo(new String[]
-                        {"300", "proc3", "1", "300", "300", "300000", "300"});
-      procInfos[3] = new ProcessStatInfo(new String[]
-                        {"400", "proc4", "100", "100", "100", "400000", "400"});
-
-      writeStatFiles(procfsRootDir, pids, procInfos);
+      procInfos[0] =
+          new ProcessStatInfo(new String[] { "100", "proc1", "1", "100", "100",
+              "100000", "100" });
+      procInfos[1] =
+          new ProcessStatInfo(new String[] { "200", "proc2", "100", "100",
+              "100", "200000", "200" });
+      procInfos[2] =
+          new ProcessStatInfo(new String[] { "300", "proc3", "1", "300", "300",
+              "300000", "300" });
+      procInfos[3] =
+          new ProcessStatInfo(new String[] { "400", "proc4", "100", "100",
+              "100", "400000", "400" });
+      // write smap information invariably for testing
+      ProcessTreeSmapMemInfo[] memInfo = new ProcessTreeSmapMemInfo[4];
+      memInfo[0] = new ProcessTreeSmapMemInfo("100");
+      memInfo[1] = new ProcessTreeSmapMemInfo("200");
+      memInfo[2] = new ProcessTreeSmapMemInfo("300");
+      memInfo[3] = new ProcessTreeSmapMemInfo("400");
+      createMemoryMappingInfo(memInfo);
+      writeStatFiles(procfsRootDir, pids, procInfos, memInfo);
 
       // crank up the process tree class.
       ProcfsBasedProcessTree processTree =
           createProcessTree("100", procfsRootDir.getAbsolutePath());
-      // build the process tree.
-      processTree.updateProcessTree();
+      setSmapsInProceTree(processTree, smapEnabled);
 
       // verify cumulative memory
-      Assert.assertEquals("Cumulative memory does not match",
-                   700000L, processTree.getCumulativeVmem());
-
+      Assert.assertEquals("Cumulative memory does not match", 700000L,
+        processTree.getCumulativeVmem());
       // 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);
+      newProcInfos[0] =
+          new ProcessStatInfo(new String[] { "500", "proc5", "100", "100",
+              "100", "500000", "500" });
+      ProcessTreeSmapMemInfo[] newMemInfos = new ProcessTreeSmapMemInfo[1];
+      newMemInfos[0] = new ProcessTreeSmapMemInfo("500");
+      createMemoryMappingInfo(newMemInfos);
+      writeStatFiles(procfsRootDir, newPids, newProcInfos, newMemInfos);
 
       // check memory includes the new process.
       processTree.updateProcessTree();
       Assert.assertEquals("Cumulative vmem does not include new process",
-                   1200000L, processTree.getCumulativeVmem());
-      long cumuRssMem = ProcfsBasedProcessTree.PAGE_SIZE > 0 ?
-                        1200L * ProcfsBasedProcessTree.PAGE_SIZE : 0L;
-      Assert.assertEquals("Cumulative rssmem does not include new process",
-                   cumuRssMem, processTree.getCumulativeRssmem());
+        1200000L, processTree.getCumulativeVmem());
+      if (!smapEnabled) {
+        long cumuRssMem =
+            ProcfsBasedProcessTree.PAGE_SIZE > 0
+                ? 1200L * ProcfsBasedProcessTree.PAGE_SIZE : 0L;
+        Assert.assertEquals("Cumulative rssmem does not include new process",
+          cumuRssMem, processTree.getCumulativeRssmem());
+      } else {
+        Assert.assertEquals("Cumulative rssmem does not include new process",
+          100 * KB_TO_BYTES * 4, 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));
-      cumuRssMem = ProcfsBasedProcessTree.PAGE_SIZE > 0 ?
-                   700L * ProcfsBasedProcessTree.PAGE_SIZE : 0L;
-      Assert.assertEquals("Cumulative rssmem shouldn't have included new process",
-                   cumuRssMem, processTree.getCumulativeRssmem(1));
+      Assert.assertEquals(
+        "Cumulative vmem shouldn't have included new process", 700000L,
+        processTree.getCumulativeVmem(1));
+      if (!smapEnabled) {
+        long cumuRssMem =
+            ProcfsBasedProcessTree.PAGE_SIZE > 0
+                ? 700L * ProcfsBasedProcessTree.PAGE_SIZE : 0L;
+        Assert.assertEquals(
+          "Cumulative rssmem shouldn't have included new process", cumuRssMem,
+          processTree.getCumulativeRssmem(1));
+      } else {
+        Assert.assertEquals(
+          "Cumulative rssmem shouldn't have included new process",
+          100 * KB_TO_BYTES * 3, processTree.getCumulativeRssmem(1));
+      }
 
       // one more process
-      newPids = new String[]{ "600" };
+      newPids = new String[] { "600" };
       setupPidDirs(procfsRootDir, newPids);
 
       newProcInfos = new ProcessStatInfo[1];
-      newProcInfos[0] = new ProcessStatInfo(new String[]
-                      {"600", "proc6", "100", "100", "100", "600000", "600"});
-      writeStatFiles(procfsRootDir, newPids, newProcInfos);
+      newProcInfos[0] =
+          new ProcessStatInfo(new String[] { "600", "proc6", "100", "100",
+              "100", "600000", "600" });
+      newMemInfos = new ProcessTreeSmapMemInfo[1];
+      newMemInfos[0] = new ProcessTreeSmapMemInfo("600");
+      createMemoryMappingInfo(newMemInfos);
+      writeStatFiles(procfsRootDir, newPids, newProcInfos, newMemInfos);
 
       // refresh process tree
       processTree.updateProcessTree();
 
       // processes older than 2 iterations should be same as before.
-      Assert.assertEquals("Cumulative vmem shouldn't have included new processes",
-                   700000L, processTree.getCumulativeVmem(2));
-      cumuRssMem = ProcfsBasedProcessTree.PAGE_SIZE > 0 ?
-                   700L * ProcfsBasedProcessTree.PAGE_SIZE : 0L;
-      Assert.assertEquals("Cumulative rssmem shouldn't have included new processes",
-                   cumuRssMem, processTree.getCumulativeRssmem(2));
+      Assert.assertEquals(
+        "Cumulative vmem shouldn't have included new processes", 700000L,
+        processTree.getCumulativeVmem(2));
+      if (!smapEnabled) {
+        long cumuRssMem =
+            ProcfsBasedProcessTree.PAGE_SIZE > 0
+                ? 700L * ProcfsBasedProcessTree.PAGE_SIZE : 0L;
+        Assert.assertEquals(
+          "Cumulative rssmem shouldn't have included new processes",
+          cumuRssMem, processTree.getCumulativeRssmem(2));
+      } else {
+        Assert.assertEquals(
+          "Cumulative rssmem shouldn't have included new processes",
+          100 * KB_TO_BYTES * 3, processTree.getCumulativeRssmem(2));
+      }
 
       // processes older than 1 iteration should not include new process,
       // but include process 500
-      Assert.assertEquals("Cumulative vmem shouldn't have included new processes",
-                   1200000L, processTree.getCumulativeVmem(1));
-      cumuRssMem = ProcfsBasedProcessTree.PAGE_SIZE > 0 ?
-                   1200L * ProcfsBasedProcessTree.PAGE_SIZE : 0L;
-      Assert.assertEquals("Cumulative rssmem shouldn't have included new processes",
-                   cumuRssMem, processTree.getCumulativeRssmem(1));
+      Assert.assertEquals(
+        "Cumulative vmem shouldn't have included new processes", 1200000L,
+        processTree.getCumulativeVmem(1));
+      if (!smapEnabled) {
+        long cumuRssMem =
+            ProcfsBasedProcessTree.PAGE_SIZE > 0
+                ? 1200L * ProcfsBasedProcessTree.PAGE_SIZE : 0L;
+        Assert.assertEquals(
+          "Cumulative rssmem shouldn't have included new processes",
+          cumuRssMem, processTree.getCumulativeRssmem(1));
+      } else {
+        Assert.assertEquals(
+          "Cumulative rssmem shouldn't have included new processes",
+          100 * KB_TO_BYTES * 4, processTree.getCumulativeRssmem(1));
+      }
 
       // no processes older than 3 iterations, this should be 0
-      Assert.assertEquals("Getting non-zero vmem for processes older than 3 iterations",
-                    0L, processTree.getCumulativeVmem(3));
-      Assert.assertEquals("Getting non-zero rssmem for processes older than 3 iterations",
-                    0L, processTree.getCumulativeRssmem(3));
+      Assert.assertEquals(
+        "Getting non-zero vmem for processes older than 3 iterations", 0L,
+        processTree.getCumulativeVmem(3));
+      Assert.assertEquals(
+        "Getting non-zero rssmem for processes older than 3 iterations", 0L,
+        processTree.getCumulativeRssmem(3));
+      Assert.assertEquals(
+        "Getting non-zero rssmem for processes older than 3 iterations", 0L,
+        processTree.getCumulativeRssmem(3));
     } finally {
       FileUtil.fullyDelete(procfsRootDir);
     }
@@ -522,10 +661,12 @@ public class TestProcfsBasedProcessTree {
    * Verifies ProcfsBasedProcessTree.checkPidPgrpidForMatch() in case of
    * 'constructProcessInfo() returning null' by not writing stat file for the
    * mock process
-   * @throws IOException if there was a problem setting up the
-   *                      fake procfs directories or files.
+   *
+   * @throws IOException
+   *           if there was a problem setting up the fake procfs directories or
+   *           files.
    */
-  @Test (timeout = 30000)
+  @Test(timeout = 30000)
   public void testDestroyProcessTree() throws IOException {
     // test process
     String pid = "100";
@@ -539,8 +680,8 @@ public class TestProcfsBasedProcessTree {
       createProcessTree(pid, procfsRootDir.getAbsolutePath());
 
       // Let us not create stat file for pid 100.
-      Assert.assertTrue(ProcfsBasedProcessTree.checkPidPgrpidForMatch(
-            pid, procfsRootDir.getAbsolutePath()));
+      Assert.assertTrue(ProcfsBasedProcessTree.checkPidPgrpidForMatch(pid,
+        procfsRootDir.getAbsolutePath()));
     } finally {
       FileUtil.fullyDelete(procfsRootDir);
     }
@@ -551,9 +692,8 @@ public class TestProcfsBasedProcessTree {
    *
    * @throws IOException
    */
-  @Test (timeout = 30000)
-  public void testProcessTreeDump()
-      throws IOException {
+  @Test(timeout = 30000)
+  public void testProcessTreeDump() throws IOException {
 
     String[] pids = { "100", "200", "300", "400", "500", "600" };
 
@@ -566,18 +706,32 @@ public class TestProcfsBasedProcessTree {
       int numProcesses = pids.length;
       // Processes 200, 300, 400 and 500 are descendants of 100. 600 is not.
       ProcessStatInfo[] procInfos = new ProcessStatInfo[numProcesses];
-      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"});
+      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" });
+
+      ProcessTreeSmapMemInfo[] memInfos = new ProcessTreeSmapMemInfo[6];
+      memInfos[0] = new ProcessTreeSmapMemInfo("100");
+      memInfos[1] = new ProcessTreeSmapMemInfo("200");
+      memInfos[2] = new ProcessTreeSmapMemInfo("300");
+      memInfos[3] = new ProcessTreeSmapMemInfo("400");
+      memInfos[4] = new ProcessTreeSmapMemInfo("500");
+      memInfos[5] = new ProcessTreeSmapMemInfo("600");
 
       String[] cmdLines = new String[numProcesses];
       cmdLines[0] = "proc1 arg1 arg2";
@@ -587,11 +741,12 @@ public class TestProcfsBasedProcessTree {
       cmdLines[4] = "proc5 arg9 arg10";
       cmdLines[5] = "proc6 arg11 arg12";
 
-      writeStatFiles(procfsRootDir, pids, procInfos);
+      createMemoryMappingInfo(memInfos);
+      writeStatFiles(procfsRootDir, pids, procInfos, memInfos);
       writeCmdLineFiles(procfsRootDir, pids, cmdLines);
 
-      ProcfsBasedProcessTree processTree = createProcessTree(
-          "100", procfsRootDir.getAbsolutePath());
+      ProcfsBasedProcessTree processTree =
+          createProcessTree("100", procfsRootDir.getAbsolutePath());
       // build the process tree.
       processTree.updateProcessTree();
 
@@ -600,50 +755,52 @@ public class TestProcfsBasedProcessTree {
 
       LOG.info("Process-tree dump follows: \n" + processTreeDump);
       Assert.assertTrue("Process-tree dump doesn't start with a proper header",
-          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"));
+        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++) {
         ProcessStatInfo p = procInfos[i];
         Assert.assertTrue(
-            "Process-tree dump doesn't contain the cmdLineDump of process "
-                + p.pid, processTreeDump.contains("\t|- " + p.pid + " "
-                + p.ppid + " " + p.pgrpId + " " + p.session + " (" + p.name
-                + ") " + p.utime + " " + p.stime + " " + p.vmem + " "
-                + p.rssmemPage + " " + cmdLines[i]));
+          "Process-tree dump doesn't contain the cmdLineDump of process "
+              + p.pid,
+          processTreeDump.contains("\t|- " + p.pid + " " + p.ppid + " "
+              + p.pgrpId + " " + p.session + " (" + p.name + ") " + p.utime
+              + " " + p.stime + " " + p.vmem + " " + p.rssmemPage + " "
+              + cmdLines[i]));
       }
 
       // 600 should not be in the dump
       ProcessStatInfo p = procInfos[5];
       Assert.assertFalse(
-          "Process-tree dump shouldn't contain the cmdLineDump of process "
-              + p.pid, processTreeDump.contains("\t|- " + p.pid + " " + p.ppid
-              + " " + p.pgrpId + " " + p.session + " (" + p.name + ") "
-              + p.utime + " " + p.stime + " " + p.vmem + " " + cmdLines[5]));
+        "Process-tree dump shouldn't contain the cmdLineDump of process "
+            + p.pid,
+        processTreeDump.contains("\t|- " + p.pid + " " + p.ppid + " "
+            + p.pgrpId + " " + p.session + " (" + p.name + ") " + p.utime + " "
+            + p.stime + " " + p.vmem + " " + cmdLines[5]));
     } finally {
       FileUtil.fullyDelete(procfsRootDir);
     }
   }
 
   protected static boolean isSetsidAvailable() {
-      ShellCommandExecutor shexec = null;
-      boolean setsidSupported = true;
-      try {
-        String[] args = {"setsid", "bash", "-c", "echo $$"};
-        shexec = new ShellCommandExecutor(args);
-        shexec.execute();
-      } catch (IOException ioe) {
-        LOG.warn("setsid is not available on this machine. So not using it.");
-        setsidSupported = false;
-      } finally { // handle the exit code
-        LOG.info("setsid exited with exit code " + shexec.getExitCode());
-      }
-      return setsidSupported;
+    ShellCommandExecutor shexec = null;
+    boolean setsidSupported = true;
+    try {
+      String[] args = { "setsid", "bash", "-c", "echo $$" };
+      shexec = new ShellCommandExecutor(args);
+      shexec.execute();
+    } catch (IOException ioe) {
+      LOG.warn("setsid is not available on this machine. So not using it.");
+      setsidSupported = false;
+    } finally { // handle the exit code
+      LOG.info("setsid exited with exit code " + shexec.getExitCode());
+    }
+    return setsidSupported;
   }
 
   /**
-   * Is the root-process alive?
-   * Used only in tests.
+   * Is the root-process alive? Used only in tests.
+   *
    * @return true if the root-process is alive, false otherwise.
    */
   private static boolean isAlive(String pid) {
@@ -662,16 +819,16 @@ public class TestProcfsBasedProcessTree {
 
   private static void sendSignal(String pid, int signal) throws IOException {
     ShellCommandExecutor shexec = null;
-      String[] arg = { "kill", "-" + signal, pid };
-      shexec = new ShellCommandExecutor(arg);
+    String[] arg = { "kill", "-" + signal, pid };
+    shexec = new ShellCommandExecutor(arg);
     shexec.execute();
   }
 
   /**
-   * Is any of the subprocesses in the process-tree alive?
-   * Used only in tests.
-   * @return true if any of the processes in the process-tree is
-   *           alive, false otherwise.
+   * Is any of the subprocesses in the process-tree alive? Used only in tests.
+   *
+   * @return true if any of the processes in the process-tree is alive, false
+   *         otherwise.
    */
   private static boolean isAnyProcessInTreeAlive(
       ProcfsBasedProcessTree processTree) {
@@ -685,11 +842,13 @@ public class TestProcfsBasedProcessTree {
 
   /**
    * Create a directory to mimic the procfs file system's root.
-   * @param procfsRootDir root directory to create.
-   * @throws IOException if could not delete the procfs root directory
+   *
+   * @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));
@@ -701,18 +860,22 @@ public class TestProcfsBasedProcessTree {
 
   /**
    * Create PID directories under the specified procfs root directory
-   * @param procfsRootDir root directory of procfs file system
-   * @param pids the PID directories to create.
-   * @throws IOException If PID dirs could not be created
+   *
+   * @param procfsRootDir
+   *          root directory of procfs file system
+   * @param pids
+   *          the PID directories to create.
+   * @throws IOException
+   *           If PID dirs could not be created
    */
   public static void setupPidDirs(File procfsRootDir, String[] pids)
-                      throws IOException {
+      throws IOException {
     for (String pid : pids) {
       File pidDir = new File(procfsRootDir, pid);
       pidDir.mkdir();
       if (!pidDir.exists()) {
-        throw new IOException ("couldn't make process directory under " +
-            "fake procfs");
+        throw new IOException("couldn't make process directory under "
+            + "fake procfs");
       } else {
         LOG.info("created pid dir");
       }
@@ -720,43 +883,67 @@ public class TestProcfsBasedProcessTree {
   }
 
   /**
-   * Write stat files under the specified pid directories with data
-   * setup in the corresponding ProcessStatInfo objects
-   * @param procfsRootDir root directory of procfs file system
-   * @param pids the PID directories under which to create the stat file
-   * @param procs corresponding ProcessStatInfo objects whose data should be
-   *              written to the stat files.
-   * @throws IOException if stat files could not be written
+   * Write stat files under the specified pid directories with data setup in the
+   * corresponding ProcessStatInfo objects
+   *
+   * @param procfsRootDir
+   *          root directory of procfs file system
+   * @param pids
+   *          the PID directories under which to create the stat file
+   * @param procs
+   *          corresponding ProcessStatInfo objects whose data should be written
+   *          to the stat files.
+   * @throws IOException
+   *           if stat files could not be written
    */
   public static void writeStatFiles(File procfsRootDir, String[] pids,
-                              ProcessStatInfo[] procs) throws IOException {
-    for (int i=0; i<pids.length; i++) {
+      ProcessStatInfo[] procs, ProcessTreeSmapMemInfo[] smaps)
+      throws IOException {
+    for (int i = 0; i < pids.length; i++) {
       File statFile =
           new File(new File(procfsRootDir, pids[i]),
-              ProcfsBasedProcessTree.PROCFS_STAT_FILE);
+            ProcfsBasedProcessTree.PROCFS_STAT_FILE);
       BufferedWriter bw = null;
       try {
         FileWriter fw = new FileWriter(statFile);
         bw = new BufferedWriter(fw);
         bw.write(procs[i].getStatLine());
-        LOG.info("wrote stat file for " + pids[i] +
-                  " with contents: " + procs[i].getStatLine());
+        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.
         if (bw != null) {
           bw.close();
         }
       }
+      if (smaps != null) {
+        File smapFile =
+            new File(new File(procfsRootDir, pids[i]),
+              ProcfsBasedProcessTree.SMAPS);
+        bw = null;
+        try {
+          FileWriter fw = new FileWriter(smapFile);
+          bw = new BufferedWriter(fw);
+          bw.write(smaps[i].toString());
+          bw.flush();
+          LOG.info("wrote smap file for " + pids[i] + " with contents: "
+              + smaps[i].toString());
+        } finally {
+          // not handling exception - will throw an error and fail the test.
+          if (bw != null) {
+            bw.close();
+          }
+        }
+      }
     }
   }
 
   private static void writeCmdLineFiles(File procfsRootDir, String[] pids,
-      String[] cmdLines)
-      throws IOException {
+      String[] cmdLines) throws IOException {
     for (int i = 0; i < pids.length; i++) {
       File statFile =
           new File(new File(procfsRootDir, pids[i]),
-              ProcfsBasedProcessTree.PROCFS_CMDLINE_FILE);
+            ProcfsBasedProcessTree.PROCFS_CMDLINE_FILE);
       BufferedWriter bw = null;
       try {
         bw = new BufferedWriter(new FileWriter(statFile));

+ 1 - 1
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

@@ -131,7 +131,7 @@ public class TestContainersMonitor extends BaseContainerManagerTest {
       procs[6] = new TestProcfsBasedProcessTree.ProcessStatInfo(
           new String[] { "700", "proc7", "600", "600", "600", "100000" });
       // write stat files.
-      TestProcfsBasedProcessTree.writeStatFiles(procfsRootDir, pids, procs);
+      TestProcfsBasedProcessTree.writeStatFiles(procfsRootDir, pids, procs, null);
 
       // vmem limit
       long limit = 700000;