소스 검색

MAPREDUCE-2517. Add system tests to Gridmix. (Vinay Thota via amarrk)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-1@1222572 13f79535-47bb-0310-9956-ffa450edef68
Amar Kamat 13 년 전
부모
커밋
9c6b52f581
84개의 변경된 파일5817개의 추가작업 그리고 464개의 파일을 삭제
  1. 2 0
      CHANGES.txt
  2. 1 0
      build.xml
  3. 1 0
      ivy/libraries.properties
  4. 79 6
      src/contrib/build-contrib.xml
  5. 1 1
      src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/DistributedCacheEmulator.java
  6. 1 1
      src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/Gridmix.java
  7. 0 75
      src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/GridMixConfig.java
  8. 234 0
      src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/GridmixSystemTestCase.java
  9. 107 0
      src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestCPUEmulationForMapsAndReducesWithCustomInterval.java
  10. 106 0
      src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestCPUEmulationForMapsAndReducesWithDefaultInterval.java
  11. 104 0
      src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestCPUEmulationForMapsWithCustomInterval.java
  12. 103 0
      src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestCPUEmulationForMapsWithDefaultInterval.java
  13. 96 0
      src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestCompressionEmulationEnableForAllTypesOfJobs.java
  14. 98 0
      src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestCompressionEmulationForCompressInAndUncompressOut.java
  15. 93 0
      src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestCompressionEmulationForUncompressInAndCompressOut.java
  16. 65 0
      src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestDisableGridmixEmulationOfHighRam.java
  17. 95 0
      src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestEmulationOfHDFSAndLocalFSDCFiles.java
  18. 91 0
      src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestEmulationOfHDFSDCFileUsesMultipleJobs.java
  19. 92 0
      src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestEmulationOfHDFSDCFilesWithDifferentVisibilities.java
  20. 64 0
      src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestEmulationOfHighRamAndNormalMRJobs.java
  21. 93 0
      src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestEmulationOfLocalFSDCFiles.java
  22. 83 56
      src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestGridMixDataGeneration.java
  23. 40 27
      src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestGridMixFilePool.java
  24. 175 0
      src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestGridmixCompressedInputGeneration.java
  25. 102 0
      src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestGridmixCompressionEmulationWithCompressInput.java
  26. 89 0
      src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestGridmixEmulationOfHDFSPrivateDCFile.java
  27. 92 0
      src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestGridmixEmulationOfHDFSPublicDCFile.java
  28. 64 0
      src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestGridmixEmulationOfHighRamJobsCase1.java
  29. 67 0
      src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestGridmixEmulationOfHighRamJobsCase2.java
  30. 64 0
      src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestGridmixEmulationOfHighRamJobsCase3.java
  31. 92 0
      src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestGridmixEmulationOfMultipleHDFSPrivateDCFiles.java
  32. 93 0
      src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestGridmixEmulationOfMultipleHDFSPublicDCFiles.java
  33. 67 0
      src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestGridmixWith10minTrace.java
  34. 62 0
      src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestGridmixWith12minTrace.java
  35. 59 0
      src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestGridmixWith1minTrace.java
  36. 64 0
      src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestGridmixWith2minStreamingJobTrace.java
  37. 68 0
      src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestGridmixWith3minStreamingJobTrace.java
  38. 62 0
      src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestGridmixWith3minTrace.java
  39. 65 0
      src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestGridmixWith5minStreamingJobTrace.java
  40. 62 0
      src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestGridmixWith5minTrace.java
  41. 62 0
      src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestGridmixWith7minTrace.java
  42. 106 0
      src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestMemEmulForMapsAndReducesWithCustomIntrvl.java
  43. 106 0
      src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestMemEmulForMapsAndReducesWithDefaultIntrvl.java
  44. 108 0
      src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestMemEmulForMapsWithCustomHeapMemoryRatio.java
  45. 106 0
      src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestMemEmulForMapsWithCustomIntrvl.java
  46. 104 0
      src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestMemEmulForMapsWithDefaultIntrvl.java
  47. 0 282
      src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/UtilsForGridmix.java
  48. 286 0
      src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/test/system/GridMixConfig.java
  49. 11 15
      src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/test/system/GridMixRunMode.java
  50. 86 0
      src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/test/system/GridmixJobStory.java
  51. 82 0
      src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/test/system/GridmixJobSubmission.java
  52. 1239 0
      src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/test/system/GridmixJobVerification.java
  53. 524 0
      src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/test/system/UtilsForGridmix.java
  54. BIN
      src/contrib/gridmix/src/test/system/resources/2m_stream_trace.json.gz
  55. BIN
      src/contrib/gridmix/src/test/system/resources/3m_stream_trace.json.gz
  56. BIN
      src/contrib/gridmix/src/test/system/resources/5m_stream_trace.json.gz
  57. BIN
      src/contrib/gridmix/src/test/system/resources/compression_case1_trace.json.gz
  58. BIN
      src/contrib/gridmix/src/test/system/resources/compression_case2_trace.json.gz
  59. BIN
      src/contrib/gridmix/src/test/system/resources/compression_case3_trace.json.gz
  60. BIN
      src/contrib/gridmix/src/test/system/resources/compression_case4_trace.json.gz
  61. BIN
      src/contrib/gridmix/src/test/system/resources/cpu_emul_case1.json.gz
  62. BIN
      src/contrib/gridmix/src/test/system/resources/cpu_emul_case2.json.gz
  63. BIN
      src/contrib/gridmix/src/test/system/resources/distcache_case1_trace.json.gz
  64. BIN
      src/contrib/gridmix/src/test/system/resources/distcache_case2_trace.json.gz
  65. BIN
      src/contrib/gridmix/src/test/system/resources/distcache_case3_trace.json.gz
  66. BIN
      src/contrib/gridmix/src/test/system/resources/distcache_case4_trace.json.gz
  67. BIN
      src/contrib/gridmix/src/test/system/resources/distcache_case5_trace.json.gz
  68. BIN
      src/contrib/gridmix/src/test/system/resources/distcache_case6_trace.json.gz
  69. BIN
      src/contrib/gridmix/src/test/system/resources/distcache_case7_trace.json.gz
  70. BIN
      src/contrib/gridmix/src/test/system/resources/distcache_case8_trace.json.gz
  71. BIN
      src/contrib/gridmix/src/test/system/resources/distcache_case9_trace.json.gz
  72. BIN
      src/contrib/gridmix/src/test/system/resources/highram_mr_jobs_case1.json.gz
  73. BIN
      src/contrib/gridmix/src/test/system/resources/highram_mr_jobs_case2.json.gz
  74. BIN
      src/contrib/gridmix/src/test/system/resources/highram_mr_jobs_case3.json.gz
  75. BIN
      src/contrib/gridmix/src/test/system/resources/highram_mr_jobs_case4.json.gz
  76. BIN
      src/contrib/gridmix/src/test/system/resources/mem_emul_case1.json.gz
  77. BIN
      src/contrib/gridmix/src/test/system/resources/mem_emul_case2.json.gz
  78. BIN
      src/contrib/gridmix/src/test/system/resources/trace_10m.json.gz
  79. BIN
      src/contrib/gridmix/src/test/system/resources/trace_12m.json.gz
  80. BIN
      src/contrib/gridmix/src/test/system/resources/trace_1m.json.gz
  81. BIN
      src/contrib/gridmix/src/test/system/resources/trace_3m.json.gz
  82. BIN
      src/contrib/gridmix/src/test/system/resources/trace_5m.json.gz
  83. BIN
      src/contrib/gridmix/src/test/system/resources/trace_7m.json.gz
  84. 1 1
      src/test/aop/build/aop.xml

+ 2 - 0
CHANGES.txt

@@ -75,6 +75,8 @@ Release 1.1.0 - unreleased
 
   IMPROVEMENTS
 
+    MAPREDUCE-2517. Add system tests to Gridmix. (Vinay Thota via amarrk)
+
     MAPREDUCE-3008. [Gridmix] Improve cumulative CPU usage emulation for
                     short running tasks. (amarrk)
 

+ 1 - 0
build.xml

@@ -1594,6 +1594,7 @@
     <mkdir dir="${dist.dir}/share/${name}/contrib"/>
     <mkdir dir="${dist.dir}/share/${name}/webapps"/>
     <mkdir dir="${dist.dir}/share/${name}/templates/conf"/>
+    <mkdir dir="${dist.dir}/native"/>
 
     <copy todir="${dist.dir}/share/${name}/templates/conf" includeEmptyDirs="false">
       <fileset dir="${basedir}/src/packages/templates/conf">

+ 1 - 0
ivy/libraries.properties

@@ -23,6 +23,7 @@ ant-task.version=2.0.10
 
 asm.version=3.2
 aspectj.version=1.6.5
+aspectj.version=1.6.11
 
 checkstyle.version=4.2
 

+ 79 - 6
src/contrib/build-contrib.xml

@@ -35,6 +35,8 @@
   <property name="src.test" location="${root}/src/test"/>
   <property name="src.test.data" location="${root}/src/test/data"/>
   <!-- Property added for contrib system tests -->
+  <property name="build-fi.dir" location="${hadoop.root}/build-fi"/>
+  <property name="system-test-build-dir" location="${build-fi.dir}/system"/>
   <property name="src.test.system" location="${root}/src/test/system"/>
 
   <property name="src.examples" location="${root}/src/examples"/>
@@ -359,14 +361,85 @@
          <isset property="testcase" />
        </and>
     </condition>
-    <macro-test-runner test.file="${test.all.tests.file}"
-                       classpath="test.system.classpath"
-                       test.dir="${build.test.system}"
-                       fileset.dir="${hadoop.root}/src/contrib/${name}/src/test/system"
-                       hadoop.conf.dir.deployed="${hadoop.conf.dir.deployed}">
-    </macro-test-runner>
+     <property name="test.junit.jvmargs" value="-ea" />
+    <macro-system-test-runner test.file="${test.all.tests.file}"
+                     classpath="test.system.classpath"
+                     test.dir="${build.test.system}"
+                     fileset.dir="${hadoop.root}/src/contrib/${name}/src/test/system"
+                     hadoop.conf.dir.deployed="${hadoop.conf.dir.deployed}">
+  </macro-system-test-runner>
   </target>
 
+  <macrodef name="macro-system-test-runner">
+    <attribute name="test.file" />
+    <attribute name="classpath" />
+    <attribute name="test.dir" />
+    <attribute name="fileset.dir" />
+    <attribute name="hadoop.conf.dir.deployed" default="" />
+    <sequential>
+      <delete dir="@{test.dir}/data"/>
+      <mkdir dir="@{test.dir}/data"/>
+      <delete dir="@{test.dir}/logs"/>
+      <mkdir dir="@{test.dir}/logs"/>
+      <copy file="${test.src.dir}/hadoop-policy.xml"
+        todir="@{test.dir}/extraconf" />
+      <copy file="${test.src.dir}/fi-site.xml"
+        todir="@{test.dir}/extraconf" />
+      <junit showoutput="${test.output}"
+        printsummary="${test.junit.printsummary}"
+        haltonfailure="${test.junit.haltonfailure}"
+        fork="yes"
+        forkmode="${test.junit.fork.mode}"
+        maxmemory="${test.junit.maxmemory}"
+        dir="${basedir}" timeout="${test.timeout}"
+        errorProperty="tests.failed" failureProperty="tests.failed">
+        <jvmarg value="${test.junit.jvmargs}" />
+        <sysproperty key="java.net.preferIPv4Stack" value="true"/>
+        <sysproperty key="test.build.data" value="@{test.dir}/data"/>
+        <sysproperty key="test.tools.input.dir" value = "${test.tools.input.dir}"/>
+        <sysproperty key="test.cache.data" value="${test.cache.data}"/>
+        <sysproperty key="test.debug.data" value="${test.debug.data}"/>
+        <sysproperty key="hadoop.log.dir" value="@{test.dir}/logs"/>
+        <sysproperty key="test.src.dir" value="@{fileset.dir}"/>
+        <sysproperty key="taskcontroller-path" value="${taskcontroller-path}"/>
+        <sysproperty key="taskcontroller-ugi" value="${taskcontroller-ugi}"/>
+        <sysproperty key="test.build.extraconf" value="@{test.dir}/extraconf" />
+        <sysproperty key="hadoop.policy.file" value="hadoop-policy.xml"/>
+        <sysproperty key="java.library.path"
+          value="${build.native}/lib:${lib.dir}/native/${build.platform}"/>
+        <sysproperty key="install.c++.examples" value="${install.c++.examples}"/>
+        <syspropertyset dynamic="no">
+          <propertyref name="hadoop.tmp.dir"/>
+        </syspropertyset>
+        <!-- set compile.c++ in the child jvm only if it is set -->
+        <syspropertyset dynamic="no">
+          <propertyref name="compile.c++"/>
+        </syspropertyset>
+        <!-- Pass probability specifications to the spawn JVM -->
+        <syspropertyset id="FaultProbabilityProperties">
+          <propertyref regex="fi.*"/>
+        </syspropertyset>
+        <sysproperty key="test.system.hdrc.deployed.hadoopconfdir"
+                     value="@{hadoop.conf.dir.deployed}" />
+        <classpath refid="@{classpath}"/>
+        <formatter type="${test.junit.output.format}" />
+        <batchtest todir="@{test.dir}" unless="testcase">
+          <fileset dir="@{fileset.dir}"
+            excludes="**/${test.exclude}.java aop/** system/**">
+            <patternset>
+              <includesfile name="@{test.file}"/>
+            </patternset>
+          </fileset>
+        </batchtest>
+        <batchtest todir="@{test.dir}" if="testcase">
+          <fileset dir="@{fileset.dir}" includes="**/${testcase}.java"/>
+        </batchtest>
+      </junit>
+      <antcall target="checkfailure"/>
+    </sequential>
+  </macrodef>
+
+
   <target name="checkfailure" if="tests.failed">
     <touch file="${build.contrib.dir}/testsfailed"/>
     <fail unless="continueOnFailure">Contrib Tests failed!</fail>

+ 1 - 1
src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/DistributedCacheEmulator.java

