You are viewing a plain text version of this content. The canonical link for it is here.
Posted to mapreduce-commits@hadoop.apache.org by am...@apache.org on 2012/04/12 09:17:34 UTC

svn commit: r1325145 - in /hadoop/common/trunk/hadoop-mapreduce-project: ./ src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/emulators/resourceusage/ src/contrib/gridmix/src/test/org/apache/hadoop/mapred/gridmix/

Author: amarrk
Date: Thu Apr 12 07:17:34 2012
New Revision: 1325145

URL: http://svn.apache.org/viewvc?rev=1325145&view=rev
Log:
MAPREDUCE-4083. [Gridmix] NPE in cpu emulation. (amarrk)

Modified:
    hadoop/common/trunk/hadoop-mapreduce-project/CHANGES.txt
    hadoop/common/trunk/hadoop-mapreduce-project/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/emulators/resourceusage/CumulativeCpuUsageEmulatorPlugin.java
    hadoop/common/trunk/hadoop-mapreduce-project/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/emulators/resourceusage/TotalHeapUsageEmulatorPlugin.java
    hadoop/common/trunk/hadoop-mapreduce-project/src/contrib/gridmix/src/test/org/apache/hadoop/mapred/gridmix/TestGridmixMemoryEmulation.java
    hadoop/common/trunk/hadoop-mapreduce-project/src/contrib/gridmix/src/test/org/apache/hadoop/mapred/gridmix/TestResourceUsageEmulators.java

Modified: hadoop/common/trunk/hadoop-mapreduce-project/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-mapreduce-project/CHANGES.txt?rev=1325145&r1=1325144&r2=1325145&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-mapreduce-project/CHANGES.txt (original)
+++ hadoop/common/trunk/hadoop-mapreduce-project/CHANGES.txt Thu Apr 12 07:17:34 2012
@@ -52,6 +52,8 @@ Trunk (unreleased changes)
 
   BUG FIXES
 
+    MAPREDUCE-4083. [Gridmix] NPE in cpu emulation. (amarrk)
+
     MAPREDUCE-4087. [Gridmix] GenerateDistCacheData job of Gridmix can
                     become slow in some cases (ravigummadi).
 

