You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by sr...@apache.org on 2012/04/28 03:00:05 UTC

svn commit: r1331645 - in /hadoop/common/branches/branch-1-win: ./ src/core/org/apache/hadoop/util/ src/mapred/org/apache/hadoop/mapred/ src/test/org/apache/hadoop/mapred/ src/test/org/apache/hadoop/util/

Author: sradia
Date: Sat Apr 28 01:00:04 2012
New Revision: 1331645

URL: http://svn.apache.org/viewvc?rev=1331645&view=rev
Log:
    MAPREDUCE-4204 Refactor ProcfsBasedProcessTree to make the resource collection object pluggable (Bikas Saha via Sanjay)

Added:
    hadoop/common/branches/branch-1-win/src/core/org/apache/hadoop/util/ResourceCalculatorProcessTree.java
Modified:
    hadoop/common/branches/branch-1-win/CHANGES.txt
    hadoop/common/branches/branch-1-win/src/core/org/apache/hadoop/util/LinuxResourceCalculatorPlugin.java
    hadoop/common/branches/branch-1-win/src/core/org/apache/hadoop/util/ProcfsBasedProcessTree.java
    hadoop/common/branches/branch-1-win/src/mapred/org/apache/hadoop/mapred/TaskMemoryManagerThread.java
    hadoop/common/branches/branch-1-win/src/mapred/org/apache/hadoop/mapred/TaskTracker.java
    hadoop/common/branches/branch-1-win/src/test/org/apache/hadoop/mapred/TestTaskTrackerMemoryManager.java
    hadoop/common/branches/branch-1-win/src/test/org/apache/hadoop/util/TestProcfsBasedProcessTree.java

Modified: hadoop/common/branches/branch-1-win/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-1-win/CHANGES.txt?rev=1331645&r1=1331644&r2=1331645&view=diff
==============================================================================
--- hadoop/common/branches/branch-1-win/CHANGES.txt (original)
+++ hadoop/common/branches/branch-1-win/CHANGES.txt Sat Apr 28 01:00:04 2012
@@ -7,11 +7,13 @@ Hadoop Change Log
 branch-hadoop-1-win - unreleased
   IMPROVEMENTS
 
-    HADOOP-8223 - Initial patch for branch-1-win (David Lao via Sanjay)
+    HADOOP-8223 Initial patch for branch-1-win (David Lao via Sanjay)
 
-    HADOOP-8234 - Enable user group mappings on Windows (Bikas Saha via Sanjay)
+    HADOOP-8234 Enable user group mappings on Windows (Bikas Saha via Sanjay)
 
-    MAPREDUCE-4201 - Getting PID not working on Windows. Termination of Task/TaskJVM's not working (Bikas Saha via Sanjay)
+    MAPREDUCE-4201 Getting PID not working on Windows. Termination of Task/TaskJVM's not working (Bikas Saha via Sanjay)
+
+    MAPREDUCE-4204 Refactor ProcfsBasedProcessTree to make the resource collection object pluggable (Bikas Saha via Sanjay)
 
 Release 1.1.0 - unreleased
 

Modified: hadoop/common/branches/branch-1-win/src/core/org/apache/hadoop/util/LinuxResourceCalculatorPlugin.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-1-win/src/core/org/apache/hadoop/util/LinuxResourceCalculatorPlugin.java?rev=1331645&r1=1331644&r2=1331645&view=diff
==============================================================================
--- hadoop/common/branches/branch-1-win/src/core/org/apache/hadoop/util/LinuxResourceCalculatorPlugin.java (original)
+++ hadoop/common/branches/branch-1-win/src/core/org/apache/hadoop/util/LinuxResourceCalculatorPlugin.java Sat Apr 28 01:00:04 2012
@@ -86,7 +86,7 @@ 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;

