Browse Source

YARN-3427. Remove deprecated methods from ResourceCalculatorProcessTree
(Contributed by Miklos Szegedi via Daniel Templeton)

Daniel Templeton 8 years ago
parent
commit
15acdbdba2

+ 0 - 12
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ProcfsBasedProcessTree.java

@@ -341,12 +341,6 @@ public class ProcfsBasedProcessTree extends ResourceCalculatorProcessTree {
     return total;
   }
 
-  @Override
-  @SuppressWarnings("deprecation")
-  public long getCumulativeVmem(int olderThanAge) {
-    return getVirtualMemorySize(olderThanAge);
-  }
-
   @Override
   public long getRssMemorySize(int olderThanAge) {
     if (PAGE_SIZE < 0) {
@@ -368,12 +362,6 @@ public class ProcfsBasedProcessTree extends ResourceCalculatorProcessTree {
     return isAvailable ? totalPages * PAGE_SIZE : UNAVAILABLE; // convert # pages to byte
   }
 
-  @Override
-  @SuppressWarnings("deprecation")
-  public long getCumulativeRssmem(int olderThanAge) {
-    return getRssMemorySize(olderThanAge);
-  }
-
   /**
    * Get the resident set size (RSS) memory used by all the processes
    * in the process-tree that are older than the passed in age. RSS is

+ 0 - 54
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ResourceCalculatorProcessTree.java

@@ -79,18 +79,6 @@ public abstract class ResourceCalculatorProcessTree extends Configured {
     return getVirtualMemorySize(0);
   }
 
-  /**
-   * Get the virtual memory used by all the processes in the
-   * process-tree.
-   *
-   * @return virtual memory used by the process-tree in bytes,
-   * {@link #UNAVAILABLE} if it cannot be calculated.
-   */
-  @Deprecated
-  public long getCumulativeVmem() {
-    return getCumulativeVmem(0);
-  }
-
   /**
    * Get the resident set size (rss) memory used by all the processes
    * in the process-tree.
@@ -102,18 +90,6 @@ public abstract class ResourceCalculatorProcessTree extends Configured {
     return getRssMemorySize(0);
   }
 
-  /**
-   * Get the resident set size (rss) memory used by all the processes
-   * in the process-tree.
-   *
-   * @return rss memory used by the process-tree in bytes,
-   * {@link #UNAVAILABLE} if it cannot be calculated.
-   */
-  @Deprecated
-  public long getCumulativeRssmem() {
-    return getCumulativeRssmem(0);
-  }
-
   /**
    * Get the virtual memory used by all the processes in the
    * process-tree that are older than the passed in age.
@@ -128,21 +104,6 @@ public abstract class ResourceCalculatorProcessTree extends Configured {
     return UNAVAILABLE;
   }
 
-  /**
-   * Get the 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 virtual memory used by the process-tree in bytes for
-   * processes older than the specified age, {@link #UNAVAILABLE} if it
-   * cannot be calculated.
-   */
-  @Deprecated
-  public long getCumulativeVmem(int olderThanAge) {
-    return UNAVAILABLE;
-  }
-
   /**
    * Get the resident set size (rss) memory used by all the processes
    * in the process-tree that are older than the passed in age.
@@ -157,21 +118,6 @@ public abstract class ResourceCalculatorProcessTree extends Configured {
     return UNAVAILABLE;
   }
 
-  /**
-   * Get the 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 rss memory used by the process-tree in bytes for
-   * processes older than specified age, {@link #UNAVAILABLE} if it cannot be
-   * calculated.
-   */
-  @Deprecated
-  public long getCumulativeRssmem(int olderThanAge) {
-    return UNAVAILABLE;
-  }
-
   /**
    * Get the CPU time in millisecond used by all the processes in the
    * process-tree since the process-tree was created

+ 0 - 12
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/WindowsBasedProcessTree.java

@@ -214,12 +214,6 @@ public class WindowsBasedProcessTree extends ResourceCalculatorProcessTree {
     return total;
   }
 
-  @Override
-  @SuppressWarnings("deprecation")
-  public long getCumulativeVmem(int olderThanAge) {
-    return getVirtualMemorySize(olderThanAge);
-  }
-
   @Override
   public long getRssMemorySize(int olderThanAge) {
     long total = UNAVAILABLE;
@@ -236,12 +230,6 @@ public class WindowsBasedProcessTree extends ResourceCalculatorProcessTree {
     return total;
   }
 
-  @Override
-  @SuppressWarnings("deprecation")
-  public long getCumulativeRssmem(int olderThanAge) {
-    return getRssMemorySize(olderThanAge);
-  }
-
   @Override
   public long getCumulativeCpuTime() {
     for (ProcessInfo p : processTree.values()) {

+ 0 - 64
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestProcfsBasedProcessTree.java

@@ -236,10 +236,6 @@ public class TestProcfsBasedProcessTree {
       "vmem for the gone-process is " + p.getVirtualMemorySize()
           + " . It should be UNAVAILABLE(-1).",
           p.getVirtualMemorySize() == UNAVAILABLE);
-    Assert.assertTrue(
-      "vmem (old API) for the gone-process is " + p.getCumulativeVmem()
-          + " . It should be UNAVAILABLE(-1).",
-          p.getCumulativeVmem() == UNAVAILABLE);
     Assert.assertTrue(p.toString().equals("[ ]"));
   }
 
@@ -456,9 +452,6 @@ public class TestProcfsBasedProcessTree {
                   ResourceCalculatorProcessTree.UNAVAILABLE;
       Assert.assertEquals("rss memory does not match", cumuRssMem,
         processTree.getRssMemorySize());
-      // verify old API
-      Assert.assertEquals("rss memory (old API) does not match", cumuRssMem,
-        processTree.getCumulativeRssmem());
 
       // verify cumulative cpu time
       long cumuCpuTime =
@@ -477,9 +470,6 @@ public class TestProcfsBasedProcessTree {
       // anon (exclude r-xs,r--s)
       Assert.assertEquals("rss memory does not match",
           (20 * KB_TO_BYTES * 3), processTree.getRssMemorySize());
-      // verify old API
-      Assert.assertEquals("rss memory (old API) does not match",
-          (20 * KB_TO_BYTES * 3), processTree.getCumulativeRssmem());
 
       // test the cpu time again to see if it cumulates
       procInfos[0] =
@@ -545,7 +535,6 @@ public class TestProcfsBasedProcessTree {
     testMemForOlderProcesses(true);
   }
 
-  @SuppressWarnings("deprecation")
   private void testMemForOlderProcesses(boolean smapEnabled) throws IOException {
     // initial list of processes
     String[] pids = { "100", "200", "300", "400" };
@@ -589,8 +578,6 @@ public class TestProcfsBasedProcessTree {
       // verify virtual memory
       Assert.assertEquals("Virtual memory does not match", 700000L,
         processTree.getVirtualMemorySize());
-      Assert.assertEquals("Virtual memory (old API) does not match", 700000L,
-        processTree.getCumulativeVmem());
 
       // write one more process as child of 100.
       String[] newPids = { "500" };
@@ -609,8 +596,6 @@ public class TestProcfsBasedProcessTree {
       processTree.updateProcessTree();
       Assert.assertEquals("vmem does not include new process",
         1200000L, processTree.getVirtualMemorySize());
-      Assert.assertEquals("vmem (old API) does not include new process",
-        1200000L, processTree.getCumulativeVmem());
       if (!smapEnabled) {
         long cumuRssMem =
             ProcfsBasedProcessTree.PAGE_SIZE > 0
@@ -618,25 +603,15 @@ public class TestProcfsBasedProcessTree {
                     ResourceCalculatorProcessTree.UNAVAILABLE;
         Assert.assertEquals("rssmem does not include new process",
           cumuRssMem, processTree.getRssMemorySize());
-        // verify old API
-        Assert.assertEquals("rssmem (old API) does not include new process",
-          cumuRssMem, processTree.getCumulativeRssmem());
       } else {
         Assert.assertEquals("rssmem does not include new process",
             20 * KB_TO_BYTES * 4, processTree.getRssMemorySize());
-        // verify old API
-        Assert.assertEquals("rssmem (old API) does not include new process",
-            20 * KB_TO_BYTES * 4, processTree.getCumulativeRssmem());
       }
 
       // however processes older than 1 iteration will retain the older value
       Assert.assertEquals(
         "vmem shouldn't have included new process", 700000L,
         processTree.getVirtualMemorySize(1));
-      // verify old API
-      Assert.assertEquals(
-          "vmem (old API) shouldn't have included new process", 700000L,
-          processTree.getCumulativeVmem(1));
       if (!smapEnabled) {
         long cumuRssMem =
             ProcfsBasedProcessTree.PAGE_SIZE > 0
@@ -645,18 +620,10 @@ public class TestProcfsBasedProcessTree {
         Assert.assertEquals(
           "rssmem shouldn't have included new process", cumuRssMem,
           processTree.getRssMemorySize(1));
-        // Verify old API
-        Assert.assertEquals(
-          "rssmem (old API) shouldn't have included new process", cumuRssMem,
-          processTree.getCumulativeRssmem(1));
       } else {
         Assert.assertEquals(
           "rssmem shouldn't have included new process",
             20 * KB_TO_BYTES * 3, processTree.getRssMemorySize(1));
-        // Verify old API
-        Assert.assertEquals(
-          "rssmem (old API) shouldn't have included new process",
-            20 * KB_TO_BYTES * 3, processTree.getCumulativeRssmem(1));
       }
 
       // one more process
@@ -679,10 +646,6 @@ public class TestProcfsBasedProcessTree {
       Assert.assertEquals(
         "vmem shouldn't have included new processes", 700000L,
         processTree.getVirtualMemorySize(2));
-      // verify old API
-      Assert.assertEquals(
-        "vmem (old API) shouldn't have included new processes", 700000L,
-        processTree.getCumulativeVmem(2));
       if (!smapEnabled) {
         long cumuRssMem =
             ProcfsBasedProcessTree.PAGE_SIZE > 0
@@ -691,18 +654,10 @@ public class TestProcfsBasedProcessTree {
         Assert.assertEquals(
           "rssmem shouldn't have included new processes",
           cumuRssMem, processTree.getRssMemorySize(2));
-        // Verify old API
-        Assert.assertEquals(
-          "rssmem (old API) shouldn't have included new processes",
-          cumuRssMem, processTree.getCumulativeRssmem(2));
       } else {
         Assert.assertEquals(
           "rssmem shouldn't have included new processes",
             20 * KB_TO_BYTES * 3, processTree.getRssMemorySize(2));
-        // Verify old API
-        Assert.assertEquals(
-          "rssmem (old API) shouldn't have included new processes",
-            20 * KB_TO_BYTES * 3, processTree.getCumulativeRssmem(2));
       }
 
       // processes older than 1 iteration should not include new process,
@@ -710,10 +665,6 @@ public class TestProcfsBasedProcessTree {
       Assert.assertEquals(
         "vmem shouldn't have included new processes", 1200000L,
         processTree.getVirtualMemorySize(1));
-      // verify old API
-      Assert.assertEquals(
-        "vmem (old API) shouldn't have included new processes", 1200000L,
-        processTree.getCumulativeVmem(1));
       if (!smapEnabled) {
         long cumuRssMem =
             ProcfsBasedProcessTree.PAGE_SIZE > 0
@@ -722,34 +673,19 @@ public class TestProcfsBasedProcessTree {
         Assert.assertEquals(
           "rssmem shouldn't have included new processes",
           cumuRssMem, processTree.getRssMemorySize(1));
-        // verify old API
-        Assert.assertEquals(
-          "rssmem (old API) shouldn't have included new processes",
-          cumuRssMem, processTree.getCumulativeRssmem(1));
       } else {
         Assert.assertEquals(
           "rssmem shouldn't have included new processes",
             20 * KB_TO_BYTES * 4, processTree.getRssMemorySize(1));
-        Assert.assertEquals(
-          "rssmem (old API) shouldn't have included new processes",
-            20 * KB_TO_BYTES * 4, processTree.getCumulativeRssmem(1));
       }
 
       // no processes older than 3 iterations
       Assert.assertEquals(
           "Getting non-zero vmem for processes older than 3 iterations",
           0, processTree.getVirtualMemorySize(3));
-      // verify old API
-      Assert.assertEquals(
-          "Getting non-zero vmem (old API) for processes older than 3 iterations",
-          0, processTree.getCumulativeVmem(3));
       Assert.assertEquals(
           "Getting non-zero rssmem for processes older than 3 iterations",
           0, processTree.getRssMemorySize(3));
-      // verify old API
-      Assert.assertEquals(
-          "Getting non-zero rssmem (old API) for processes older than 3 iterations",
-          0, processTree.getCumulativeRssmem(3));
     } finally {
       FileUtil.fullyDelete(procfsRootDir);
     }

+ 0 - 12
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestWindowsBasedProcessTree.java

@@ -56,13 +56,9 @@ public class TestWindowsBasedProcessTree {
     pTree.infoStr = "3524,1024,1024,500\r\n2844,1024,1024,500\r\n";
     pTree.updateProcessTree();
     assertTrue(pTree.getVirtualMemorySize() == 2048);
-    assertTrue(pTree.getCumulativeVmem() == 2048);
     assertTrue(pTree.getVirtualMemorySize(0) == 2048);
-    assertTrue(pTree.getCumulativeVmem(0) == 2048);
     assertTrue(pTree.getRssMemorySize() == 2048);
-    assertTrue(pTree.getCumulativeRssmem() == 2048);
     assertTrue(pTree.getRssMemorySize(0) == 2048);
-    assertTrue(pTree.getCumulativeRssmem(0) == 2048);
     assertTrue(pTree.getCumulativeCpuTime() == 1000);
     assertTrue(pTree.getCpuUsagePercent() == ResourceCalculatorProcessTree.UNAVAILABLE);
 
@@ -71,13 +67,9 @@ public class TestWindowsBasedProcessTree {
     testClock.setTime(elapsedTimeBetweenUpdatesMsec);
     pTree.updateProcessTree();
     assertTrue(pTree.getVirtualMemorySize() == 3072);
-    assertTrue(pTree.getCumulativeVmem() == 3072);
     assertTrue(pTree.getVirtualMemorySize(1) == 2048);
-    assertTrue(pTree.getCumulativeVmem(1) == 2048);
     assertTrue(pTree.getRssMemorySize() == 3072);
-    assertTrue(pTree.getCumulativeRssmem() == 3072);
     assertTrue(pTree.getRssMemorySize(1) == 2048);
-    assertTrue(pTree.getCumulativeRssmem(1) == 2048);
     assertTrue(pTree.getCumulativeCpuTime() == 3000);
     assertTrue(pTree.getCpuUsagePercent() == 200);
     Assert.assertEquals("Percent CPU time is not correct",
@@ -88,13 +80,9 @@ public class TestWindowsBasedProcessTree {
     testClock.setTime(elapsedTimeBetweenUpdatesMsec);
     pTree.updateProcessTree();
     assertTrue(pTree.getVirtualMemorySize() == 2048);
-    assertTrue(pTree.getCumulativeVmem() == 2048);
     assertTrue(pTree.getVirtualMemorySize(2) == 2048);
-    assertTrue(pTree.getCumulativeVmem(2) == 2048);
     assertTrue(pTree.getRssMemorySize() == 2048);
-    assertTrue(pTree.getCumulativeRssmem() == 2048);
     assertTrue(pTree.getRssMemorySize(2) == 2048);
-    assertTrue(pTree.getCumulativeRssmem(2) == 2048);
     assertTrue(pTree.getCumulativeCpuTime() == 4000);
     Assert.assertEquals("Percent CPU time is not correct",
         pTree.getCpuUsagePercent(), 0, 0.01);