@@ -360,7 +360,7 @@ class DistributedCacheEmulator {
    * @return true if the path provided is of a local file system based
    *              distributed cache file
    */
-  private boolean isLocalDistCacheFile(String filePath, String user,
+  protected static boolean isLocalDistCacheFile(String filePath, String user,
                                        boolean visibility) {
     return (!visibility && filePath.contains(user + "/.staging"));
   }

+ 1 - 1
src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/Gridmix.java

@@ -124,7 +124,7 @@ public class Gridmix extends Configured implements Tool {
     summarizer = new Summarizer(args);
   }
   
-  Gridmix() {
+  public Gridmix() {
     summarizer = new Summarizer();
   }
   

+ 0 - 75
src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/GridMixConfig.java

@@ -1,75 +0,0 @@
-/**
- * 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.mapred.gridmix;
-
-public class GridMixConfig {
-  /**
-   *  Gridmix logger mode.
-   */
-  public static final String GRIDMIX_LOG_MODE = 
-      "log4j.logger.org.apache.hadoop.mapred.gridmix";
-
-  /**
-   *  Gridmix output directory.
-   */
-  public static final String GRIDMIX_OUTPUT_DIR = 
-      "gridmix.output.directory";
-
-  /**
-   * Gridmix job type (LOADJOB/SLEEPJOB).
-   */
-  public static final String GRIDMIX_JOB_TYPE = 
-      "gridmix.job.type";
-
-  /**
-   *  Gridmix submission use queue.
-   */
-  public static final String GRIDMIX_JOB_SUBMISSION_QUEUE_IN_TRACE = 
-      "gridmix.job-submission.use-queue-in-trace";
-  
-  /**
-   *  Gridmix user resolver(RoundRobinUserResolver/
-   *  SubmitterUserResolver/EchoUserResolver).
-   */
-  public static final String GRIDMIX_USER_RESOLVER = 
-      "gridmix.user.resolve.class";
-
-  /**
-   *  Gridmix queue depth.
-   */
-  public static final String GRIDMIX_QUEUE_DEPTH = 
-      "gridmix.client.pending.queue.depth";
-  
-  /**
-   * Gridmix generate bytes per file.
-   */
-  public static final String GRIDMIX_BYTES_PER_FILE = 
-      "gridmix.gen.bytes.per.file";
-  
-  /**
-   *  Gridmix job submission policy(STRESS/REPLAY/SERIAL).
-   */
-  public static final String GRIDMIX_SUBMISSION_POLICY =
-      "gridmix.job-submission.policy";
-
-  /**
-   *  Gridmix minimum file size.
-   */
-  public static final String GRIDMIX_MINIMUM_FILE_SIZE =
-      "gridmix.min.file.size";
-}

+ 234 - 0
src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/GridmixSystemTestCase.java

@@ -0,0 +1,234 @@
+/**
+ * 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.mapred.gridmix;
+
+import org.apache.commons.logging.LogFactory;
+import org.apache.commons.logging.Log;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.test.system.MRCluster;
+import org.apache.hadoop.mapreduce.test.system.JTProtocol;
+import org.apache.hadoop.mapreduce.test.system.JTClient;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.gridmix.test.system.GridmixJobSubmission;
+import org.apache.hadoop.mapred.gridmix.test.system.GridmixJobVerification;
+import org.apache.hadoop.mapred.gridmix.test.system.GridMixRunMode;
+import org.apache.hadoop.mapred.gridmix.test.system.GridMixConfig;
+import org.apache.hadoop.mapred.gridmix.test.system.UtilsForGridmix;
+import org.apache.hadoop.mapred.gridmix.test.system.GridmixJobStory;
+import org.apache.hadoop.tools.rumen.ZombieJob;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.mapreduce.JobID;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+import java.util.Iterator;
+import java.util.Map;
+import java.util.List;
+import java.util.Set;
+import java.io.IOException;
+import org.junit.Assert;
+
+/**
+ * Run and verify the Gridmix jobs for given a trace.
+ */
+public class GridmixSystemTestCase {
+  private static final Log LOG = 
+      LogFactory.getLog(GridmixSystemTestCase.class);
+  protected static Configuration  conf = new Configuration();
+  protected static MRCluster cluster;
+  protected static int cSize;
+  protected static JTClient jtClient;
+  protected static JTProtocol rtClient;
+  protected static Path gridmixDir;
+  protected static Map<String, String> map;
+  protected static GridmixJobSubmission gridmixJS;
+  protected static GridmixJobVerification gridmixJV;
+  protected static List<JobID> jobids;
+  
+  @BeforeClass
+  public static void before() throws Exception {
+    String [] excludeExpList = {"java.net.ConnectException", 
+                                "java.io.IOException",
+                               };
+    cluster = MRCluster.createCluster(conf);
+    cluster.setExcludeExpList(excludeExpList);
+    cluster.setUp();
+    cSize = cluster.getTTClients().size();
+    jtClient = cluster.getJTClient();
+    rtClient = jtClient.getProxy();
+    gridmixDir = new Path("herriot-gridmix");
+    UtilsForGridmix.createDirs(gridmixDir, rtClient.getDaemonConf());
+    map = UtilsForGridmix.getMRTraces(rtClient.getDaemonConf());
+  }
+
+  @AfterClass
+  public static void after() throws Exception {
+    UtilsForGridmix.cleanup(gridmixDir, rtClient.getDaemonConf());
+    org.apache.hadoop.fs.FileUtil.fullyDelete(new java.io.File(System.
+        getProperty("java.io.tmpdir") + "/gridmix-st/"));
+    cluster.tearDown();
+
+    /* Clean up the proxy user directories if gridmix run with 
+      RoundRobinUserResovler mode.*/
+    // The below code commented due to a bug in mapreduce framework.
+    /*if (gridmixJV != null 
+       && gridmixJV.getJobUserResolver().contains("RoundRobin")) {
+       List<String> proxyUsers = 
+           UtilsForGridmix.listProxyUsers(gridmixJS.getJobConf(),
+           UserGroupInformation.getLoginUser().getShortUserName());
+       for (int index = 0; index < proxyUsers.size(); index++){
+         UtilsForGridmix.cleanup(new Path("hdfs:///user/" + 
+             proxyUsers.get(index)), 
+             rtClient.getDaemonConf());
+       }
+    }*/
+  }
+  
+  /**
+   * Run the gridmix with specified runtime parameters and 
+   * verify the jobs the after completion of execution.
+   * @param runtimeValues - common runtime arguments for gridmix.
+   * @param otherValues - test specific runtime arguments for gridmix.
+   * @param tracePath - path of a trace file.
+   * @throws Exception - if an exception occurs.
+   */
+  public static void runGridmixAndVerify(String[] runtimeValues, 
+     String [] otherValues, String tracePath) throws Exception {
+     runGridmixAndVerify(runtimeValues, otherValues, tracePath , 
+         GridMixRunMode.DATA_GENERATION_AND_RUN_GRIDMIX.getValue());
+  }
+  
+  /**
+   * Run the gridmix with specified runtime parameters and
+   * verify the jobs the after completion of execution.
+   * @param runtimeValues - common runtime arguments for gridmix.
+   * @param otherValues - test specific runtime arguments for gridmix.
+   * @param tracePath - path of a trace file.
+   * @param mode - 1 for data generation, 2 for run the gridmix and 3 for
+   * data generation and run the gridmix.
+   * @throws Exception - if an exception occurs.
+   */
+  public static void runGridmixAndVerify(String [] runtimeValues, 
+      String [] otherValues, String tracePath, int mode) throws Exception {
+    List<JobID> jobids = runGridmix(runtimeValues, otherValues, mode);
+    gridmixJV = new GridmixJobVerification(new Path(tracePath), 
+                                           gridmixJS.getJobConf(), jtClient);
+    gridmixJV.verifyGridmixJobsWithJobStories(jobids);  
+  }
+
+  /**
+   * Run the gridmix with user specified mode.
+   * @param runtimeValues - common runtime parameters for gridmix.
+   * @param otherValues - test specifix runtime parameters for gridmix.
+   * @param mode -  1 for data generation, 2 for run the gridmix and 3 for
+   * data generation and run the gridmix.
+   * @return - list of gridmix job ids.
+   * @throws Exception - if an exception occurs.
+   */
+  public static List<JobID> runGridmix(String[] runtimeValues, 
+     String[] otherValues, int mode) throws Exception {
+    gridmixJS = new GridmixJobSubmission(rtClient.getDaemonConf(),
+       jtClient, gridmixDir);
+    gridmixJS.submitJobs(runtimeValues, otherValues, mode);
+    LOG.info("Gridmix JobCount:" + gridmixJS.getGridmixJobCount());
+    List<JobID> jobids = 
+        UtilsForGridmix.listGridmixJobIDs(jtClient.getClient(), 
+                                          gridmixJS.getGridmixJobCount());
+    return jobids;
+  }
+  
+  /**
+   * get the trace file based on given regular expression.
+   * @param regExp - trace file file pattern. 
+   * @return - trace file as string.
+   * @throws IOException - if an I/O error occurs.
+   */
+  public static String getTraceFile(String regExp) throws IOException {
+    List<String> listTraces = UtilsForGridmix.listMRTraces(
+        rtClient.getDaemonConf());
+    Iterator<String> ite = listTraces.iterator();
+    while(ite.hasNext()) {
+      String traceFile = ite.next();
+      if (traceFile.indexOf(regExp)>=0) {
+        return traceFile;
+      }
+    }
+    return null;
+  }
+
+  /**
+   * Validate the task memory parameters.
+   * @param tracePath - trace file.
+   * @param isTraceHasHighRamJobs - true if trace has high ram job(s) 
+   *                                otherwise its false 
+   */
+  @SuppressWarnings("deprecation")
+  public static void validateTaskMemoryParamters(String tracePath,
+      boolean isTraceHasHighRamJobs) throws IOException {
+    if (isTraceHasHighRamJobs) {
+      GridmixJobStory gjs = new GridmixJobStory(new Path(tracePath),
+                                                rtClient.getDaemonConf());
+      Set<JobID> jobids = gjs.getZombieJobs().keySet();
+      boolean isHighRamFlag = false;
+      for (JobID jobid :jobids) {
+        ZombieJob zombieJob = gjs.getZombieJobs().get(jobid);
+        JobConf origJobConf = zombieJob.getJobConf();
+        int origMapFactor =
+            GridmixJobVerification.getMapFactor(origJobConf);
+        int origReduceFactor =
+            GridmixJobVerification.getReduceFactor(origJobConf);
+        if (origMapFactor >= 2 || origReduceFactor >= 2) {
+          isHighRamFlag = true;
+          long TaskMapMemInMB =
+              GridmixJobVerification.getScaledTaskMemInMB(
+                      GridMixConfig.JOB_MAP_MEMORY_MB,
+                      GridMixConfig.CLUSTER_MAP_MEMORY,
+                      origJobConf, rtClient.getDaemonConf());
+
+          long TaskReduceMemInMB =
+              GridmixJobVerification.getScaledTaskMemInMB(
+                      GridMixConfig.JOB_REDUCE_MEMORY_MB,
+                      GridMixConfig.CLUSTER_REDUCE_MEMORY,
+                      origJobConf, rtClient.getDaemonConf());
+          long taskMapLimitInMB =
+              conf.getLong(GridMixConfig.CLUSTER_MAX_MAP_MEMORY,
+                           JobConf.DISABLED_MEMORY_LIMIT);
+
+          long taskReduceLimitInMB =
+              conf.getLong(GridMixConfig.CLUSTER_MAX_REDUCE_MEMORY,
+                           JobConf.DISABLED_MEMORY_LIMIT);
+
+          GridmixJobVerification.verifyMemoryLimits(TaskMapMemInMB,
+                                                    taskMapLimitInMB);
+          GridmixJobVerification.verifyMemoryLimits(TaskReduceMemInMB,
+                                                    taskReduceLimitInMB);
+        }
+      }
+      Assert.assertTrue("Trace doesn't have atleast one high ram job.",
+                        isHighRamFlag);
+    }
+  }
+
+  public static boolean isLocalDistCache(String fileName, String userName, 
+                                         boolean visibility) {
+    return DistributedCacheEmulator.isLocalDistCacheFile(fileName, 
+                                                         userName, visibility);
+  }
+}
+

+ 107 - 0
src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestCPUEmulationForMapsAndReducesWithCustomInterval.java

@@ -0,0 +1,107 @@
+/**
+ * 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.mapred.gridmix;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.mapred.gridmix.test.system.GridMixConfig;
+import org.apache.hadoop.mapred.gridmix.test.system.GridMixRunMode;
+import org.apache.hadoop.mapred.gridmix.test.system.UtilsForGridmix;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.junit.Test;
+import org.junit.Assert;
+
+/**
+ * Test cpu emulation with default interval for gridmix jobs 
+ * against different input data, submission policies and user resolvers.
+ * Verify the cpu resource metrics of both maps and reduces phase of
+ * Gridmix jobs with their corresponding original job in the input trace.
+ */
+public class TestCPUEmulationForMapsAndReducesWithCustomInterval 
+    extends GridmixSystemTestCase { 
+  private static final Log LOG = LogFactory
+      .getLog("TestCPUEmulationForMapsAndReducesWithCustomInterval.class");
+  int execMode = GridMixRunMode.DATA_GENERATION_AND_RUN_GRIDMIX.getValue();
+
+ /**
+   * Generate compressed input and run {@link Gridmix} by turning on the 
+   * cpu emulation feature with default setting. The {@link Gridmix} 
+   * should use the following runtime parameters.
+   * Submission Policy : STRESS, UserResovler: RoundRobinUserResolver. 
+   * Once the {@link Gridmix} run is complete, verify cpu resource metrics of 
+   * {@link Gridmix} jobs with their corresponding original job in a trace.
+   * @throws Exception - if an error occurs.
+   */
+  @Test
+  public void  testCPUEmulationForMapsAndReducesWithCompressedInputCase7() 
+      throws Exception {
+    final long inputSizeInMB = 1024 * 7;
+    String tracePath = getTraceFile("cpu_emul_case2");
+    Assert.assertNotNull("Trace file not found!", tracePath);
+    String [] runtimeValues = 
+            { "LOADJOB",
+              RoundRobinUserResolver.class.getName(),
+              "STRESS",
+              inputSizeInMB + "m",
+              "file://" + UtilsForGridmix.getProxyUsersFile(conf),
+              tracePath};
+
+    String [] otherArgs = { 
+            "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=false",
+            "-D", JobContext.JOB_CANCEL_DELEGATION_TOKEN + "=false",
+            "-D", GridMixConfig.GRIDMIX_CPU_CUSTOM_INTERVAL + "=0.35F",
+            "-D", GridMixConfig.GRIDMIX_CPU_EMULATION + "=" + 
+                  GridMixConfig.GRIDMIX_CPU_EMULATION_PLUGIN};
+
+    runGridmixAndVerify(runtimeValues, otherArgs, tracePath, execMode); 
+  }
+  
+  /**
+   * Generate uncompressed input and run {@link Gridmix} by turning on the 
+   * cpu emulation feature with default setting. The {@link Gridmix} 
+   * should use the following runtime parameters.
+   * Submission Policy : SERIAL, UserResovler: SubmitterUserResolver 
+   * Once the {@link Gridmix} run is complete, verify cpu resource metrics of 
+   * {@link Gridmix} jobs with their corresponding original job in a trace.
+   * @throws Exception - if an error occurs.
+   */
+  @Test
+  public void  testCPUEmulatonForMapsAndReducesWithUncompressedInputCase8() 
+      throws Exception {
+    final long inputSizeInMB = cSize * 300;
+    String tracePath = getTraceFile("cpu_emul_case2");
+    Assert.assertNotNull("Trace file not found.", tracePath);
+    String [] runtimeValues = 
+              { "LOADJOB", 
+                SubmitterUserResolver.class.getName(), 
+                "SERIAL", 
+                inputSizeInMB + "m",
+                tracePath};
+
+    String [] otherArgs = {
+            "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=false",
+            "-D", GridMixConfig.GRIDMIX_COMPRESSION_ENABLE + "=false",
+            "-D", JobContext.JOB_CANCEL_DELEGATION_TOKEN + "=false",
+            "-D", GridMixConfig.GRIDMIX_CPU_CUSTOM_INTERVAL + "=0.4F",
+            "-D", GridMixConfig.GRIDMIX_CPU_EMULATION + "=" + 
+                  GridMixConfig.GRIDMIX_CPU_EMULATION_PLUGIN     };
+
+    runGridmixAndVerify(runtimeValues, otherArgs, tracePath, execMode); 
+  }
+}
+

+ 106 - 0
src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestCPUEmulationForMapsAndReducesWithDefaultInterval.java

@@ -0,0 +1,106 @@
+/**
+ * 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.mapred.gridmix;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.mapred.gridmix.test.system.GridMixConfig;
+import org.apache.hadoop.mapred.gridmix.test.system.GridMixRunMode;
+import org.apache.hadoop.mapred.gridmix.test.system.UtilsForGridmix;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.junit.Test;
+import org.junit.Assert;
+
+/**
+ * Test cpu emulation with default interval for gridmix jobs 
+ * against different input data, submission policies and user resolvers.
+ * Verify the cpu resource metrics for both maps and reduces of
+ * Gridmix jobs with their corresponding original job in the input trace.
+ */
+public class TestCPUEmulationForMapsAndReducesWithDefaultInterval 
+                                            extends GridmixSystemTestCase { 
+  private static final Log LOG = 
+          LogFactory.getLog(
+              "TestCPUEmulationForMapsAndReducesWithDefaultInterval.class");
+  int execMode = GridMixRunMode.DATA_GENERATION_AND_RUN_GRIDMIX.getValue();
+
+ /**
+   * Generate compressed input and run {@link Gridmix} by turning on the 
+   * cpu emulation feature with default setting. The {@link Gridmix} 
+   * should use the following runtime parameters.
+   * Submission Policy : REPLAY, UserResovler: RoundRobinUserResolver. 
+   * Once the {@link Gridmix} run is complete, verify cpu resource metrics of 
+   * {@link Gridmix} jobs with their corresponding original jobs in the trace.
+   * @throws Exception - if an error occurs.
+   */
+  @Test
+  public void  testCPUEmulationForMapsAndReducesWithCompressedInputCase5() 
+      throws Exception {
+    final long inputSizeInMB = 7168;
+    String tracePath = getTraceFile("cpu_emul_case2");
+    Assert.assertNotNull("Trace file not found!", tracePath);
+    String [] runtimeValues = 
+            { "LOADJOB",
+              RoundRobinUserResolver.class.getName(),
+              "REPLAY",
+              inputSizeInMB + "m",
+              "file://" + UtilsForGridmix.getProxyUsersFile(conf),
+              tracePath};
+
+    String [] otherArgs = { 
+            "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=false",
+            "-D", JobContext.JOB_CANCEL_DELEGATION_TOKEN + "=false",
+            "-D", GridMixConfig.GRIDMIX_CPU_EMULATION + "=" + 
+                  GridMixConfig.GRIDMIX_CPU_EMULATION_PLUGIN};
+
+    runGridmixAndVerify(runtimeValues, otherArgs, tracePath, execMode); 
+  }
+  
+  /**
+   * Generate uncompressed input and run {@link Gridmix} by turning on the 
+   * cpu emulation feature with default settings. The {@link Gridmix} 
+   * should use the following runtime parameters.
+   * Submission Policy : STRESS, UserResovler: SubmitterUserResolver 
+   * Once the Gridmix run is complete, verify cpu resource metrics of 
+   * {@link Gridmix} jobs with their corresponding original jobs in the trace.
+   * @throws Exception - if an error occurs.
+   */
+  @Test
+  public void  testCPUEmulatonForMapsAndReducesWithUncompressedInputCase6() 
+      throws Exception {
+    final long inputSizeInMB = cSize * 400;
+    String tracePath = getTraceFile("cpu_emul_case2");
+    Assert.assertNotNull("Trace file not found!", tracePath);
+    String [] runtimeValues = 
+              { "LOADJOB", 
+                SubmitterUserResolver.class.getName(), 
+                "STRESS",
+                inputSizeInMB + "m",
+                tracePath};
+
+    String [] otherArgs = {
+            "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=false",
+            "-D", GridMixConfig.GRIDMIX_COMPRESSION_ENABLE + "=false",
+            "-D", JobContext.JOB_CANCEL_DELEGATION_TOKEN + "=false",
+            "-D", GridMixConfig.GRIDMIX_CPU_EMULATION + "=" + 
+                  GridMixConfig.GRIDMIX_CPU_EMULATION_PLUGIN     };
+    
+    runGridmixAndVerify(runtimeValues, otherArgs, tracePath, execMode); 
+  }
+}
+

+ 104 - 0
src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestCPUEmulationForMapsWithCustomInterval.java

@@ -0,0 +1,104 @@
+/**
+ * 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.mapred.gridmix;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.mapred.gridmix.test.system.GridMixConfig;
+import org.apache.hadoop.mapred.gridmix.test.system.GridMixRunMode;
+import org.apache.hadoop.mapred.gridmix.test.system.UtilsForGridmix;
+import org.junit.Test;
+import org.junit.Assert;
+
+/**
+ * Test the {@link Gridmix} cpu emulation with custom interval for 
+ * gridmix jobs against different input data, submission policies and 
+ * user resolvers. Verify the map phase cpu metrics of gridmix jobs 
+ * against their original job in the trace. 
+ */
+public class TestCPUEmulationForMapsWithCustomInterval 
+                                            extends GridmixSystemTestCase {
+  private static final Log LOG = 
+      LogFactory.getLog("TestCPUEmulationForMapsWithCustomInterval.class");
+  int execMode = GridMixRunMode.DATA_GENERATION_AND_RUN_GRIDMIX.getValue();
+
+  /**
+   * Generate compressed input and run {@link Gridmix} by turning on 
+   * cpu emulation feature with custom setting. The {@link Gridmix} should 
+   * use the following runtime parameters while running gridmix jobs.
+   * Submission Policy : STRESS, User Resolver Mode : SumitterUserResolver
+   * Once {@link Gridmix} run is complete, verify maps phase cpu resource 
+   * metrics of {@link Gridmix} jobs with their corresponding original
+   * in the trace.
+   * @throws Exception - if an error occurs.
+   */
+  @Test
+  public void  testCPUEmulatonForMapsWithCompressedInputCase3() 
+      throws Exception { 
+    final long inputSizeInMB = 1024 * 7;
+    String tracePath = getTraceFile("cpu_emul_case1");
+    Assert.assertNotNull("Trace file not found!", tracePath);
+    String [] runtimeValues = {"LOADJOB",
+                               SubmitterUserResolver.class.getName(),
+                               "STRESS",
+                               inputSizeInMB + "m",
+                               tracePath};
+
+    String [] otherArgs = {
+        "-D", GridMixConfig.GRIDMIX_CPU_EMULATION + "=" +
+              GridMixConfig.GRIDMIX_CPU_EMULATION_PLUGIN,
+        "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=false",
+        "-D", GridMixConfig.GRIDMIX_CPU_CUSTOM_INTERVAL + "=0.25F"};
+
+    runGridmixAndVerify(runtimeValues, otherArgs, tracePath, execMode);
+  }
+
+  /**
+   * Generate uncompressed input and run {@link Gridmix} by turning on 
+   * cpu emulation feature with custom settings. The {@link Gridmix} 
+   * should use the following runtime paramters while running gridmix jobs.
+   * Submission Policy: REPLAY  User Resolver Mode: RoundRobinUserResolver
+   * Once {@link Gridmix} run is complete, verify the map phase cpu resource 
+   * metrics of {@link Gridmix} jobs with their corresponding jobs
+   * in the original trace.
+   * @throws Exception - if an error occurs.
+   */
+  @Test
+  public void testCPUEmulatonForMapsUnCompressedInputCase4() 
+      throws Exception { 
+    final long inputSizeInMB = cSize * 200;
+    String tracePath = getTraceFile("cpu_emul_case1");
+    Assert.assertNotNull("Trace file not found!", tracePath);
+    String [] runtimeValues = 
+           {"LOADJOB",
+            RoundRobinUserResolver.class.getName(),
+            "REPLAY",
+            inputSizeInMB + "m",
+            "file://" + UtilsForGridmix.getProxyUsersFile(conf),
+            tracePath};
+
+    String [] otherArgs = {
+        "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=false",
+        "-D", GridMixConfig.GRIDMIX_COMPRESSION_ENABLE + "=false",
+        "-D", GridMixConfig.GRIDMIX_CPU_EMULATION + "=" + 
+              GridMixConfig.GRIDMIX_CPU_EMULATION_PLUGIN,
+        "-D", GridMixConfig.GRIDMIX_CPU_CUSTOM_INTERVAL + "=0.35F"};
+
+    runGridmixAndVerify(runtimeValues, otherArgs, tracePath, execMode);
+  }
+}

+ 103 - 0
src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestCPUEmulationForMapsWithDefaultInterval.java

@@ -0,0 +1,103 @@
+/**
+ * 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.mapred.gridmix;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.mapred.gridmix.test.system.GridMixConfig;
+import org.apache.hadoop.mapred.gridmix.test.system.GridMixRunMode;
+import org.apache.hadoop.mapred.gridmix.test.system.UtilsForGridmix;
+import org.junit.Test;
+import org.junit.Assert;
+
+/**
+ * Test the {@link Gridmix} cpu emulation with default settings for 
+ * gridmix jobs against different input data, submission policies and 
+ * user resolvers. Verify the map phase cpu metrics of gridmix jobs 
+ * against their original jobs in the trace. 
+ */
+public class TestCPUEmulationForMapsWithDefaultInterval 
+    extends GridmixSystemTestCase { 
+  private static final Log LOG = 
+      LogFactory.getLog("TestCPUEmulationForMapsWithDefaultInterval.class");
+  int execMode = GridMixRunMode.DATA_GENERATION_AND_RUN_GRIDMIX.getValue();
+
+  /**
+   * Generate compressed input and run {@link Gridmix} by turning on cpu 
+   * emulation feature with default settings. The {@link Gridmix} should 
+   * use the following runtime parameters while running the gridmix jobs.
+   * Submission Policy: STRESS, UserResolver: SubmitterUserResolver. 
+   * Once the {@link Gridmix} run is complete, verify map phase cpu metrics of 
+   * {@link Gridmix} jobs with their corresponding original job in a trace.
+   * @throws Exception - if an error occurs.
+   */
+  @Test
+  public void testCPUEmulatonForMapsWithCompressedInputCase1() 
+      throws Exception {
+    final long inputSizeInMB = 1024 * 6;
+    String tracePath = getTraceFile("cpu_emul_case1");
+    Assert.assertNotNull("Trace file not found!", tracePath);
+    String [] runtimeValues = { "LOADJOB", 
+                                SubmitterUserResolver.class.getName(), 
+                                "STRESS", 
+                                inputSizeInMB + "m", 
+                                tracePath};
+
+    String [] otherArgs = { 
+            "-D", GridMixConfig.GRIDMIX_CPU_EMULATION + "=" + 
+                  GridMixConfig.GRIDMIX_CPU_EMULATION_PLUGIN,
+            "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=false",
+            "-D", GridMixConfig.GRIDMIX_HIGH_RAM_JOB_ENABLE + "=false"};
+
+    runGridmixAndVerify(runtimeValues, otherArgs, tracePath, execMode); 
+  }
+  
+  /**
+   * Generate uncompressed input and run {@link Gridmix} by turning on 
+   * cpu emulation feature with default settings. The {@link Gridmix} 
+   * should use the following runtime parameters while running Gridmix jobs.
+   * Submission Policy: REPLAY, UserResolver: RoundRobinUserResolver
+   * Once the Gridmix run is complete, verify cpu resource metrics of 
+   * {@link Gridmix} jobs with their corresponding original job in a trace.
+   * @throws Exception - if an error occurs.
+   */
+  @Test
+  public void testCPUEmulatonForMapsWithUnCompressedInputCase2() 
+      throws Exception { 
+    final long inputSizeInMB = cSize * 200;
+    String tracePath = getTraceFile("cpu_emul_case1");
+    Assert.assertNotNull("Trace file not found!", tracePath);
+    String [] runtimeValues = 
+            { "LOADJOB",
+              RoundRobinUserResolver.class.getName(),
+              "REPLAY",
+              inputSizeInMB + "m",
+              "file://" + UtilsForGridmix.getProxyUsersFile(conf),
+              tracePath};
+
+    String [] otherArgs = { 
+            "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=false",
+            "-D", GridMixConfig.GRIDMIX_COMPRESSION_ENABLE + "=false",
+            "-D", GridMixConfig.GRIDMIX_HIGH_RAM_JOB_ENABLE + "=false",
+            "-D", GridMixConfig.GRIDMIX_CPU_EMULATION + "=" + 
+                  GridMixConfig.GRIDMIX_CPU_EMULATION_PLUGIN};
+
+    runGridmixAndVerify(runtimeValues, otherArgs, tracePath, 
+           GridMixRunMode.DATA_GENERATION_AND_RUN_GRIDMIX.getValue());
+  }
+}

+ 96 - 0
src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestCompressionEmulationEnableForAllTypesOfJobs.java

@@ -0,0 +1,96 @@
+/**
+ * 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.mapred.gridmix;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.mapred.gridmix.test.system.GridMixConfig;
+import org.apache.hadoop.mapred.gridmix.test.system.GridMixRunMode;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Verify the compression emulation for all the jobs in the trace 
+ * irrespective of compressed inputs.
+ */
+public class TestCompressionEmulationEnableForAllTypesOfJobs 
+    extends GridmixSystemTestCase { 
+  private static final Log LOG = 
+      LogFactory.getLog(
+          "TestCompressionEmulationEnableForAllTypesOfJobs.class");
+
+  /**
+   *  Generate compressed input data and verify the compression emulation
+   *  for all the jobs in the trace irrespective of whether the original
+   *  job uses the compressed input or not.Also use the custom compression
+   *  ratios for map input, map output and reduce output.
+   * @throws Exception - if an error occurs.
+   */
+  @Test
+  public void testInputCompressionEmualtionEnableForAllJobsWithCustomRatios() 
+      throws Exception { 
+    final long inputSizeInMB = 1024 * 6;
+    final String tracePath = getTraceFile("compression_case4_trace");
+    Assert.assertNotNull("Trace file has not found.", tracePath);
+    final String [] runtimeValues = {"LOADJOB",
+                                     SubmitterUserResolver.class.getName(),
+                                     "REPLAY",
+                                     inputSizeInMB + "m",
+                                     tracePath};
+
+    final String [] otherArgs = {
+        "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=false",
+        "-D", GridMixConfig.GRIDMIX_COMPRESSION_ENABLE + "=true",
+        "-D", GridmixJob.GRIDMIX_HIGHRAM_EMULATION_ENABLE + "=false",
+        "-D", GridMixConfig.GRIDMIX_INPUT_DECOMPRESS_ENABLE + "=true",
+        "-D", GridMixConfig.GRIDMIX_INPUT_COMPRESS_RATIO + "=0.46",
+        "-D", GridMixConfig.GRIDMIX_INTERMEDIATE_COMPRESSION_RATIO + "=0.35",
+        "-D", GridMixConfig.GRIDMIX_OUTPUT_COMPRESSION_RATIO + "=0.36"
+    };
+
+    runGridmixAndVerify(runtimeValues, otherArgs, tracePath, 
+        GridMixRunMode.DATA_GENERATION_AND_RUN_GRIDMIX.getValue());
+  }
+  
+  /**
+   *  Use existing compressed input data and turn off the compression 
+   *  emulation. Verify the compression emulation whether it uses 
+   *  by the jobs or not.
+   * @throws Exception - if an error occurs.
+   */
+  @Test
+  public void testInputCompressionEmulationDisableAllJobs() 
+      throws Exception { 
+     final String tracePath = getTraceFile("compression_case4_trace");
+     Assert.assertNotNull("Trace file has not found.", tracePath);
+     final String [] runtimeValues = {"LOADJOB",
+                                      SubmitterUserResolver.class.getName(),
+                                      "SERIAL",
+                                      tracePath};
+
+    final String [] otherArgs = {
+        "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=false",
+        "-D", GridmixJob.GRIDMIX_HIGHRAM_EMULATION_ENABLE + "=false",
+        "-D", GridMixConfig.GRIDMIX_COMPRESSION_ENABLE + "=false"
+    };
+
+    runGridmixAndVerify(runtimeValues, otherArgs, tracePath, 
+                        GridMixRunMode.RUN_GRIDMIX.getValue());
+  }  
+}
+

+ 98 - 0
src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestCompressionEmulationForCompressInAndUncompressOut.java

@@ -0,0 +1,98 @@
+/**
+ * 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.mapred.gridmix;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.mapred.gridmix.test.system.GridMixConfig;
+import org.apache.hadoop.mapred.gridmix.test.system.GridMixRunMode;
+import org.apache.hadoop.mapred.gridmix.test.system.UtilsForGridmix;
+import org.junit.Assert;
+import org.junit.Test;
+/**
+ * Verify the gridmix jobs compression ratio's of input, 
+ * intermediate input and with default/custom ratios.Also verify
+ * the compressed output file format is enabled or not.
+ *
+ */
+public class TestCompressionEmulationForCompressInAndUncompressOut 
+    extends GridmixSystemTestCase { 
+  private static final Log LOG = 
+      LogFactory.getLog(
+          "TestCompressionEmulationForCompressInAndUncompressOut.class");
+  final long inputSizeInMB = 1024 * 6;
+
+  /**
+   * Generate a compressed input data and verify the compression ratios 
+   * of map input and map output against default compression ratios 
+   * and also verify the whether the compressed output file output format 
+   * is enabled or not.
+   * @throws Exception -if an error occurs.
+   */
+  @Test
+  public void testCompressionEmulationOfCompressedInputWithDefaultRatios() 
+      throws Exception {
+    final String tracePath = getTraceFile("compression_case2_trace");
+    Assert.assertNotNull("Trace file has not found.", tracePath);
+    final String [] runtimeValues = {"LOADJOB",
+                                     SubmitterUserResolver.class.getName(),
+                                     "STRESS",
+                                     inputSizeInMB + "m",
+                                     tracePath};
+
+    final String [] otherArgs = {
+        "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=false",
+        "-D", GridmixJob.GRIDMIX_HIGHRAM_EMULATION_ENABLE + "=false",
+        "-D", GridMixConfig.GRIDMIX_COMPRESSION_ENABLE + "=true"
+    };
+
+    runGridmixAndVerify(runtimeValues, otherArgs, tracePath, 
+        GridMixRunMode.DATA_GENERATION_AND_RUN_GRIDMIX.getValue());
+  }
+  
+  /**
+   * Use existing compressed input data and verify the compression ratios 
+   * of input and intermediate input against custom compression ratios 
+   * and also verify the compressed output file output format is enabled or not.
+   * @throws Exception -if an error occurs.
+   */
+  @Test
+  public void testCompressionEmulationOfCompressedInputWithCustomRatios() 
+      throws Exception {
+    final String tracePath = getTraceFile("compression_case2_trace");
+    Assert.assertNotNull("Trace file has not found.", tracePath);
+    UtilsForGridmix.cleanup(gridmixDir, rtClient.getDaemonConf());
+    final String [] runtimeValues = {"LOADJOB",
+                                     SubmitterUserResolver.class.getName(),
+                                     "STRESS",
+                                     inputSizeInMB + "m",
+                                     tracePath};
+
+    final String [] otherArgs = {
+        "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=false",
+        "-D", GridMixConfig.GRIDMIX_COMPRESSION_ENABLE + "=true",
+        "-D", GridMixConfig.GRIDMIX_INPUT_DECOMPRESS_ENABLE + "=true",
+        "-D", GridmixJob.GRIDMIX_HIGHRAM_EMULATION_ENABLE + "=false",
+        "-D", GridMixConfig.GRIDMIX_INPUT_COMPRESS_RATIO + "=0.58",
+        "-D", GridMixConfig.GRIDMIX_INTERMEDIATE_COMPRESSION_RATIO + "=0.42"
+    };
+
+    runGridmixAndVerify(runtimeValues, otherArgs, tracePath, 
+        GridMixRunMode.DATA_GENERATION_AND_RUN_GRIDMIX.getValue());
+  }
+}
+

+ 93 - 0
src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestCompressionEmulationForUncompressInAndCompressOut.java

@@ -0,0 +1,93 @@
+/**
+ * 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.mapred.gridmix;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.mapred.gridmix.test.system.GridMixConfig;
+import org.apache.hadoop.mapred.gridmix.test.system.GridMixRunMode;
+import org.apache.hadoop.mapred.gridmix.test.system.UtilsForGridmix;
+import org.junit.Assert;
+import org.junit.Test;
+/**
+ * Verify the gridmix jobs compression ratio's of reduce output and 
+ * with default and custom ratios.
+ */
+public class TestCompressionEmulationForUncompressInAndCompressOut
+   extends GridmixSystemTestCase { 
+   private static final Log LOG = 
+       LogFactory.getLog(
+           "TestCompressionEmulationForUncompressInAndCompressOut.class");
+   final long inputSizeInMB = 1024 * 6;
+
+  /**
+   * Generate a uncompressed input data and verify the compression ratios 
+   * of reduce output against default output compression ratio.
+   * @throws Exception -if an error occurs.
+   */
+  @Test
+  public void testCompressionEmulationOfCompressedOuputWithDefaultRatios() 
+      throws Exception { 
+    final String tracePath = getTraceFile("compression_case3_trace");
+    Assert.assertNotNull("Trace file has not found.", tracePath);
+    final String [] runtimeValues = 
+                     {"LOADJOB",
+                      RoundRobinUserResolver.class.getName(),
+                      "REPLAY",
+                      inputSizeInMB + "m",
+                      "file://" + UtilsForGridmix.getProxyUsersFile(conf),
+                      tracePath};
+
+    final String [] otherArgs = {
+        "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=false",
+        "-D", GridmixJob.GRIDMIX_HIGHRAM_EMULATION_ENABLE + "=false",
+        "-D", GridMixConfig.GRIDMIX_COMPRESSION_ENABLE + "=true"
+    };
+
+    runGridmixAndVerify(runtimeValues, otherArgs, tracePath,
+        GridMixRunMode.DATA_GENERATION_AND_RUN_GRIDMIX.getValue());
+  }
+  
+  /**
+   * Use existing uncompressed input data and verify the compression ratio 
+   * of reduce output against custom output compression ratio and also verify 
+   * the compression output file output format.
+   * @throws Exception -if an error occurs.
+   */
+  @Test
+  public void testCompressionEmulationOfCompressedOutputWithCustomRatios() 
+      throws Exception {
+    final String tracePath = getTraceFile("compression_case3_trace");
+    Assert.assertNotNull("Trace file has not found.", tracePath);
+    UtilsForGridmix.cleanup(gridmixDir, rtClient.getDaemonConf());
+    final String [] runtimeValues = { "LOADJOB",
+                                      SubmitterUserResolver.class.getName(),
+                                      "STRESS",
+                                      inputSizeInMB + "m",
+                                      tracePath };
+
+    final String [] otherArgs = { 
+        "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=false",
+        "-D", GridMixConfig.GRIDMIX_COMPRESSION_ENABLE + "=true",
+        "-D", GridmixJob.GRIDMIX_HIGHRAM_EMULATION_ENABLE + "=false",
+        "-D", GridMixConfig.GRIDMIX_OUTPUT_COMPRESSION_RATIO + "=0.38"
+    };
+
+    runGridmixAndVerify(runtimeValues, otherArgs, tracePath, 
+        GridMixRunMode.DATA_GENERATION_AND_RUN_GRIDMIX.getValue());
+  }
+}

+ 65 - 0
src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestDisableGridmixEmulationOfHighRam.java

@@ -0,0 +1,65 @@
+/**
+ * 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.mapred.gridmix;
+
+import org.apache.commons.logging.LogFactory;
+import org.apache.commons.logging.Log;
+import org.apache.hadoop.mapred.gridmix.test.system.GridMixConfig;
+import org.apache.hadoop.mapred.gridmix.GridmixJob;
+import org.junit.Test;
+import org.junit.Assert;
+
+/**
+ * Run the {@link Gridmix} with a high ram jobs trace by disabling the 
+ * emulation of high ram  and verify each {@link Gridmix} job 
+ * whether it honors the high ram or not. In disable mode it should 
+ * should not honor the high ram and run it as a normal job.
+ */
+public class TestDisableGridmixEmulationOfHighRam 
+    extends GridmixSystemTestCase { 
+  private static final Log LOG = 
+      LogFactory.getLog("TestDisableGridmixEmulationOfHighRam.class");
+
+  /**
+   * Generate input data and run {@link Gridmix} with a high ram jobs trace 
+   * as a load job and STRESS submission policy in a SubmitterUserResolver 
+   * mode. Verify each {@link Gridmix} job whether it honors the 
+   * high ram or not after completion of execution. In disable mode the
+   * jobs should not honor the high ram. 
+   * @throws Exception - if an error occurs.
+   */
+  @Test
+  public void testEmulationOfHighRamForReducersOfMRJobs() 
+      throws Exception { 
+    final long inputSizeInMB = cSize * 250;
+    String tracePath = getTraceFile("highram_mr_jobs_case3");
+    Assert.assertNotNull("Trace file has not found.", tracePath);
+    String [] runtimeValues = {"LOADJOB",
+                               SubmitterUserResolver.class.getName(),
+                               "SERIAL",
+                               inputSizeInMB + "m",
+                               tracePath};
+
+    String [] otherArgs = {
+               "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=false", 
+               "-D", GridMixConfig.GRIDMIX_COMPRESSION_ENABLE + "=false", 
+               "-D", GridmixJob.GRIDMIX_HIGHRAM_EMULATION_ENABLE + "=false"};
+
+    runGridmixAndVerify(runtimeValues, otherArgs, tracePath);
+  }
+}

+ 95 - 0
src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestEmulationOfHDFSAndLocalFSDCFiles.java

@@ -0,0 +1,95 @@
+/**
+ * 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.mapred.gridmix;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.mapred.gridmix.test.system.GridMixConfig;
+import org.apache.hadoop.mapred.gridmix.test.system.GridMixRunMode;
+import org.apache.hadoop.mapred.gridmix.test.system.UtilsForGridmix;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Verify the emulation of HDFS and Local FS distributed cache files against
+ * the given input trace file.
+ */
+public class TestEmulationOfHDFSAndLocalFSDCFiles extends 
+    GridmixSystemTestCase {
+  private static final Log LOG = 
+      LogFactory.getLog("TestEmulationOfLocalFSDCFiles.class");
+
+  /**
+   * Generate the input data and distributed cache files for HDFS and 
+   * local FS. Verify the gridmix emulation of HDFS and Local FS 
+   * distributed cache files in RoundRobinUserResolver mode with STRESS
+   * submission policy.
+   * @throws Exception - if an error occurs.
+   */
+  @Test
+  public void testGenerateDataEmulateHDFSAndLocalFSDCFiles() 
+     throws Exception  {
+    final long inputSizeInMB = 1024 * 6;
+    final String tracePath = getTraceFile("distcache_case8_trace");
+    Assert.assertNotNull("Trace file has not found.", tracePath);
+    final String [] runtimeValues = 
+                     {"LOADJOB",
+                      RoundRobinUserResolver.class.getName(),
+                      "STRESS",
+                      inputSizeInMB + "m",
+                      "file://" + UtilsForGridmix.getProxyUsersFile(conf),
+                      tracePath};
+
+    final String [] otherArgs = {
+       "-D", JobContext.JOB_CANCEL_DELEGATION_TOKEN + "=false",
+       "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=true",
+       "-D", GridmixJob.GRIDMIX_HIGHRAM_EMULATION_ENABLE + "=false",
+       "-D", GridMixConfig.GRIDMIX_COMPRESSION_ENABLE + "=false"
+    };
+    runGridmixAndVerify(runtimeValues, otherArgs, tracePath, 
+        GridMixRunMode.DATA_GENERATION_AND_RUN_GRIDMIX.getValue());
+  }
+  
+  /**
+   * Use existing input and distributed cache files for HDFS and
+   * local FS. Verify the gridmix emulation of HDFS and Local FS
+   * distributed cache files in SubmitterUserResolver mode with REPLAY
+   * submission policy.
+   * @throws Exception - if an error occurs.
+   */
+  @Test
+  public void testEmulationOfHDFSAndLocalFSDCFiles() 
+     throws Exception  {
+    final String tracePath = getTraceFile("distcache_case8_trace");
+    Assert.assertNotNull("Trace file has not found.", tracePath);
+    final String [] runtimeValues ={"LOADJOB",
+                                    SubmitterUserResolver.class.getName(),
+                                    "STRESS",
+                                    tracePath};
+
+    final String [] otherArgs = { 
+       "-D", JobContext.JOB_CANCEL_DELEGATION_TOKEN + "=false",
+       "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=true",
+       "-D", GridmixJob.GRIDMIX_HIGHRAM_EMULATION_ENABLE + "=false",
+       "-D", GridMixConfig.GRIDMIX_COMPRESSION_ENABLE + "=false"
+    };
+    runGridmixAndVerify(runtimeValues, otherArgs, tracePath, 
+                        GridMixRunMode.RUN_GRIDMIX.getValue());
+  }
+}

+ 91 - 0
src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestEmulationOfHDFSDCFileUsesMultipleJobs.java

@@ -0,0 +1,91 @@
+/**
+ * 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.mapred.gridmix;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.mapred.gridmix.test.system.GridMixConfig;
+import org.apache.hadoop.mapred.gridmix.test.system.GridMixRunMode;
+import org.apache.hadoop.mapred.gridmix.test.system.UtilsForGridmix;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Verify the Gridmix emulation of HDFS distributed cache file which uses 
+ * different jobs that are submitted with different users.
+ */
+public class TestEmulationOfHDFSDCFileUsesMultipleJobs extends 
+    GridmixSystemTestCase {
+  private static final Log LOG = 
+      LogFactory.getLog("TestEmulationOfHDFSDCFileUsesMultipleJobs.class");
+
+  /**
+   * Generate the input data and HDFS distributed cache file based 
+   * on given input trace. Verify the Gridmix emulation of HDFS
+   * distributed cache file in RoundRobinResolver mode with 
+   * STRESS submission policy.
+   * @throws Exception - if an error occurs.
+   */
+  @Test
+  public void testGenerateAndEmulationOfHDFSDCFile() 
+     throws Exception { 
+    final long inputSizeInMB = 1024 * 6;
+    final String tracePath = getTraceFile("distcache_case9_trace");
+    Assert.assertNotNull("Trace file has not found.", tracePath);
+    final String [] runtimeValues = 
+                     {"LOADJOB",
+                      RoundRobinUserResolver.class.getName(),
+                      "STRESS",
+                      inputSizeInMB + "m",
+                      "file://" + UtilsForGridmix.getProxyUsersFile(conf),
+                      tracePath};
+
+    final String [] otherArgs = {
+        "-D", JobContext.JOB_CANCEL_DELEGATION_TOKEN + "=false",
+        "-D", GridmixJob.GRIDMIX_HIGHRAM_EMULATION_ENABLE + "=false",
+        "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=true"
+    };
+    runGridmixAndVerify(runtimeValues, otherArgs, tracePath, 
+        GridMixRunMode.DATA_GENERATION_AND_RUN_GRIDMIX.getValue());
+  }
+  
+  /**
+   * Verify the Gridmix emulation of HDFS distributed cache
+   * file in SubmitterUserResolver mode with STRESS submission policy 
+   * by using the existing input data and HDFS distributed cache file. 
+   * @throws Exception - if an error occurs.
+   */
+  @Test
+  public void testGridmixEmulationOfHDFSPublicDCFile() 
+      throws Exception {
+    final String tracePath = getTraceFile("distcache_case9_trace");
+    Assert.assertNotNull("Trace file has not found.", tracePath);
+    final String [] runtimeValues = {"LOADJOB",
+                                     SubmitterUserResolver.class.getName(),
+                                     "STRESS",
+                                     tracePath};
+
+    final String [] otherArgs = {
+      "-D", GridmixJob.GRIDMIX_HIGHRAM_EMULATION_ENABLE + "=false",
+      "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=true"
+    };
+    runGridmixAndVerify(runtimeValues, otherArgs, tracePath, 
+                        GridMixRunMode.RUN_GRIDMIX.getValue());
+  }
+}

+ 92 - 0
src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestEmulationOfHDFSDCFilesWithDifferentVisibilities.java

@@ -0,0 +1,92 @@
+/**
+ * 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.mapred.gridmix;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.mapred.gridmix.test.system.GridMixConfig;
+import org.apache.hadoop.mapred.gridmix.test.system.GridMixRunMode;
+import org.apache.hadoop.mapred.gridmix.test.system.UtilsForGridmix;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ *  Verify the Gridmix emulation of HDFS distributed cache files of 
+ *  different visibilities. 
+ */
+
+public class TestEmulationOfHDFSDCFilesWithDifferentVisibilities 
+    extends GridmixSystemTestCase {
+  private static final Log LOG = 
+     LogFactory.getLog(
+         "TestEmulationOfHDFSDCFilesWithDifferentVisibilities.class");
+  
+  /**
+   * Generate input data and HDFS distributed cache files of different
+   * visibilities based on given input trace. Verify the Gridmix emulation 
+   * of HDFS distributed cache files of different visibilities in 
+   * RoundRobinUserResolver mode with SERIAL submission policy.
+   * @throws Exception - if an error occurs.
+   */
+  @Test
+  public void testGenerateAndEmulateOfHDFSDCFilesWithDiffVisibilities() 
+     throws Exception {
+    final long INPUT_SIZE = 1024 * 9;
+    final String tracePath = getTraceFile("distcache_case5_trace");
+    Assert.assertNotNull("Trace file was not found.", tracePath);
+    final String [] runtimeValues = 
+                     { "LOADJOB",
+                       RoundRobinUserResolver.class.getName(),
+                       "STRESS",
+                       INPUT_SIZE+"m",
+                       "file://" + UtilsForGridmix.getProxyUsersFile(conf),
+                       tracePath};
+
+    final String [] otherArgs = { 
+        "-D", JobContext.JOB_CANCEL_DELEGATION_TOKEN + "=false",
+        "-D", GridmixJob.GRIDMIX_HIGHRAM_EMULATION_ENABLE + "=false",
+        "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE +  "=true"
+    };
+    runGridmixAndVerify(runtimeValues, otherArgs, tracePath, 
+        GridMixRunMode.DATA_GENERATION_AND_RUN_GRIDMIX.getValue());
+  }
+  
+  /**
+   * Disable the distributed cache emulation and verify the Gridmix jobs
+   * whether it emulates or not. 
+   * @throws Exception
+   */
+  @Test
+  public void testHDFSDCFilesWithoutEnableDCEmulation() 
+     throws Exception {
+    final String tracePath = getTraceFile("distcache_case6_trace");
+    Assert.assertNotNull("Trace file was not found.", tracePath);
+    final String [] runtimeValues ={ "LOADJOB",
+                                     SubmitterUserResolver.class.getName(),
+                                     "STRESS",
+                                     tracePath};
+    final String [] otherArgs = {
+        "-D", GridmixJob.GRIDMIX_HIGHRAM_EMULATION_ENABLE + "=false",
+        "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=false"
+    };
+    runGridmixAndVerify(runtimeValues, otherArgs, tracePath, 
+                        GridMixRunMode.RUN_GRIDMIX.getValue());
+  }
+}
+

+ 64 - 0
src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestEmulationOfHighRamAndNormalMRJobs.java

@@ -0,0 +1,64 @@
+/**
+ * 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.mapred.gridmix;
+
+import org.apache.commons.logging.LogFactory;
+import org.apache.commons.logging.Log;
+import org.apache.hadoop.mapred.gridmix.test.system.GridMixConfig;
+import org.apache.hadoop.mapred.gridmix.test.system.GridMixRunMode;
+import org.apache.hadoop.mapred.gridmix.test.system.UtilsForGridmix;
+import org.junit.Test;
+import org.junit.Assert;
+
+/**
+ * Run the {@link Gridmix} with combination of high ram and normal jobs of
+ * trace and verify whether high ram jobs{@link Gridmix} are honoring or not.
+ * Normal MR jobs should not honors the high ram emulation.
+ */
+public class TestEmulationOfHighRamAndNormalMRJobs
+    extends GridmixSystemTestCase { 
+  private static final Log LOG = 
+      LogFactory.getLog("TestEmulationOfHighRamAndNormalMRJobs.class");
+
+  /**
+   * Generate input data and run the combination normal and high ram 
+   * {@link Gridmix} jobs as load job and STRESS submission policy 
+   * in a SubmitterUserResolver mode. Verify whether each {@link Gridmix} 
+   * job honors the high ram or not after completion of execution. 
+   * @throws Exception - if an error occurs.
+   */
+  @Test
+  public void testEmulationOfHighRamForReducersOfMRJobs() 
+      throws Exception { 
+    final long inputSizeInMB = cSize * 250;
+    String tracePath = getTraceFile("highram_mr_jobs_case4");
+    Assert.assertNotNull("Trace file has not found.", tracePath);
+    String [] runtimeArgs = {"LOADJOB",
+                               SubmitterUserResolver.class.getName(),
+                               "SERIAL",
+                               inputSizeInMB + "m",
+                               tracePath};
+    String [] otherArgs = {
+            "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=false", 
+            "-D", GridMixConfig.GRIDMIX_COMPRESSION_ENABLE + "=false", 
+            "-D", GridMixConfig.GRIDMIX_HIGH_RAM_JOB_ENABLE + "=true"};
+
+    validateTaskMemoryParamters(tracePath, true);
+    runGridmixAndVerify(runtimeArgs, otherArgs, tracePath);
+  }
+}

+ 93 - 0
src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestEmulationOfLocalFSDCFiles.java

@@ -0,0 +1,93 @@
+/**
+ * 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.mapred.gridmix;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.mapred.gridmix.test.system.GridMixConfig;
+import org.apache.hadoop.mapred.gridmix.test.system.GridMixRunMode;
+import org.apache.hadoop.mapred.gridmix.test.system.UtilsForGridmix;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Verify the emulation of local FS distributed cache files.
+ *
+ */
+public class TestEmulationOfLocalFSDCFiles extends GridmixSystemTestCase { 
+  private static final Log LOG = 
+      LogFactory.getLog("TestEmulationOfLocalFSDCFiles.class");
+
+  /**
+   * Generate the input data and distributer cache files.Verify the 
+   * gridmix emulation of local file system distributed cache files 
+   * in RoundRobinUserResolver mode with REPLAY submission policy.
+   * @throws Exception - if an error occurs.
+   */
+  @Test
+  public void testGenerateInputAndEmulateLocalFSDCFile() 
+     throws Exception { 
+    final long inputSizeInMB = 1024 * 6;
+    final String tracePath = getTraceFile("distcache_case7_trace");
+    Assert.assertNotNull("Trace file has not found.", tracePath);
+    final String [] runtimeValues = 
+                     {"LOADJOB",
+                      RoundRobinUserResolver.class.getName(),
+                      "REPLAY",
+                      inputSizeInMB + "m",
+                      "file://" + UtilsForGridmix.getProxyUsersFile(conf),
+                      tracePath};
+
+    final String [] otherArgs = {
+       "-D", JobContext.JOB_CANCEL_DELEGATION_TOKEN + "=false",
+       "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=true",
+       "-D", GridmixJob.GRIDMIX_HIGHRAM_EMULATION_ENABLE + "=false",
+       "-D", GridMixConfig.GRIDMIX_COMPRESSION_ENABLE + "=false"
+    };
+    runGridmixAndVerify(runtimeValues, otherArgs, tracePath, 
+        GridMixRunMode.DATA_GENERATION_AND_RUN_GRIDMIX.getValue());
+  }
+  
+  /**
+   * Use existing input and local distributed cache files and  verify 
+   * the gridmix emulation of local file system distributed cache 
+   * files in SubmitterUserResolver mode with STRESS
+   * Submission policy.
+   * @throws Exception - if an error occurs.
+   */
+  @Test
+  public void testEmulationOfLocalFSDCFile() 
+     throws Exception  {
+    final String tracePath = getTraceFile("distcache_case7_trace");
+    Assert.assertNotNull("Trace file has not found.", tracePath);
+    final String [] runtimeValues = {"LOADJOB",
+                                     SubmitterUserResolver.class.getName(),
+                                     "STRESS",
+                                     tracePath};
+
+    final String [] otherArgs = {
+       "-D",JobContext.JOB_CANCEL_DELEGATION_TOKEN + "=false",
+      "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=true",
+      "-D", GridmixJob.GRIDMIX_HIGHRAM_EMULATION_ENABLE + "=false",
+      "-D", GridMixConfig.GRIDMIX_COMPRESSION_ENABLE + "=false"
+    };
+    runGridmixAndVerify(runtimeValues, otherArgs, tracePath, 
+                        GridMixRunMode.RUN_GRIDMIX.getValue());
+  }
+}

+ 83 - 56
src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestGridMixDataGeneration.java

@@ -27,6 +27,9 @@ import org.apache.hadoop.mapred.JobStatus;
 import org.apache.hadoop.mapred.gridmix.RoundRobinUserResolver;
 import org.apache.hadoop.mapred.gridmix.EchoUserResolver;
 import org.apache.hadoop.mapred.gridmix.SubmitterUserResolver;
+import org.apache.hadoop.mapred.gridmix.test.system.UtilsForGridmix;
+import org.apache.hadoop.mapred.gridmix.test.system.GridMixRunMode;
+import org.apache.hadoop.mapred.gridmix.test.system.GridMixConfig;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.FileSystem;
@@ -55,15 +58,14 @@ public class TestGridMixDataGeneration {
   @BeforeClass
   public static void before() throws Exception {
     String [] excludeExpList = {"java.net.ConnectException", 
-        "java.io.IOException"};
+                                "java.io.IOException"};
     cluster = MRCluster.createCluster(conf);
     cluster.setExcludeExpList(excludeExpList);
     cluster.setUp();
     cSize = cluster.getTTClients().size();
     jtClient = cluster.getJTClient();
     rtClient = jtClient.getProxy();
-    gridmixDir = new Path("hdfs:///user/" + UtilsForGridmix.getUserName()
-       + "/herriot-gridmix");
+    gridmixDir = new Path("herriot-gridmix");
     UtilsForGridmix.createDirs(gridmixDir, rtClient.getDaemonConf());
   }
 
@@ -82,17 +84,24 @@ public class TestGridMixDataGeneration {
   @Test
   public void testGenerateDataWithSTRESSSubmission() throws Exception {
     conf = rtClient.getDaemonConf();
-    final long inputSize = cSize * 128;
-    String [] runtimeValues ={"LOADJOB",
-       SubmitterUserResolver.class.getName(),
-       "STRESS",
-       inputSize+"m",
-       "file:///dev/null"}; 
+    final long inputSizeInMB = cSize * 128;
+    String [] runtimeValues = {"LOADJOB",
+                               SubmitterUserResolver.class.getName(),
+                               "STRESS",
+                               inputSizeInMB + "m",
+                               "file:///dev/null"};
 
-    int exitCode = UtilsForGridmix.runGridmixJob(gridmixDir, 
-      conf,GridMixRunMode.DATA_GENERATION, runtimeValues);
+    String [] otherArgs = {
+        "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=false", 
+        "-D", GridmixJob.GRIDMIX_HIGHRAM_EMULATION_ENABLE + "=false",
+        "-D", GridMixConfig.GRIDMIX_COMPRESSION_ENABLE + "=false"
+    };
+    int exitCode = 
+        UtilsForGridmix.runGridmixJob(gridmixDir, conf, 
+            GridMixRunMode.DATA_GENERATION.getValue(), 
+            runtimeValues, otherArgs);
     Assert.assertEquals("Data generation has failed.", 0 , exitCode);
-    checkGeneratedDataAndJobStatus(inputSize);
+    checkGeneratedDataAndJobStatus(inputSizeInMB);
   }
   
   /**
@@ -104,18 +113,27 @@ public class TestGridMixDataGeneration {
   @Test
   public void testGenerateDataWithREPLAYSubmission() throws Exception {
     conf = rtClient.getDaemonConf();
-    final long inputSize = cSize * 300;
-    String [] runtimeValues ={"LOADJOB",
-       RoundRobinUserResolver.class.getName(),
-       "REPLAY",
-       inputSize +"m",
-       "file://" + UtilsForGridmix.getProxyUsersFile(cluster.getHadoopProxyUsers()),
-       "file:///dev/null"};
+    final long inputSizeInMB = cSize * 300;
+    String [] runtimeValues = 
+               {"LOADJOB",
+                RoundRobinUserResolver.class.getName(),
+                "REPLAY",
+                inputSizeInMB +"m",
+                "file://" + UtilsForGridmix.getProxyUsersFile(conf),
+                "file:///dev/null"};
     
-    int exitCode = UtilsForGridmix.runGridmixJob(gridmixDir, 
-       conf,GridMixRunMode.DATA_GENERATION, runtimeValues);
+    String [] otherArgs = {
+        "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=false", 
+        "-D", GridmixJob.GRIDMIX_HIGHRAM_EMULATION_ENABLE + "=false",
+        "-D", GridMixConfig.GRIDMIX_COMPRESSION_ENABLE + "=false"
+    };
+
+    int exitCode = 
+        UtilsForGridmix.runGridmixJob(gridmixDir, conf, 
+            GridMixRunMode.DATA_GENERATION.getValue(), 
+            runtimeValues, otherArgs);
     Assert.assertEquals("Data generation has failed.", 0 , exitCode);
-    checkGeneratedDataAndJobStatus(inputSize); 
+    checkGeneratedDataAndJobStatus(inputSizeInMB); 
   }
   
   /**
@@ -128,72 +146,81 @@ public class TestGridMixDataGeneration {
   @Test
   public void testGenerateDataWithSERIALSubmission() throws Exception {
     conf = rtClient.getDaemonConf();
-    int perNodeSize = 500; // 500 mb per node data
-    final long inputSize = cSize * perNodeSize;
-    String [] runtimeValues ={"LOADJOB",
-       EchoUserResolver.class.getName(),
-       "SERIAL",
-       inputSize + "m",
-       "file:///dev/null"};
-    int bytesPerFile = 200; // 200 mb per file of data
+    long perNodeSizeInMB = 500; // 500 mb per node data
+    final long inputSizeInMB = cSize * perNodeSizeInMB;
+    String [] runtimeValues ={"LOADJOB", 
+                              EchoUserResolver.class.getName(), 
+                              "SERIAL", 
+                              inputSizeInMB + "m", 
+                              "file:///dev/null"};
+    long bytesPerFile = 200  * 1024 * 1024; // 200 mb per file of data
     String [] otherArgs = {
-      "-D", GridMixConfig.GRIDMIX_BYTES_PER_FILE + 
-      "=" + (bytesPerFile * 1024 * 1024)
+        "-D", GridMixConfig.GRIDMIX_BYTES_PER_FILE + "=" + bytesPerFile, 
+        "-D", GridmixJob.GRIDMIX_HIGHRAM_EMULATION_ENABLE + "=false",
+        "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=false", 
+        "-D", GridMixConfig.GRIDMIX_COMPRESSION_ENABLE + "=false"
     };
-    int exitCode = UtilsForGridmix.runGridmixJob(gridmixDir, 
-       conf,GridMixRunMode.DATA_GENERATION, runtimeValues,otherArgs);
+    int exitCode = 
+        UtilsForGridmix.runGridmixJob(gridmixDir, conf, 
+            GridMixRunMode.DATA_GENERATION.getValue(), 
+            runtimeValues, otherArgs);
     Assert.assertEquals("Data generation has failed.", 0 , exitCode);
     LOG.info("Verify the eache file size in a generate data.");
-    verifyEachNodeSize(new Path(gridmixDir,"input"));
-    verifyNumOfFilesGeneratedInEachNode(new Path(gridmixDir,"input"), 
-       perNodeSize, bytesPerFile);
-    checkGeneratedDataAndJobStatus(inputSize);
+    verifyEachNodeSize(new Path(gridmixDir, "input"), perNodeSizeInMB);
+    verifyNumOfFilesGeneratedInEachNode(new Path(gridmixDir, "input"), 
+                                        perNodeSizeInMB, bytesPerFile);
+    checkGeneratedDataAndJobStatus(inputSizeInMB);
   }
   
   private void checkGeneratedDataAndJobStatus(long inputSize) 
-     throws IOException {
+      throws IOException {
     LOG.info("Verify the generated data size.");
-    long dataSize = getDataSize(new Path(gridmixDir,"input"));
+    long dataSizeInMB = getDataSizeInMB(new Path(gridmixDir,"input"));
     Assert.assertTrue("Generate data has not matched with given size",
-       dataSize + 0.1 > inputSize || dataSize - 0.1 < inputSize);
+       dataSizeInMB + 0.1 > inputSize || dataSizeInMB - 0.1 < inputSize);
  
     JobClient jobClient = jtClient.getClient();
+    int len = jobClient.getAllJobs().length;
     LOG.info("Verify the job status after completion of job.");
     Assert.assertEquals("Job has not succeeded.", JobStatus.SUCCEEDED, 
-       jobClient.getAllJobs()[0].getRunState());
+                        jobClient.getAllJobs()[len-1].getRunState());
   }
   
-  private void verifyEachNodeSize(Path inputDir) throws IOException {
+  private void verifyEachNodeSize(Path inputDir, long dataSizePerNode) 
+      throws IOException {
     FileSystem fs = inputDir.getFileSystem(conf);
     FileStatus [] fstatus = fs.listStatus(inputDir);
     for (FileStatus fstat : fstatus) {
       if ( fstat.isDir()) {
-        long fileSize = getDataSize(fstat.getPath());
-        Assert.assertTrue("The Size has not " + 
-           " matched with given per node file size(500mb)", 
-           fileSize + 0.1 > 500 || fileSize - 0.1 < 500);
+        long fileSize = getDataSizeInMB(fstat.getPath());
+        Assert.assertTrue("The Size has not matched with given "
+                         + "per node file size(" + dataSizePerNode +"MB)", 
+                         fileSize + 0.1 > dataSizePerNode 
+                         || fileSize - 0.1 < dataSizePerNode);
       }
     }    
   }
 
   private void verifyNumOfFilesGeneratedInEachNode(Path inputDir, 
-     int nodeSize, int fileSize) throws IOException {
-    int expFileCount = Math.round(nodeSize/fileSize) + 
-       ((nodeSize%fileSize != 0)? 1:0);
+      long nodeSize, long fileSize) throws IOException {
+    long fileCount = (nodeSize * 1024 * 1024)/fileSize;
+    long expFileCount = Math.round(fileCount);
+    expFileCount = expFileCount + ((nodeSize%fileSize != 0)? 1:0);
     FileSystem fs = inputDir.getFileSystem(conf);
     FileStatus [] fstatus = fs.listStatus(inputDir);
     for (FileStatus fstat : fstatus) {
-      if ( fstat.isDir()) {
+      if (fstat.isDir()) {
         FileSystem nodeFs = fstat.getPath().getFileSystem(conf);
-        long actFileCount = nodeFs.getContentSummary(fstat.getPath())
-           .getFileCount();
-        Assert.assertEquals("File count has not matched.", 
-           expFileCount, actFileCount);
+        LOG.info("getPath():" + fstat.getPath().toString());
+        long actFileCount = nodeFs.getContentSummary(
+            fstat.getPath()).getFileCount();
+        Assert.assertEquals("File count has not matched.", expFileCount, 
+                            actFileCount);
       }
     }
   }
 
-  private static long getDataSize(Path inputDir) throws IOException {
+  private static long getDataSizeInMB(Path inputDir) throws IOException {
     FileSystem fs = inputDir.getFileSystem(conf);
     ContentSummary csmry = fs.getContentSummary(inputDir);
     long dataSize = csmry.getLength();

+ 40 - 27
src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestGridMixFilePool.java

@@ -23,6 +23,9 @@ import org.apache.hadoop.mapreduce.test.system.MRCluster;
 import org.apache.hadoop.mapreduce.test.system.JTClient;
 import org.apache.hadoop.mapreduce.test.system.JTProtocol;
 import org.apache.hadoop.mapred.gridmix.FilePool;
+import org.apache.hadoop.mapred.gridmix.test.system.UtilsForGridmix;
+import org.apache.hadoop.mapred.gridmix.test.system.GridMixRunMode;
+import org.apache.hadoop.mapred.gridmix.test.system.GridMixConfig;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.FileStatus;
 import org.junit.Assert;
@@ -33,8 +36,8 @@ import java.io.IOException;
 import java.util.ArrayList;
 
 public class TestGridMixFilePool {
-  private static final Log LOG = LogFactory
-     .getLog(TestGridMixFilePool.class);
+  private static final Log LOG = 
+      LogFactory.getLog(TestGridMixFilePool.class);
   private static Configuration conf = new Configuration();
   private static MRCluster cluster;
   private static JTProtocol remoteClient;
@@ -45,15 +48,14 @@ public class TestGridMixFilePool {
   @BeforeClass
   public static void before() throws Exception {
     String []  excludeExpList = {"java.net.ConnectException", 
-       "java.io.IOException"};
+                                 "java.io.IOException"};
     cluster = MRCluster.createCluster(conf);
     cluster.setExcludeExpList(excludeExpList);
     cluster.setUp();
     jtClient = cluster.getJTClient();
     remoteClient = jtClient.getProxy();
     clusterSize = cluster.getTTClients().size();
-    gridmixDir = new Path("hdfs:///user/" + UtilsForGridmix.getUserName()
-       + "/herriot-gridmix");
+    gridmixDir = new Path("herriot-gridmix");
     UtilsForGridmix.createDirs(gridmixDir, remoteClient.getDaemonConf());
   }
 
@@ -66,50 +68,61 @@ public class TestGridMixFilePool {
   @Test
   public void testFilesCountAndSizesForSpecifiedFilePool() throws Exception {
     conf = remoteClient.getDaemonConf();
-    final long inputSize = clusterSize * 200;
+    final long inputSizeInMB = clusterSize * 200;
     int [] fileSizesInMB = {50, 100, 400, 50, 300, 10, 60, 40, 20 ,10 , 500};
     long targetSize = Long.MAX_VALUE;
-    final int expFileCount = 13;
+    final int expFileCount = clusterSize + 4;
     String [] runtimeValues ={"LOADJOB",
-       SubmitterUserResolver.class.getName(),
-       "STRESS",
-       inputSize+"m",
-       "file:///dev/null"}; 
+                              SubmitterUserResolver.class.getName(),
+                              "STRESS",
+                              inputSizeInMB + "m",
+                              "file:///dev/null"}; 
 
-    int exitCode = UtilsForGridmix.runGridmixJob(gridmixDir, 
-       conf,GridMixRunMode.DATA_GENERATION, runtimeValues);
+    String [] otherArgs = {
+        "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=false",
+        "-D", GridmixJob.GRIDMIX_HIGHRAM_EMULATION_ENABLE + "=false",
+        "-D", GridMixConfig.GRIDMIX_COMPRESSION_ENABLE + "=false"
+    };
+
+    // Generate the input data by using gridmix framework.
+    int exitCode = 
+        UtilsForGridmix.runGridmixJob(gridmixDir, conf, 
+            GridMixRunMode.DATA_GENERATION.getValue(), 
+            runtimeValues, otherArgs);
     Assert.assertEquals("Data generation has failed.", 0 , exitCode);
-    // create files for given sizes.
-    createFiles(new Path(gridmixDir,"input"),fileSizesInMB);
+    // Create the files without using gridmix input generation with 
+    // above mentioned sizes in a array.
+    createFiles(new Path(gridmixDir, "input"), fileSizesInMB);
     conf.setLong(FilePool.GRIDMIX_MIN_FILE, 100 * 1024 * 1024);
-    FilePool fpool = new FilePool(conf,new Path(gridmixDir,"input"));
+    FilePool fpool = new FilePool(conf, new Path(gridmixDir, "input"));
     fpool.refresh();
-    verifyFilesSizeAndCountForSpecifiedPool(expFileCount,targetSize, fpool);
+    verifyFilesSizeAndCountForSpecifiedPool(expFileCount, targetSize, fpool);
   }
   
   private void createFiles(Path inputDir, int [] fileSizes) 
-     throws Exception {
+      throws Exception { 
     for (int size : fileSizes) {
       UtilsForGridmix.createFile(size, inputDir, conf);
     }
   }
   
   private void verifyFilesSizeAndCountForSpecifiedPool(int expFileCount, 
-     long minFileSize, FilePool pool) throws IOException {
+      long minFileSize, FilePool pool) throws IOException {
     final ArrayList<FileStatus> files = new ArrayList<FileStatus>();
-    long  actFilesSize = pool.getInputFiles(minFileSize, files)/(1024 * 1024);
-    long expFilesSize = 3100 ;
-    Assert.assertEquals("Files Size has not matched for specified pool.",
-       expFilesSize, actFilesSize);
-    int actFileCount = files.size();    
-    Assert.assertEquals("File count has not matched.", 
-       expFileCount, actFileCount);
+    long filesSizeInBytes = pool.getInputFiles(minFileSize, files);
+    long actFilesSizeInMB = filesSizeInBytes / (1024 * 1024);
+    long expFilesSizeInMB = (clusterSize * 200) + 1300;
+    Assert.assertEquals("Files Size has not matched for specified pool.", 
+                        expFilesSizeInMB, actFilesSizeInMB);
+    int actFileCount = files.size();
+    Assert.assertEquals("File count has not matched.", expFileCount, 
+                        actFileCount);
     int count = 0;
     for (FileStatus fstat : files) {
       String fp = fstat.getPath().toString();
       count = count + ((fp.indexOf("datafile_") > 0)? 0 : 1);
     }
     Assert.assertEquals("Total folders are not matched with cluster size", 
-            clusterSize, count);
+                        clusterSize, count);
   }
 }

+ 175 - 0
src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestGridmixCompressedInputGeneration.java

@@ -0,0 +1,175 @@
+/**
+ * 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.mapred.gridmix;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.mapred.JobClient;
+import org.apache.hadoop.mapred.JobStatus;
+import org.apache.hadoop.mapred.gridmix.Gridmix;
+import org.apache.hadoop.mapred.gridmix.test.system.GridMixConfig;
+import org.apache.hadoop.mapred.gridmix.test.system.GridMixRunMode;
+import org.apache.hadoop.mapred.gridmix.test.system.UtilsForGridmix;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Verify the Gridmix generated input if compression emulation turn on.
+ */
+public class TestGridmixCompressedInputGeneration 
+    extends GridmixSystemTestCase { 
+
+  private static final Log LOG = 
+      LogFactory.getLog("TestGridmixCompressedInputGeneration.class");
+
+  /**
+   * Generate input data and verify whether input files are compressed
+   * or not.
+   * @throws Exception - if an error occurs.
+   */
+  @Test
+  public void testGridmixCompressionInputGeneration() throws Exception {
+    final long inputSizeInMB = 1024 * 7;
+    final String [] runtimeValues = {"LOADJOB",
+                                     SubmitterUserResolver.class.getName(),
+                                     "STRESS",
+                                     inputSizeInMB  + "m",
+                                     "file:///dev/null"};
+    final String [] otherArgs = { 
+        "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=false",
+        "-D", GridMixConfig.GRIDMIX_COMPRESSION_ENABLE + "=true"
+    };
+    LOG.info("Verify the generated compressed input data.");
+    runAndVerify(true, inputSizeInMB, runtimeValues, otherArgs);
+  }
+
+  /**
+   * Disable compression emulation and verify whether input files are 
+   * compressed or not.
+   * @throws Exception
+   */
+  @Test
+  public void testGridmixInputGenerationWithoutCompressionEnable() 
+      throws Exception { 
+    UtilsForGridmix.cleanup(gridmixDir, rtClient.getDaemonConf());
+    final long inputSizeInMB = 1024 * 6;
+    final String [] runtimeValues = {"LOADJOB",
+                                     SubmitterUserResolver.class.getName(),
+                                     "STRESS",
+                                     inputSizeInMB + "m",
+                                     "file:///dev/null"};
+    final String [] otherArgs = {
+        "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=false",
+        "-D", GridMixConfig.GRIDMIX_COMPRESSION_ENABLE + "=false"
+    };
+
+    LOG.info("Verify the generated uncompressed input data.");
+    runAndVerify(false, inputSizeInMB, runtimeValues, otherArgs);
+  }
+  
+  private void runAndVerify(boolean isCompressed, long INPUT_SIZE, 
+      String [] runtimeValues, String [] otherArgs) throws Exception { 
+    int exitCode = 
+        UtilsForGridmix.runGridmixJob(gridmixDir, conf, 
+                                      GridMixRunMode.DATA_GENERATION.getValue(),
+                                      runtimeValues,otherArgs);
+    Assert.assertEquals("Data generation has failed.", 0, exitCode);
+    verifyJobStatus();
+    verifyInputDataSize(INPUT_SIZE);
+    verifyInputFiles(isCompressed);
+  }
+  
+  private void verifyInputFiles(boolean isCompressed) throws IOException { 
+    List<String> inputFiles = 
+        getInputFiles(conf, Gridmix.getGridmixInputDataPath(gridmixDir));
+    for (String inputFile: inputFiles) {
+      boolean fileStatus = (inputFile.contains(".gz") 
+                         || inputFile.contains(".tgz")
+                         || inputFile.contains(".deflate"))? true : false;
+      if (isCompressed) { 
+        Assert.assertTrue("Compressed input split file was not found.",
+                          fileStatus);
+      } else {
+        Assert.assertFalse("Uncompressed input split file was not found.",
+                           fileStatus);
+      }
+    }
+  }
+
+  private void verifyInputDataSize(long INPUT_SIZE) throws IOException {
+    long actDataSize = 
+        getInputDataSizeInMB(conf, Gridmix.getGridmixInputDataPath(gridmixDir));
+    double ratio = ((double)actDataSize)/INPUT_SIZE;
+    long expDataSize = (long)(INPUT_SIZE * ratio);
+    Assert.assertEquals("Generated data has not matched with given size.", 
+                        expDataSize, actDataSize);
+  }
+
+  private void verifyJobStatus() throws IOException { 
+    JobClient jobClient = jtClient.getClient();
+    int len = jobClient.getAllJobs().length;
+    LOG.info("Verify the job status after completion of job...");
+    Assert.assertEquals("Job has not succeeded.", JobStatus.SUCCEEDED, 
+                        jobClient.getAllJobs()[len -1].getRunState());
+  }
+
+  private long getInputDataSizeInMB(Configuration conf, Path inputDir) 
+      throws IOException { 
+    FileSystem fs = inputDir.getFileSystem(conf);
+    ContentSummary csmry = fs.getContentSummary(inputDir);
+    long dataSize = csmry.getLength();
+    dataSize = dataSize/(1024 * 1024);
+    return dataSize;
+  }
+
+  private List<String> getInputFiles(Configuration conf, Path inputDir) 
+      throws IOException {
+    FileSystem fs = inputDir.getFileSystem(conf);
+    FileStatus [] listStatus = fs.listStatus(inputDir);
+    List<String> files = new ArrayList<String>();
+    for (FileStatus fileStat : listStatus) {
+      files.add(getInputFile(fileStat, conf));
+    }
+    return files;
+  }
+
+  private String getInputFile(FileStatus fstatus, Configuration conf) 
+      throws IOException {
+    String fileName = null;
+    if (!fstatus.isDir()) {
+      fileName = fstatus.getPath().getName();
+      LOG.info("fileName++++:" + fileName);
+    } else {
+      FileSystem fs = fstatus.getPath().getFileSystem(conf);
+      FileStatus [] listStatus = fs.listStatus(fstatus.getPath());
+      for (FileStatus fileStat : listStatus) {
+         return getInputFile(fileStat, conf);
+      }
+    }
+    return fileName;
+  }
+}
+

+ 102 - 0
src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestGridmixCompressionEmulationWithCompressInput.java

@@ -0,0 +1,102 @@
+/**
+ * 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.mapred.gridmix;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.mapred.gridmix.test.system.GridMixConfig;
+import org.apache.hadoop.mapred.gridmix.test.system.GridMixRunMode;
+import org.apache.hadoop.mapred.gridmix.test.system.UtilsForGridmix;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Verify the gridmix jobs compression ratios of map input, 
+ * map output and reduce output with default and user specified 
+ * compression ratios.
+ *
+ */
+public class TestGridmixCompressionEmulationWithCompressInput 
+    extends GridmixSystemTestCase {
+  private static final Log LOG = 
+      LogFactory.getLog(
+              "TestGridmixCompressionEmulationWithCompressInput.class");
+  final long inputSizeInMB = 1024 * 6;
+
+  /**
+   * Generate compressed input data and verify the map input, 
+   * map output and reduce output compression ratios of gridmix jobs 
+   * against the default compression ratios. 
+   * @throws Exception - if an error occurs.
+   */
+  @Test
+  public void testGridmixCompressionRatiosAgainstDefaultCompressionRatio() 
+      throws Exception { 
+    final String tracePath = getTraceFile("compression_case1_trace");
+    Assert.assertNotNull("Trace file has not found.", tracePath);
+
+    final String [] runtimeValues = 
+                     {"LOADJOB",
+                      RoundRobinUserResolver.class.getName(),
+                      "STRESS",
+                      inputSizeInMB + "m",
+                      "file://" + UtilsForGridmix.getProxyUsersFile(conf),
+                      tracePath};
+
+    final String [] otherArgs = { 
+        "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=false",
+        "-D", GridmixJob.GRIDMIX_HIGHRAM_EMULATION_ENABLE + "=false",
+        "-D", GridMixConfig.GRIDMIX_COMPRESSION_ENABLE + "=true"
+    };
+    runGridmixAndVerify(runtimeValues, otherArgs, tracePath,
+        GridMixRunMode.DATA_GENERATION_AND_RUN_GRIDMIX.getValue());
+  }
+  
+  /**
+   * Verify map input, map output and  reduce output compression ratios of
+   * gridmix jobs against user specified compression ratios. 
+   * @throws Exception - if an error occurs.
+   */
+  @Test
+  public void testGridmixOuputCompressionRatiosAgainstCustomRatios() 
+      throws Exception { 
+    final String tracePath = getTraceFile("compression_case1_trace");
+    Assert.assertNotNull("Trace file has not found.", tracePath);
+    UtilsForGridmix.cleanup(gridmixDir, rtClient.getDaemonConf());
+
+    final String [] runtimeValues = 
+                     {"LOADJOB",
+                      RoundRobinUserResolver.class.getName(),
+                      "STRESS",
+                      inputSizeInMB + "m",
+                      "file://" + UtilsForGridmix.getProxyUsersFile(conf),
+                      tracePath};
+
+    final String [] otherArgs = {
+        "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=false",
+        "-D", GridMixConfig.GRIDMIX_COMPRESSION_ENABLE + "=true",
+        "-D", GridMixConfig.GRIDMIX_INPUT_DECOMPRESS_ENABLE + "=true",
+        "-D", GridmixJob.GRIDMIX_HIGHRAM_EMULATION_ENABLE + "=false",
+        "-D", GridMixConfig.GRIDMIX_INPUT_COMPRESS_RATIO + "=0.68",
+        "-D", GridMixConfig.GRIDMIX_INTERMEDIATE_COMPRESSION_RATIO + "=0.35",
+        "-D", GridMixConfig.GRIDMIX_OUTPUT_COMPRESSION_RATIO + "=0.40"
+    };
+    runGridmixAndVerify(runtimeValues, otherArgs, tracePath, 
+        GridMixRunMode.DATA_GENERATION_AND_RUN_GRIDMIX.getValue());
+  }
+}
+

+ 89 - 0
src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestGridmixEmulationOfHDFSPrivateDCFile.java

@@ -0,0 +1,89 @@
+/**
+ * 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.mapred.gridmix;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.mapred.gridmix.test.system.GridMixConfig;
+import org.apache.hadoop.mapred.gridmix.test.system.GridMixRunMode;
+import org.apache.hadoop.mapred.gridmix.test.system.UtilsForGridmix;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Verify the Gridmix emulation of HDFS private distributed cache file.
+ */
+public class TestGridmixEmulationOfHDFSPrivateDCFile 
+    extends GridmixSystemTestCase { 
+  private static final Log LOG = 
+      LogFactory.getLog("TestGridmixEmulationOfHDFSPrivateDCFile.class");
+  /**
+   * Generate input data and single HDFS private distributed cache 
+   * file based on given input trace.Verify the Gridmix emulation of 
+   * single private HDFS distributed cache file in RoundRobinUserResolver 
+   * mode with STRESS submission policy.
+   * @throws Exception - if an error occurs.
+   */
+  @Test
+  public void testGenerateAndEmulateOfHDFSPrivateDCFile() 
+      throws Exception {
+    final long inputSizeInMB = 8192;
+    final String tracePath = getTraceFile("distcache_case3_trace");
+    Assert.assertNotNull("Trace file was not found.", tracePath);
+    final String [] runtimeValues = 
+                     {"LOADJOB",
+                      RoundRobinUserResolver.class.getName(),
+                      "STRESS",
+                      inputSizeInMB + "m",
+                      "file://" + UtilsForGridmix.getProxyUsersFile(conf),
+                      tracePath};
+
+    final String [] otherArgs = {
+        "-D", JobContext.JOB_CANCEL_DELEGATION_TOKEN + "=false",
+        "-D", GridmixJob.GRIDMIX_HIGHRAM_EMULATION_ENABLE + "=false",
+        "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=true"
+    };
+    runGridmixAndVerify(runtimeValues, otherArgs, tracePath, 
+        GridMixRunMode.DATA_GENERATION_AND_RUN_GRIDMIX.getValue());
+  }
+  /**
+   * Verify the Gridmix emulation of single HDFS private distributed 
+   * cache file in SubmitterUserResolver mode with REPLAY submission 
+   * policy by using the existing input data and HDFS private 
+   * distributed cache file.
+   * @throws Exception - if an error occurs.
+   */
+  @Test
+  public void testGridmixEmulationOfHDFSPrivateDCFile() 
+      throws Exception {
+    final String tracePath = getTraceFile("distcache_case3_trace");
+    Assert.assertNotNull("Trace file was not found.", tracePath);
+    final String [] runtimeValues ={"LOADJOB",
+                                    SubmitterUserResolver.class.getName(),
+                                    "REPLAY",
+                                    tracePath};
+    final String [] otherArgs = {
+        "-D", GridmixJob.GRIDMIX_HIGHRAM_EMULATION_ENABLE + "=false",
+        "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=true"
+    };
+    runGridmixAndVerify(runtimeValues, otherArgs, tracePath, 
+                        GridMixRunMode.RUN_GRIDMIX.getValue());
+  }
+}
+

+ 92 - 0
src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestGridmixEmulationOfHDFSPublicDCFile.java

@@ -0,0 +1,92 @@
+/**
+ * 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.mapred.gridmix;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.mapred.gridmix.test.system.GridMixConfig;
+import org.apache.hadoop.mapred.gridmix.test.system.GridMixRunMode;
+import org.apache.hadoop.mapred.gridmix.test.system.UtilsForGridmix;
+//import org.apache.hadoop.mapreduce.MRJobConfig;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Verify the Gridmix emulation of HDFS public distributed cache file.
+ */
+public class TestGridmixEmulationOfHDFSPublicDCFile 
+    extends GridmixSystemTestCase { 
+  private static final Log LOG = 
+      LogFactory.getLog("TestGridmixEmulationOfHDFSPublicDCFile.class");
+
+  /**
+   * Generate the input data and HDFS distributed cache file based 
+   * on given input trace. Verify the Gridmix emulation of single HDFS
+   * public distributed cache file in SubmitterUserResolver mode with 
+   * STRESS submission policy.
+   * @throws Exception - if an error occurs.
+   */
+  @Test
+  public void testGenerateAndEmulationOfSingleHDFSDCFile() 
+      throws Exception { 
+    final long inputSizeInMB = 7168;
+    final String tracePath = getTraceFile("distcache_case1_trace");
+    Assert.assertNotNull("Trace file was not found.", tracePath);
+    final String [] runtimeValues = {"LOADJOB",
+                                     SubmitterUserResolver.class.getName(),
+                                     "STRESS",
+                                     inputSizeInMB + "m",
+                                     tracePath};
+
+    final String [] otherArgs = { 
+      "-D", JobContext.JOB_CANCEL_DELEGATION_TOKEN + "=false",
+      "-D", GridmixJob.GRIDMIX_HIGHRAM_EMULATION_ENABLE + "=false",
+      "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=true"
+    };
+    runGridmixAndVerify(runtimeValues, otherArgs, tracePath, 
+        GridMixRunMode.DATA_GENERATION_AND_RUN_GRIDMIX.getValue());
+  }
+  
+  /**
+   * Verify the Gridmix emulation of Single HDFS public distributed cache
+   * file in RoundRobinUserResolver mode with REPLAY submission policy 
+   * by using the existing input data and HDFS public distributed cache file. 
+   * @throws Exception - if an error occurs.
+   */
+  @Test
+  public void testGridmixEmulationOfSingleHDFSPublicDCFile() 
+      throws Exception {
+    final String tracePath = getTraceFile("distcache_case1_trace");
+    Assert.assertNotNull("Trace file was not found.", tracePath);
+    final String [] runtimeValues = 
+                     { "LOADJOB",
+                       RoundRobinUserResolver.class.getName(),
+                       "REPLAY",
+                       "file://" + UtilsForGridmix.getProxyUsersFile(conf),
+                       tracePath};
+
+    final String [] otherArgs = {
+       "-D", GridmixJob.GRIDMIX_HIGHRAM_EMULATION_ENABLE + "=false",
+       "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=true"
+    };
+    runGridmixAndVerify(runtimeValues, otherArgs, tracePath, 
+                        GridMixRunMode.RUN_GRIDMIX.getValue());
+  }
+}
+

+ 64 - 0
src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestGridmixEmulationOfHighRamJobsCase1.java

@@ -0,0 +1,64 @@
+/**
+ * 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.mapred.gridmix;
+
+import org.apache.commons.logging.LogFactory;
+import org.apache.commons.logging.Log;
+import org.apache.hadoop.mapred.gridmix.test.system.GridMixConfig;
+import org.apache.hadoop.mapred.gridmix.GridmixJob;
+import org.junit.Test;
+import org.junit.Assert;
+
+/**
+ * Run the {@link Gridmix} with a high ram jobs trace and 
+ * verify each {@link Gridmix} job whether it honors the high ram or not.
+ * In the trace the jobs should use the high ram for both maps and reduces.
+ */
+public class TestGridmixEmulationOfHighRamJobsCase1 
+    extends GridmixSystemTestCase { 
+  private static final Log LOG = 
+      LogFactory.getLog("TestGridmixEmulationOfHighRamJobsCase1.class");
+
+ /**
+   * Generate input data and run {@link Gridmix} with a high ram jobs trace 
+   * as a load job and STRESS submission policy in a SubmitterUserResolver 
+   * mode. Verify each {@link Gridmix} job whether it honors the high ram or not
+   * after completion of execution. 
+   * @throws Exception - if an error occurs.
+   */
+  @Test
+  public void testEmulationOfHighRamForMapsAndReducesOfMRJobs() 
+      throws Exception { 
+    final long inputSizeInMB = cSize * 400;
+    String tracePath = getTraceFile("highram_mr_jobs_case1");
+    Assert.assertNotNull("Trace file has not found.", tracePath);
+    String [] runtimeValues = {"LOADJOB",
+                               SubmitterUserResolver.class.getName(),
+                               "STRESS",
+                               inputSizeInMB + "m",
+                               tracePath};
+
+    String [] otherArgs = {
+               "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=false", 
+               "-D", GridMixConfig.GRIDMIX_COMPRESSION_ENABLE + "=false", 
+               "-D", GridmixJob.GRIDMIX_HIGHRAM_EMULATION_ENABLE + "=true"};
+
+    validateTaskMemoryParamters(tracePath, true);
+    runGridmixAndVerify(runtimeValues, otherArgs, tracePath);
+  }
+}

+ 67 - 0
src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestGridmixEmulationOfHighRamJobsCase2.java

@@ -0,0 +1,67 @@
+/**
+ * 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.mapred.gridmix;
+
+import org.apache.commons.logging.LogFactory;
+import org.apache.commons.logging.Log;
+import org.apache.hadoop.mapred.gridmix.test.system.GridMixConfig;
+import org.apache.hadoop.mapred.gridmix.GridmixJob;
+import org.apache.hadoop.mapred.gridmix.test.system.UtilsForGridmix;
+import org.junit.Test;
+import org.junit.Assert;
+
+/**
+ * Run the {@link Gridmix} with a high ram jobs trace and 
+ * verify each {@link Gridmix} job whether it honors the high ram or not.
+ * In the trace the jobs should use the high ram only for maps.
+ */
+public class TestGridmixEmulationOfHighRamJobsCase2 
+    extends GridmixSystemTestCase { 
+  private static final Log LOG = 
+      LogFactory.getLog("TestGridmixEmulationOfHighRamJobsCase2.class");
+
+ /**
+   * Generate input data and run {@link Gridmix} with a high ram jobs trace 
+   * as a load job and REPALY submission policy in a RoundRobinUserResolver 
+   * mode. Verify each {@link Gridmix} job whether it honors the high ram or not
+   * after completion of execution. 
+   * @throws Exception - if an error occurs.
+   */
+  @Test
+  public void testEmulationOfHighRamForMapsOfMRJobs() 
+      throws Exception { 
+    final long inputSizeInMB = cSize * 300;
+    String tracePath = getTraceFile("highram_mr_jobs_case2");
+    Assert.assertNotNull("Trace file has not found.", tracePath);
+    String [] runtimeValues = 
+               {"LOADJOB",
+                RoundRobinUserResolver.class.getName(),
+                "REPLAY",
+                inputSizeInMB + "m",
+                "file://" + UtilsForGridmix.getProxyUsersFile(conf),
+                tracePath};
+
+    String [] otherArgs = {
+               "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=false", 
+               "-D", GridMixConfig.GRIDMIX_COMPRESSION_ENABLE + "=false", 
+               "-D", GridmixJob.GRIDMIX_HIGHRAM_EMULATION_ENABLE + "=true"};
+
+    validateTaskMemoryParamters(tracePath, true);
+    runGridmixAndVerify(runtimeValues, otherArgs, tracePath);
+  }
+}

+ 64 - 0
src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestGridmixEmulationOfHighRamJobsCase3.java

@@ -0,0 +1,64 @@
+/**
+ * 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.mapred.gridmix;
+
+import org.apache.commons.logging.LogFactory;
+import org.apache.commons.logging.Log;
+import org.apache.hadoop.mapred.gridmix.test.system.GridMixConfig;
+import org.apache.hadoop.mapred.gridmix.GridmixJob;
+import org.junit.Test;
+import org.junit.Assert;
+
+/**
+ * Run the {@link Gridmix} with a high ram jobs trace and 
+ * verify each {@link Gridmix} job whether it honors the high ram or not.
+ * In the trace the jobs should use the high ram only for reducers.
+ */
+public class TestGridmixEmulationOfHighRamJobsCase3 
+    extends GridmixSystemTestCase { 
+  private static final Log LOG = 
+      LogFactory.getLog(TestGridmixEmulationOfHighRamJobsCase3.class);
+
+ /**
+   * Generate input data and run {@link Gridmix} with a high ram jobs trace 
+   * as a load job and SERIAL submission policy in a SubmitterUserResolver 
+   * mode. Verify each {@link Gridmix} job whether it honors the 
+   * high ram or not after completion of execution. 
+   * @throws Exception - if an error occurs.
+   */
+  @Test
+  public void testEmulationOfHighRamForReducersOfMRJobs() 
+      throws Exception { 
+    final long inputSizeInMB = cSize * 250;
+    String tracePath = getTraceFile("highram_mr_jobs_case3");
+    Assert.assertNotNull("Trace file has not found.", tracePath);
+    String [] runtimeValues = {"LOADJOB",
+                               SubmitterUserResolver.class.getName(),
+                               "SERIAL",
+                               inputSizeInMB + "m",
+                               tracePath};
+
+    String [] otherArgs = {
+               "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=false", 
+               "-D", GridMixConfig.GRIDMIX_COMPRESSION_ENABLE + "=false", 
+               "-D", GridmixJob.GRIDMIX_HIGHRAM_EMULATION_ENABLE + "=true"};
+
+    validateTaskMemoryParamters(tracePath, true);
+    runGridmixAndVerify(runtimeValues, otherArgs, tracePath);
+  }
+}

+ 92 - 0
src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestGridmixEmulationOfMultipleHDFSPrivateDCFiles.java

@@ -0,0 +1,92 @@
+/**
+ * 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.mapred.gridmix;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.mapred.gridmix.test.system.GridMixConfig;
+import org.apache.hadoop.mapred.gridmix.test.system.GridMixRunMode;
+import org.apache.hadoop.mapred.gridmix.test.system.UtilsForGridmix;
+//import org.apache.hadoop.mapreduce.MRJobConfig;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Verify the Gridmix emulation of Multiple HDFS private distributed 
+ * cache files.
+ */
+public class TestGridmixEmulationOfMultipleHDFSPrivateDCFiles 
+    extends GridmixSystemTestCase {
+  private static final Log LOG = 
+      LogFactory.getLog(
+          "TestGridmixEmulationOfMultipleHDFSPrivateDCFiles.class");
+
+  /**
+   * Generate input data and multiple HDFS private distributed cache 
+   * files based on given input trace.Verify the Gridmix emulation of 
+   * multiple private HDFS distributed cache files in RoundRobinUserResolver 
+   * mode with SERIAL submission policy.
+   * @throws Exception - if an error occurs.
+   */
+  @Test
+  public void testGenerateAndEmulationOfMultipleHDFSPrivateDCFiles() 
+      throws Exception {
+    final long inputSize = 6144;
+    final String tracePath = getTraceFile("distcache_case4_trace");
+    Assert.assertNotNull("Trace file was not found.", tracePath);
+    final String [] runtimeValues = 
+                     {"LOADJOB",
+                      RoundRobinUserResolver.class.getName(),
+                      "SERIAL",
+                      inputSize+"m",
+                      "file://" + UtilsForGridmix.getProxyUsersFile(conf),
+                      tracePath};
+    final String [] otherArgs = {
+        "-D", JobContext.JOB_CANCEL_DELEGATION_TOKEN + "=false",
+        "-D", GridmixJob.GRIDMIX_HIGHRAM_EMULATION_ENABLE + "=false",
+        "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=true"
+    };
+    runGridmixAndVerify(runtimeValues, otherArgs, tracePath, 
+        GridMixRunMode.DATA_GENERATION_AND_RUN_GRIDMIX.getValue());
+  }
+
+  /**
+   * Verify the Gridmix emulation of multiple HDFS private distributed 
+   * cache files in SubmitterUserResolver mode with STRESS submission 
+   * policy by using the existing input data and HDFS private 
+   * distributed cache files.
+   * @throws Exception - if an error occurs.
+   */
+  @Test
+  public void testGridmixEmulationOfMultipleHDFSPrivateDCFiles() 
+      throws Exception {
+    final String tracePath = getTraceFile("distcache_case4_trace");
+    Assert.assertNotNull("Trace file was not found.", tracePath);
+    final String [] runtimeValues = {"LOADJOB",
+                                     SubmitterUserResolver.class.getName(),
+                                     "STRESS",
+                                     tracePath};
+    final String [] otherArgs = {
+        "-D", GridmixJob.GRIDMIX_HIGHRAM_EMULATION_ENABLE + "=false",
+        "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=true"
+    };
+    runGridmixAndVerify(runtimeValues, otherArgs, tracePath, 
+                        GridMixRunMode.RUN_GRIDMIX.getValue());
+  }
+}

+ 93 - 0
src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestGridmixEmulationOfMultipleHDFSPublicDCFiles.java

@@ -0,0 +1,93 @@
+/**
+ * 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.mapred.gridmix;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.mapred.gridmix.test.system.GridMixConfig;
+import org.apache.hadoop.mapred.gridmix.test.system.GridMixRunMode;
+import org.apache.hadoop.mapred.gridmix.test.system.UtilsForGridmix;
+//import org.apache.hadoop.mapreduce.MRJobConfig;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.junit.Assert;
+import org.junit.Test;
+import java.io.IOException;
+
+/**
+ * Verify the Gridmix emulation of Multiple HDFS public distributed 
+ * cache files.
+ */
+public class TestGridmixEmulationOfMultipleHDFSPublicDCFiles 
+    extends GridmixSystemTestCase { 
+  private static final Log LOG = 
+      LogFactory.getLog(
+          "TestGridmixEmulationOfMultipleHDFSPublicDCFiles.class");
+
+  /**
+   * Generate the compressed input data and dist cache files based 
+   * on input trace. Verify the Gridmix emulation of
+   * multiple HDFS public distributed cache file.
+   * @throws Exception - if an error occurs.
+   */
+  @Test
+  public void testGenerateAndEmulationOfMultipleHDFSDCFiles() 
+      throws Exception  {
+    final long inputSizeInMB = 7168;
+    final String tracePath = getTraceFile("distcache_case2_trace");
+    Assert.assertNotNull("Trace file was not found.", tracePath);
+    final String [] runtimeValues = 
+                     {"LOADJOB",
+                      RoundRobinUserResolver.class.getName(),
+                      "STRESS",
+                      inputSizeInMB + "m",
+                      "file://" + UtilsForGridmix.getProxyUsersFile(conf),
+                      tracePath};
+
+    final String [] otherArgs = { 
+       "-D", JobContext.JOB_CANCEL_DELEGATION_TOKEN + "=false",
+       "-D", GridmixJob.GRIDMIX_HIGHRAM_EMULATION_ENABLE + "=false",
+       "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=true"
+    };
+    runGridmixAndVerify(runtimeValues, otherArgs, tracePath, 
+        GridMixRunMode.DATA_GENERATION_AND_RUN_GRIDMIX.getValue());
+  }
+  
+  /**
+   * Verify the Gridmix emulation of Single HDFS public distributed cache file 
+   * by using an existing input compressed data and HDFS dist cache file. 
+   * @throws Exception - if an error occurs.
+   */
+  @Test
+  public void testGridmixEmulationOfMulitpleHDFSPublicDCFile() 
+      throws Exception {
+    final String tracePath = getTraceFile("distcache_case2_trace");
+    Assert.assertNotNull("Trace file was not found.", tracePath);
+    final String [] runtimeValues = {"LOADJOB",
+                                     SubmitterUserResolver.class.getName(),
+                                     "SERIAL",
+                                     tracePath};
+
+    final String [] otherArgs = {
+      "-D", GridmixJob.GRIDMIX_HIGHRAM_EMULATION_ENABLE + "=false",
+      "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=true"
+    };
+    runGridmixAndVerify(runtimeValues, otherArgs,  tracePath, 
+                        GridMixRunMode.RUN_GRIDMIX.getValue());
+  }
+}
+

+ 67 - 0
src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestGridmixWith10minTrace.java

@@ -0,0 +1,67 @@
+/**
+ * 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.mapred.gridmix;
+
+import org.apache.commons.logging.LogFactory;
+import org.apache.commons.logging.Log;
+import org.apache.hadoop.mapred.gridmix.test.system.UtilsForGridmix;
+import org.apache.hadoop.mapred.gridmix.test.system.GridMixConfig;
+import org.junit.Test;
+
+/**
+ * Run the Gridmix with 10 minutes MR jobs trace and 
+ * verify each job history against the corresponding job story 
+ * in a given trace file.
+ */
+public class TestGridmixWith10minTrace extends GridmixSystemTestCase {
+  private static final Log LOG = 
+      LogFactory.getLog(TestGridmixWith10minTrace.class);
+
+  /**
+   * Generate data and run gridmix by sleep jobs with STRESS submission 
+   * policy in a RoundRobinUserResolver mode against 10 minutes trace file.
+   * Verify each Gridmix job history with a corresponding job story 
+   * in a trace file after completion of all the jobs execution.
+   * @throws Exception - if an error occurs.
+   */
+  @Test
+  public void testGridmixWith10minTrace() throws Exception {
+    final long inputSizeInMB = cSize * 250;
+    final long minFileSize = 200 * 1024 * 1024;
+    String [] runtimeValues =
+               {"SLEEPJOB",
+                RoundRobinUserResolver.class.getName(),
+                "SERIAL",
+                inputSizeInMB + "m",
+                "file://" + UtilsForGridmix.getProxyUsersFile(conf),
+                map.get("10m")};
+
+    String [] otherArgs = {
+        "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=false",
+        "-D", GridMixConfig.GRIDMIX_COMPRESSION_ENABLE + "=false",
+        "-D", GridmixJob.GRIDMIX_HIGHRAM_EMULATION_ENABLE + "=false",
+        "-D", GridMixConfig.GRIDMIX_MINIMUM_FILE_SIZE + "=" + minFileSize,
+        "-D", GridMixConfig.GRIDMIX_JOB_SUBMISSION_QUEUE_IN_TRACE + "=false",
+        "-D", GridMixConfig.GRIDMIX_SLEEPJOB_MAPTASK_ONLY + "=true",
+        "-D", GridMixConfig.GRIDMIX_SLEEP_MAP_MAX_TIME + "=10"
+    };
+    String tracePath = map.get("10m");
+    runGridmixAndVerify(runtimeValues, otherArgs,tracePath);
+  }
+}

+ 62 - 0
src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestGridmixWith12minTrace.java

@@ -0,0 +1,62 @@
+/**
+ * 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.mapred.gridmix;
+
+import org.apache.commons.logging.LogFactory;
+import org.apache.commons.logging.Log;
+import org.apache.hadoop.mapred.gridmix.test.system.GridMixConfig;
+import org.junit.Test;
+
+/**
+ * Run the Gridmix with 12 minutes MR job traces and 
+ * verify each job history against the corresponding job story 
+ * in a given trace file.
+ */
+public class TestGridmixWith12minTrace extends GridmixSystemTestCase {
+  private static final Log LOG = 
+      LogFactory.getLog(TestGridmixWith12minTrace.class);
+ 
+  /**
+   * Generate data and run gridmix sleep jobs with REPLAY submission 
+   * policy in a SubmitterUserResolver mode against 12 minutes trace file.
+   * Verify each Gridmix job history with a corresponding job story 
+   * in a trace file after completion of all the jobs execution.
+   * @throws Exception - if an error occurs.
+   */
+  @Test
+  public void testGridmixWith12minTrace() throws Exception {
+    final long inputSizeInMB = cSize * 150;
+    String [] runtimeValues = {"SLEEPJOB",
+                               SubmitterUserResolver.class.getName(),
+                               "REPLAY",
+                               inputSizeInMB + "m",
+                               map.get("12m")};
+
+    String [] otherArgs = {
+        "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=false",
+        "-D", GridMixConfig.GRIDMIX_COMPRESSION_ENABLE + "=false",
+        "-D", GridmixJob.GRIDMIX_HIGHRAM_EMULATION_ENABLE + "=false",
+        "-D", GridMixConfig.GRIDMIX_SLEEP_MAP_MAX_TIME + "=10",
+        "-D", GridMixConfig.GRIDMIX_SLEEP_REDUCE_MAX_TIME + "=5"
+    };
+
+    String tracePath = map.get("12m");
+    runGridmixAndVerify(runtimeValues, otherArgs, tracePath);
+  }
+}

+ 59 - 0
src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestGridmixWith1minTrace.java

@@ -0,0 +1,59 @@
+/**
+ * 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.mapred.gridmix;
+
+import org.apache.commons.logging.LogFactory;
+import org.apache.commons.logging.Log;
+import org.junit.Test;
+import org.apache.hadoop.mapred.gridmix.test.system.GridMixConfig;
+
+/**
+ * Run the Gridmix with 1 minute MR jobs trace and 
+ * verify each job history against the corresponding job story 
+ * in a given trace file.
+ */
+public class TestGridmixWith1minTrace extends GridmixSystemTestCase{
+  private static final Log LOG = 
+      LogFactory.getLog(TestGridmixWith1minTrace.class);
+
+  /**
+   * Generate data and run gridmix by load job with STRESS submission policy
+   * in a SubmitterUserResolver mode against 1 minute trace file. 
+   * Verify each Gridmix job history with a corresponding job story in the 
+   * trace after completion of all the jobs execution.
+   * @throws Exception - if an error occurs.
+   */
+  @Test
+  public void testGridmixWith1minTrace() throws Exception {
+    final long inputSizeInMB = cSize * 400;
+    String [] runtimeValues = {"LOADJOB",
+                               SubmitterUserResolver.class.getName(),
+                               "STRESS",
+                               inputSizeInMB + "m",
+                               map.get("1m")};
+
+    String [] otherArgs = {
+        "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=false",
+        "-D", GridmixJob.GRIDMIX_HIGHRAM_EMULATION_ENABLE + "=false",
+        "-D", GridMixConfig.GRIDMIX_COMPRESSION_ENABLE + "=false"
+    };
+
+    String tracePath = map.get("1m");
+    runGridmixAndVerify(runtimeValues, otherArgs, tracePath);
+  }
+}

+ 64 - 0
src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestGridmixWith2minStreamingJobTrace.java

@@ -0,0 +1,64 @@
+/**
+ * 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.mapred.gridmix;
+
+import org.apache.commons.logging.LogFactory;
+import org.apache.commons.logging.Log;
+import org.apache.hadoop.mapred.gridmix.test.system.GridMixConfig;
+import org.junit.Test;
+import org.junit.Assert;
+
+/**
+ * Run the Gridmix with 2 minutes job trace which has been generated with 
+ * streaming jobs histories and verify each job history against 
+ * the corresponding job story in a given trace file.
+ */
+public class TestGridmixWith2minStreamingJobTrace 
+    extends GridmixSystemTestCase {
+  private static final Log LOG = 
+      LogFactory.getLog("TestGridmixWith2minStreamingJobTrace.class");
+
+  /**
+   * Generate input data and run Gridmix by load job with STRESS submission 
+   * policy in a SubmitterUserResolver mode against 2 minutes job 
+   * trace file of streaming jobs. Verify each Gridmix job history with 
+   * a corresponding job story in a trace file after completion of all 
+   * the jobs execution.  
+   * @throws Exception - if an error occurs.
+   */
+  @Test
+  public void testGridmixWith2minStreamJobTrace() throws Exception {
+    final long inputSizeInMB = cSize * 250;
+    final long minFileSize = 150 * 1024 * 1024;
+    String tracePath = getTraceFile("2m_stream");
+    Assert.assertNotNull("Trace file has not found.", tracePath);
+    String [] runtimeValues = {"LOADJOB",
+                               SubmitterUserResolver.class.getName(),
+                               "STRESS",
+                               inputSizeInMB + "m",
+                               tracePath};
+    String [] otherArgs = {
+        "-D", GridMixConfig.GRIDMIX_JOB_SUBMISSION_QUEUE_IN_TRACE + "=true",
+        "-D", GridMixConfig.GRIDMIX_MINIMUM_FILE_SIZE + "=" + minFileSize,
+        "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=false",
+        "-D", GridmixJob.GRIDMIX_HIGHRAM_EMULATION_ENABLE + "=false",
+        "-D", GridMixConfig.GRIDMIX_COMPRESSION_ENABLE + "=false"
+    };
+    runGridmixAndVerify(runtimeValues, otherArgs, tracePath);
+  }
+}

+ 68 - 0
src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestGridmixWith3minStreamingJobTrace.java

@@ -0,0 +1,68 @@
+/**
+ * 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.mapred.gridmix;
+
+import org.apache.commons.logging.LogFactory;
+import org.apache.commons.logging.Log;
+import org.apache.hadoop.mapred.gridmix.test.system.GridMixConfig;
+import org.apache.hadoop.mapred.gridmix.test.system.UtilsForGridmix;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Run the Gridmix with 3 minutes job trace which has been generated with 
+ * streaming jobs histories and verify each job history against 
+ * corresponding job story in a given trace file.
+ */
+public class TestGridmixWith3minStreamingJobTrace 
+    extends GridmixSystemTestCase {
+  private static final Log LOG = 
+     LogFactory.getLog("TestGridmixWith3minStreamingJobTrace.class");
+
+  /**
+   * Generate input data and run gridmix by load job with REPLAY submission 
+   * policy in a RoundRobinUserResolver mode against 3 minutes job trace file 
+   * of streaming job. Verify each gridmix job history with a corresponding 
+   * job story in a trace file after completion of all the jobs execution.
+   * @throws Exception - if an error occurs.
+   */
+  @Test
+  public void testGridmixWith3minStreamJobTrace() throws Exception {
+    final long inputSizeInMB = cSize * 200;
+    final long bytesPerFile = 150 * 1024 * 1024;
+    String tracePath = getTraceFile("3m_stream");
+    Assert.assertNotNull("Trace file has not found.", tracePath);
+    String [] runtimeValues = 
+               {"LOADJOB",
+                RoundRobinUserResolver.class.getName(),
+                "REPLAY",
+                inputSizeInMB + "m",
+                "file://" + UtilsForGridmix.getProxyUsersFile(conf),
+                tracePath};
+
+    String [] otherArgs = { 
+        "-D", GridMixConfig.GRIDMIX_JOB_SUBMISSION_QUEUE_IN_TRACE + "=true",
+        "-D", GridMixConfig.GRIDMIX_BYTES_PER_FILE + "=" + bytesPerFile,
+        "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=false",
+        "-D", GridmixJob.GRIDMIX_HIGHRAM_EMULATION_ENABLE + "=false",
+        "-D", GridMixConfig.GRIDMIX_COMPRESSION_ENABLE + "=false"
+    };
+
+    runGridmixAndVerify(runtimeValues, otherArgs, tracePath);
+  }
+}

+ 62 - 0
src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestGridmixWith3minTrace.java

@@ -0,0 +1,62 @@
+/**
+ * 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.mapred.gridmix;
+
+import org.apache.commons.logging.LogFactory;
+import org.apache.commons.logging.Log;
+import org.apache.hadoop.mapred.gridmix.test.system.UtilsForGridmix;
+import org.apache.hadoop.mapred.gridmix.test.system.GridMixConfig;
+import org.junit.Test;
+
+/**
+ * Run the Gridmix with 3 minutes MR jobs trace and 
+ * verify each job history against the corresponding job story 
+ * in a given trace file.
+ */
+public class TestGridmixWith3minTrace extends GridmixSystemTestCase {
+  private static final Log LOG = 
+      LogFactory.getLog(TestGridmixWith3minTrace.class);
+
+  /**
+   * Generate data and run gridmix by load job with REPLAY submission 
+   * policy in a RoundRobinUserResolver mode by using 3 minutes trace file. 
+   * Verify each Gridmix job history with a corresponding job story in 
+   * a trace after completion of all the jobs execution.  
+   * @throws Exception - if an error occurs.
+   */
+  @Test
+  public void testGridmixWith3minTrace() throws Exception {
+    final long inputSizeInMB = cSize * 200;
+    String [] runtimeValues = 
+              {"LOADJOB",
+               RoundRobinUserResolver.class.getName(),
+               "REPLAY",
+               inputSizeInMB + "m",
+               "file://" + UtilsForGridmix.getProxyUsersFile(conf),
+               map.get("3m")};
+
+    String [] otherArgs = {
+        "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=false",
+        "-D", GridmixJob.GRIDMIX_HIGHRAM_EMULATION_ENABLE + "=false",
+        "-D", GridMixConfig.GRIDMIX_COMPRESSION_ENABLE + "=false"
+    };
+
+    String tracePath = map.get("3m");
+    runGridmixAndVerify(runtimeValues, otherArgs, tracePath);  
+  }
+}

+ 65 - 0
src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestGridmixWith5minStreamingJobTrace.java

@@ -0,0 +1,65 @@
+/**
+ * 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.mapred.gridmix;
+
+import org.apache.commons.logging.LogFactory;
+import org.apache.commons.logging.Log;
+import org.apache.hadoop.mapred.gridmix.test.system.GridMixConfig;
+import org.junit.Test;
+import org.junit.Assert;
+
+/**
+ * Run the Gridmix with 5 minutes job trace which has been generated with 
+ * streaming jobs histories and verify each job history against 
+ * corresponding job story in a given trace file.
+ */
+public class TestGridmixWith5minStreamingJobTrace 
+    extends GridmixSystemTestCase {
+  private static final Log LOG = 
+      LogFactory.getLog("TestGridmixWith5minStreamingJobTrace.class");
+
+  /**
+   * Generate input data and run gridmix by load job with SERIAL submission 
+   * policy in a SubmitterUserResolver mode against 5 minutes job trace file 
+   * of streaming job. Verify each gridmix job history with a corresponding 
+   * job story in a trace file after completion of all the jobs execution.
+   * @throws Exception - if an error occurs.
+   */
+  @Test
+  public void testGridmixWith5minStreamJobTrace() throws Exception {
+    String tracePath = getTraceFile("5m_stream");
+    Assert.assertNotNull("Trace file has not found.", tracePath);
+    final long inputSizeInMB = cSize * 200;
+    final long bytesPerFile = 150 * 1024 * 1024;
+    String [] runtimeValues = {"LOADJOB", 
+                              SubmitterUserResolver.class.getName(), 
+                              "SERIAL", 
+                              inputSizeInMB + "m",
+                              tracePath};
+
+    String [] otherArgs = {
+        "-D", GridMixConfig.GRIDMIX_KEY_FRC + "=0.5f",
+        "-D", GridMixConfig.GRIDMIX_BYTES_PER_FILE + "=" + bytesPerFile,
+        "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=false",
+        "-D", GridmixJob.GRIDMIX_HIGHRAM_EMULATION_ENABLE + "=false",
+        "-D", GridMixConfig.GRIDMIX_COMPRESSION_ENABLE + "=false"
+    };
+
+    runGridmixAndVerify(runtimeValues, otherArgs, tracePath);
+  }
+}

+ 62 - 0
src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestGridmixWith5minTrace.java

@@ -0,0 +1,62 @@
+/**
+ * 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.mapred.gridmix;
+
+import org.apache.commons.logging.LogFactory;
+import org.apache.commons.logging.Log;
+import org.apache.hadoop.mapred.gridmix.test.system.GridMixConfig;
+import org.junit.Test;
+
+/**
+ * Run the Gridmix with 5 minutes MR jobs trace and 
+ * verify each job history against the corresponding job story 
+ * in a given trace file.
+ */
+public class TestGridmixWith5minTrace extends GridmixSystemTestCase {
+  private static final Log LOG = 
+      LogFactory.getLog(TestGridmixWith5minTrace.class);
+
+  /**
+   * Generate data and run gridmix by load job with SERIAL submission 
+   * policy in a SubmitterUserResolver mode against 5 minutes trace file. 
+   * Verify each Gridmix job history with a corresponding job story 
+   * in a trace file after completion of all the jobs.  
+   * @throws Exception - if an error occurs.
+   */
+  @Test
+  public void testGridmixWith5minTrace() throws Exception {
+    final long inputSizeInMB = cSize * 300;
+    final long minFileSize = 100 * 1024 * 1024;
+    String [] runtimeValues ={"LOADJOB", 
+                              SubmitterUserResolver.class.getName(), 
+                              "SERIAL", 
+                              inputSizeInMB + "m", 
+                              map.get("5m")};
+
+    String [] otherArgs = {
+        "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=false",
+        "-D", GridMixConfig.GRIDMIX_COMPRESSION_ENABLE + "=false",
+        "-D", GridmixJob.GRIDMIX_HIGHRAM_EMULATION_ENABLE + "=false",
+        "-D", GridMixConfig.GRIDMIX_MINIMUM_FILE_SIZE + "=" + minFileSize
+    };
+
+    String tracePath = map.get("5m");
+    runGridmixAndVerify(runtimeValues, otherArgs, tracePath);
+  }
+}
+

+ 62 - 0
src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestGridmixWith7minTrace.java

@@ -0,0 +1,62 @@
+/**
+ * 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.mapred.gridmix;
+
+import org.apache.commons.logging.LogFactory;
+import org.apache.commons.logging.Log;
+import org.apache.hadoop.mapred.gridmix.test.system.GridMixConfig;
+import org.junit.Test;
+
+/**
+ * Run the Gridmix with 7 minutes MR jobs trace and 
+ * verify each job history against the corresponding job story 
+ * in a given trace file.
+ */
+public class TestGridmixWith7minTrace extends GridmixSystemTestCase {
+  private static final Log LOG = 
+      LogFactory.getLog(TestGridmixWith7minTrace.class);
+
+  /**
+   * Generate data and run gridmix by sleep job with STRESS submission 
+   * policy in a SubmitterUserResolver mode against 7 minute trace file.
+   * Verify each Gridmix job history with a corresponding job story 
+   * in a trace file after completion of all the jobs execution.
+   * @throws Exception - if an error occurs.
+   */
+  @Test
+  public void testGridmixWith7minTrace() throws Exception {
+    final long inputSizeInMB = cSize * 400;
+    final long minFileSize = 200 * 1024 * 1024;
+    String [] runtimeValues ={"SLEEPJOB",
+                              SubmitterUserResolver.class.getName(),
+                              "STRESS",
+                              inputSizeInMB + "m",
+                              map.get("7m")};
+
+    String [] otherArgs = {
+        "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=false",
+        "-D", GridmixJob.GRIDMIX_HIGHRAM_EMULATION_ENABLE + "=false",
+        "-D", GridMixConfig.GRIDMIX_COMPRESSION_ENABLE + "=false",
+        "-D", GridMixConfig.GRIDMIX_MINIMUM_FILE_SIZE + "=" + minFileSize,
+        "-D", GridMixConfig.GRIDMIX_JOB_SUBMISSION_QUEUE_IN_TRACE + "=false"
+    };
+    String tracePath = map.get("7m");
+    runGridmixAndVerify(runtimeValues, otherArgs, tracePath);
+  }
+}

+ 106 - 0
src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestMemEmulForMapsAndReducesWithCustomIntrvl.java

@@ -0,0 +1,106 @@
+/**
+ * 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.mapred.gridmix;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.mapred.gridmix.test.system.GridMixConfig;
+import org.apache.hadoop.mapred.gridmix.test.system.GridMixRunMode;
+import org.apache.hadoop.mapred.gridmix.test.system.UtilsForGridmix;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.junit.Test;
+import org.junit.Assert;
+
+/**
+ * Test the {@link Gridmix} memory emulation feature for the jobs with 
+ * custom progress interval, different input data, submission policies
+ * and user resolver modes. Verify the total heap usage of map and reduce
+ * tasks of the jobs with corresponding original job in the trace. 
+ */
+public class TestMemEmulForMapsAndReducesWithCustomIntrvl 
+    extends GridmixSystemTestCase { 
+  private static final Log LOG = 
+      LogFactory.getLog("TestMemEmulForMapsAndReducesWithCustomIntrvl.class");
+  /**
+   * Generate compressed input and run {@link Gridmix} by turning on the
+   * memory emulation with custom progress interval. The {@link Gridmix} 
+   * should use the following runtime parameters while running the jobs.
+   * Submission Policy : STRESS, User Resolver Mode : SumitterUserResolver
+   * Verify maps and reduces total heap memory usage of {@link Gridmix} jobs 
+   * with corresponding original job in the trace. 
+   * @throws Exception - if an error occurs.
+   */
+  @Test
+  public void testMemoryEmulationForReducesWithCompressedInputCase7() 
+      throws Exception { 
+    final long inputSizeInMB = 1024 * 7;
+    String tracePath = getTraceFile("mem_emul_case2");
+    Assert.assertNotNull("Trace file has not found.", tracePath);
+    String [] runtimeValues = 
+            { "LOADJOB",
+              RoundRobinUserResolver.class.getName(),
+              "STRESS",
+              inputSizeInMB + "m",
+              "file://" + UtilsForGridmix.getProxyUsersFile(conf),
+              tracePath};
+
+    String [] otherArgs = {
+            "-D", GridMixConfig.GRIDMIX_MEMORY_EMULATION + "=" + 
+                GridMixConfig.GRIDMIX_MEMORY_EMULATION_PLUGIN,
+            "-D", GridMixConfig.GRIDMIX_HEAP_MEMORY_CUSTOM_INTRVL + "=0.3F",
+            "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=false",
+            "-D", JobContext.JOB_CANCEL_DELEGATION_TOKEN + "=false"};
+
+    runGridmixAndVerify(runtimeValues, otherArgs, tracePath, 
+           GridMixRunMode.DATA_GENERATION_AND_RUN_GRIDMIX.getValue());
+  }
+  
+  /**
+   * Generate uncompressed input and run {@link Gridmix} by turning on the
+   * memory emulation with custom progress interval. The {@link Gridmix}
+   * should use the following runtime parameters while running the jobs.
+   * Submission Policy : STRESS, User Resolver Mode : SumitterUserResolver
+   * Verify maps and reduces total heap memory usage of {@link Gridmix} jobs 
+   * with corresponding original job in the trace. 
+   * @throws Exception - if an error occurs.
+   */
+  @Test
+  public void testMemoryEmulationForReducesWithUncompressedInputCase8()
+      throws Exception {
+    final long inputSizeInMB = cSize * 300;
+    String tracePath = getTraceFile("mem_emul_case2");
+    Assert.assertNotNull("Trace file has not found.", tracePath);
+    String [] runtimeValues = 
+              { "LOADJOB", 
+                SubmitterUserResolver.class.getName(), 
+                "REPLAY", 
+                inputSizeInMB + "m",
+                tracePath};
+
+    String [] otherArgs = {
+            "-D", GridMixConfig.GRIDMIX_MEMORY_EMULATION + "=" + 
+                  GridMixConfig.GRIDMIX_MEMORY_EMULATION_PLUGIN,
+            "-D", GridMixConfig.GRIDMIX_HEAP_MEMORY_CUSTOM_INTRVL + "=0.2F",
+            "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=false",
+            "-D", GridMixConfig.GRIDMIX_COMPRESSION_ENABLE + "=false",
+            "-D", JobContext.JOB_CANCEL_DELEGATION_TOKEN + "=false"};
+
+    runGridmixAndVerify(runtimeValues, otherArgs, tracePath, 
+            GridMixRunMode.DATA_GENERATION_AND_RUN_GRIDMIX.getValue());
+  }
+}

+ 106 - 0
src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestMemEmulForMapsAndReducesWithDefaultIntrvl.java

@@ -0,0 +1,106 @@
+/**
+ * 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.mapred.gridmix;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.mapred.gridmix.test.system.GridMixConfig;
+import org.apache.hadoop.mapred.gridmix.test.system.GridMixRunMode;
+import org.apache.hadoop.mapred.gridmix.test.system.UtilsForGridmix;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.junit.Test;
+import org.junit.Assert;
+
+/**
+ * Test the {@link Gridmix} memory emulation feature for gridmix jobs
+ * with default progress interval, different input data, submission 
+ * policies and user resolver modes. Verify the total heap usage of
+ * map and reduce tasks of the jobs with corresponding original
+ * job in the trace. 
+ */
+public class TestMemEmulForMapsAndReducesWithDefaultIntrvl 
+    extends GridmixSystemTestCase { 
+  private static final Log LOG = 
+      LogFactory.getLog("TestMemEmulForMapsAndReducesWithDefaultIntrvl.class");
+
+  /**
+   * Generate compressed input and run {@link Gridmix} by turning on the
+   * memory emulation with default progress interval. The {@link Gridmix} 
+   * should use the following runtime parameters while running the jobs.
+   * Submission Policy : STRESS, User Resolver Mode : SumitterUserResolver
+   * Verify maps and reduces total heap memory usage of {@link Gridmix} jobs 
+   * with corresponding original job in the trace. 
+   * @throws Exception - if an error occurs.
+   */
+  @Test
+  public void testMemoryEmulationForReducesWithCompressedInputCase5() 
+      throws Exception { 
+    final long inputSizeInMB = 1024 * 7;
+    String tracePath = getTraceFile("mem_emul_case2");
+    Assert.assertNotNull("Trace file has not found.", tracePath);
+    String [] runtimeValues = 
+            { "LOADJOB",
+              RoundRobinUserResolver.class.getName(),
+              "STRESS",
+              inputSizeInMB + "m",
+              "file://" + UtilsForGridmix.getProxyUsersFile(conf),
+              tracePath};
+
+    String [] otherArgs = {
+            "-D", GridMixConfig.GRIDMIX_MEMORY_EMULATION + "=" + 
+                GridMixConfig.GRIDMIX_MEMORY_EMULATION_PLUGIN,
+            "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=false",
+            "-D", JobContext.JOB_CANCEL_DELEGATION_TOKEN + "=false"};
+
+    runGridmixAndVerify(runtimeValues, otherArgs, tracePath, 
+           GridMixRunMode.DATA_GENERATION_AND_RUN_GRIDMIX.getValue());
+  }
+
+  /**
+   * Generate uncompressed input and run {@link Gridmix} by turning on the
+   * memory emulation with default progress interval. The {@link Gridmix} 
+   * should use the following runtime parameters while running the jobs. 
+   * Submission Policy : STRESS, User Resolver Mode : SumitterUserResolver
+   * Verify maps and reduces total heap memory usage of {@link Gridmix} jobs 
+   * with corresponding original job in the trace. 
+   * @throws Exception - if an error occurs.
+   */
+  @Test
+  public void testMemoryEmulationForReducesWithUncompressedInputCase6()
+      throws Exception {
+    final long inputSizeInMB = cSize * 300;
+    String tracePath = getTraceFile("mem_emul_case2");
+    Assert.assertNotNull("Trace file has not found.", tracePath);
+    String [] runtimeValues = 
+              { "LOADJOB", 
+                SubmitterUserResolver.class.getName(), 
+                "REPLAY", 
+                inputSizeInMB + "m",
+                tracePath};
+
+    String [] otherArgs = {
+            "-D", GridMixConfig.GRIDMIX_MEMORY_EMULATION + "=" + 
+                  GridMixConfig.GRIDMIX_MEMORY_EMULATION_PLUGIN,
+            "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=false",
+            "-D", GridMixConfig.GRIDMIX_COMPRESSION_ENABLE + "=false",
+            "-D", JobContext.JOB_CANCEL_DELEGATION_TOKEN + "=false"};
+
+    runGridmixAndVerify(runtimeValues, otherArgs, tracePath, 
+            GridMixRunMode.DATA_GENERATION_AND_RUN_GRIDMIX.getValue());
+  }
+}

+ 108 - 0
src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestMemEmulForMapsWithCustomHeapMemoryRatio.java

@@ -0,0 +1,108 @@
+/**
+ * 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.mapred.gridmix;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.mapred.gridmix.test.system.GridMixConfig;
+import org.apache.hadoop.mapred.gridmix.test.system.GridMixRunMode;
+import org.apache.hadoop.mapred.gridmix.test.system.UtilsForGridmix;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.junit.Test;
+import org.junit.Assert;
+
+/**
+ * Test the {@link Gridmix} memory emulation feature for {@link Gridmix} jobs 
+ * with default progress interval, custom heap memory ratio, different input 
+ * data, submission policies and user resolver modes. Verify the total heap 
+ * usage of map and reduce tasks of the jobs with corresponding the original job 
+ * in the trace. 
+ */
+public class TestMemEmulForMapsWithCustomHeapMemoryRatio 
+    extends GridmixSystemTestCase { 
+  private static final Log LOG = 
+      LogFactory.getLog("TestMemEmulForMapsWithCustomHeapMemoryRatio.class");
+
+  /**
+   * Generate compressed input and run {@link Gridmix} by turning on the
+   * memory emulation. The {@link Gridmix} should use the following runtime 
+   * parameters while running the jobs.
+   * Submission Policy : STRESS, User Resolver Mode : SumitterUserResolver
+   * Verify total heap memory usage of the tasks of {@link Gridmix} jobs with 
+   * corresponding original job in the trace. 
+   * @throws Exception - if an error occurs.
+   */
+  @Test
+  public void testMemoryEmulationForMapsWithCompressedInputCase1() 
+     throws Exception {
+    final long inputSizeInMB = 1024 * 7;
+    String tracePath = getTraceFile("mem_emul_case2");
+    Assert.assertNotNull("Trace file has not found.", tracePath);
+    String [] runtimeValues = 
+            { "LOADJOB",
+              SubmitterUserResolver.class.getName(),
+              "STRESS",
+              inputSizeInMB + "m",
+              tracePath};
+
+    String [] otherArgs = { 
+            "-D", GridMixConfig.GRIDMIX_MEMORY_EMULATION + "="  +
+                  GridMixConfig.GRIDMIX_MEMORY_EMULATION_PLUGIN,
+            "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=false",
+            "-D", JobContext.JOB_CANCEL_DELEGATION_TOKEN + "=false",
+            "-D", GridMixConfig.GRIDMIX_HEAP_FREE_MEMORY_RATIO + "=0.5F"};
+
+    runGridmixAndVerify(runtimeValues, otherArgs, tracePath, 
+           GridMixRunMode.DATA_GENERATION_AND_RUN_GRIDMIX.getValue());
+  }
+  
+  /**
+   * Generate uncompressed input and run {@link Gridmix} by turning on the
+   * memory emulation. The {@link Gridmix} should use the following runtime 
+   * parameters while running the jobs.
+   *  Submission Policy : STRESS, User Resolver Mode : RoundRobinUserResolver
+   * Verify total heap memory usage of tasks of {@link Gridmix} jobs with 
+   * corresponding original job in the trace. 
+   * @throws Exception - if an error occurs.
+   */
+  @Test
+  public void testMemoryEmulationForMapsWithUncompressedInputCase2() 
+      throws Exception {
+    final long inputSizeInMB = cSize * 300;
+    String tracePath = getTraceFile("mem_emul_case2");
+    Assert.assertNotNull("Trace file has not found.", tracePath);
+    String [] runtimeValues = 
+              { "LOADJOB", 
+                RoundRobinUserResolver.class.getName(), 
+                "STRESS",
+                inputSizeInMB + "m",
+                "file://" + UtilsForGridmix.getProxyUsersFile(conf),
+                tracePath};
+
+    String [] otherArgs = {
+            "-D", GridMixConfig.GRIDMIX_MEMORY_EMULATION + "=" +  
+                  GridMixConfig.GRIDMIX_MEMORY_EMULATION_PLUGIN,
+            "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=false",
+            "-D", GridMixConfig.GRIDMIX_COMPRESSION_ENABLE + "=false",
+            "-D", JobContext.JOB_CANCEL_DELEGATION_TOKEN + "=false",
+            "-D", GridMixConfig.GRIDMIX_HEAP_FREE_MEMORY_RATIO + "=0.4F"};
+
+    runGridmixAndVerify(runtimeValues, otherArgs, tracePath, 
+            GridMixRunMode.DATA_GENERATION_AND_RUN_GRIDMIX.getValue());
+  }
+}

+ 106 - 0
src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestMemEmulForMapsWithCustomIntrvl.java

@@ -0,0 +1,106 @@
+/**
+ * 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.mapred.gridmix;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.mapred.gridmix.test.system.GridMixConfig;
+import org.apache.hadoop.mapred.gridmix.test.system.GridMixRunMode;
+import org.apache.hadoop.mapred.gridmix.test.system.UtilsForGridmix;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.junit.Test;
+import org.junit.Assert;
+
+/**
+ * Test the {@link Gridmix} memory emulation feature for {@link Gridmix} jobs 
+ * with custom progress interval, different input data, submission policies 
+ * and user resolver modes. Verify the total heap usage of map tasks of
+ * the jobs with corresponding the original job in the trace. 
+ */
+public class TestMemEmulForMapsWithCustomIntrvl extends GridmixSystemTestCase { 
+  private static final Log LOG = 
+      LogFactory.getLog("TestMemEmulForMapsWithCustomIntrvl.class");
+
+  /**
+   * Generate compressed input and run {@link Gridmix} by turning on the
+   * memory emulation with custom progress interval. The {@link Gridmix}
+   * should use the following runtime parameters while running the jobs.
+   * Submission Policy : STRESS, User Resolver Mode : SumitterUserResolver
+   * Verify maps total heap memory usage of {@link Gridmix} jobs with 
+   * corresponding original job in the trace. 
+   * @throws Exception - if an error occurs.
+   */
+  @Test
+  public void testMemoryEmulationForMapsWithCompressedInputCase3() 
+     throws Exception {
+    final long inputSizeInMB = 1024 * 7;
+    String tracePath = getTraceFile("mem_emul_case1");
+    Assert.assertNotNull("Trace file has not found.", tracePath);
+    String [] runtimeValues = 
+            { "LOADJOB",
+              SubmitterUserResolver.class.getName(),
+              "STRESS",
+              inputSizeInMB + "m",
+              tracePath};
+
+    String [] otherArgs = { 
+            "-D", GridMixConfig.GRIDMIX_MEMORY_EMULATION + "="  +
+                  GridMixConfig.GRIDMIX_MEMORY_EMULATION_PLUGIN,
+            "-D", GridMixConfig.GRIDMIX_HEAP_MEMORY_CUSTOM_INTRVL + "=0.2F",
+            "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=false",
+            "-D", JobContext.JOB_CANCEL_DELEGATION_TOKEN + "=false"};
+
+    runGridmixAndVerify(runtimeValues, otherArgs, tracePath, 
+           GridMixRunMode.DATA_GENERATION_AND_RUN_GRIDMIX.getValue());
+  }
+
+  /**
+   * Generate uncompressed input and run {@link Gridmix} by turning on the
+   * memory emulation with custom progress interval. The {@link Gridmix} 
+   * should use the following runtime parameters while running the jobs.
+   *  Submission Policy : STRESS, User Resolver Mode : RoundRobinUserResolver
+   * Verify maps total heap memory usage of {@link Gridmix} jobs with 
+   * corresponding original job in the trace. 
+   * @throws Exception - if an error occurs.
+   */
+  @Test
+  public void testMemoryEmulationForMapsWithUncompressedInputCase4() 
+      throws Exception {
+    final long inputSizeInMB = cSize * 300;
+    String tracePath = getTraceFile("mem_emul_case1");
+    Assert.assertNotNull("Trace file has not found.", tracePath);
+    String [] runtimeValues = 
+              { "LOADJOB", 
+                RoundRobinUserResolver.class.getName(), 
+                "STRESS",
+                inputSizeInMB + "m",
+                "file://" + UtilsForGridmix.getProxyUsersFile(conf),
+                tracePath};
+
+    String [] otherArgs = {
+            "-D", GridMixConfig.GRIDMIX_MEMORY_EMULATION + "=" +  
+                  GridMixConfig.GRIDMIX_MEMORY_EMULATION_PLUGIN,
+            "-D", GridMixConfig.GRIDMIX_HEAP_MEMORY_CUSTOM_INTRVL + "=0.3F",
+            "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=false",
+            "-D", GridMixConfig.GRIDMIX_COMPRESSION_ENABLE + "=false",
+            "-D", JobContext.JOB_CANCEL_DELEGATION_TOKEN + "=false"};
+
+    runGridmixAndVerify(runtimeValues, otherArgs, tracePath, 
+            GridMixRunMode.DATA_GENERATION_AND_RUN_GRIDMIX.getValue());
+  }
+}

+ 104 - 0
src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/TestMemEmulForMapsWithDefaultIntrvl.java

@@ -0,0 +1,104 @@
+/**
+ * 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.mapred.gridmix;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.mapred.gridmix.test.system.GridMixConfig;
+import org.apache.hadoop.mapred.gridmix.test.system.GridMixRunMode;
+import org.apache.hadoop.mapred.gridmix.test.system.UtilsForGridmix;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.junit.Test;
+import org.junit.Assert;
+
+/**
+ * Test the {@link Gridmix} memory emulation feature for {@link Gridmix} jobs 
+ * with default progress interval, different input data, submission policies 
+ * and user resolver modes. Verify the total heap usage of map tasks of the 
+ * jobs with corresponding original job in the trace. 
+ */
+public class TestMemEmulForMapsWithDefaultIntrvl extends GridmixSystemTestCase { 
+  private static final Log LOG = 
+          LogFactory.getLog("TestMemEmulForMapsWithDefaultIntrvl.class");
+
+  /**
+   * Generate compressed input and run {@link Gridmix} by turning on the
+   * memory emulation with default progress interval. The {@link Gridmix} 
+   * should use the following runtime parameters while running the jobs.
+   * Submission Policy : STRESS, User Resolver Mode : SumitterUserResolver
+   * Verify maps total heap memory usage of {@link Gridmix} jobs with 
+   * corresponding original job in the trace. 
+   * @throws Exception - if an error occurs.
+   */
+  @Test
+  public void testMemoryEmulationForMapsWithCompressedInputCase1() 
+     throws Exception {
+    final long inputSizeInMB = 1024 * 7;
+    String tracePath = getTraceFile("mem_emul_case1");
+    Assert.assertNotNull("Trace file has not found.", tracePath);
+    String [] runtimeValues = 
+            { "LOADJOB",
+              SubmitterUserResolver.class.getName(),
+              "STRESS",
+              inputSizeInMB + "m",
+              tracePath};
+
+    String [] otherArgs = { 
+            "-D", GridMixConfig.GRIDMIX_MEMORY_EMULATION + "="  +
+                  GridMixConfig.GRIDMIX_MEMORY_EMULATION_PLUGIN,
+            "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=false",
+            "-D", JobContext.JOB_CANCEL_DELEGATION_TOKEN + "=false"};
+
+    runGridmixAndVerify(runtimeValues, otherArgs, tracePath, 
+           GridMixRunMode.DATA_GENERATION_AND_RUN_GRIDMIX.getValue());
+  }
+  
+  /**
+   * Generate uncompressed input and run {@link Gridmix} by turning on the
+   * memory emulation with default progress interval. The {@link Gridmix} 
+   * should use the following runtime parameters while running the jobs.
+   *  Submission Policy : STRESS, User Resolver Mode : RoundRobinUserResolver
+   * Verify maps total heap memory usage of {@link Gridmix} jobs with 
+   * corresponding original job in the trace. 
+   * @throws Exception - if an error occurs.
+   */
+  @Test
+  public void testMemoryEmulationForMapsWithUncompressedInputCase2() 
+      throws Exception {
+    final long inputSizeInMB = cSize * 300;
+    String tracePath = getTraceFile("mem_emul_case1");
+    Assert.assertNotNull("Trace file has not found.", tracePath);
+    String [] runtimeValues = 
+              { "LOADJOB", 
+                RoundRobinUserResolver.class.getName(), 
+                "STRESS",
+                inputSizeInMB + "m",
+                "file://" + UtilsForGridmix.getProxyUsersFile(conf),
+                tracePath};
+
+    String [] otherArgs = {
+            "-D", GridMixConfig.GRIDMIX_MEMORY_EMULATION + "=" +  
+                  GridMixConfig.GRIDMIX_MEMORY_EMULATION_PLUGIN,
+            "-D", GridMixConfig.GRIDMIX_DISTCACHE_ENABLE + "=false",
+            "-D", GridMixConfig.GRIDMIX_COMPRESSION_ENABLE + "=false",
+            "-D", JobContext.JOB_CANCEL_DELEGATION_TOKEN + "=false"};
+
+    runGridmixAndVerify(runtimeValues, otherArgs, tracePath, 
+            GridMixRunMode.DATA_GENERATION_AND_RUN_GRIDMIX.getValue());
+  }
+}

+ 0 - 282
src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/UtilsForGridmix.java

@@ -1,282 +0,0 @@
-/**
- * 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.mapred.gridmix;
-
-import java.io.IOException;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.util.ToolRunner;
-import org.apache.hadoop.mapred.gridmix.Gridmix;
-import org.apache.hadoop.conf.Configuration;
-import java.util.Date;
-import java.text.SimpleDateFormat;
-import java.io.OutputStream;
-import java.util.Set;
-import java.util.List;
-import java.util.Iterator;
-import java.util.Map;
-import java.io.File;
-import java.io.FileOutputStream;
-import org.apache.hadoop.test.system.ProxyUserDefinitions;
-import org.apache.hadoop.test.system.ProxyUserDefinitions.GroupsAndHost;
-
-/**
- * Gridmix utilities.
- */
-public class UtilsForGridmix {
-  private static final Log LOG = LogFactory.getLog(UtilsForGridmix.class);
-
-  /**
-   * cleanup the folder or file.
-   * @param path - folder or file path.
-   * @param conf - cluster configuration 
-   * @throws IOException - If an I/O error occurs.
-   */
-  public static void cleanup(Path path, Configuration conf) 
-     throws IOException {
-    FileSystem fs = path.getFileSystem(conf);
-    fs.delete(path, true);
-    fs.close();
-  }
-
-  /**
-   * Get the login user.
-   * @return - login user as string..
-   * @throws IOException - if an I/O error occurs.
-   */
-  public static String getUserName() throws IOException {
-    return UserGroupInformation.getLoginUser().getUserName();
-  }
-  
-  /**
-   * Get the argument list for gridmix job.
-   * @param gridmixDir - gridmix parent directory.
-   * @param gridmixRunMode - gridmix modes either 1,2,3.
-   * @param values - gridmix runtime values.
-   * @param otherArgs - gridmix other generic args.
-   * @return - argument list as string array.
-   */
-  public static String [] getArgsList(Path gridmixDir, int gridmixRunMode, 
-    String [] values, String [] otherArgs) {
-    
-    String [] runtimeArgs = {
-       "-D", GridMixConfig.GRIDMIX_LOG_MODE + 
-       "=DEBUG",
-       "-D", GridMixConfig.GRIDMIX_OUTPUT_DIR + 
-       "=" + new Path(gridmixDir,"gridmix").toString(),
-       "-D", GridMixConfig.GRIDMIX_JOB_SUBMISSION_QUEUE_IN_TRACE 
-       + "=true",
-       "-D", GridMixConfig.GRIDMIX_JOB_TYPE 
-       + "=" + values[0],
-       "-D", GridMixConfig.GRIDMIX_USER_RESOLVER + 
-       "=" + values[1],
-       "-D", GridMixConfig.GRIDMIX_SUBMISSION_POLICY + 
-       "=" + values[2]
-    };
-    String [] classArgs;
-    if ((gridmixRunMode == GridMixRunMode.DATA_GENERATION || 
-      gridmixRunMode == GridMixRunMode.DATA_GENERATION_AND_RUN_GRIDMIX) && 
-      values[1].indexOf("RoundRobinUserResolver") > 0) {
-     classArgs = new String[]{
-        "-generate", values[3], 
-        "-users", values[4], 
-        new Path(gridmixDir,"input").toString(), 
-        values[5]};
-    } else if (gridmixRunMode == GridMixRunMode.DATA_GENERATION ||
-       gridmixRunMode == GridMixRunMode.DATA_GENERATION_AND_RUN_GRIDMIX){
-      classArgs = new String[]{
-         "-generate", values[3], new Path(gridmixDir,"input").toString(),
-         values[4]};
-    } else if(gridmixRunMode == GridMixRunMode.RUN_GRIDMIX 
-       && values[1].indexOf("RoundRobinUserResolver") > 0) {
-      classArgs = new String[]{         
-         "-users", values[3], 
-         new Path(gridmixDir,"input").toString(),
-         values[4]};
-    } else {
-      classArgs = new String[]{
-         new Path(gridmixDir,"input").toString(),values[3]};
-    }
-    
-    String [] args = new String [runtimeArgs.length + 
-       classArgs.length + ((otherArgs != null)?otherArgs.length:0)];
-    System.arraycopy(runtimeArgs, 0, args, 0, runtimeArgs.length);
-    if (otherArgs !=null) {
-      System.arraycopy(otherArgs, 0, args, runtimeArgs.length, 
-         otherArgs.length);
-      System.arraycopy(classArgs, 0, args, (runtimeArgs.length + 
-         otherArgs.length), classArgs.length);
-    } else {
-      System.arraycopy(classArgs, 0, args, runtimeArgs.length, 
-         classArgs.length);
-    }
-    return args;
-  }
-  
-  /**
-   * Create a file with specified size in mb.
-   * @param sizeInMB - file size in mb.
-   * @param inputDir - input directory.
-   * @param conf - cluster configuration.
-   * @throws Exception - if an exception occurs.
-   */
-  public static void createFile(int sizeInMB, Path inputDir, 
-     Configuration conf) throws Exception {
-    Date d = new Date();
-    SimpleDateFormat sdf = new SimpleDateFormat("ddMMyy_HHmmssS");
-    String formatDate = sdf.format(d);
-    FileSystem fs = inputDir.getFileSystem(conf);
-    OutputStream out = fs.create(new Path(inputDir,"datafile_" + formatDate));
-    final byte[] b = new byte[1024 * 1024];
-    for (int index = 0; index < sizeInMB; index++) {
-       out.write(b);
-    }    
-    out.close();
-    fs.close();
-  }
-  
-  /**
-   * Create directories for a path.
-   * @param path - directories path.
-   * @param conf  - cluster configuration.
-   * @throws IOException  - if an I/O error occurs.
-   */
-  public static void createDirs(Path path,Configuration conf) 
-     throws IOException {
-    FileSystem fs = path.getFileSystem(conf);
-    if (!fs.exists(path)) {
-       fs.mkdirs(path);
-    }
-  }
-  
-  /**
-   * Run the Gridmix job with given runtime arguments.
-   * @param gridmixDir - Gridmix parent directory.
-   * @param conf - cluster configuration.
-   * @param gridmixRunMode - gridmix run mode either 1,2,3
-   * @param runtimeValues -gridmix runtime values.
-   * @return - gridmix status either 0 or 1.
-   * @throws Exception
-   */
-  public static int runGridmixJob(Path gridmixDir, Configuration conf, 
-     int gridmixRunMode, String [] runtimeValues) throws Exception {
-    return runGridmixJob(gridmixDir, conf, gridmixRunMode, runtimeValues, null);
-  }
-  /**
-   * Run the Gridmix job with given runtime arguments.
-   * @param gridmixDir - Gridmix parent directory
-   * @param conf - cluster configuration.
-   * @param gridmixRunMode - gridmix run mode.
-   * @param runtimeValues - gridmix runtime values.
-   * @param otherArgs - gridmix other generic args.
-   * @return - gridmix status either 0 or 1.
-   * @throws Exception
-   */
-  
-  public static int runGridmixJob(Path gridmixDir, Configuration conf, 
-     int gridmixRunMode, String [] runtimeValues, 
-     String [] otherArgs) throws Exception {
-    Path  outputDir = new Path(gridmixDir, "gridmix");
-    Path inputDir = new Path(gridmixDir, "input");
-    LOG.info("Cleanup the data if data already exists.");
-    switch (gridmixRunMode) {
-      case GridMixRunMode.DATA_GENERATION :
-        cleanup(inputDir, conf);
-        cleanup(outputDir, conf);
-        break;
-      case GridMixRunMode.DATA_GENERATION_AND_RUN_GRIDMIX :
-        cleanup(inputDir, conf);
-        cleanup(outputDir, conf);
-        break;
-      case GridMixRunMode.RUN_GRIDMIX :
-        cleanup(outputDir, conf);
-        break;
-    }
-
-    final String [] args = UtilsForGridmix.getArgsList(gridmixDir,
-       gridmixRunMode, runtimeValues, otherArgs);
-    Gridmix gridmix = new Gridmix();
-    LOG.info("Submit a Gridmix job in " + runtimeValues[1] + 
-    " mode for " + GridMixRunMode.getMode(gridmixRunMode));
-    int exitCode = ToolRunner.run(conf, gridmix, args);
-    return exitCode;
-  }
-
-  /**
-   * Get the proxy users file.
-   * @param ProxyUserDefinitions - proxy users data container.
-   * @return String - proxy users file.
-   * @Exception - if no proxy users found in configuration.
-   */
-  public static String getProxyUsersFile(ProxyUserDefinitions pud)
-      throws Exception  {
-     String fileName = buildProxyUsersFile(pud.getProxyUsers());
-     if (fileName == null) {
-        LOG.error("Proxy users file not found.");
-        throw new Exception("Proxy users file not found.");
-     } else {
-        return fileName;
-     }
-  }
-
-  /**
-   * Build the proxy users file. 
-   * @param proxyUserData - groups and hostnames against proxy user.
-   * @return String - proxy users file.
-   * @throws Exception - if an exception occurs.
-   */
-  public static String buildProxyUsersFile(final Map<String, GroupsAndHost> 
-      proxyUserData) throws Exception {
-     FileOutputStream fos = null;
-     File file = null;
-     StringBuffer input = new StringBuffer();
-     Set users = proxyUserData.keySet();
-     Iterator itr = users.iterator();
-     while (itr.hasNext()) {
-        String user = itr.next().toString();
-        input.append(user);
-        final GroupsAndHost gah = proxyUserData.get(user);
-        final List <String> groups = gah.getGroups();
-        for (String group : groups) {
-           input.append(",");
-           input.append(group);
-        }
-        input.append("\n");
-     }
-     if (input.length() > 0) {
-        try {
-           file = File.createTempFile("proxyusers",null);
-           fos = new FileOutputStream(file);
-           fos.write(input.toString().getBytes());
-        } catch(IOException ioexp) {
-           LOG.warn(ioexp.getMessage());
-           return null;
-        } finally {
-           fos.close();
-           file.deleteOnExit();
-        }
-        LOG.info("file.toString():" + file.toString());
-        return file.toString();
-     } else {
-        return null;
-     }
-  }
-}

+ 286 - 0
src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/test/system/GridMixConfig.java

@@ -0,0 +1,286 @@
+/**
+ * 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.mapred.gridmix.test.system;
+
+import org.apache.hadoop.mapred.gridmix.Gridmix;
+import org.apache.hadoop.mapred.gridmix.JobCreator;
+import org.apache.hadoop.mapred.gridmix.SleepJob;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.JobTracker;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.filecache.DistributedCache;
+import org.apache.hadoop.mapred.gridmix.emulators.resourceusage.*;
+
+/**
+ * Gridmix system tests configurations. 
+ */
+public class GridMixConfig {
+
+  /**
+   *  Gridmix original job id.
+   */
+  public static final String GRIDMIX_ORIGINAL_JOB_ID = Gridmix.ORIGINAL_JOB_ID;
+
+  /**
+   *  Gridmix output directory.
+   */
+  public static final String GRIDMIX_OUTPUT_DIR = Gridmix.GRIDMIX_OUT_DIR; 
+
+  /**
+   * Gridmix job type (LOADJOB/SLEEPJOB).
+   */
+  public static final String GRIDMIX_JOB_TYPE = JobCreator.GRIDMIX_JOB_TYPE;
+
+  /**
+   *  Gridmix submission use queue.
+   */
+  /* In Gridmix package the visibility of below mentioned 
+  properties are protected and it have not visible outside 
+  the package. However,it should required for system tests, 
+  so it's re-defining in system tests config file.*/
+  public static final String GRIDMIX_JOB_SUBMISSION_QUEUE_IN_TRACE = 
+      "gridmix.job-submission.use-queue-in-trace";
+  
+  /**
+   *  Gridmix user resolver(RoundRobinUserResolver/
+   *  SubmitterUserResolver/EchoUserResolver).
+   */
+  public static final String GRIDMIX_USER_RESOLVER = Gridmix.GRIDMIX_USR_RSV;
+
+  /**
+   *  Gridmix queue depth.
+   */
+  public static final String GRIDMIX_QUEUE_DEPTH = Gridmix.GRIDMIX_QUE_DEP;
+
+  /* In Gridmix package the visibility of below mentioned 
+  property is protected and it should not available for 
+  outside the package. However,it should required for 
+  system tests, so it's re-defining in system tests config file.*/
+  /**
+   * Gridmix generate bytes per file.
+   */
+  public static final String GRIDMIX_BYTES_PER_FILE = 
+      "gridmix.gen.bytes.per.file";
+  
+  /**
+   *  Gridmix job submission policy(STRESS/REPLAY/SERIAL).
+   */
+
+  public static final String GRIDMIX_SUBMISSION_POLICY =
+      "gridmix.job-submission.policy";
+
+  /**
+   *  Gridmix minimum file size.
+   */
+  public static final String GRIDMIX_MINIMUM_FILE_SIZE =
+      "gridmix.min.file.size";
+
+  /**
+   * Gridmix key fraction.
+   */
+  public static final String GRIDMIX_KEY_FRC = 
+      "gridmix.key.fraction";
+
+  /**
+   * Gridmix compression enable
+   */
+  public static final String GRIDMIX_COMPRESSION_ENABLE =
+      "gridmix.compression-emulation.enable";
+  /**
+   * Gridmix distcache enable
+   */
+  public static final String GRIDMIX_DISTCACHE_ENABLE = 
+      "gridmix.distributed-cache-emulation.enable";
+
+  /**
+   * Gridmix input decompression enable.
+   */
+  public static final String GRIDMIX_INPUT_DECOMPRESS_ENABLE = 
+    "gridmix.compression-emulation.input-decompression.enable";
+
+  /**
+   * Gridmix input compression ratio.
+   */
+  public static final String GRIDMIX_INPUT_COMPRESS_RATIO = 
+    "gridmix.compression-emulation.map-input.decompression-ratio";
+
+  /**
+   * Gridmix intermediate compression ratio.
+   */
+  public static final String GRIDMIX_INTERMEDIATE_COMPRESSION_RATIO = 
+    "gridmix.compression-emulation.map-output.compression-ratio";
+
+  /**
+   * Gridmix output compression ratio.
+   */
+  public static final String GRIDMIX_OUTPUT_COMPRESSION_RATIO = 
+      "gridmix.compression-emulation.reduce-output.compression-ratio";
+
+  /**
+   * Gridmix distributed cache visibilities.
+   */
+  public static final String GRIDMIX_DISTCACHE_VISIBILITIES = 
+      JobContext.CACHE_FILE_VISIBILITIES;
+
+  /**
+   * Gridmix distributed cache files.
+   */
+  public static final String GRIDMIX_DISTCACHE_FILES = 
+      DistributedCache.CACHE_FILES;
+  
+  /**
+   * Gridmix distributed cache files size.
+   */
+  public static final String GRIDMIX_DISTCACHE_FILESSIZE = 
+      DistributedCache.CACHE_FILES_SIZES;
+
+  /**
+   * Gridmix distributed cache files time stamp.
+   */
+  public static final String GRIDMIX_DISTCACHE_TIMESTAMP =
+      DistributedCache.CACHE_FILES_TIMESTAMPS;
+
+  /**
+   *  Gridmix logger mode.
+   */
+  public static final String GRIDMIX_LOG_MODE =
+      "log4j.logger.org.apache.hadoop.mapred.gridmix";
+
+  /**
+   * Gridmix sleep job map task only.
+   */
+  public static final String GRIDMIX_SLEEPJOB_MAPTASK_ONLY = 
+      SleepJob.SLEEPJOB_MAPTASK_ONLY;
+
+  /**
+   * Gridmix sleep map maximum time.
+   */
+  public static final String GRIDMIX_SLEEP_MAP_MAX_TIME = 
+      SleepJob.GRIDMIX_SLEEP_MAX_MAP_TIME;
+
+  /**
+   * Gridmix sleep reduce maximum time.
+   */
+  public static final String GRIDMIX_SLEEP_REDUCE_MAX_TIME = 
+      SleepJob.GRIDMIX_SLEEP_MAX_REDUCE_TIME;
+
+  /**
+   * Gridmix high ram job emulation enable.
+   */
+  public static final String GRIDMIX_HIGH_RAM_JOB_ENABLE = 
+      "gridmix.highram-emulation.enable";
+
+  /**
+   * Job map memory in mb.
+   */
+  public static final String JOB_MAP_MEMORY_MB = 
+       JobConf.MAPRED_JOB_MAP_MEMORY_MB_PROPERTY;
+
+  /**
+   * Job reduce memory in mb.
+   */
+  public static final String JOB_REDUCE_MEMORY_MB = 
+      JobConf.MAPRED_JOB_REDUCE_MEMORY_MB_PROPERTY;
+
+  /**
+   * Cluster map memory in mb. 
+   */
+  public static final String CLUSTER_MAP_MEMORY = 
+      JobTracker.MAPRED_CLUSTER_MAP_MEMORY_MB_PROPERTY;
+
+  /**
+   * Cluster reduce memory in mb.
+   */
+  public static final String CLUSTER_REDUCE_MEMORY = 
+      JobTracker.MAPRED_CLUSTER_REDUCE_MEMORY_MB_PROPERTY;
+
+  /**
+   * Cluster maximum map memory.
+   */
+  public static final String CLUSTER_MAX_MAP_MEMORY = 
+      JobTracker.MAPRED_CLUSTER_MAX_MAP_MEMORY_MB_PROPERTY;
+
+  /**
+   * Cluster maximum reduce memory.
+   */
+  public static final String CLUSTER_MAX_REDUCE_MEMORY = 
+      JobTracker.MAPRED_CLUSTER_MAX_REDUCE_MEMORY_MB_PROPERTY;
+
+  /**
+   * Gridmix cpu emulation.
+   */
+  public static final String GRIDMIX_CPU_EMULATION =
+      ResourceUsageMatcher.RESOURCE_USAGE_EMULATION_PLUGINS;
+
+  /**
+   *  Gridmix cpu usage emulation plugin.
+   */
+  public static final String GRIDMIX_CPU_EMULATION_PLUGIN =
+      CumulativeCpuUsageEmulatorPlugin.class.getName();
+
+  /**
+   * Gridmix cpu emulation custom interval.
+   */
+  public static final String GRIDMIX_CPU_CUSTOM_INTERVAL =
+      CumulativeCpuUsageEmulatorPlugin.CPU_EMULATION_PROGRESS_INTERVAL;
+
+  /**
+   * Gridmix cpu emulation lower limit.
+   */
+  public static final int GRIDMIX_CPU_EMULATION_LOWER_LIMIT = 70;
+
+  /**
+   * Gridmix cpu emulation upper limit.
+   */
+  public static final int GRIDMIX_CPU_EMULATION_UPPER_LIMIT = 130;
+
+  /**
+  * Gridmix heap memory custom interval
+  */
+  public static final String GRIDMIX_HEAP_MEMORY_CUSTOM_INTRVL =
+      TotalHeapUsageEmulatorPlugin.HEAP_EMULATION_PROGRESS_INTERVAL;
+
+  /**
+   *  Gridmix heap free memory ratio
+   */
+  public static final String GRIDMIX_HEAP_FREE_MEMORY_RATIO =
+      TotalHeapUsageEmulatorPlugin.MIN_HEAP_FREE_RATIO;
+
+  /**
+   *  Gridmix memory emulation plugin
+   */
+  public static final String GRIDMIX_MEMORY_EMULATION_PLUGIN =
+      TotalHeapUsageEmulatorPlugin.class.getName();
+
+  /**
+   *  Gridmix memory emulation
+   */
+  public static final String GRIDMIX_MEMORY_EMULATION =
+      ResourceUsageMatcher.RESOURCE_USAGE_EMULATION_PLUGINS;
+
+  /**
+   *  Gridmix memory emulation lower limit.
+   */
+  public static int GRIDMIX_MEMORY_EMULATION_LOWER_LIMIT = 70;
+
+  /**
+   * Gridmix memory emulation upper limit. 
+   */
+  public static int GRIDMIX_MEMORY_EMULATION_UPPER_LIMIT = 130;
+}
+

+ 11 - 15
src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/GridMixRunMode.java → src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/test/system/GridMixRunMode.java

@@ -15,24 +15,20 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.hadoop.mapred.gridmix;
+package org.apache.hadoop.mapred.gridmix.test.system;
 /**
  * Gridmix run modes. 
  *
  */
-public class GridMixRunMode {
-   public static final int DATA_GENERATION = 1;
-   public static final int RUN_GRIDMIX = 2;
-   public static final int DATA_GENERATION_AND_RUN_GRIDMIX = 3;
-   private static String [] modeStr = {"DATA GENERATION",
-      "RUNNING GRIDMIX",
-      "DATA GENERATION AND RUNNING GRIDMIX"};
-   /**
-    * Get the appropriate message against the mode.
-    * @param mode - grimdix run mode either 1 or 2 or 3.
-    * @return - message as string.
-    */
-   public static String getMode(int mode){
-     return modeStr[mode-1];
+public enum GridMixRunMode {
+   DATA_GENERATION(1), RUN_GRIDMIX(2), DATA_GENERATION_AND_RUN_GRIDMIX(3);
+   private int mode;
+
+   GridMixRunMode (int mode) {
+      this.mode = mode;
+   }
+   
+   public int getValue() {
+     return mode;
    }
 }

+ 86 - 0
src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/test/system/GridmixJobStory.java

@@ -0,0 +1,86 @@
+/**
+ * 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.mapred.gridmix.test.system;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.JobID;
+import org.apache.hadoop.tools.rumen.ZombieJobProducer;
+import org.apache.hadoop.tools.rumen.ZombieJob;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+/**
+ * Build the job stories with a given trace file. 
+ */
+public class GridmixJobStory {
+  private static Log LOG = LogFactory.getLog(GridmixJobStory.class);
+  private Path path;
+  private Map<JobID, ZombieJob> zombieJobs;
+  private Configuration conf;
+  
+  public GridmixJobStory(Path path, Configuration conf) {
+    this.path = path;
+    this.conf = conf;
+    try {
+       zombieJobs = buildJobStories();
+       if(zombieJobs == null) {
+          throw new NullPointerException("No jobs found in a " 
+              + " given trace file.");
+       }
+    } catch (IOException ioe) {
+      LOG.warn("Error:" + ioe.getMessage());
+    } catch (NullPointerException npe) {
+      LOG.warn("Error:" + npe.getMessage());
+    }
+  }
+  
+  /**
+   * Get the zombie jobs as a map.
+   * @return the zombie jobs map.
+   */
+  public Map<JobID, ZombieJob> getZombieJobs() {
+    return zombieJobs;
+  }
+  
+  /**
+   * Get the zombie job of a given job id.
+   * @param jobId - gridmix job id.
+   * @return - the zombie job object.
+   */
+  public ZombieJob getZombieJob(JobID jobId) {
+    return zombieJobs.get(jobId);
+  }
+  
+  private Map<JobID, ZombieJob> buildJobStories() throws IOException {
+    ZombieJobProducer zjp = new ZombieJobProducer(path,null, conf);
+    Map<JobID, ZombieJob> hm = new HashMap<JobID, ZombieJob>();
+    ZombieJob zj = zjp.getNextJob();
+    while (zj != null) {
+      hm.put(zj.getJobID(),zj);
+      zj = zjp.getNextJob();
+    }
+    if (hm.size() == 0) {
+      return null;
+    } else {
+      return hm;
+    }
+  }
+}

+ 82 - 0
src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/test/system/GridmixJobSubmission.java

@@ -0,0 +1,82 @@
+/**
+ * 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.mapred.gridmix.test.system;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.test.system.JTClient;
+import org.junit.Assert;
+
+/**
+ * Submit the gridmix jobs. 
+ */
+public class GridmixJobSubmission {
+  private static final Log LOG = 
+      LogFactory.getLog(GridmixJobSubmission.class);
+  private int gridmixJobCount;
+  private Configuration conf;
+  private Path gridmixDir;
+  private JTClient jtClient;
+
+  public GridmixJobSubmission(Configuration conf, JTClient jtClient , 
+                              Path gridmixDir) { 
+    this.conf = conf;
+    this.jtClient = jtClient;
+    this.gridmixDir = gridmixDir;
+  }
+  
+  /**
+   * Submit the gridmix jobs.
+   * @param runtimeArgs - gridmix common runtime arguments.
+   * @param otherArgs - gridmix other runtime arguments.
+   * @param traceInterval - trace time interval.
+   * @throws Exception
+   */
+  public void submitJobs(String [] runtimeArgs, 
+                         String [] otherArgs, int mode) throws Exception {
+    int prvJobCount = jtClient.getClient().getAllJobs().length;
+    int exitCode = -1;
+    if (otherArgs == null) {
+      exitCode = UtilsForGridmix.runGridmixJob(gridmixDir, conf, 
+                                               mode, runtimeArgs);
+    } else {
+      exitCode = UtilsForGridmix.runGridmixJob(gridmixDir, conf, mode,
+                                               runtimeArgs, otherArgs);
+    }
+    Assert.assertEquals("Gridmix jobs have failed.", 0 , exitCode);
+    gridmixJobCount = jtClient.getClient().getAllJobs().length - prvJobCount;
+  }
+
+  /**
+   * Get the submitted jobs count.
+   * @return count of no. of jobs submitted for a trace.
+   */
+  public int getGridmixJobCount() {
+     return gridmixJobCount;
+  }
+
+  /**
+   * Get the job configuration.
+   * @return Configuration of a submitted job.
+   */
+  public Configuration getJobConf() {
+    return conf;
+  }
+}

+ 1239 - 0
src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/test/system/GridmixJobVerification.java

@@ -0,0 +1,1239 @@
+/**
+ * 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.mapred.gridmix.test.system;
+
+import java.io.IOException;
+import java.io.File;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import java.util.Collections;
+import java.util.Set;
+import java.util.ArrayList;
+import java.util.Arrays;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapred.Counters;
+import org.apache.hadoop.mapred.Counters.Counter;
+import org.apache.hadoop.mapred.Counters.Group;
+import org.apache.hadoop.mapred.Task;
+import org.apache.hadoop.mapred.DefaultJobHistoryParser;
+import org.apache.hadoop.mapred.JobHistory;
+import org.apache.hadoop.mapreduce.JobID;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.mapreduce.test.system.JTClient;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.tools.rumen.LoggedJob;
+import org.apache.hadoop.tools.rumen.ZombieJob;
+import org.apache.hadoop.tools.rumen.TaskInfo;
+import org.junit.Assert;
+import java.text.ParseException;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.mapred.gridmix.GridmixSystemTestCase;
+
+/**
+ * Verifying each Gridmix job with corresponding job story in a trace file.
+ */
+public class GridmixJobVerification {
+
+  private static Log LOG = LogFactory.getLog(GridmixJobVerification.class);
+  private Path path;
+  private Configuration conf;
+  private JTClient jtClient;
+  private String userResolverVal;
+  static final String origJobIdKey = GridMixConfig.GRIDMIX_ORIGINAL_JOB_ID;
+  static final String jobSubKey = GridMixConfig.GRIDMIX_SUBMISSION_POLICY;
+  static final String jobTypeKey = GridMixConfig.GRIDMIX_JOB_TYPE;
+  static final String mapTaskKey = GridMixConfig.GRIDMIX_SLEEPJOB_MAPTASK_ONLY;
+  static final String usrResolver = GridMixConfig.GRIDMIX_USER_RESOLVER;
+  static final String fileOutputFormatKey = "mapred.output.compress";
+  static final String fileInputFormatKey = "mapred.input.dir";
+  static final String compEmulKey = GridMixConfig.GRIDMIX_COMPRESSION_ENABLE;
+  static final String inputDecompKey = 
+      GridMixConfig.GRIDMIX_INPUT_DECOMPRESS_ENABLE;
+  static final String mapInputCompRatio = 
+      GridMixConfig.GRIDMIX_INPUT_COMPRESS_RATIO;
+  static final String mapOutputCompRatio = 
+      GridMixConfig.GRIDMIX_INTERMEDIATE_COMPRESSION_RATIO;
+  static final String reduceOutputCompRatio = 
+      GridMixConfig.GRIDMIX_OUTPUT_COMPRESSION_RATIO;
+  private Map<String, List<JobConf>> simuAndOrigJobsInfo = 
+      new HashMap<String, List<JobConf>>();
+
+  /**
+   * Gridmix job verification constructor
+   * @param path - path of the gridmix output directory.
+   * @param conf - cluster configuration.
+   * @param jtClient - jobtracker client.
+   */
+  public GridmixJobVerification(Path path, Configuration conf, 
+     JTClient jtClient) {
+    this.path = path;
+    this.conf = conf;
+    this.jtClient = jtClient;
+  }
+  
+  /**
+   * It verifies the Gridmix jobs with corresponding job story in a trace file.
+   * @param jobids - gridmix job ids.
+   * @throws IOException - if an I/O error occurs.
+   * @throws ParseException - if an parse error occurs.
+   */
+  public void verifyGridmixJobsWithJobStories(List<JobID> jobids) 
+      throws Exception {
+
+    SortedMap <Long, String> origSubmissionTime = new TreeMap <Long, String>();
+    SortedMap <Long, String> simuSubmissionTime = new TreeMap<Long, String>();
+    GridmixJobStory gjs = new GridmixJobStory(path, conf);
+    final Iterator<JobID> ite = jobids.iterator();
+    File destFolder = new File(System.getProperty("java.io.tmpdir") 
+                              + "/gridmix-st/");
+    destFolder.mkdir();
+    while (ite.hasNext()) {
+      JobID simuJobId = ite.next();
+      
+      JobHistory.JobInfo jhInfo = getSimulatedJobHistory(simuJobId);
+      Assert.assertNotNull("Job history not found.", jhInfo);
+      Counters counters = 
+          Counters.fromEscapedCompactString(jhInfo.getValues()
+              .get(JobHistory.Keys.COUNTERS));
+      JobConf simuJobConf = getSimulatedJobConf(simuJobId, destFolder);
+      int cnt = 1;
+      do {
+        if (simuJobConf != null) {
+          break;
+        }
+        Thread.sleep(100);
+        simuJobConf = getSimulatedJobConf(simuJobId, destFolder);
+        cnt++;
+      } while(cnt < 30);
+
+      String origJobId = simuJobConf.get(origJobIdKey);
+      LOG.info("OriginalJobID<->CurrentJobID:" 
+              + origJobId + "<->" + simuJobId);
+
+      if (userResolverVal == null) {
+        userResolverVal = simuJobConf.get(usrResolver);
+      }
+    
+      ZombieJob zombieJob = gjs.getZombieJob(JobID.forName(origJobId));
+      Map<String, Long> mapJobCounters = getJobMapCounters(zombieJob);
+      Map<String, Long> reduceJobCounters = getJobReduceCounters(zombieJob);
+      if (simuJobConf.get(jobSubKey).contains("REPLAY")) {
+          origSubmissionTime.put(zombieJob.getSubmissionTime(), 
+                                 origJobId.toString() + "^" + simuJobId); 
+          simuSubmissionTime.put(Long.parseLong(jhInfo.getValues().get(JobHistory.Keys.SUBMIT_TIME)), 
+                                 origJobId.toString() + "^" + simuJobId); ;
+      }
+
+      LOG.info("Verifying the job <" + simuJobId + "> and wait for a while...");
+      verifySimulatedJobSummary(zombieJob, jhInfo, simuJobConf);
+      verifyJobMapCounters(counters, mapJobCounters, simuJobConf);
+      verifyJobReduceCounters(counters, reduceJobCounters, simuJobConf); 
+      verifyCompressionEmulation(zombieJob.getJobConf(), simuJobConf, counters, 
+                                 reduceJobCounters, mapJobCounters);
+      verifyDistributeCache(zombieJob,simuJobConf);
+      setJobDistributedCacheInfo(simuJobId.toString(), simuJobConf, 
+         zombieJob.getJobConf());
+      verifyHighRamMemoryJobs(zombieJob, simuJobConf);
+      verifyCPUEmulationOfJobs(zombieJob, jhInfo, simuJobConf);
+      verifyMemoryEmulationOfJobs(zombieJob, jhInfo, simuJobConf);
+      LOG.info("Done.");
+    }
+    verifyDistributedCacheBetweenJobs(simuAndOrigJobsInfo);
+  }
+
+  /**
+   * Verify the job submission order between the jobs in replay mode.
+   * @param origSubmissionTime - sorted map of original jobs submission times.
+   * @param simuSubmissionTime - sorted map of simulated jobs submission times.
+   */
+  public void verifyJobSumissionTime(SortedMap<Long, String> origSubmissionTime, 
+      SortedMap<Long, String> simuSubmissionTime) { 
+    Assert.assertEquals("Simulated job's submission time count has " 
+                     + "not match with Original job's submission time count.", 
+                     origSubmissionTime.size(), simuSubmissionTime.size());
+    for ( int index = 0; index < origSubmissionTime.size(); index ++) {
+        String origAndSimuJobID = origSubmissionTime.get(index);
+        String simuAndorigJobID = simuSubmissionTime.get(index);
+        Assert.assertEquals("Simulated jobs have not submitted in same " 
+                           + "order as original jobs submitted in REPLAY mode.", 
+                           origAndSimuJobID, simuAndorigJobID);
+    }
+  }
+
+  /**
+   * It verifies the simulated job map counters.
+   * @param counters - Original job map counters.
+   * @param mapJobCounters - Simulated job map counters.
+   * @param jobConf - Simulated job configuration.
+   * @throws ParseException - If an parser error occurs.
+   */
+  public void verifyJobMapCounters(Counters counters, 
+     Map<String,Long> mapCounters, JobConf jobConf) throws ParseException {
+    if (!jobConf.get(jobTypeKey, "LOADJOB").equals("SLEEPJOB")) {
+      Assert.assertEquals("Map input records have not matched.",
+                          mapCounters.get("MAP_INPUT_RECS").longValue(), 
+                          getCounterValue(counters, "MAP_INPUT_RECORDS"));
+    } else {
+      Assert.assertTrue("Map Input Bytes are zero", 
+                        getCounterValue(counters,"HDFS_BYTES_READ") != 0);
+      Assert.assertNotNull("Map Input Records are zero", 
+                           getCounterValue(counters, "MAP_INPUT_RECORDS")!=0);
+    }
+  }
+
+  /**
+   *  It verifies the simulated job reduce counters.
+   * @param counters - Original job reduce counters.
+   * @param reduceCounters - Simulated job reduce counters.
+   * @param jobConf - simulated job configuration.
+   * @throws ParseException - if an parser error occurs.
+   */
+  public void verifyJobReduceCounters(Counters counters, 
+     Map<String,Long> reduceCounters, JobConf jobConf) throws ParseException {
+    if (jobConf.get(jobTypeKey, "LOADJOB").equals("SLEEPJOB")) {
+      Assert.assertTrue("Reduce output records are not zero for sleep job.",
+          getCounterValue(counters, "REDUCE_OUTPUT_RECORDS") == 0);
+      Assert.assertTrue("Reduce output bytes are not zero for sleep job.", 
+          getCounterValue(counters,"HDFS_BYTES_WRITTEN") == 0);
+    }
+  }
+
+  /**
+   * It verifies the gridmix simulated job summary.
+   * @param zombieJob - Original job summary.
+   * @param jhInfo  - Simulated job history info.
+   * @param jobConf - simulated job configuration.
+   * @throws IOException - if an I/O error occurs.
+   */
+  public void verifySimulatedJobSummary(ZombieJob zombieJob, 
+     JobHistory.JobInfo jhInfo, JobConf jobConf) throws IOException {
+    Assert.assertEquals("Job id has not matched", zombieJob.getJobID(), 
+                        JobID.forName(jobConf.get(origJobIdKey)));
+
+    Assert.assertEquals("Job maps have not matched", String.valueOf(zombieJob.getNumberMaps()),
+                        jhInfo.getValues().get(JobHistory.Keys.TOTAL_MAPS));
+
+    if (!jobConf.getBoolean(mapTaskKey, false)) { 
+      Assert.assertEquals("Job reducers have not matched", 
+          String.valueOf(zombieJob.getNumberReduces()), jhInfo.getValues().get(JobHistory.Keys.TOTAL_REDUCES));
+    } else {
+      Assert.assertEquals("Job reducers have not matched",
+                          0, Integer.parseInt(jhInfo.getValues().get(JobHistory.Keys.TOTAL_REDUCES)));
+    }
+
+    Assert.assertEquals("Job status has not matched.", 
+                        zombieJob.getOutcome().name(), 
+                        convertJobStatus(jhInfo.getValues().get(JobHistory.Keys.JOB_STATUS)));
+
+    LoggedJob loggedJob = zombieJob.getLoggedJob();
+    Assert.assertEquals("Job priority has not matched.", 
+                        loggedJob.getPriority().toString(), 
+                        jhInfo.getValues().get(JobHistory.Keys.JOB_PRIORITY));
+
+    if (jobConf.get(usrResolver).contains("RoundRobin")) {
+       String user = UserGroupInformation.getLoginUser().getShortUserName();
+       Assert.assertTrue(jhInfo.getValues().get(JobHistory.Keys.JOBID).toString() 
+                        + " has not impersonate with other user.", 
+                        !jhInfo.getValues().get(JobHistory.Keys.USER).equals(user));
+    }
+  }
+
+  /**
+   * Get the original job map counters from a trace.
+   * @param zombieJob - Original job story.
+   * @return - map counters as a map.
+   */
+  public Map<String, Long> getJobMapCounters(ZombieJob zombieJob) {
+    long expMapInputBytes = 0;
+    long expMapOutputBytes = 0;
+    long expMapInputRecs = 0;
+    long expMapOutputRecs = 0;
+    Map<String,Long> mapCounters = new HashMap<String,Long>();
+    for (int index = 0; index < zombieJob.getNumberMaps(); index ++) {
+      TaskInfo mapTask = zombieJob.getTaskInfo(TaskType.MAP, index);
+      expMapInputBytes += mapTask.getInputBytes();
+      expMapOutputBytes += mapTask.getOutputBytes();
+      expMapInputRecs += mapTask.getInputRecords();
+      expMapOutputRecs += mapTask.getOutputRecords();
+    }
+    mapCounters.put("MAP_INPUT_BYTES", expMapInputBytes);
+    mapCounters.put("MAP_OUTPUT_BYTES", expMapOutputBytes);
+    mapCounters.put("MAP_INPUT_RECS", expMapInputRecs);
+    mapCounters.put("MAP_OUTPUT_RECS", expMapOutputRecs);
+    return mapCounters;
+  }
+  
+  /**
+   * Get the original job reduce counters from a trace.
+   * @param zombieJob - Original job story.
+   * @return - reduce counters as a map.
+   */
+  public Map<String,Long> getJobReduceCounters(ZombieJob zombieJob) {
+    long expReduceInputBytes = 0;
+    long expReduceOutputBytes = 0;
+    long expReduceInputRecs = 0;
+    long expReduceOutputRecs = 0;
+    Map<String,Long> reduceCounters = new HashMap<String,Long>();
+    for (int index = 0; index < zombieJob.getNumberReduces(); index ++) {
+      TaskInfo reduceTask = zombieJob.getTaskInfo(TaskType.REDUCE, index);
+      expReduceInputBytes += reduceTask.getInputBytes();
+      expReduceOutputBytes += reduceTask.getOutputBytes();
+      expReduceInputRecs += reduceTask.getInputRecords();
+      expReduceOutputRecs += reduceTask.getOutputRecords();
+    }
+    reduceCounters.put("REDUCE_INPUT_BYTES", expReduceInputBytes);
+    reduceCounters.put("REDUCE_OUTPUT_BYTES", expReduceOutputBytes);
+    reduceCounters.put("REDUCE_INPUT_RECS", expReduceInputRecs);
+    reduceCounters.put("REDUCE_OUTPUT_RECS", expReduceOutputRecs);
+    return reduceCounters;
+  }
+
+  /**
+   * Get the simulated job configuration of a job.
+   * @param simulatedJobID - Simulated job id.
+   * @param tmpJHFolder - temporary job history folder location.
+   * @return - simulated job configuration.
+   * @throws IOException - If an I/O error occurs.
+   */
+  public JobConf getSimulatedJobConf(JobID simulatedJobID, File tmpJHFolder) 
+      throws IOException, InterruptedException {
+    FileSystem fs = null;
+    try {
+
+      String historyFilePath = jtClient.getProxy()
+          .getJobHistoryLocationForRetiredJob(simulatedJobID);
+      int cnt = 0;
+      do {
+        if (historyFilePath != null) {
+           break;
+        }
+        Thread.sleep(100);
+        historyFilePath = jtClient.getProxy()
+            .getJobHistoryLocationForRetiredJob(simulatedJobID);
+        cnt++;
+      } while( cnt < 30 );
+      Assert.assertNotNull("History file has not available for the job ["
+              + simulatedJobID + "] for 3 secs.", historyFilePath);
+      Path jhpath = new Path(historyFilePath);
+      LOG.info("Parent:" + jhpath.getParent());
+      fs = jhpath.getFileSystem(conf);
+      fs.copyToLocalFile(jhpath,new Path(tmpJHFolder.toString()));
+      fs.copyToLocalFile(new Path(jhpath.getParent() + "/" + simulatedJobID + "_conf.xml"), 
+                         new Path(tmpJHFolder.toString()));
+      JobConf jobConf = new JobConf();
+      jobConf.addResource(new Path(tmpJHFolder.toString() 
+                         + "/" + simulatedJobID + "_conf.xml"));
+      jobConf.reloadConfiguration();
+      return jobConf;
+
+    }finally {
+      fs.close();
+    }
+  }
+
+  /**
+   * Get the simulated job history of a job.
+   * @param simulatedJobID - simulated job id.
+   * @return - simulated job information.
+   * @throws IOException - if an I/O error occurs.
+   */
+  public JobHistory.JobInfo getSimulatedJobHistory(JobID simulatedJobID) 
+      throws IOException, InterruptedException {
+    FileSystem fs = null;
+    try {
+      String historyFilePath = jtClient.getProxy().
+          getJobHistoryLocationForRetiredJob(simulatedJobID);
+      int cnt = 0;
+      do {
+        if (historyFilePath != null) {
+          break;
+        }
+        Thread.sleep(100);
+        historyFilePath = jtClient.getProxy()
+            .getJobHistoryLocationForRetiredJob(simulatedJobID);
+        cnt++;
+      } while( cnt < 30 ); 
+      LOG.info("HistoryFilePath:" + historyFilePath);
+      Assert.assertNotNull("History file path has not found for a job[" 
+              + simulatedJobID + "] for 3 secs.");
+      Path jhpath = new Path(historyFilePath);
+      fs = jhpath.getFileSystem(conf);
+      JobHistory.JobInfo jobInfo = 
+          new JobHistory.JobInfo(simulatedJobID.toString());
+      DefaultJobHistoryParser.parseJobTasks(historyFilePath, jobInfo, fs);
+      return jobInfo;
+    } finally {
+      fs.close();
+    }
+  }
+
+  /**
+   * It verifies the cpu resource usage of gridmix jobs against
+   * the original job cpu resource usage.
+   * @param origJobHistory - Original job history.
+   * @param simuJobHistoryInfo - Simulated job history.
+   * @param simuJobConf - simulated job configuration.
+   */
+  public void verifyCPUEmulationOfJobs(ZombieJob origJobHistory,
+                                   JobHistory.JobInfo simuJobHistoryInfo,
+                                   JobConf simuJobConf) throws Exception {
+    boolean isCPUEmulON = false;
+    if (simuJobConf.get(GridMixConfig.GRIDMIX_CPU_EMULATION) != null) {
+      isCPUEmulON = 
+          simuJobConf.get(GridMixConfig.GRIDMIX_CPU_EMULATION).
+              contains(GridMixConfig.GRIDMIX_CPU_EMULATION_PLUGIN);
+    }
+   
+    if (isCPUEmulON) {
+      Map<String,Long> origJobMetrics =
+                       getOriginalJobCPUMetrics(origJobHistory);
+      Map<String,Long> simuJobMetrics =
+                       getSimulatedJobCPUMetrics(simuJobHistoryInfo);
+
+      long origMapUsage = origJobMetrics.get("MAP");
+      LOG.info("Total cpu usage of Maps for a original job:" + origMapUsage);
+
+      long origReduceUsage = origJobMetrics.get("REDUCE");
+      LOG.info("Total cpu usage of Reduces for a original job:" 
+              + origReduceUsage);
+
+      long simuMapUsage = simuJobMetrics.get("MAP");
+      LOG.info("Total cpu usage of Maps for a simulated job:" + simuMapUsage);
+
+      long simuReduceUsage = simuJobMetrics.get("REDUCE");
+      LOG.info("Total cpu usage of Reduces for a simulated job:" 
+              + simuReduceUsage);
+
+      int mapCount = Integer.parseInt(
+          simuJobHistoryInfo.getValues().get(JobHistory.Keys.TOTAL_MAPS));
+      int reduceCount = Integer.parseInt(
+          simuJobHistoryInfo.getValues().get(JobHistory.Keys.TOTAL_REDUCES));
+
+      if (mapCount > 0) {
+        double mapEmulFactor = (simuMapUsage * 100) / origMapUsage;
+        long mapEmulAccuracy = Math.round(mapEmulFactor);
+        LOG.info("CPU emulation accuracy for maps in job " +
+                 simuJobHistoryInfo.getValues().get(JobHistory.Keys.JOBID) +
+                 ":"+ mapEmulAccuracy + "%");
+        Assert.assertTrue("Map-side cpu emulaiton inaccurate!" +
+                          " Actual cpu usage: " + simuMapUsage +
+                          " Expected cpu usage: " + origMapUsage, mapEmulAccuracy 
+                          >= GridMixConfig.GRIDMIX_CPU_EMULATION_LOWER_LIMIT
+                          && mapEmulAccuracy 
+                          <= GridMixConfig.GRIDMIX_CPU_EMULATION_UPPER_LIMIT);
+      }
+
+      if (reduceCount >0) {
+        double reduceEmulFactor = (simuReduceUsage * 100) / origReduceUsage;
+        long reduceCPUUsage = simuReduceUsage / 1000;
+        LOG.info("Reduce CPU Usage:" + reduceCPUUsage);
+        LOG.info("Reduce emulation factor:" + reduceEmulFactor);
+        long reduceEmulAccuracy = Math.round(reduceEmulFactor);
+        LOG.info("CPU emulation accuracy for reduces in job " +
+                 simuJobHistoryInfo.getValues().get(JobHistory.Keys.JOBID) + 
+                 ": " + reduceEmulAccuracy + "%");
+        if ( reduceCPUUsage >= 10 ) {
+          Assert.assertTrue("Reduce side cpu emulaiton inaccurate!" +
+                            " Actual cpu usage:" + simuReduceUsage +
+                            "Expected cpu usage: " + origReduceUsage,
+                            reduceEmulAccuracy 
+                            >= GridMixConfig.GRIDMIX_CPU_EMULATION_LOWER_LIMIT
+                            && reduceEmulAccuracy 
+                            <= GridMixConfig.GRIDMIX_CPU_EMULATION_UPPER_LIMIT);
+        } else {
+          Assert.assertTrue("Reduce side cpu emulaiton inaccurate!" +
+                            " Actual cpu usage:" + simuReduceUsage +
+                            "Expected cpu usage: " + origReduceUsage,
+                            reduceEmulAccuracy
+                            >= 60 && reduceEmulAccuracy <= 100);
+        } 
+      }
+    }
+  }
+
+  /**
+   * It verifies the heap memory resource usage of gridmix jobs with
+   * corresponding original job in the trace.
+   * @param zombieJob - Original job history.
+   * @param jhInfo - Simulated job history.
+   * @param simuJobConf - simulated job configuration.
+   */
+  public void verifyMemoryEmulationOfJobs(ZombieJob zombieJob,
+          JobHistory.JobInfo jhInfo, JobConf simuJobConf) throws Exception {
+    long origJobMapsTHU = 0;
+    long origJobReducesTHU = 0;
+    long simuJobMapsTHU = 0;
+    long simuJobReducesTHU = 0;
+    boolean isMemEmulOn = false;
+    String strHeapRatio = "0.3F";
+
+    if (simuJobConf.get(GridMixConfig.GRIDMIX_MEMORY_EMULATION) != null) {
+      isMemEmulOn =
+          simuJobConf.get(GridMixConfig.GRIDMIX_MEMORY_EMULATION).
+              contains(GridMixConfig.GRIDMIX_MEMORY_EMULATION_PLUGIN);
+    }
+    
+    if (isMemEmulOn) {
+      
+      for (int index = 0; index < zombieJob.getNumberMaps(); index ++) {
+        TaskInfo mapTask = zombieJob.getTaskInfo(TaskType.MAP, index);
+        if (mapTask.getResourceUsageMetrics().getHeapUsage() > 0) {
+          origJobMapsTHU +=
+                  mapTask.getResourceUsageMetrics().getHeapUsage();
+        }
+      }
+      LOG.info("Total Heap Usage of Maps for original job: " 
+              + origJobMapsTHU);
+
+      for (int index = 0; index < zombieJob.getNumberReduces(); index ++) {
+        TaskInfo reduceTask = zombieJob.getTaskInfo(TaskType.REDUCE, index);
+        if (reduceTask.getResourceUsageMetrics().getHeapUsage() > 0) {
+          origJobReducesTHU +=
+                  reduceTask.getResourceUsageMetrics().getHeapUsage();
+        }
+      }
+      LOG.info("Total Heap Usage of Reduces for original job: " 
+              + origJobReducesTHU);
+      
+      Counters mapCounters = 
+          Counters.fromEscapedCompactString(jhInfo.getValues()
+                  .get(JobHistory.Keys.MAP_COUNTERS));
+
+      Counters reduceCounters = 
+          Counters.fromEscapedCompactString(jhInfo.getValues()
+                  .get(JobHistory.Keys.REDUCE_COUNTERS));
+
+      simuJobMapsTHU = 
+          getCounterValue(mapCounters, 
+              Task.Counter.COMMITTED_HEAP_BYTES.toString());
+      LOG.info("Simulated Job Maps Total Heap Usage: " + simuJobMapsTHU);
+
+      simuJobReducesTHU = 
+          getCounterValue(reduceCounters, 
+              Task.Counter.COMMITTED_HEAP_BYTES.toString());
+      LOG.info("Simulated Jobs Reduces Total Heap Usage: " + simuJobReducesTHU);
+
+      long mapCount = 
+          Integer.parseInt(jhInfo.getValues()
+                  .get(JobHistory.Keys.TOTAL_MAPS));
+      long reduceCount = 
+          Integer.parseInt(jhInfo.getValues()
+                  .get(JobHistory.Keys.TOTAL_REDUCES));
+
+      if (simuJobConf.get(GridMixConfig
+          .GRIDMIX_HEAP_FREE_MEMORY_RATIO) != null) {
+        strHeapRatio = "0.3F";
+      }
+
+      if (mapCount > 0) {
+        double mapEmulFactor = (simuJobMapsTHU * 100) / origJobMapsTHU;
+        long mapEmulAccuracy = Math.round(mapEmulFactor);
+        LOG.info("Maps memory emulation accuracy of a job:"
+                + mapEmulAccuracy + "%");
+        Assert.assertTrue("Map phase total memory emulation had crossed the "
+                         + "configured max limit.", mapEmulAccuracy
+                         <= GridMixConfig.GRIDMIX_MEMORY_EMULATION_UPPER_LIMIT);
+        Assert.assertTrue("Map phase total memory emulation had not crossed "
+                         + "the configured min limit.", mapEmulAccuracy
+                         >= GridMixConfig.GRIDMIX_MEMORY_EMULATION_LOWER_LIMIT);
+        double expHeapRatio = Double.parseDouble(strHeapRatio);
+        LOG.info("expHeapRatio for maps:" + expHeapRatio);
+        double actHeapRatio =
+                ((double)Math.abs(origJobMapsTHU - simuJobMapsTHU)) ;
+        actHeapRatio /= origJobMapsTHU;
+          LOG.info("actHeapRatio for maps:" + actHeapRatio);
+          Assert.assertTrue("Simulate job maps heap ratio not matched.",
+                            actHeapRatio <= expHeapRatio);
+      }
+
+      if (reduceCount >0) {
+        double reduceEmulFactor = (simuJobReducesTHU * 100) / origJobReducesTHU;
+        long reduceEmulAccuracy = Math.round(reduceEmulFactor);
+        LOG.info("Reduces memory emulation accuracy of a job:"
+                + reduceEmulAccuracy + "%");
+        Assert.assertTrue("Reduce phase total memory emulation had crossed "
+                         + "configured max limit.", reduceEmulAccuracy
+                         <= GridMixConfig.GRIDMIX_MEMORY_EMULATION_UPPER_LIMIT);
+        Assert.assertTrue("Reduce phase total memory emulation had not "
+                         + "crosssed configured min limit.", reduceEmulAccuracy
+                         >= GridMixConfig.GRIDMIX_MEMORY_EMULATION_LOWER_LIMIT);
+        double expHeapRatio = Double.parseDouble(strHeapRatio);
+        LOG.info("expHeapRatio for reduces:" + expHeapRatio);
+        double actHeapRatio =
+                ((double)Math.abs(origJobReducesTHU - simuJobReducesTHU));
+        actHeapRatio /= origJobReducesTHU;
+          LOG.info("actHeapRatio for reduces:" + actHeapRatio);
+          Assert.assertTrue("Simulate job reduces heap ratio not matched.",
+                            actHeapRatio <= expHeapRatio);
+      }
+    }
+  }
+
+  /**
+   *  Get the simulated job cpu metrics.
+   * @param jhInfo - Simulated job history
+   * @return - cpu metrics as a map.
+   * @throws Exception - if an error occurs.
+   */
+  private Map<String,Long> getSimulatedJobCPUMetrics(
+                           JobHistory.JobInfo jhInfo) throws Exception {
+    Map<String, Long> resourceMetrics = new HashMap<String, Long>();
+    Counters mapCounters = Counters.fromEscapedCompactString(
+        jhInfo.getValues().get(JobHistory.Keys.MAP_COUNTERS));
+    long mapCPUUsage =
+        getCounterValue(mapCounters,
+                        Task.Counter.CPU_MILLISECONDS.toString());
+    resourceMetrics.put("MAP", mapCPUUsage);
+
+    Counters reduceCounters = Counters.fromEscapedCompactString(
+        jhInfo.getValues().get(JobHistory.Keys.REDUCE_COUNTERS));
+    long reduceCPUUsage =
+        getCounterValue(reduceCounters,
+                        Task.Counter.CPU_MILLISECONDS.toString());
+    resourceMetrics.put("REDUCE", reduceCPUUsage);
+    return resourceMetrics;
+  }
+
+  /**
+   * Get the original job cpu metrics.
+   * @param zombieJob - original job history.
+   * @return - cpu metrics as map.
+   */
+  private Map<String, Long> getOriginalJobCPUMetrics(ZombieJob zombieJob) {
+    long mapTotalCPUUsage = 0;
+    long reduceTotalCPUUsage = 0;
+    Map<String,Long> resourceMetrics = new HashMap<String,Long>();
+
+    for (int index = 0; index < zombieJob.getNumberMaps(); index++) {
+      TaskInfo mapTask = zombieJob.getTaskInfo(TaskType.MAP, index);
+      if (mapTask.getResourceUsageMetrics().getCumulativeCpuUsage() > 0) {
+        mapTotalCPUUsage +=
+            mapTask.getResourceUsageMetrics().getCumulativeCpuUsage();
+      }
+    }
+    resourceMetrics.put("MAP", mapTotalCPUUsage);
+
+    for (int index = 0; index < zombieJob.getNumberReduces(); index++) {
+      TaskInfo reduceTask = zombieJob.getTaskInfo(TaskType.REDUCE, index);
+      if (reduceTask.getResourceUsageMetrics().getCumulativeCpuUsage() > 0) {
+        reduceTotalCPUUsage +=
+            reduceTask.getResourceUsageMetrics().getCumulativeCpuUsage();
+      }
+    }
+    resourceMetrics.put("REDUCE", reduceTotalCPUUsage);
+    return resourceMetrics;
+  }
+
+  /**
+   * Get the user resolver of a job.
+   */
+  public String getJobUserResolver() {
+    return userResolverVal;
+  }
+
+  /**
+   * It verifies the compression ratios of mapreduce jobs.
+   * @param origJobConf - original job configuration.
+   * @param simuJobConf - simulated job configuration.
+   * @param counters  - simulated job counters.
+   * @param origReduceCounters - original job reduce counters.
+   * @param origMapCounters - original job map counters.
+   * @throws ParseException - if a parser error occurs.
+   * @throws IOException - if an I/O error occurs.
+   */
+  public void verifyCompressionEmulation(JobConf origJobConf, 
+                                         JobConf simuJobConf,Counters counters, 
+                                         Map<String, Long> origReduceCounters, 
+                                         Map<String, Long> origMapJobCounters) 
+                                         throws ParseException,IOException { 
+    if (simuJobConf.getBoolean(compEmulKey, false)) {
+      String inputDir = origJobConf.get(fileInputFormatKey);
+      Assert.assertNotNull(fileInputFormatKey + " is Null",inputDir);
+      long simMapInputBytes = getCounterValue(counters, "HDFS_BYTES_READ");
+      long uncompressedInputSize = origMapJobCounters.get("MAP_INPUT_BYTES"); 
+      long simReduceInputBytes =
+            getCounterValue(counters, "REDUCE_SHUFFLE_BYTES");
+        long simMapOutputBytes = getCounterValue(counters, "MAP_OUTPUT_BYTES");
+
+      // Verify input compression whether it's enable or not.
+      if (inputDir.contains(".gz") || inputDir.contains(".tgz") 
+         || inputDir.contains(".bz")) { 
+        Assert.assertTrue("Input decompression attribute has been not set for " 
+                         + "for compressed input",
+                         simuJobConf.getBoolean(inputDecompKey, false));
+
+        float INPUT_COMP_RATIO = 
+            getExpectedCompressionRatio(simuJobConf, mapInputCompRatio);
+        float INTERMEDIATE_COMP_RATIO = 
+            getExpectedCompressionRatio(simuJobConf, mapOutputCompRatio);
+
+        // Verify Map Input Compression Ratio.
+        assertMapInputCompressionRatio(simMapInputBytes, uncompressedInputSize, 
+                                       INPUT_COMP_RATIO);
+
+        // Verify Map Output Compression Ratio.
+        assertMapOuputCompressionRatio(simReduceInputBytes, simMapOutputBytes, 
+                                       INTERMEDIATE_COMP_RATIO);
+      } else {
+        Assert.assertEquals("MAP input bytes has not matched.", 
+                            convertBytes(uncompressedInputSize), 
+                            convertBytes(simMapInputBytes));
+      }
+
+      Assert.assertEquals("Simulated job output format has not matched with " 
+                         + "original job output format.",
+                         origJobConf.getBoolean(fileOutputFormatKey,false), 
+                         simuJobConf.getBoolean(fileOutputFormatKey,false));
+
+      if (simuJobConf.getBoolean(fileOutputFormatKey,false)) { 
+        float OUTPUT_COMP_RATIO = 
+            getExpectedCompressionRatio(simuJobConf, reduceOutputCompRatio);
+
+         //Verify reduce output compression ratio.
+         long simReduceOutputBytes = 
+             getCounterValue(counters, "HDFS_BYTES_WRITTEN");
+         long origReduceOutputBytes = 
+             origReduceCounters.get("REDUCE_OUTPUT_BYTES");
+         assertReduceOutputCompressionRatio(simReduceOutputBytes, 
+                                            origReduceOutputBytes, 
+                                            OUTPUT_COMP_RATIO);
+      }
+    }
+  }
+
+  private void assertMapInputCompressionRatio(long simMapInputBytes, 
+                                   long origMapInputBytes, 
+                                   float expInputCompRatio) { 
+    LOG.info("***Verify the map input bytes compression ratio****");
+    LOG.info("Simulated job's map input bytes(REDUCE_SHUFFLE_BYTES): " 
+            + simMapInputBytes);
+    LOG.info("Original job's map input bytes: " + origMapInputBytes);
+
+    final float actInputCompRatio = 
+        getActualCompressionRatio(simMapInputBytes, origMapInputBytes);
+    LOG.info("Expected Map Input Compression Ratio:" + expInputCompRatio);
+    LOG.info("Actual Map Input Compression Ratio:" + actInputCompRatio);
+
+    float diffVal = (float)(expInputCompRatio * 0.06);
+    LOG.info("Expected Difference of Map Input Compression Ratio is <= " + 
+            + diffVal);
+    float delta = Math.abs(expInputCompRatio - actInputCompRatio);
+    LOG.info("Actual Difference of Map Iput Compression Ratio:" + delta);
+    Assert.assertTrue("Simulated job input compression ratio has mismatched.", 
+                      delta <= diffVal);
+    LOG.info("******Done******");
+  }
+
+  private void assertMapOuputCompressionRatio(long simReduceInputBytes, 
+                                              long simMapoutputBytes, 
+                                              float expMapOuputCompRatio) { 
+    LOG.info("***Verify the map output bytes compression ratio***");
+    LOG.info("Simulated job reduce input bytes:" + simReduceInputBytes);
+    LOG.info("Simulated job map output bytes:" + simMapoutputBytes);
+
+    final float actMapOutputCompRatio = 
+        getActualCompressionRatio(simReduceInputBytes, simMapoutputBytes);
+    LOG.info("Expected Map Output Compression Ratio:" + expMapOuputCompRatio);
+    LOG.info("Actual Map Output Compression Ratio:" + actMapOutputCompRatio);
+
+    float diffVal = 0.05f;
+    LOG.info("Expected Difference Of Map Output Compression Ratio is <= " 
+            + diffVal);
+    float delta = Math.abs(expMapOuputCompRatio - actMapOutputCompRatio);
+    LOG.info("Actual Difference Of Map Ouput Compression Ratio :" + delta);
+
+    Assert.assertTrue("Simulated job map output compression ratio " 
+                     + "has not been matched.", delta <= diffVal);
+    LOG.info("******Done******");
+  }
+
+  private void assertReduceOutputCompressionRatio(long simReduceOutputBytes, 
+      long origReduceOutputBytes , float expOutputCompRatio ) {
+      LOG.info("***Verify the reduce output bytes compression ratio***");
+      final float actOuputputCompRatio = 
+          getActualCompressionRatio(simReduceOutputBytes, origReduceOutputBytes);
+      LOG.info("Simulated job's reduce output bytes:" + simReduceOutputBytes);
+      LOG.info("Original job's reduce output bytes:" + origReduceOutputBytes);
+      LOG.info("Expected output compression ratio:" + expOutputCompRatio);
+      LOG.info("Actual output compression ratio:" + actOuputputCompRatio);
+      long diffVal = (long)(origReduceOutputBytes * 0.15);
+      long delta = Math.abs(origReduceOutputBytes - simReduceOutputBytes);
+      LOG.info("Expected difference of output compressed bytes is <= " 
+              + diffVal);
+      LOG.info("Actual difference of compressed ouput bytes:" + delta);
+      Assert.assertTrue("Simulated job reduce output compression ratio " +
+         "has not been matched.", delta <= diffVal);
+      LOG.info("******Done******");
+  }
+
+  private float getExpectedCompressionRatio(JobConf simuJobConf, 
+                                            String RATIO_TYPE) {
+    // Default decompression ratio is 0.50f irrespective of original 
+    //job compression ratio.
+    if (simuJobConf.get(RATIO_TYPE) != null) {
+      return Float.parseFloat(simuJobConf.get(RATIO_TYPE));
+    } else {
+      return 0.50f;
+    }
+  }
+
+  private float getActualCompressionRatio(long compressBytes, 
+                                          long uncompessBytes) {
+    double ratio = ((double)compressBytes) / uncompessBytes; 
+    int significant = (int)Math.round(ratio * 100);
+    return ((float)significant)/100; 
+  }
+
+  /**
+   * Verify the distributed cache files between the jobs in a gridmix run.
+   * @param jobsInfo - jobConfs of simulated and original jobs as a map.
+   */
+  public void verifyDistributedCacheBetweenJobs(
+      Map<String,List<JobConf>> jobsInfo) {
+     if (jobsInfo.size() > 1) {
+       Map<String, Integer> simJobfilesOccurBtnJobs = 
+           getDistcacheFilesOccurenceBetweenJobs(jobsInfo, 0);
+       Map<String, Integer> origJobfilesOccurBtnJobs = 
+           getDistcacheFilesOccurenceBetweenJobs(jobsInfo, 1);
+       List<Integer> simuOccurList = 
+           getMapValuesAsList(simJobfilesOccurBtnJobs);
+       Collections.sort(simuOccurList);
+       List<Integer> origOccurList = 
+           getMapValuesAsList(origJobfilesOccurBtnJobs);
+       Collections.sort(origOccurList);
+       Assert.assertEquals("The unique count of distibuted cache files in " 
+                        + "simulated jobs have not matched with the unique "
+                        + "count of original jobs distributed files ", 
+                        simuOccurList.size(), origOccurList.size());
+       int index = 0;
+       for (Integer origDistFileCount : origOccurList) {
+         Assert.assertEquals("Distributed cache file reused in simulated " 
+                            + "jobs has not matched with reused of distributed"
+                            + "cache file in original jobs.",
+                            origDistFileCount, simuOccurList.get(index));
+         index ++;
+       }
+     }
+  }
+
+  /**
+   * Get the unique distributed cache files and occurrence between the jobs.
+   * @param jobsInfo - job's configurations as a map.
+   * @param jobConfIndex - 0 for simulated job configuration and 
+   *                       1 for original jobs configuration.
+   * @return  - unique distributed cache files and occurrences as map.
+   */
+  private Map<String, Integer> getDistcacheFilesOccurenceBetweenJobs(
+      Map<String, List<JobConf>> jobsInfo, int jobConfIndex) {
+    Map<String,Integer> filesOccurBtnJobs = new HashMap <String,Integer>();
+    Set<String> jobIds = jobsInfo.keySet();
+    Iterator<String > ite = jobIds.iterator();
+    while (ite.hasNext()) {
+      String jobId = ite.next();
+      List<JobConf> jobconfs = jobsInfo.get(jobId);
+      String [] distCacheFiles = jobconfs.get(jobConfIndex).get(
+          GridMixConfig.GRIDMIX_DISTCACHE_FILES).split(",");
+      String [] distCacheFileTimeStamps = jobconfs.get(jobConfIndex).get(
+          GridMixConfig.GRIDMIX_DISTCACHE_TIMESTAMP).split(",");
+      String [] distCacheFileVisib = jobconfs.get(jobConfIndex).get(
+          GridMixConfig.GRIDMIX_DISTCACHE_VISIBILITIES).split(",");
+      int indx = 0;
+      for (String distCacheFile : distCacheFiles) {
+        String fileAndSize = distCacheFile + "^" 
+                           + distCacheFileTimeStamps[indx] + "^" 
+                           + jobconfs.get(jobConfIndex).getUser();
+        if (filesOccurBtnJobs.get(fileAndSize) != null) {
+          int count = filesOccurBtnJobs.get(fileAndSize);
+          count ++;
+          filesOccurBtnJobs.put(fileAndSize, count);
+        } else {
+          filesOccurBtnJobs.put(fileAndSize, 1);
+        }
+      }
+    }
+    return filesOccurBtnJobs;
+  }
+
+  /**
+   * It verifies the distributed cache emulation of  a job.
+   * @param zombieJob - Original job story.
+   * @param simuJobConf - Simulated job configuration.
+   */
+  public void verifyDistributeCache(ZombieJob zombieJob, 
+                                    JobConf simuJobConf) throws IOException {
+    if (simuJobConf.getBoolean(GridMixConfig.GRIDMIX_DISTCACHE_ENABLE, false)) {
+      JobConf origJobConf = zombieJob.getJobConf();
+      assertFileVisibility(simuJobConf);
+      assertDistcacheFiles(simuJobConf,origJobConf);
+      assertFileSizes(simuJobConf,origJobConf);
+      assertFileStamps(simuJobConf,origJobConf);
+    } else {
+      Assert.assertNull("Configuration has distributed cache visibilites" 
+          + "without enabled distributed cache emulation.", 
+          simuJobConf.get(GridMixConfig.GRIDMIX_DISTCACHE_VISIBILITIES));
+      Assert.assertNull("Configuration has distributed cache files time " 
+          + "stamps without enabled distributed cache emulation.", 
+          simuJobConf.get(GridMixConfig.GRIDMIX_DISTCACHE_TIMESTAMP));
+      Assert.assertNull("Configuration has distributed cache files paths" 
+          + "without enabled distributed cache emulation.", 
+          simuJobConf.get(GridMixConfig.GRIDMIX_DISTCACHE_FILES));
+      Assert.assertNull("Configuration has distributed cache files sizes" 
+          + "without enabled distributed cache emulation.", 
+          simuJobConf.get(GridMixConfig.GRIDMIX_DISTCACHE_FILESSIZE));
+    }
+  }
+
+  private void assertFileStamps(JobConf simuJobConf, JobConf origJobConf) {
+    //Verify simulated jobs against distributed cache files time stamps.
+    String [] origDCFTS = 
+        origJobConf.get(GridMixConfig.GRIDMIX_DISTCACHE_TIMESTAMP).split(",");
+    String [] simuDCFTS = 
+        simuJobConf.get(GridMixConfig.GRIDMIX_DISTCACHE_TIMESTAMP).split(",");
+    for (int index = 0; index < origDCFTS.length; index++) { 
+      Assert.assertTrue("Invalid time stamps between original "
+          +"and simulated job", Long.parseLong(origDCFTS[index]) 
+          < Long.parseLong(simuDCFTS[index]));
+    }
+  }
+
+  private void assertFileVisibility(JobConf simuJobConf ) {
+    // Verify simulated jobs against distributed cache files visibilities.
+    String [] distFiles = 
+        simuJobConf.get(GridMixConfig.GRIDMIX_DISTCACHE_FILES).split(",");
+    String [] simuDistVisibilities = 
+        simuJobConf.get(GridMixConfig.GRIDMIX_DISTCACHE_VISIBILITIES).split(",");
+    List<Boolean> expFileVisibility = new ArrayList<Boolean >();
+    int index = 0;
+    for (String distFile : distFiles) {
+      boolean isLocalDistCache = GridmixSystemTestCase.isLocalDistCache(
+                                 distFile, 
+                                 simuJobConf.getUser(), 
+                                 Boolean.valueOf(simuDistVisibilities[index]));
+      if (!isLocalDistCache) {
+        expFileVisibility.add(true);
+      } else {
+        expFileVisibility.add(false);
+      }
+      index ++;
+    }
+    index = 0;
+    for (String actFileVisibility :  simuDistVisibilities) {
+      Assert.assertEquals("Simulated job distributed cache file " 
+                         + "visibilities has not matched.", 
+                         expFileVisibility.get(index),
+                         Boolean.valueOf(actFileVisibility));
+      index ++;
+    }
+  }
+  
+  private void assertDistcacheFiles(JobConf simuJobConf, JobConf origJobConf) 
+      throws IOException {
+    //Verify simulated jobs against distributed cache files.
+    String [] origDistFiles = origJobConf.get(
+        GridMixConfig.GRIDMIX_DISTCACHE_FILES).split(",");
+    String [] simuDistFiles = simuJobConf.get(
+        GridMixConfig.GRIDMIX_DISTCACHE_FILES).split(",");
+    String [] simuDistVisibilities = simuJobConf.get(
+        GridMixConfig.GRIDMIX_DISTCACHE_VISIBILITIES).split(",");
+    Assert.assertEquals("No. of simulatued job's distcache files mismacted" 
+                       + "with no.of original job's distcache files", 
+                       origDistFiles.length, simuDistFiles.length);
+
+    int index = 0;
+    for (String simDistFile : simuDistFiles) {
+      Path distPath = new Path(simDistFile);
+      boolean isLocalDistCache = 
+          GridmixSystemTestCase.isLocalDistCache(simDistFile,
+              simuJobConf.getUser(),
+              Boolean.valueOf(simuDistVisibilities[index]));
+      if (!isLocalDistCache) {
+        FileSystem fs = distPath.getFileSystem(conf);
+        FileStatus fstat = fs.getFileStatus(distPath);
+        FsPermission permission = fstat.getPermission();
+        Assert.assertTrue("HDFS distributed cache file has wrong " 
+                         + "permissions for users.", 
+                         FsAction.READ_WRITE.SYMBOL 
+                         == permission.getUserAction().SYMBOL);
+        Assert.assertTrue("HDFS distributed cache file has wrong " 
+                         + "permissions for groups.", 
+                         FsAction.READ.SYMBOL 
+                         == permission.getGroupAction().SYMBOL);
+        Assert.assertTrue("HDSFS distributed cache file has wrong " 
+                         + "permissions for others.", 
+                         FsAction.READ.SYMBOL 
+                         == permission.getOtherAction().SYMBOL);
+      }
+      index++;
+    }
+  }
+
+  private void assertFileSizes(JobConf simuJobConf, JobConf origJobConf) { 
+    // Verify simulated jobs against distributed cache files size.
+    List<String> origDistFilesSize = 
+        Arrays.asList(origJobConf.get(
+            GridMixConfig.GRIDMIX_DISTCACHE_FILESSIZE).split(","));
+    Collections.sort(origDistFilesSize);
+
+    List<String> simuDistFilesSize = 
+        Arrays.asList(simuJobConf.get(
+            GridMixConfig.GRIDMIX_DISTCACHE_FILESSIZE).split(","));
+    Collections.sort(simuDistFilesSize);
+
+    Assert.assertEquals("Simulated job's file size list has not " 
+                       + "matched with the Original job's file size list.",
+                       origDistFilesSize.size(),
+                       simuDistFilesSize.size());
+
+    for (int index = 0; index < origDistFilesSize.size(); index ++) {
+       Assert.assertEquals("Simulated job distcache file size has not " 
+                          + "matched with original job distcache file size.", 
+                          origDistFilesSize.get(index), 
+                          simuDistFilesSize.get(index));
+    }
+  }
+
+  private void setJobDistributedCacheInfo(String jobId, JobConf simuJobConf, 
+     JobConf origJobConf) { 
+    if (simuJobConf.get(GridMixConfig.GRIDMIX_DISTCACHE_FILES) != null) {
+      List<JobConf> jobConfs = new ArrayList<JobConf>();
+      jobConfs.add(simuJobConf);
+      jobConfs.add(origJobConf);
+      simuAndOrigJobsInfo.put(jobId,jobConfs);
+    }
+  }
+
+  private List<Integer> getMapValuesAsList(Map<String,Integer> jobOccurs) { 
+    List<Integer> occursList = new ArrayList<Integer>();
+    Set<String> files = jobOccurs.keySet();
+    Iterator<String > ite = files.iterator();
+    while (ite.hasNext()) {
+      String file = ite.next(); 
+      occursList.add(jobOccurs.get(file));
+    }
+    return occursList;
+  }
+
+  /**
+   * It verifies the high ram gridmix jobs.
+   * @param zombieJob - Original job story.
+   * @param simuJobConf - Simulated job configuration.
+   */
+  @SuppressWarnings("deprecation")
+  public void verifyHighRamMemoryJobs(ZombieJob zombieJob,
+                                      JobConf simuJobConf) {
+    JobConf origJobConf = zombieJob.getJobConf();
+    int origMapFactor = getMapFactor(origJobConf);
+    int origReduceFactor = getReduceFactor(origJobConf);
+    boolean isHighRamEnable = 
+        simuJobConf.getBoolean(GridMixConfig.GRIDMIX_HIGH_RAM_JOB_ENABLE, 
+                               false);
+    if (isHighRamEnable) {
+        if (origMapFactor >= 2 && origReduceFactor >= 2) {
+          assertGridMixHighRamJob(simuJobConf, origJobConf, 1);
+        } else if(origMapFactor >= 2) {
+          assertGridMixHighRamJob(simuJobConf, origJobConf, 2);
+        } else if(origReduceFactor >= 2) {
+          assertGridMixHighRamJob(simuJobConf, origJobConf, 3);
+        }
+    } else {
+        if (origMapFactor >= 2 && origReduceFactor >= 2) {
+              assertGridMixHighRamJob(simuJobConf, origJobConf, 4);
+        } else if(origMapFactor >= 2) {
+              assertGridMixHighRamJob(simuJobConf, origJobConf, 5);
+        } else if(origReduceFactor >= 2) {
+              assertGridMixHighRamJob(simuJobConf, origJobConf, 6);
+        }
+    }
+  }
+
+  /**
+   * Get the value for identifying the slots used by the map.
+   * @param jobConf - job configuration
+   * @return - map factor value.
+   */
+  public static int getMapFactor(Configuration jobConf) {
+    long clusterMapMem = 
+        Long.parseLong(jobConf.get(GridMixConfig.CLUSTER_MAP_MEMORY));
+    long jobMapMem = 
+        Long.parseLong(jobConf.get(GridMixConfig.JOB_MAP_MEMORY_MB));
+    return (int)Math.ceil((double)jobMapMem / clusterMapMem);  
+  }
+
+  /**
+   * Get the value for identifying the slots used by the reduce.
+   * @param jobConf - job configuration.
+   * @return - reduce factor value.
+   */
+  public static int getReduceFactor(Configuration jobConf) {
+    long clusterReduceMem = 
+        Long.parseLong(jobConf.get(GridMixConfig.CLUSTER_REDUCE_MEMORY));
+    long jobReduceMem = 
+        Long.parseLong(jobConf.get(GridMixConfig.JOB_REDUCE_MEMORY_MB));
+    return (int)Math.ceil((double)jobReduceMem / clusterReduceMem);
+  }
+
+  @SuppressWarnings("deprecation")
+  private void assertGridMixHighRamJob(JobConf simuJobConf, 
+                                       Configuration origConf, int option) {
+    int simuMapFactor = getMapFactor(simuJobConf);
+    int simuReduceFactor = getReduceFactor(simuJobConf);
+    /**
+     *  option 1 : Both map and reduce honors the high ram.
+     *  option 2 : Map only honors the high ram.
+     *  option 3 : Reduce only honors the high ram.
+     *  option 4 : Both map and reduce should not honors the high ram
+     *             in disable state.
+     *  option 5 : Map should not honors the high ram in disable state.
+     *  option 6 : Reduce should not honors the high ram in disable state.
+     */
+    switch (option) {
+      case 1 :
+               Assert.assertTrue("Gridmix job has not honored the high "
+                                + "ram for map.", simuMapFactor >= 2 
+                                && simuMapFactor == getMapFactor(origConf));
+               Assert.assertTrue("Gridmix job has not honored the high "
+                                + "ram for reduce.", simuReduceFactor >= 2 
+                                && simuReduceFactor 
+                                == getReduceFactor(origConf));
+               break;
+      case 2 :
+               Assert.assertTrue("Gridmix job has not honored the high "
+                                + "ram for map.", simuMapFactor >= 2 
+                                && simuMapFactor == getMapFactor(origConf));
+               break;
+      case 3 :
+               Assert.assertTrue("Girdmix job has not honored the high "
+                                + "ram for reduce.", simuReduceFactor >= 2 
+                                && simuReduceFactor 
+                                == getReduceFactor(origConf));
+               break;
+      case 4 :
+               Assert.assertTrue("Gridmix job has honored the high "
+                                + "ram for map in emulation disable state.", 
+                                simuMapFactor < 2 
+                                && simuMapFactor != getMapFactor(origConf));
+               Assert.assertTrue("Gridmix job has honored the high "
+                                + "ram for reduce in emulation disable state.", 
+                                simuReduceFactor < 2 
+                                && simuReduceFactor 
+                                != getReduceFactor(origConf));
+               break;
+      case 5 :
+               Assert.assertTrue("Gridmix job has honored the high "
+                                + "ram for map in emulation disable state.", 
+                                simuMapFactor < 2 
+                                && simuMapFactor != getMapFactor(origConf));
+               break;
+      case 6 :
+               Assert.assertTrue("Girdmix job has honored the high "
+                                + "ram for reduce in emulation disable state.", 
+                                simuReduceFactor < 2 
+                                && simuReduceFactor 
+                                != getReduceFactor(origConf));
+               break;
+    }
+  }
+
+  /**
+   * Get task memory after scaling based on cluster configuration.
+   * @param jobTaskKey - Job task key attribute.
+   * @param clusterTaskKey - Cluster task key attribute.
+   * @param origConf - Original job configuration.
+   * @param simuConf - Simulated job configuration.
+   * @return scaled task memory value.
+   */
+  @SuppressWarnings("deprecation")
+  public static long getScaledTaskMemInMB(String jobTaskKey, 
+                                          String clusterTaskKey, 
+                                          Configuration origConf, 
+                                          Configuration simuConf) { 
+    long simuClusterTaskValue = 
+        simuConf.getLong(clusterTaskKey, JobConf.DISABLED_MEMORY_LIMIT);
+    long origClusterTaskValue = 
+        origConf.getLong(clusterTaskKey, JobConf.DISABLED_MEMORY_LIMIT);
+    long origJobTaskValue = 
+        origConf.getLong(jobTaskKey, JobConf.DISABLED_MEMORY_LIMIT);
+    double scaleFactor = 
+        Math.ceil((double)origJobTaskValue / origClusterTaskValue);
+    long simulatedJobValue = (long)(scaleFactor * simuClusterTaskValue);
+    return simulatedJobValue;
+  }
+
+  /**
+   * It Verifies the memory limit of a task.
+   * @param TaskMemInMB - task memory limit.
+   * @param taskLimitInMB - task upper limit.
+   */
+  public static void verifyMemoryLimits(long TaskMemInMB, long taskLimitInMB) {
+    if (TaskMemInMB > taskLimitInMB) {
+      Assert.fail("Simulated job's task memory exceeds the " 
+                 + "upper limit of task virtual memory.");
+    }
+  }
+
+  private String convertJobStatus(String jobStatus) {
+    if (jobStatus.equals("SUCCEEDED")) { 
+      return "SUCCESS";
+    } else {
+      return jobStatus;
+    }
+  }
+  
+  private String convertBytes(long bytesValue) {
+    int units = 1024;
+    if( bytesValue < units ) {
+      return String.valueOf(bytesValue)+ "B";
+    } else {
+      // it converts the bytes into either KB or MB or GB or TB etc.
+      int exp = (int)(Math.log(bytesValue) / Math.log(units));
+      return String.format("%1d%sB",(long)(bytesValue / Math.pow(units, exp)), 
+          "KMGTPE".charAt(exp -1));
+    }
+  }
+ 
+
+  private long getCounterValue(Counters counters, String key) 
+     throws ParseException { 
+    for (String groupName : counters.getGroupNames()) {
+       Group totalGroup = counters.getGroup(groupName);
+       Iterator<Counter> itrCounter = totalGroup.iterator();
+       while (itrCounter.hasNext()) {
+         Counter counter = itrCounter.next();
+         if (counter.getName().equals(key)) {
+           return counter.getValue();
+         }
+       }
+    }
+    return 0;
+  }
+}
+

+ 524 - 0
src/contrib/gridmix/src/test/system/org/apache/hadoop/mapred/gridmix/test/system/UtilsForGridmix.java

@@ -0,0 +1,524 @@
+/**
+ * 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.mapred.gridmix.test.system;
+
+import java.io.IOException;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.hadoop.mapred.gridmix.Gridmix;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapred.JobClient;
+import org.apache.hadoop.mapred.JobStatus;
+import org.apache.hadoop.mapred.RunningJob;
+import org.apache.hadoop.mapreduce.JobID;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Arrays;
+import java.net.URI;
+import java.text.SimpleDateFormat;
+import java.io.OutputStream;
+import java.util.Set;
+import java.util.List;
+import java.util.Iterator;
+import java.util.Map;
+import java.io.File;
+import java.io.FileOutputStream;
+import org.apache.hadoop.test.system.ProxyUserDefinitions;
+import org.apache.hadoop.test.system.ProxyUserDefinitions.GroupsAndHost;
+
+/**
+ * Gridmix utilities.
+ */
+public class UtilsForGridmix {
+  private static final Log LOG = LogFactory.getLog(UtilsForGridmix.class);
+  private static final Path DEFAULT_TRACES_PATH =
+    new Path(System.getProperty("user.dir") + "/src/test/system/resources/");
+
+  /**
+   * cleanup the folder or file.
+   * @param path - folder or file path.
+   * @param conf - cluster configuration 
+   * @throws IOException - If an I/O error occurs.
+   */
+  public static void cleanup(Path path, Configuration conf) 
+     throws IOException {
+    FileSystem fs = path.getFileSystem(conf);
+    fs.delete(path, true);
+    fs.close();
+  }
+
+  /**
+   * Get the login user.
+   * @return - login user as string..
+   * @throws IOException - if an I/O error occurs.
+   */
+  public static String getUserName() throws IOException {
+    return UserGroupInformation.getLoginUser().getUserName();
+  }
+  
+  /**
+   * Get the argument list for gridmix job.
+   * @param gridmixDir - gridmix parent directory.
+   * @param gridmixRunMode - gridmix modes either 1,2,3.
+   * @param values - gridmix runtime values.
+   * @param otherArgs - gridmix other generic args.
+   * @return - argument list as string array.
+   */
+  public static String [] getArgsList(Path gridmixDir, int gridmixRunMode, 
+                                      String [] values, String [] otherArgs) {
+    String [] runtimeArgs = { 
+        "-D", GridMixConfig.GRIDMIX_LOG_MODE + "=DEBUG", 
+        "-D", GridMixConfig.GRIDMIX_OUTPUT_DIR + "=gridmix", 
+        "-D", GridMixConfig.GRIDMIX_JOB_SUBMISSION_QUEUE_IN_TRACE + "=true", 
+        "-D", GridMixConfig.GRIDMIX_JOB_TYPE + "=" + values[0], 
+        "-D", GridMixConfig.GRIDMIX_USER_RESOLVER + "=" + values[1], 
+        "-D", GridMixConfig.GRIDMIX_SUBMISSION_POLICY + "=" + values[2]
+    };
+
+    String [] classArgs;
+    if ((gridmixRunMode == GridMixRunMode.DATA_GENERATION.getValue() 
+       || gridmixRunMode 
+       == GridMixRunMode.DATA_GENERATION_AND_RUN_GRIDMIX.getValue()) 
+       && values[1].indexOf("RoundRobinUserResolver") > 0) { 
+      classArgs = new String[] { 
+          "-generate", values[3], 
+          "-users", values[4], 
+          gridmixDir.toString(), 
+          values[5]
+      };
+    } else if (gridmixRunMode == GridMixRunMode.DATA_GENERATION.getValue() 
+              || gridmixRunMode 
+              == GridMixRunMode.DATA_GENERATION_AND_RUN_GRIDMIX.getValue()) { 
+      classArgs = new String[] { 
+          "-generate", values[3], 
+          gridmixDir.toString(), 
+          values[4]
+      };
+    } else if (gridmixRunMode == GridMixRunMode.RUN_GRIDMIX.getValue() 
+              && values[1].indexOf("RoundRobinUserResolver") > 0) { 
+      classArgs = new String[] { 
+          "-users", values[3], 
+          gridmixDir.toString(), 
+          values[4]
+      };
+    } else { 
+      classArgs = new String[] { 
+         gridmixDir.toString(),values[3]
+      };
+    }
+
+    String [] args = new String [runtimeArgs.length + 
+       classArgs.length + ((otherArgs != null)?otherArgs.length:0)];
+    System.arraycopy(runtimeArgs, 0, args, 0, runtimeArgs.length);
+
+    if (otherArgs != null) {
+      System.arraycopy(otherArgs, 0, args, runtimeArgs.length, 
+                       otherArgs.length);
+      System.arraycopy(classArgs, 0, args, (runtimeArgs.length + 
+                       otherArgs.length), classArgs.length);
+    } else {
+      System.arraycopy(classArgs, 0, args, runtimeArgs.length, 
+                       classArgs.length);
+    }
+    return args;
+  }
+  
+  /**
+   * Create a file with specified size in mb.
+   * @param sizeInMB - file size in mb.
+   * @param inputDir - input directory.
+   * @param conf - cluster configuration.
+   * @throws Exception - if an exception occurs.
+   */
+  public static void createFile(int sizeInMB, Path inputDir, 
+      Configuration conf) throws Exception {
+    Date d = new Date();
+    SimpleDateFormat sdf = new SimpleDateFormat("ddMMyy_HHmmssS");
+    String formatDate = sdf.format(d);
+    FileSystem fs = inputDir.getFileSystem(conf);
+    OutputStream out = fs.create(new Path(inputDir,"datafile_" + formatDate));
+    final byte[] b = new byte[1024 * 1024];
+    for (int index = 0; index < sizeInMB; index++) { 
+      out.write(b);
+    }    
+    out.close();
+    fs.close();
+  }
+  
+  /**
+   * Create directories for a path.
+   * @param path - directories path.
+   * @param conf  - cluster configuration.
+   * @throws IOException  - if an I/O error occurs.
+   */
+  public static void createDirs(Path path,Configuration conf) 
+     throws IOException { 
+    FileSystem fs = path.getFileSystem(conf);
+    if (!fs.exists(path)) { 
+       fs.mkdirs(path);
+       fs.setPermission(path,new FsPermission(FsAction.ALL,
+           FsAction.ALL,FsAction.ALL));
+    }
+  }
+  
+  /**
+   * Run the Gridmix job with given runtime arguments.
+   * @param gridmixDir - Gridmix parent directory.
+   * @param conf - cluster configuration.
+   * @param gridmixRunMode - gridmix run mode either 1,2,3
+   * @param runtimeValues -gridmix runtime values.
+   * @return - gridmix status either 0 or 1.
+   * @throws Exception
+   */
+  public static int runGridmixJob(Path gridmixDir, Configuration conf, 
+     int gridmixRunMode, String [] runtimeValues) throws Exception {
+    return runGridmixJob(gridmixDir, conf, gridmixRunMode, runtimeValues, null);
+  }
+  /**
+   * Run the Gridmix job with given runtime arguments.
+   * @param gridmixDir - Gridmix parent directory
+   * @param conf - cluster configuration.
+   * @param gridmixRunMode - gridmix run mode.
+   * @param runtimeValues - gridmix runtime values.
+   * @param otherArgs - gridmix other generic args.
+   * @return - gridmix status either 0 or 1.
+   * @throws Exception
+   */
+  
+  public static int runGridmixJob(Path gridmixDir, Configuration conf, 
+                                  int gridmixRunMode, String [] runtimeValues, 
+                                  String [] otherArgs) throws Exception {
+    Path  outputDir = new Path(gridmixDir, "gridmix");
+    Path inputDir = new Path(gridmixDir, "input");
+    LOG.info("Cleanup the data if data already exists.");
+    String modeName = new String();
+    switch (gridmixRunMode) { 
+      case 1 : 
+        cleanup(inputDir, conf);
+        cleanup(outputDir, conf);
+        modeName = GridMixRunMode.DATA_GENERATION.name();
+        break;
+      case 2 : 
+        cleanup(outputDir, conf);
+        modeName = GridMixRunMode.RUN_GRIDMIX.name();
+        break;
+      case 3 : 
+        cleanup(inputDir, conf);
+        cleanup(outputDir, conf);
+        modeName = GridMixRunMode.DATA_GENERATION_AND_RUN_GRIDMIX.name();
+        break;
+    }
+
+    final String [] args = 
+        UtilsForGridmix.getArgsList(gridmixDir, gridmixRunMode, 
+                                    runtimeValues, otherArgs);
+    Gridmix gridmix = new Gridmix();
+    LOG.info("Submit a Gridmix job in " + runtimeValues[1] 
+            + " mode for " + modeName);
+    int exitCode = ToolRunner.run(conf, gridmix, args);
+    return exitCode;
+  }
+
+  /**
+   * Get the proxy users file.
+   * @param conf - cluster configuration.
+   * @return String - proxy users file.
+   * @Exception - if no proxy users found in configuration.
+   */
+  public static String getProxyUsersFile(Configuration conf) 
+      throws Exception {
+     ProxyUserDefinitions pud = getProxyUsersData(conf);
+     String fileName = buildProxyUsersFile(pud.getProxyUsers());
+     if (fileName == null) { 
+        LOG.error("Proxy users file not found.");
+        throw new Exception("Proxy users file not found.");
+     } else { 
+        return fileName;
+     }
+  }
+  
+  /**
+  * List the current gridmix jobid's.
+  * @param client - job client.
+  * @param execJobCount - number of executed jobs.
+  * @return - list of gridmix jobid's.
+  */
+ public static List<JobID> listGridmixJobIDs(JobClient client, 
+     int execJobCount) throws Exception { 
+   List<JobID> jobids = new ArrayList<JobID>();
+   JobStatus [] jobStatus = client.getAllJobs();
+   int numJobs = jobStatus.length;
+   for (int index = 0; index < 31; index++) {
+     Thread.sleep(1000);
+     jobStatus = client.getAllJobs();
+     numJobs = jobStatus.length;
+   }
+   for (int index = 1; index <= execJobCount; index++) {
+     JobStatus js = jobStatus[numJobs - index];
+     JobID jobid = js.getJobID();
+     RunningJob runJob = client.getJob(jobid.toString());
+     String jobName = runJob.getJobName();
+     if (!jobName.equals("GRIDMIX_GENERATE_INPUT_DATA") && 
+         !jobName.equals("GRIDMIX_GENERATE_DISTCACHE_DATA")) {
+       jobids.add(jobid);
+     }
+   }
+   return (jobids.size() == 0)? null : jobids;
+ }
+
+ /**
+  * List the proxy users. 
+  * @param conf
+  * @return
+  * @throws Exception
+  */
+ public static List<String> listProxyUsers(Configuration conf,
+     String loginUser) throws Exception {
+   List<String> proxyUsers = new ArrayList<String>();
+   ProxyUserDefinitions pud = getProxyUsersData(conf);
+   Map<String, GroupsAndHost> usersData = pud.getProxyUsers();
+   Collection users = usersData.keySet();
+   Iterator<String> itr = users.iterator();
+   while (itr.hasNext()) { 
+     String user = itr.next();
+     if (!user.equals(loginUser)){ proxyUsers.add(user); };
+   }
+   return proxyUsers;
+ }
+
+  private static String buildProxyUsersFile(final Map<String, GroupsAndHost> 
+      proxyUserData) throws Exception { 
+     FileOutputStream fos = null;
+     File file = null;
+     StringBuffer input = new StringBuffer();
+     Set users = proxyUserData.keySet();
+     Iterator itr = users.iterator();
+     while (itr.hasNext()) { 
+       String user = itr.next().toString();
+       if (!user.equals(
+           UserGroupInformation.getLoginUser().getShortUserName())) {
+         input.append(user);
+         final GroupsAndHost gah = proxyUserData.get(user);
+         final List <String> groups = gah.getGroups();
+         for (String group : groups) { 
+           input.append(",");
+           input.append(group);
+         }
+         input.append("\n");
+       }
+     }
+     if (input.length() > 0) { 
+        try {
+           file = File.createTempFile("proxyusers", null);
+           fos = new FileOutputStream(file);
+           fos.write(input.toString().getBytes());
+        } catch(IOException ioexp) { 
+           LOG.warn(ioexp.getMessage());
+           return null;
+        } finally {
+           fos.close();
+           file.deleteOnExit();
+        }
+        LOG.info("file.toString():" + file.toString());
+        return file.toString();
+     } else {
+        return null;
+     }
+  }
+
+  private static ProxyUserDefinitions getProxyUsersData(Configuration conf)
+      throws Exception { 
+    Iterator itr = conf.iterator();
+    List<String> proxyUsersData = new ArrayList<String>();
+    while (itr.hasNext()) { 
+      String property = itr.next().toString();
+      if (property.indexOf("hadoop.proxyuser") >= 0 
+         && property.indexOf("groups=") >= 0) { 
+        proxyUsersData.add(property.split("\\.")[2]);
+      }
+    }
+
+    if (proxyUsersData.size() == 0) { 
+       LOG.error("No proxy users found in the configuration.");
+       throw new Exception("No proxy users found in the configuration.");
+    }
+
+    ProxyUserDefinitions pud = new ProxyUserDefinitions() { 
+       public boolean writeToFile(URI filePath) throws IOException { 
+           throw new UnsupportedOperationException("No such methood exists.");
+       };
+    };
+
+     for (String userName : proxyUsersData) { 
+        List<String> groups = Arrays.asList(conf.get("hadoop.proxyuser." + 
+            userName + ".groups").split("//,"));
+        List<String> hosts = Arrays.asList(conf.get("hadoop.proxyuser." + 
+            userName + ".hosts").split("//,"));
+        ProxyUserDefinitions.GroupsAndHost definitions = 
+            pud.new GroupsAndHost();
+        definitions.setGroups(groups);
+        definitions.setHosts(hosts);
+        pud.addProxyUser(userName, definitions);
+     }
+     return pud;
+  }
+
+  /**
+   *  Gives the list of paths for MR traces against different time 
+   *  intervals.It fetches only the paths which followed the below 
+   *  file convention.
+   *    Syntax : &lt;FileName&gt;_&lt;TimeIntervals&gt;.json.gz
+   *  There is a restriction in a  file and user has to  
+   *  follow  the below convention for time interval.
+   *    Syntax: &lt;numeric&gt;[m|h|d] 
+   *    e.g : for 10 minutes trace should specify 10m, 
+   *    same way for 1 hour traces should specify 1h, 
+   *    for 1 day traces should specify 1d.
+   *
+   * @param conf - cluster configuration.
+   * @return - list of MR paths as key/value pair based on time interval.
+   * @throws IOException - if an I/O error occurs.
+   */
+  public static Map<String, String> getMRTraces(Configuration conf) 
+     throws IOException { 
+    return getMRTraces(conf, DEFAULT_TRACES_PATH);
+  }
+  
+  /**
+   *  It gives the list of paths for MR traces against different time 
+   *  intervals. It fetches only the paths which followed the below 
+   *  file convention.
+   *    Syntax : &lt;FileNames&gt;_&lt;TimeInterval&gt;.json.gz
+   *  There is a restriction in a file and user has to follow the 
+   *  below convention for time interval. 
+   *    Syntax: &lt;numeric&gt;[m|h|d] 
+   *    e.g : for 10 minutes trace should specify 10m,
+   *    same way for 1 hour traces should specify 1h, 
+   *    for 1 day  traces should specify 1d.
+   *
+   * @param conf - cluster configuration object.
+   * @param tracesPath - MR traces path.
+   * @return - list of MR paths as key/value pair based on time interval.
+   * @throws IOException - If an I/O error occurs.
+   */
+  public static Map<String,String> getMRTraces(Configuration conf, 
+      Path tracesPath) throws IOException { 
+     Map <String, String> jobTraces = new HashMap <String, String>();
+     final FileSystem fs = FileSystem.getLocal(conf);
+     final FileStatus fstat[] = fs.listStatus(tracesPath);
+     for (FileStatus fst : fstat) { 
+        final String fileName = fst.getPath().getName();
+        if (fileName.endsWith("m.json.gz") 
+            || fileName.endsWith("h.json.gz") 
+            || fileName.endsWith("d.json.gz")) { 
+           jobTraces.put(fileName.substring(fileName.indexOf("_") + 1, 
+              fileName.indexOf(".json.gz")), fst.getPath().toString());
+        }
+     }
+     if (jobTraces.size() == 0) { 
+        LOG.error("No traces found in " + tracesPath.toString() + " path.");
+        throw new IOException("No traces found in " 
+                             + tracesPath.toString() + " path.");
+     }
+     return jobTraces;
+  }
+  
+  /**
+   * It list the all the MR traces path irrespective of time.
+   * @param conf - cluster configuration.
+   * @param tracesPath - MR traces path
+   * @return - MR paths as a list.
+   * @throws IOException - if an I/O error occurs.
+   */
+  public static List<String> listMRTraces(Configuration conf, 
+      Path tracesPath) throws IOException {
+     List<String> jobTraces = new ArrayList<String>();
+     final FileSystem fs = FileSystem.getLocal(conf);
+     final FileStatus fstat[] = fs.listStatus(tracesPath);
+     for (FileStatus fst : fstat) {
+        jobTraces.add(fst.getPath().toString());
+     }
+     if (jobTraces.size() == 0) {
+        LOG.error("No traces found in " + tracesPath.toString() + " path.");
+        throw new IOException("No traces found in " 
+                             + tracesPath.toString() + " path.");
+     }
+     return jobTraces;
+  }
+  
+  /**
+   * It list the all the MR traces path irrespective of time.
+   * @param conf - cluster configuration.
+   * @param tracesPath - MR traces path
+   * @return - MR paths as a list.
+   * @throws IOException - if an I/O error occurs.
+   */
+  public static List<String> listMRTraces(Configuration conf) 
+      throws IOException { 
+     return listMRTraces(conf, DEFAULT_TRACES_PATH);
+  }
+
+  /**
+   * Gives the list of MR traces for given time interval.
+   * The time interval should be following convention.
+   *   Syntax : &lt;numeric&gt;[m|h|d]
+   *   e.g : 10m or 1h or 2d etc.
+   * @param conf - cluster configuration
+   * @param timeInterval - trace time interval.
+   * @param tracesPath - MR traces Path.
+   * @return - MR paths as a list for a given time interval.
+   * @throws IOException - If an I/O error occurs.
+   */
+  public static List<String> listMRTracesByTime(Configuration conf, 
+      String timeInterval, Path tracesPath) throws IOException { 
+     List<String> jobTraces = new ArrayList<String>();
+     final FileSystem fs = FileSystem.getLocal(conf);
+     final FileStatus fstat[] = fs.listStatus(tracesPath);
+     for (FileStatus fst : fstat) { 
+        final String fileName = fst.getPath().getName();
+        if (fileName.indexOf(timeInterval) >= 0) { 
+           jobTraces.add(fst.getPath().toString());
+        }
+     }
+     return jobTraces;
+  }
+  
+  /**
+   * Gives the list of MR traces for given time interval.
+   * The time interval should be following convention.
+   *   Syntax : &lt;numeric&gt;[m|h|d]
+   *   e.g : 10m or 1h or 2d etc.
+   * @param conf - cluster configuration
+   * @param timeInterval - trace time interval.
+   * @return - MR paths as a list for a given time interval.
+   * @throws IOException - If an I/O error occurs.
+   */
+  public static List<String> listMRTracesByTime(Configuration conf, 
+      String timeInterval) throws IOException { 
+     return listMRTracesByTime(conf, timeInterval, DEFAULT_TRACES_PATH);
+  }
+}

BIN
src/contrib/gridmix/src/test/system/resources/2m_stream_trace.json.gz


BIN
src/contrib/gridmix/src/test/system/resources/3m_stream_trace.json.gz


BIN
src/contrib/gridmix/src/test/system/resources/5m_stream_trace.json.gz


BIN
src/contrib/gridmix/src/test/system/resources/compression_case1_trace.json.gz


BIN
src/contrib/gridmix/src/test/system/resources/compression_case2_trace.json.gz


BIN
src/contrib/gridmix/src/test/system/resources/compression_case3_trace.json.gz


BIN
src/contrib/gridmix/src/test/system/resources/compression_case4_trace.json.gz


BIN
src/contrib/gridmix/src/test/system/resources/cpu_emul_case1.json.gz


BIN
src/contrib/gridmix/src/test/system/resources/cpu_emul_case2.json.gz


BIN
src/contrib/gridmix/src/test/system/resources/distcache_case1_trace.json.gz


BIN
src/contrib/gridmix/src/test/system/resources/distcache_case2_trace.json.gz


BIN
src/contrib/gridmix/src/test/system/resources/distcache_case3_trace.json.gz


BIN
src/contrib/gridmix/src/test/system/resources/distcache_case4_trace.json.gz


BIN
src/contrib/gridmix/src/test/system/resources/distcache_case5_trace.json.gz


BIN
src/contrib/gridmix/src/test/system/resources/distcache_case6_trace.json.gz


BIN
src/contrib/gridmix/src/test/system/resources/distcache_case7_trace.json.gz


BIN
src/contrib/gridmix/src/test/system/resources/distcache_case8_trace.json.gz


BIN
src/contrib/gridmix/src/test/system/resources/distcache_case9_trace.json.gz


BIN
src/contrib/gridmix/src/test/system/resources/highram_mr_jobs_case1.json.gz


BIN
src/contrib/gridmix/src/test/system/resources/highram_mr_jobs_case2.json.gz


BIN
src/contrib/gridmix/src/test/system/resources/highram_mr_jobs_case3.json.gz


BIN
src/contrib/gridmix/src/test/system/resources/highram_mr_jobs_case4.json.gz


BIN
src/contrib/gridmix/src/test/system/resources/mem_emul_case1.json.gz


BIN
src/contrib/gridmix/src/test/system/resources/mem_emul_case2.json.gz


BIN
src/contrib/gridmix/src/test/system/resources/trace_10m.json.gz


BIN
src/contrib/gridmix/src/test/system/resources/trace_12m.json.gz


BIN
src/contrib/gridmix/src/test/system/resources/trace_1m.json.gz


BIN
src/contrib/gridmix/src/test/system/resources/trace_3m.json.gz


BIN
src/contrib/gridmix/src/test/system/resources/trace_5m.json.gz


BIN
src/contrib/gridmix/src/test/system/resources/trace_7m.json.gz


+ 1 - 1
src/test/aop/build/aop.xml

@@ -19,7 +19,7 @@
   <property name="build-fi.dir" value="${basedir}/build-fi"/>
   <property name="hadoop-fi.jar" location="${build.dir}/${final.name}-fi.jar" />
   <property name="compile-inject.output" value="${build-fi.dir}/compile-fi.log"/>
-  <property name="aspectversion" value="1.6.5"/>
+  <property name="aspectversion" value="1.6.11"/>
   <property file="${basedir}/build.properties"/>
 
   <!-- Properties related to system fault injection and tests -->