Modified: hadoop/common/trunk/hadoop-mapreduce-project/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/emulators/resourceusage/CumulativeCpuUsageEmulatorPlugin.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-mapreduce-project/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/emulators/resourceusage/CumulativeCpuUsageEmulatorPlugin.java?rev=1325145&r1=1325144&r2=1325145&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-mapreduce-project/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/emulators/resourceusage/CumulativeCpuUsageEmulatorPlugin.java (original)
+++ hadoop/common/trunk/hadoop-mapreduce-project/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/emulators/resourceusage/CumulativeCpuUsageEmulatorPlugin.java Thu Apr 12 07:17:34 2012
@@ -235,7 +235,9 @@ implements ResourceUsageEmulatorPlugin {
   
   @Override
   public float getProgress() {
-    return Math.min(1f, ((float)getCurrentCPUUsage())/targetCpuUsage);
+    return enabled 
+           ? Math.min(1f, ((float)getCurrentCPUUsage())/targetCpuUsage)
+           : 1.0f;
   }
   
   @Override
@@ -297,6 +299,9 @@ implements ResourceUsageEmulatorPlugin {
   public void initialize(Configuration conf, ResourceUsageMetrics metrics,
                          ResourceCalculatorPlugin monitor,
                          Progressive progress) {
+    this.monitor = monitor;
+    this.progress = progress;
+    
     // get the target CPU usage
     targetCpuUsage = metrics.getCumulativeCpuUsage();
     if (targetCpuUsage <= 0 ) {
@@ -306,8 +311,6 @@ implements ResourceUsageEmulatorPlugin {
       enabled = true;
     }
     
-    this.monitor = monitor;
-    this.progress = progress;
     emulationInterval =  conf.getFloat(CPU_EMULATION_PROGRESS_INTERVAL, 
                                        DEFAULT_EMULATION_FREQUENCY);
     

Modified: hadoop/common/trunk/hadoop-mapreduce-project/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/emulators/resourceusage/TotalHeapUsageEmulatorPlugin.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-mapreduce-project/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/emulators/resourceusage/TotalHeapUsageEmulatorPlugin.java?rev=1325145&r1=1325144&r2=1325145&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-mapreduce-project/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/emulators/resourceusage/TotalHeapUsageEmulatorPlugin.java (original)
+++ hadoop/common/trunk/hadoop-mapreduce-project/src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/emulators/resourceusage/TotalHeapUsageEmulatorPlugin.java Thu Apr 12 07:17:34 2012
@@ -188,7 +188,9 @@ implements ResourceUsageEmulatorPlugin {
   
   @Override
   public float getProgress() {
-    return Math.min(1f, ((float)getTotalHeapUsageInMB())/targetHeapUsageInMB);
+    return enabled 
+           ? Math.min(1f, ((float)getTotalHeapUsageInMB())/targetHeapUsageInMB)
+           : 1.0f;
   }
   
   @Override
@@ -237,6 +239,8 @@ implements ResourceUsageEmulatorPlugin {
   public void initialize(Configuration conf, ResourceUsageMetrics metrics,
                          ResourceCalculatorPlugin monitor,
                          Progressive progress) {
+    this.progress = progress;
+    
     // get the target heap usage
     targetHeapUsageInMB = metrics.getHeapUsage() / ONE_MB;
     if (targetHeapUsageInMB <= 0 ) {
@@ -248,7 +252,6 @@ implements ResourceUsageEmulatorPlugin {
       enabled = true;
     }
     
-    this.progress = progress;
     emulationInterval = 
       conf.getFloat(HEAP_EMULATION_PROGRESS_INTERVAL, 
                     DEFAULT_EMULATION_PROGRESS_INTERVAL);

Modified: hadoop/common/trunk/hadoop-mapreduce-project/src/contrib/gridmix/src/test/org/apache/hadoop/mapred/gridmix/TestGridmixMemoryEmulation.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-mapreduce-project/src/contrib/gridmix/src/test/org/apache/hadoop/mapred/gridmix/TestGridmixMemoryEmulation.java?rev=1325145&r1=1325144&r2=1325145&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-mapreduce-project/src/contrib/gridmix/src/test/org/apache/hadoop/mapred/gridmix/TestGridmixMemoryEmulation.java (original)
+++ hadoop/common/trunk/hadoop-mapreduce-project/src/contrib/gridmix/src/test/org/apache/hadoop/mapred/gridmix/TestGridmixMemoryEmulation.java Thu Apr 12 07:17:34 2012
@@ -171,6 +171,11 @@ public class TestGridmixMemoryEmulation 
     assertEquals("Disabled heap usage emulation plugin works!", 
                  heapUsagePre, heapUsagePost);
     
+    // test with get progress
+    float progress = heapPlugin.getProgress();
+    assertEquals("Invalid progress of disabled cumulative heap usage emulation "
+                 + "plugin!", 1.0f, progress, 0f);
+    
     // test with wrong/invalid configuration
     Boolean failed = null;
     invalidUsage = 

Modified: hadoop/common/trunk/hadoop-mapreduce-project/src/contrib/gridmix/src/test/org/apache/hadoop/mapred/gridmix/TestResourceUsageEmulators.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-mapreduce-project/src/contrib/gridmix/src/test/org/apache/hadoop/mapred/gridmix/TestResourceUsageEmulators.java?rev=1325145&r1=1325144&r2=1325145&view=diff
==============================================================================
--- hadoop/common/trunk/hadoop-mapreduce-project/src/contrib/gridmix/src/test/org/apache/hadoop/mapred/gridmix/TestResourceUsageEmulators.java (original)
+++ hadoop/common/trunk/hadoop-mapreduce-project/src/contrib/gridmix/src/test/org/apache/hadoop/mapred/gridmix/TestResourceUsageEmulators.java Thu Apr 12 07:17:34 2012
@@ -32,7 +32,6 @@ import org.apache.hadoop.mapreduce.TaskT
 import org.apache.hadoop.mapreduce.server.tasktracker.TTConfig;
 import org.apache.hadoop.mapreduce.task.MapContextImpl;
 import org.apache.hadoop.mapreduce.util.ResourceCalculatorPlugin;
-import org.apache.hadoop.yarn.util.ResourceCalculatorPlugin.ProcResourceValues;
 import org.apache.hadoop.tools.rumen.ResourceUsageMetrics;
 import org.apache.hadoop.mapred.DummyResourceCalculatorPlugin;
 import org.apache.hadoop.mapred.gridmix.LoadJob.ResourceUsageMatcherRunner;
@@ -484,6 +483,11 @@ public class TestResourceUsageEmulators 
     assertEquals("Disabled cumulative CPU usage emulation plugin works!", 
                  cpuUsagePre, cpuUsagePost);
     
+    // test with get progress
+    float progress = cpuPlugin.getProgress();
+    assertEquals("Invalid progress of disabled cumulative CPU usage emulation " 
+                 + "plugin!", 1.0f, progress, 0f);
+    
     // test with valid resource usage value
     ResourceUsageMetrics metrics = createMetrics(targetCpuUsage);