Modified: hadoop/common/branches/branch-1-win/src/core/org/apache/hadoop/util/ProcfsBasedProcessTree.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-1-win/src/core/org/apache/hadoop/util/ProcfsBasedProcessTree.java?rev=1331645&r1=1331644&r2=1331645&view=diff
==============================================================================
--- hadoop/common/branches/branch-1-win/src/core/org/apache/hadoop/util/ProcfsBasedProcessTree.java (original)
+++ hadoop/common/branches/branch-1-win/src/core/org/apache/hadoop/util/ProcfsBasedProcessTree.java Sat Apr 28 01:00:04 2012
@@ -39,7 +39,7 @@ import org.apache.hadoop.util.Shell.Shel
 /**
  * A Proc file-system based ProcessTree. Works only on Linux.
  */
-public class ProcfsBasedProcessTree extends ProcessTree {
+public class ProcfsBasedProcessTree extends ResourceCalculatorProcessTree {
 
   static final Log LOG = LogFactory
       .getLog(ProcfsBasedProcessTree.class);
@@ -92,15 +92,12 @@ public class ProcfsBasedProcessTree exte
   private Long cpuTime = 0L;
 
   private Map<String, ProcessInfo> processTree = new HashMap<String, ProcessInfo>();
-
-  public ProcfsBasedProcessTree(String pid) {
-    this(pid, false);
-  }
   
-  public ProcfsBasedProcessTree(String pid, boolean setsidUsed) {
+  public ProcfsBasedProcessTree(String pid) {
     this(pid,PROCFS);
   }
 
+  // exposing procfs dir for testing overrides
   public ProcfsBasedProcessTree(String pid, String procfsDir) {
     this.pid = getValidPID(pid);
     this.procfsDir = procfsDir;
@@ -132,7 +129,8 @@ public class ProcfsBasedProcessTree exte
    * 
    * @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();
@@ -204,27 +202,27 @@ public class ProcfsBasedProcessTree exte
   }
 
   /**
-   * Is the root-process alive?
+   * Is the root-process alive? Used in testing
    * 
    * @return true if the root-process is alive, false otherwise.
    */
-  public boolean isAlive() {
+  boolean isAlive() {
     if (pid.equals(deadPid)) {
       return false;
     } else {
-      return isAlive(pid);
+      return ProcessTree.isAlive(pid);
     }
   }
 
   /**
    * Is any of the subprocesses in the process-tree alive?
-   * 
+   * Used in testing
    * @return true if any of the processes in the process-tree is
    *           alive, false otherwise.
    */
-  public boolean isAnyProcessInTreeAlive() {
+  boolean isAnyProcessInTreeAlive() {
     for (String pId : processTree.keySet()) {
-      if (isAlive(pId)) {
+      if (ProcessTree.isAlive(pId)) {
         return true;
       }
     }
@@ -240,6 +238,7 @@ public class ProcfsBasedProcessTree exte
    * @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.
@@ -259,29 +258,6 @@ public class ProcfsBasedProcessTree exte
 
   /**
    * 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
@@ -289,6 +265,7 @@ public class ProcfsBasedProcessTree exte
    * @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()) {
@@ -309,6 +286,7 @@ public class ProcfsBasedProcessTree exte
    *          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;
@@ -329,6 +307,7 @@ public class ProcfsBasedProcessTree exte
    * @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;
@@ -336,7 +315,7 @@ public class ProcfsBasedProcessTree exte
     long incJiffies = 0;
     for (ProcessInfo p : processTree.values()) {
       if (p != null) {
-        incJiffies += p.dtime;
+        incJiffies += p.getDtime();
       }
     }
     cpuTime += incJiffies * JIFFY_LENGTH_IN_MILLIS;

Added: hadoop/common/branches/branch-1-win/src/core/org/apache/hadoop/util/ResourceCalculatorProcessTree.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-1-win/src/core/org/apache/hadoop/util/ResourceCalculatorProcessTree.java?rev=1331645&view=auto
==============================================================================
--- hadoop/common/branches/branch-1-win/src/core/org/apache/hadoop/util/ResourceCalculatorProcessTree.java (added)
+++ hadoop/common/branches/branch-1-win/src/core/org/apache/hadoop/util/ResourceCalculatorProcessTree.java Sat Apr 28 01:00:04 2012
@@ -0,0 +1,137 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.util;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * Interface class to obtain process resource usage
+ *
+ */
+public abstract class ResourceCalculatorProcessTree {
+  static final Log LOG = LogFactory
+      .getLog(ResourceCalculatorProcessTree.class);
+  
+  /**
+   * Get the process-tree with latest state. If the root-process is not alive,
+   * an empty tree will be returned.
+   * 
+   * Each call to this function should increment the age of the running
+   * processes that already exist in the process tree. Age is used other API's
+   * of the interface.
+   * 
+   * @return the process-tree with latest state.
+   */
+  public abstract ResourceCalculatorProcessTree getProcessTree();
+
+  /**
+   * Get a dump of the process-tree.
+   * 
+   * @return a string concatenating the dump of information of all the processes
+   *         in the process-tree
+   */
+  public abstract String getProcessTreeDump();
+  
+  /**
+   * 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() {
+    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() {
+    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.
+   */
+  public abstract long getCumulativeVmem(int olderThanAge);
+
+  /**
+   * Get the cumulative resident set size (rss) memory used by all the processes
+   * in the process-tree that are older than the passed in age.
+   *
+   * @param olderThanAge processes above this age are included in the
+   *                      memory addition
+   * @return cumulative rss memory used by the process-tree in bytes,
+   *          for processes older than this age. return 0 if it cannot be
+   *          calculated
+   */
+  public abstract long getCumulativeRssmem(int olderThanAge);
+
+  /**
+   * Get the CPU time in millisecond used by all the processes in the
+   * process-tree since the process-tree created
+   *
+   * @return cumulative CPU time in millisecond since the process-tree created
+   *         return 0 if it cannot be calculated
+   */
+  public abstract long getCumulativeCpuTime();
+  
+  /**
+   * Checks if the ResourceCalculatorProcessTree is available on this system.
+   * 
+   * @return true if ResourceCalculatorProcessTree is available. False
+   *         otherwise.
+   */
+  public static boolean isAvailable() {
+    if(ProcfsBasedProcessTree.isAvailable())
+      return true;
+    
+    return false;
+  }
+
+  /**
+   * Get the ResourceCalculatorProcessTree configure it. This method will try
+   * and return a ResourceCalculatorProcessTree available for this system.
+   * 
+   * @param pid
+   *          process pid of the root of the process tree
+   * @param conf
+   *          configure the object with this.
+   * @return ResourceCalculatorProcessTree
+   */
+  public static ResourceCalculatorProcessTree getResourceCalculatorProcessTree(
+      String pid, Configuration conf) {
+    if(ProcfsBasedProcessTree.isAvailable())
+      return new ProcfsBasedProcessTree(pid);
+
+    // Not supported on this system.
+    return null;
+  }
+}

Modified: hadoop/common/branches/branch-1-win/src/mapred/org/apache/hadoop/mapred/TaskMemoryManagerThread.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-1-win/src/mapred/org/apache/hadoop/mapred/TaskMemoryManagerThread.java?rev=1331645&r1=1331644&r2=1331645&view=diff
==============================================================================
--- hadoop/common/branches/branch-1-win/src/mapred/org/apache/hadoop/mapred/TaskMemoryManagerThread.java (original)
+++ hadoop/common/branches/branch-1-win/src/mapred/org/apache/hadoop/mapred/TaskMemoryManagerThread.java Sat Apr 28 01:00:04 2012
@@ -30,6 +30,7 @@ import org.apache.hadoop.mapred.TaskTrac
 import org.apache.hadoop.mapred.TaskTracker.TaskInProgress;
 import org.apache.hadoop.util.ProcessTree;
 import org.apache.hadoop.util.ProcfsBasedProcessTree;
+import org.apache.hadoop.util.ResourceCalculatorProcessTree;
 import org.apache.hadoop.util.StringUtils;
 
 /**
@@ -44,6 +45,7 @@ class TaskMemoryManagerThread extends Th
   private long monitoringInterval;
 
   private long maxMemoryAllowedForAllTasks;
+  private JobConf conf;
 
   private Map<TaskAttemptID, ProcessTreeInfo> processTreeInfoMap;
   private Map<TaskAttemptID, ProcessTreeInfo> tasksToBeAdded;
@@ -53,7 +55,7 @@ class TaskMemoryManagerThread extends Th
     "Memory usage of ProcessTree %s for task-id %s : %d bytes, " +
       "limit : %d bytes";
   
-  public TaskMemoryManagerThread(TaskTracker taskTracker) {
+  public TaskMemoryManagerThread(TaskTracker taskTracker, JobConf conf) {
     
     this(taskTracker.getTotalMemoryAllottedForTasksOnTT() * 1024 * 1024L,
       taskTracker.getJobConf().getLong(
@@ -61,6 +63,7 @@ class TaskMemoryManagerThread extends Th
         5000L));
 
     this.taskTracker = taskTracker;
+    this.conf = conf;
   }
 
   // mainly for test purposes. note that the tasktracker variable is
@@ -95,7 +98,7 @@ class TaskMemoryManagerThread extends Th
   private static class ProcessTreeInfo {
     private TaskAttemptID tid;
     private String pid;
-    private ProcfsBasedProcessTree pTree;
+    private ResourceCalculatorProcessTree pTree;
     private long memLimit;
     private String pidFile;
 
@@ -119,11 +122,11 @@ class TaskMemoryManagerThread extends Th
       this.pid = pid;
     }
 
-    public ProcfsBasedProcessTree getProcessTree() {
+    public ResourceCalculatorProcessTree getProcessTree() {
       return pTree;
     }
 
-    public void setProcessTree(ProcfsBasedProcessTree pTree) {
+    public void setProcessTree(ResourceCalculatorProcessTree pTree) {
       this.pTree = pTree;
     }
 
@@ -183,8 +186,10 @@ class TaskMemoryManagerThread extends Th
               // itself is still retained in runningTasks till successful
               // transmission to JT
 
-              ProcfsBasedProcessTree pt = 
-                new ProcfsBasedProcessTree(pId, ProcessTree.isSetsidAvailable);
+              ResourceCalculatorProcessTree pt = ResourceCalculatorProcessTree
+                  .getResourceCalculatorProcessTree(pId, conf);
+              // we would not be running unless a valid calculator is available
+              assert pt != null;
               LOG.debug("Tracking ProcessTree " + pId + " for the first time");
 
               ptInfo.setPid(pId);
@@ -199,7 +204,7 @@ class TaskMemoryManagerThread extends Th
 
           LOG.debug("Constructing ProcessTree for : PID = " + pId + " TID = "
               + tid);
-          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
@@ -315,7 +320,7 @@ class TaskMemoryManagerThread extends Th
   }
 
   // method provided just for easy testing purposes
-  boolean isProcessTreeOverLimit(ProcfsBasedProcessTree pTree, 
+  boolean isProcessTreeOverLimit(ResourceCalculatorProcessTree pTree, 
                                     String tId, long limit) {
     long currentMemUsage = pTree.getCumulativeVmem();
     // as processes begin with an age 1, we want to see if there are processes
@@ -341,7 +346,7 @@ class TaskMemoryManagerThread extends Th
       TaskAttemptID tid = task.getTask().getTaskID();
       if (processTreeInfoMap.containsKey(tid)) {
         ProcessTreeInfo ptInfo = processTreeInfoMap.get(tid);
-        ProcfsBasedProcessTree pTree = ptInfo.getProcessTree();
+        ResourceCalculatorProcessTree pTree = ptInfo.getProcessTree();
         memoryStillInUsage -= pTree.getCumulativeVmem();
         tasksToKill.add(tid);
       }

Modified: hadoop/common/branches/branch-1-win/src/mapred/org/apache/hadoop/mapred/TaskTracker.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-1-win/src/mapred/org/apache/hadoop/mapred/TaskTracker.java?rev=1331645&r1=1331644&r2=1331645&view=diff
==============================================================================
--- hadoop/common/branches/branch-1-win/src/mapred/org/apache/hadoop/mapred/TaskTracker.java (original)
+++ hadoop/common/branches/branch-1-win/src/mapred/org/apache/hadoop/mapred/TaskTracker.java Sat Apr 28 01:00:04 2012
@@ -99,6 +99,7 @@ import org.apache.hadoop.util.ProcfsBase
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.util.ResourceCalculatorProcessTree;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.VersionInfo;
 import org.apache.hadoop.util.DiskChecker.DiskErrorException;
@@ -4201,15 +4202,15 @@ public class TaskTracker implements MRCo
     // start the taskMemoryManager thread only if enabled
     setTaskMemoryManagerEnabledFlag();
     if (isTaskMemoryManagerEnabled()) {
-      taskMemoryManager = new TaskMemoryManagerThread(this);
+      taskMemoryManager = new TaskMemoryManagerThread(this, fConf);
       taskMemoryManager.setDaemon(true);
       taskMemoryManager.start();
     }
   }
 
   void setTaskMemoryManagerEnabledFlag() {
-    if (!ProcfsBasedProcessTree.isAvailable()) {
-      LOG.info("ProcessTree implementation is missing on this system. "
+    if (!ResourceCalculatorProcessTree.isAvailable()) {
+      LOG.info("ResourceCalculatorProcessTree implementation is missing on this system. "
           + "TaskMemoryManager is disabled.");
       taskMemoryManagerEnabled = false;
       return;

Modified: hadoop/common/branches/branch-1-win/src/test/org/apache/hadoop/mapred/TestTaskTrackerMemoryManager.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-1-win/src/test/org/apache/hadoop/mapred/TestTaskTrackerMemoryManager.java?rev=1331645&r1=1331644&r2=1331645&view=diff
==============================================================================
--- hadoop/common/branches/branch-1-win/src/test/org/apache/hadoop/mapred/TestTaskTrackerMemoryManager.java (original)
+++ hadoop/common/branches/branch-1-win/src/test/org/apache/hadoop/mapred/TestTaskTrackerMemoryManager.java Sat Apr 28 01:00:04 2012
@@ -35,6 +35,7 @@ import org.apache.hadoop.hdfs.server.nam
 import org.apache.hadoop.examples.SleepJob;
 import org.apache.hadoop.util.MemoryCalculatorPlugin;
 import org.apache.hadoop.util.ProcfsBasedProcessTree;
+import org.apache.hadoop.util.ResourceCalculatorProcessTree;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.TestProcfsBasedProcessTree;
 import org.apache.hadoop.util.ToolRunner;
@@ -119,11 +120,10 @@ public class TestTaskTrackerMemoryManage
     }
   }
 
-  private boolean isProcfsBasedTreeAvailable() {
+  private boolean isResourceCalculatorTreeAvailable() {
     try {
-      if (!ProcfsBasedProcessTree.isAvailable()) {
-        LOG.info("Currently ProcessTree has only one implementation "
-            + "ProcfsBasedProcessTree, which is not available on this "
+      if (!ResourceCalculatorProcessTree.isAvailable()) {
+        LOG.info("ResourceCalculatorProcessTree is not available on this "
             + "system. Not testing");
         return false;
       }
@@ -143,7 +143,7 @@ public class TestTaskTrackerMemoryManage
   public void testTTLimitsDisabled()
       throws Exception {
     // Run the test only if memory management is enabled
-    if (!isProcfsBasedTreeAvailable()) {
+    if (!isResourceCalculatorTreeAvailable()) {
       return;
     }
 
@@ -165,7 +165,7 @@ public class TestTaskTrackerMemoryManage
   public void testTasksWithinLimits()
       throws Exception {
     // Run the test only if memory management is enabled
-    if (!isProcfsBasedTreeAvailable()) {
+    if (!isResourceCalculatorTreeAvailable()) {
       return;
     }
 
@@ -196,7 +196,7 @@ public class TestTaskTrackerMemoryManage
       throws Exception {
 
     // Run the test only if memory management is enabled
-    if (!isProcfsBasedTreeAvailable()) {
+    if (!isResourceCalculatorTreeAvailable()) {
       return;
     }
 
@@ -225,7 +225,7 @@ public class TestTaskTrackerMemoryManage
     throws Exception {
     
     // Run the test only if memory management is enabled
-    if (!isProcfsBasedTreeAvailable()) {
+    if (!isResourceCalculatorTreeAvailable()) {
       return;
     }
     // Start cluster with proper configuration.
@@ -316,7 +316,7 @@ public class TestTaskTrackerMemoryManage
       throws Exception {
 
     // Run the test only if memory management is enabled
-    if (!isProcfsBasedTreeAvailable()) {
+    if (!isResourceCalculatorTreeAvailable()) {
       return;
     }
 

Modified: hadoop/common/branches/branch-1-win/src/test/org/apache/hadoop/util/TestProcfsBasedProcessTree.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-1-win/src/test/org/apache/hadoop/util/TestProcfsBasedProcessTree.java?rev=1331645&r1=1331644&r2=1331645&view=diff
==============================================================================
--- hadoop/common/branches/branch-1-win/src/test/org/apache/hadoop/util/TestProcfsBasedProcessTree.java (original)
+++ hadoop/common/branches/branch-1-win/src/test/org/apache/hadoop/util/TestProcfsBasedProcessTree.java Sat Apr 28 01:00:04 2012
@@ -22,7 +22,6 @@ import java.io.BufferedWriter;
 import java.io.File;
 import java.io.FileWriter;
 import java.io.IOException;
-import java.util.Arrays;
 import java.util.Random;
 import java.util.Vector;
 import java.util.regex.Matcher;
@@ -148,9 +147,8 @@ public class TestProcfsBasedProcessTree 
     t.start();
     String pid = getRogueTaskPID();
     LOG.info("Root process pid: " + pid);
-    ProcfsBasedProcessTree p = new ProcfsBasedProcessTree(pid,
-        ProcessTree.isSetsidAvailable);
-    p = p.getProcessTree(); // initialize
+    ProcfsBasedProcessTree p = new ProcfsBasedProcessTree(pid);
+    p.getProcessTree(); // initialize
     LOG.info("ProcessTree: " + p.toString());
     File leaf = new File(lowestDescendant);
     //wait till lowest descendant process of Rougue Task starts execution
@@ -162,7 +160,7 @@ public class TestProcfsBasedProcessTree 
       }
     }
     
-    p = p.getProcessTree(); // reconstruct
+    p.getProcessTree(); // reconstruct
     LOG.info("ProcessTree: " + p.toString());
 
     // Get the process-tree dump
@@ -204,7 +202,7 @@ public class TestProcfsBasedProcessTree 
     }
 
     // ProcessTree is gone now. Any further calls should be sane.
-    p = p.getProcessTree();
+    p.getProcessTree();
     assertFalse("ProcessTree must have been gone", p.isAlive());
     assertTrue("Cumulative vmem for the gone-process is "
         + p.getCumulativeVmem() + " . It should be zero.", p