浏览代码

MAPREDUCE-3118. Backport Gridmix and Rumen features to branch-0.20-security (Ravi Gummadi via amarrk)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-0.20-security@1185694 13f79535-47bb-0310-9956-ffa450edef68
Amar Kamat 14 年之前
父节点
当前提交
e755f16fe4
共有 85 个文件被更改,包括 10155 次插入505 次删除
  1. 2 0
      CHANGES.txt
  2. 3 0
      build.xml
  3. 2 0
      src/contrib/build-contrib.xml
  4. 34 4
      src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/AvgRecordFactory.java
  5. 116 0
      src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/ClusterSummarizer.java
  6. 573 0
      src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/CompressionEmulationUtil.java
  7. 543 0
      src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/DistributedCacheEmulator.java
  8. 10 6
      src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/EchoUserResolver.java
  9. 307 0
      src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/ExecutionSummarizer.java
  10. 4 6
      src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/FileQueue.java
  11. 111 3
      src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/GenerateData.java
  12. 259 0
      src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/GenerateDistCacheData.java
  13. 267 41
      src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/Gridmix.java
  14. 245 15
      src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/GridmixJob.java
  15. 44 1
      src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/GridmixKey.java
  16. 57 1
      src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/GridmixRecord.java
  17. 13 1
      src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/InputStriper.java
  18. 53 7
      src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/JobCreator.java
  19. 28 7
      src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/JobFactory.java
  20. 2 2
      src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/JobMonitor.java
  21. 1 1
      src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/JobSubmitter.java
  22. 213 5
      src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/LoadJob.java
  23. 32 1
      src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/LoadSplit.java
  24. 25 0
      src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/Progressive.java
  25. 337 0
      src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/PseudoLocalFs.java
  26. 147 0
      src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/RandomTextDataGenerator.java
  27. 51 31
      src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/RoundRobinUserResolver.java
  28. 5 0
      src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/SleepJob.java
  29. 4 3
      src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/Statistics.java
  30. 11 2
      src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/SubmitterUserResolver.java
  31. 75 0
      src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/Summarizer.java
  32. 18 11
      src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/UserResolver.java
  33. 315 0
      src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/emulators/resourceusage/CumulativeCpuUsageEmulatorPlugin.java
  34. 63 0
      src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/emulators/resourceusage/ResourceUsageEmulatorPlugin.java
  35. 80 0
      src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/emulators/resourceusage/ResourceUsageMatcher.java
  36. 258 0
      src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/emulators/resourceusage/TotalHeapUsageEmulatorPlugin.java
  37. 二进制
      src/contrib/gridmix/src/test/data/wordcount.json.gz
  38. 10 5
      src/contrib/gridmix/src/test/org/apache/hadoop/mapred/gridmix/DebugJobProducer.java
  39. 563 0
      src/contrib/gridmix/src/test/org/apache/hadoop/mapred/gridmix/TestCompressionEmulationUtils.java
  40. 498 0
      src/contrib/gridmix/src/test/org/apache/hadoop/mapred/gridmix/TestDistCacheEmulation.java
  41. 453 0
      src/contrib/gridmix/src/test/org/apache/hadoop/mapred/gridmix/TestGridmixMemoryEmulation.java
  42. 3 2
      src/contrib/gridmix/src/test/org/apache/hadoop/mapred/gridmix/TestGridmixRecord.java
  43. 195 54
      src/contrib/gridmix/src/test/org/apache/hadoop/mapred/gridmix/TestGridmixSubmission.java
  44. 371 0
      src/contrib/gridmix/src/test/org/apache/hadoop/mapred/gridmix/TestGridmixSummary.java
  45. 202 0
      src/contrib/gridmix/src/test/org/apache/hadoop/mapred/gridmix/TestHighRamJob.java
  46. 233 0
      src/contrib/gridmix/src/test/org/apache/hadoop/mapred/gridmix/TestPseudoLocalFs.java
  47. 84 0
      src/contrib/gridmix/src/test/org/apache/hadoop/mapred/gridmix/TestRandomTextDataGenerator.java
  48. 612 0
      src/contrib/gridmix/src/test/org/apache/hadoop/mapred/gridmix/TestResourceUsageEmulators.java
  49. 109 33
      src/contrib/gridmix/src/test/org/apache/hadoop/mapred/gridmix/TestUserResolve.java
  50. 9 0
      src/core/org/apache/hadoop/util/Progress.java
  51. 757 139
      src/docs/src/documentation/content/xdocs/gridmix.xml
  52. 454 0
      src/docs/src/documentation/content/xdocs/rumen.xml
  53. 1 0
      src/docs/src/documentation/content/xdocs/site.xml
  54. 2 2
      src/mapred/org/apache/hadoop/mapred/JobConf.java
  55. 2 2
      src/mapred/org/apache/hadoop/mapred/JobTracker.java
  56. 10 0
      src/mapred/org/apache/hadoop/mapred/Reporter.java
  57. 5 0
      src/mapred/org/apache/hadoop/mapred/Task.java
  58. 6 0
      src/mapred/org/apache/hadoop/mapreduce/StatusReporter.java
  59. 4 0
      src/mapred/org/apache/hadoop/mapreduce/TaskInputOutputContext.java
  60. 4 0
      src/mapred/org/apache/hadoop/mapreduce/lib/map/MultithreadedMapper.java
  61. 11 2
      src/test/org/apache/hadoop/mapred/UtilsForTests.java
  62. 4 0
      src/test/org/apache/hadoop/mapreduce/MapReduceTestUtil.java
  63. 404 52
      src/test/org/apache/hadoop/tools/rumen/TestRumenJobTraces.java
  64. 二进制
      src/test/tools/data/rumen/small-trace-test/counters-test-trace.json.gz
  65. 二进制
      src/test/tools/data/rumen/small-trace-test/dispatch-trace-output.json.gz
  66. 二进制
      src/test/tools/data/rumen/small-trace-test/job-tracker-logs-trace-output.gz
  67. 4 1
      src/test/tools/data/rumen/small-trace-test/sample-conf.file.xml
  68. 3 0
      src/test/tools/data/rumen/small-trace-test/truncated-trace-output
  69. 二进制
      src/test/tools/data/rumen/small-trace-test/v20-resource-usage-log.gz
  70. 2 2
      src/tools/org/apache/hadoop/tools/rumen/ClusterStory.java
  71. 1 1
      src/tools/org/apache/hadoop/tools/rumen/DeskewedJobTraceReader.java
  72. 34 0
      src/tools/org/apache/hadoop/tools/rumen/HadoopLogsAnalyzer.java
  73. 10 2
      src/tools/org/apache/hadoop/tools/rumen/JobBuilder.java
  74. 3 18
      src/tools/org/apache/hadoop/tools/rumen/JobConfigurationParser.java
  75. 1 1
      src/tools/org/apache/hadoop/tools/rumen/JobHistoryParserFactory.java
  76. 51 0
      src/tools/org/apache/hadoop/tools/rumen/LoggedJob.java
  77. 45 0
      src/tools/org/apache/hadoop/tools/rumen/LoggedTaskAttempt.java
  78. 1 1
      src/tools/org/apache/hadoop/tools/rumen/Node.java
  79. 5 0
      src/tools/org/apache/hadoop/tools/rumen/ParsedConfigFile.java
  80. 160 0
      src/tools/org/apache/hadoop/tools/rumen/ResourceUsageMetrics.java
  81. 1 2
      src/tools/org/apache/hadoop/tools/rumen/TaskAttemptInfo.java
  82. 14 0
      src/tools/org/apache/hadoop/tools/rumen/TaskInfo.java
  83. 111 36
      src/tools/org/apache/hadoop/tools/rumen/TraceBuilder.java
  84. 18 2
      src/tools/org/apache/hadoop/tools/rumen/ZombieJob.java
  85. 377 0
      src/tools/org/apache/hadoop/tools/rumen/package-info.java

+ 2 - 0
CHANGES.txt

@@ -4,6 +4,8 @@ Release 0.20.206.0 - unreleased
 
 
   NEW FEATURES
   NEW FEATURES
 
 
+    MAPREDUCE-3118. Backport Gridmix and Rumen features to 
+                    branch-0.20-security (Ravi Gummadi via amarrk)
   BUG FIXES
   BUG FIXES
 
 
     HDFS-2305. Running multiple 2NNs can result in corrupt file system. (atm)
     HDFS-2305. Running multiple 2NNs can result in corrupt file system. (atm)

+ 3 - 0
build.xml

@@ -1291,6 +1291,7 @@
         <packageset dir="${mapred.src.dir}"/>
         <packageset dir="${mapred.src.dir}"/>
         <packageset dir="${hdfs.src.dir}"/>        	
         <packageset dir="${hdfs.src.dir}"/>        	
     	<packageset dir="${examples.dir}"/>
     	<packageset dir="${examples.dir}"/>
+    	<packageset dir="${tools.src}"/>
 
 
     	<packageset dir="src/contrib/streaming/src/java"/>
     	<packageset dir="src/contrib/streaming/src/java"/>
     	<packageset dir="src/contrib/data_join/src/java"/>
     	<packageset dir="src/contrib/data_join/src/java"/>
@@ -1371,6 +1372,8 @@
        <packageset dir="src/core"/>
        <packageset dir="src/core"/>
        <packageset dir="src/mapred"/>
        <packageset dir="src/mapred"/>
        <packageset dir="src/tools"/>
        <packageset dir="src/tools"/>
+       <packageset dir="${tools.src}"/>
+       <packageset dir="${tools.src}"/>
        <classpath >
        <classpath >
          <path refid="classpath" />
          <path refid="classpath" />
          <path refid="jdiff-classpath" />
          <path refid="jdiff-classpath" />

+ 2 - 0
src/contrib/build-contrib.xml

@@ -33,6 +33,7 @@
 
 
   <property name="src.dir"  location="${root}/src/java"/>
   <property name="src.dir"  location="${root}/src/java"/>
   <property name="src.test" location="${root}/src/test"/>
   <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 added for contrib system tests -->
   <property name="src.test.system" location="${root}/src/test/system"/>
   <property name="src.test.system" location="${root}/src/test/system"/>
 
 
@@ -289,6 +290,7 @@
       
       
       <sysproperty key="test.build.data" value="${build.test}/data"/>
       <sysproperty key="test.build.data" value="${build.test}/data"/>
       <sysproperty key="build.test" value="${build.test}"/>
       <sysproperty key="build.test" value="${build.test}"/>
+      <sysproperty key="src.test.data" value="${src.test.data}"/>
       <sysproperty key="contrib.name" value="${name}"/>
       <sysproperty key="contrib.name" value="${name}"/>
       
       
       <!-- requires fork=yes for: 
       <!-- requires fork=yes for: 

+ 34 - 4
src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/AvgRecordFactory.java

@@ -40,6 +40,8 @@ class AvgRecordFactory extends RecordFactory {
   private final int keyLen;
   private final int keyLen;
   private long accBytes = 0L;
   private long accBytes = 0L;
   private long accRecords = 0L;
   private long accRecords = 0L;
+  private int unspilledBytes = 0;
+  private int minSpilledBytes = 0;
 
 
   /**
   /**
    * @param targetBytes Expected byte count.
    * @param targetBytes Expected byte count.
@@ -48,6 +50,14 @@ class AvgRecordFactory extends RecordFactory {
    */
    */
   public AvgRecordFactory(long targetBytes, long targetRecords,
   public AvgRecordFactory(long targetBytes, long targetRecords,
       Configuration conf) {
       Configuration conf) {
+    this(targetBytes, targetRecords, conf, 0);
+  }
+  
+  /**
+   * @param minSpilledBytes Minimum amount of data expected per record
+   */
+  public AvgRecordFactory(long targetBytes, long targetRecords,
+      Configuration conf, int minSpilledBytes) {
     this.targetBytes = targetBytes;
     this.targetBytes = targetBytes;
     this.targetRecords = targetRecords <= 0 && this.targetBytes >= 0
     this.targetRecords = targetRecords <= 0 && this.targetBytes >= 0
       ? Math.max(1,
       ? Math.max(1,
@@ -58,6 +68,7 @@ class AvgRecordFactory extends RecordFactory {
     avgrec = (int) Math.min(Integer.MAX_VALUE, tmp + 1);
     avgrec = (int) Math.min(Integer.MAX_VALUE, tmp + 1);
     keyLen = Math.max(1,
     keyLen = Math.max(1,
         (int)(tmp * Math.min(1.0f, conf.getFloat(GRIDMIX_KEY_FRC, 0.1f))));
         (int)(tmp * Math.min(1.0f, conf.getFloat(GRIDMIX_KEY_FRC, 0.1f))));
+    this.minSpilledBytes = minSpilledBytes;
   }
   }
 
 
   @Override
   @Override
@@ -67,14 +78,33 @@ class AvgRecordFactory extends RecordFactory {
     }
     }
     final int reclen = accRecords++ >= step ? avgrec - 1 : avgrec;
     final int reclen = accRecords++ >= step ? avgrec - 1 : avgrec;
     final int len = (int) Math.min(targetBytes - accBytes, reclen);
     final int len = (int) Math.min(targetBytes - accBytes, reclen);
+    
+    unspilledBytes += len;
+    
     // len != reclen?
     // len != reclen?
     if (key != null) {
     if (key != null) {
-      key.setSize(keyLen);
-      val.setSize(len - key.getSize());
+      if (unspilledBytes < minSpilledBytes && accRecords < targetRecords) {
+        key.setSize(1);
+        val.setSize(1);
+        accBytes += key.getSize() + val.getSize();
+        unspilledBytes -= (key.getSize() + val.getSize());
+      } else {
+        key.setSize(keyLen);
+        val.setSize(unspilledBytes - key.getSize());
+        accBytes += unspilledBytes;
+        unspilledBytes = 0;
+      }
     } else {
     } else {
-      val.setSize(len);
+      if (unspilledBytes < minSpilledBytes && accRecords < targetRecords) {
+        val.setSize(1);
+        accBytes += val.getSize();
+        unspilledBytes -= val.getSize();
+      } else {
+        val.setSize(unspilledBytes);
+        accBytes += unspilledBytes;
+        unspilledBytes = 0;
+      }
     }
     }
-    accBytes += len;
     return true;
     return true;
   }
   }
 
 

+ 116 - 0
src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/ClusterSummarizer.java

@@ -0,0 +1,116 @@
+/**
+ * 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.lang.time.FastDateFormat;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.mapred.JobTracker;
+import org.apache.hadoop.mapred.gridmix.Statistics.ClusterStats;
+
+/**
+ * Summarizes the Hadoop cluster used in this {@link Gridmix} run. 
+ * Statistics that are reported are
+ * <ul>
+ *   <li>Total number of active trackers in the cluster</li>
+ *   <li>Total number of blacklisted trackers in the cluster</li>
+ *   <li>Max map task capacity of the cluster</li>
+ *   <li>Max reduce task capacity of the cluster</li>
+ * </ul>
+ * 
+ * Apart from these statistics, {@link JobTracker} and {@link FileSystem} 
+ * addresses are also recorded in the summary.
+ */
+class ClusterSummarizer implements StatListener<ClusterStats> {
+  static final Log LOG = LogFactory.getLog(ClusterSummarizer.class);
+  
+  private int numBlacklistedTrackers;
+  private int numActiveTrackers;
+  private int maxMapTasks;
+  private int maxReduceTasks;
+  private String jobTrackerInfo = Summarizer.NA;
+  private String namenodeInfo = Summarizer.NA;
+  
+  @Override
+  @SuppressWarnings("deprecation")
+  public void update(ClusterStats item) {
+    try {
+      numBlacklistedTrackers = item.getStatus().getBlacklistedTrackers();
+      numActiveTrackers = item.getStatus().getTaskTrackers();
+      maxMapTasks = item.getStatus().getMaxMapTasks();
+      maxReduceTasks = item.getStatus().getMaxReduceTasks();
+    } catch (Exception e) {
+      long time = System.currentTimeMillis();
+      LOG.info("Error in processing cluster status at " 
+               + FastDateFormat.getInstance().format(time));
+    }
+  }
+  
+  /**
+   * Summarizes the cluster used for this {@link Gridmix} run.
+   */
+  @Override
+  public String toString() {
+    StringBuilder builder = new StringBuilder();
+    builder.append("Cluster Summary:-");
+    builder.append("\nJobTracker: ").append(getJobTrackerInfo());
+    builder.append("\nFileSystem: ").append(getNamenodeInfo());
+    builder.append("\nNumber of blacklisted trackers: ")
+           .append(getNumBlacklistedTrackers());
+    builder.append("\nNumber of active trackers: ")
+           .append(getNumActiveTrackers());
+    builder.append("\nMax map task capacity: ")
+           .append(getMaxMapTasks());
+    builder.append("\nMax reduce task capacity: ").append(getMaxReduceTasks());
+    builder.append("\n\n");
+    return builder.toString();
+  }
+  
+  void start(Configuration conf) {
+    jobTrackerInfo = conf.get("mapred.job.tracker");
+    namenodeInfo = conf.get(CommonConfigurationKeys.FS_DEFAULT_NAME_KEY);
+  }
+  
+  // Getters
+  protected int getNumBlacklistedTrackers() {
+    return numBlacklistedTrackers;
+  }
+  
+  protected int getNumActiveTrackers() {
+    return numActiveTrackers;
+  }
+  
+  protected int getMaxMapTasks() {
+    return maxMapTasks;
+  }
+  
+  protected int getMaxReduceTasks() {
+    return maxReduceTasks;
+  }
+  
+  protected String getJobTrackerInfo() {
+    return jobTrackerInfo;
+  }
+  
+  protected String getNamenodeInfo() {
+    return namenodeInfo;
+  }
+}

+ 573 - 0
src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/CompressionEmulationUtil.java

@@ -0,0 +1,573 @@
+/**
+ * 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.DataOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.compress.CodecPool;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.io.compress.CompressionCodecFactory;
+import org.apache.hadoop.io.compress.CompressionInputStream;
+import org.apache.hadoop.io.compress.Decompressor;
+import org.apache.hadoop.io.compress.GzipCodec;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.Utils;
+import org.apache.hadoop.mapred.gridmix.GenerateData.DataStatistics;
+import org.apache.hadoop.mapred.gridmix.GenerateData.GenDataFormat;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.util.StringUtils;
+
+/**
+ * This is a utility class for all the compression related modules.
+ */
+class CompressionEmulationUtil {
+  static final Log LOG = LogFactory.getLog(CompressionEmulationUtil.class);
+  
+  /**
+   * Enable compression usage in GridMix runs.
+   */
+  private static final String COMPRESSION_EMULATION_ENABLE = 
+    "gridmix.compression-emulation.enable";
+  
+  /**
+   * Enable input data decompression.
+   */
+  private static final String INPUT_DECOMPRESSION_EMULATION_ENABLE = 
+    "gridmix.compression-emulation.input-decompression.enable";
+  
+  /**
+   * Configuration property for setting the compression ratio for map input 
+   * data.
+   */
+  private static final String GRIDMIX_MAP_INPUT_COMPRESSION_RATIO = 
+    "gridmix.compression-emulation.map-input.decompression-ratio";
+  
+  /**
+   * Configuration property for setting the compression ratio of map output.
+   */
+  private static final String GRIDMIX_MAP_OUTPUT_COMPRESSION_RATIO = 
+    "gridmix.compression-emulation.map-output.compression-ratio";
+  
+  /**
+   * Configuration property for setting the compression ratio of reduce output.
+   */
+  private static final String GRIDMIX_REDUCE_OUTPUT_COMPRESSION_RATIO = 
+    "gridmix.compression-emulation.reduce-output.compression-ratio";
+  
+  /**
+   * Default compression ratio.
+   */
+  static final float DEFAULT_COMPRESSION_RATIO = 0.5F;
+  
+  private static final CompressionRatioLookupTable COMPRESSION_LOOKUP_TABLE = 
+    new CompressionRatioLookupTable();
+  
+  /**
+   * This is a {@link Mapper} implementation for generating random text data.
+   * It uses {@link RandomTextDataGenerator} for generating text data and the
+   * output files are compressed.
+   */
+  public static class RandomTextDataMapper
+  extends Mapper<NullWritable, LongWritable, Text, Text> {
+    private RandomTextDataGenerator rtg;
+
+    @Override
+    protected void setup(Context context)
+        throws IOException, InterruptedException {
+      Configuration conf = context.getConfiguration();
+      int listSize = 
+        RandomTextDataGenerator.getRandomTextDataGeneratorListSize(conf);
+      int wordSize = 
+        RandomTextDataGenerator.getRandomTextDataGeneratorWordSize(conf);
+      rtg = new RandomTextDataGenerator(listSize, wordSize);
+    }
+    
+    /**
+     * Emits random words sequence of desired size. Note that the desired output
+     * size is passed as the value parameter to this map.
+     */
+    @Override
+    public void map(NullWritable key, LongWritable value, Context context)
+    throws IOException, InterruptedException {
+      //TODO Control the extra data written ..
+      //TODO Should the key\tvalue\n be considered for measuring size?
+      //     Can counters like BYTES_WRITTEN be used? What will be the value of
+      //     such counters in LocalJobRunner?
+      for (long bytes = value.get(); bytes > 0;) {
+        String randomKey = rtg.getRandomWord();
+        String randomValue = rtg.getRandomWord();
+        context.write(new Text(randomKey), new Text(randomValue));
+        bytes -= (randomValue.getBytes().length + randomKey.getBytes().length);
+      }
+    }
+  }
+  
+  /**
+   * Configure the {@link Job} for enabling compression emulation.
+   */
+  static void configure(final Job job) throws IOException, InterruptedException,
+                                              ClassNotFoundException {
+    // set the random text mapper
+    job.setMapperClass(RandomTextDataMapper.class);
+    job.setNumReduceTasks(0);
+    job.setMapOutputKeyClass(Text.class);
+    job.setMapOutputValueClass(Text.class);
+    job.setInputFormatClass(GenDataFormat.class);
+    job.setJarByClass(GenerateData.class);
+
+    // set the output compression true
+    FileOutputFormat.setCompressOutput(job, true);
+    try {
+      FileInputFormat.addInputPath(job, new Path("ignored"));
+    } catch (IOException e) {
+      LOG.error("Error while adding input path ", e);
+    }
+  }
+
+  /**
+   * This is the lookup table for mapping compression ratio to the size of the 
+   * word in the {@link RandomTextDataGenerator}'s dictionary. 
+   * 
+   * Note that this table is computed (empirically) using a dictionary of 
+   * default length i.e {@value RandomTextDataGenerator#DEFAULT_LIST_SIZE}.
+   */
+  private static class CompressionRatioLookupTable {
+    private static Map<Float, Integer> map = new HashMap<Float, Integer>(60);
+    private static final float MIN_RATIO = 0.07F;
+    private static final float MAX_RATIO = 0.68F;
+    
+    // add the empirically obtained data points in the lookup table
+    CompressionRatioLookupTable() {
+      map.put(.07F,30);
+      map.put(.08F,25);
+      map.put(.09F,60);
+      map.put(.10F,20);
+      map.put(.11F,70);
+      map.put(.12F,15);
+      map.put(.13F,80);
+      map.put(.14F,85);
+      map.put(.15F,90);
+      map.put(.16F,95);
+      map.put(.17F,100);
+      map.put(.18F,105);
+      map.put(.19F,110);
+      map.put(.20F,115);
+      map.put(.21F,120);
+      map.put(.22F,125);
+      map.put(.23F,130);
+      map.put(.24F,140);
+      map.put(.25F,145);
+      map.put(.26F,150);
+      map.put(.27F,155);
+      map.put(.28F,160);
+      map.put(.29F,170);
+      map.put(.30F,175);
+      map.put(.31F,180);
+      map.put(.32F,190);
+      map.put(.33F,195);
+      map.put(.34F,205);
+      map.put(.35F,215);
+      map.put(.36F,225);
+      map.put(.37F,230);
+      map.put(.38F,240);
+      map.put(.39F,250);
+      map.put(.40F,260);
+      map.put(.41F,270);
+      map.put(.42F,280);
+      map.put(.43F,295);
+      map.put(.44F,310);
+      map.put(.45F,325);
+      map.put(.46F,335);
+      map.put(.47F,355);
+      map.put(.48F,375);
+      map.put(.49F,395);
+      map.put(.50F,420);
+      map.put(.51F,440);
+      map.put(.52F,465);
+      map.put(.53F,500);
+      map.put(.54F,525);
+      map.put(.55F,550);
+      map.put(.56F,600);
+      map.put(.57F,640);
+      map.put(.58F,680);
+      map.put(.59F,734);
+      map.put(.60F,813);
+      map.put(.61F,905);
+      map.put(.62F,1000);
+      map.put(.63F,1055);
+      map.put(.64F,1160);
+      map.put(.65F,1355);
+      map.put(.66F,1510);
+      map.put(.67F,1805);
+      map.put(.68F,2170);
+    }
+    
+    /**
+     * Returns the size of the word in {@link RandomTextDataGenerator}'s 
+     * dictionary that can generate text with the desired compression ratio.
+     * 
+     * @throws RuntimeException If ratio is less than {@value #MIN_RATIO} or 
+     *                          greater than {@value #MAX_RATIO}.
+     */
+    int getWordSizeForRatio(float ratio) {
+      ratio = standardizeCompressionRatio(ratio);
+      if (ratio >= MIN_RATIO && ratio <= MAX_RATIO) {
+        return map.get(ratio);
+      } else {
+        throw new RuntimeException("Compression ratio should be in the range [" 
+          + MIN_RATIO + "," + MAX_RATIO + "]. Configured compression ratio is " 
+          + ratio + ".");
+      }
+    }
+  }
+  
+  /**
+   * Setup the data generator's configuration to generate compressible random 
+   * text data with the desired compression ratio.
+   * Note that the compression ratio, if configured, will set the 
+   * {@link RandomTextDataGenerator}'s list-size and word-size based on 
+   * empirical values using the compression ratio set in the configuration. 
+   * 
+   * Hence to achieve the desired compression ratio, 
+   * {@link RandomTextDataGenerator}'s list-size will be set to the default 
+   * value i.e {@value RandomTextDataGenerator#DEFAULT_LIST_SIZE}.
+   */
+  static void setupDataGeneratorConfig(Configuration conf) {
+    boolean compress = isCompressionEmulationEnabled(conf);
+    if (compress) {
+      float ratio = getMapInputCompressionEmulationRatio(conf);
+      LOG.info("GridMix is configured to generate compressed input data with "
+               + " a compression ratio of " + ratio);
+      int wordSize = COMPRESSION_LOOKUP_TABLE.getWordSizeForRatio(ratio);
+      RandomTextDataGenerator.setRandomTextDataGeneratorWordSize(conf, 
+                                                                 wordSize);
+
+      // since the compression ratios are computed using the default value of 
+      // list size
+      RandomTextDataGenerator.setRandomTextDataGeneratorListSize(conf, 
+          RandomTextDataGenerator.DEFAULT_LIST_SIZE);
+    }
+  }
+  
+  /**
+   * Returns a {@link RandomTextDataGenerator} that generates random 
+   * compressible text with the desired compression ratio.
+   */
+  static RandomTextDataGenerator getRandomTextDataGenerator(float ratio, 
+                                                            long seed) {
+    int wordSize = COMPRESSION_LOOKUP_TABLE.getWordSizeForRatio(ratio);
+    RandomTextDataGenerator rtg = 
+      new RandomTextDataGenerator(RandomTextDataGenerator.DEFAULT_LIST_SIZE, 
+            seed, wordSize);
+    return rtg;
+  }
+  
+  /** Publishes compression related data statistics. Following statistics are
+   * published
+   * <ul>
+   *   <li>Total compressed input data size</li>
+   *   <li>Number of compressed input data files</li>
+   *   <li>Compression Ratio</li>
+   *   <li>Text data dictionary size</li>
+   *   <li>Random text word size</li>
+   * </ul>
+   */
+  static DataStatistics publishCompressedDataStatistics(Path inputDir, 
+                          Configuration conf, long uncompressedDataSize) 
+  throws IOException {
+    FileSystem fs = inputDir.getFileSystem(conf);
+    CompressionCodecFactory compressionCodecs = 
+      new CompressionCodecFactory(conf);
+
+    // iterate over compressed files and sum up the compressed file sizes
+    long compressedDataSize = 0;
+    int numCompressedFiles = 0;
+    // obtain input data file statuses
+    FileStatus[] outFileStatuses = 
+      fs.listStatus(inputDir, new Utils.OutputFileUtils.OutputFilesFilter());
+    for (FileStatus status : outFileStatuses) {
+      // check if the input file is compressed
+      if (compressionCodecs != null) {
+        CompressionCodec codec = compressionCodecs.getCodec(status.getPath());
+        if (codec != null) {
+          ++numCompressedFiles;
+          compressedDataSize += status.getLen();
+        }
+      }
+    }
+
+    LOG.info("Gridmix is configured to use compressed input data.");
+    // publish the input data size
+    LOG.info("Total size of compressed input data : " 
+             + StringUtils.humanReadableInt(compressedDataSize));
+    LOG.info("Total number of compressed input data files : " 
+             + numCompressedFiles);
+
+    if (numCompressedFiles == 0) {
+      throw new RuntimeException("No compressed file found in the input" 
+          + " directory : " + inputDir.toString() + ". To enable compression"
+          + " emulation, run Gridmix either with "
+          + " an input directory containing compressed input file(s) or" 
+          + " use the -generate option to (re)generate it. If compression"
+          + " emulation is not desired, disable it by setting '" 
+          + COMPRESSION_EMULATION_ENABLE + "' to 'false'.");
+    }
+    
+    // publish compression ratio only if its generated in this gridmix run
+    if (uncompressedDataSize > 0) {
+      // compute the compression ratio
+      double ratio = ((double)compressedDataSize) / uncompressedDataSize;
+
+      // publish the compression ratio
+      LOG.info("Input Data Compression Ratio : " + ratio);
+    }
+    
+    return new DataStatistics(compressedDataSize, numCompressedFiles, true);
+  }
+  
+  /**
+   * Enables/Disables compression emulation.
+   * @param conf Target configuration where the parameter 
+   * {@value #COMPRESSION_EMULATION_ENABLE} will be set. 
+   * @param val The value to be set.
+   */
+  static void setCompressionEmulationEnabled(Configuration conf, boolean val) {
+    conf.setBoolean(COMPRESSION_EMULATION_ENABLE, val);
+  }
+  
+  /**
+   * Checks if compression emulation is enabled or not. Default is {@code true}.
+   */
+  static boolean isCompressionEmulationEnabled(Configuration conf) {
+    return conf.getBoolean(COMPRESSION_EMULATION_ENABLE, true);
+  }
+  
+  /**
+   * Enables/Disables input decompression emulation.
+   * @param conf Target configuration where the parameter 
+   * {@value #INPUT_DECOMPRESSION_EMULATION_ENABLE} will be set. 
+   * @param val The value to be set.
+   */
+  static void setInputCompressionEmulationEnabled(Configuration conf, 
+                                                  boolean val) {
+    conf.setBoolean(INPUT_DECOMPRESSION_EMULATION_ENABLE, val);
+  }
+  
+  /**
+   * Check if input decompression emulation is enabled or not. 
+   * Default is {@code false}.
+   */
+  static boolean isInputCompressionEmulationEnabled(Configuration conf) {
+    return conf.getBoolean(INPUT_DECOMPRESSION_EMULATION_ENABLE, false);
+  }
+  
+  /**
+   * Set the map input data compression ratio in the given conf.
+   */
+  static void setMapInputCompressionEmulationRatio(Configuration conf, 
+                                                   float ratio) {
+    conf.setFloat(GRIDMIX_MAP_INPUT_COMPRESSION_RATIO, ratio);
+  }
+  
+  /**
+   * Get the map input data compression ratio using the given configuration.
+   * If the compression ratio is not set in the configuration then use the 
+   * default value i.e {@value #DEFAULT_COMPRESSION_RATIO}.
+   */
+  static float getMapInputCompressionEmulationRatio(Configuration conf) {
+    return conf.getFloat(GRIDMIX_MAP_INPUT_COMPRESSION_RATIO, 
+                         DEFAULT_COMPRESSION_RATIO);
+  }
+  
+  /**
+   * Set the map output data compression ratio in the given configuration.
+   */
+  static void setMapOutputCompressionEmulationRatio(Configuration conf, 
+                                                    float ratio) {
+    conf.setFloat(GRIDMIX_MAP_OUTPUT_COMPRESSION_RATIO, ratio);
+  }
+  
+  /**
+   * Get the map output data compression ratio using the given configuration.
+   * If the compression ratio is not set in the configuration then use the 
+   * default value i.e {@value #DEFAULT_COMPRESSION_RATIO}.
+   */
+  static float getMapOutputCompressionEmulationRatio(Configuration conf) {
+    return conf.getFloat(GRIDMIX_MAP_OUTPUT_COMPRESSION_RATIO, 
+                         DEFAULT_COMPRESSION_RATIO);
+  }
+  
+  /**
+   * Set the reduce output data compression ratio in the given configuration.
+   */
+  static void setReduceOutputCompressionEmulationRatio(Configuration conf, 
+                                                       float ratio) {
+    conf.setFloat(GRIDMIX_REDUCE_OUTPUT_COMPRESSION_RATIO, ratio);
+  }
+  
+  /**
+   * Get the reduce output data compression ratio using the given configuration.
+   * If the compression ratio is not set in the configuration then use the 
+   * default value i.e {@value #DEFAULT_COMPRESSION_RATIO}.
+   */
+  static float getReduceOutputCompressionEmulationRatio(Configuration conf) {
+    return conf.getFloat(GRIDMIX_REDUCE_OUTPUT_COMPRESSION_RATIO, 
+                         DEFAULT_COMPRESSION_RATIO);
+  }
+  
+  /**
+   * Standardize the compression ratio i.e round off the compression ratio to
+   * only 2 significant digits.
+   */
+  static float standardizeCompressionRatio(float ratio) {
+    // round off to 2 significant digits
+    int significant = (int)Math.round(ratio * 100);
+    return ((float)significant)/100;
+  }
+  
+  /**
+   * Returns a {@link InputStream} for a file that might be compressed.
+   */
+  static InputStream getPossiblyDecompressedInputStream(Path file, 
+                                                        Configuration conf,
+                                                        long offset)
+  throws IOException {
+    FileSystem fs = file.getFileSystem(conf);
+    if (isCompressionEmulationEnabled(conf)
+        && isInputCompressionEmulationEnabled(conf)) {
+      CompressionCodecFactory compressionCodecs = 
+        new CompressionCodecFactory(conf);
+      CompressionCodec codec = compressionCodecs.getCodec(file);
+      if (codec != null) {
+        Decompressor decompressor = CodecPool.getDecompressor(codec);
+        if (decompressor != null) {
+          CompressionInputStream in = 
+            codec.createInputStream(fs.open(file), decompressor);
+          //TODO Seek doesnt work with compressed input stream. 
+          //     Use SplittableCompressionCodec?
+          return (InputStream)in;
+        }
+      }
+    }
+    FSDataInputStream in = fs.open(file);
+    in.seek(offset);
+    return (InputStream)in;
+  }
+  
+  /**
+   * Returns a {@link OutputStream} for a file that might need 
+   * compression.
+   */
+  static OutputStream getPossiblyCompressedOutputStream(Path file, 
+                                                        Configuration conf)
+  throws IOException {
+    FileSystem fs = file.getFileSystem(conf);
+    JobConf jConf = new JobConf(conf);
+    if (org.apache.hadoop.mapred.FileOutputFormat.getCompressOutput(jConf)) {
+      // get the codec class
+      Class<? extends CompressionCodec> codecClass =
+        org.apache.hadoop.mapred.FileOutputFormat
+                                .getOutputCompressorClass(jConf, 
+                                                          GzipCodec.class);
+      // get the codec implementation
+      CompressionCodec codec = ReflectionUtils.newInstance(codecClass, conf);
+
+      // add the appropriate extension
+      file = file.suffix(codec.getDefaultExtension());
+
+      if (isCompressionEmulationEnabled(conf)) {
+        FSDataOutputStream fileOut = fs.create(file, false);
+        return new DataOutputStream(codec.createOutputStream(fileOut));
+      }
+    }
+    return fs.create(file, false);
+  }
+  
+  /**
+   * Extracts compression/decompression related configuration parameters from 
+   * the source configuration to the target configuration.
+   */
+  static void configureCompressionEmulation(Configuration source, 
+                                            Configuration target) {
+    // enable output compression
+    target.setBoolean("mapred.output.compress",
+    		source.getBoolean("mapred.output.compress", false));
+
+    // set the job output compression codec
+    String jobOutputCompressionCodec = 
+      source.get("mapred.output.compression.codec");
+    if (jobOutputCompressionCodec != null) {
+      target.set("mapred.output.compression.codec", jobOutputCompressionCodec);
+    }
+
+    // set the job output compression type
+    String jobOutputCompressionType = 
+      source.get("mapred.output.compression.type");
+    if (jobOutputCompressionType != null) {
+      target.set("mapred.output.compression.type", jobOutputCompressionType);
+    }
+
+    // enable map output compression
+    target.setBoolean("mapred.compress.map.output",
+        source.getBoolean("mapred.compress.map.output", false));
+
+    // set the map output compression codecs
+    String mapOutputCompressionCodec = 
+      source.get("mapred.map.output.compression.codec");
+    if (mapOutputCompressionCodec != null) {
+      target.set("mapred.map.output.compression.codec", 
+                 mapOutputCompressionCodec);
+    }
+
+    // enable input decompression
+    //TODO replace with mapInputBytes and hdfsBytesRead
+    Path[] inputs = 
+      org.apache.hadoop.mapred.FileInputFormat
+         .getInputPaths(new JobConf(source));
+    boolean needsCompressedInput = false;
+    CompressionCodecFactory compressionCodecs = 
+      new CompressionCodecFactory(source);
+    for (Path input : inputs) {
+      CompressionCodec codec = compressionCodecs.getCodec(input);
+      if (codec != null) {
+        needsCompressedInput = true;
+      }
+    }
+    setInputCompressionEmulationEnabled(target, needsCompressedInput);
+  }
+}

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

@@ -0,0 +1,543 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.filecache.DistributedCache;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.MD5Hash;
+import org.apache.hadoop.io.SequenceFile;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.tools.rumen.JobStory;
+import org.apache.hadoop.tools.rumen.JobStoryProducer;
+import org.apache.hadoop.tools.rumen.Pre21JobHistoryConstants;
+
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Emulation of Distributed Cache Usage in gridmix.
+ * <br> Emulation of Distributed Cache Load in gridmix will put load on
+ * TaskTrackers and affects execution time of tasks because of localization of
+ * distributed cache files by TaskTrackers.
+ * <br> Gridmix creates distributed cache files for simulated jobs by launching
+ * a MapReduce job {@link GenerateDistCacheData} in advance i.e. before
+ * launching simulated jobs.
+ * <br> The distributed cache file paths used in the original cluster are mapped
+ * to unique file names in the simulated cluster.
+ * <br> All HDFS-based distributed cache files generated by gridmix are
+ * public distributed cache files. But Gridmix makes sure that load incurred due
+ * to localization of private distributed cache files on the original cluster
+ * is also faithfully simulated. Gridmix emulates the load due to private
+ * distributed cache files by mapping private distributed cache files of
+ * different users in the original cluster to different public distributed cache
+ * files in the simulated cluster.
+ *
+ * <br> The configuration properties like
+ * {@link DistributedCache#CACHE_FILES},
+ * {@link JobContext#CACHE_FILE_VISIBILITIES},
+ * {@link DistributedCache#CACHE_FILES_SIZES} and
+ * {@link DistributedCache#CACHE_FILES_TIMESTAMPS} obtained from trace are used
+ * to decide
+ * <li> file size of each distributed cache file to be generated
+ * <li> whether a distributed cache file is already seen in this trace file
+ * <li> whether a distributed cache file was considered public or private.
+ * <br>
+ * <br> Gridmix configures these generated files as distributed cache files for
+ * the simulated jobs.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+class DistributedCacheEmulator {
+  private static final Log LOG =
+      LogFactory.getLog(DistributedCacheEmulator.class);
+
+  static final long AVG_BYTES_PER_MAP = 128 * 1024 * 1024L;// 128MB
+
+  // If at least 1 distributed cache file is missing in the expected
+  // distributed cache dir, Gridmix cannot proceed with emulation of
+  // distributed cache load.
+  int MISSING_DIST_CACHE_FILES_ERROR = 1;
+
+  private Path distCachePath;
+
+  /**
+   * Map between simulated cluster's distributed cache file paths and their
+   * file sizes. Unique distributed cache files are entered into this map.
+   * 2 distributed cache files are considered same if and only if their
+   * file paths, visibilities and timestamps are same.
+   */
+  private Map<String, Long> distCacheFiles = new HashMap<String, Long>();
+
+  /**
+   * Configuration property for whether gridmix should emulate
+   * distributed cache usage or not. Default value is true.
+   */
+  static final String GRIDMIX_EMULATE_DISTRIBUTEDCACHE =
+      "gridmix.distributed-cache-emulation.enable";
+
+  // Whether to emulate distributed cache usage or not
+  boolean emulateDistributedCache = true;
+
+  // Whether to generate distributed cache data or not
+  boolean generateDistCacheData = false;
+
+  Configuration conf; // gridmix configuration
+
+  // Pseudo local file system where local FS based distributed cache files are
+  // created by gridmix.
+  FileSystem pseudoLocalFs = null;
+
+  /**
+   * @param conf gridmix configuration
+   * @param ioPath &lt;ioPath&gt;/distributedCache/ is the gridmix Distributed
+   *               Cache directory
+   */
+  public DistributedCacheEmulator(Configuration conf, Path ioPath) {
+    this.conf = conf;
+    distCachePath = new Path(ioPath, "distributedCache");
+    this.conf.setClass("fs.pseudo.impl", PseudoLocalFs.class, FileSystem.class);
+  }
+
+  /**
+   * This is to be called before any other method of DistributedCacheEmulator.
+   * <br> Checks if emulation of distributed cache load is needed and is feasible.
+   *  Sets the flags generateDistCacheData and emulateDistributedCache to the
+   *  appropriate values.
+   * <br> Gridmix does not emulate distributed cache load if
+   * <ol><li> the specific gridmix job type doesn't need emulation of
+   * distributed cache load OR
+   * <li> the trace is coming from a stream instead of file OR
+   * <li> the distributed cache dir where distributed cache data is to be
+   * generated by gridmix is on local file system OR
+   * <li> execute permission is not there for any of the ascendant directories
+   * of &lt;ioPath&gt; till root. This is because for emulation of distributed
+   * cache load, distributed cache files created under
+   * &lt;ioPath/distributedCache/public/&gt; should be considered by hadoop
+   * as public distributed cache files.
+   * <li> creation of pseudo local file system fails.</ol>
+   * <br> For (2), (3), (4) and (5), generation of distributed cache data
+   * is also disabled.
+   * 
+   * @param traceIn trace file path. If this is '-', then trace comes from the
+   *                stream stdin.
+   * @param jobCreator job creator of gridmix jobs of a specific type
+   * @param generate  true if -generate option was specified
+   * @throws IOException
+   */
+  void init(String traceIn, JobCreator jobCreator, boolean generate)
+      throws IOException {
+    emulateDistributedCache = jobCreator.canEmulateDistCacheLoad()
+        && conf.getBoolean(GRIDMIX_EMULATE_DISTRIBUTEDCACHE, true);
+    generateDistCacheData = generate;
+
+    if (generateDistCacheData || emulateDistributedCache) {
+      if ("-".equals(traceIn)) {// trace is from stdin
+        LOG.warn("Gridmix will not emulate Distributed Cache load because "
+            + "the input trace source is a stream instead of file.");
+        emulateDistributedCache = generateDistCacheData = false;
+      } else if (FileSystem.getLocal(conf).getUri().getScheme().equals(
+          distCachePath.toUri().getScheme())) {// local FS
+        LOG.warn("Gridmix will not emulate Distributed Cache load because "
+            + "<iopath> provided is on local file system.");
+        emulateDistributedCache = generateDistCacheData = false;
+      } else {
+        // Check if execute permission is there for all the ascendant
+        // directories of distCachePath till root.
+        FileSystem fs = FileSystem.get(conf);
+        Path cur = distCachePath.getParent();
+        while (cur != null) {
+          if (cur.toString().length() > 0) {
+            FsPermission perm = fs.getFileStatus(cur).getPermission();
+            if (!perm.getOtherAction().and(FsAction.EXECUTE).equals(
+                FsAction.EXECUTE)) {
+              LOG.warn("Gridmix will not emulate Distributed Cache load "
+                  + "because the ascendant directory (of distributed cache "
+                  + "directory) " + cur + " doesn't have execute permission "
+                  + "for others.");
+              emulateDistributedCache = generateDistCacheData = false;
+              break;
+            }
+          }
+          cur = cur.getParent();
+        }
+      }
+    }
+
+    // Check if pseudo local file system can be created
+    try {
+      pseudoLocalFs = FileSystem.get(new URI("pseudo:///"), conf);
+    } catch (URISyntaxException e) {
+      LOG.warn("Gridmix will not emulate Distributed Cache load because "
+          + "creation of pseudo local file system failed.");
+      e.printStackTrace();
+      emulateDistributedCache = generateDistCacheData = false;
+      return;
+    }
+  }
+
+  /**
+   * @return true if gridmix should emulate distributed cache load
+   */
+  boolean shouldEmulateDistCacheLoad() {
+    return emulateDistributedCache;
+  }
+
+  /**
+   * @return true if gridmix should generate distributed cache data
+   */
+  boolean shouldGenerateDistCacheData() {
+    return generateDistCacheData;
+  }
+
+  /**
+   * @return the distributed cache directory path
+   */
+  Path getDistributedCacheDir() {
+    return distCachePath;
+  }
+
+  /**
+   * Create distributed cache directories.
+   * Also create a file that contains the list of distributed cache files
+   * that will be used as distributed cache files for all the simulated jobs.
+   * @param jsp job story producer for the trace
+   * @return exit code
+   * @throws IOException
+   */
+  int setupGenerateDistCacheData(JobStoryProducer jsp)
+      throws IOException {
+
+    createDistCacheDirectory();
+    return buildDistCacheFilesList(jsp);
+  }
+
+  /**
+   * Create distributed cache directory where distributed cache files will be
+   * created by the MapReduce job {@link GenerateDistCacheData#JOB_NAME}.
+   * @throws IOException
+   */
+  private void createDistCacheDirectory() throws IOException {
+    FileSystem fs = FileSystem.get(conf);
+    FileSystem.mkdirs(fs, distCachePath, new FsPermission((short) 0777));
+  }
+
+  /**
+   * Create the list of unique distributed cache files needed for all the
+   * simulated jobs and write the list to a special file.
+   * @param jsp job story producer for the trace
+   * @return exit code
+   * @throws IOException
+   */
+  private int buildDistCacheFilesList(JobStoryProducer jsp) throws IOException {
+    // Read all the jobs from the trace file and build the list of unique
+    // distributed cache files.
+    JobStory jobStory;
+    while ((jobStory = jsp.getNextJob()) != null) {
+      if (jobStory.getOutcome() == Pre21JobHistoryConstants.Values.SUCCESS && 
+         jobStory.getSubmissionTime() >= 0) {
+        updateHDFSDistCacheFilesList(jobStory);
+      }
+    }
+    jsp.close();
+
+    return writeDistCacheFilesList();
+  }
+
+  /**
+   * For the job to be simulated, identify the needed distributed cache files by
+   * mapping original cluster's distributed cache file paths to the simulated cluster's
+   * paths and add these paths in the map {@code distCacheFiles}.
+   *<br>
+   * JobStory should contain distributed cache related properties like
+   * <li> {@link DistributedCache#CACHE_FILES}
+   * <li> {@link JobContext#CACHE_FILE_VISIBILITIES}
+   * <li> {@link DistributedCache#CACHE_FILES_SIZES}
+   * <li> {@link DistributedCache#CACHE_FILES_TIMESTAMPS}
+   * <li> {@link DistributedCache#CLASSPATH_FILES}
+   *
+   * <li> {@link DistributedCache#CACHE_ARCHIVES}
+   * <li> {@link JobContext#CACHE_ARCHIVES_VISIBILITIES}
+   * <li> {@link DistributedCache#CACHE_ARCHIVES_SIZES}
+   * <li> {@link DistributedCache#CACHE_ARCHIVES_TIMESTAMPS}
+   * <li> {@link DistributedCache#CLASSPATH_ARCHIVES}
+   *
+   * <li> {@link DistributedCache#CACHE_SYMLINK}
+   *
+   * @param jobdesc JobStory of original job obtained from trace
+   * @throws IOException
+   */
+  void updateHDFSDistCacheFilesList(JobStory jobdesc) throws IOException {
+
+    // Map original job's distributed cache file paths to simulated cluster's
+    // paths, to be used by this simulated job.
+    JobConf jobConf = jobdesc.getJobConf();
+
+    String[] files = jobConf.getStrings(DistributedCache.CACHE_FILES);
+    if (files != null) {
+
+      String[] fileSizes = jobConf.getStrings(
+                               DistributedCache.CACHE_FILES_SIZES);
+      String[] visibilities =
+        jobConf.getStrings(JobContext.CACHE_FILE_VISIBILITIES);
+      String[] timeStamps =
+        jobConf.getStrings(DistributedCache.CACHE_FILES_TIMESTAMPS);
+
+      FileSystem fs = FileSystem.get(conf);
+      String user = jobConf.getUser();
+      for (int i = 0; i < files.length; i++) {
+        // Check if visibilities are available because older hadoop versions
+        // didn't have public, private Distributed Caches separately.
+        boolean visibility =
+            (visibilities == null) ? true : Boolean.valueOf(visibilities[i]);
+        if (isLocalDistCacheFile(files[i], user, visibility)) {
+          // local FS based distributed cache file.
+          // Create this file on the pseudo local FS on the fly (i.e. when the
+          // simulated job is submitted).
+          continue;
+        }
+        // distributed cache file on hdfs
+        String mappedPath = mapDistCacheFilePath(files[i], timeStamps[i],
+                                                 visibility, user);
+
+        // No need to add a distributed cache file path to the list if
+        // (1) the mapped path is already there in the list OR
+        // (2) the file with the mapped path already exists.
+        // In any of the above 2 cases, file paths, timestamps, file sizes and
+        // visibilities match. File sizes should match if file paths and
+        // timestamps match because single file path with single timestamp
+        // should correspond to a single file size.
+        if (distCacheFiles.containsKey(mappedPath) ||
+            fs.exists(new Path(mappedPath))) {
+          continue;
+        }
+        distCacheFiles.put(mappedPath, Long.valueOf(fileSizes[i]));
+      }
+    }
+  }
+
+  /**
+   * Check if the file path provided was constructed by MapReduce for a
+   * distributed cache file on local file system.
+   * @param filePath path of the distributed cache file
+   * @param user job submitter of the job for which &lt;filePath&gt; is a
+   *             distributed cache file
+   * @param visibility <code>true</code> for public distributed cache file
+   * @return true if the path provided is of a local file system based
+   *              distributed cache file
+   */
+  private boolean isLocalDistCacheFile(String filePath, String user,
+                                       boolean visibility) {
+    return (!visibility && filePath.contains(user + "/.staging"));
+  }
+
+  /**
+   * Map the HDFS based distributed cache file path from original cluster to
+   * a unique file name on the simulated cluster.
+   * <br> Unique  distributed file names on simulated cluster are generated
+   * using original cluster's <li>file path, <li>timestamp and <li> the
+   * job-submitter for private distributed cache file.
+   * <br> This implies that if on original cluster, a single HDFS file
+   * considered as two private distributed cache files for two jobs of
+   * different users, then the corresponding simulated jobs will have two
+   * different files of the same size in public distributed cache, one for each
+   * user. Both these simulated jobs will not share these distributed cache
+   * files, thus leading to the same load as seen in the original cluster.
+   * @param file distributed cache file path
+   * @param timeStamp time stamp of dist cachce file
+   * @param isPublic true if this distributed cache file is a public
+   *                 distributed cache file
+   * @param user job submitter on original cluster
+   * @return the mapped path on simulated cluster
+   */
+  private String mapDistCacheFilePath(String file, String timeStamp,
+      boolean isPublic, String user) {
+    String id = file + timeStamp;
+    if (!isPublic) {
+      // consider job-submitter for private distributed cache file
+      id = id.concat(user);
+    }
+    return new Path(distCachePath, MD5Hash.digest(id).toString()).toUri()
+               .getPath();
+  }
+
+  /**
+   * Write the list of distributed cache files in the decreasing order of
+   * file sizes into the sequence file. This file will be input to the job
+   * {@link GenerateDistCacheData}.
+   * Also validates if -generate option is missing and distributed cache files
+   * are missing.
+   * @return exit code
+   * @throws IOException
+   */
+  private int writeDistCacheFilesList()
+      throws IOException {
+    // Sort the distributed cache files in the decreasing order of file sizes.
+    List dcFiles = new ArrayList(distCacheFiles.entrySet());
+    Collections.sort(dcFiles, new Comparator() {
+      public int compare(Object dc1, Object dc2) {
+        return ((Comparable) ((Map.Entry) (dc2)).getValue())
+            .compareTo(((Map.Entry) (dc1)).getValue());
+      }
+    });
+
+    // write the sorted distributed cache files to the sequence file
+    FileSystem fs = FileSystem.get(conf);
+    Path distCacheFilesList = new Path(distCachePath, "_distCacheFiles.txt");
+    conf.set(GenerateDistCacheData.GRIDMIX_DISTCACHE_FILE_LIST,
+        distCacheFilesList.toString());
+    SequenceFile.Writer src_writer = SequenceFile.createWriter(fs, conf,
+        distCacheFilesList, LongWritable.class, BytesWritable.class,
+        SequenceFile.CompressionType.NONE);
+
+    // Total number of unique distributed cache files
+    int fileCount = dcFiles.size();
+    long byteCount = 0;// Total size of all distributed cache files
+    long bytesSync = 0;// Bytes after previous sync;used to add sync marker
+
+    for (Iterator it = dcFiles.iterator(); it.hasNext();) {
+      Map.Entry entry = (Map.Entry)it.next();
+      LongWritable fileSize =
+          new LongWritable(Long.valueOf(entry.getValue().toString()));
+      BytesWritable filePath =
+          new BytesWritable(entry.getKey().toString().getBytes());
+
+      byteCount += fileSize.get();
+      bytesSync += fileSize.get();
+      if (bytesSync > AVG_BYTES_PER_MAP) {
+        src_writer.sync();
+        bytesSync = fileSize.get();
+      }
+      src_writer.append(fileSize, filePath);
+    }
+    if (src_writer != null) {
+      src_writer.close();
+    }
+    // Set delete on exit for 'dist cache files list' as it is not needed later.
+    fs.deleteOnExit(distCacheFilesList);
+
+    conf.setInt(GenerateDistCacheData.GRIDMIX_DISTCACHE_FILE_COUNT, fileCount);
+    conf.setLong(GenerateDistCacheData.GRIDMIX_DISTCACHE_BYTE_COUNT, byteCount);
+    LOG.info("Number of HDFS based distributed cache files to be generated is "
+        + fileCount + ". Total size of HDFS based distributed cache files "
+        + "to be generated is " + byteCount);
+
+    if (!shouldGenerateDistCacheData() && fileCount > 0) {
+      LOG.error("Missing " + fileCount + " distributed cache files under the "
+          + " directory\n" + distCachePath + "\nthat are needed for gridmix"
+          + " to emulate distributed cache load. Either use -generate\noption"
+          + " to generate distributed cache data along with input data OR "
+          + "disable\ndistributed cache emulation by configuring '"
+          + DistributedCacheEmulator.GRIDMIX_EMULATE_DISTRIBUTEDCACHE
+          + "' to false.");
+      return MISSING_DIST_CACHE_FILES_ERROR;
+    }
+    return 0;
+  }
+
+  /**
+   * If gridmix needs to emulate distributed cache load, then configure
+   * distributed cache files of a simulated job by mapping the original
+   * cluster's distributed cache file paths to the simulated cluster's paths and
+   * setting these mapped paths in the job configuration of the simulated job.
+   * <br>
+   * Configure local FS based distributed cache files through the property
+   * "tmpfiles" and hdfs based distributed cache files through the property
+   * {@link DistributedCache#CACHE_FILES}.
+   * @param conf configuration for the simulated job to be run
+   * @param jobConf job configuration of original cluster's job, obtained from
+   *                trace
+   * @throws IOException
+   */
+  void configureDistCacheFiles(Configuration conf, JobConf jobConf)
+      throws IOException {
+    if (shouldEmulateDistCacheLoad()) {
+
+      String[] files = jobConf.getStrings(DistributedCache.CACHE_FILES);
+      if (files != null) {
+        // hdfs based distributed cache files to be configured for simulated job
+        List<String> cacheFiles = new ArrayList<String>();
+        // local FS based distributed cache files to be configured for
+        // simulated job
+        List<String> localCacheFiles = new ArrayList<String>();
+
+        String[] visibilities =
+          jobConf.getStrings(JobContext.CACHE_FILE_VISIBILITIES);
+        String[] timeStamps =
+          jobConf.getStrings(DistributedCache.CACHE_FILES_TIMESTAMPS);
+        String[] fileSizes =
+          jobConf.getStrings(DistributedCache.CACHE_FILES_SIZES);
+
+        String user = jobConf.getUser();
+        for (int i = 0; i < files.length; i++) {
+          // Check if visibilities are available because older hadoop versions
+          // didn't have public, private Distributed Caches separately.
+          boolean visibility =
+            (visibilities == null) ? true : Boolean.valueOf(visibilities[i]);
+          if (isLocalDistCacheFile(files[i], user, visibility)) {
+            // local FS based distributed cache file.
+            // Create this file on the pseudo local FS.
+            String fileId = MD5Hash.digest(files[i] + timeStamps[i]).toString();
+            long fileSize = Long.valueOf(fileSizes[i]);
+            Path mappedLocalFilePath =
+                PseudoLocalFs.generateFilePath(fileId, fileSize)
+                    .makeQualified(pseudoLocalFs.getUri(),
+                                   pseudoLocalFs.getWorkingDirectory());
+            pseudoLocalFs.create(mappedLocalFilePath);
+            localCacheFiles.add(mappedLocalFilePath.toUri().toString());
+          } else {
+            // hdfs based distributed cache file.
+            // Get the mapped HDFS path on simulated cluster
+            String mappedPath = mapDistCacheFilePath(files[i], timeStamps[i],
+                                                     visibility, user);
+            cacheFiles.add(mappedPath);
+          }
+        }
+        if (cacheFiles.size() > 0) {
+          // configure hdfs based distributed cache files for simulated job
+          conf.setStrings(DistributedCache.CACHE_FILES,
+                          cacheFiles.toArray(new String[cacheFiles.size()]));
+        }
+        if (localCacheFiles.size() > 0) {
+          // configure local FS based distributed cache files for simulated job
+          conf.setStrings("tmpfiles", localCacheFiles.toArray(
+                                        new String[localCacheFiles.size()]));
+        }
+      }
+    }
+  }
+}

+ 10 - 6
src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/EchoUserResolver.java

@@ -19,15 +19,9 @@ package org.apache.hadoop.mapred.gridmix;
 
 
 import java.io.IOException;
 import java.io.IOException;
 import java.net.URI;
 import java.net.URI;
-import java.util.Collections;
-import java.util.List;
-import java.util.ArrayList;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.ShellBasedUnixGroupsMapping;
-import org.apache.hadoop.security.Groups;
-import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 
 
@@ -50,4 +44,14 @@ public class EchoUserResolver implements UserResolver {
       UserGroupInformation ugi) {
       UserGroupInformation ugi) {
     return ugi;
     return ugi;
   }
   }
+
+  /**
+   * {@inheritDoc}
+   * <br><br>
+   * Since {@link EchoUserResolver} simply returns the user's name passed as
+   * the argument, it doesn't need a target list of users.
+   */
+  public boolean needsTargetUsersList() {
+    return false;
+  }
 }
 }

+ 307 - 0
src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/ExecutionSummarizer.java

@@ -0,0 +1,307 @@
+/**
+ * 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.lang.time.FastDateFormat;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.MD5Hash;
+import org.apache.hadoop.mapred.gridmix.GenerateData.DataStatistics;
+import org.apache.hadoop.mapred.gridmix.Statistics.JobStats;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.util.StringUtils;
+
+/**
+ * Summarizes a {@link Gridmix} run. Statistics that are reported are
+ * <ul>
+ *   <li>Total number of jobs in the input trace</li>
+ *   <li>Trace signature</li>
+ *   <li>Total number of jobs processed from the input trace</li>
+ *   <li>Total number of jobs submitted</li>
+ *   <li>Total number of successful and failed jobs</li>
+ *   <li>Total number of map/reduce tasks launched</li>
+ *   <li>Gridmix start & end time</li>
+ *   <li>Total time for the Gridmix run (data-generation and simulation)</li>
+ *   <li>Gridmix Configuration (i.e job-type, submission-type, resolver)</li>
+ * </ul>
+ */
+class ExecutionSummarizer implements StatListener<JobStats> {
+  static final Log LOG = LogFactory.getLog(ExecutionSummarizer.class);
+  private static final FastDateFormat UTIL = FastDateFormat.getInstance();
+  
+  private int numJobsInInputTrace;
+  private int totalSuccessfulJobs;
+  private int totalFailedJobs;
+  private int totalMapTasksLaunched;
+  private int totalReduceTasksLaunched;
+  private long totalSimulationTime;
+  private long totalRuntime;
+  private final String commandLineArgs;
+  private long startTime;
+  private long endTime;
+  private long simulationStartTime;
+  private String inputTraceLocation;
+  private String inputTraceSignature;
+  private String jobSubmissionPolicy;
+  private String resolver;
+  private DataStatistics dataStats;
+  private String expectedDataSize;
+  
+  /**
+   * Basic constructor initialized with the runtime arguments. 
+   */
+  ExecutionSummarizer(String[] args) {
+    startTime = System.currentTimeMillis();
+    // flatten the args string and store it
+    commandLineArgs = 
+      org.apache.commons.lang.StringUtils.join(args, ' '); 
+  }
+  
+  /**
+   * Default constructor. 
+   */
+  ExecutionSummarizer() {
+    startTime = System.currentTimeMillis();
+    commandLineArgs = Summarizer.NA; 
+  }
+  
+  void start(Configuration conf) {
+    simulationStartTime = System.currentTimeMillis();
+  }
+  
+  private void processJobState(JobStats stats) throws Exception {
+    Job job = stats.getJob();
+    if (job.isSuccessful()) {
+      ++totalSuccessfulJobs;
+    } else {
+      ++totalFailedJobs;
+    }
+  }
+  
+  private void processJobTasks(JobStats stats) throws Exception {
+    totalMapTasksLaunched += stats.getNoOfMaps();
+    Job job = stats.getJob();
+    totalReduceTasksLaunched += job.getNumReduceTasks();
+  }
+  
+  private void process(JobStats stats) {
+    try {
+      // process the job run state
+      processJobState(stats);
+      
+      // process the tasks information
+      processJobTasks(stats);
+    } catch (Exception e) {
+      LOG.info("Error in processing job " + stats.getJob().getJobID() + ".");
+    }
+  }
+  
+  @Override
+  public void update(JobStats item) {
+    // process only if the simulation has started
+    if (simulationStartTime > 0) {
+      process(item);
+      totalSimulationTime = 
+        System.currentTimeMillis() - getSimulationStartTime();
+    }
+  }
+  
+  // Generates a signature for the trace file based on
+  //   - filename
+  //   - modification time
+  //   - file length
+  //   - owner
+  protected static String getTraceSignature(String input) throws IOException {
+    Path inputPath = new Path(input);
+    FileSystem fs = inputPath.getFileSystem(new Configuration());
+    FileStatus status = fs.getFileStatus(inputPath);
+    Path qPath = fs.makeQualified(status.getPath());
+    String traceID = status.getModificationTime() + qPath.toString()
+                     + status.getOwner() + status.getLen();
+    return MD5Hash.digest(traceID).toString();
+  }
+  
+  @SuppressWarnings("unchecked")
+  void finalize(JobFactory factory, String inputPath, long dataSize, 
+                UserResolver userResolver, DataStatistics stats,
+                Configuration conf) 
+  throws IOException {
+    numJobsInInputTrace = factory.numJobsInTrace;
+    endTime = System.currentTimeMillis();
+    Path inputTracePath = new Path(inputPath);
+    FileSystem fs = inputTracePath.getFileSystem(conf);
+    inputTraceLocation = fs.makeQualified(inputTracePath).toString();
+    inputTraceSignature = getTraceSignature(inputTraceLocation);
+    jobSubmissionPolicy = Gridmix.getJobSubmissionPolicy(conf).name();
+    resolver = userResolver.getClass().getName();
+    if (dataSize > 0) {
+      expectedDataSize = StringUtils.humanReadableInt(dataSize);
+    } else {
+      expectedDataSize = Summarizer.NA;
+    }
+    dataStats = stats;
+    totalRuntime = System.currentTimeMillis() - getStartTime();
+  }
+  
+  /**
+   * Summarizes the current {@link Gridmix} run.
+   */
+  @Override
+  public String toString() {
+    StringBuilder builder = new StringBuilder();
+    builder.append("Execution Summary:-");
+    builder.append("\nInput trace: ").append(getInputTraceLocation());
+    builder.append("\nInput trace signature: ")
+           .append(getInputTraceSignature());
+    builder.append("\nTotal number of jobs in trace: ")
+           .append(getNumJobsInTrace());
+    builder.append("\nExpected input data size: ")
+           .append(getExpectedDataSize());
+    builder.append("\nInput data statistics: ")
+           .append(getInputDataStatistics());
+    builder.append("\nTotal number of jobs processed: ")
+           .append(getNumSubmittedJobs());
+    builder.append("\nTotal number of successful jobs: ")
+           .append(getNumSuccessfulJobs());
+    builder.append("\nTotal number of failed jobs: ")
+           .append(getNumFailedJobs());
+    builder.append("\nTotal number of map tasks launched: ")
+           .append(getNumMapTasksLaunched());
+    builder.append("\nTotal number of reduce task launched: ")
+           .append(getNumReduceTasksLaunched());
+    builder.append("\nGridmix start time: ")
+           .append(UTIL.format(getStartTime()));
+    builder.append("\nGridmix end time: ").append(UTIL.format(getEndTime()));
+    builder.append("\nGridmix simulation start time: ")
+           .append(UTIL.format(getStartTime()));
+    builder.append("\nGridmix runtime: ")
+           .append(StringUtils.formatTime(getRuntime()));
+    builder.append("\nTime spent in initialization (data-gen etc): ")
+           .append(StringUtils.formatTime(getInitTime()));
+    builder.append("\nTime spent in simulation: ")
+           .append(StringUtils.formatTime(getSimulationTime()));
+    builder.append("\nGridmix configuration parameters: ")
+           .append(getCommandLineArgsString());
+    builder.append("\nGridmix job submission policy: ")
+           .append(getJobSubmissionPolicy());
+    builder.append("\nGridmix resolver: ").append(getUserResolver());
+    builder.append("\n\n");
+    return builder.toString();
+  }
+  
+  // Gets the stringified version of DataStatistics
+  static String stringifyDataStatistics(DataStatistics stats) {
+    if (stats != null) {
+      StringBuffer buffer = new StringBuffer();
+      String compressionStatus = stats.isDataCompressed() 
+                                 ? "Compressed" 
+                                 : "Uncompressed";
+      buffer.append(compressionStatus).append(" input data size: ");
+      buffer.append(StringUtils.humanReadableInt(stats.getDataSize()));
+      buffer.append(", ");
+      buffer.append("Number of files: ").append(stats.getNumFiles());
+
+      return buffer.toString();
+    } else {
+      return Summarizer.NA;
+    }
+  }
+  
+  // Getters
+  protected String getExpectedDataSize() {
+    return expectedDataSize;
+  }
+  
+  protected String getUserResolver() {
+    return resolver;
+  }
+  
+  protected String getInputDataStatistics() {
+    return stringifyDataStatistics(dataStats);
+  }
+  
+  protected String getInputTraceSignature() {
+    return inputTraceSignature;
+  }
+  
+  protected String getInputTraceLocation() {
+    return inputTraceLocation;
+  }
+  
+  protected int getNumJobsInTrace() {
+    return numJobsInInputTrace;
+  }
+  
+  protected int getNumSuccessfulJobs() {
+    return totalSuccessfulJobs;
+  }
+  
+  protected int getNumFailedJobs() {
+    return totalFailedJobs;
+  }
+  
+  protected int getNumSubmittedJobs() {
+    return totalSuccessfulJobs + totalFailedJobs;
+  }
+  
+  protected int getNumMapTasksLaunched() {
+    return totalMapTasksLaunched;
+  }
+  
+  protected int getNumReduceTasksLaunched() {
+    return totalReduceTasksLaunched;
+  }
+  
+  protected long getStartTime() {
+    return startTime;
+  }
+  
+  protected long getEndTime() {
+    return endTime;
+  }
+  
+  protected long getInitTime() {
+    return simulationStartTime - startTime;
+  }
+  
+  protected long getSimulationStartTime() {
+    return simulationStartTime;
+  }
+  
+  protected long getSimulationTime() {
+    return totalSimulationTime;
+  }
+  
+  protected long getRuntime() {
+    return totalRuntime;
+  }
+  
+  protected String getCommandLineArgsString() {
+    return commandLineArgs;
+  }
+  
+  protected String getJobSubmissionPolicy() {
+    return jobSubmissionPolicy;
+  }
+}

+ 4 - 6
src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/FileQueue.java

@@ -21,8 +21,6 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStream;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.IOUtils;
 
 
@@ -34,7 +32,7 @@ class FileQueue extends InputStream {
 
 
   private int idx = -1;
   private int idx = -1;
   private long curlen = -1L;
   private long curlen = -1L;
-  private FSDataInputStream input;
+  private InputStream input;
   private final byte[] z = new byte[1];
   private final byte[] z = new byte[1];
   private final Path[] paths;
   private final Path[] paths;
   private final long[] lengths;
   private final long[] lengths;
@@ -64,9 +62,9 @@ class FileQueue extends InputStream {
     idx = (idx + 1) % paths.length;
     idx = (idx + 1) % paths.length;
     curlen = lengths[idx];
     curlen = lengths[idx];
     final Path file = paths[idx];
     final Path file = paths[idx];
-    final FileSystem fs = file.getFileSystem(conf);
-    input = fs.open(file);
-    input.seek(startoffset[idx]);
+    input = 
+      CompressionEmulationUtil.getPossiblyDecompressedInputStream(file, 
+                                 conf, startoffset[idx]);
   }
   }
 
 
   @Override
   @Override

+ 111 - 3
src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/GenerateData.java

@@ -30,8 +30,10 @@ import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 import java.util.regex.Pattern;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.LongWritable;
@@ -41,6 +43,7 @@ import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.mapred.ClusterStatus;
 import org.apache.hadoop.mapred.ClusterStatus;
 import org.apache.hadoop.mapred.JobClient;
 import org.apache.hadoop.mapred.JobClient;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.Utils;
 import org.apache.hadoop.mapreduce.InputFormat;
 import org.apache.hadoop.mapreduce.InputFormat;
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.Job;
@@ -52,6 +55,7 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
 import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.StringUtils;
 
 
 // TODO can replace with form of GridmixJob
 // TODO can replace with form of GridmixJob
 class GenerateData extends GridmixJob {
 class GenerateData extends GridmixJob {
@@ -86,14 +90,103 @@ class GenerateData extends GridmixJob {
    * Replication of generated data.
    * Replication of generated data.
    */
    */
   public static final String GRIDMIX_GEN_REPLICATION = "gridmix.gen.replicas";
   public static final String GRIDMIX_GEN_REPLICATION = "gridmix.gen.replicas";
+  static final String JOB_NAME = "GRIDMIX_GENERATE_INPUT_DATA";
 
 
   public GenerateData(Configuration conf, Path outdir, long genbytes)
   public GenerateData(Configuration conf, Path outdir, long genbytes)
       throws IOException {
       throws IOException {
-    super(conf, 0L, "GRIDMIX_GENDATA");
+    super(conf, 0L, JOB_NAME);
     job.getConfiguration().setLong(GRIDMIX_GEN_BYTES, genbytes);
     job.getConfiguration().setLong(GRIDMIX_GEN_BYTES, genbytes);
     FileOutputFormat.setOutputPath(job, outdir);
     FileOutputFormat.setOutputPath(job, outdir);
   }
   }
 
 
+  /**
+   * Represents the input data characteristics.
+   */
+  static class DataStatistics {
+    private long dataSize;
+    private long numFiles;
+    private boolean isDataCompressed;
+    
+    DataStatistics(long dataSize, long numFiles, boolean isCompressed) {
+      this.dataSize = dataSize;
+      this.numFiles = numFiles;
+      this.isDataCompressed = isCompressed;
+    }
+    
+    long getDataSize() {
+      return dataSize;
+    }
+    
+    long getNumFiles() {
+      return numFiles;
+    }
+    
+    boolean isDataCompressed() {
+      return isDataCompressed;
+    }
+  }
+  
+  /**
+   * Publish the data statistics.
+   */
+  static DataStatistics publishDataStatistics(Path inputDir, long genBytes, 
+                                              Configuration conf) 
+  throws IOException {
+    if (CompressionEmulationUtil.isCompressionEmulationEnabled(conf)) {
+      return CompressionEmulationUtil.publishCompressedDataStatistics(inputDir, 
+                                        conf, genBytes);
+    } else {
+      return publishPlainDataStatistics(conf, inputDir);
+    }
+  }
+
+  /**
+   * List files recursively and get their statuses.
+   * @param path The path of the file/dir for which ls is to be done recursively
+   * @param fs FileSystem of the path
+   * @param filter the user-supplied path filter
+   * @return
+   */
+  private static List<FileStatus> listFiles(Path path, FileSystem fs,
+      PathFilter filter) throws IOException {
+    List<FileStatus> list = new ArrayList<FileStatus>();
+    FileStatus[] statuses = fs.listStatus(path, filter);
+    if (statuses != null) {
+      for (FileStatus status : statuses) {
+        if (status.isDir()) {
+          list.addAll(listFiles(status.getPath(), fs, filter));
+        } else {
+          list.add(status);
+        }
+      }
+    }
+    return list;
+  }
+
+  static DataStatistics publishPlainDataStatistics(Configuration conf, 
+                                                   Path inputDir) 
+  throws IOException {
+    FileSystem fs = inputDir.getFileSystem(conf);
+
+    // obtain input data file statuses
+    long dataSize = 0;
+    long fileCount = 0;
+    PathFilter filter = new Utils.OutputFileUtils.OutputFilesFilter();
+    List<FileStatus> statuses = listFiles(inputDir, fs, filter);
+
+    for (FileStatus fStat : statuses) {
+      dataSize += fStat.getLen();
+    }
+    fileCount = statuses.size();
+
+    // publish the plain data statistics
+    LOG.info("Total size of input data : " 
+             + StringUtils.humanReadableInt(dataSize));
+    LOG.info("Total number of input data files : " + fileCount);
+    
+    return new DataStatistics(dataSize, fileCount, false);
+  }
+  
   @Override
   @Override
   public Job call() throws IOException, InterruptedException,
   public Job call() throws IOException, InterruptedException,
                            ClassNotFoundException {
                            ClassNotFoundException {
@@ -101,6 +194,18 @@ class GenerateData extends GridmixJob {
     ugi.doAs( new PrivilegedExceptionAction <Job>() {
     ugi.doAs( new PrivilegedExceptionAction <Job>() {
        public Job run() throws IOException, ClassNotFoundException,
        public Job run() throws IOException, ClassNotFoundException,
                                InterruptedException {
                                InterruptedException {
+         // check if compression emulation is enabled
+         if (CompressionEmulationUtil
+             .isCompressionEmulationEnabled(job.getConfiguration())) {
+           CompressionEmulationUtil.configure(job);
+         } else {
+           configureRandomBytesDataGenerator();
+         }
+         job.submit();
+         return job;
+       }
+       
+       private void configureRandomBytesDataGenerator() {
         job.setMapperClass(GenDataMapper.class);
         job.setMapperClass(GenDataMapper.class);
         job.setNumReduceTasks(0);
         job.setNumReduceTasks(0);
         job.setMapOutputKeyClass(NullWritable.class);
         job.setMapOutputKeyClass(NullWritable.class);
@@ -113,12 +218,15 @@ class GenerateData extends GridmixJob {
         } catch (IOException e) {
         } catch (IOException e) {
           LOG.error("Error  while adding input path ", e);
           LOG.error("Error  while adding input path ", e);
         }
         }
-        job.submit();
-        return job;
       }
       }
     });
     });
     return job;
     return job;
   }
   }
+  
+  @Override
+  protected boolean canEmulateCompression() {
+    return false;
+  }
 
 
   public static class GenDataMapper
   public static class GenDataMapper
       extends Mapper<NullWritable,LongWritable,NullWritable,BytesWritable> {
       extends Mapper<NullWritable,LongWritable,NullWritable,BytesWritable> {

+ 259 - 0
src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/GenerateDistCacheData.java

@@ -0,0 +1,259 @@
+/**
+ * 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 java.security.PrivilegedExceptionAction;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.SequenceFile;
+import org.apache.hadoop.mapred.ClusterStatus;
+import org.apache.hadoop.mapreduce.lib.input.FileSplit;
+import org.apache.hadoop.mapred.JobClient;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapreduce.lib.input.SequenceFileRecordReader;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
+import org.apache.hadoop.security.UserGroupInformation;
+
+/**
+ * GridmixJob that generates distributed cache files.
+ * {@link GenerateDistCacheData} expects a list of distributed cache files to be
+ * generated as input. This list is expected to be stored as a sequence file
+ * and the filename is expected to be configured using
+ * {@code gridmix.distcache.file.list}.
+ * This input file contains the list of distributed cache files and their sizes.
+ * For each record (i.e. file size and file path) in this input file,
+ * a file with the specific file size at the specific path is created.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+class GenerateDistCacheData extends GridmixJob {
+
+  /**
+   * Number of distributed cache files to be created by gridmix
+   */
+  static final String GRIDMIX_DISTCACHE_FILE_COUNT =
+      "gridmix.distcache.file.count";
+  /**
+   * Total number of bytes to be written to the distributed cache files by
+   * gridmix. i.e. Sum of sizes of all unique distributed cache files to be
+   * created by gridmix.
+   */
+  static final String GRIDMIX_DISTCACHE_BYTE_COUNT =
+      "gridmix.distcache.byte.count";
+  /**
+   * The special file created(and used) by gridmix, that contains the list of
+   * unique distributed cache files that are to be created and their sizes.
+   */
+  static final String GRIDMIX_DISTCACHE_FILE_LIST =
+      "gridmix.distcache.file.list";
+  static final String JOB_NAME = "GRIDMIX_GENERATE_DISTCACHE_DATA";
+
+  public GenerateDistCacheData(Configuration conf) throws IOException {
+    super(conf, 0L, JOB_NAME);
+  }
+
+  @Override
+  public Job call() throws IOException, InterruptedException,
+                           ClassNotFoundException {
+    UserGroupInformation ugi = UserGroupInformation.getLoginUser();
+    ugi.doAs( new PrivilegedExceptionAction <Job>() {
+       public Job run() throws IOException, ClassNotFoundException,
+                               InterruptedException {
+        job.setMapperClass(GenDCDataMapper.class);
+        job.setNumReduceTasks(0);
+        job.setMapOutputKeyClass(NullWritable.class);
+        job.setMapOutputValueClass(BytesWritable.class);
+        job.setInputFormatClass(GenDCDataFormat.class);
+        job.setOutputFormatClass(NullOutputFormat.class);
+        job.setJarByClass(GenerateDistCacheData.class);
+        try {
+          FileInputFormat.addInputPath(job, new Path("ignored"));
+        } catch (IOException e) {
+          LOG.error("Error while adding input path ", e);
+        }
+        job.submit();
+        return job;
+      }
+    });
+    return job;
+  }
+
+  @Override
+  protected boolean canEmulateCompression() {
+    return false;
+  }
+
+  public static class GenDCDataMapper
+      extends Mapper<LongWritable, BytesWritable, NullWritable, BytesWritable> {
+
+    private BytesWritable val;
+    private final Random r = new Random();
+    private FileSystem fs;
+
+    @Override
+    protected void setup(Context context)
+        throws IOException, InterruptedException {
+      val = new BytesWritable(new byte[context.getConfiguration().getInt(
+              GenerateData.GRIDMIX_VAL_BYTES, 1024 * 1024)]);
+      fs = FileSystem.get(context.getConfiguration());
+    }
+
+    // Create one distributed cache file with the needed file size.
+    // key is distributed cache file size and
+    // value is distributed cache file path.
+    @Override
+    public void map(LongWritable key, BytesWritable value, Context context)
+        throws IOException, InterruptedException {
+
+      String fileName = new String(value.getBytes(), 0, value.getLength());
+      Path path = new Path(fileName);
+
+      /**
+       * Create distributed cache file with the permissions 0755.
+       * Since the private distributed cache directory doesn't have execute
+       * permission for others, it is OK to set read permission for others for
+       * the files under that directory and still they will become 'private'
+       * distributed cache files on the simulated cluster.
+       */
+      FSDataOutputStream dos =
+          FileSystem.create(fs, path, new FsPermission((short)0755));
+
+      for (long bytes = key.get(); bytes > 0; bytes -= val.getLength()) {
+        r.nextBytes(val.getBytes());
+        val.setSize((int)Math.min(val.getLength(), bytes));
+        dos.write(val.getBytes(), 0, val.getLength());// Write to distCache file
+      }
+      dos.close();
+    }
+  }
+
+  /**
+   * InputFormat for GenerateDistCacheData.
+   * Input to GenerateDistCacheData is the special file(in SequenceFile format)
+   * that contains the list of distributed cache files to be generated along
+   * with their file sizes.
+   */
+  static class GenDCDataFormat
+      extends InputFormat<LongWritable, BytesWritable> {
+
+    // Split the special file that contains the list of distributed cache file
+    // paths and their file sizes such that each split corresponds to
+    // approximately same amount of distributed cache data to be generated.
+    // Consider numTaskTrackers * numMapSlotsPerTracker as the number of maps
+    // for this job, if there is lot of data to be generated.
+    @Override
+    public List<InputSplit> getSplits(JobContext jobCtxt) throws IOException {
+      final JobConf jobConf = new JobConf(jobCtxt.getConfiguration());
+      final JobClient client = new JobClient(jobConf);
+      ClusterStatus stat = client.getClusterStatus(true);
+      int numTrackers = stat.getTaskTrackers();
+      final int fileCount = jobConf.getInt(GRIDMIX_DISTCACHE_FILE_COUNT, -1);
+
+      // Total size of distributed cache files to be generated
+      final long totalSize = jobConf.getLong(GRIDMIX_DISTCACHE_BYTE_COUNT, -1);
+      // Get the path of the special file
+      String distCacheFileList = jobConf.get(GRIDMIX_DISTCACHE_FILE_LIST);
+      if (fileCount < 0 || totalSize < 0 || distCacheFileList == null) {
+        throw new RuntimeException("Invalid metadata: #files (" + fileCount
+            + "), total_size (" + totalSize + "), filelisturi ("
+            + distCacheFileList + ")");
+      }
+
+      Path sequenceFile = new Path(distCacheFileList);
+      FileSystem fs = sequenceFile.getFileSystem(jobConf);
+      FileStatus srcst = fs.getFileStatus(sequenceFile);
+      // Consider the number of TTs * mapSlotsPerTracker as number of mappers.
+      int numMapSlotsPerTracker =
+          jobConf.getInt("mapred.tasktracker.map.tasks.maximum", 2);
+      int numSplits = numTrackers * numMapSlotsPerTracker;
+
+      List<InputSplit> splits = new ArrayList<InputSplit>(numSplits);
+      LongWritable key = new LongWritable();
+      BytesWritable value = new BytesWritable();
+
+      // Average size of data to be generated by each map task
+      final long targetSize = Math.max(totalSize / numSplits,
+                                DistributedCacheEmulator.AVG_BYTES_PER_MAP);
+      long splitStartPosition = 0L;
+      long splitEndPosition = 0L;
+      long acc = 0L;
+      long bytesRemaining = srcst.getLen();
+      SequenceFile.Reader reader = null;
+      try {
+        reader = new SequenceFile.Reader(fs, sequenceFile, jobConf);
+        while (reader.next(key, value)) {
+
+          // If adding this file would put this split past the target size,
+          // cut the last split and put this file in the next split.
+          if (acc + key.get() > targetSize && acc != 0) {
+            long splitSize = splitEndPosition - splitStartPosition;
+            splits.add(new FileSplit(
+                sequenceFile, splitStartPosition, splitSize, (String[])null));
+            bytesRemaining -= splitSize;
+            splitStartPosition = splitEndPosition;
+            acc = 0L;
+          }
+          acc += key.get();
+          splitEndPosition = reader.getPosition();
+        }
+      } finally {
+        if (reader != null) {
+          reader.close();
+        }
+      }
+      if (bytesRemaining != 0) {
+        splits.add(new FileSplit(
+            sequenceFile, splitStartPosition, bytesRemaining, (String[])null));
+      }
+
+      return splits;
+    }
+
+    /**
+     * Returns a reader for this split of the distributed cache file list.
+     */
+    @Override
+    public RecordReader<LongWritable, BytesWritable> createRecordReader(
+        InputSplit split, final TaskAttemptContext taskContext)
+        throws IOException, InterruptedException {
+      return new SequenceFileRecordReader<LongWritable, BytesWritable>();
+    }
+  }
+}

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

@@ -33,12 +33,14 @@ import org.apache.hadoop.fs.FsShell;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.mapred.gridmix.GenerateData.DataStatistics;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.hadoop.util.ToolRunner;
+import org.apache.hadoop.tools.rumen.JobStoryProducer;
 import org.apache.hadoop.tools.rumen.ZombieJobProducer;
 import org.apache.hadoop.tools.rumen.ZombieJobProducer;
 
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
@@ -92,62 +94,143 @@ public class Gridmix extends Configured implements Tool {
    */
    */
   public static final String GRIDMIX_USR_RSV = "gridmix.user.resolve.class";
   public static final String GRIDMIX_USR_RSV = "gridmix.user.resolve.class";
 
 
+  /**
+   * Configuration property set in simulated job's configuration whose value is
+   * set to the corresponding original job's name. This is not configurable by
+   * gridmix user.
+   */
+  public static final String ORIGINAL_JOB_NAME =
+      "gridmix.job.original-job-name";
+  /**
+   * Configuration property set in simulated job's configuration whose value is
+   * set to the corresponding original job's id. This is not configurable by
+   * gridmix user.
+   */
+  public static final String ORIGINAL_JOB_ID = "gridmix.job.original-job-id";
+
+  private DistributedCacheEmulator distCacheEmulator;
+
   // Submit data structures
   // Submit data structures
   private JobFactory factory;
   private JobFactory factory;
   private JobSubmitter submitter;
   private JobSubmitter submitter;
   private JobMonitor monitor;
   private JobMonitor monitor;
   private Statistics statistics;
   private Statistics statistics;
+  private Summarizer summarizer;
 
 
   // Shutdown hook
   // Shutdown hook
   private final Shutdown sdh = new Shutdown();
   private final Shutdown sdh = new Shutdown();
 
 
+  Gridmix(String[] args) {
+    summarizer = new Summarizer(args);
+  }
+  
+  Gridmix() {
+    summarizer = new Summarizer();
+  }
+  
+  // Get the input data directory for Gridmix. Input directory is 
+  // <io-path>/input
+  static Path getGridmixInputDataPath(Path ioPath) {
+    return new Path(ioPath, "input");
+  }
+  
   /**
   /**
-   * Write random bytes at the path provided.
+   * Write random bytes at the path &lt;inputDir&gt;.
    * @see org.apache.hadoop.mapred.gridmix.GenerateData
    * @see org.apache.hadoop.mapred.gridmix.GenerateData
    */
    */
-  protected void writeInputData(long genbytes, Path ioPath)
+  protected void writeInputData(long genbytes, Path inputDir)
       throws IOException, InterruptedException {
       throws IOException, InterruptedException {
     final Configuration conf = getConf();
     final Configuration conf = getConf();
-    final GridmixJob genData = new GenerateData(conf, ioPath, genbytes);
-    submitter.add(genData);
+    
+    // configure the compression ratio if needed
+    CompressionEmulationUtil.setupDataGeneratorConfig(conf);
+    
+    final GenerateData genData = new GenerateData(conf, inputDir, genbytes);
     LOG.info("Generating " + StringUtils.humanReadableInt(genbytes) +
     LOG.info("Generating " + StringUtils.humanReadableInt(genbytes) +
         " of test data...");
         " of test data...");
+    launchGridmixJob(genData);
+    
+    FsShell shell = new FsShell(conf);
+    try {
+      LOG.info("Changing the permissions for inputPath " + inputDir.toString());
+      shell.run(new String[] {"-chmod","-R","777", inputDir.toString()});
+    } catch (Exception e) {
+      LOG.error("Couldnt change the file permissions " , e);
+      throw new IOException(e);
+    }
+    
+    LOG.info("Input data generation successful.");
+  }
+
+  /**
+   * Write random bytes in the distributed cache files that will be used by all
+   * simulated jobs of current gridmix run, if files are to be generated.
+   * Do this as part of the MapReduce job {@link GenerateDistCacheData#JOB_NAME}
+   * @see org.apache.hadoop.mapred.gridmix.GenerateDistCacheData
+   */
+  protected void writeDistCacheData(Configuration conf)
+      throws IOException, InterruptedException {
+    int fileCount =
+        conf.getInt(GenerateDistCacheData.GRIDMIX_DISTCACHE_FILE_COUNT, -1);
+    if (fileCount > 0) {// generate distributed cache files
+      final GridmixJob genDistCacheData = new GenerateDistCacheData(conf);
+      LOG.info("Generating distributed cache data of size " + conf.getLong(
+          GenerateDistCacheData.GRIDMIX_DISTCACHE_BYTE_COUNT, -1));
+      launchGridmixJob(genDistCacheData);
+    }
+  }
+
+  // Launch Input/DistCache Data Generation job and wait for completion
+  void launchGridmixJob(GridmixJob job)
+      throws IOException, InterruptedException {
+    submitter.add(job);
+
     // TODO add listeners, use for job dependencies
     // TODO add listeners, use for job dependencies
     TimeUnit.SECONDS.sleep(10);
     TimeUnit.SECONDS.sleep(10);
     try {
     try {
-      genData.getJob().waitForCompletion(false);
+      job.getJob().waitForCompletion(false);
     } catch (ClassNotFoundException e) {
     } catch (ClassNotFoundException e) {
       throw new IOException("Internal error", e);
       throw new IOException("Internal error", e);
     }
     }
-    if (!genData.getJob().isSuccessful()) {
-      throw new IOException("Data generation failed!");
-    }
-
-    FsShell shell = new FsShell(conf);
-    try {
-      LOG.info("Changing the permissions for inputPath " + ioPath.toString());
-      shell.run(new String[] {"-chmod","-R","777", ioPath.toString()});
-    } catch (Exception e) {
-      LOG.error("Couldnt change the file permissions " , e);
-      throw new IOException(e);
+    if (!job.getJob().isSuccessful()) {
+      throw new IOException(job.getJob().getJobName() + " job failed!");
     }
     }
-    LOG.info("Done.");
   }
   }
 
 
-  protected InputStream createInputStream(String in) throws IOException {
-    if ("-".equals(in)) {
-      return System.in;
+  /**
+   * Create an appropriate {@code JobStoryProducer} object for the
+   * given trace.
+   * 
+   * @param traceIn the path to the trace file. The special path
+   * &quot;-&quot; denotes the standard input stream.
+   *
+   * @param conf the configuration to be used.
+   *
+   * @throws IOException if there was an error.
+   */
+  protected JobStoryProducer createJobStoryProducer(String traceIn,
+      Configuration conf) throws IOException {
+    if ("-".equals(traceIn)) {
+      return new ZombieJobProducer(System.in, null);
     }
     }
-    final Path pin = new Path(in);
-    return pin.getFileSystem(getConf()).open(pin);
+    return new ZombieJobProducer(new Path(traceIn), null, conf);
   }
   }
 
 
+  // get the gridmix job submission policy
+  protected static GridmixJobSubmissionPolicy getJobSubmissionPolicy(
+                                                Configuration conf) {
+    return GridmixJobSubmissionPolicy.getPolicy(conf, 
+                                        GridmixJobSubmissionPolicy.STRESS);
+  }
+  
   /**
   /**
    * Create each component in the pipeline and start it.
    * Create each component in the pipeline and start it.
    * @param conf Configuration data, no keys specific to this context
    * @param conf Configuration data, no keys specific to this context
    * @param traceIn Either a Path to the trace data or &quot;-&quot; for
    * @param traceIn Either a Path to the trace data or &quot;-&quot; for
    *                stdin
    *                stdin
-   * @param ioPath Path from which input data is read
+   * @param ioPath &lt;ioPath&gt;/input/ is the dir from which input data is
+   *               read and &lt;ioPath&gt;/distributedCache/ is the gridmix
+   *               distributed cache directory.
    * @param scratchDir Path into which job output is written
    * @param scratchDir Path into which job output is written
    * @param startFlag Semaphore for starting job trace pipeline
    * @param startFlag Semaphore for starting job trace pipeline
    */
    */
@@ -155,8 +238,8 @@ public class Gridmix extends Configured implements Tool {
       Path scratchDir, CountDownLatch startFlag, UserResolver userResolver)
       Path scratchDir, CountDownLatch startFlag, UserResolver userResolver)
       throws IOException {
       throws IOException {
     try {
     try {
-      GridmixJobSubmissionPolicy policy = GridmixJobSubmissionPolicy.getPolicy(
-        conf, GridmixJobSubmissionPolicy.STRESS);
+      Path inputDir = getGridmixInputDataPath(ioPath);
+      GridmixJobSubmissionPolicy policy = getJobSubmissionPolicy(conf);
       LOG.info(" Submission policy is " + policy.name());
       LOG.info(" Submission policy is " + policy.name());
       statistics = new Statistics(conf, policy.getPollingInterval(), startFlag);
       statistics = new Statistics(conf, policy.getPollingInterval(), startFlag);
       monitor = createJobMonitor(statistics);
       monitor = createJobMonitor(statistics);
@@ -167,16 +250,24 @@ public class Gridmix extends Configured implements Tool {
         monitor, conf.getInt(
         monitor, conf.getInt(
           GRIDMIX_SUB_THR, noOfSubmitterThreads), conf.getInt(
           GRIDMIX_SUB_THR, noOfSubmitterThreads), conf.getInt(
           GRIDMIX_QUE_DEP, 5), new FilePool(
           GRIDMIX_QUE_DEP, 5), new FilePool(
-          conf, ioPath), userResolver,statistics);
+          conf, inputDir), userResolver,statistics);
       
       
-      factory = createJobFactory(
-        submitter, traceIn, scratchDir, conf, startFlag, userResolver);
+      distCacheEmulator = new DistributedCacheEmulator(conf, ioPath);
+
+      factory = createJobFactory(submitter, traceIn, scratchDir, conf,
+                                 startFlag, userResolver);
+      factory.jobCreator.setDistCacheEmulator(distCacheEmulator);
+
       if (policy==GridmixJobSubmissionPolicy.SERIAL) {
       if (policy==GridmixJobSubmissionPolicy.SERIAL) {
         statistics.addJobStatsListeners(factory);
         statistics.addJobStatsListeners(factory);
       } else {
       } else {
         statistics.addClusterStatsObservers(factory);
         statistics.addClusterStatsObservers(factory);
       }
       }
-      
+
+      // add the gridmix run summarizer to the statistics
+      statistics.addJobStatsListeners(summarizer.getExecutionSummarizer());
+      statistics.addClusterStatsObservers(summarizer.getClusterSummarizer());
+
       monitor.start();
       monitor.start();
       submitter.start();
       submitter.start();
     }catch(Exception e) {
     }catch(Exception e) {
@@ -201,9 +292,8 @@ public class Gridmix extends Configured implements Tool {
     throws IOException {
     throws IOException {
     return GridmixJobSubmissionPolicy.getPolicy(
     return GridmixJobSubmissionPolicy.getPolicy(
       conf, GridmixJobSubmissionPolicy.STRESS).createJobFactory(
       conf, GridmixJobSubmissionPolicy.STRESS).createJobFactory(
-      submitter, new ZombieJobProducer(
-        createInputStream(
-          traceIn), null), scratchDir, conf, startFlag, resolver);
+      submitter, createJobStoryProducer(traceIn, conf), scratchDir, conf,
+      startFlag, resolver);
   }
   }
 
 
   public int run(final String[] argv) throws IOException, InterruptedException {
   public int run(final String[] argv) throws IOException, InterruptedException {
@@ -217,6 +307,10 @@ public class Gridmix extends Configured implements Tool {
         return runJob(conf,argv);
         return runJob(conf,argv);
       }
       }
     });
     });
+    
+    // print the run summary
+    System.out.print("\n\n");
+    System.out.println(summarizer.toString());
     return val; 
     return val; 
   }
   }
 
 
@@ -232,6 +326,9 @@ public class Gridmix extends Configured implements Tool {
       printUsage(System.err);
       printUsage(System.err);
       return 1;
       return 1;
     }
     }
+    
+    // Should gridmix generate distributed cache data ?
+    boolean generate = false;
     long genbytes = -1L;
     long genbytes = -1L;
     String traceIn = null;
     String traceIn = null;
     Path ioPath = null;
     Path ioPath = null;
@@ -243,6 +340,7 @@ public class Gridmix extends Configured implements Tool {
       for (int i = 0; i < argv.length - 2; ++i) {
       for (int i = 0; i < argv.length - 2; ++i) {
         if ("-generate".equals(argv[i])) {
         if ("-generate".equals(argv[i])) {
           genbytes = StringUtils.TraditionalBinaryPrefix.string2long(argv[++i]);
           genbytes = StringUtils.TraditionalBinaryPrefix.string2long(argv[++i]);
+          generate = true;
         } else if ("-users".equals(argv[i])) {
         } else if ("-users".equals(argv[i])) {
           userRsrc = new URI(argv[++i]);
           userRsrc = new URI(argv[++i]);
         } else {
         } else {
@@ -250,9 +348,22 @@ public class Gridmix extends Configured implements Tool {
           return 1;
           return 1;
         }
         }
       }
       }
-      if (!userResolver.setTargetUsers(userRsrc, conf)) {
-        LOG.warn("Resource " + userRsrc + " ignored");
+
+      if (userResolver.needsTargetUsersList()) {
+        if (userRsrc != null) {
+          if (!userResolver.setTargetUsers(userRsrc, conf)) {
+            LOG.warn("Ignoring the user resource '" + userRsrc + "'.");
+          }
+        } else {
+          System.err.println("\n\n" + userResolver.getClass()
+              + " needs target user list. Use -users option." + "\n\n");
+          printUsage(System.err);
+          return 1;
+        }
+      } else if (userRsrc != null) {
+        LOG.warn("Ignoring the user resource '" + userRsrc + "'.");
       }
       }
+
       ioPath = new Path(argv[argv.length - 2]);
       ioPath = new Path(argv[argv.length - 2]);
       traceIn = argv[argv.length - 1];
       traceIn = argv[argv.length - 1];
     } catch (Exception e) {
     } catch (Exception e) {
@@ -260,17 +371,46 @@ public class Gridmix extends Configured implements Tool {
       printUsage(System.err);
       printUsage(System.err);
       return 1;
       return 1;
     }
     }
-    return start(conf, traceIn, ioPath, genbytes, userResolver);
+    return start(conf, traceIn, ioPath, genbytes, userResolver, generate);
   }
   }
 
 
+  /**
+   * 
+   * @param conf gridmix configuration
+   * @param traceIn trace file path(if it is '-', then trace comes from the
+   *                stream stdin)
+   * @param ioPath Working directory for gridmix. GenerateData job
+   *               will generate data in the directory &lt;ioPath&gt;/input/ and
+   *               distributed cache data is generated in the directory
+   *               &lt;ioPath&gt;/distributedCache/, if -generate option is
+   *               specified.
+   * @param genbytes size of input data to be generated under the directory
+   *                 &lt;ioPath&gt;/input/
+   * @param userResolver gridmix user resolver
+   * @param generate true if -generate option was specified
+   * @return exit code
+   * @throws IOException
+   * @throws InterruptedException
+   */
   int start(Configuration conf, String traceIn, Path ioPath, long genbytes,
   int start(Configuration conf, String traceIn, Path ioPath, long genbytes,
-      UserResolver userResolver) throws IOException, InterruptedException {
+      UserResolver userResolver, boolean generate)
+      throws IOException, InterruptedException {
+    DataStatistics stats = null;
     InputStream trace = null;
     InputStream trace = null;
+    final FileSystem inputFs = ioPath.getFileSystem(conf);
+    ioPath = ioPath.makeQualified(inputFs);
+
     try {
     try {
+      // Create <ioPath> with 777 permissions
+      boolean succeeded = FileSystem.mkdirs(inputFs, ioPath,
+                                            new FsPermission((short) 0777));
+      if (!succeeded) {
+        throw new IOException("Creation of <ioPath> directory "
+                              + ioPath.toUri().toString() + " failed.");
+      }
+
       Path scratchDir = new Path(ioPath, conf.get(GRIDMIX_OUT_DIR, "gridmix"));
       Path scratchDir = new Path(ioPath, conf.get(GRIDMIX_OUT_DIR, "gridmix"));
-      final FileSystem scratchFs = scratchDir.getFileSystem(conf);
-      scratchFs.mkdirs(scratchDir, new FsPermission((short) 0777));
-      scratchFs.setPermission(scratchDir, new FsPermission((short) 0777));
+
       // add shutdown hook for SIGINT, etc.
       // add shutdown hook for SIGINT, etc.
       Runtime.getRuntime().addShutdownHook(sdh);
       Runtime.getRuntime().addShutdownHook(sdh);
       CountDownLatch startFlag = new CountDownLatch(1);
       CountDownLatch startFlag = new CountDownLatch(1);
@@ -278,12 +418,30 @@ public class Gridmix extends Configured implements Tool {
         // Create, start job submission threads
         // Create, start job submission threads
         startThreads(conf, traceIn, ioPath, scratchDir, startFlag,
         startThreads(conf, traceIn, ioPath, scratchDir, startFlag,
             userResolver);
             userResolver);
+
+        Path inputDir = getGridmixInputDataPath(ioPath);
+
         // Write input data if specified
         // Write input data if specified
         if (genbytes > 0) {
         if (genbytes > 0) {
-          writeInputData(genbytes, ioPath);
+          writeInputData(genbytes, inputDir);
         }
         }
+
+        // publish the data statistics
+        stats = GenerateData.publishDataStatistics(inputDir, genbytes, conf);
+
         // scan input dir contents
         // scan input dir contents
         submitter.refreshFilePool();
         submitter.refreshFilePool();
+
+        // set up the needed things for emulation of various loads
+        int exitCode = setupEmulation(conf, traceIn, scratchDir, ioPath,
+                                      generate);
+        if (exitCode != 0) {
+          return exitCode;
+        }
+
+        // start the summarizer
+        summarizer.start(conf);
+        
         factory.start();
         factory.start();
         statistics.start();
         statistics.start();
       } catch (Throwable e) {
       } catch (Throwable e) {
@@ -313,11 +471,73 @@ public class Gridmix extends Configured implements Tool {
 
 
       }
       }
     } finally {
     } finally {
+      if (factory != null) {
+        summarizer.finalize(factory, traceIn, genbytes, userResolver, stats, 
+                            conf);
+      }
       IOUtils.cleanup(LOG, trace);
       IOUtils.cleanup(LOG, trace);
     }
     }
     return 0;
     return 0;
   }
   }
 
 
+  /**
+   * Create gridmix output directory. Setup things for emulation of
+   * various loads, if needed.
+   * @param conf gridmix configuration
+   * @param traceIn trace file path(if it is '-', then trace comes from the
+   *                stream stdin)
+   * @param scratchDir gridmix output directory
+   * @param ioPath Working directory for gridmix.
+   * @param generate true if -generate option was specified
+   * @return exit code
+   * @throws IOException
+   * @throws InterruptedException 
+   */
+  private int setupEmulation(Configuration conf, String traceIn,
+      Path scratchDir, Path ioPath, boolean generate)
+      throws IOException, InterruptedException {
+    // create scratch directory(output directory of gridmix)
+    final FileSystem scratchFs = scratchDir.getFileSystem(conf);
+    FileSystem.mkdirs(scratchFs, scratchDir, new FsPermission((short) 0777));
+
+    // Setup things needed for emulation of distributed cache load
+    return setupDistCacheEmulation(conf, traceIn, ioPath, generate);
+    // Setup emulation of other loads like CPU load, Memory load
+  }
+
+  /**
+   * Setup gridmix for emulation of distributed cache load. This includes
+   * generation of distributed cache files, if needed.
+   * @param conf gridmix configuration
+   * @param traceIn trace file path(if it is '-', then trace comes from the
+   *                stream stdin)
+   * @param ioPath &lt;ioPath&gt;/input/ is the dir where input data (a) exists
+   *               or (b) is generated. &lt;ioPath&gt;/distributedCache/ is the
+   *               folder where distributed cache data (a) exists or (b) is to be
+   *               generated by gridmix.
+   * @param generate true if -generate option was specified
+   * @return exit code
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  private int setupDistCacheEmulation(Configuration conf, String traceIn,
+      Path ioPath, boolean generate) throws IOException, InterruptedException {
+    distCacheEmulator.init(traceIn, factory.jobCreator, generate);
+    int exitCode = 0;
+    if (distCacheEmulator.shouldGenerateDistCacheData() ||
+        distCacheEmulator.shouldEmulateDistCacheLoad()) {
+
+      JobStoryProducer jsp = createJobStoryProducer(traceIn, conf);
+      exitCode = distCacheEmulator.setupGenerateDistCacheData(jsp);
+      if (exitCode == 0) {
+        // If there are files to be generated, run a MapReduce job to generate
+        // these distributed cache files of all the simulated jobs of this trace.
+        writeDistCacheData(conf);
+      }
+    }
+    return exitCode;
+  }
+
   /**
   /**
    * Handles orderly shutdown by requesting that each component in the
    * Handles orderly shutdown by requesting that each component in the
    * pipeline abort its progress, waiting for each to exit and killing
    * pipeline abort its progress, waiting for each to exit and killing
@@ -387,7 +607,7 @@ public class Gridmix extends Configured implements Tool {
   public static void main(String[] argv) throws Exception {
   public static void main(String[] argv) throws Exception {
     int res = -1;
     int res = -1;
     try {
     try {
-      res = ToolRunner.run(new Configuration(), new Gridmix(), argv);
+      res = ToolRunner.run(new Configuration(), new Gridmix(argv), argv);
     } finally {
     } finally {
       System.exit(res);
       System.exit(res);
     }
     }
@@ -416,6 +636,11 @@ public class Gridmix extends Configured implements Tool {
     ToolRunner.printGenericCommandUsage(out);
     ToolRunner.printGenericCommandUsage(out);
     out.println("Usage: gridmix [-generate <MiB>] [-users URI] [-Dname=value ...] <iopath> <trace>");
     out.println("Usage: gridmix [-generate <MiB>] [-users URI] [-Dname=value ...] <iopath> <trace>");
     out.println("  e.g. gridmix -generate 100m foo -");
     out.println("  e.g. gridmix -generate 100m foo -");
+    out.println("Options:");
+    out.println("   -generate <MiB> : Generate input data of size MiB under "
+        + "<iopath>/input/ and generate\n\t\t     distributed cache data under "
+        + "<iopath>/distributedCache/.");
+    out.println("   -users <usersResourceURI> : URI that contains the users list.");
     out.println("Configuration parameters:");
     out.println("Configuration parameters:");
     out.println("   General parameters:");
     out.println("   General parameters:");
     out.printf("       %-48s : Output directory\n", GRIDMIX_OUT_DIR);
     out.printf("       %-48s : Output directory\n", GRIDMIX_OUT_DIR);
@@ -493,3 +718,4 @@ public class Gridmix extends Configured implements Tool {
   }
   }
 
 
 }
 }
+

+ 245 - 15
src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/GridmixJob.java

@@ -17,24 +17,27 @@
  */
  */
 package org.apache.hadoop.mapred.gridmix;
 package org.apache.hadoop.mapred.gridmix;
 
 
+import java.io.DataOutputStream;
 import java.io.IOException;
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Formatter;
 import java.util.Formatter;
 import java.util.List;
 import java.util.List;
 import java.util.concurrent.Callable;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.Delayed;
 import java.util.concurrent.Delayed;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeUnit;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
 import java.security.PrivilegedExceptionAction;
 import java.security.PrivilegedExceptionAction;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.DataInputBuffer;
 import org.apache.hadoop.io.DataInputBuffer;
 import org.apache.hadoop.io.RawComparator;
 import org.apache.hadoop.io.RawComparator;
 import org.apache.hadoop.io.WritableComparator;
 import org.apache.hadoop.io.WritableComparator;
 import org.apache.hadoop.io.WritableUtils;
 import org.apache.hadoop.io.WritableUtils;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.JobTracker;
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.JobContext;
 import org.apache.hadoop.mapreduce.JobContext;
@@ -53,16 +56,17 @@ import org.apache.commons.logging.LogFactory;
  */
  */
 abstract class GridmixJob implements Callable<Job>, Delayed {
 abstract class GridmixJob implements Callable<Job>, Delayed {
 
 
-  public static final String JOBNAME = "GRIDMIX";
-  public static final String ORIGNAME = "gridmix.job.name.original";
+  // Gridmix job name format is GRIDMIX<6 digit sequence number>
+  public static final String JOB_NAME_PREFIX = "GRIDMIX";
   public static final Log LOG = LogFactory.getLog(GridmixJob.class);
   public static final Log LOG = LogFactory.getLog(GridmixJob.class);
 
 
   private static final ThreadLocal<Formatter> nameFormat =
   private static final ThreadLocal<Formatter> nameFormat =
     new ThreadLocal<Formatter>() {
     new ThreadLocal<Formatter>() {
       @Override
       @Override
       protected Formatter initialValue() {
       protected Formatter initialValue() {
-        final StringBuilder sb = new StringBuilder(JOBNAME.length() + 5);
-        sb.append(JOBNAME);
+        final StringBuilder sb =
+            new StringBuilder(JOB_NAME_PREFIX.length() + 6);
+        sb.append(JOB_NAME_PREFIX);
         return new Formatter(sb);
         return new Formatter(sb);
       }
       }
     };
     };
@@ -80,6 +84,14 @@ abstract class GridmixJob implements Callable<Job>, Delayed {
       "gridmix.job-submission.use-queue-in-trace";
       "gridmix.job-submission.use-queue-in-trace";
   protected static final String GRIDMIX_DEFAULT_QUEUE = 
   protected static final String GRIDMIX_DEFAULT_QUEUE = 
       "gridmix.job-submission.default-queue";
       "gridmix.job-submission.default-queue";
+  // configuration key to enable/disable High-Ram feature emulation
+  static final String GRIDMIX_HIGHRAM_EMULATION_ENABLE = 
+    "gridmix.highram-emulation.enable";
+  // configuration key to enable/disable task jvm options
+  static final String GRIDMIX_TASK_JVM_OPTIONS_ENABLE = 
+    "gridmix.task.jvm-options.enable";
+  private static final Pattern maxHeapPattern = 
+    Pattern.compile("-Xmx[0-9]+[kKmMgGtT]?+");
 
 
   private static void setJobQueue(Job job, String queue) {
   private static void setJobQueue(Job job, String queue) {
     if (queue != null)
     if (queue != null)
@@ -93,22 +105,56 @@ abstract class GridmixJob implements Callable<Job>, Delayed {
     this.jobdesc = jobdesc;
     this.jobdesc = jobdesc;
     this.seq = seq;
     this.seq = seq;
 
 
-    ((StringBuilder)nameFormat.get().out()).setLength(JOBNAME.length());
+    ((StringBuilder)nameFormat.get().out()).setLength(JOB_NAME_PREFIX.length());
     try {
     try {
       job = this.ugi.doAs(new PrivilegedExceptionAction<Job>() {
       job = this.ugi.doAs(new PrivilegedExceptionAction<Job>() {
         public Job run() throws IOException {
         public Job run() throws IOException {
-          Job ret = new Job(conf, nameFormat.get().format("%05d", seq)
-              .toString());
+
+          String jobId = null == jobdesc.getJobID()
+                         ? "<unknown>"
+                         : jobdesc.getJobID().toString();
+          Job ret = new Job(conf,
+                            nameFormat.get().format("%06d", seq).toString());
           ret.getConfiguration().setInt(GRIDMIX_JOB_SEQ, seq);
           ret.getConfiguration().setInt(GRIDMIX_JOB_SEQ, seq);
-          ret.getConfiguration().set(ORIGNAME,
-              null == jobdesc.getJobID() ? "<unknown>" : jobdesc.getJobID()
-                  .toString());
+
+          ret.getConfiguration().set(Gridmix.ORIGINAL_JOB_ID, jobId);
+          ret.getConfiguration().set(Gridmix.ORIGINAL_JOB_NAME,
+                                     jobdesc.getName());
           if (conf.getBoolean(GRIDMIX_USE_QUEUE_IN_TRACE, false)) {
           if (conf.getBoolean(GRIDMIX_USE_QUEUE_IN_TRACE, false)) {
             setJobQueue(ret, jobdesc.getQueueName());
             setJobQueue(ret, jobdesc.getQueueName());
           } else {
           } else {
             setJobQueue(ret, conf.get(GRIDMIX_DEFAULT_QUEUE));
             setJobQueue(ret, conf.get(GRIDMIX_DEFAULT_QUEUE));
           }
           }
 
 
+          // check if the job can emulate compression
+          if (canEmulateCompression()) {
+            // set the compression related configs if compression emulation is
+            // enabled
+            if (CompressionEmulationUtil.isCompressionEmulationEnabled(conf)) {
+              CompressionEmulationUtil.configureCompressionEmulation(
+                  jobdesc.getJobConf(), ret.getConfiguration());
+            }
+          }
+          
+          // configure high ram properties if enabled
+          if (conf.getBoolean(GRIDMIX_HIGHRAM_EMULATION_ENABLE, true)) {
+            configureHighRamProperties(jobdesc.getJobConf(), 
+                                       ret.getConfiguration());
+          }
+          
+          // configure task jvm options if enabled
+          // this knob can be turned off if there is a mismatch between the
+          // target (simulation) cluster and the original cluster. Such a 
+          // mismatch can result in job failures (due to memory issues) on the 
+          // target (simulated) cluster.
+          //
+          // TODO If configured, scale the original task's JVM (heap related)
+          //      options to suit the target (simulation) cluster
+          if (conf.getBoolean(GRIDMIX_TASK_JVM_OPTIONS_ENABLE, true)) {
+            configureTaskJVMOptions(jobdesc.getJobConf(), 
+                                    ret.getConfiguration());
+          }
+          
           return ret;
           return ret;
         }
         }
       });
       });
@@ -120,6 +166,185 @@ abstract class GridmixJob implements Callable<Job>, Delayed {
         submissionMillis, TimeUnit.MILLISECONDS);
         submissionMillis, TimeUnit.MILLISECONDS);
     outdir = new Path(outRoot, "" + seq);
     outdir = new Path(outRoot, "" + seq);
   }
   }
+  
+  @SuppressWarnings("deprecation")
+  protected static void configureTaskJVMOptions(Configuration originalJobConf,
+                                                Configuration simulatedJobConf){
+    // Get the heap related java opts used for the original job and set the 
+    // same for the simulated job.
+    //    set task task heap options
+    configureTaskJVMMaxHeapOptions(originalJobConf, simulatedJobConf, 
+                                   JobConf.MAPRED_TASK_JAVA_OPTS);
+    //  set map task heap options
+    configureTaskJVMMaxHeapOptions(originalJobConf, simulatedJobConf, 
+                                   JobConf.MAPRED_MAP_TASK_JAVA_OPTS);
+
+    //  set reduce task heap options
+    configureTaskJVMMaxHeapOptions(originalJobConf, simulatedJobConf, 
+                                   JobConf.MAPRED_REDUCE_TASK_JAVA_OPTS);
+  }
+  
+  // Configures the task's max heap options using the specified key
+  private static void configureTaskJVMMaxHeapOptions(Configuration srcConf, 
+                                                     Configuration destConf,
+                                                     String key) {
+    String srcHeapOpts = srcConf.get(key);
+    if (srcHeapOpts != null) {
+      List<String> srcMaxOptsList = new ArrayList<String>();
+      // extract the max heap options and ignore the rest
+      extractMaxHeapOpts(srcHeapOpts, srcMaxOptsList, 
+                         new ArrayList<String>());
+      if (srcMaxOptsList.size() > 0) {
+        List<String> destOtherOptsList = new ArrayList<String>();
+        // extract the other heap options and ignore the max options in the 
+        // destination configuration
+        String destHeapOpts = destConf.get(key);
+        if (destHeapOpts != null) {
+          extractMaxHeapOpts(destHeapOpts, new ArrayList<String>(), 
+                             destOtherOptsList);
+        }
+        
+        // the source configuration might have some task level max heap opts set
+        // remove these opts from the destination configuration and replace
+        // with the options set in the original configuration
+        StringBuilder newHeapOpts = new StringBuilder();
+        
+        for (String otherOpt : destOtherOptsList) {
+          newHeapOpts.append(otherOpt).append(" ");
+        }
+        
+        for (String opts : srcMaxOptsList) {
+          newHeapOpts.append(opts).append(" ");
+        }
+        
+        // set the final heap opts 
+        destConf.set(key, newHeapOpts.toString().trim());
+      }
+    }
+  }
+  
+  private static void extractMaxHeapOpts(String javaOptions,  
+      List<String> maxOpts,  List<String> others) {
+    for (String opt : javaOptions.split(" ")) {
+      Matcher matcher = maxHeapPattern.matcher(opt);
+      if (matcher.find()) {
+        maxOpts.add(opt);
+      } else {
+        others.add(opt);
+      }
+    }
+  }
+
+  // Scales the desired job-level configuration parameter. This API makes sure 
+  // that the ratio of the job level configuration parameter to the cluster 
+  // level configuration parameter is maintained in the simulated run. Hence 
+  // the values are scaled from the original cluster's configuration to the 
+  // simulated cluster's configuration for higher emulation accuracy.
+  // This kind of scaling is useful for memory parameters.
+  private static void scaleConfigParameter(Configuration sourceConf, 
+                        Configuration destConf, String clusterValueKey, 
+                        String jobValueKey, long defaultValue) {
+    long simulatedClusterDefaultValue = 
+           destConf.getLong(clusterValueKey, defaultValue);
+    
+    long originalClusterDefaultValue = 
+           sourceConf.getLong(clusterValueKey, defaultValue);
+    
+    long originalJobValue = 
+           sourceConf.getLong(jobValueKey, defaultValue);
+    
+    double scaleFactor = (double)originalJobValue/originalClusterDefaultValue;
+    
+    long simulatedJobValue = (long)(scaleFactor * simulatedClusterDefaultValue);
+    
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("For the job configuration parameter '" + jobValueKey 
+                + "' and the cluster configuration parameter '" 
+                + clusterValueKey + "', the original job's configuration value"
+                + " is scaled from '" + originalJobValue + "' to '" 
+                + simulatedJobValue + "' using the default (unit) value of "
+                + "'" + originalClusterDefaultValue + "' for the original "
+                + " cluster and '" + simulatedClusterDefaultValue + "' for the"
+                + " simulated cluster.");
+    }
+    
+    destConf.setLong(jobValueKey, simulatedJobValue);
+  }
+  
+  // Checks if the scaling of original job's memory parameter value is 
+  // valid
+  @SuppressWarnings("deprecation")
+  private static boolean checkMemoryUpperLimits(String jobKey, String limitKey,  
+                                                Configuration conf, 
+                                                boolean convertLimitToMB) {
+    if (conf.get(limitKey) != null) {
+      long limit = conf.getLong(limitKey, JobConf.DISABLED_MEMORY_LIMIT);
+      // scale only if the max memory limit is set.
+      if (limit >= 0) {
+        if (convertLimitToMB) {
+          limit /= (1024 * 1024); //Converting to MB
+        }
+        
+        long scaledConfigValue = 
+               conf.getLong(jobKey, JobConf.DISABLED_MEMORY_LIMIT);
+        
+        // check now
+        if (scaledConfigValue > limit) {
+          throw new RuntimeException("Simulated job's configuration" 
+              + " parameter '" + jobKey + "' got scaled to a value '" 
+              + scaledConfigValue + "' which exceeds the upper limit of '" 
+              + limit + "' defined for the simulated cluster by the key '" 
+              + limitKey + "'. To disable High-Ram feature emulation, set '" 
+              + GRIDMIX_HIGHRAM_EMULATION_ENABLE + "' to 'false'.");
+        }
+        return true;
+      }
+    }
+    return false;
+  }
+  
+  // Check if the parameter scaling does not exceed the cluster limits.
+  @SuppressWarnings("deprecation")
+  private static void validateTaskMemoryLimits(Configuration conf, 
+                        String jobKey, String clusterMaxKey) {
+    if (!checkMemoryUpperLimits(jobKey, 
+        JobConf.UPPER_LIMIT_ON_TASK_VMEM_PROPERTY, conf, true)) {
+      checkMemoryUpperLimits(jobKey, clusterMaxKey, conf, false);
+    }
+  }
+
+  /**
+   * Sets the high ram job properties in the simulated job's configuration.
+   */
+  @SuppressWarnings("deprecation")
+  static void configureHighRamProperties(Configuration sourceConf, 
+                                         Configuration destConf) {
+    // set the memory per map task
+    scaleConfigParameter(sourceConf, destConf, 
+                         JobTracker.MAPRED_CLUSTER_MAP_MEMORY_MB_PROPERTY,
+                         JobConf.MAPRED_JOB_MAP_MEMORY_MB_PROPERTY, 
+                         JobConf.DISABLED_MEMORY_LIMIT);
+    
+    // validate and fail early
+    validateTaskMemoryLimits(destConf,
+        JobConf.MAPRED_JOB_MAP_MEMORY_MB_PROPERTY, 
+        JobTracker.MAPRED_CLUSTER_MAX_MAP_MEMORY_MB_PROPERTY);
+    
+    // set the memory per reduce task
+    scaleConfigParameter(sourceConf, destConf, 
+                         JobTracker.MAPRED_CLUSTER_REDUCE_MEMORY_MB_PROPERTY,
+                         JobConf.MAPRED_JOB_REDUCE_MEMORY_MB_PROPERTY,
+                         JobConf.DISABLED_MEMORY_LIMIT);
+    // validate and fail early
+    validateTaskMemoryLimits(destConf,
+        JobConf.MAPRED_JOB_REDUCE_MEMORY_MB_PROPERTY, 
+        JobTracker.MAPRED_CLUSTER_MAX_REDUCE_MEMORY_MB_PROPERTY);
+  }
+
+  /**
+   * Indicates whether this {@link GridmixJob} supports compression emulation.
+   */
+  protected abstract boolean canEmulateCompression();
 
 
   protected GridmixJob(
   protected GridmixJob(
     final Configuration conf, long submissionMillis, final String name)
     final Configuration conf, long submissionMillis, final String name)
@@ -289,13 +514,18 @@ abstract class GridmixJob implements Callable<Job>, Delayed {
         TaskAttemptContext job) throws IOException {
         TaskAttemptContext job) throws IOException {
 
 
       Path file = getDefaultWorkFile(job, "");
       Path file = getDefaultWorkFile(job, "");
-      FileSystem fs = file.getFileSystem(job.getConfiguration());
-      final FSDataOutputStream fileOut = fs.create(file, false);
+      final DataOutputStream fileOut;
+
+      fileOut = 
+        new DataOutputStream(CompressionEmulationUtil
+            .getPossiblyCompressedOutputStream(file, job.getConfiguration()));
+
       return new RecordWriter<K,GridmixRecord>() {
       return new RecordWriter<K,GridmixRecord>() {
         @Override
         @Override
         public void write(K ignored, GridmixRecord value)
         public void write(K ignored, GridmixRecord value)
             throws IOException {
             throws IOException {
-          value.writeRandom(fileOut, value.getSize());
+          // Let the Gridmix record fill itself.
+          value.write(fileOut);
         }
         }
         @Override
         @Override
         public void close(TaskAttemptContext ctxt) throws IOException {
         public void close(TaskAttemptContext ctxt) throws IOException {

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

@@ -25,6 +25,7 @@ import org.apache.hadoop.io.DataInputBuffer;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableUtils;
 import org.apache.hadoop.io.WritableUtils;
 import org.apache.hadoop.io.WritableComparator;
 import org.apache.hadoop.io.WritableComparator;
+import org.apache.hadoop.tools.rumen.ResourceUsageMetrics;
 
 
 class GridmixKey extends GridmixRecord {
 class GridmixKey extends GridmixRecord {
   static final byte REDUCE_SPEC = 0;
   static final byte REDUCE_SPEC = 0;
@@ -115,6 +116,22 @@ class GridmixKey extends GridmixRecord {
     setSize(origSize);
     setSize(origSize);
   }
   }
 
 
+  /**
+   * Get the {@link ResourceUsageMetrics} stored in the key.
+   */
+  public ResourceUsageMetrics getReduceResourceUsageMetrics() {
+    assert REDUCE_SPEC == getType();
+    return spec.metrics;
+  }
+  
+  /**
+   * Store the {@link ResourceUsageMetrics} in the key.
+   */
+  public void setReduceResourceUsageMetrics(ResourceUsageMetrics metrics) {
+    assert REDUCE_SPEC == getType();
+    spec.setResourceUsageSpecification(metrics);
+  }
+  
   public byte getType() {
   public byte getType() {
     return type;
     return type;
   }
   }
@@ -195,18 +212,35 @@ class GridmixKey extends GridmixRecord {
     long rec_in;
     long rec_in;
     long rec_out;
     long rec_out;
     long bytes_out;
     long bytes_out;
+    private ResourceUsageMetrics metrics = null;
+    private int sizeOfResourceUsageMetrics = 0;
     public Spec() { }
     public Spec() { }
 
 
     public void set(Spec other) {
     public void set(Spec other) {
       rec_in = other.rec_in;
       rec_in = other.rec_in;
       bytes_out = other.bytes_out;
       bytes_out = other.bytes_out;
       rec_out = other.rec_out;
       rec_out = other.rec_out;
+      setResourceUsageSpecification(other.metrics);
     }
     }
 
 
+    /**
+     * Sets the {@link ResourceUsageMetrics} for this {@link Spec}.
+     */
+    public void setResourceUsageSpecification(ResourceUsageMetrics metrics) {
+      this.metrics = metrics;
+      if (metrics != null) {
+        this.sizeOfResourceUsageMetrics = metrics.size();
+      } else {
+        this.sizeOfResourceUsageMetrics = 0;
+      }
+    }
+    
     public int getSize() {
     public int getSize() {
       return WritableUtils.getVIntSize(rec_in) +
       return WritableUtils.getVIntSize(rec_in) +
              WritableUtils.getVIntSize(rec_out) +
              WritableUtils.getVIntSize(rec_out) +
-             WritableUtils.getVIntSize(bytes_out);
+             WritableUtils.getVIntSize(bytes_out) +
+             WritableUtils.getVIntSize(sizeOfResourceUsageMetrics) +
+             sizeOfResourceUsageMetrics;
     }
     }
 
 
     @Override
     @Override
@@ -214,6 +248,11 @@ class GridmixKey extends GridmixRecord {
       rec_in = WritableUtils.readVLong(in);
       rec_in = WritableUtils.readVLong(in);
       rec_out = WritableUtils.readVLong(in);
       rec_out = WritableUtils.readVLong(in);
       bytes_out = WritableUtils.readVLong(in);
       bytes_out = WritableUtils.readVLong(in);
+      sizeOfResourceUsageMetrics =  WritableUtils.readVInt(in);
+      if (sizeOfResourceUsageMetrics > 0) {
+        metrics = new ResourceUsageMetrics();
+        metrics.readFields(in);
+      }
     }
     }
 
 
     @Override
     @Override
@@ -221,6 +260,10 @@ class GridmixKey extends GridmixRecord {
       WritableUtils.writeVLong(out, rec_in);
       WritableUtils.writeVLong(out, rec_in);
       WritableUtils.writeVLong(out, rec_out);
       WritableUtils.writeVLong(out, rec_out);
       WritableUtils.writeVLong(out, bytes_out);
       WritableUtils.writeVLong(out, bytes_out);
+      WritableUtils.writeVInt(out, sizeOfResourceUsageMetrics);
+      if (sizeOfResourceUsageMetrics > 0) {
+        metrics.write(out);
+      }
     }
     }
   }
   }
 
 

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

@@ -28,6 +28,7 @@ import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.io.WritableComparator;
 import org.apache.hadoop.io.WritableComparator;
 import org.apache.hadoop.io.WritableUtils;
 import org.apache.hadoop.io.WritableUtils;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 
 
 class GridmixRecord implements WritableComparable<GridmixRecord> {
 class GridmixRecord implements WritableComparable<GridmixRecord> {
 
 
@@ -39,6 +40,10 @@ class GridmixRecord implements WritableComparable<GridmixRecord> {
   private final DataOutputBuffer dob =
   private final DataOutputBuffer dob =
     new DataOutputBuffer(Long.SIZE / Byte.SIZE);
     new DataOutputBuffer(Long.SIZE / Byte.SIZE);
   private byte[] literal = dob.getData();
   private byte[] literal = dob.getData();
+  private boolean compressible = false;
+  private float compressionRatio = 
+    CompressionEmulationUtil.DEFAULT_COMPRESSION_RATIO;
+  private RandomTextDataGenerator rtg = null;
 
 
   GridmixRecord() {
   GridmixRecord() {
     this(1, 0L);
     this(1, 0L);
@@ -57,6 +62,19 @@ class GridmixRecord implements WritableComparable<GridmixRecord> {
     setSizeInternal(size);
     setSizeInternal(size);
   }
   }
 
 
+  void setCompressibility(boolean compressible, float ratio) {
+    this.compressible = compressible;
+    this.compressionRatio = ratio;
+    // Initialize the RandomTextDataGenerator once for every GridMix record
+    // Note that RandomTextDataGenerator is needed only when the GridMix record
+    // is configured to generate compressible text data.
+    if (compressible) {
+      rtg = 
+        CompressionEmulationUtil.getRandomTextDataGenerator(ratio, 
+                                   RandomTextDataGenerator.DEFAULT_SEED);
+    }
+  }
+  
   private void setSizeInternal(int size) {
   private void setSizeInternal(int size) {
     this.size = Math.max(1, size);
     this.size = Math.max(1, size);
     try {
     try {
@@ -79,6 +97,39 @@ class GridmixRecord implements WritableComparable<GridmixRecord> {
     return (x ^= (x << 17));
     return (x ^= (x << 17));
   }
   }
 
 
+  /**
+   * Generate random text data that can be compressed. If the record is marked
+   * compressible (via {@link FileOutputFormat#COMPRESS}), only then the 
+   * random data will be text data else 
+   * {@link GridmixRecord#writeRandom(DataOutput, int)} will be invoked.
+   */
+  private void writeRandomText(DataOutput out, final int size) 
+  throws IOException {
+    long tmp = seed;
+    out.writeLong(tmp);
+    int i = size - (Long.SIZE / Byte.SIZE);
+    //TODO Should we use long for size. What if the data is more than 4G?
+    
+    String randomWord = rtg.getRandomWord();
+    byte[] bytes = randomWord.getBytes("UTF-8");
+    long randomWordSize = bytes.length;
+    while (i >= randomWordSize) {
+      out.write(bytes);
+      i -= randomWordSize;
+      
+      // get the next random word
+      randomWord = rtg.getRandomWord();
+      bytes = randomWord.getBytes("UTF-8");
+      // determine the random word size
+      randomWordSize = bytes.length;
+    }
+    
+    // pad the remaining bytes
+    if (i > 0) {
+      out.write(bytes, 0, i);
+    }
+  }
+  
   public void writeRandom(DataOutput out, final int size) throws IOException {
   public void writeRandom(DataOutput out, final int size) throws IOException {
     long tmp = seed;
     long tmp = seed;
     out.writeLong(tmp);
     out.writeLong(tmp);
@@ -120,8 +171,13 @@ class GridmixRecord implements WritableComparable<GridmixRecord> {
     WritableUtils.writeVInt(out, size);
     WritableUtils.writeVInt(out, size);
     final int payload = size - WritableUtils.getVIntSize(size);
     final int payload = size - WritableUtils.getVIntSize(size);
     if (payload > Long.SIZE / Byte.SIZE) {
     if (payload > Long.SIZE / Byte.SIZE) {
-      writeRandom(out, payload);
+      if (compressible) {
+        writeRandomText(out, payload);
+      } else {
+        writeRandom(out, payload);
+      }
     } else if (payload > 0) {
     } else if (payload > 0) {
+      //TODO What is compressible is turned on? LOG is a bad idea!
       out.write(literal, 0, payload);
       out.write(literal, 0, payload);
     }
     }
   }
   }

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

@@ -25,9 +25,12 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.List;
 import java.util.Map.Entry;
 import java.util.Map.Entry;
 
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.io.compress.CompressionCodecFactory;
 
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
@@ -42,6 +45,7 @@ class InputStriper {
   long currentStart;
   long currentStart;
   FileStatus current;
   FileStatus current;
   final List<FileStatus> files = new ArrayList<FileStatus>();
   final List<FileStatus> files = new ArrayList<FileStatus>();
+  final Configuration conf = new Configuration();
 
 
   /**
   /**
    * @param inputDir Pool from which files are requested.
    * @param inputDir Pool from which files are requested.
@@ -91,7 +95,15 @@ class InputStriper {
       }
       }
       currentStart += fromFile;
       currentStart += fromFile;
       bytes -= fromFile;
       bytes -= fromFile;
-      if (current.getLen() - currentStart == 0) {
+      // Switch to a new file if
+      //  - the current file is uncompressed and completely used
+      //  - the current file is compressed
+      
+      CompressionCodecFactory compressionCodecs = 
+        new CompressionCodecFactory(conf);
+      CompressionCodec codec = compressionCodecs.getCodec(current.getPath());
+      if (current.getLen() - currentStart == 0
+          || codec != null) {
         current = files.get(++idx % files.size());
         current = files.get(++idx % files.size());
         currentStart = 0;
         currentStart = 0;
       }
       }

+ 53 - 7
src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/JobCreator.java

@@ -18,31 +18,42 @@
 
 
 package org.apache.hadoop.mapred.gridmix;
 package org.apache.hadoop.mapred.gridmix;
 
 
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.ClusterStatus;
 import org.apache.hadoop.mapred.ClusterStatus;
 import org.apache.hadoop.mapred.JobClient;
 import org.apache.hadoop.mapred.JobClient;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.gridmix.GenerateData.GenSplit;
-import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.tools.rumen.JobStory;
 import org.apache.hadoop.tools.rumen.JobStory;
 
 
 import java.io.IOException;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.ArrayList;
-import java.util.Random;
 import java.util.regex.Matcher;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 import java.util.regex.Pattern;
 
 
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
 public enum JobCreator {
 public enum JobCreator {
 
 
   LOADJOB {
   LOADJOB {
     @Override
     @Override
     public GridmixJob createGridmixJob(
     public GridmixJob createGridmixJob(
-      Configuration conf, long submissionMillis, JobStory jobdesc, Path outRoot,
-      UserGroupInformation ugi, int seq) throws IOException {
+      Configuration gridmixConf, long submissionMillis, JobStory jobdesc,
+      Path outRoot, UserGroupInformation ugi, int seq) throws IOException {
+
+      // Build configuration for this simulated job
+      Configuration conf = new Configuration(gridmixConf);
+      dce.configureDistCacheFiles(conf, jobdesc.getJobConf());
       return new LoadJob(conf, submissionMillis, jobdesc, outRoot, ugi, seq);
       return new LoadJob(conf, submissionMillis, jobdesc, outRoot, ugi, seq);
-    }},
+    }
+
+    @Override
+    public boolean canEmulateDistCacheLoad() {
+      return true;
+    }
+  },
 
 
   SLEEPJOB {
   SLEEPJOB {
     private String[] hosts;
     private String[] hosts;
@@ -72,12 +83,30 @@ public enum JobCreator {
       }
       }
       return new SleepJob(conf, submissionMillis, jobdesc, outRoot, ugi, seq,
       return new SleepJob(conf, submissionMillis, jobdesc, outRoot, ugi, seq,
           numLocations, hosts);
           numLocations, hosts);
-    }};
+    }
+
+    @Override
+    public boolean canEmulateDistCacheLoad() {
+      return false;
+    }
+  };
 
 
   public static final String GRIDMIX_JOB_TYPE = "gridmix.job.type";
   public static final String GRIDMIX_JOB_TYPE = "gridmix.job.type";
   public static final String SLEEPJOB_RANDOM_LOCATIONS = 
   public static final String SLEEPJOB_RANDOM_LOCATIONS = 
     "gridmix.sleep.fake-locations";
     "gridmix.sleep.fake-locations";
 
 
+  /**
+   * Create Gridmix simulated job.
+   * @param conf configuration of simulated job
+   * @param submissionMillis At what time submission of this simulated job be
+   *                         done
+   * @param jobdesc JobStory obtained from trace
+   * @param outRoot gridmix output directory
+   * @param ugi UGI of job submitter of this simulated job
+   * @param seq job sequence number
+   * @return the created simulated job
+   * @throws IOException
+   */
   public abstract GridmixJob createGridmixJob(
   public abstract GridmixJob createGridmixJob(
     final Configuration conf, long submissionMillis, final JobStory jobdesc,
     final Configuration conf, long submissionMillis, final JobStory jobdesc,
     Path outRoot, UserGroupInformation ugi, final int seq) throws IOException;
     Path outRoot, UserGroupInformation ugi, final int seq) throws IOException;
@@ -86,4 +115,21 @@ public enum JobCreator {
     Configuration conf, JobCreator defaultPolicy) {
     Configuration conf, JobCreator defaultPolicy) {
     return conf.getEnum(GRIDMIX_JOB_TYPE, defaultPolicy);
     return conf.getEnum(GRIDMIX_JOB_TYPE, defaultPolicy);
   }
   }
+
+  /**
+   * @return true if gridmix simulated jobs of this job type can emulate
+   *         distributed cache load
+   */
+  abstract boolean canEmulateDistCacheLoad();
+
+  DistributedCacheEmulator dce;
+  /**
+   * This method is to be called before calling any other method in JobCreator
+   * except canEmulateDistCacheLoad(), especially if canEmulateDistCacheLoad()
+   * returns true for that job type.
+   * @param e Distributed Cache Emulator
+   */
+  void setDistCacheEmulator(DistributedCacheEmulator e) {
+    this.dce = e;
+  }
 }
 }

+ 28 - 7
src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/JobFactory.java

@@ -63,6 +63,7 @@ abstract class JobFactory<T> implements Gridmix.Component<Void>,StatListener<T>{
   protected final JobStoryProducer jobProducer;
   protected final JobStoryProducer jobProducer;
   protected final ReentrantLock lock = new ReentrantLock(true);
   protected final ReentrantLock lock = new ReentrantLock(true);
   protected final JobCreator jobCreator;
   protected final JobCreator jobCreator;
+  protected int numJobsInTrace = 0;
 
 
   /**
   /**
    * Creating a new instance does not start the thread.
    * Creating a new instance does not start the thread.
@@ -112,7 +113,7 @@ abstract class JobFactory<T> implements Gridmix.Component<Void>,StatListener<T>{
     public MinTaskInfo(TaskInfo info) {
     public MinTaskInfo(TaskInfo info) {
       super(info.getInputBytes(), info.getInputRecords(),
       super(info.getInputBytes(), info.getInputRecords(),
             info.getOutputBytes(), info.getOutputRecords(),
             info.getOutputBytes(), info.getOutputRecords(),
-            info.getTaskMemory());
+            info.getTaskMemory(), info.getResourceUsageMetrics());
     }
     }
     public long getInputBytes() {
     public long getInputBytes() {
       return Math.max(0, super.getInputBytes());
       return Math.max(0, super.getInputBytes());
@@ -168,13 +169,33 @@ abstract class JobFactory<T> implements Gridmix.Component<Void>,StatListener<T>{
 
 
   protected abstract Thread createReaderThread() ;
   protected abstract Thread createReaderThread() ;
 
 
+  //gets the next job from the trace and does some bookkeeping for the same
+  private JobStory getNextJobFromTrace() throws IOException {
+    JobStory story = jobProducer.getNextJob();
+    if (story != null) {
+      ++numJobsInTrace;
+    }
+    return story;
+  }
+
   protected JobStory getNextJobFiltered() throws IOException {
   protected JobStory getNextJobFiltered() throws IOException {
-    JobStory job;
-    do {
-      job = jobProducer.getNextJob();
-    } while (job != null
-        && (job.getOutcome() != Pre21JobHistoryConstants.Values.SUCCESS ||
-            job.getSubmissionTime() < 0));
+    JobStory job = getNextJobFromTrace();
+    while (job != null &&
+           (job.getOutcome() != Pre21JobHistoryConstants.Values.SUCCESS ||
+            job.getSubmissionTime() < 0)) {
+      if (LOG.isDebugEnabled()) {
+        String reason = null;
+        if (job.getOutcome() != Pre21JobHistoryConstants.Values.SUCCESS) {
+          reason = "STATE (" + job.getOutcome().name() + ") ";
+        }
+        if (job.getSubmissionTime() < 0) {
+          reason += "SUBMISSION-TIME (" + job.getSubmissionTime() + ")";
+        }
+        LOG.debug("Ignoring job " + job.getJobID() + " from the input trace."
+                  + " Reason: " + reason == null ? "N/A" : reason);
+      }
+      job = getNextJobFromTrace();
+    }
     return null == job ? null : new FilterJobStory(job) {
     return null == job ? null : new FilterJobStory(job) {
         @Override
         @Override
         public TaskInfo getTaskInfo(TaskType taskType, int taskNumber) {
         public TaskInfo getTaskInfo(TaskType taskType, int taskNumber) {

+ 2 - 2
src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/JobMonitor.java

@@ -78,13 +78,13 @@ class JobMonitor implements Gridmix.Component<Job> {
   }
   }
 
 
   /**
   /**
-   * Add a submission failed job , such tht it can be communicated
+   * Add a submission failed job , such that it can be communicated
    * back to serial.
    * back to serial.
    * TODO: Cleaner solution for this problem
    * TODO: Cleaner solution for this problem
    * @param job
    * @param job
    */
    */
   public void submissionFailed(Job job) {
   public void submissionFailed(Job job) {
-    LOG.info(" Job submission failed notify if anyone is waiting " + job);
+    LOG.info("Job submission failed notification for job " + job.getJobID());
     this.statistics.add(job);
     this.statistics.add(job);
   }
   }
 
 

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

@@ -127,7 +127,7 @@ class JobSubmitter implements Gridmix.Component<GridmixJob> {
         monitor.submissionFailed(job.getJob());
         monitor.submissionFailed(job.getJob());
       } catch(Exception e) {
       } catch(Exception e) {
         //Due to some exception job wasnt submitted.
         //Due to some exception job wasnt submitted.
-        LOG.info(" Job " + job.getJob() + " submission failed " , e);
+        LOG.info(" Job " + job.getJob().getJobID() + " submission failed " , e);
         monitor.submissionFailed(job.getJob());
         monitor.submissionFailed(job.getJob());
       } finally {
       } finally {
         sem.release();
         sem.release();

+ 213 - 5
src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/LoadJob.java

@@ -22,6 +22,9 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.NullWritable;
 import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.TaskTracker;
+import org.apache.hadoop.mapred.gridmix.emulators.resourceusage.ResourceUsageMatcher;
 import org.apache.hadoop.mapreduce.InputFormat;
 import org.apache.hadoop.mapreduce.InputFormat;
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.Job;
@@ -30,10 +33,13 @@ import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.hadoop.mapreduce.RecordReader;
 import org.apache.hadoop.mapreduce.RecordReader;
 import org.apache.hadoop.mapreduce.Reducer;
 import org.apache.hadoop.mapreduce.Reducer;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskInputOutputContext;
 import org.apache.hadoop.mapreduce.TaskType;
 import org.apache.hadoop.mapreduce.TaskType;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.util.ResourceCalculatorPlugin;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.tools.rumen.JobStory;
 import org.apache.hadoop.tools.rumen.JobStory;
+import org.apache.hadoop.tools.rumen.ResourceUsageMetrics;
 import org.apache.hadoop.tools.rumen.TaskInfo;
 import org.apache.hadoop.tools.rumen.TaskInfo;
 
 
 import java.io.IOException;
 import java.io.IOException;
@@ -83,6 +89,106 @@ class LoadJob extends GridmixJob {
     return job;
     return job;
   }
   }
 
 
+  @Override
+  protected boolean canEmulateCompression() {
+    return true;
+  }
+  
+  /**
+   * This is a progress based resource usage matcher.
+   */
+  @SuppressWarnings("unchecked")
+  static class ResourceUsageMatcherRunner extends Thread {
+    private final ResourceUsageMatcher matcher;
+    private final Progressive progress;
+    private final long sleepTime;
+    private static final String SLEEP_CONFIG = 
+      "gridmix.emulators.resource-usage.sleep-duration";
+    private static final long DEFAULT_SLEEP_TIME = 100; // 100ms
+    
+    ResourceUsageMatcherRunner(final TaskInputOutputContext context, 
+                               ResourceUsageMetrics metrics) {
+      Configuration conf = context.getConfiguration();
+      
+      // set the resource calculator plugin
+      Class<? extends ResourceCalculatorPlugin> clazz =
+        conf.getClass(TaskTracker.TT_RESOURCE_CALCULATOR_PLUGIN,
+                      null, ResourceCalculatorPlugin.class);
+      ResourceCalculatorPlugin plugin = 
+        ResourceCalculatorPlugin.getResourceCalculatorPlugin(clazz, conf);
+      
+      // set the other parameters
+      this.sleepTime = conf.getLong(SLEEP_CONFIG, DEFAULT_SLEEP_TIME);
+      progress = new Progressive() {
+        @Override
+        public float getProgress() {
+          return context.getProgress();
+        }
+      };
+      
+      // instantiate a resource-usage-matcher
+      matcher = new ResourceUsageMatcher();
+      matcher.configure(conf, plugin, metrics, progress);
+    }
+    
+    protected void match() throws Exception {
+      // match the resource usage
+      matcher.matchResourceUsage();
+    }
+    
+    @Override
+    public void run() {
+      LOG.info("Resource usage matcher thread started.");
+      try {
+        while (progress.getProgress() < 1) {
+          // match
+          match();
+          
+          // sleep for some time
+          try {
+            Thread.sleep(sleepTime);
+          } catch (Exception e) {}
+        }
+        
+        // match for progress = 1
+        match();
+        LOG.info("Resource usage emulation complete! Matcher exiting");
+      } catch (Exception e) {
+        LOG.info("Exception while running the resource-usage-emulation matcher"
+                 + " thread! Exiting.", e);
+      }
+    }
+  }
+  
+  // Makes sure that the TaskTracker doesn't kill the map/reduce tasks while
+  // they are emulating
+  private static class StatusReporter extends Thread {
+    private TaskInputOutputContext context;
+    StatusReporter(TaskInputOutputContext context) {
+      this.context = context;
+    }
+
+    @Override
+    public void run() {
+      LOG.info("Status reporter thread started.");
+      try {
+        while (context.getProgress() < 1) {
+          // report progress
+          context.progress();
+
+          // sleep for some time
+          try {
+            Thread.sleep(100); // sleep for 100ms
+          } catch (Exception e) {}
+        }
+        
+        LOG.info("Status reporter thread exiting");
+      } catch (Exception e) {
+        LOG.info("Exception while running the status reporter thread!", e);
+      }
+    }
+  }
+  
   public static class LoadMapper
   public static class LoadMapper
       extends Mapper<NullWritable,GridmixRecord,GridmixKey,GridmixRecord> {
       extends Mapper<NullWritable,GridmixRecord,GridmixKey,GridmixRecord> {
 
 
@@ -95,6 +201,9 @@ class LoadJob extends GridmixJob {
     private final GridmixKey key = new GridmixKey();
     private final GridmixKey key = new GridmixKey();
     private final GridmixRecord val = new GridmixRecord();
     private final GridmixRecord val = new GridmixRecord();
 
 
+    private ResourceUsageMatcherRunner matcher = null;
+    private StatusReporter reporter = null;
+    
     @Override
     @Override
     protected void setup(Context ctxt)
     protected void setup(Context ctxt)
         throws IOException, InterruptedException {
         throws IOException, InterruptedException {
@@ -104,6 +213,20 @@ class LoadJob extends GridmixJob {
       final long[] reduceBytes = split.getOutputBytes();
       final long[] reduceBytes = split.getOutputBytes();
       final long[] reduceRecords = split.getOutputRecords();
       final long[] reduceRecords = split.getOutputRecords();
 
 
+      // enable gridmix map output record for compression
+      final boolean emulateMapOutputCompression = 
+        CompressionEmulationUtil.isCompressionEmulationEnabled(conf)
+        && conf.getBoolean("mapred.compress.map.output", false);
+      float compressionRatio = 1.0f;
+      if (emulateMapOutputCompression) {
+        compressionRatio = 
+          CompressionEmulationUtil.getMapOutputCompressionEmulationRatio(conf);
+        LOG.info("GridMix is configured to use a compression ratio of " 
+                 + compressionRatio + " for the map output data.");
+        key.setCompressibility(true, compressionRatio);
+        val.setCompressibility(true, compressionRatio);
+      }
+      
       long totalRecords = 0L;
       long totalRecords = 0L;
       final int nReduces = ctxt.getNumReduceTasks();
       final int nReduces = ctxt.getNumReduceTasks();
       if (nReduces > 0) {
       if (nReduces > 0) {
@@ -114,17 +237,30 @@ class LoadJob extends GridmixJob {
           if (i == id) {
           if (i == id) {
             spec.bytes_out = split.getReduceBytes(idx);
             spec.bytes_out = split.getReduceBytes(idx);
             spec.rec_out = split.getReduceRecords(idx);
             spec.rec_out = split.getReduceRecords(idx);
+            spec.setResourceUsageSpecification(
+                   split.getReduceResourceUsageMetrics(idx));
             ++idx;
             ++idx;
             id += maps;
             id += maps;
           }
           }
+          // set the map output bytes such that the final reduce input bytes 
+          // match the expected value obtained from the original job
+          long mapOutputBytes = reduceBytes[i];
+          if (emulateMapOutputCompression) {
+            mapOutputBytes /= compressionRatio;
+          }
           reduces.add(new IntermediateRecordFactory(
           reduces.add(new IntermediateRecordFactory(
-              new AvgRecordFactory(reduceBytes[i], reduceRecords[i], conf),
+              new AvgRecordFactory(mapOutputBytes, reduceRecords[i], conf, 
+                                   5*1024),
               i, reduceRecords[i], spec, conf));
               i, reduceRecords[i], spec, conf));
           totalRecords += reduceRecords[i];
           totalRecords += reduceRecords[i];
         }
         }
       } else {
       } else {
-        reduces.add(new AvgRecordFactory(reduceBytes[0], reduceRecords[0],
-              conf));
+        long mapOutputBytes = reduceBytes[0];
+        if (emulateMapOutputCompression) {
+          mapOutputBytes /= compressionRatio;
+        }
+        reduces.add(new AvgRecordFactory(mapOutputBytes, reduceRecords[0],
+                    conf, 5*1024));
         totalRecords = reduceRecords[0];
         totalRecords = reduceRecords[0];
       }
       }
       final long splitRecords = split.getInputRecords();
       final long splitRecords = split.getInputRecords();
@@ -134,6 +270,13 @@ class LoadJob extends GridmixJob {
         : splitRecords;
         : splitRecords;
       ratio = totalRecords / (1.0 * inputRecords);
       ratio = totalRecords / (1.0 * inputRecords);
       acc = 0.0;
       acc = 0.0;
+      
+      matcher = new ResourceUsageMatcherRunner(ctxt, 
+                      split.getMapResourceUsageMetrics());
+      
+      // start the status reporter thread
+      reporter = new StatusReporter(ctxt);
+      reporter.start();
     }
     }
 
 
     @Override
     @Override
@@ -151,6 +294,13 @@ class LoadJob extends GridmixJob {
         }
         }
         context.write(key, val);
         context.write(key, val);
         acc -= 1.0;
         acc -= 1.0;
+        
+        // match inline
+        try {
+          matcher.match();
+        } catch (Exception e) {
+          LOG.debug("Error in resource usage emulation! Message: ", e);
+        }
       }
       }
     }
     }
 
 
@@ -162,8 +312,18 @@ class LoadJob extends GridmixJob {
         while (factory.next(key, val)) {
         while (factory.next(key, val)) {
           context.write(key, val);
           context.write(key, val);
           key.setSeed(r.nextLong());
           key.setSeed(r.nextLong());
+          
+          // match inline
+          try {
+            matcher.match();
+          } catch (Exception e) {
+            LOG.debug("Error in resource usage emulation! Message: ", e);
+          }
         }
         }
       }
       }
+      
+      // start the matcher thread since the map phase ends here
+      matcher.start();
     }
     }
   }
   }
 
 
@@ -177,6 +337,9 @@ class LoadJob extends GridmixJob {
     private double ratio;
     private double ratio;
     private RecordFactory factory;
     private RecordFactory factory;
 
 
+    private ResourceUsageMatcherRunner matcher = null;
+    private StatusReporter reporter = null;
+    
     @Override
     @Override
     protected void setup(Context context)
     protected void setup(Context context)
         throws IOException, InterruptedException {
         throws IOException, InterruptedException {
@@ -187,20 +350,48 @@ class LoadJob extends GridmixJob {
       long outBytes = 0L;
       long outBytes = 0L;
       long outRecords = 0L;
       long outRecords = 0L;
       long inRecords = 0L;
       long inRecords = 0L;
+      ResourceUsageMetrics metrics = new ResourceUsageMetrics();
       for (GridmixRecord ignored : context.getValues()) {
       for (GridmixRecord ignored : context.getValues()) {
         final GridmixKey spec = context.getCurrentKey();
         final GridmixKey spec = context.getCurrentKey();
         inRecords += spec.getReduceInputRecords();
         inRecords += spec.getReduceInputRecords();
         outBytes += spec.getReduceOutputBytes();
         outBytes += spec.getReduceOutputBytes();
         outRecords += spec.getReduceOutputRecords();
         outRecords += spec.getReduceOutputRecords();
+        if (spec.getReduceResourceUsageMetrics() != null) {
+          metrics = spec.getReduceResourceUsageMetrics();
+        }
       }
       }
       if (0 == outRecords && inRecords > 0) {
       if (0 == outRecords && inRecords > 0) {
         LOG.info("Spec output bytes w/o records. Using input record count");
         LOG.info("Spec output bytes w/o records. Using input record count");
         outRecords = inRecords;
         outRecords = inRecords;
       }
       }
+      
+      // enable gridmix reduce output record for compression
+      Configuration conf = context.getConfiguration();
+      if (CompressionEmulationUtil.isCompressionEmulationEnabled(conf)
+          && FileOutputFormat.getCompressOutput(context)) {
+        float compressionRatio = 
+          CompressionEmulationUtil
+            .getReduceOutputCompressionEmulationRatio(conf);
+        LOG.info("GridMix is configured to use a compression ratio of " 
+                 + compressionRatio + " for the reduce output data.");
+        val.setCompressibility(true, compressionRatio);
+        
+        // Set the actual output data size to make sure that the actual output 
+        // data size is same after compression
+        outBytes /= compressionRatio;
+      }
+      
       factory =
       factory =
-        new AvgRecordFactory(outBytes, outRecords, context.getConfiguration());
+        new AvgRecordFactory(outBytes, outRecords, 
+                             context.getConfiguration(), 5*1024);
       ratio = outRecords / (1.0 * inRecords);
       ratio = outRecords / (1.0 * inRecords);
       acc = 0.0;
       acc = 0.0;
+      
+      matcher = new ResourceUsageMatcherRunner(context, metrics);
+      
+      // start the status reporter thread
+      reporter = new StatusReporter(context);
+      reporter.start();
     }
     }
     @Override
     @Override
     protected void reduce(GridmixKey key, Iterable<GridmixRecord> values,
     protected void reduce(GridmixKey key, Iterable<GridmixRecord> values,
@@ -210,6 +401,13 @@ class LoadJob extends GridmixJob {
         while (acc >= 1.0 && factory.next(null, val)) {
         while (acc >= 1.0 && factory.next(null, val)) {
           context.write(NullWritable.get(), val);
           context.write(NullWritable.get(), val);
           acc -= 1.0;
           acc -= 1.0;
+          
+          // match inline
+          try {
+            matcher.match();
+          } catch (Exception e) {
+            LOG.debug("Error in resource usage emulation! Message: ", e);
+          }
         }
         }
       }
       }
     }
     }
@@ -220,6 +418,13 @@ class LoadJob extends GridmixJob {
       while (factory.next(null, val)) {
       while (factory.next(null, val)) {
         context.write(NullWritable.get(), val);
         context.write(NullWritable.get(), val);
         val.setSeed(r.nextLong());
         val.setSeed(r.nextLong());
+        
+        // match inline
+        try {
+          matcher.match();
+        } catch (Exception e) {
+          LOG.debug("Error in resource usage emulation! Message: ", e);
+        }
       }
       }
     }
     }
   }
   }
@@ -311,11 +516,13 @@ class LoadJob extends GridmixJob {
       final int nSpec = reds / maps + ((reds % maps) > i ? 1 : 0);
       final int nSpec = reds / maps + ((reds % maps) > i ? 1 : 0);
       final long[] specBytes = new long[nSpec];
       final long[] specBytes = new long[nSpec];
       final long[] specRecords = new long[nSpec];
       final long[] specRecords = new long[nSpec];
+      final ResourceUsageMetrics[] metrics = new ResourceUsageMetrics[nSpec];
       for (int j = 0; j < nSpec; ++j) {
       for (int j = 0; j < nSpec; ++j) {
         final TaskInfo info =
         final TaskInfo info =
           jobdesc.getTaskInfo(TaskType.REDUCE, i + j * maps);
           jobdesc.getTaskInfo(TaskType.REDUCE, i + j * maps);
         specBytes[j] = info.getOutputBytes();
         specBytes[j] = info.getOutputBytes();
         specRecords[j] = info.getOutputRecords();
         specRecords[j] = info.getOutputRecords();
+        metrics[j] = info.getResourceUsageMetrics();
         if (LOG.isDebugEnabled()) {
         if (LOG.isDebugEnabled()) {
           LOG.debug(String.format("SPEC(%d) %d -> %d %d %d", id(), i,
           LOG.debug(String.format("SPEC(%d) %d -> %d %d %d", id(), i,
               i + j * maps, info.getOutputRecords(), info.getOutputBytes()));
               i + j * maps, info.getOutputRecords(), info.getOutputBytes()));
@@ -326,7 +533,8 @@ class LoadJob extends GridmixJob {
               info.getInputBytes(), 3), maps, i,
               info.getInputBytes(), 3), maps, i,
             info.getInputBytes(), info.getInputRecords(),
             info.getInputBytes(), info.getInputRecords(),
             info.getOutputBytes(), info.getOutputRecords(),
             info.getOutputBytes(), info.getOutputRecords(),
-            reduceByteRatio, reduceRecordRatio, specBytes, specRecords));
+            reduceByteRatio, reduceRecordRatio, specBytes, specRecords,
+            info.getResourceUsageMetrics(), metrics));
     }
     }
     pushDescription(id(), splits);
     pushDescription(id(), splits);
   }
   }

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

@@ -22,6 +22,7 @@ import java.io.DataOutput;
 import java.io.IOException;
 import java.io.IOException;
 
 
 import org.apache.hadoop.io.WritableUtils;
 import org.apache.hadoop.io.WritableUtils;
+import org.apache.hadoop.tools.rumen.ResourceUsageMetrics;
 
 
 class LoadSplit extends CombineFileSplit {
 class LoadSplit extends CombineFileSplit {
   private int id;
   private int id;
@@ -39,6 +40,9 @@ class LoadSplit extends CombineFileSplit {
   private long[] reduceOutputBytes = new long[0];
   private long[] reduceOutputBytes = new long[0];
   private long[] reduceOutputRecords = new long[0];
   private long[] reduceOutputRecords = new long[0];
 
 
+  private ResourceUsageMetrics mapMetrics;
+  private ResourceUsageMetrics[] reduceMetrics;
+
   LoadSplit() {
   LoadSplit() {
     super();
     super();
   }
   }
@@ -46,7 +50,9 @@ class LoadSplit extends CombineFileSplit {
   public LoadSplit(CombineFileSplit cfsplit, int maps, int id,
   public LoadSplit(CombineFileSplit cfsplit, int maps, int id,
       long inputBytes, long inputRecords, long outputBytes,
       long inputBytes, long inputRecords, long outputBytes,
       long outputRecords, double[] reduceBytes, double[] reduceRecords,
       long outputRecords, double[] reduceBytes, double[] reduceRecords,
-      long[] reduceOutputBytes, long[] reduceOutputRecords)
+      long[] reduceOutputBytes, long[] reduceOutputRecords,
+      ResourceUsageMetrics metrics,
+      ResourceUsageMetrics[] rMetrics)
       throws IOException {
       throws IOException {
     super(cfsplit);
     super(cfsplit);
     this.id = id;
     this.id = id;
@@ -60,6 +66,8 @@ class LoadSplit extends CombineFileSplit {
     nSpec = reduceOutputBytes.length;
     nSpec = reduceOutputBytes.length;
     this.reduceOutputBytes = reduceOutputBytes;
     this.reduceOutputBytes = reduceOutputBytes;
     this.reduceOutputRecords = reduceOutputRecords;
     this.reduceOutputRecords = reduceOutputRecords;
+    this.mapMetrics = metrics;
+    this.reduceMetrics = rMetrics;
   }
   }
 
 
   public int getId() {
   public int getId() {
@@ -97,6 +105,15 @@ class LoadSplit extends CombineFileSplit {
   public long getReduceRecords(int i) {
   public long getReduceRecords(int i) {
     return reduceOutputRecords[i];
     return reduceOutputRecords[i];
   }
   }
+  
+  public ResourceUsageMetrics getMapResourceUsageMetrics() {
+    return mapMetrics;
+  }
+  
+  public ResourceUsageMetrics getReduceResourceUsageMetrics(int i) {
+    return reduceMetrics[i];
+  }
+  
   @Override
   @Override
   public void write(DataOutput out) throws IOException {
   public void write(DataOutput out) throws IOException {
     super.write(out);
     super.write(out);
@@ -116,6 +133,12 @@ class LoadSplit extends CombineFileSplit {
       WritableUtils.writeVLong(out, reduceOutputBytes[i]);
       WritableUtils.writeVLong(out, reduceOutputBytes[i]);
       WritableUtils.writeVLong(out, reduceOutputRecords[i]);
       WritableUtils.writeVLong(out, reduceOutputRecords[i]);
     }
     }
+    mapMetrics.write(out);
+    int numReduceMetrics = (reduceMetrics == null) ? 0 : reduceMetrics.length;
+    WritableUtils.writeVInt(out, numReduceMetrics);
+    for (int i = 0; i < numReduceMetrics; ++i) {
+      reduceMetrics[i].write(out);
+    }
   }
   }
   @Override
   @Override
   public void readFields(DataInput in) throws IOException {
   public void readFields(DataInput in) throws IOException {
@@ -144,5 +167,13 @@ class LoadSplit extends CombineFileSplit {
       reduceOutputBytes[i] = WritableUtils.readVLong(in);
       reduceOutputBytes[i] = WritableUtils.readVLong(in);
       reduceOutputRecords[i] = WritableUtils.readVLong(in);
       reduceOutputRecords[i] = WritableUtils.readVLong(in);
     }
     }
+    mapMetrics = new ResourceUsageMetrics();
+    mapMetrics.readFields(in);
+    int numReduceMetrics = WritableUtils.readVInt(in);
+    reduceMetrics = new ResourceUsageMetrics[numReduceMetrics];
+    for (int i = 0; i < numReduceMetrics; ++i) {
+      reduceMetrics[i] = new ResourceUsageMetrics();
+      reduceMetrics[i].readFields(in);
+    }
   }
   }
 }
 }

+ 25 - 0
src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/Progressive.java

@@ -0,0 +1,25 @@
+/**
+ * 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;
+
+/**
+ * Used to track progress of tasks.
+ */
+public interface Progressive {
+  public float getProgress();
+}

+ 337 - 0
src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/PseudoLocalFs.java

@@ -0,0 +1,337 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Random;
+import java.net.URI;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PositionedReadable;
+import org.apache.hadoop.fs.Seekable;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.util.Progressable;
+
+/**
+ * Pseudo local file system that generates random data for any file on the fly
+ * instead of storing files on disk. So opening same file multiple times will
+ * not give same file content. There are no directories in this file system
+ * other than the root and all the files are under root i.e. "/". All file URIs
+ * on pseudo local file system should be of the format <code>
+ * pseudo:///&lt;name&gt;.&lt;fileSize&gt;</code> where name is a unique name
+ * and &lt;fileSize&gt; is a number representing the size of the file in bytes.
+ */
+class PseudoLocalFs extends FileSystem {
+  Path home;
+  /**
+   * The creation time and modification time of all files in
+   * {@link PseudoLocalFs} is same.
+   */
+  private static final long TIME = System.currentTimeMillis();
+  private static final String HOME_DIR = "/";
+  private static final long BLOCK_SIZE  = 4 * 1024 * 1024L; // 4 MB
+  private static final int DEFAULT_BUFFER_SIZE = 1024  * 1024; // 1MB
+
+  static final URI NAME = URI.create("pseudo:///");
+
+  PseudoLocalFs() {
+    this(new Path(HOME_DIR));
+  }
+
+  PseudoLocalFs(Path home) {
+    super();
+    this.home = home;
+  }
+
+  @Override
+  public URI getUri() {
+    return NAME;
+  }
+
+  @Override
+  public Path getHomeDirectory() {
+    return home;
+  }
+
+  @Override
+  public Path getWorkingDirectory() {
+    return getHomeDirectory();
+  }
+
+  /**
+   * Generates a valid pseudo local file path from the given <code>fileId</code>
+   * and <code>fileSize</code>.
+   * @param fileId unique file id string
+   * @param fileSize file size
+   * @return the generated relative path
+   */
+  static Path generateFilePath(String fileId, long fileSize) {
+    return new Path(fileId + "." + fileSize);
+  }
+
+  /**
+   * Creating a pseudo local file is nothing but validating the file path.
+   * Actual data of the file is generated on the fly when client tries to open
+   * the file for reading.
+   * @param path file path to be created
+   */
+  @Override
+  public FSDataOutputStream create(Path path) throws IOException {
+    try {
+      validateFileNameFormat(path);
+    } catch (FileNotFoundException e) {
+      throw new IOException("File creation failed for " + path);
+    }
+    return null;
+  }
+
+  /**
+   * Validate if the path provided is of expected format of Pseudo Local File
+   * System based files.
+   * @param path file path
+   * @return the file size
+   * @throws FileNotFoundException
+   */
+  long validateFileNameFormat(Path path) throws FileNotFoundException {
+    path = path.makeQualified(this);
+    boolean valid = true;
+    long fileSize = 0;
+    if (!path.toUri().getScheme().equals(getUri().getScheme())) {
+      valid = false;
+    } else {
+      String[] parts = path.toUri().getPath().split("\\.");
+      try {
+        fileSize = Long.valueOf(parts[parts.length - 1]);
+        valid = (fileSize >= 0);
+      } catch (NumberFormatException e) {
+        valid = false;
+      }
+    }
+    if (!valid) {
+      throw new FileNotFoundException("File " + path
+          + " does not exist in pseudo local file system");
+    }
+    return fileSize;
+  }
+
+  /**
+   * @See create(Path) for details
+   */
+  @Override
+  public FSDataInputStream open(Path path, int bufferSize) throws IOException {
+    long fileSize = validateFileNameFormat(path);
+    InputStream in = new RandomInputStream(fileSize, bufferSize);
+    return new FSDataInputStream(in);
+  }
+
+  /**
+   * @See create(Path) for details
+   */
+  @Override
+  public FSDataInputStream open(Path path) throws IOException {
+    return open(path, DEFAULT_BUFFER_SIZE);
+  }
+
+  @Override
+  public FileStatus getFileStatus(Path path) throws IOException {
+    long fileSize = validateFileNameFormat(path);
+    return new FileStatus(fileSize, false, 1, BLOCK_SIZE, TIME, path);
+  }
+
+  @Override
+  public boolean exists(Path path) {
+    try{
+      validateFileNameFormat(path);
+    } catch (FileNotFoundException e) {
+      return false;
+    }
+    return true;
+  }
+
+  @Override
+  public FSDataOutputStream create(Path path, FsPermission permission,
+      boolean overwrite, int bufferSize, short replication, long blockSize,
+      Progressable progress) throws IOException {
+    return create(path);
+  }
+
+  @Override
+  public FileStatus[] listStatus(Path path) throws FileNotFoundException,
+      IOException {
+    return new FileStatus[] {getFileStatus(path)};
+  }
+
+  /**
+   * Input Stream that generates specified number of random bytes.
+   */
+  static class RandomInputStream extends InputStream
+      implements Seekable, PositionedReadable {
+
+    private final Random r = new Random();
+    private BytesWritable val = null;
+    private int positionInVal = 0;// current position in the buffer 'val'
+
+    private long totalSize = 0;// total number of random bytes to be generated
+    private long curPos = 0;// current position in this stream
+
+    /**
+     * @param size total number of random bytes to be generated in this stream
+     * @param bufferSize the buffer size. An internal buffer array of length
+     * <code>bufferSize</code> is created. If <code>bufferSize</code> is not a
+     * positive number, then a default value of 1MB is used.
+     */
+    RandomInputStream(long size, int bufferSize) {
+      totalSize = size;
+      if (bufferSize <= 0) {
+        bufferSize = DEFAULT_BUFFER_SIZE;
+      }
+      val = new BytesWritable(new byte[bufferSize]);
+    }
+
+    @Override
+    public int read() throws IOException {
+      byte[] b = new byte[1];
+      if (curPos < totalSize) {
+        if (positionInVal < val.getLength()) {// use buffered byte
+          b[0] = val.getBytes()[positionInVal++];
+          ++curPos;
+        } else {// generate data
+          int num = read(b);
+          if (num < 0) {
+            return num;
+          }
+        }
+      } else {
+        return -1;
+      }
+      return b[0];
+    }
+
+    @Override
+    public int read(byte[] bytes) throws IOException {
+      return read(bytes, 0, bytes.length);
+    }
+
+    @Override
+    public int read(byte[] bytes, int off, int len) throws IOException {
+      if (curPos == totalSize) {
+        return -1;// EOF
+      }
+      int numBytes = len;
+      if (numBytes > (totalSize - curPos)) {// position in file is close to EOF
+        numBytes = (int)(totalSize - curPos);
+      }
+      if (numBytes > (val.getLength() - positionInVal)) {
+        // need to generate data into val
+        r.nextBytes(val.getBytes());
+        positionInVal = 0;
+      }
+
+      System.arraycopy(val.getBytes(), positionInVal, bytes, off, numBytes);
+      curPos += numBytes;
+      positionInVal += numBytes;
+      return numBytes;
+    }
+
+    @Override
+    public int available() {
+      return (int)(val.getLength() - positionInVal);
+    }
+
+    @Override
+    public int read(long position, byte[] buffer, int offset, int length)
+        throws IOException {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void readFully(long position, byte[] buffer) throws IOException {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void readFully(long position, byte[] buffer, int offset, int length)
+        throws IOException {
+      throw new UnsupportedOperationException();
+    }
+
+    /**
+     * Get the current position in this stream/pseudo-file
+     * @return the position in this stream/pseudo-file
+     * @throws IOException
+     */
+    @Override
+    public long getPos() throws IOException {
+      return curPos;
+    }
+
+    @Override
+    public void seek(long pos) throws IOException {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public boolean seekToNewSource(long targetPos) throws IOException {
+      throw new UnsupportedOperationException();
+    }
+  }
+
+  @Override
+  public FSDataOutputStream append(Path path, int bufferSize,
+      Progressable progress) throws IOException {
+    throw new UnsupportedOperationException("Append is not supported"
+        + " in pseudo local file system.");
+  }
+
+  @Override
+  public boolean mkdirs(Path f, FsPermission permission) throws IOException {
+    throw new UnsupportedOperationException("Mkdirs is not supported"
+        + " in pseudo local file system.");
+  }
+
+  @Override
+  public boolean rename(Path src, Path dst) throws IOException {
+    throw new UnsupportedOperationException("Rename is not supported"
+        + " in pseudo local file system.");
+  }
+
+  @Override
+  public boolean delete(Path path, boolean recursive) {
+    throw new UnsupportedOperationException("File deletion is not supported "
+        + "in pseudo local file system.");
+  }
+
+  @Override
+  public void setWorkingDirectory(Path newDir) {
+    throw new UnsupportedOperationException("SetWorkingDirectory "
+        + "is not supported in pseudo local file system.");
+  }
+
+  @Override
+  public boolean delete(Path f) throws IOException {//dummy implementation
+    return true;
+  }
+}

+ 147 - 0
src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/RandomTextDataGenerator.java

@@ -0,0 +1,147 @@
+/**
+ * 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.util.Arrays;
+import java.util.List;
+import java.util.Random;
+
+import org.apache.commons.lang.RandomStringUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * A random text generator. The words are simply sequences of alphabets.
+ */
+class RandomTextDataGenerator {
+  static final Log LOG = LogFactory.getLog(RandomTextDataGenerator.class);
+  
+  /**
+   * Configuration key for random text data generator's list size.
+   */
+  static final String GRIDMIX_DATAGEN_RANDOMTEXT_LISTSIZE = 
+    "gridmix.datagenerator.randomtext.listsize";
+  
+  /**
+   * Configuration key for random text data generator's word size.
+   */
+  static final String GRIDMIX_DATAGEN_RANDOMTEXT_WORDSIZE = 
+    "gridmix.datagenerator.randomtext.wordsize";
+  
+  /**
+   * Default random text data generator's list size.
+   */
+  static final int DEFAULT_LIST_SIZE = 200;
+  
+  /**
+   * Default random text data generator's word size.
+   */
+  static final int DEFAULT_WORD_SIZE = 10;
+  
+  /**
+   * Default random text data generator's seed.
+   */
+  static final long DEFAULT_SEED = 0L;
+  
+  /**
+   * A list of random words
+   */
+  private String[] words;
+  private Random random;
+  
+  /**
+   * Constructor for {@link RandomTextDataGenerator} with default seed.
+   * @param size the total number of words to consider.
+   * @param wordSize Size of each word
+   */
+  RandomTextDataGenerator(int size, int wordSize) {
+    this(size, DEFAULT_SEED , wordSize);
+  }
+  
+  /**
+   * Constructor for {@link RandomTextDataGenerator}.
+   * @param size the total number of words to consider.
+   * @param seed Random number generator seed for repeatability
+   * @param wordSize Size of each word
+   */
+  RandomTextDataGenerator(int size, Long seed, int wordSize) {
+    random = new Random(seed);
+    words = new String[size];
+    
+    //TODO change the default with the actual stats
+    //TODO do u need varied sized words?
+    for (int i = 0; i < size; ++i) {
+      words[i] = 
+        RandomStringUtils.random(wordSize, 0, 0, true, false, null, random);
+    }
+  }
+  
+  /**
+   * Get the configured random text data generator's list size.
+   */
+  static int getRandomTextDataGeneratorListSize(Configuration conf) {
+    return conf.getInt(GRIDMIX_DATAGEN_RANDOMTEXT_LISTSIZE, DEFAULT_LIST_SIZE);
+  }
+  
+  /**
+   * Set the random text data generator's list size.
+   */
+  static void setRandomTextDataGeneratorListSize(Configuration conf, 
+                                                 int listSize) {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Random text data generator is configured to use a dictionary " 
+                + " with " + listSize + " words");
+    }
+    conf.setInt(GRIDMIX_DATAGEN_RANDOMTEXT_LISTSIZE, listSize);
+  }
+  
+  /**
+   * Get the configured random text data generator word size.
+   */
+  static int getRandomTextDataGeneratorWordSize(Configuration conf) {
+    return conf.getInt(GRIDMIX_DATAGEN_RANDOMTEXT_WORDSIZE, DEFAULT_WORD_SIZE);
+  }
+  
+  /**
+   * Set the random text data generator word size.
+   */
+  static void setRandomTextDataGeneratorWordSize(Configuration conf, 
+                                                 int wordSize) {
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Random text data generator is configured to use a dictionary " 
+                + " with words of length " + wordSize);
+    }
+    conf.setInt(GRIDMIX_DATAGEN_RANDOMTEXT_WORDSIZE, wordSize);
+  }
+  
+  /**
+   * Returns a randomly selected word from a list of random words.
+   */
+  String getRandomWord() {
+    int index = random.nextInt(words.length);
+    return words[index];
+  }
+  
+  /**
+   * This is mainly for testing.
+   */
+  List<String> getRandomWords() {
+    return Arrays.asList(words);
+  }
+}

+ 51 - 31
src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/RoundRobinUserResolver.java

@@ -38,12 +38,18 @@ public class RoundRobinUserResolver implements UserResolver {
 
 
   private int uidx = 0;
   private int uidx = 0;
   private List<UserGroupInformation> users = Collections.emptyList();
   private List<UserGroupInformation> users = Collections.emptyList();
-  private final HashMap<UserGroupInformation,UserGroupInformation> usercache =
-    new HashMap<UserGroupInformation,UserGroupInformation>();
+
+  /**
+   *  Mapping between user names of original cluster and UGIs of proxy users of
+   *  simulated cluster
+   */
+  private final HashMap<String,UserGroupInformation> usercache =
+      new HashMap<String,UserGroupInformation>();
   
   
   /**
   /**
-   * Userlist assumes one UGI per line, each UGI matching
-   * &lt;username&gt;,&lt;group&gt;[,group]*
+   * Userlist assumes one user per line.
+   * Each line in users-list-file is of the form &lt;username&gt;[,group]* 
+   * <br> Group names are ignored(they are not parsed at all).
    */
    */
   private List<UserGroupInformation> parseUserList(
   private List<UserGroupInformation> parseUserList(
       URI userUri, Configuration conf) throws IOException {
       URI userUri, Configuration conf) throws IOException {
@@ -54,64 +60,78 @@ public class RoundRobinUserResolver implements UserResolver {
     final Path userloc = new Path(userUri.toString());
     final Path userloc = new Path(userUri.toString());
     final Text rawUgi = new Text();
     final Text rawUgi = new Text();
     final FileSystem fs = userloc.getFileSystem(conf);
     final FileSystem fs = userloc.getFileSystem(conf);
-    final ArrayList<UserGroupInformation> ret = new ArrayList();
+    final ArrayList<UserGroupInformation> ugiList =
+        new ArrayList<UserGroupInformation>();
 
 
     LineReader in = null;
     LineReader in = null;
     try {
     try {
-      final ArrayList<String> groups = new ArrayList();
       in = new LineReader(fs.open(userloc));
       in = new LineReader(fs.open(userloc));
-      while (in.readLine(rawUgi) > 0) {
+      while (in.readLine(rawUgi) > 0) {//line is of the form username[,group]*
+        // e is end position of user name in this line
         int e = rawUgi.find(",");
         int e = rawUgi.find(",");
-        if (e <= 0) {
+        if (rawUgi.getLength() == 0 || e == 0) {
           throw new IOException("Missing username: " + rawUgi);
           throw new IOException("Missing username: " + rawUgi);
         }
         }
+        if (e == -1) {
+          e = rawUgi.getLength();
+        }
         final String username = Text.decode(rawUgi.getBytes(), 0, e);
         final String username = Text.decode(rawUgi.getBytes(), 0, e);
-        int s = e;
-        while ((e = rawUgi.find(",", ++s)) != -1) {
-          groups.add(Text.decode(rawUgi.getBytes(), s, e - s));
-          s = e;
+        UserGroupInformation ugi = null;
+        try {
+          ugi = UserGroupInformation.createProxyUser(username,
+                    UserGroupInformation.getLoginUser());
+        } catch (IOException ioe) {
+          LOG.error("Error while creating a proxy user " ,ioe);
         }
         }
-        groups.add(Text.decode(rawUgi.getBytes(), s, rawUgi.getLength() - s));
-        if (groups.size() == 0) {
-          throw new IOException("Missing groups: " + rawUgi);
+        if (ugi != null) {
+          ugiList.add(ugi);
         }
         }
-        ret.add(UserGroupInformation.createRemoteUser(username));
+        // No need to parse groups, even if they exist. Go to next line
       }
       }
     } finally {
     } finally {
       if (in != null) {
       if (in != null) {
         in.close();
         in.close();
       }
       }
     }
     }
-    return ret;
+    return ugiList;
   }
   }
 
 
   @Override
   @Override
   public synchronized boolean setTargetUsers(URI userloc, Configuration conf)
   public synchronized boolean setTargetUsers(URI userloc, Configuration conf)
       throws IOException {
       throws IOException {
+    uidx = 0;
     users = parseUserList(userloc, conf);
     users = parseUserList(userloc, conf);
     if (users.size() == 0) {
     if (users.size() == 0) {
-      throw new IOException("Empty user list");
+      throw new IOException(buildEmptyUsersErrorMsg(userloc));
     }
     }
-    usercache.keySet().retainAll(users);
+    usercache.clear();
     return true;
     return true;
   }
   }
 
 
+  static String buildEmptyUsersErrorMsg(URI userloc) {
+    return "Empty user list is not allowed for RoundRobinUserResolver. Provided"
+    + " user resource URI '" + userloc + "' resulted in an empty user list.";
+  }
+
   @Override
   @Override
   public synchronized UserGroupInformation getTargetUgi(
   public synchronized UserGroupInformation getTargetUgi(
       UserGroupInformation ugi) {
       UserGroupInformation ugi) {
-    UserGroupInformation ret = usercache.get(ugi);
-    if (null == ret) {
-      ret = users.get(uidx++ % users.size());
-      usercache.put(ugi, ret);
+    // UGI of proxy user
+    UserGroupInformation targetUGI = usercache.get(ugi.getUserName());
+    if (targetUGI == null) {
+      targetUGI = users.get(uidx++ % users.size());
+      usercache.put(ugi.getUserName(), targetUGI);
     }
     }
-    UserGroupInformation val = null;
-    try {
-      val = UserGroupInformation.createProxyUser(
-        ret.getUserName(), UserGroupInformation.getLoginUser());
-    } catch (IOException e) {
-      LOG.error("Error while creating the proxy user " ,e);
-    }
-    return val;
+    return targetUGI;
   }
   }
 
 
+  /**
+   * {@inheritDoc}
+   * <p>
+   * {@link RoundRobinUserResolver} needs to map the users in the
+   * trace to the provided list of target users. So user list is needed.
+   */
+  public boolean needsTargetUsersList() {
+    return true;
+  }
 }
 }

+ 5 - 0
src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/SleepJob.java

@@ -93,6 +93,11 @@ public class SleepJob extends GridmixJob {
         Long.MAX_VALUE);
         Long.MAX_VALUE);
   }
   }
 
 
+  @Override
+  protected boolean canEmulateCompression() {
+    return false;
+  }
+  
   @Override
   @Override
   public Job call()
   public Job call()
     throws IOException, InterruptedException, ClassNotFoundException {
     throws IOException, InterruptedException, ClassNotFoundException {

+ 4 - 3
src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/Statistics.java

@@ -189,9 +189,10 @@ public class Statistics implements Component<Job> {
         try {
         try {
           jobCompleted.await(jtPollingInterval, TimeUnit.MILLISECONDS);
           jobCompleted.await(jtPollingInterval, TimeUnit.MILLISECONDS);
         } catch (InterruptedException ie) {
         } catch (InterruptedException ie) {
-          LOG.error(
-            "Statistics interrupt while waiting for polling " + ie.getCause(),
-            ie);
+          if (!shutdown) {
+            LOG.error("Statistics interrupt while waiting for completion of "
+                + "a job.", ie);
+          }
           return;
           return;
         } finally {
         } finally {
           lock.unlock();
           lock.unlock();

+ 11 - 2
src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/SubmitterUserResolver.java

@@ -32,13 +32,13 @@ public class SubmitterUserResolver implements UserResolver {
   
   
   private UserGroupInformation ugi = null;
   private UserGroupInformation ugi = null;
 
 
-  public SubmitterUserResolver() {
+  public SubmitterUserResolver() throws IOException {
     LOG.info(" Current user resolver is SubmitterUserResolver ");
     LOG.info(" Current user resolver is SubmitterUserResolver ");
+    ugi = UserGroupInformation.getLoginUser();
   }
   }
 
 
   public synchronized boolean setTargetUsers(URI userdesc, Configuration conf)
   public synchronized boolean setTargetUsers(URI userdesc, Configuration conf)
       throws IOException {
       throws IOException {
-    ugi = UserGroupInformation.getLoginUser();
     return false;
     return false;
   }
   }
 
 
@@ -47,4 +47,13 @@ public class SubmitterUserResolver implements UserResolver {
     return this.ugi;
     return this.ugi;
   }
   }
 
 
+  /**
+   * {@inheritDoc}
+   * <p>
+   * Since {@link SubmitterUserResolver} returns the user name who is running
+   * gridmix, it doesn't need a target list of users.
+   */
+  public boolean needsTargetUsersList() {
+    return false;
+  }
 }
 }

+ 75 - 0
src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/Summarizer.java

@@ -0,0 +1,75 @@
+/**
+ * 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.hadoop.conf.Configuration;
+import org.apache.hadoop.mapred.gridmix.GenerateData.DataStatistics;
+
+/**
+ * Summarizes various aspects of a {@link Gridmix} run.
+ */
+class Summarizer {
+  private ExecutionSummarizer executionSummarizer;
+  private ClusterSummarizer clusterSummarizer;
+  protected static final String NA = "N/A";
+  
+  Summarizer() {
+    this(new String[]{NA});
+  }
+  
+  Summarizer(String[] args) {
+    executionSummarizer = new ExecutionSummarizer(args);
+    clusterSummarizer = new ClusterSummarizer();
+  }
+  
+  ExecutionSummarizer getExecutionSummarizer() {
+    return executionSummarizer;
+  }
+  
+  ClusterSummarizer getClusterSummarizer() {
+    return clusterSummarizer;
+  }
+  
+  void start(Configuration conf) {
+    executionSummarizer.start(conf);
+    clusterSummarizer.start(conf);
+  }
+  
+  /**
+   * This finalizes the summarizer.
+   */
+  @SuppressWarnings("unchecked")
+  void finalize(JobFactory factory, String path, long size, 
+                UserResolver resolver, DataStatistics stats, Configuration conf)
+  throws IOException {
+    executionSummarizer.finalize(factory, path, size, resolver, stats, conf);
+  }
+  
+  /**
+   * Summarizes the current {@link Gridmix} run and the cluster used. 
+   */
+  @Override
+  public String toString() {
+    StringBuilder builder = new StringBuilder();
+    builder.append(executionSummarizer.toString());
+    builder.append(clusterSummarizer.toString());
+    return builder.toString();
+  }
+}

+ 18 - 11
src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/UserResolver.java

@@ -19,29 +19,27 @@ package org.apache.hadoop.mapred.gridmix;
 
 
 import java.io.IOException;
 import java.io.IOException;
 import java.net.URI;
 import java.net.URI;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.util.LineReader;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
 
 
 /**
 /**
  * Maps users in the trace to a set of valid target users on the test cluster.
  * Maps users in the trace to a set of valid target users on the test cluster.
  */
  */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
 public interface UserResolver {
 public interface UserResolver {
 
 
   /**
   /**
    * Configure the user map given the URI and configuration. The resolver's
    * Configure the user map given the URI and configuration. The resolver's
    * contract will define how the resource will be interpreted, but the default
    * contract will define how the resource will be interpreted, but the default
    * will typically interpret the URI as a {@link org.apache.hadoop.fs.Path}
    * will typically interpret the URI as a {@link org.apache.hadoop.fs.Path}
-   * listing target users. 
-   * @param userdesc URI (possibly null) from which user information may be
-   * loaded per the subclass contract.
+   * listing target users.
+   * This method should be called only if {@link #needsTargetUsersList()}
+   * returns true.
+   * @param userdesc URI from which user information may be loaded per the
+   * subclass contract.
    * @param conf The tool configuration.
    * @param conf The tool configuration.
    * @return true if the resource provided was used in building the list of
    * @return true if the resource provided was used in building the list of
    * target users
    * target users
@@ -55,4 +53,13 @@ public interface UserResolver {
    */
    */
   public UserGroupInformation getTargetUgi(UserGroupInformation ugi);
   public UserGroupInformation getTargetUgi(UserGroupInformation ugi);
 
 
+  /**
+   * Indicates whether this user resolver needs a list of target users to be
+   * provided.
+   *
+   * @return true if a list of target users is to be provided for this
+   * user resolver
+   */
+  public boolean needsTargetUsersList();
+
 }
 }

+ 315 - 0
src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/emulators/resourceusage/CumulativeCpuUsageEmulatorPlugin.java

@@ -0,0 +1,315 @@
+/**
+ * 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.emulators.resourceusage;
+
+import java.io.IOException;
+import java.util.Random;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapred.gridmix.Progressive;
+import org.apache.hadoop.util.ResourceCalculatorPlugin;
+import org.apache.hadoop.tools.rumen.ResourceUsageMetrics;
+
+/**
+ * <p>A {@link ResourceUsageEmulatorPlugin} that emulates the cumulative CPU 
+ * usage by performing certain CPU intensive operations. Performing such CPU 
+ * intensive operations essentially uses up some CPU. Every 
+ * {@link ResourceUsageEmulatorPlugin} is configured with a feedback module i.e 
+ * a {@link ResourceCalculatorPlugin}, to monitor the resource usage.</p>
+ * 
+ * <p>{@link CumulativeCpuUsageEmulatorPlugin} emulates the CPU usage in steps. 
+ * The frequency of emulation can be configured via 
+ * {@link #CPU_EMULATION_PROGRESS_INTERVAL}.
+ * CPU usage values are matched via emulation only on the interval boundaries.
+ * </p>
+ *  
+ * {@link CumulativeCpuUsageEmulatorPlugin} is a wrapper program for managing 
+ * the CPU usage emulation feature. It internally uses an emulation algorithm 
+ * (called as core and described using {@link CpuUsageEmulatorCore}) for 
+ * performing the actual emulation. Multiple calls to this core engine should 
+ * use up some amount of CPU.<br>
+ * 
+ * <p>{@link CumulativeCpuUsageEmulatorPlugin} provides a calibration feature 
+ * via {@link #initialize(Configuration, ResourceUsageMetrics, 
+ *                        ResourceCalculatorPlugin, Progressive)} to calibrate 
+ *  the plugin and its core for the underlying hardware. As a result of 
+ *  calibration, every call to the emulation engine's core should roughly use up
+ *  1% of the total usage value to be emulated. This makes sure that the 
+ *  underlying hardware is profiled before use and that the plugin doesn't 
+ *  accidently overuse the CPU. With 1% as the unit emulation target value for 
+ *  the core engine, there will be roughly 100 calls to the engine resulting in 
+ *  roughly 100 calls to the feedback (resource usage monitor) module. 
+ *  Excessive usage of the feedback module is discouraged as 
+ *  it might result into excess CPU usage resulting into no real CPU emulation.
+ *  </p>
+ */
+public class CumulativeCpuUsageEmulatorPlugin 
+implements ResourceUsageEmulatorPlugin {
+  protected CpuUsageEmulatorCore emulatorCore;
+  private ResourceCalculatorPlugin monitor;
+  private Progressive progress;
+  private boolean enabled = true;
+  private float emulationInterval; // emulation interval
+  private long targetCpuUsage = 0;
+  private float lastSeenProgress = 0;
+  private long lastSeenCpuUsageCpuUsage = 0;
+  
+  // Configuration parameters
+  public static final String CPU_EMULATION_PROGRESS_INTERVAL = 
+    "gridmix.emulators.resource-usage.cpu.emulation-interval";
+  private static final float DEFAULT_EMULATION_FREQUENCY = 0.1F; // 10 times
+
+  /**
+   * This is the core CPU usage emulation algorithm. This is the core engine
+   * which actually performs some CPU intensive operations to consume some
+   * amount of CPU. Multiple calls of {@link #compute()} should help the 
+   * plugin emulate the desired level of CPU usage. This core engine can be
+   * calibrated using the {@link #calibrate(ResourceCalculatorPlugin, long)}
+   * API to suit the underlying hardware better. It also can be used to optimize
+   * the emulation cycle.
+   */
+  public interface CpuUsageEmulatorCore {
+    /**
+     * Performs some computation to use up some CPU.
+     */
+    public void compute();
+    
+    /**
+     * Allows the core to calibrate itself.
+     */
+    public void calibrate(ResourceCalculatorPlugin monitor, 
+                          long totalCpuUsage);
+  }
+  
+  /**
+   * This is the core engine to emulate the CPU usage. The only responsibility 
+   * of this class is to perform certain math intensive operations to make sure 
+   * that some desired value of CPU is used.
+   */
+  public static class DefaultCpuUsageEmulator implements CpuUsageEmulatorCore {
+    // number of times to loop for performing the basic unit computation
+    private int numIterations;
+    private final Random random;
+    
+    /**
+     * This is to fool the JVM and make it think that we need the value 
+     * stored in the unit computation i.e {@link #compute()}. This will prevent
+     * the JVM from optimizing the code.
+     */
+    protected double returnValue;
+    
+    /**
+     * Initialized the {@link DefaultCpuUsageEmulator} with default values. 
+     * Note that the {@link DefaultCpuUsageEmulator} should be calibrated 
+     * (see {@link #calibrate(ResourceCalculatorPlugin, long)}) when initialized
+     * using this constructor.
+     */
+    public DefaultCpuUsageEmulator() {
+      this(-1);
+    }
+    
+    DefaultCpuUsageEmulator(int numIterations) {
+      this.numIterations = numIterations;
+      random = new Random();
+    }
+    
+    /**
+     * This will consume some desired level of CPU. This API will try to use up
+     * 'X' percent of the target cumulative CPU usage. Currently X is set to 
+     * 10%.
+     */
+    public void compute() {
+      for (int i = 0; i < numIterations; ++i) {
+        performUnitComputation();
+      }
+    }
+    
+    // Perform unit computation. The complete CPU emulation will be based on 
+    // multiple invocations to this unit computation module.
+    protected void performUnitComputation() {
+      //TODO can this be configurable too. Users/emulators should be able to 
+      // pick and choose what MATH operations to run.
+      // Example :
+      //           BASIC : ADD, SUB, MUL, DIV
+      //           ADV   : SQRT, SIN, COSIN..
+      //           COMPO : (BASIC/ADV)*
+      // Also define input generator. For now we can use the random number 
+      // generator. Later this can be changed to accept multiple sources.
+      
+      int randomData = random.nextInt();
+      int randomDataCube = randomData * randomData * randomData;
+      double randomDataCubeRoot = Math.cbrt(randomData);
+      returnValue = Math.log(Math.tan(randomDataCubeRoot 
+                                      * Math.exp(randomDataCube)) 
+                             * Math.sqrt(randomData));
+    }
+    
+    /**
+     * This will calibrate the algorithm such that a single invocation of
+     * {@link #compute()} emulates roughly 1% of the total desired resource 
+     * usage value.
+     */
+    public void calibrate(ResourceCalculatorPlugin monitor, 
+                          long totalCpuUsage) {
+      long initTime = monitor.getProcResourceValues().getCumulativeCpuTime();
+      
+      long defaultLoopSize = 0;
+      long finalTime = initTime;
+      
+      //TODO Make this configurable
+      while (finalTime - initTime < 100) { // 100 ms
+        ++defaultLoopSize;
+        performUnitComputation(); //perform unit computation
+        finalTime = monitor.getProcResourceValues().getCumulativeCpuTime();
+      }
+      
+      long referenceRuntime = finalTime - initTime;
+      
+      // time for one loop = (final-time - init-time) / total-loops
+      float timePerLoop = ((float)referenceRuntime) / defaultLoopSize;
+      
+      // compute the 1% of the total CPU usage desired
+      //TODO Make this configurable
+      long onePercent = totalCpuUsage / 100;
+      
+      // num-iterations for 1% = (total-desired-usage / 100) / time-for-one-loop
+      numIterations = Math.max(1, (int)((float)onePercent/timePerLoop));
+      
+      System.out.println("Calibration done. Basic computation runtime : " 
+          + timePerLoop + " milliseconds. Optimal number of iterations (1%): " 
+          + numIterations);
+    }
+  }
+  
+  public CumulativeCpuUsageEmulatorPlugin() {
+    this(new DefaultCpuUsageEmulator());
+  }
+  
+  /**
+   * For testing.
+   */
+  public CumulativeCpuUsageEmulatorPlugin(CpuUsageEmulatorCore core) {
+    emulatorCore = core;
+  }
+  
+  // Note that this weighing function uses only the current progress. In future,
+  // this might depend on progress, emulation-interval and expected target.
+  private float getWeightForProgressInterval(float progress) {
+    // we want some kind of exponential growth function that gives less weight
+    // on lower progress boundaries but high (exact emulation) near progress 
+    // value of 1.
+    // so here is how the current growth function looks like
+    //    progress    weight
+    //      0.1       0.0001
+    //      0.2       0.0016
+    //      0.3       0.0081
+    //      0.4       0.0256
+    //      0.5       0.0625
+    //      0.6       0.1296
+    //      0.7       0.2401
+    //      0.8       0.4096
+    //      0.9       0.6561
+    //      1.0       1.000
+    
+    return progress * progress * progress * progress;
+  }
+  
+  @Override
+  //TODO Multi-threading for speedup?
+  public void emulate() throws IOException, InterruptedException {
+    if (enabled) {
+      float currentProgress = progress.getProgress();
+      if (lastSeenProgress < currentProgress 
+          && ((currentProgress - lastSeenProgress) >= emulationInterval
+              || currentProgress == 1)) {
+        // Estimate the final cpu usage
+        //
+        //   Consider the following
+        //     Cl/Cc/Cp : Last/Current/Projected Cpu usage
+        //     Pl/Pc/Pp : Last/Current/Projected progress
+        //   Then
+        //     (Cp-Cc)/(Pp-Pc) = (Cc-Cl)/(Pc-Pl)
+        //   Solving this for Cp, we get
+        //     Cp = Cc + (1-Pc)*(Cc-Cl)/Pc-Pl)
+        //   Note that (Cc-Cl)/(Pc-Pl) is termed as 'rate' in the following 
+        //   section
+        
+        long currentCpuUsage = 
+          monitor.getProcResourceValues().getCumulativeCpuTime();
+        // estimate the cpu usage rate
+        float rate = (currentCpuUsage - lastSeenCpuUsageCpuUsage)
+                     / (currentProgress - lastSeenProgress);
+        long projectedUsage = 
+          currentCpuUsage + (long)((1 - currentProgress) * rate);
+        
+        if (projectedUsage < targetCpuUsage) {
+          // determine the correction factor between the current usage and the
+          // expected usage and add some weight to the target
+          long currentWeighedTarget = 
+            (long)(targetCpuUsage 
+                   * getWeightForProgressInterval(currentProgress));
+          
+          while (monitor.getProcResourceValues().getCumulativeCpuTime() 
+                 < currentWeighedTarget) {
+            emulatorCore.compute();
+            // sleep for 100ms
+            try {
+              Thread.sleep(100);
+            } catch (InterruptedException ie) {
+              String message = 
+                "CumulativeCpuUsageEmulatorPlugin got interrupted. Exiting.";
+              throw new RuntimeException(message);
+            }
+          }
+        }
+        
+        // set the last seen progress
+        lastSeenProgress = progress.getProgress();
+        // set the last seen usage
+        lastSeenCpuUsageCpuUsage = 
+          monitor.getProcResourceValues().getCumulativeCpuTime();
+      }
+    }
+  }
+
+  @Override
+  public void initialize(Configuration conf, ResourceUsageMetrics metrics,
+                         ResourceCalculatorPlugin monitor,
+                         Progressive progress) {
+    // get the target CPU usage
+    targetCpuUsage = metrics.getCumulativeCpuUsage();
+    if (targetCpuUsage <= 0 ) {
+      enabled = false;
+      return;
+    } else {
+      enabled = true;
+    }
+    
+    this.monitor = monitor;
+    this.progress = progress;
+    emulationInterval =  conf.getFloat(CPU_EMULATION_PROGRESS_INTERVAL, 
+                                       DEFAULT_EMULATION_FREQUENCY);
+    
+    // calibrate the core cpu-usage utility
+    emulatorCore.calibrate(monitor, targetCpuUsage);
+    
+    // initialize the states
+    lastSeenProgress = 0;
+    lastSeenCpuUsageCpuUsage = 0;
+  }
+}

+ 63 - 0
src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/emulators/resourceusage/ResourceUsageEmulatorPlugin.java

@@ -0,0 +1,63 @@
+/**
+ * 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.emulators.resourceusage;
+
+import java.io.IOException;
+
+import org.apache.hadoop.mapred.gridmix.Progressive;
+import org.apache.hadoop.util.ResourceCalculatorPlugin;
+import org.apache.hadoop.tools.rumen.ResourceUsageMetrics;
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * <p>Each resource to be emulated should have a corresponding implementation 
+ * class that implements {@link ResourceUsageEmulatorPlugin}.</p>
+ * <br><br>
+ * {@link ResourceUsageEmulatorPlugin} will be configured using the 
+ * {@link #initialize(Configuration, ResourceUsageMetrics, 
+ *                    ResourceCalculatorPlugin, Progressive)} call.
+ * Every 
+ * {@link ResourceUsageEmulatorPlugin} is also configured with a feedback module
+ * i.e a {@link ResourceCalculatorPlugin}, to monitor the current resource 
+ * usage. {@link ResourceUsageMetrics} decides the final resource usage value to
+ * emulate. {@link Progressive} keeps track of the task's progress.</p>
+ * 
+ * <br><br>
+ * 
+ * For configuring GridMix to load and and use a resource usage emulator, 
+ * see {@link ResourceUsageMatcher}. 
+ */
+public interface ResourceUsageEmulatorPlugin {
+  /**
+   * Initialize the plugin. This might involve
+   *   - initializing the variables
+   *   - calibrating the plugin
+   */
+  void initialize(Configuration conf, ResourceUsageMetrics metrics, 
+                  ResourceCalculatorPlugin monitor,
+                  Progressive progress);
+
+  /**
+   * Emulate the resource usage to match the usage target. The plugin can use
+   * the given {@link ResourceCalculatorPlugin} to query for the current 
+   * resource usage.
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  void emulate() throws IOException, InterruptedException;
+}

+ 80 - 0
src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/emulators/resourceusage/ResourceUsageMatcher.java

@@ -0,0 +1,80 @@
+/**
+ * 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.emulators.resourceusage;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapred.gridmix.Progressive;
+import org.apache.hadoop.util.ResourceCalculatorPlugin;
+import org.apache.hadoop.tools.rumen.ResourceUsageMetrics;
+import org.apache.hadoop.util.ReflectionUtils;
+
+/**
+ * <p>This is the driver class for managing all the resource usage emulators.
+ * {@link ResourceUsageMatcher} expects a comma separated list of 
+ * {@link ResourceUsageEmulatorPlugin} implementations specified using 
+ * {@link #RESOURCE_USAGE_EMULATION_PLUGINS} as the configuration parameter.</p>
+ * 
+ * <p>Note that the order in which the emulators are invoked is same as the 
+ * order in which they are configured.
+ */
+public class ResourceUsageMatcher {
+  /**
+   * Configuration key to set resource usage emulators.
+   */
+  public static final String RESOURCE_USAGE_EMULATION_PLUGINS =
+    "gridmix.emulators.resource-usage.plugins";
+  
+  private List<ResourceUsageEmulatorPlugin> emulationPlugins = 
+    new ArrayList<ResourceUsageEmulatorPlugin>();
+  
+  /**
+   * Configure the {@link ResourceUsageMatcher} to load the configured plugins
+   * and initialize them.
+   */
+  @SuppressWarnings("unchecked")
+  public void configure(Configuration conf, ResourceCalculatorPlugin monitor, 
+                        ResourceUsageMetrics metrics, Progressive progress) {
+    Class[] plugins = conf.getClasses(RESOURCE_USAGE_EMULATION_PLUGINS);
+//, null, ResourceUsageEmulatorPlugin.class);
+    if (plugins == null) {
+      System.out.println("No resource usage emulator plugins configured.");
+    } else {
+      for (Class<? extends ResourceUsageEmulatorPlugin> plugin : plugins) {
+        if (plugin != null) {
+          emulationPlugins.add(ReflectionUtils.newInstance(plugin, conf));
+        }
+      }
+    }
+
+    // initialize the emulators once all the configured emulator plugins are
+    // loaded
+    for (ResourceUsageEmulatorPlugin emulator : emulationPlugins) {
+      emulator.initialize(conf, metrics, monitor, progress);
+    }
+  }
+  
+  public void matchResourceUsage() throws Exception {
+    for (ResourceUsageEmulatorPlugin emulator : emulationPlugins) {
+      // match the resource usage
+      emulator.emulate();
+    }
+  }
+}

+ 258 - 0
src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/emulators/resourceusage/TotalHeapUsageEmulatorPlugin.java

@@ -0,0 +1,258 @@
+/**
+ * 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.emulators.resourceusage;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapred.gridmix.Progressive;
+import org.apache.hadoop.util.ResourceCalculatorPlugin;
+import org.apache.hadoop.tools.rumen.ResourceUsageMetrics;
+
+/**
+ * <p>A {@link ResourceUsageEmulatorPlugin} that emulates the total heap 
+ * usage by loading the JVM heap memory. Adding smaller chunks of data to the 
+ * heap will essentially use up some heap space thus forcing the JVM to expand 
+ * its heap and thus resulting into increase in the heap usage.</p>
+ * 
+ * <p>{@link TotalHeapUsageEmulatorPlugin} emulates the heap usage in steps. 
+ * The frequency of emulation can be configured via 
+ * {@link #HEAP_EMULATION_PROGRESS_INTERVAL}.
+ * Heap usage values are matched via emulation only at specific interval 
+ * boundaries.
+ * </p>
+ *  
+ * {@link TotalHeapUsageEmulatorPlugin} is a wrapper program for managing 
+ * the heap usage emulation feature. It internally uses an emulation algorithm 
+ * (called as core and described using {@link HeapUsageEmulatorCore}) for 
+ * performing the actual emulation. Multiple calls to this core engine should 
+ * use up some amount of heap.
+ */
+public class TotalHeapUsageEmulatorPlugin 
+implements ResourceUsageEmulatorPlugin {
+  // Configuration parameters
+  //  the core engine to emulate heap usage
+  protected HeapUsageEmulatorCore emulatorCore;
+  //  the progress bar
+  private Progressive progress;
+  //  decides if this plugin can emulate heap usage or not
+  private boolean enabled = true;
+  //  the progress boundaries/interval where emulation should be done
+  private float emulationInterval;
+  //  target heap usage to emulate
+  private long targetHeapUsageInMB = 0;
+  
+  /**
+   * The frequency (based on task progress) with which memory-emulation code is
+   * run. If the value is set to 0.1 then the emulation will happen at 10% of 
+   * the task's progress. The default value of this parameter is 
+   * {@link #DEFAULT_EMULATION_PROGRESS_INTERVAL}.
+   */
+  public static final String HEAP_EMULATION_PROGRESS_INTERVAL = 
+    "gridmix.emulators.resource-usage.heap.emulation-interval";
+  
+  // Default value for emulation interval
+  private static final float DEFAULT_EMULATION_PROGRESS_INTERVAL = 0.1F; // 10 %
+
+  private float prevEmulationProgress = 0F;
+  
+  /**
+   * The minimum buffer reserved for other non-emulation activities.
+   */
+  public static final String MIN_HEAP_FREE_RATIO = 
+    "gridmix.emulators.resource-usage.heap.min-free-ratio";
+  
+  private float minFreeHeapRatio;
+  
+  private static final float DEFAULT_MIN_FREE_HEAP_RATIO = 0.3F;
+  
+  /**
+   * Determines the unit increase per call to the core engine's load API. This
+   * is expressed as a percentage of the difference between the expected total 
+   * heap usage and the current usage. 
+   */
+  public static final String HEAP_LOAD_RATIO = 
+    "gridmix.emulators.resource-usage.heap.load-ratio";
+  
+  private float heapLoadRatio;
+  
+  private static final float DEFAULT_HEAP_LOAD_RATIO = 0.1F;
+  
+  public static int ONE_MB = 1024 * 1024;
+  
+  /**
+   * Defines the core heap usage emulation algorithm. This engine is expected
+   * to perform certain memory intensive operations to consume some
+   * amount of heap. {@link #load(long)} should load the current heap and 
+   * increase the heap usage by the specified value. This core engine can be 
+   * initialized using the {@link #initialize(ResourceCalculatorPlugin, long)} 
+   * API to suit the underlying hardware better.
+   */
+  public interface HeapUsageEmulatorCore {
+    /**
+     * Performs some memory intensive operations to use up some heap.
+     */
+    public void load(long sizeInMB);
+    
+    /**
+     * Initialize the core.
+     */
+    public void initialize(ResourceCalculatorPlugin monitor, 
+                           long totalHeapUsageInMB);
+    
+    /**
+     * Reset the resource usage
+     */
+    public void reset();
+  }
+  
+  /**
+   * This is the core engine to emulate the heap usage. The only responsibility 
+   * of this class is to perform certain memory intensive operations to make 
+   * sure that some desired value of heap is used.
+   */
+  public static class DefaultHeapUsageEmulator 
+  implements HeapUsageEmulatorCore {
+    // store the unit loads in a list
+    protected static ArrayList<Object> heapSpace = new ArrayList<Object>();
+    
+    /**
+     * Increase heap usage by current process by the given amount.
+     * This is done by creating objects each of size 1MB.
+     */
+    public void load(long sizeInMB) {
+      for (long i = 0; i < sizeInMB; ++i) {
+        // Create another String object of size 1MB
+        heapSpace.add((Object)new byte[ONE_MB]);
+      }
+    }
+    
+    /**
+     * This will initialize the core and check if the core can emulate the 
+     * desired target on the underlying hardware.
+     */
+    public void initialize(ResourceCalculatorPlugin monitor, 
+                           long totalHeapUsageInMB) {
+      long maxPhysicalMemoryInMB = monitor.getPhysicalMemorySize() / ONE_MB ;
+      if(maxPhysicalMemoryInMB < totalHeapUsageInMB) {
+        throw new RuntimeException("Total heap the can be used is " 
+            + maxPhysicalMemoryInMB 
+            + " bytes while the emulator is configured to emulate a total of " 
+            + totalHeapUsageInMB + " bytes");
+      }
+    }
+    
+    /**
+     * Clear references to all the GridMix-allocated special objects so that 
+     * heap usage is reduced.
+     */
+    @Override
+    public void reset() {
+      heapSpace.clear();
+    }
+  }
+  
+  public TotalHeapUsageEmulatorPlugin() {
+    this(new DefaultHeapUsageEmulator());
+  }
+  
+  /**
+   * For testing.
+   */
+  public TotalHeapUsageEmulatorPlugin(HeapUsageEmulatorCore core) {
+    emulatorCore = core;
+  }
+  
+  protected long getTotalHeapUsageInMB() {
+    return Runtime.getRuntime().totalMemory() / ONE_MB;
+  }
+  
+  protected long getMaxHeapUsageInMB() {
+    return Runtime.getRuntime().maxMemory() / ONE_MB;
+  }
+  
+  @Override
+  public void emulate() throws IOException, InterruptedException {
+    if (enabled) {
+      float currentProgress = progress.getProgress();
+      if (prevEmulationProgress < currentProgress 
+          && ((currentProgress - prevEmulationProgress) >= emulationInterval
+              || currentProgress == 1)) {
+
+        long maxHeapSizeInMB = getMaxHeapUsageInMB();
+        long committedHeapSizeInMB = getTotalHeapUsageInMB();
+        
+        // Increase committed heap usage, if needed
+        // Using a linear weighing function for computing the expected usage
+        long expectedHeapUsageInMB = 
+          Math.min(maxHeapSizeInMB,
+                   (long) (targetHeapUsageInMB * currentProgress));
+        if (expectedHeapUsageInMB < maxHeapSizeInMB
+            && committedHeapSizeInMB < expectedHeapUsageInMB) {
+          long bufferInMB = (long)(minFreeHeapRatio * expectedHeapUsageInMB);
+          long currentDifferenceInMB = 
+            expectedHeapUsageInMB - committedHeapSizeInMB;
+          long currentIncrementLoadSizeInMB = 
+                (long)(currentDifferenceInMB * heapLoadRatio);
+          // Make sure that at least 1 MB is incremented.
+          currentIncrementLoadSizeInMB = 
+            Math.max(1, currentIncrementLoadSizeInMB);
+          while (committedHeapSizeInMB + bufferInMB < expectedHeapUsageInMB) {
+            // add blocks in order of X% of the difference, X = 10% by default
+            emulatorCore.load(currentIncrementLoadSizeInMB);
+            committedHeapSizeInMB = getTotalHeapUsageInMB();
+          }
+        }
+        
+        // store the emulation progress boundary
+        prevEmulationProgress = currentProgress;
+      }
+      
+      // reset the core so that the garbage is reclaimed
+      emulatorCore.reset();
+    }
+  }
+
+  @Override
+  public void initialize(Configuration conf, ResourceUsageMetrics metrics,
+                         ResourceCalculatorPlugin monitor,
+                         Progressive progress) {
+    // get the target heap usage
+    targetHeapUsageInMB = metrics.getHeapUsage() / ONE_MB;
+    if (targetHeapUsageInMB <= 0 ) {
+      enabled = false;
+      return;
+    } else {
+      // calibrate the core heap-usage utility
+      emulatorCore.initialize(monitor, targetHeapUsageInMB);
+      enabled = true;
+    }
+    
+    this.progress = progress;
+    emulationInterval = 
+      conf.getFloat(HEAP_EMULATION_PROGRESS_INTERVAL, 
+                    DEFAULT_EMULATION_PROGRESS_INTERVAL);
+    
+    minFreeHeapRatio = conf.getFloat(MIN_HEAP_FREE_RATIO, 
+                                     DEFAULT_MIN_FREE_HEAP_RATIO);
+    
+    heapLoadRatio = conf.getFloat(HEAP_LOAD_RATIO, DEFAULT_HEAP_LOAD_RATIO);
+    
+    prevEmulationProgress = 0;
+  }
+}

二进制
src/contrib/gridmix/src/test/data/wordcount.json.gz


+ 10 - 5
src/contrib/gridmix/src/test/org/apache/hadoop/mapred/gridmix/DebugJobProducer.java

@@ -146,7 +146,7 @@ public class DebugJobProducer implements JobStoryProducer {
       final long seed = r.nextLong();
       final long seed = r.nextLong();
       r.setSeed(seed);
       r.setSeed(seed);
       id = seq.getAndIncrement();
       id = seq.getAndIncrement();
-      name = String.format("MOCKJOB%05d", id);
+      name = String.format("MOCKJOB%06d", id);
       this.conf = conf;
       this.conf = conf;
       LOG.info(name + " (" + seed + ")");
       LOG.info(name + " (" + seed + ")");
       submitTime = timestamp.addAndGet(
       submitTime = timestamp.addAndGet(
@@ -209,9 +209,14 @@ public class DebugJobProducer implements JobStoryProducer {
 
 
    @Override
    @Override
    public String getUser() {
    public String getUser() {
-     String s = String.format("foobar%d", id);
-     GridmixTestUtils.createHomeAndStagingDirectory(s,(JobConf)conf);
-     return s;
+     // Obtain user name from job configuration, if available.
+     // Otherwise use dummy user names.
+     String user = conf.get("user.name");
+     if (user == null) {
+       user = String.format("foobar%d", id);
+     }
+     GridmixTestUtils.createHomeAndStagingDirectory(user, (JobConf)conf);
+     return user;
    }
    }
 
 
    @Override
    @Override
@@ -285,7 +290,7 @@ public class DebugJobProducer implements JobStoryProducer {
 
 
     @Override
     @Override
     public org.apache.hadoop.mapred.JobConf getJobConf() {
     public org.apache.hadoop.mapred.JobConf getJobConf() {
-      throw new UnsupportedOperationException();
+      return new JobConf(conf);
     }
     }
 
 
     @Override
     @Override

+ 563 - 0
src/contrib/gridmix/src/test/org/apache/hadoop/mapred/gridmix/TestCompressionEmulationUtils.java

@@ -0,0 +1,563 @@
+/**
+ * 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.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.io.compress.GzipCodec;
+import org.apache.hadoop.mapred.ClusterStatus;
+import org.apache.hadoop.mapred.JobClient;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.Utils;
+import org.apache.hadoop.mapred.gridmix.CompressionEmulationUtil.RandomTextDataMapper;
+import org.apache.hadoop.mapred.gridmix.GenerateData.GenSplit;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+
+import static org.junit.Assert.*;
+import org.junit.Test;
+
+/**
+ * Test {@link CompressionEmulationUtil}
+ */
+public class TestCompressionEmulationUtils {
+  //TODO Remove this once LocalJobRunner can run Gridmix.
+  static class CustomInputFormat extends GenerateData.GenDataFormat {
+    @Override
+    public List<InputSplit> getSplits(JobContext jobCtxt) throws IOException {
+      // get the total data to be generated
+      long toGen =
+        jobCtxt.getConfiguration().getLong(GenerateData.GRIDMIX_GEN_BYTES, -1);
+      if (toGen < 0) {
+        throw new IOException("Invalid/missing generation bytes: " + toGen);
+      }
+      // get the total number of mappers configured
+      int totalMappersConfigured =
+        jobCtxt.getConfiguration().getInt("mapred.map.tasks", -1);
+      if (totalMappersConfigured < 0) {
+        throw new IOException("Invalid/missing num mappers: " 
+                              + totalMappersConfigured);
+      }
+      
+      final long bytesPerTracker = toGen / totalMappersConfigured;
+      final ArrayList<InputSplit> splits = 
+        new ArrayList<InputSplit>(totalMappersConfigured);
+      for (int i = 0; i < totalMappersConfigured; ++i) {
+        splits.add(new GenSplit(bytesPerTracker, 
+                   new String[] { "tracker_local" }));
+      }
+      return splits;
+    }
+  }
+  
+  /**
+   * Test {@link RandomTextDataMapper} via {@link CompressionEmulationUtil}.
+   */
+  @Test
+  public void testRandomCompressedTextDataGenerator() throws Exception {
+    int wordSize = 10;
+    int listSize = 20;
+    long dataSize = 10*1024*1024;
+    
+    Configuration conf = new Configuration();
+    CompressionEmulationUtil.setCompressionEmulationEnabled(conf, true);
+    CompressionEmulationUtil.setInputCompressionEmulationEnabled(conf, true);
+    
+    // configure the RandomTextDataGenerator to generate desired sized data
+    conf.setInt(RandomTextDataGenerator.GRIDMIX_DATAGEN_RANDOMTEXT_LISTSIZE, 
+                listSize);
+    conf.setInt(RandomTextDataGenerator.GRIDMIX_DATAGEN_RANDOMTEXT_WORDSIZE, 
+                wordSize);
+    conf.setLong(GenerateData.GRIDMIX_GEN_BYTES, dataSize);
+    
+    FileSystem lfs = FileSystem.getLocal(conf);
+    
+    // define the test's root temp directory
+    Path rootTempDir =
+        new Path(System.getProperty("test.build.data", "/tmp")).makeQualified(
+            lfs.getUri(), lfs.getWorkingDirectory());
+
+    Path tempDir = new Path(rootTempDir, "TestRandomCompressedTextDataGenr");
+    lfs.delete(tempDir, true);
+    
+    runDataGenJob(conf, tempDir);
+    
+    // validate the output data
+    FileStatus[] files = 
+      lfs.listStatus(tempDir, new Utils.OutputFileUtils.OutputFilesFilter());
+    long size = 0;
+    long maxLineSize = 0;
+    
+    for (FileStatus status : files) {
+      InputStream in = 
+        CompressionEmulationUtil
+          .getPossiblyDecompressedInputStream(status.getPath(), conf, 0);
+      BufferedReader reader = new BufferedReader(new InputStreamReader(in));
+      String line = reader.readLine();
+      if (line != null) {
+        long lineSize = line.getBytes().length;
+        if (lineSize > maxLineSize) {
+          maxLineSize = lineSize;
+        }
+        while (line != null) {
+          for (String word : line.split("\\s")) {
+            size += word.getBytes().length;
+          }
+          line = reader.readLine();
+        }
+      }
+      reader.close();
+    }
+
+    assertTrue(size >= dataSize);
+    assertTrue(size <= dataSize + maxLineSize);
+  }
+  
+  /**
+   * Runs a GridMix data-generation job.
+   */
+  private static void runDataGenJob(Configuration conf, Path tempDir) 
+  throws IOException, ClassNotFoundException, InterruptedException {
+    JobConf jobConf = new JobConf(conf);
+    JobClient client = new JobClient(jobConf);
+    
+    // get the local job runner
+    jobConf.setInt("mapred.map.tasks", 1);
+    
+    Job job = new Job(jobConf);
+
+    CompressionEmulationUtil.configure(job);
+    job.setInputFormatClass(CustomInputFormat.class);
+    
+    // set the output path
+    FileOutputFormat.setOutputPath(job, tempDir);
+    
+    // submit and wait for completion
+    job.submit();
+    int ret = job.waitForCompletion(true) ? 0 : 1;
+
+    assertEquals("Job Failed", 0, ret);
+  }
+  
+  /**
+   * Test if {@link RandomTextDataGenerator} can generate random text data 
+   * with the desired compression ratio. This involves
+   *   - using {@link CompressionEmulationUtil} to configure the MR job for 
+   *     generating the random text data with the desired compression ratio
+   *   - running the MR job
+   *   - test {@link RandomTextDataGenerator}'s output and match the output size
+   *     (compressed) with the expected compression ratio.
+   */
+  private void testCompressionRatioConfigure(float ratio)
+  throws Exception {
+    long dataSize = 10*1024*1024;
+    
+    Configuration conf = new Configuration();
+    CompressionEmulationUtil.setCompressionEmulationEnabled(conf, true);
+    CompressionEmulationUtil.setInputCompressionEmulationEnabled(conf, true);
+    
+    conf.setLong(GenerateData.GRIDMIX_GEN_BYTES, dataSize);
+    
+    float expectedRatio = CompressionEmulationUtil.DEFAULT_COMPRESSION_RATIO;
+    if (ratio > 0) {
+      // set the compression ratio in the conf
+      CompressionEmulationUtil.setMapInputCompressionEmulationRatio(conf, ratio);
+      expectedRatio = 
+        CompressionEmulationUtil.standardizeCompressionRatio(ratio);
+    }
+    
+    // invoke the utility to map from ratio to word-size
+    CompressionEmulationUtil.setupDataGeneratorConfig(conf);
+    
+    FileSystem lfs = FileSystem.getLocal(conf);
+    
+    // define the test's root temp directory
+    Path rootTempDir =
+        new Path(System.getProperty("test.build.data", "/tmp")).makeQualified(
+            lfs.getUri(), lfs.getWorkingDirectory());
+
+    Path tempDir = 
+      new Path(rootTempDir, "TestCustomRandomCompressedTextDataGenr");
+    lfs.delete(tempDir, true);
+    
+    runDataGenJob(conf, tempDir);
+    
+    // validate the output data
+    FileStatus[] files = 
+      lfs.listStatus(tempDir, new Utils.OutputFileUtils.OutputFilesFilter());
+    long size = 0;
+    
+    for (FileStatus status : files) {
+      size += status.getLen();
+    }
+
+    float compressionRatio = ((float)size)/dataSize;
+    float stdRatio = 
+      CompressionEmulationUtil.standardizeCompressionRatio(compressionRatio);
+    
+    assertEquals(expectedRatio, stdRatio, 0.0D);
+  }
+  
+  /**
+   * Test compression ratio with multiple compression ratios.
+   */
+  @Test
+  public void testCompressionRatios() throws Exception {
+    // test default compression ratio i.e 0.5
+    testCompressionRatioConfigure(0F);
+    // test for a sample compression ratio of 0.2
+    testCompressionRatioConfigure(0.2F);
+    // test for a sample compression ratio of 0.4
+    testCompressionRatioConfigure(0.4F);
+    // test for a sample compression ratio of 0.65
+    testCompressionRatioConfigure(0.65F);
+    // test for a compression ratio of 0.682 which should be standardized
+    // to round(0.682) i.e 0.68
+    testCompressionRatioConfigure(0.682F);
+    // test for a compression ratio of 0.567 which should be standardized
+    // to round(0.567) i.e 0.57
+    testCompressionRatioConfigure(0.567F);
+    
+    // test with a compression ratio of 0.01 which less than the min supported
+    // value of 0.07
+    boolean failed = false;
+    try {
+      testCompressionRatioConfigure(0.01F);
+    } catch (RuntimeException re) {
+      failed = true;
+    }
+    assertTrue("Compression ratio min value (0.07) check failed!", failed);
+    
+    // test with a compression ratio of 0.01 which less than the max supported
+    // value of 0.68
+    failed = false;
+    try {
+      testCompressionRatioConfigure(0.7F);
+    } catch (RuntimeException re) {
+      failed = true;
+    }
+    assertTrue("Compression ratio max value (0.68) check failed!", failed);
+  }
+  
+  /**
+   * Test compression ratio standardization.
+   */
+  @Test
+  public void testCompressionRatioStandardization() throws Exception {
+    assertEquals(0.55F, 
+        CompressionEmulationUtil.standardizeCompressionRatio(0.55F), 0.0D);
+    assertEquals(0.65F, 
+        CompressionEmulationUtil.standardizeCompressionRatio(0.652F), 0.0D);
+    assertEquals(0.78F, 
+        CompressionEmulationUtil.standardizeCompressionRatio(0.777F), 0.0D);
+    assertEquals(0.86F, 
+        CompressionEmulationUtil.standardizeCompressionRatio(0.855F), 0.0D);
+  }
+  
+  /**
+   * Test map input compression ratio configuration utilities.
+   */
+  @Test
+  public void testInputCompressionRatioConfiguration() throws Exception {
+    Configuration conf = new Configuration();
+    float ratio = 0.567F;
+    CompressionEmulationUtil.setMapInputCompressionEmulationRatio(conf, ratio);
+    assertEquals(ratio, 
+        CompressionEmulationUtil.getMapInputCompressionEmulationRatio(conf), 
+        0.0D);
+  }
+  
+  /**
+   * Test map output compression ratio configuration utilities.
+   */
+  @Test
+  public void testIntermediateCompressionRatioConfiguration() 
+  throws Exception {
+    Configuration conf = new Configuration();
+    float ratio = 0.567F;
+    CompressionEmulationUtil.setMapOutputCompressionEmulationRatio(conf, ratio);
+    assertEquals(ratio, 
+        CompressionEmulationUtil.getMapOutputCompressionEmulationRatio(conf), 
+        0.0D);
+  }
+  
+  /**
+   * Test reduce output compression ratio configuration utilities.
+   */
+  @Test
+  public void testOutputCompressionRatioConfiguration() throws Exception {
+    Configuration conf = new Configuration();
+    float ratio = 0.567F;
+    CompressionEmulationUtil.setReduceOutputCompressionEmulationRatio(conf, 
+                                                                      ratio);
+    assertEquals(ratio, 
+        CompressionEmulationUtil.getReduceOutputCompressionEmulationRatio(conf),
+        0.0D);
+  }
+  
+  /**
+   * Test compressible {@link GridmixRecord}.
+   */
+  @Test
+  public void testCompressibleGridmixRecord() throws IOException {
+    JobConf conf = new JobConf();
+    CompressionEmulationUtil.setCompressionEmulationEnabled(conf, true);
+    CompressionEmulationUtil.setInputCompressionEmulationEnabled(conf, true);
+    
+    FileSystem lfs = FileSystem.getLocal(conf);
+    int dataSize = 1024 * 1024 * 10; // 10 MB
+    float ratio = 0.357F;
+    
+    // define the test's root temp directory
+    Path rootTempDir =
+        new Path(System.getProperty("test.build.data", "/tmp")).makeQualified(
+            lfs.getUri(), lfs.getWorkingDirectory());
+
+    Path tempDir = new Path(rootTempDir, 
+                            "TestPossiblyCompressibleGridmixRecord");
+    lfs.delete(tempDir, true);
+    
+    // define a compressible GridmixRecord
+    GridmixRecord record = new GridmixRecord(dataSize, 0);
+    record.setCompressibility(true, ratio); // enable compression
+    
+    conf.setClass("mapred.output.compression.codec", GzipCodec.class, 
+                  CompressionCodec.class);
+    org.apache.hadoop.mapred.FileOutputFormat.setCompressOutput(conf, true);
+    
+    // write the record to a file
+    Path recordFile = new Path(tempDir, "record");
+    OutputStream outStream = CompressionEmulationUtil
+                               .getPossiblyCompressedOutputStream(recordFile, 
+                                                                  conf);    
+    DataOutputStream out = new DataOutputStream(outStream);
+    record.write(out);
+    out.close();
+    outStream.close();
+    
+    // open the compressed stream for reading
+    Path actualRecordFile = recordFile.suffix(".gz");
+    InputStream in = 
+      CompressionEmulationUtil
+        .getPossiblyDecompressedInputStream(actualRecordFile, conf, 0);
+    
+    // get the compressed file size
+    long compressedFileSize = lfs.listStatus(actualRecordFile)[0].getLen();
+    
+    GridmixRecord recordRead = new GridmixRecord();
+    recordRead.readFields(new DataInputStream(in));
+    
+    assertEquals("Record size mismatch in a compressible GridmixRecord",
+                 dataSize, recordRead.getSize());
+    assertTrue("Failed to generate a compressible GridmixRecord",
+               recordRead.getSize() > compressedFileSize);
+    
+    // check if the record can generate data with the desired compression ratio
+    float seenRatio = ((float)compressedFileSize)/dataSize;
+    assertEquals(CompressionEmulationUtil.standardizeCompressionRatio(ratio), 
+        CompressionEmulationUtil.standardizeCompressionRatio(seenRatio), 1.0D);
+  }
+  
+  /**
+   * Test 
+   * {@link CompressionEmulationUtil#isCompressionEmulationEnabled(
+   *          org.apache.hadoop.conf.Configuration)}.
+   */
+  @Test
+  public void testIsCompressionEmulationEnabled() {
+    Configuration conf = new Configuration();
+    // Check default values
+    assertTrue(CompressionEmulationUtil.isCompressionEmulationEnabled(conf));
+    
+    // Check disabled
+    CompressionEmulationUtil.setCompressionEmulationEnabled(conf, false);
+    assertFalse(CompressionEmulationUtil.isCompressionEmulationEnabled(conf));
+    
+    // Check enabled
+    CompressionEmulationUtil.setCompressionEmulationEnabled(conf, true);
+    assertTrue(CompressionEmulationUtil.isCompressionEmulationEnabled(conf));
+  }
+  
+  /**
+   * Test 
+   * {@link CompressionEmulationUtil#getPossiblyDecompressedInputStream(Path, 
+   *                                   Configuration, long)}
+   *  and
+   *  {@link CompressionEmulationUtil#getPossiblyCompressedOutputStream(Path, 
+   *                                    Configuration)}.
+   */
+  @Test
+  public void testPossiblyCompressedDecompressedStreams() throws IOException {
+    JobConf conf = new JobConf();
+    FileSystem lfs = FileSystem.getLocal(conf);
+    String inputLine = "Hi Hello!";
+
+    CompressionEmulationUtil.setCompressionEmulationEnabled(conf, true);
+    CompressionEmulationUtil.setInputCompressionEmulationEnabled(conf, true);
+    conf.setBoolean("mapred.output.compress", true);
+    conf.setClass("mapred.output.compression.codec", GzipCodec.class, 
+                  CompressionCodec.class);
+
+    // define the test's root temp directory
+    Path rootTempDir =
+        new Path(System.getProperty("test.build.data", "/tmp")).makeQualified(
+            lfs.getUri(), lfs.getWorkingDirectory());
+
+    Path tempDir =
+      new Path(rootTempDir, "TestPossiblyCompressedDecompressedStreams");
+    lfs.delete(tempDir, true);
+
+    // create a compressed file
+    Path compressedFile = new Path(tempDir, "test");
+    OutputStream out = 
+      CompressionEmulationUtil.getPossiblyCompressedOutputStream(compressedFile, 
+                                                                 conf);
+    BufferedWriter writer = new BufferedWriter(new OutputStreamWriter(out));
+    writer.write(inputLine);
+    writer.close();
+    
+    // now read back the data from the compressed stream
+    compressedFile = compressedFile.suffix(".gz");
+    InputStream in = 
+      CompressionEmulationUtil
+        .getPossiblyDecompressedInputStream(compressedFile, conf, 0);
+    BufferedReader reader = new BufferedReader(new InputStreamReader(in));
+    String readLine = reader.readLine();
+    assertEquals("Compression/Decompression error", inputLine, readLine);
+    reader.close();
+  }
+  
+  /**
+   * Test if 
+   * {@link CompressionEmulationUtil#configureCompressionEmulation(
+   *        org.apache.hadoop.mapred.JobConf, org.apache.hadoop.mapred.JobConf)}
+   *  can extract compression related configuration parameters.
+   */
+  @Test
+  public void testExtractCompressionConfigs() {
+    JobConf source = new JobConf();
+    JobConf target = new JobConf();
+    
+    // set the default values
+    source.setBoolean("mapred.output.compress", false);
+    source.set("mapred.output.compression.codec", "MyDefaultCodec");
+    source.set("mapred.output.compression.type", "MyDefaultType");
+    source.setBoolean("mapred.compress.map.output", false); 
+    source.set("mapred.map.output.compression.codec", "MyDefaultCodec2");
+    
+    CompressionEmulationUtil.configureCompressionEmulation(source, target);
+    
+    // check default values
+    assertFalse(target.getBoolean("mapred.output.compress", true));
+    assertEquals("MyDefaultCodec",
+                 target.get("mapred.output.compression.codec"));
+    assertEquals("MyDefaultType", target.get("mapred.output.compression.type"));
+    assertFalse(target.getBoolean("mapred.compress.map.output", true));
+    assertEquals("MyDefaultCodec2", 
+                 target.get("mapred.map.output.compression.codec"));
+    assertFalse(CompressionEmulationUtil
+                .isInputCompressionEmulationEnabled(target));
+    
+    // set new values
+    source.setBoolean("mapred.output.compress", true);
+    source.set("mapred.output.compression.codec", "MyCodec");
+    source.set("mapred.output.compression.type", "MyType");
+    source.setBoolean("mapred.compress.map.output", true);
+    source.set("mapred.map.output.compression.codec", "MyCodec2");
+    org.apache.hadoop.mapred.FileInputFormat.setInputPaths(source, "file.gz");
+    
+    target = new JobConf(); // reset
+    CompressionEmulationUtil.configureCompressionEmulation(source, target);
+    
+    // check new values
+    assertTrue(target.getBoolean("mapred.output.compress", false));
+    assertEquals("MyCodec",
+                 target.get("mapred.output.compression.codec"));
+    assertEquals("MyType", target.get("mapred.output.compression.type"));
+    assertTrue(target.getBoolean("mapred.compress.map.output", false));
+    assertEquals("MyCodec2", 
+                 target.get("mapred.map.output.compression.codec"));
+    assertTrue(CompressionEmulationUtil
+               .isInputCompressionEmulationEnabled(target));
+  }
+  
+  /**
+   * Test of {@link FileQueue} can identify compressed file and provide
+   * readers to extract uncompressed data only if input-compression is enabled.
+   */
+  @Test
+  public void testFileQueueDecompression() throws IOException {
+    JobConf conf = new JobConf();
+    FileSystem lfs = FileSystem.getLocal(conf);
+    String inputLine = "Hi Hello!";
+    
+    CompressionEmulationUtil.setCompressionEmulationEnabled(conf, true);
+    CompressionEmulationUtil.setInputCompressionEmulationEnabled(conf, true);
+    org.apache.hadoop.mapred.FileOutputFormat.setCompressOutput(conf, true);
+    org.apache.hadoop.mapred.FileOutputFormat.setOutputCompressorClass(conf, 
+                                                GzipCodec.class);
+
+    // define the test's root temp directory
+    Path rootTempDir =
+        new Path(System.getProperty("test.build.data", "/tmp")).makeQualified(
+            lfs.getUri(), lfs.getWorkingDirectory());
+
+    Path tempDir = new Path(rootTempDir, "TestFileQueueDecompression");
+    lfs.delete(tempDir, true);
+
+    // create a compressed file
+    Path compressedFile = new Path(tempDir, "test");
+    OutputStream out = 
+      CompressionEmulationUtil.getPossiblyCompressedOutputStream(compressedFile, 
+                                                                 conf);
+    BufferedWriter writer = new BufferedWriter(new OutputStreamWriter(out));
+    writer.write(inputLine);
+    writer.close();
+    
+    compressedFile = compressedFile.suffix(".gz");
+    // now read back the data from the compressed stream using FileQueue
+    long fileSize = lfs.listStatus(compressedFile)[0].getLen();
+    CombineFileSplit split = 
+      new CombineFileSplit(new Path[] {compressedFile}, new long[] {fileSize});
+    FileQueue queue = new FileQueue(split, conf);
+    byte[] bytes = new byte[inputLine.getBytes().length];
+    queue.read(bytes);
+    queue.close();
+    String readLine = new String(bytes);
+    assertEquals("Compression/Decompression error", inputLine, readLine);
+  }
+}

+ 498 - 0
src/contrib/gridmix/src/test/org/apache/hadoop/mapred/gridmix/TestDistCacheEmulation.java

@@ -0,0 +1,498 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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 static org.junit.Assert.*;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.filecache.DistributedCache;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.MapContext;
+import org.apache.hadoop.mapreduce.MapReduceTestUtil;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.lib.input.FileSplit;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * Validate emulation of distributed cache load in gridmix simulated jobs.
+ *
+ */
+public class TestDistCacheEmulation {
+
+  private DistributedCacheEmulator dce = null;
+
+  @BeforeClass
+  public static void init() throws IOException {
+    GridmixTestUtils.initCluster();
+  }
+
+  @AfterClass
+  public static void shutDown() throws IOException {
+    GridmixTestUtils.shutdownCluster();
+  }
+
+  /**
+   * Validate the dist cache files generated by GenerateDistCacheData job.
+   * @param jobConf configuration of GenerateDistCacheData job.
+   * @param sortedFileSizes array of sorted distributed cache file sizes 
+   * @throws IOException 
+   * @throws FileNotFoundException 
+   */
+  private void validateDistCacheData(JobConf jobConf, long[] sortedFileSizes)
+      throws FileNotFoundException, IOException {
+    Path distCachePath = dce.getDistributedCacheDir();
+    String filesListFile =
+        jobConf.get(GenerateDistCacheData.GRIDMIX_DISTCACHE_FILE_LIST);
+    FileSystem fs = FileSystem.get(jobConf);
+
+    // Validate the existence of Distributed Cache files list file directly
+    // under distributed cache directory
+    Path listFile = new Path(filesListFile);
+    assertTrue("Path of Distributed Cache files list file is wrong.",
+        distCachePath.equals(listFile.getParent().makeQualified(fs)));
+
+    // Delete the dist cache files list file
+    assertTrue("Failed to delete distributed Cache files list file " + listFile,
+               fs.delete(listFile));
+
+    List<Long> fileSizes = new ArrayList<Long>();
+    for (long size : sortedFileSizes) {
+      fileSizes.add(size);
+    }
+    // validate dist cache files after deleting the 'files list file'
+    validateDistCacheFiles(fileSizes, distCachePath);
+  }
+
+  /**
+   * Validate private/public distributed cache files.
+   * @param filesSizesExpected list of sizes of expected dist cache files
+   * @param distCacheDir the distributed cache dir to be validated
+   * @throws IOException 
+   * @throws FileNotFoundException 
+   */
+  private void validateDistCacheFiles(List filesSizesExpected,
+      Path distCacheDir) throws FileNotFoundException, IOException {
+    //RemoteIterator<LocatedFileStatus> iter =
+    FileStatus[] statuses = GridmixTestUtils.dfs.listStatus(distCacheDir);
+    int numFiles = filesSizesExpected.size();
+    assertEquals("Number of files under distributed cache dir is wrong.",
+                 numFiles, statuses.length);
+    for (int i = 0; i < numFiles; i++) {
+      FileStatus stat = statuses[i];
+      assertTrue("File size of distributed cache file "
+          + stat.getPath().toUri().getPath() + " is wrong.",
+          filesSizesExpected.remove(stat.getLen()));
+
+      FsPermission perm = stat.getPermission();
+      assertEquals("Wrong permissions for distributed cache file "
+          + stat.getPath().toUri().getPath(),
+          new FsPermission((short)0644), perm);
+    }
+  }
+
+  /**
+   * Configures 5 HDFS-based dist cache files and 1 local-FS-based dist cache
+   * file in the given Configuration object <code>conf</code>.
+   * @param conf configuration where dist cache config properties are to be set
+   * @return array of sorted HDFS-based distributed cache file sizes
+   * @throws IOException
+   */
+  private long[] configureDummyDistCacheFiles(Configuration conf)
+      throws IOException {
+    String user = UserGroupInformation.getCurrentUser().getShortUserName();
+    conf.set("user.name", user);
+    // Set some dummy dist cache files in gridmix configuration so that they go
+    // into the configuration of JobStory objects.
+    String[] distCacheFiles = {"hdfs:///tmp/file1.txt",
+                               "/tmp/" + user + "/.staging/job_1/file2.txt",
+                               "hdfs:///user/user1/file3.txt",
+                               "/home/user2/file4.txt",
+                               "subdir1/file5.txt",
+                               "subdir2/file6.gz"};
+    String[] fileSizes = {"400", "2500", "700", "1200", "1500", "500"};
+
+    String[] visibilities = {"true", "false", "false", "true", "true", "false"};
+    String[] timeStamps = {"1234", "2345", "34567", "5434", "125", "134"};
+
+    conf.setStrings(DistributedCache.CACHE_FILES, distCacheFiles);
+    conf.setStrings(DistributedCache.CACHE_FILES_SIZES, fileSizes);
+    conf.setStrings(JobContext.CACHE_FILE_VISIBILITIES, visibilities);
+    conf.setStrings(DistributedCache.CACHE_FILES_TIMESTAMPS, timeStamps);
+
+    // local FS based dist cache file whose path contains <user>/.staging is
+    // not created on HDFS. So file size 2500 is not added to sortedFileSizes.
+    long[] sortedFileSizes = new long[] {1500, 1200, 700, 500, 400};
+    return sortedFileSizes;
+  }
+
+  /**
+   * Runs setupGenerateDistCacheData() on a new DistrbutedCacheEmulator and
+   * and returns the jobConf. Fills the array <code>sortedFileSizes</code> that
+   * can be used for validation.
+   * Validation of exit code from setupGenerateDistCacheData() is done.
+   * @param generate true if -generate option is specified
+   * @param sortedFileSizes sorted HDFS-based distributed cache file sizes
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  private JobConf runSetupGenerateDistCacheData(boolean generate,
+      long[] sortedFileSizes) throws IOException, InterruptedException {
+    Configuration conf = new Configuration();
+    long[] fileSizes = configureDummyDistCacheFiles(conf);
+    System.arraycopy(fileSizes, 0, sortedFileSizes, 0, fileSizes.length);
+
+    // Job stories of all 3 jobs will have same dist cache files in their
+    // configurations
+    final int numJobs = 3;
+    DebugJobProducer jobProducer = new DebugJobProducer(numJobs, conf);
+
+    JobConf jobConf =
+        GridmixTestUtils.mrCluster.createJobConf(new JobConf(conf));
+    Path ioPath = new Path("testSetupGenerateDistCacheData")
+                    .makeQualified(GridmixTestUtils.dfs);
+    FileSystem fs = FileSystem.get(jobConf);
+    if (fs.exists(ioPath)) {
+      fs.delete(ioPath, true);
+    }
+    FileSystem.mkdirs(fs, ioPath, new FsPermission((short)0777));
+
+    dce = createDistributedCacheEmulator(jobConf, ioPath, generate);
+    int exitCode = dce.setupGenerateDistCacheData(jobProducer);
+    int expectedExitCode = generate ? 0 : dce.MISSING_DIST_CACHE_FILES_ERROR;
+    assertEquals("setupGenerateDistCacheData failed.",
+                 expectedExitCode, exitCode);
+
+    // reset back
+    resetDistCacheConfigProperties(jobConf);
+    return jobConf;
+  }
+
+  /**
+   * Reset the config properties related to Distributed Cache in the given
+   * job configuration <code>jobConf</code>.
+   * @param jobConf job configuration
+   */
+  private void resetDistCacheConfigProperties(JobConf jobConf) {
+    // reset current/latest property names
+    jobConf.setStrings(DistributedCache.CACHE_FILES, "");
+    jobConf.setStrings(DistributedCache.CACHE_FILES_SIZES, "");
+    jobConf.setStrings(DistributedCache.CACHE_FILES_TIMESTAMPS, "");
+    jobConf.setStrings(JobContext.CACHE_FILE_VISIBILITIES, "");
+    // reset old property names
+    jobConf.setStrings("mapred.cache.files", "");
+    jobConf.setStrings("mapred.cache.files.filesizes", "");
+    jobConf.setStrings("mapred.cache.files.visibilities", "");
+    jobConf.setStrings("mapred.cache.files.timestamps", "");
+  }
+
+  /**
+   * Validate GenerateDistCacheData job if it creates dist cache files properly.
+   * @throws Exception
+   */
+  @Test
+  public void testGenerateDistCacheData() throws Exception {
+    long[] sortedFileSizes = new long[5];
+    JobConf jobConf =
+        runSetupGenerateDistCacheData(true, sortedFileSizes);
+    GridmixJob gridmixJob = new GenerateDistCacheData(jobConf);
+    Job job = gridmixJob.call();
+    assertEquals("Number of reduce tasks in GenerateDistCacheData is not 0.",
+        0, job.getNumReduceTasks());
+    assertTrue("GenerateDistCacheData job failed.",
+        job.waitForCompletion(false));
+    validateDistCacheData(jobConf, sortedFileSizes);
+  }
+
+  /**
+   *  Validate setupGenerateDistCacheData by validating
+   *  <li> permissions of the distributed cache directories and
+   *  <li> content of the generated sequence file. This includes validation of
+   *       dist cache file paths and their file sizes.
+   */
+  private void validateSetupGenDC(JobConf jobConf, long[] sortedFileSizes)
+      throws IOException, InterruptedException {
+    // build things needed for validation
+    long sumOfFileSizes = 0;
+    for (int i = 0; i < sortedFileSizes.length; i++) {
+      sumOfFileSizes += sortedFileSizes[i];
+    }
+
+    FileSystem fs = FileSystem.get(jobConf);
+    assertEquals("Number of distributed cache files to be generated is wrong.",
+        sortedFileSizes.length,
+        jobConf.getInt(GenerateDistCacheData.GRIDMIX_DISTCACHE_FILE_COUNT, -1));
+    assertEquals("Total size of dist cache files to be generated is wrong.",
+        sumOfFileSizes, jobConf.getLong(
+        GenerateDistCacheData.GRIDMIX_DISTCACHE_BYTE_COUNT, -1));
+    Path filesListFile = new Path(jobConf.get(
+        GenerateDistCacheData.GRIDMIX_DISTCACHE_FILE_LIST));
+    FileStatus stat = fs.getFileStatus(filesListFile);
+    assertEquals("Wrong permissions of dist Cache files list file "
+        + filesListFile, new FsPermission((short)0644), stat.getPermission());
+
+    InputSplit split =
+        new FileSplit(filesListFile, 0, stat.getLen(), (String[])null);
+    TaskAttemptContext taskContext =
+        MapReduceTestUtil.createDummyMapTaskAttemptContext(jobConf);
+    RecordReader<LongWritable, BytesWritable> reader =
+      new GenerateDistCacheData.GenDCDataFormat().createRecordReader(
+      split, taskContext);
+    MapContext<LongWritable, BytesWritable, NullWritable, BytesWritable>
+        mapContext = new MapContext<LongWritable, BytesWritable,
+        NullWritable, BytesWritable>(jobConf, taskContext.getTaskAttemptID(),
+        reader, null, null, MapReduceTestUtil.createDummyReporter(), split);
+    reader.initialize(split, mapContext);
+
+    // start validating setupGenerateDistCacheData
+    doValidateSetupGenDC(reader, fs, sortedFileSizes);
+  }
+
+  /**
+   *  Validate setupGenerateDistCacheData by validating
+   *  <li> permissions of the distributed cache directory and
+   *  <li> content of the generated sequence file. This includes validation of
+   *       dist cache file paths and their file sizes.
+   */
+  private void doValidateSetupGenDC(RecordReader<LongWritable, BytesWritable>
+      reader, FileSystem fs, long[] sortedFileSizes)
+      throws IOException, InterruptedException {
+
+    // Validate permissions of dist cache directory
+    Path distCacheDir = dce.getDistributedCacheDir();
+    assertEquals("Wrong permissions for distributed cache dir " + distCacheDir,
+        fs.getFileStatus(distCacheDir).getPermission()
+        .getOtherAction().and(FsAction.EXECUTE), FsAction.EXECUTE);
+
+    // Validate the content of the sequence file generated by
+    // dce.setupGenerateDistCacheData().
+    LongWritable key = new LongWritable();
+    BytesWritable val = new BytesWritable();
+    for (int i = 0; i < sortedFileSizes.length; i++) {
+      assertTrue("Number of files written to the sequence file by "
+          + "setupGenerateDistCacheData is less than the expected.",
+          reader.nextKeyValue());
+      key = reader.getCurrentKey();
+      val = reader.getCurrentValue();
+      long fileSize = key.get();
+      String file = new String(val.getBytes(), 0, val.getLength());
+
+      // Dist Cache files should be sorted based on file size.
+      assertEquals("Dist cache file size is wrong.",
+          sortedFileSizes[i], fileSize);
+
+      // Validate dist cache file path.
+
+      // parent dir of dist cache file
+      Path parent = new Path(file).getParent().makeQualified(fs);
+      // should exist in dist cache dir
+      assertTrue("Public dist cache file path is wrong.",
+          distCacheDir.equals(parent));
+    }
+  }
+
+  /**
+   *  Test if DistributedCacheEmulator's setup of GenerateDistCacheData is
+   *  working as expected.
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  @Test
+  public void testSetupGenerateDistCacheData()
+      throws IOException, InterruptedException {
+    long[] sortedFileSizes = new long[5];
+    JobConf jobConf = runSetupGenerateDistCacheData(true, sortedFileSizes);
+    validateSetupGenDC(jobConf, sortedFileSizes);
+
+    // Verify if correct exit code is seen when -generate option is missing and
+    // distributed cache files are missing in the expected path.
+    runSetupGenerateDistCacheData(false, sortedFileSizes);
+  }
+
+  /**
+   *  Create DistributedCacheEmulator object and do the initialization by
+   *  calling init() on it with dummy trace. Also configure the pseudo local FS.
+   */
+  private DistributedCacheEmulator createDistributedCacheEmulator(
+      Configuration conf, Path ioPath, boolean generate) throws IOException {
+    DistributedCacheEmulator dce =
+        new DistributedCacheEmulator(conf, ioPath);
+    JobCreator jobCreator = JobCreator.getPolicy(conf, JobCreator.LOADJOB);
+    jobCreator.setDistCacheEmulator(dce);
+    dce.init("dummytrace", jobCreator, generate);
+    return dce;
+  }
+
+  /**
+   *  Test the configuration property for disabling/enabling emulation of
+   *  distributed cache load.
+   */
+  @Test
+  public void testDistCacheEmulationConfigurability() throws IOException {
+    Configuration conf = new Configuration();
+    JobConf jobConf = GridmixTestUtils.mrCluster.createJobConf(
+        new JobConf(conf));
+    Path ioPath = new Path("testDistCacheEmulationConfigurability")
+        .makeQualified(GridmixTestUtils.dfs);
+    FileSystem fs = FileSystem.get(jobConf);
+    FileSystem.mkdirs(fs, ioPath, new FsPermission((short)0777));
+
+    // default config
+    dce = createDistributedCacheEmulator(jobConf, ioPath, false);
+    assertTrue("Default configuration of "
+        + DistributedCacheEmulator.GRIDMIX_EMULATE_DISTRIBUTEDCACHE
+        + " is wrong.", dce.shouldEmulateDistCacheLoad());
+
+    // config property set to false
+    jobConf.setBoolean(
+        DistributedCacheEmulator.GRIDMIX_EMULATE_DISTRIBUTEDCACHE, false);
+    dce = createDistributedCacheEmulator(jobConf, ioPath, false);
+    assertFalse("Disabling of emulation of distributed cache load by setting "
+        + DistributedCacheEmulator.GRIDMIX_EMULATE_DISTRIBUTEDCACHE
+        + " to false is not working.", dce.shouldEmulateDistCacheLoad());
+  }
+
+  /**
+   * Verify if DistributedCacheEmulator can configure distributed cache files
+   * for simulated job if job conf from trace had no dist cache files.
+   * @param conf configuration for the simulated job to be run
+   * @param jobConf job configuration of original cluster's job, obtained from
+   *                trace
+   * @throws IOException
+   */
+  private void validateJobConfWithOutDCFiles(Configuration conf,
+      JobConf jobConf) throws IOException {
+    // Validate if Gridmix can configure dist cache files properly if there are
+    // no HDFS-based dist cache files and localFS-based dist cache files in
+    // trace for a job.
+    dce.configureDistCacheFiles(conf, jobConf);
+    assertNull("Distributed cache files configured by GridMix is wrong.",
+               conf.get(DistributedCache.CACHE_FILES));
+    assertNull("Distributed cache files configured by Gridmix through -files "
+               + "option is wrong.", conf.get("tmpfiles"));
+  }
+
+  /**
+   * Verify if DistributedCacheEmulator can configure distributed cache files
+   * for simulated job if job conf from trace had HDFS-based dist cache files
+   * and local-FS-based dist cache files.
+   * @param conf configuration for the simulated job to be run
+   * @param jobConf job configuration of original cluster's job, obtained from
+   *                trace
+   * @throws IOException
+   */
+  private void validateJobConfWithDCFiles(Configuration conf,
+      JobConf jobConf) throws IOException {
+    long[] sortedFileSizes = configureDummyDistCacheFiles(jobConf);
+
+    // 1 local FS based dist cache file and 5 HDFS based dist cache files. So
+    // total expected dist cache files count is 6.
+    assertEquals("Gridmix is not able to extract dist cache file sizes.",
+                 6, jobConf.getStrings(DistributedCache.CACHE_FILES_SIZES).length);
+    assertEquals("Gridmix is not able to extract dist cache file visibilities.",
+                 6, jobConf.getStrings(
+                      JobContext.CACHE_FILE_VISIBILITIES).length);
+
+    dce.configureDistCacheFiles(conf, jobConf);
+
+    assertEquals("Configuring of HDFS-based dist cache files by gridmix is "
+                 + "wrong.", sortedFileSizes.length,
+                 conf.getStrings(DistributedCache.CACHE_FILES).length);
+    assertEquals("Configuring of local-FS-based dist cache files by gridmix is "
+                 + "wrong.", 1, conf.getStrings("tmpfiles").length);
+  }
+
+  /**
+   * Verify if configureDistCacheFiles() works fine when there are distributed
+   * cache files set but visibilities are not set. This is to handle history
+   * traces of older hadoop version where there are no private/public
+   * Distributed Caches.
+   * @throws IOException
+   */
+  private void validateWithOutVisibilities() throws IOException {
+    Configuration conf = new Configuration();// configuration for simulated job
+    JobConf jobConf = new JobConf();
+    String user = "user1";
+    jobConf.setUser(user);
+    String[] files = {"/tmp/hdfs1.txt", "/tmp/"+ user + "/.staging/file1"};
+    jobConf.setStrings(DistributedCache.CACHE_FILES, files);
+    jobConf.setStrings(DistributedCache.CACHE_FILES_SIZES, "12,200");
+    jobConf.setStrings(DistributedCache.CACHE_FILES_TIMESTAMPS, "56789,98345");
+    dce.configureDistCacheFiles(conf, jobConf);
+    assertEquals("Configuring of HDFS-based dist cache files by gridmix is "
+                 + "wrong.", files.length,
+                 conf.getStrings(DistributedCache.CACHE_FILES).length);
+    assertNull("Configuring of local-FS-based dist cache files by gridmix is "
+               + "wrong.", conf.get("tmpfiles"));
+  }
+
+  /**
+   * Test if Gridmix can configure config properties related to Distributed
+   * Cache properly.
+   * @throws IOException
+   */
+  @Test
+  public void testDistCacheFilesConfiguration() throws IOException {
+    Configuration conf = new Configuration();
+    JobConf jobConf = GridmixTestUtils.mrCluster.createJobConf(
+                        new JobConf(conf));
+    Path ioPath = new Path("testDistCacheEmulationConfigurability")
+                    .makeQualified(GridmixTestUtils.dfs);
+    FileSystem fs = FileSystem.get(jobConf);
+    FileSystem.mkdirs(fs, ioPath, new FsPermission((short)0777));
+
+    // default config
+    dce = createDistributedCacheEmulator(jobConf, ioPath, false);
+    assertTrue("Default configuration of "
+               + DistributedCacheEmulator.GRIDMIX_EMULATE_DISTRIBUTEDCACHE
+               + " is wrong.", dce.shouldEmulateDistCacheLoad());
+
+    // Validate if DistributedCacheEmulator can handle a JobStory with out
+    // Distributed Cache files properly.
+    validateJobConfWithOutDCFiles(conf, jobConf);
+
+    // Validate if Gridmix can configure dist cache files properly if there are
+    // HDFS-based dist cache files and localFS-based dist cache files in trace
+    // for a job.
+    validateJobConfWithDCFiles(conf, jobConf);
+    
+    // Use new JobConf as JobStory conf and check if configureDistCacheFiles()
+    // doesn't throw NPE when there are dist cache files set but visibilities
+    // are not set.
+    validateWithOutVisibilities();
+  }
+}

+ 453 - 0
src/contrib/gridmix/src/test/org/apache/hadoop/mapred/gridmix/TestGridmixMemoryEmulation.java

@@ -0,0 +1,453 @@
+/**
+ * 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.junit.Test;
+import static org.junit.Assert.*;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.DummyResourceCalculatorPlugin;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.gridmix.DebugJobProducer.MockJob;
+import org.apache.hadoop.mapred.gridmix.TestHighRamJob.DummyGridmixJob;
+import org.apache.hadoop.mapred.gridmix.TestResourceUsageEmulators.FakeProgressive;
+import org.apache.hadoop.mapred.gridmix.emulators.resourceusage.TotalHeapUsageEmulatorPlugin;
+import org.apache.hadoop.mapred.gridmix.emulators.resourceusage.TotalHeapUsageEmulatorPlugin.DefaultHeapUsageEmulator;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.util.ResourceCalculatorPlugin;
+import org.apache.hadoop.tools.rumen.ResourceUsageMetrics;
+
+/**
+ * Test Gridmix memory emulation.
+ */
+public class TestGridmixMemoryEmulation {
+  /**
+   * This is a dummy class that fakes heap usage.
+   */
+  private static class FakeHeapUsageEmulatorCore 
+  extends DefaultHeapUsageEmulator {
+    private int numCalls = 0;
+    
+    @Override
+    public void load(long sizeInMB) {
+      ++numCalls;
+      super.load(sizeInMB);
+    }
+    
+    // Get the total number of times load() was invoked
+    int getNumCalls() {
+      return numCalls;
+    }
+    
+    // Get the total number of 1mb objects stored within
+    long getHeapUsageInMB() {
+      return heapSpace.size();
+    }
+    
+    @Override
+    public void reset() {
+      // no op to stop emulate() from resetting
+    }
+    
+    /**
+     * For re-testing purpose.
+     */
+    void resetFake() {
+      numCalls = 0;
+      super.reset();
+    }
+  }
+
+  /**
+   * This is a dummy class that fakes the heap usage emulator plugin.
+   */
+  private static class FakeHeapUsageEmulatorPlugin 
+  extends TotalHeapUsageEmulatorPlugin {
+    private FakeHeapUsageEmulatorCore core;
+    
+    public FakeHeapUsageEmulatorPlugin(FakeHeapUsageEmulatorCore core) {
+      super(core);
+      this.core = core;
+    }
+    
+    @Override
+    protected long getMaxHeapUsageInMB() {
+      return Long.MAX_VALUE / ONE_MB;
+    }
+    
+    @Override
+    protected long getTotalHeapUsageInMB() {
+      return core.getHeapUsageInMB();
+    }
+  }
+  
+  /**
+   * Test {@link TotalHeapUsageEmulatorPlugin}'s core heap usage emulation 
+   * engine.
+   */
+  @Test
+  public void testHeapUsageEmulator() throws IOException {
+    FakeHeapUsageEmulatorCore heapEmulator = new FakeHeapUsageEmulatorCore();
+    
+    long testSizeInMB = 10; // 10 mb
+    long previousHeap = heapEmulator.getHeapUsageInMB();
+    heapEmulator.load(testSizeInMB);
+    long currentHeap = heapEmulator.getHeapUsageInMB();
+    
+    // check if the heap has increased by expected value
+    assertEquals("Default heap emulator failed to load 10mb", 
+                 previousHeap + testSizeInMB, currentHeap);
+    
+    // test reset
+    heapEmulator.resetFake();
+    assertEquals("Default heap emulator failed to reset", 
+                 0, heapEmulator.getHeapUsageInMB());
+  }
+
+  /**
+   * Test {@link TotalHeapUsageEmulatorPlugin}.
+   */
+  @Test
+  public void testTotalHeapUsageEmulatorPlugin() throws Exception {
+    Configuration conf = new Configuration();
+    // set the dummy resource calculator for testing
+    ResourceCalculatorPlugin monitor = new DummyResourceCalculatorPlugin();
+    long maxHeapUsage = 1024 * TotalHeapUsageEmulatorPlugin.ONE_MB; // 1GB
+    conf.setLong(DummyResourceCalculatorPlugin.MAXPMEM_TESTING_PROPERTY, 
+                 maxHeapUsage);
+    monitor.setConf(conf);
+    
+    // no buffer to be reserved
+    conf.setFloat(TotalHeapUsageEmulatorPlugin.MIN_HEAP_FREE_RATIO, 0F);
+    // only 1 call to be made per cycle
+    conf.setFloat(TotalHeapUsageEmulatorPlugin.HEAP_LOAD_RATIO, 1F);
+    long targetHeapUsageInMB = 200; // 200mb
+    
+    // fake progress indicator
+    FakeProgressive fakeProgress = new FakeProgressive();
+    
+    // fake heap usage generator
+    FakeHeapUsageEmulatorCore fakeCore = new FakeHeapUsageEmulatorCore();
+    
+    // a heap usage emulator with fake core
+    FakeHeapUsageEmulatorPlugin heapPlugin = 
+      new FakeHeapUsageEmulatorPlugin(fakeCore);
+    
+    // test with invalid or missing resource usage value
+    ResourceUsageMetrics invalidUsage = 
+      TestResourceUsageEmulators.createMetrics(0);
+    heapPlugin.initialize(conf, invalidUsage, null, null);
+    
+    // test if disabled heap emulation plugin's emulate() call is a no-operation
+    // this will test if the emulation plugin is disabled or not
+    int numCallsPre = fakeCore.getNumCalls();
+    long heapUsagePre = fakeCore.getHeapUsageInMB();
+    heapPlugin.emulate();
+    int numCallsPost = fakeCore.getNumCalls();
+    long heapUsagePost = fakeCore.getHeapUsageInMB();
+    
+    //  test if no calls are made heap usage emulator core
+    assertEquals("Disabled heap usage emulation plugin works!", 
+                 numCallsPre, numCallsPost);
+    //  test if no calls are made heap usage emulator core
+    assertEquals("Disabled heap usage emulation plugin works!", 
+                 heapUsagePre, heapUsagePost);
+    
+    // test with wrong/invalid configuration
+    Boolean failed = null;
+    invalidUsage = 
+      TestResourceUsageEmulators.createMetrics(maxHeapUsage 
+                                   + TotalHeapUsageEmulatorPlugin.ONE_MB);
+    try {
+      heapPlugin.initialize(conf, invalidUsage, monitor, null);
+      failed = false;
+    } catch (Exception e) {
+      failed = true;
+    }
+    assertNotNull("Fail case failure!", failed);
+    assertTrue("Expected failure!", failed); 
+    
+    // test with valid resource usage value
+    ResourceUsageMetrics metrics = 
+      TestResourceUsageEmulators.createMetrics(targetHeapUsageInMB 
+                                   * TotalHeapUsageEmulatorPlugin.ONE_MB);
+    
+    // test with default emulation interval
+    // in every interval, the emulator will add 100% of the expected usage 
+    // (since gridmix.emulators.resource-usage.heap.load-ratio=1)
+    // so at 10%, emulator will add 10% (difference), at 20% it will add 10% ...
+    // So to emulate 200MB, it will add
+    //   20mb + 20mb + 20mb + 20mb + .. = 200mb 
+    testEmulationAccuracy(conf, fakeCore, monitor, metrics, heapPlugin, 200, 
+                          10);
+    
+    // test with custom value for emulation interval of 20%
+    conf.setFloat(TotalHeapUsageEmulatorPlugin.HEAP_EMULATION_PROGRESS_INTERVAL,
+                  0.2F);
+    //  40mb + 40mb + 40mb + 40mb + 40mb = 200mb
+    testEmulationAccuracy(conf, fakeCore, monitor, metrics, heapPlugin, 200, 5);
+    
+    // test with custom value of free heap ratio and load ratio = 1
+    conf.setFloat(TotalHeapUsageEmulatorPlugin.HEAP_LOAD_RATIO, 1F);
+    conf.setFloat(TotalHeapUsageEmulatorPlugin.MIN_HEAP_FREE_RATIO, 0.5F);
+    //  40mb + 0mb + 80mb + 0mb + 0mb = 120mb
+    testEmulationAccuracy(conf, fakeCore, monitor, metrics, heapPlugin, 120, 2);
+    
+    // test with custom value of heap load ratio and min free heap ratio = 0
+    conf.setFloat(TotalHeapUsageEmulatorPlugin.HEAP_LOAD_RATIO, 0.5F);
+    conf.setFloat(TotalHeapUsageEmulatorPlugin.MIN_HEAP_FREE_RATIO, 0F);
+    // 20mb (call#1) + 20mb (call#1) + 20mb (call#2) + 20mb (call#2) +.. = 200mb
+    testEmulationAccuracy(conf, fakeCore, monitor, metrics, heapPlugin, 200, 
+                          10);
+    
+    // test with custom value of free heap ratio = 0.3 and load ratio = 0.5
+    conf.setFloat(TotalHeapUsageEmulatorPlugin.MIN_HEAP_FREE_RATIO, 0.25F);
+    conf.setFloat(TotalHeapUsageEmulatorPlugin.HEAP_LOAD_RATIO, 0.5F);
+    // 20mb (call#1) + 20mb (call#1) + 30mb (call#2) + 0mb (call#2) 
+    // + 30mb (call#3) + 0mb (call#3) + 35mb (call#4) + 0mb (call#4)
+    // + 37mb (call#5) + 0mb (call#5) = 162mb
+    testEmulationAccuracy(conf, fakeCore, monitor, metrics, heapPlugin, 162, 6);
+    
+    // test if emulation interval boundary is respected
+    fakeProgress = new FakeProgressive(); // initialize
+    conf.setFloat(TotalHeapUsageEmulatorPlugin.MIN_HEAP_FREE_RATIO, 0F);
+    conf.setFloat(TotalHeapUsageEmulatorPlugin.HEAP_LOAD_RATIO, 1F);
+    conf.setFloat(TotalHeapUsageEmulatorPlugin.HEAP_EMULATION_PROGRESS_INTERVAL,
+                  0.25F);
+    heapPlugin.initialize(conf, metrics, monitor, fakeProgress);
+    fakeCore.resetFake();
+    // take a snapshot after the initialization
+    long initHeapUsage = fakeCore.getHeapUsageInMB();
+    long initNumCallsUsage = fakeCore.getNumCalls();
+    // test with 0 progress
+    testEmulationBoundary(0F, fakeCore, fakeProgress, heapPlugin, initHeapUsage, 
+                          initNumCallsUsage, "[no-op, 0 progress]");
+    // test with 24% progress
+    testEmulationBoundary(0.24F, fakeCore, fakeProgress, heapPlugin, 
+                          initHeapUsage, initNumCallsUsage, 
+                          "[no-op, 24% progress]");
+    // test with 25% progress
+    testEmulationBoundary(0.25F, fakeCore, fakeProgress, heapPlugin, 
+        targetHeapUsageInMB / 4, 1, "[op, 25% progress]");
+    // test with 80% progress
+    testEmulationBoundary(0.80F, fakeCore, fakeProgress, heapPlugin, 
+        (targetHeapUsageInMB * 4) / 5, 2, "[op, 80% progress]");
+    
+    // now test if the final call with 100% progress ramps up the heap usage
+    testEmulationBoundary(1F, fakeCore, fakeProgress, heapPlugin, 
+        targetHeapUsageInMB, 3, "[op, 100% progress]");
+  }
+
+  // test whether the heap usage emulator achieves the desired target using
+  // desired calls to the underling core engine.
+  private static void testEmulationAccuracy(Configuration conf, 
+                        FakeHeapUsageEmulatorCore fakeCore,
+                        ResourceCalculatorPlugin monitor,
+                        ResourceUsageMetrics metrics,
+                        TotalHeapUsageEmulatorPlugin heapPlugin,
+                        long expectedTotalHeapUsageInMB,
+                        long expectedTotalNumCalls)
+  throws Exception {
+    FakeProgressive fakeProgress = new FakeProgressive();
+    fakeCore.resetFake();
+    heapPlugin.initialize(conf, metrics, monitor, fakeProgress);
+    int numLoops = 0;
+    while (fakeProgress.getProgress() < 1) {
+      ++numLoops;
+      float progress = numLoops / 100.0F;
+      fakeProgress.setProgress(progress);
+      heapPlugin.emulate();
+    }
+    
+    // test if the resource plugin shows the expected usage
+    assertEquals("Cumulative heap usage emulator plugin failed (total usage)!", 
+                 expectedTotalHeapUsageInMB, fakeCore.getHeapUsageInMB(), 1L);
+    // test if the resource plugin shows the expected num calls
+    assertEquals("Cumulative heap usage emulator plugin failed (num calls)!", 
+                 expectedTotalNumCalls, fakeCore.getNumCalls(), 0L);
+  }
+
+  // tests if the heap usage emulation plugin emulates only at the expected
+  // progress gaps
+  private static void testEmulationBoundary(float progress, 
+      FakeHeapUsageEmulatorCore fakeCore, FakeProgressive fakeProgress, 
+      TotalHeapUsageEmulatorPlugin heapPlugin, long expectedTotalHeapUsageInMB, 
+      long expectedTotalNumCalls, String info) throws Exception {
+    fakeProgress.setProgress(progress);
+    heapPlugin.emulate();
+    // test heap usage
+    assertEquals("Emulation interval test for heap usage failed " + info + "!", 
+                 expectedTotalHeapUsageInMB, fakeCore.getHeapUsageInMB(), 0L);
+    // test num calls
+    assertEquals("Emulation interval test for heap usage failed " + info + "!", 
+                 expectedTotalNumCalls, fakeCore.getNumCalls(), 0L);
+  }
+  
+  /**
+   * Test the specified task java heap options.
+   */
+  @SuppressWarnings("deprecation")
+  private void testJavaHeapOptions(String mapOptions, 
+      String reduceOptions, String taskOptions, String defaultMapOptions, 
+      String defaultReduceOptions, String defaultTaskOptions, 
+      String expectedMapOptions, String expectedReduceOptions, 
+      String expectedTaskOptions) throws Exception {
+    Configuration simulatedConf = new Configuration(false);
+//    // reset the configuration parameters
+//    simulatedConf.set(JobConf.MAPRED_MAP_TASK_JAVA_OPTS, "");
+//    simulatedConf.set(JobConf.MAPRED_REDUCE_TASK_JAVA_OPTS, "");
+//    simulatedConf.set(JobConf.MAPRED_TASK_JAVA_OPTS, "");
+    
+    // set the default map task options
+    if (defaultMapOptions != null) {
+      simulatedConf.set(JobConf.MAPRED_MAP_TASK_JAVA_OPTS, defaultMapOptions);
+    }
+    // set the default reduce task options
+    if (defaultReduceOptions != null) {
+      simulatedConf.set(JobConf.MAPRED_REDUCE_TASK_JAVA_OPTS,
+                        defaultReduceOptions);
+    }
+    // set the default task options
+    if (defaultTaskOptions != null) {
+      simulatedConf.set(JobConf.MAPRED_TASK_JAVA_OPTS, defaultTaskOptions);
+    }
+    
+    Configuration originalConf = new Configuration(false);
+//    // reset the configuration parameters
+//    originalConf.set(JobConf.MAPRED_MAP_TASK_JAVA_OPTS, "");
+//    originalConf.set(JobConf.MAPRED_REDUCE_TASK_JAVA_OPTS, "");
+//    originalConf.set(JobConf.MAPRED_TASK_JAVA_OPTS, "");
+    
+    // set the map task options
+    if (mapOptions != null) {
+      originalConf.set(JobConf.MAPRED_MAP_TASK_JAVA_OPTS, mapOptions);
+    }
+    // set the reduce task options
+    if (reduceOptions != null) {
+      originalConf.set(JobConf.MAPRED_REDUCE_TASK_JAVA_OPTS, reduceOptions);
+    }
+    // set the task options
+    if (taskOptions != null) {
+      originalConf.set(JobConf.MAPRED_TASK_JAVA_OPTS, taskOptions);
+    }
+    
+    // configure the task jvm's heap options
+    GridmixJob.configureTaskJVMOptions(originalConf, simulatedConf);
+    
+    assertEquals("Map heap options mismatch!", expectedMapOptions, 
+                 simulatedConf.get(JobConf.MAPRED_MAP_TASK_JAVA_OPTS));
+    assertEquals("Reduce heap options mismatch!", expectedReduceOptions, 
+                 simulatedConf.get(JobConf.MAPRED_REDUCE_TASK_JAVA_OPTS));
+    assertEquals("Task heap options mismatch!", expectedTaskOptions, 
+                 simulatedConf.get(JobConf.MAPRED_TASK_JAVA_OPTS));
+  }
+  
+  /**
+   * Test task-level java heap options configuration in {@link GridmixJob}.
+   */
+  @Test
+  public void testJavaHeapOptions() throws Exception {
+    // test missing opts
+    testJavaHeapOptions(null, null, null, null, null, null, null, null, 
+                        null);
+    
+    // test original heap opts and missing default opts
+    testJavaHeapOptions("-Xms10m", "-Xms20m", "-Xms30m", null, null, null,
+                        null, null, null);
+    
+    // test missing opts with default opts
+    testJavaHeapOptions(null, null, null, "-Xms10m", "-Xms20m", "-Xms30m",
+                        "-Xms10m", "-Xms20m", "-Xms30m");
+    
+    // test empty option
+    testJavaHeapOptions("", "", "", null, null, null, null, null, null);
+    
+    // test empty default option and no original heap options
+    testJavaHeapOptions(null, null, null, "", "", "", "", "", "");
+    
+    // test empty opts and default opts
+    testJavaHeapOptions("", "", "", "-Xmx10m -Xms1m", "-Xmx50m -Xms2m", 
+                        "-Xms2m -Xmx100m", "-Xmx10m -Xms1m", "-Xmx50m -Xms2m", 
+                        "-Xms2m -Xmx100m");
+    
+    // test custom heap opts with no default opts
+    testJavaHeapOptions("-Xmx10m", "-Xmx20m", "-Xmx30m", null, null, null,
+                        "-Xmx10m", "-Xmx20m", "-Xmx30m");
+    
+    // test heap opts with default opts (multiple value)
+    testJavaHeapOptions("-Xms5m -Xmx200m", "-Xms15m -Xmx300m", 
+                        "-Xms25m -Xmx50m", "-XXabc", "-XXxyz", "-XXdef", 
+                        "-XXabc -Xmx200m", "-XXxyz -Xmx300m", "-XXdef -Xmx50m");
+    
+    // test heap opts with default opts (duplication of -Xmx)
+    testJavaHeapOptions("-Xms5m -Xmx200m", "-Xms15m -Xmx300m", 
+                        "-Xms25m -Xmx50m", "-XXabc -Xmx500m", "-XXxyz -Xmx600m",
+                        "-XXdef -Xmx700m", "-XXabc -Xmx200m", "-XXxyz -Xmx300m",
+                        "-XXdef -Xmx50m");
+    
+    // test heap opts with default opts (single value)
+    testJavaHeapOptions("-Xmx10m", "-Xmx20m", "-Xmx50m", "-Xms2m", 
+                        "-Xms3m", "-Xms5m", "-Xms2m -Xmx10m", "-Xms3m -Xmx20m",
+                        "-Xms5m -Xmx50m");
+    
+    // test heap opts with default opts (duplication of -Xmx)
+    testJavaHeapOptions("-Xmx10m", "-Xmx20m", "-Xmx50m", "-Xmx2m", 
+                        "-Xmx3m", "-Xmx5m", "-Xmx10m", "-Xmx20m", "-Xmx50m");
+  }
+  
+  /**
+   * Test disabled task heap options configuration in {@link GridmixJob}.
+   */
+  @Test
+  @SuppressWarnings("deprecation")
+  public void testJavaHeapOptionsDisabled() throws Exception {
+    Configuration gridmixConf = new Configuration();
+    gridmixConf.setBoolean(GridmixJob.GRIDMIX_TASK_JVM_OPTIONS_ENABLE, false);
+    
+    // set the default values of simulated job
+    gridmixConf.set(JobConf.MAPRED_MAP_TASK_JAVA_OPTS, "-Xmx1m");
+    gridmixConf.set(JobConf.MAPRED_REDUCE_TASK_JAVA_OPTS, "-Xmx2m");
+    gridmixConf.set(JobConf.MAPRED_TASK_JAVA_OPTS, "-Xmx3m");
+    
+    // set the default map and reduce task options for original job
+    final JobConf originalConf = new JobConf();
+    originalConf.set(JobConf.MAPRED_MAP_TASK_JAVA_OPTS, "-Xmx10m");
+    originalConf.set(JobConf.MAPRED_REDUCE_TASK_JAVA_OPTS, "-Xmx20m");
+    originalConf.set(JobConf.MAPRED_TASK_JAVA_OPTS, "-Xmx30m");
+    
+    // define a mock job
+    MockJob story = new MockJob(originalConf) {
+      public JobConf getJobConf() {
+        return originalConf;
+      }
+    };
+    
+    GridmixJob job = new DummyGridmixJob(gridmixConf, story);
+    Job simulatedJob = job.getJob();
+    Configuration simulatedConf = simulatedJob.getConfiguration();
+    
+    assertEquals("Map heap options works when disabled!", "-Xmx1m", 
+                 simulatedConf.get(JobConf.MAPRED_MAP_TASK_JAVA_OPTS));
+    assertEquals("Reduce heap options works when disabled!", "-Xmx2m", 
+                 simulatedConf.get(JobConf.MAPRED_REDUCE_TASK_JAVA_OPTS));
+    assertEquals("Task heap options works when disabled!", "-Xmx3m", 
+                 simulatedConf.get(JobConf.MAPRED_TASK_JAVA_OPTS));
+  }
+}

+ 3 - 2
src/contrib/gridmix/src/test/org/apache/hadoop/mapred/gridmix/TestGridmixRecord.java

@@ -176,7 +176,8 @@ public class TestGridmixRecord {
       a.setReduceOutputBytes(out_bytes);
       a.setReduceOutputBytes(out_bytes);
       final int min = WritableUtils.getVIntSize(in_rec)
       final int min = WritableUtils.getVIntSize(in_rec)
                     + WritableUtils.getVIntSize(out_rec)
                     + WritableUtils.getVIntSize(out_rec)
-                    + WritableUtils.getVIntSize(out_bytes);
+                    + WritableUtils.getVIntSize(out_bytes)
+                    + WritableUtils.getVIntSize(0);
       assertEquals(min + 2, a.fixedBytes()); // meta + vint min
       assertEquals(min + 2, a.fixedBytes()); // meta + vint min
       final int size = r.nextInt(1024) + a.fixedBytes() + 1;
       final int size = r.nextInt(1024) + a.fixedBytes() + 1;
       setSerialize(a, r.nextLong(), size, out);
       setSerialize(a, r.nextLong(), size, out);
@@ -207,7 +208,7 @@ public class TestGridmixRecord {
 
 
   @Test
   @Test
   public void testKeySpec() throws Exception {
   public void testKeySpec() throws Exception {
-    final int min = 5;
+    final int min = 6;
     final int max = 300;
     final int max = 300;
     final GridmixKey a = new GridmixKey(GridmixKey.REDUCE_SPEC, 1, 0L);
     final GridmixKey a = new GridmixKey(GridmixKey.REDUCE_SPEC, 1, 0L);
     final GridmixKey b = new GridmixKey(GridmixKey.REDUCE_SPEC, 1, 0L);
     final GridmixKey b = new GridmixKey(GridmixKey.REDUCE_SPEC, 1, 0L);

+ 195 - 54
src/contrib/gridmix/src/test/org/apache/hadoop/mapred/gridmix/TestGridmixSubmission.java

@@ -23,6 +23,8 @@ import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.mapred.Counters;
 import org.apache.hadoop.mapred.Counters;
@@ -34,6 +36,7 @@ import org.apache.hadoop.mapred.TaskReport;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.TaskType;
 import org.apache.hadoop.mapreduce.TaskType;
 import org.apache.hadoop.tools.rumen.JobStory;
 import org.apache.hadoop.tools.rumen.JobStory;
+import org.apache.hadoop.tools.rumen.JobStoryProducer;
 import org.apache.hadoop.tools.rumen.TaskInfo;
 import org.apache.hadoop.tools.rumen.TaskInfo;
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.log4j.Level;
 import org.apache.log4j.Level;
@@ -41,13 +44,16 @@ import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.Test;
 
 
+import java.io.InputStream;
 import java.io.IOException;
 import java.io.IOException;
+import java.text.DecimalFormat;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Arrays;
 import java.util.HashMap;
 import java.util.HashMap;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.LinkedBlockingQueue;
+import java.util.zip.GZIPInputStream;
 
 
 import static org.apache.hadoop.mapred.Task.Counter.MAP_INPUT_RECORDS;
 import static org.apache.hadoop.mapred.Task.Counter.MAP_INPUT_RECORDS;
 import static org.apache.hadoop.mapred.Task.Counter.MAP_OUTPUT_BYTES;
 import static org.apache.hadoop.mapred.Task.Counter.MAP_OUTPUT_BYTES;
@@ -103,17 +109,10 @@ public class TestGridmixSubmission {
         GridmixTestUtils.mrCluster.createJobConf());
         GridmixTestUtils.mrCluster.createJobConf());
       for (Job job : succeeded) {
       for (Job job : succeeded) {
         final String jobname = job.getJobName();
         final String jobname = job.getJobName();
-        if ("GRIDMIX_GENDATA".equals(jobname)) {
-          if (!job.getConfiguration().getBoolean(
-            GridmixJob.GRIDMIX_USE_QUEUE_IN_TRACE, true)) {
-            assertEquals(
-              " Improper queue for " + job.getJobName(),
-              job.getConfiguration().get("mapred.job.queue.name"), "q1");
-          } else {
-            assertEquals(
-              " Improper queue for " + job.getJobName(),
-              job.getConfiguration().get("mapred.job.queue.name"), "default");
-          }
+        final String jobName = job.getJobName();
+        Configuration conf = job.getConfiguration();
+        if (GenerateData.JOB_NAME.equals(jobName)) {
+          verifyQueue(conf, jobName);
           final Path in = new Path("foo").makeQualified(GridmixTestUtils.dfs);
           final Path in = new Path("foo").makeQualified(GridmixTestUtils.dfs);
           final Path out = new Path("/gridmix").makeQualified(GridmixTestUtils.dfs);
           final Path out = new Path("/gridmix").makeQualified(GridmixTestUtils.dfs);
           final ContentSummary generated = GridmixTestUtils.dfs.getContentSummary(in);
           final ContentSummary generated = GridmixTestUtils.dfs.getContentSummary(in);
@@ -123,37 +122,55 @@ public class TestGridmixSubmission {
           FileStatus[] outstat = GridmixTestUtils.dfs.listStatus(out);
           FileStatus[] outstat = GridmixTestUtils.dfs.listStatus(out);
           assertEquals("Mismatched job count", NJOBS, outstat.length);
           assertEquals("Mismatched job count", NJOBS, outstat.length);
           continue;
           continue;
+        } else if (GenerateDistCacheData.JOB_NAME.equals(jobName)) {
+          verifyQueue(conf, jobName);
+          continue;
         }
         }
-        
-        if (!job.getConfiguration().getBoolean(
-          GridmixJob.GRIDMIX_USE_QUEUE_IN_TRACE, true)) {
-          assertEquals(" Improper queue for  " + job.getJobName() + " " ,
-          job.getConfiguration().get("mapred.job.queue.name"),"q1" );
+
+        if (!conf.getBoolean(
+            GridmixJob.GRIDMIX_USE_QUEUE_IN_TRACE, true)) {
+          assertEquals(" Improper queue for  " + jobName + " " ,
+              conf.get("mapred.queue.name"), "q1" );
         } else {
         } else {
-          assertEquals(
-            " Improper queue for  " + job.getJobName() + " ",
-            job.getConfiguration().get("mapred.job.queue.name"), sub.get(
-              job.getConfiguration().get(GridmixJob.ORIGNAME)).getQueueName());
+          assertEquals(" Improper queue for  " + jobName + " ",
+              conf.get("mapred.queue.name"),
+              sub.get(conf.get(Gridmix.ORIGINAL_JOB_ID)).getQueueName());
         }
         }
 
 
-        final JobStory spec =
-          sub.get(job.getConfiguration().get(GridmixJob.ORIGNAME));
-        assertNotNull("No spec for " + job.getJobName(), spec);
-        assertNotNull("No counters for " + job.getJobName(), job.getCounters());
-        final String specname = spec.getName();
-        final FileStatus stat = GridmixTestUtils.dfs.getFileStatus(new Path(
-          GridmixTestUtils.DEST, "" +
-              Integer.valueOf(specname.substring(specname.length() - 5))));
-        assertEquals("Wrong owner for " + job.getJobName(), spec.getUser(),
-            stat.getOwner());
-
+        final String originalJobId = conf.get(Gridmix.ORIGINAL_JOB_ID);
+        final JobStory spec = sub.get(originalJobId);
+        assertNotNull("No spec for " + jobName, spec);
+        assertNotNull("No counters for " + jobName, job.getCounters());
+        final String originalJobName = spec.getName();
+        System.out.println("originalJobName=" + originalJobName
+            + ";GridmixJobName=" + jobName + ";originalJobID=" + originalJobId);
+        assertTrue("Original job name is wrong.", originalJobName.equals(
+            conf.get(Gridmix.ORIGINAL_JOB_NAME)));
+
+        // Gridmix job seqNum contains 6 digits
+        int seqNumLength = 6;
+        String jobSeqNum = new DecimalFormat("000000").format(
+            conf.getInt(GridmixJob.GRIDMIX_JOB_SEQ, -1));
+        // Original job name is of the format MOCKJOB<6 digit sequence number>
+        // because MockJob jobNames are of this format.
+        assertTrue(originalJobName.substring(
+            originalJobName.length() - seqNumLength).equals(jobSeqNum));
+
+        assertTrue("Gridmix job name is not in the expected format.",
+            jobName.equals(
+                GridmixJob.JOB_NAME_PREFIX + jobSeqNum));
+        final FileStatus stat =
+          GridmixTestUtils.dfs.getFileStatus(
+            new Path(GridmixTestUtils.DEST, "" + Integer.valueOf(jobSeqNum)));
+        assertEquals("Wrong owner for " + jobName, spec.getUser(),
+                     stat.getOwner());
         final int nMaps = spec.getNumberMaps();
         final int nMaps = spec.getNumberMaps();
         final int nReds = spec.getNumberReduces();
         final int nReds = spec.getNumberReduces();
 
 
         // TODO Blocked by MAPREDUCE-118
         // TODO Blocked by MAPREDUCE-118
         if (true) return;
         if (true) return;
         // TODO
         // TODO
-        System.out.println(jobname + ": " + nMaps + "/" + nReds);
+        System.out.println(jobName + ": " + nMaps + "/" + nReds);
         final TaskReport[] mReports =
         final TaskReport[] mReports =
           client.getMapTaskReports(JobID.downgrade(job.getJobID()));
           client.getMapTaskReports(JobID.downgrade(job.getJobID()));
         assertEquals("Mismatched map count", nMaps, mReports.length);
         assertEquals("Mismatched map count", nMaps, mReports.length);
@@ -168,6 +185,18 @@ public class TestGridmixSubmission {
       }
       }
     }
     }
 
 
+    // Verify if correct job queue is used
+    private void verifyQueue(Configuration conf, String jobName) {
+      if (!conf.getBoolean(
+          GridmixJob.GRIDMIX_USE_QUEUE_IN_TRACE, true)) {
+        assertEquals(" Improper queue for " + jobName,
+            conf.get("mapred.job.queue.name"), "q1");
+      } else {
+        assertEquals(" Improper queue for " + jobName,
+            conf.get("mapred.job.queue.name"), "default");
+      }
+    }
+
     public void check(final TaskType type, Job job, JobStory spec,
     public void check(final TaskType type, Job job, JobStory spec,
           final TaskReport[] runTasks,
           final TaskReport[] runTasks,
           long extraInputBytes, int extraInputRecords,
           long extraInputBytes, int extraInputRecords,
@@ -325,19 +354,118 @@ public class TestGridmixSubmission {
     }
     }
   }
   }
 
 
+  /**
+   * Verifies that the given {@code JobStory} corresponds to the checked-in
+   * WordCount {@code JobStory}. The verification is effected via JUnit
+   * assertions.
+   *
+   * @param js the candidate JobStory.
+   */
+  private void verifyWordCountJobStory(JobStory js) {
+    assertNotNull("Null JobStory", js);
+    String expectedJobStory = "WordCount:johndoe:default:1285322645148:3:1";
+    String actualJobStory = js.getName() + ":" + js.getUser() + ":"
+      + js.getQueueName() + ":" + js.getSubmissionTime() + ":"
+      + js.getNumberMaps() + ":" + js.getNumberReduces();
+    assertEquals("Unexpected JobStory", expectedJobStory, actualJobStory);
+  }
+
+  /**
+   * Expands a file compressed using {@code gzip}.
+   *
+   * @param fs the {@code FileSystem} corresponding to the given
+   * file.
+   *
+   * @param in the path to the compressed file.
+   *
+   * @param out the path to the uncompressed output.
+   *
+   * @throws Exception if there was an error during the operation.
+   */
+  private void expandGzippedTrace(FileSystem fs, Path in, Path out)
+    throws Exception {
+    byte[] buff = new byte[4096];
+    GZIPInputStream gis = new GZIPInputStream(fs.open(in));
+    FSDataOutputStream fsdos = fs.create(out);
+    int numRead;
+    while ((numRead = gis.read(buff, 0, buff.length)) != -1) {
+      fsdos.write(buff, 0, numRead);
+    }
+    gis.close();
+    fsdos.close();
+  }
+
+  /**
+   * Tests the reading of traces in GridMix3. These traces are generated
+   * by Rumen and are in the JSON format. The traces can optionally be
+   * compressed and uncompressed traces can also be passed to GridMix3 via
+   * its standard input stream. The testing is effected via JUnit assertions.
+   *
+   * @throws Exception if there was an error.
+   */
+  @Test
+  public void testTraceReader() throws Exception {
+    Configuration conf = new Configuration();
+    FileSystem lfs = FileSystem.getLocal(conf);
+    Path rootInputDir = new Path(System.getProperty("src.test.data"));
+    rootInputDir
+      = rootInputDir.makeQualified(lfs.getUri(), lfs.getWorkingDirectory());
+    Path rootTempDir
+      = new Path(System.getProperty("test.build.data",
+        System.getProperty("java.io.tmpdir")), "testTraceReader");
+    rootTempDir
+      = rootTempDir.makeQualified(lfs.getUri(), lfs.getWorkingDirectory());
+    Path inputFile = new Path(rootInputDir, "wordcount.json.gz");
+    Path tempFile = new Path(rootTempDir, "gridmix3-wc.json");
+
+    InputStream origStdIn = System.in;
+    InputStream tmpIs = null;
+    try {
+      DebugGridmix dgm = new DebugGridmix();
+      JobStoryProducer jsp
+        = dgm.createJobStoryProducer(inputFile.toString(), conf);
+
+      System.out.println("Verifying JobStory from compressed trace...");
+      verifyWordCountJobStory(jsp.getNextJob());
+
+      expandGzippedTrace(lfs, inputFile, tempFile);
+      jsp = dgm.createJobStoryProducer(tempFile.toString(), conf);
+      System.out.println("Verifying JobStory from uncompressed trace...");
+      verifyWordCountJobStory(jsp.getNextJob());
+
+      tmpIs = lfs.open(tempFile);
+      System.setIn(tmpIs);
+      System.out.println("Verifying JobStory from trace in standard input...");
+      jsp = dgm.createJobStoryProducer("-", conf);
+      verifyWordCountJobStory(jsp.getNextJob());
+    } finally {
+      System.setIn(origStdIn);
+      if (tmpIs != null) {
+	tmpIs.close();
+      }
+      lfs.delete(rootTempDir, true);
+    }
+  }
+
   @Test
   @Test
   public void testReplaySubmit() throws Exception {
   public void testReplaySubmit() throws Exception {
     policy = GridmixJobSubmissionPolicy.REPLAY;
     policy = GridmixJobSubmissionPolicy.REPLAY;
     System.out.println(" Replay started at " + System.currentTimeMillis());
     System.out.println(" Replay started at " + System.currentTimeMillis());
-    doSubmission(false);
+    doSubmission(false, false);
     System.out.println(" Replay ended at " + System.currentTimeMillis());
     System.out.println(" Replay ended at " + System.currentTimeMillis());
+
+    System.out.println(" Replay started with default output path at time "
+        + System.currentTimeMillis());
+    doSubmission(false, true);
+    System.out.println(" Replay ended with default output path at time "
+        + System.currentTimeMillis());
   }
   }
   
   
   @Test
   @Test
   public void testStressSubmit() throws Exception {
   public void testStressSubmit() throws Exception {
     policy = GridmixJobSubmissionPolicy.STRESS;
     policy = GridmixJobSubmissionPolicy.STRESS;
     System.out.println(" Stress started at " + System.currentTimeMillis());
     System.out.println(" Stress started at " + System.currentTimeMillis());
-    doSubmission(false);
+    doSubmission(false, false);
     System.out.println(" Stress ended at " + System.currentTimeMillis());
     System.out.println(" Stress ended at " + System.currentTimeMillis());
   }
   }
 
 
@@ -346,7 +474,7 @@ public class TestGridmixSubmission {
     policy = GridmixJobSubmissionPolicy.STRESS;
     policy = GridmixJobSubmissionPolicy.STRESS;
     System.out.println(
     System.out.println(
       " Stress with default q started at " + System.currentTimeMillis());
       " Stress with default q started at " + System.currentTimeMillis());
-    doSubmission(true);
+    doSubmission(true, false);
     System.out.println(
     System.out.println(
       " Stress with default q ended at " + System.currentTimeMillis());
       " Stress with default q ended at " + System.currentTimeMillis());
   }
   }
@@ -355,26 +483,39 @@ public class TestGridmixSubmission {
   public void testSerialSubmit() throws Exception {
   public void testSerialSubmit() throws Exception {
     policy = GridmixJobSubmissionPolicy.SERIAL;
     policy = GridmixJobSubmissionPolicy.SERIAL;
     System.out.println("Serial started at " + System.currentTimeMillis());
     System.out.println("Serial started at " + System.currentTimeMillis());
-    doSubmission(false);
+    doSubmission(false, false);
     System.out.println("Serial ended at " + System.currentTimeMillis());
     System.out.println("Serial ended at " + System.currentTimeMillis());
   }
   }
 
 
-  private void doSubmission(boolean useDefaultQueue) throws Exception {
+  private void doSubmission(boolean useDefaultQueue,
+      boolean defaultOutputPath) throws Exception {
     final Path in = new Path("foo").makeQualified(GridmixTestUtils.dfs);
     final Path in = new Path("foo").makeQualified(GridmixTestUtils.dfs);
     final Path out = GridmixTestUtils.DEST.makeQualified(GridmixTestUtils.dfs);
     final Path out = GridmixTestUtils.DEST.makeQualified(GridmixTestUtils.dfs);
     final Path root = new Path("/user");
     final Path root = new Path("/user");
     Configuration conf = null;
     Configuration conf = null;
+
     try{
     try{
-    final String[] argv = {
-      "-D" + FilePool.GRIDMIX_MIN_FILE + "=0",
-      "-D" + Gridmix.GRIDMIX_OUT_DIR + "=" + out,
-      "-D" + Gridmix.GRIDMIX_USR_RSV + "=" + EchoUserResolver.class.getName(),
-      "-generate", String.valueOf(GENDATA) + "m",
-      in.toString(),
-      "-" // ignored by DebugGridmix
-    };
-    DebugGridmix client = new DebugGridmix();
-    conf = new Configuration();
+      ArrayList<String> argsList = new ArrayList<String>();
+
+      argsList.add("-D" + FilePool.GRIDMIX_MIN_FILE + "=0");
+      argsList.add("-D" + Gridmix.GRIDMIX_USR_RSV + "="
+          + EchoUserResolver.class.getName());
+
+      // Set the config property gridmix.output.directory only if
+      // defaultOutputPath is false. If defaultOutputPath is true, then
+      // let us allow gridmix to use the path foo/gridmix/ as output dir.
+      if (!defaultOutputPath) {
+        argsList.add("-D" + Gridmix.GRIDMIX_OUT_DIR + "=" + out);
+      }
+      argsList.add("-generate");
+      argsList.add(String.valueOf(GENDATA) + "m");
+      argsList.add(in.toString());
+      argsList.add("-"); // ignored by DebugGridmix
+
+      String[] argv = argsList.toArray(new String[argsList.size()]);
+
+      DebugGridmix client = new DebugGridmix();
+      conf = new Configuration();
       conf.setEnum(GridmixJobSubmissionPolicy.JOB_SUBMISSION_POLICY,policy);
       conf.setEnum(GridmixJobSubmissionPolicy.JOB_SUBMISSION_POLICY,policy);
       if (useDefaultQueue) {
       if (useDefaultQueue) {
         conf.setBoolean(GridmixJob.GRIDMIX_USE_QUEUE_IN_TRACE, false);
         conf.setBoolean(GridmixJob.GRIDMIX_USE_QUEUE_IN_TRACE, false);
@@ -382,13 +523,13 @@ public class TestGridmixSubmission {
       } else {
       } else {
         conf.setBoolean(GridmixJob.GRIDMIX_USE_QUEUE_IN_TRACE, true);
         conf.setBoolean(GridmixJob.GRIDMIX_USE_QUEUE_IN_TRACE, true);
       }
       }
-    conf = GridmixTestUtils.mrCluster.createJobConf(new JobConf(conf));
-    // allow synthetic users to create home directories
-    GridmixTestUtils.dfs.mkdirs(root, new FsPermission((short)0777));
-    GridmixTestUtils.dfs.setPermission(root, new FsPermission((short)0777));
-    int res = ToolRunner.run(conf, client, argv);
-    assertEquals("Client exited with nonzero status", 0, res);
-    client.checkMonitor();
+      conf = GridmixTestUtils.mrCluster.createJobConf(new JobConf(conf));
+      // allow synthetic users to create home directories
+      GridmixTestUtils.dfs.mkdirs(root, new FsPermission((short)0777));
+      GridmixTestUtils.dfs.setPermission(root, new FsPermission((short)0777));
+      int res = ToolRunner.run(conf, client, argv);
+      assertEquals("Client exited with nonzero status", 0, res);
+      client.checkMonitor();
      } catch (Exception e) {
      } catch (Exception e) {
        e.printStackTrace();
        e.printStackTrace();
      } finally {
      } finally {

+ 371 - 0
src/contrib/gridmix/src/test/org/apache/hadoop/mapred/gridmix/TestGridmixSummary.java

@@ -0,0 +1,371 @@
+/**
+ * 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 static org.junit.Assert.*;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.mapred.JobClient;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.UtilsForTests;
+import org.apache.hadoop.mapred.gridmix.GenerateData.DataStatistics;
+import org.apache.hadoop.mapred.gridmix.Statistics.ClusterStats;
+import org.apache.hadoop.mapred.gridmix.Statistics.JobStats;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.tools.rumen.JobStory;
+import org.apache.hadoop.tools.rumen.JobStoryProducer;
+import org.junit.Test;
+
+/**
+ * Test {@link ExecutionSummarizer} and {@link ClusterSummarizer}.
+ */
+public class TestGridmixSummary {
+  
+  /**
+   * Test {@link DataStatistics}.
+   */
+  @Test
+  public void testDataStatistics() throws Exception {
+    // test data-statistics getters with compression enabled
+    DataStatistics stats = new DataStatistics(10, 2, true);
+    assertEquals("Data size mismatch", 10, stats.getDataSize());
+    assertEquals("Num files mismatch", 2, stats.getNumFiles());
+    assertTrue("Compression configuration mismatch", stats.isDataCompressed());
+    
+    // test data-statistics getters with compression disabled
+    stats = new DataStatistics(100, 5, false);
+    assertEquals("Data size mismatch", 100, stats.getDataSize());
+    assertEquals("Num files mismatch", 5, stats.getNumFiles());
+    assertFalse("Compression configuration mismatch", stats.isDataCompressed());
+    
+    // test publish data stats
+    Configuration conf = new Configuration();
+    Path rootTempDir = new Path(System.getProperty("test.build.data", "/tmp"));
+    Path testDir = new Path(rootTempDir, "testDataStatistics");
+    FileSystem fs = testDir.getFileSystem(conf);
+    fs.delete(testDir, true);
+    Path testInputDir = new Path(testDir, "test");
+    fs.mkdirs(testInputDir);
+    
+    // test empty folder (compression = true)
+    CompressionEmulationUtil.setCompressionEmulationEnabled(conf, true);
+    Boolean failed = null;
+    try {
+      GenerateData.publishDataStatistics(testInputDir, 1024L, conf);
+      failed = false;
+    } catch (RuntimeException e) {
+      failed = true;
+    }
+    assertNotNull("Expected failure!", failed);
+    assertTrue("Compression data publishing error", failed);
+    
+    // test with empty folder (compression = off)
+    CompressionEmulationUtil.setCompressionEmulationEnabled(conf, false);
+    stats = GenerateData.publishDataStatistics(testInputDir, 1024L, conf);
+    assertEquals("Data size mismatch", 0, stats.getDataSize());
+    assertEquals("Num files mismatch", 0, stats.getNumFiles());
+    assertFalse("Compression configuration mismatch", stats.isDataCompressed());
+    
+    // test with some plain input data (compression = off)
+    CompressionEmulationUtil.setCompressionEmulationEnabled(conf, false);
+    Path inputDataFile = new Path(testInputDir, "test");
+    long size = 
+      UtilsForTests.createTmpFileDFS(fs, inputDataFile, 
+          FsPermission.createImmutable((short)777), "hi hello bye").size();
+    stats = GenerateData.publishDataStatistics(testInputDir, -1, conf);
+    assertEquals("Data size mismatch", size, stats.getDataSize());
+    assertEquals("Num files mismatch", 1, stats.getNumFiles());
+    assertFalse("Compression configuration mismatch", stats.isDataCompressed());
+    
+    // test with some plain input data (compression = on)
+    CompressionEmulationUtil.setCompressionEmulationEnabled(conf, true);
+    failed = null;
+    try {
+      GenerateData.publishDataStatistics(testInputDir, 1234L, conf);
+      failed = false;
+    } catch (RuntimeException e) {
+      failed = true;
+    }
+    assertNotNull("Expected failure!", failed);
+    assertTrue("Compression data publishing error", failed);
+    
+    // test with some compressed input data (compression = off)
+    CompressionEmulationUtil.setCompressionEmulationEnabled(conf, false);
+    fs.delete(inputDataFile, false);
+    inputDataFile = new Path(testInputDir, "test.gz");
+    size = 
+      UtilsForTests.createTmpFileDFS(fs, inputDataFile, 
+          FsPermission.createImmutable((short)777), "hi hello").size();
+    stats =  GenerateData.publishDataStatistics(testInputDir, 1234L, conf);
+    assertEquals("Data size mismatch", size, stats.getDataSize());
+    assertEquals("Num files mismatch", 1, stats.getNumFiles());
+    assertFalse("Compression configuration mismatch", stats.isDataCompressed());
+    
+    // test with some compressed input data (compression = on)
+    CompressionEmulationUtil.setCompressionEmulationEnabled(conf, true);
+    stats = GenerateData.publishDataStatistics(testInputDir, 1234L, conf);
+    assertEquals("Data size mismatch", size, stats.getDataSize());
+    assertEquals("Num files mismatch", 1, stats.getNumFiles());
+    assertTrue("Compression configuration mismatch", stats.isDataCompressed());
+  }
+  
+  /**
+   * A fake {@link JobFactory}.
+   */
+  @SuppressWarnings("unchecked")
+  private static class FakeJobFactory extends JobFactory {
+    /**
+     * A fake {@link JobStoryProducer} for {@link FakeJobFactory}.
+     */
+    private static class FakeJobStoryProducer implements JobStoryProducer {
+      @Override
+      public void close() throws IOException {
+      }
+
+      @Override
+      public JobStory getNextJob() throws IOException {
+        return null;
+      }
+    }
+    
+    FakeJobFactory(Configuration conf) {
+      super(null, new FakeJobStoryProducer(), null, conf, null, null);
+    }
+    
+    @Override
+    public void update(Object item) {
+    }
+    
+    @Override
+    protected Thread createReaderThread() {
+      return null;
+    }
+  }
+  
+  /**
+   * Test {@link ExecutionSummarizer}.
+   */
+  @Test
+  @SuppressWarnings("unchecked")
+  public void testExecutionSummarizer() throws IOException {
+    Configuration conf = new Configuration();
+    
+    ExecutionSummarizer es = new ExecutionSummarizer();
+    assertEquals("ExecutionSummarizer init failed", 
+                 Summarizer.NA, es.getCommandLineArgsString());
+    
+    long startTime = System.currentTimeMillis();
+    // test configuration parameters
+    String[] initArgs = new String[] {"-Xmx20m", "-Dtest.args='test'"};
+    es = new ExecutionSummarizer(initArgs);
+    
+    assertEquals("ExecutionSummarizer init failed", 
+                 "-Xmx20m -Dtest.args='test'", 
+                 es.getCommandLineArgsString());
+    
+    // test start time
+    assertTrue("Start time mismatch", es.getStartTime() >= startTime);
+    assertTrue("Start time mismatch", 
+               es.getStartTime() <= System.currentTimeMillis());
+    
+    // test start() of ExecutionSummarizer
+    es.update(null);
+    assertEquals("ExecutionSummarizer init failed", 0, 
+                 es.getSimulationStartTime());
+    testExecutionSummarizer(0, 0, 0, 0, 0, 0, es);
+    
+    long simStartTime = System.currentTimeMillis();
+    es.start(null);
+    assertTrue("Simulation start time mismatch", 
+               es.getSimulationStartTime() >= simStartTime);
+    assertTrue("Simulation start time mismatch", 
+               es.getSimulationStartTime() <= System.currentTimeMillis());
+    
+    // test with job stats
+    JobStats stats = generateFakeJobStats(1, 10, true);
+    es.update(stats);
+    testExecutionSummarizer(1, 10, 0, 1, 1, 0, es);
+    
+    // test with failed job 
+    stats = generateFakeJobStats(5, 1, false);
+    es.update(stats);
+    testExecutionSummarizer(6, 11, 0, 2, 1, 1, es);
+    
+    // test finalize
+    //  define a fake job factory
+    JobFactory factory = new FakeJobFactory(conf);
+    
+    // fake the num jobs in trace
+    factory.numJobsInTrace = 3;
+    
+    Path rootTempDir = new Path(System.getProperty("test.build.data", "/tmp"));
+    Path testDir = new Path(rootTempDir, "testGridmixSummary");
+    Path testTraceFile = new Path(testDir, "test-trace.json");
+    FileSystem fs = FileSystem.getLocal(conf);
+    fs.create(testTraceFile).close();
+    
+    // finalize the summarizer
+    UserResolver resolver = new RoundRobinUserResolver();
+    DataStatistics dataStats = new DataStatistics(100, 2, true);
+    String policy = GridmixJobSubmissionPolicy.REPLAY.name();
+    conf.set(GridmixJobSubmissionPolicy.JOB_SUBMISSION_POLICY, policy);
+    es.finalize(factory, testTraceFile.toString(), 1024L, resolver, dataStats, 
+                conf);
+    
+    // test num jobs in trace
+    assertEquals("Mismtach in num jobs in trace", 3, es.getNumJobsInTrace());
+    
+    // test trace signature
+    String tid = 
+      ExecutionSummarizer.getTraceSignature(testTraceFile.toString());
+    assertEquals("Mismatch in trace signature", 
+                 tid, es.getInputTraceSignature());
+    // test trace location
+    Path qPath = fs.makeQualified(testTraceFile);
+    assertEquals("Mismatch in trace signature", 
+                 qPath.toString(), es.getInputTraceLocation());
+    // test expected data size
+    assertEquals("Mismatch in expected data size", 
+                 "1.0k", es.getExpectedDataSize());
+    // test input data statistics
+    assertEquals("Mismatch in input data statistics", 
+                 ExecutionSummarizer.stringifyDataStatistics(dataStats), 
+                 es.getInputDataStatistics());
+    // test user resolver
+    assertEquals("Mismatch in user resolver", 
+                 resolver.getClass().getName(), es.getUserResolver());
+    // test policy
+    assertEquals("Mismatch in policy", policy, es.getJobSubmissionPolicy());
+    
+    // test data stringification using large data
+    es.finalize(factory, testTraceFile.toString(), 1024*1024*1024*10L, resolver,
+                dataStats, conf);
+    assertEquals("Mismatch in expected data size", 
+                 "10.0g", es.getExpectedDataSize());
+    
+    // test trace signature uniqueness
+    //  touch the trace file
+    fs.delete(testTraceFile, false);
+    //  sleep for 1 sec
+    try {
+      Thread.sleep(1000);
+    } catch (InterruptedException ie) {}
+    fs.create(testTraceFile).close();
+    es.finalize(factory, testTraceFile.toString(), 0L, resolver, dataStats, 
+                conf);
+    // test missing expected data size
+    assertEquals("Mismatch in trace signature", 
+                 Summarizer.NA, es.getExpectedDataSize());
+    assertFalse("Mismatch in trace signature", 
+                tid.equals(es.getInputTraceSignature()));
+    // get the new identifier
+    tid = ExecutionSummarizer.getTraceSignature(testTraceFile.toString());
+    assertEquals("Mismatch in trace signature", 
+                 tid, es.getInputTraceSignature());
+    
+    testTraceFile = new Path(testDir, "test-trace2.json");
+    fs.create(testTraceFile).close();
+    es.finalize(factory, testTraceFile.toString(), 0L, resolver, dataStats, 
+                conf);
+    assertFalse("Mismatch in trace signature", 
+                tid.equals(es.getInputTraceSignature()));
+    // get the new identifier
+    tid = ExecutionSummarizer.getTraceSignature(testTraceFile.toString());
+    assertEquals("Mismatch in trace signature", 
+                 tid, es.getInputTraceSignature());
+    
+  }
+  
+  // test the ExecutionSummarizer
+  private static void testExecutionSummarizer(int numMaps, int numReds,
+      int totalJobsInTrace, int totalJobSubmitted, int numSuccessfulJob, 
+      int numFailedJobs, ExecutionSummarizer es) {
+    assertEquals("ExecutionSummarizer test failed [num-maps]", 
+                 numMaps, es.getNumMapTasksLaunched());
+    assertEquals("ExecutionSummarizer test failed [num-reducers]", 
+                 numReds, es.getNumReduceTasksLaunched());
+    assertEquals("ExecutionSummarizer test failed [num-jobs-in-trace]", 
+                 totalJobsInTrace, es.getNumJobsInTrace());
+    assertEquals("ExecutionSummarizer test failed [num-submitted jobs]", 
+                 totalJobSubmitted, es.getNumSubmittedJobs());
+    assertEquals("ExecutionSummarizer test failed [num-successful-jobs]", 
+                 numSuccessfulJob, es.getNumSuccessfulJobs());
+    assertEquals("ExecutionSummarizer test failed [num-failed jobs]", 
+                 numFailedJobs, es.getNumFailedJobs());
+  }
+  
+  // generate fake job stats
+  @SuppressWarnings("deprecation")
+  private static JobStats generateFakeJobStats(final int numMaps, 
+      final int numReds, final boolean isSuccessful) 
+  throws IOException {
+    // A fake job 
+    Job fakeJob = new Job() {
+      @Override
+      public int getNumReduceTasks() {
+        return numReds;
+      };
+      
+      @Override
+      public boolean isSuccessful() throws IOException {
+        return isSuccessful;
+      };
+    };
+    return new JobStats(numMaps, fakeJob);
+  }
+  
+  /**
+   * Test {@link ClusterSummarizer}.
+   */
+  @Test
+  @SuppressWarnings("deprecation")
+  public void testClusterSummarizer() throws IOException {
+    ClusterSummarizer cs = new ClusterSummarizer();
+    Configuration conf = new Configuration();
+    
+    String jt = "test-jt:1234";
+    String nn = "test-nn:5678";
+    conf.set("mapred.job.tracker", jt);
+    conf.set(CommonConfigurationKeys.FS_DEFAULT_NAME_KEY, nn);
+    cs.start(conf);
+    
+    assertEquals("JT name mismatch", jt, cs.getJobTrackerInfo());
+    assertEquals("NN name mismatch", nn, cs.getNamenodeInfo());
+    
+    ClusterStats cstats = ClusterStats.getClusterStats();
+    conf.set("mapred.job.tracker", "local");
+    conf.set(CommonConfigurationKeys.FS_DEFAULT_NAME_KEY, "local");
+    JobClient jc = new JobClient(new JobConf(conf));
+    cstats.setClusterMetric(jc.getClusterStatus());
+    
+    cs.update(cstats);
+    
+    // test
+    assertEquals("Cluster summary test failed!", 1, cs.getMaxMapTasks());
+    assertEquals("Cluster summary test failed!", 1, cs.getMaxReduceTasks());
+    assertEquals("Cluster summary test failed!", 1, cs.getNumActiveTrackers());
+    assertEquals("Cluster summary test failed!", 0, 
+                 cs.getNumBlacklistedTrackers());
+  }
+}

+ 202 - 0
src/contrib/gridmix/src/test/org/apache/hadoop/mapred/gridmix/TestHighRamJob.java

@@ -0,0 +1,202 @@
+/**
+ * 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 static org.junit.Assert.*;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.JobTracker;
+import org.apache.hadoop.mapred.gridmix.DebugJobProducer.MockJob;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.tools.rumen.JobStory;
+import org.junit.Test;
+
+/**
+ * Test if Gridmix correctly configures the simulated job's configuration for
+ * high ram job properties.
+ */
+public class TestHighRamJob {
+  /**
+   * A dummy {@link GridmixJob} that opens up the simulated job for testing.
+   */
+  protected static class DummyGridmixJob extends GridmixJob {
+    public DummyGridmixJob(Configuration conf, JobStory desc) 
+    throws IOException {
+      super(conf, System.currentTimeMillis(), desc, new Path("test"), 
+            UserGroupInformation.getCurrentUser(), -1);
+    }
+    
+    /**
+     * Do nothing since this is a dummy gridmix job.
+     */
+    @Override
+    public Job call() throws Exception {
+      return null;
+    }
+    
+    @Override
+    protected boolean canEmulateCompression() {
+      // return false as we don't need compression
+      return false;
+    }
+    
+    protected Job getJob() {
+      // open the simulated job for testing
+      return job;
+    }
+  }
+  
+  private static void testHighRamConfig(long jobMapMB, long jobReduceMB, 
+      long clusterMapMB, long clusterReduceMB, long simulatedClusterMapMB, 
+      long simulatedClusterReduceMB, long expectedMapMB, long expectedReduceMB, 
+      Configuration gConf) 
+  throws IOException {
+    Configuration simulatedJobConf = new Configuration(gConf);
+    simulatedJobConf.setLong(JobTracker.MAPRED_CLUSTER_MAP_MEMORY_MB_PROPERTY,
+                             simulatedClusterMapMB);
+    simulatedJobConf.setLong(
+        JobTracker.MAPRED_CLUSTER_REDUCE_MEMORY_MB_PROPERTY,
+        simulatedClusterReduceMB);
+    
+    // define a source conf
+    Configuration sourceConf = new Configuration();
+    
+    // configure the original job
+    sourceConf.setLong(JobConf.MAPRED_JOB_MAP_MEMORY_MB_PROPERTY, jobMapMB);
+    sourceConf.setLong(JobTracker.MAPRED_CLUSTER_MAP_MEMORY_MB_PROPERTY,
+                       clusterMapMB);
+    sourceConf.setLong(JobConf.MAPRED_JOB_REDUCE_MEMORY_MB_PROPERTY,
+                       jobReduceMB);
+    sourceConf.setLong(JobTracker.MAPRED_CLUSTER_REDUCE_MEMORY_MB_PROPERTY,
+                       clusterReduceMB);
+    
+    // define a mock job
+    MockJob story = new MockJob(sourceConf);
+    
+    GridmixJob job = new DummyGridmixJob(simulatedJobConf, story);
+    Job simulatedJob = job.getJob();
+    Configuration simulatedConf = simulatedJob.getConfiguration();
+    
+    // check if the high ram properties are not set
+    assertEquals(expectedMapMB, simulatedConf.getLong(
+        JobConf.MAPRED_JOB_MAP_MEMORY_MB_PROPERTY,
+        JobConf.DISABLED_MEMORY_LIMIT));
+    assertEquals(expectedReduceMB,
+        simulatedConf.getLong(JobConf.MAPRED_JOB_REDUCE_MEMORY_MB_PROPERTY,
+        JobConf.DISABLED_MEMORY_LIMIT));
+  }
+  
+  /**
+   * Tests high ram job properties configuration.
+   */
+  @SuppressWarnings("deprecation")
+  @Test
+  public void testHighRamFeatureEmulation() throws IOException {
+    // define the gridmix conf
+    Configuration gridmixConf = new Configuration();
+    
+    // test : check high ram emulation disabled
+    gridmixConf.setBoolean(GridmixJob.GRIDMIX_HIGHRAM_EMULATION_ENABLE, false);
+    testHighRamConfig(10, 20, 5, 10, JobConf.DISABLED_MEMORY_LIMIT, 
+                      JobConf.DISABLED_MEMORY_LIMIT, 
+                      JobConf.DISABLED_MEMORY_LIMIT, 
+                      JobConf.DISABLED_MEMORY_LIMIT, gridmixConf);
+    
+    // test : check with high ram enabled (default) and no scaling
+    gridmixConf = new Configuration();
+    // set the deprecated max memory limit
+    gridmixConf.setLong(JobConf.UPPER_LIMIT_ON_TASK_VMEM_PROPERTY, 
+                        20*1024*1024);
+    testHighRamConfig(10, 20, 5, 10, 5, 10, 10, 20, gridmixConf);
+    
+    // test : check with high ram enabled and scaling
+    gridmixConf = new Configuration();
+    // set the new max map/reduce memory limits
+    gridmixConf.setLong(JobTracker.MAPRED_CLUSTER_MAX_MAP_MEMORY_MB_PROPERTY,
+                        100);
+    gridmixConf.setLong(JobTracker.MAPRED_CLUSTER_MAX_REDUCE_MEMORY_MB_PROPERTY,
+                        300);
+    testHighRamConfig(10, 45, 5, 15, 50, 100, 100, 300, gridmixConf);
+    
+    // test : check with high ram enabled and map memory scaling mismatch 
+    //        (deprecated)
+    gridmixConf = new Configuration();
+    gridmixConf.setLong(JobConf.UPPER_LIMIT_ON_TASK_VMEM_PROPERTY, 
+                        70*1024*1024);
+    Boolean failed = null;
+    try {
+      testHighRamConfig(10, 45, 5, 15, 50, 100, 100, 300, gridmixConf);
+      failed = false;
+    } catch (Exception e) {
+      failed = true;
+    }
+    assertNotNull(failed);
+    assertTrue("Exception expected for exceeding map memory limit "
+               + "(deprecation)!", failed);
+    
+    // test : check with high ram enabled and reduce memory scaling mismatch 
+    //        (deprecated)
+    gridmixConf = new Configuration();
+    gridmixConf.setLong(JobConf.UPPER_LIMIT_ON_TASK_VMEM_PROPERTY, 
+                        150*1024*1024);
+    failed = null;
+    try {
+      testHighRamConfig(10, 45, 5, 15, 50, 100, 100, 300, gridmixConf);
+      failed = false;
+    } catch (Exception e) {
+      failed = true;
+    }
+    assertNotNull(failed);
+    assertTrue("Exception expected for exceeding reduce memory limit "
+               + "(deprecation)!", failed);
+    
+    // test : check with high ram enabled and scaling mismatch on map limits
+    gridmixConf = new Configuration();
+    gridmixConf.setLong(JobTracker.MAPRED_CLUSTER_MAX_MAP_MEMORY_MB_PROPERTY,
+                        70);
+    failed = null;
+    try {
+      testHighRamConfig(10, 45, 5, 15, 50, 100, 100, 300, gridmixConf);
+      failed = false;
+    } catch (Exception e) {
+      failed = true;
+    }
+    assertNotNull(failed);
+    assertTrue("Exception expected for exceeding map memory limit!", failed);
+    
+    // test : check with high ram enabled and scaling mismatch on reduce 
+    //        limits
+    gridmixConf = new Configuration();
+    gridmixConf.setLong(JobTracker.MAPRED_CLUSTER_MAX_REDUCE_MEMORY_MB_PROPERTY,
+                        200);
+    failed = null;
+    try {
+      testHighRamConfig(10, 45, 5, 15, 50, 100, 100, 300, gridmixConf);
+      failed = false;
+    } catch (Exception e) {
+      failed = true;
+    }
+    assertNotNull(failed);
+    assertTrue("Exception expected for exceeding reduce memory limit!", failed);
+  }
+}

+ 233 - 0
src/contrib/gridmix/src/test/org/apache/hadoop/mapred/gridmix/TestPseudoLocalFs.java

@@ -0,0 +1,233 @@
+/**
+ * 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 static org.junit.Assert.*;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.junit.Test;
+
+/**
+ * Test the basic functionality of PseudoLocalFs
+ */
+public class TestPseudoLocalFs {
+
+  /**
+   * Test if a file on PseudoLocalFs of a specific size can be opened and read.
+   * Validate the size of the data read.
+   * Test the read methods of {@link PseudoLocalFs.RandomInputStream}.
+   * @throws Exception
+   */
+  @Test
+  public void testPseudoLocalFsFileSize() throws Exception {
+    long fileSize = 10000;
+    Path path = PseudoLocalFs.generateFilePath("myPsedoFile", fileSize);
+    PseudoLocalFs pfs = new PseudoLocalFs();
+    pfs.create(path);
+
+    // Read 1 byte at a time and validate file size.
+    InputStream in = pfs.open(path, 0);
+    long totalSize = 0;
+
+    while (in.read() >= 0) {
+      ++totalSize;
+    }
+    in.close();
+    assertEquals("File size mismatch with read().", fileSize, totalSize);
+
+    // Read data from PseudoLocalFs-based file into buffer to
+    // validate read(byte[]) and file size.
+    in = pfs.open(path, 0);
+    totalSize = 0;
+    byte[] b = new byte[1024];
+    int bytesRead = in.read(b);
+    while (bytesRead >= 0) {
+      totalSize += bytesRead;
+      bytesRead = in.read(b);
+    }
+    assertEquals("File size mismatch with read(byte[]).", fileSize, totalSize);
+  }
+
+  /**
+   * Validate if file status is obtained for correctly formed file paths on
+   * PseudoLocalFs and also verify if appropriate exception is thrown for
+   * invalid file paths.
+   * @param pfs Pseudo Local File System
+   * @param path file path for which getFileStatus() is to be called
+   * @param shouldSucceed <code>true</code> if getFileStatus() should succeed
+   * @throws IOException
+   */
+  private void validateGetFileStatus(FileSystem pfs, Path path,
+      boolean shouldSucceed) throws IOException {
+    boolean expectedExceptionSeen = false;
+    FileStatus stat = null;
+    try {
+      stat = pfs.getFileStatus(path);
+    } catch(FileNotFoundException e) {
+      expectedExceptionSeen = true;
+    }
+    if (shouldSucceed) {
+      assertFalse("getFileStatus() has thrown Exception for valid file name "
+                  + path, expectedExceptionSeen);
+      assertNotNull("Missing file status for a valid file.", stat);
+
+      // validate fileSize
+      String[] parts = path.toUri().getPath().split("\\.");
+      long expectedFileSize = Long.valueOf(parts[parts.length - 1]);
+      assertEquals("Invalid file size.", expectedFileSize, stat.getLen());
+    } else {
+      assertTrue("getFileStatus() did not throw Exception for invalid file "
+                 + " name " + path, expectedExceptionSeen);
+    }
+  }
+
+  /**
+   * Validate if file creation succeeds for correctly formed file paths on
+   * PseudoLocalFs and also verify if appropriate exception is thrown for
+   * invalid file paths.
+   * @param pfs Pseudo Local File System
+   * @param path file path for which create() is to be called
+   * @param shouldSucceed <code>true</code> if create() should succeed
+   * @throws IOException
+   */
+  private void validateCreate(FileSystem pfs, Path path,
+      boolean shouldSucceed) throws IOException {
+    boolean expectedExceptionSeen = false;
+    try {
+      pfs.create(path);
+    } catch(IOException e) {
+      expectedExceptionSeen = true;
+    }
+    if (shouldSucceed) {
+      assertFalse("create() has thrown Exception for valid file name "
+                  + path, expectedExceptionSeen);
+    } else {
+      assertTrue("create() did not throw Exception for invalid file name "
+                 + path, expectedExceptionSeen);
+    }
+  }
+
+  /**
+   * Validate if opening of file succeeds for correctly formed file paths on
+   * PseudoLocalFs and also verify if appropriate exception is thrown for
+   * invalid file paths.
+   * @param pfs Pseudo Local File System
+   * @param path file path for which open() is to be called
+   * @param shouldSucceed <code>true</code> if open() should succeed
+   * @throws IOException
+   */
+  private void validateOpen(FileSystem pfs, Path path,
+      boolean shouldSucceed) throws IOException {
+    boolean expectedExceptionSeen = false;
+    try {
+      pfs.open(path);
+    } catch(IOException e) {
+      expectedExceptionSeen = true;
+    }
+    if (shouldSucceed) {
+      assertFalse("open() has thrown Exception for valid file name "
+                  + path, expectedExceptionSeen);
+    } else {
+      assertTrue("open() did not throw Exception for invalid file name "
+                 + path, expectedExceptionSeen);
+    }
+  }
+
+  /**
+   * Validate if exists() returns <code>true</code> for correctly formed file
+   * paths on PseudoLocalFs and returns <code>false</code> for improperly
+   * formed file paths.
+   * @param pfs Pseudo Local File System
+   * @param path file path for which exists() is to be called
+   * @param shouldSucceed expected return value of exists(&lt;path&gt;)
+   * @throws IOException
+   */
+  private void validateExists(FileSystem pfs, Path path,
+      boolean shouldSucceed) throws IOException {
+    boolean ret = pfs.exists(path);
+    if (shouldSucceed) {
+      assertTrue("exists() returned false for valid file name " + path, ret);
+    } else {
+      assertFalse("exists() returned true for invalid file name " + path, ret);
+    }
+  }
+
+  /**
+   *  Test Pseudo Local File System methods like getFileStatus(), create(),
+   *  open(), exists() for <li> valid file paths and <li> invalid file paths.
+   * @throws IOException
+   */
+  @Test
+  public void testPseudoLocalFsFileNames() throws IOException {
+    PseudoLocalFs pfs = new PseudoLocalFs();
+    Configuration conf = new Configuration();
+    conf.setClass("fs.pseudo.impl", PseudoLocalFs.class, FileSystem.class);
+
+    Path path = new Path("pseudo:///myPsedoFile.1234");
+    FileSystem testFs = path.getFileSystem(conf);
+    assertEquals("Failed to obtain a pseudo local file system object from path",
+                 pfs.getUri().getScheme(), testFs.getUri().getScheme());
+
+    // Validate PseudoLocalFS operations on URI of some other file system
+    path = new Path("file:///myPsedoFile.12345");
+    validateGetFileStatus(pfs, path, false);
+    validateCreate(pfs, path, false);
+    validateOpen(pfs, path, false);
+    validateExists(pfs, path, false);
+
+    path = new Path("pseudo:///myPsedoFile");//.<fileSize> missing
+    validateGetFileStatus(pfs, path, false);
+    validateCreate(pfs, path, false);
+    validateOpen(pfs, path, false);
+    validateExists(pfs, path, false);
+
+    // thing after final '.' is not a number
+    path = new Path("pseudo:///myPsedoFile.txt");
+    validateGetFileStatus(pfs, path, false);
+    validateCreate(pfs, path, false);
+    validateOpen(pfs, path, false);
+    validateExists(pfs, path, false);
+
+    // Generate valid file name(relative path) and validate operations on it
+    long fileSize = 231456;
+    path = PseudoLocalFs.generateFilePath("my.Psedo.File", fileSize);
+    // Validate the above generateFilePath()
+    assertEquals("generateFilePath() failed.", fileSize,
+                 pfs.validateFileNameFormat(path));
+
+    validateGetFileStatus(pfs, path, true);
+    validateCreate(pfs, path, true);
+    validateOpen(pfs, path, true);
+    validateExists(pfs, path, true);
+
+    // Validate operations on valid qualified path
+    path = new Path("myPsedoFile.1237");
+    path = path.makeQualified(pfs);
+    validateGetFileStatus(pfs, path, true);
+    validateCreate(pfs, path, true);
+    validateOpen(pfs, path, true);
+    validateExists(pfs, path, true);
+  }
+}

+ 84 - 0
src/contrib/gridmix/src/test/org/apache/hadoop/mapred/gridmix/TestRandomTextDataGenerator.java

@@ -0,0 +1,84 @@
+/**
+ * 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.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.hadoop.mapred.gridmix.RandomTextDataGenerator;
+
+import static org.junit.Assert.*;
+import org.junit.Test;
+
+/**
+ * Test {@link RandomTextDataGenerator}.
+ */
+public class TestRandomTextDataGenerator {
+  /**
+   * Test if {@link RandomTextDataGenerator} can generate random words of 
+   * desired size.
+   */
+  @Test
+  public void testRandomTextDataGenerator() {
+    RandomTextDataGenerator rtdg = new RandomTextDataGenerator(10, 0L, 5);
+    List<String> words = rtdg.getRandomWords();
+
+    // check the size
+    assertEquals("List size mismatch", 10, words.size());
+
+    // check the words
+    Set<String> wordsSet = new HashSet<String>(words);
+    assertEquals("List size mismatch due to duplicates", 10, wordsSet.size());
+
+    // check the word lengths
+    for (String word : wordsSet) {
+      assertEquals("Word size mismatch", 5, word.length());
+    }
+  }
+  
+  /**
+   * Test if {@link RandomTextDataGenerator} can generate same words given the
+   * same list-size, word-length and seed.
+   */
+  @Test
+  public void testRandomTextDataGeneratorRepeatability() {
+    RandomTextDataGenerator rtdg1 = new RandomTextDataGenerator(10, 0L, 5);
+    List<String> words1 = rtdg1.getRandomWords();
+
+    RandomTextDataGenerator rtdg2 = new RandomTextDataGenerator(10, 0L, 5);
+    List<String> words2 = rtdg2.getRandomWords();
+    
+    assertTrue("List mismatch", words1.equals(words2));
+  }
+  
+  /**
+   * Test if {@link RandomTextDataGenerator} can generate different words given 
+   * different seeds.
+   */
+  @Test
+  public void testRandomTextDataGeneratorUniqueness() {
+    RandomTextDataGenerator rtdg1 = new RandomTextDataGenerator(10, 1L, 5);
+    Set<String> words1 = new HashSet(rtdg1.getRandomWords());
+
+    RandomTextDataGenerator rtdg2 = new RandomTextDataGenerator(10, 0L, 5);
+    Set<String> words2 = new HashSet(rtdg2.getRandomWords());
+    
+    assertFalse("List size mismatch across lists", words1.equals(words2));
+  }
+}

+ 612 - 0
src/contrib/gridmix/src/test/org/apache/hadoop/mapred/gridmix/TestResourceUsageEmulators.java

@@ -0,0 +1,612 @@
+/**
+ * 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.junit.Test;
+import static org.junit.Assert.*;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.MapContext;
+import org.apache.hadoop.mapreduce.StatusReporter;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.hadoop.mapreduce.TaskInputOutputContext;
+import org.apache.hadoop.util.ResourceCalculatorPlugin;
+import org.apache.hadoop.util.ResourceCalculatorPlugin.ProcResourceValues;
+import org.apache.hadoop.tools.rumen.ResourceUsageMetrics;
+import org.apache.hadoop.util.DummyResourceCalculatorPlugin;
+import org.apache.hadoop.mapred.TaskTracker;
+import org.apache.hadoop.mapred.gridmix.LoadJob.ResourceUsageMatcherRunner;
+import org.apache.hadoop.mapred.gridmix.emulators.resourceusage.CumulativeCpuUsageEmulatorPlugin;
+import org.apache.hadoop.mapred.gridmix.emulators.resourceusage.ResourceUsageEmulatorPlugin;
+import org.apache.hadoop.mapred.gridmix.emulators.resourceusage.ResourceUsageMatcher;
+import org.apache.hadoop.mapred.gridmix.emulators.resourceusage.CumulativeCpuUsageEmulatorPlugin.DefaultCpuUsageEmulator;
+
+/**
+ * Test Gridmix's resource emulator framework and supported plugins.
+ */
+public class TestResourceUsageEmulators {
+  /**
+   * A {@link ResourceUsageEmulatorPlugin} implementation for testing purpose.
+   * It essentially creates a file named 'test' in the test directory.
+   */
+  static class TestResourceUsageEmulatorPlugin 
+  implements ResourceUsageEmulatorPlugin {
+    static final Path rootTempDir =
+        new Path(System.getProperty("test.build.data", "/tmp"));
+    static final Path tempDir = 
+      new Path(rootTempDir, "TestResourceUsageEmulatorPlugin");
+    static final String DEFAULT_IDENTIFIER = "test";
+    
+    private Path touchPath = null;
+    private FileSystem fs = null;
+    
+    @Override
+    public void emulate() throws IOException, InterruptedException {
+      // add some time between 2 calls to emulate()
+      try {
+        Thread.sleep(1000); // sleep for 1s
+      } catch (Exception e){}
+      
+      try {
+        fs.delete(touchPath, false); // delete the touch file
+        //TODO Search for a better touch utility
+        fs.create(touchPath).close(); // recreate it
+      } catch (Exception e) {
+        throw new RuntimeException(e);
+      }
+    }
+    
+    protected String getIdentifier() {
+      return DEFAULT_IDENTIFIER;
+    }
+    
+    private static Path getFilePath(String id) {
+      return new Path(tempDir, id);
+    }
+    
+    private static Path getInitFilePath(String id) {
+      return new Path(tempDir, id + ".init");
+    }
+    
+    @Override
+    public void initialize(Configuration conf, ResourceUsageMetrics metrics,
+        ResourceCalculatorPlugin monitor, Progressive progress) {
+      // add some time between 2 calls to initialize()
+      try {
+        Thread.sleep(1000); // sleep for 1s
+      } catch (Exception e){}
+      
+      try {
+        fs = FileSystem.getLocal(conf);
+        
+        Path initPath = getInitFilePath(getIdentifier());
+        fs.delete(initPath, false); // delete the old file
+        fs.create(initPath).close(); // create a new one
+        
+        touchPath = getFilePath(getIdentifier());
+        fs.delete(touchPath, false);
+      } catch (Exception e) {
+        
+      } finally {
+        if (fs != null) {
+          try {
+            fs.deleteOnExit(tempDir);
+          } catch (IOException ioe){}
+        }
+      }
+    }
+    
+    // test if the emulation framework successfully loaded this plugin
+    static long testInitialization(String id, Configuration conf) 
+    throws IOException {
+      Path testPath = getInitFilePath(id);
+      FileSystem fs = FileSystem.getLocal(conf);
+      return fs.exists(testPath) 
+             ? fs.getFileStatus(testPath).getModificationTime() 
+             : 0;
+    }
+    
+    // test if the emulation framework successfully loaded this plugin
+    static long testEmulation(String id, Configuration conf) 
+    throws IOException {
+      Path testPath = getFilePath(id);
+      FileSystem fs = FileSystem.getLocal(conf);
+      return fs.exists(testPath) 
+             ? fs.getFileStatus(testPath).getModificationTime() 
+             : 0;
+    }
+  }
+  
+  /**
+   * Test implementation of {@link ResourceUsageEmulatorPlugin} which creates
+   * a file named 'others' in the test directory.
+   */
+  static class TestOthers extends TestResourceUsageEmulatorPlugin {
+    static final String ID = "others";
+    
+    @Override
+    protected String getIdentifier() {
+      return ID;
+    }
+  }
+  
+  /**
+   * Test implementation of {@link ResourceUsageEmulatorPlugin} which creates
+   * a file named 'cpu' in the test directory.
+   */
+  static class TestCpu extends TestResourceUsageEmulatorPlugin {
+    static final String ID = "cpu";
+    
+    @Override
+    protected String getIdentifier() {
+      return ID;
+    }
+  }
+  
+  /**
+   * Test {@link ResourceUsageMatcher}.
+   */
+  @Test
+  public void testResourceUsageMatcher() throws Exception {
+    ResourceUsageMatcher matcher = new ResourceUsageMatcher();
+    Configuration conf = new Configuration();
+    conf.setClass(ResourceUsageMatcher.RESOURCE_USAGE_EMULATION_PLUGINS, 
+                  TestResourceUsageEmulatorPlugin.class, 
+                  ResourceUsageEmulatorPlugin.class);
+    long currentTime = System.currentTimeMillis();
+    
+    matcher.configure(conf, null, null, null);
+    
+    matcher.matchResourceUsage();
+    
+    String id = TestResourceUsageEmulatorPlugin.DEFAULT_IDENTIFIER;
+    long result = 
+      TestResourceUsageEmulatorPlugin.testInitialization(id, conf);
+    assertTrue("Resource usage matcher failed to initialize the configured"
+               + " plugin", result > currentTime);
+    result = TestResourceUsageEmulatorPlugin.testEmulation(id, conf);
+    assertTrue("Resource usage matcher failed to load and emulate the"
+               + " configured plugin", result > currentTime);
+    
+    // test plugin order to first emulate cpu and then others
+    conf.setStrings(ResourceUsageMatcher.RESOURCE_USAGE_EMULATION_PLUGINS, 
+                    TestCpu.class.getName() + "," + TestOthers.class.getName());
+    
+    matcher.configure(conf, null, null, null);
+
+    // test the initialization order
+    long time1 = 
+           TestResourceUsageEmulatorPlugin.testInitialization(TestCpu.ID, conf);
+    long time2 = 
+           TestResourceUsageEmulatorPlugin.testInitialization(TestOthers.ID, 
+                                                              conf);
+    assertTrue("Resource usage matcher failed to initialize the configured"
+               + " plugins in order", time1 < time2);
+    
+    matcher.matchResourceUsage();
+
+    // Note that the cpu usage emulator plugin is configured 1st and then the
+    // others plugin.
+    time1 = 
+      TestResourceUsageEmulatorPlugin.testInitialization(TestCpu.ID, conf);
+    time2 = 
+      TestResourceUsageEmulatorPlugin.testInitialization(TestOthers.ID, 
+                                                         conf);
+    assertTrue("Resource usage matcher failed to load the configured plugins", 
+               time1 < time2);
+  }
+  
+  /**
+   * Fakes the cumulative usage using {@link FakeCpuUsageEmulatorCore}.
+   */
+  static class FakeResourceUsageMonitor extends DummyResourceCalculatorPlugin {
+    private FakeCpuUsageEmulatorCore core;
+    
+    public FakeResourceUsageMonitor(FakeCpuUsageEmulatorCore core) {
+      this.core = core;
+    }
+    
+    /**
+     * A dummy CPU usage monitor. Every call to 
+     * {@link ResourceCalculatorPlugin#getCumulativeCpuTime()} will return the 
+     * value of {@link FakeCpuUsageEmulatorCore#getNumCalls()}.
+     */
+    @Override
+    public long getCumulativeCpuTime() {
+      return core.getCpuUsage();
+    }
+
+    /**
+     * Returns a {@link ProcResourceValues} with cumulative cpu usage  
+     * computed using {@link #getCumulativeCpuTime()}.
+     */
+    @Override
+    public ProcResourceValues getProcResourceValues() {
+      long usageValue = getCumulativeCpuTime();
+      return new ProcResourceValues(usageValue, -1, -1);
+    }
+  }
+  
+  /**
+   * A dummy {@link Progressive} implementation that allows users to set the
+   * progress for testing. The {@link Progressive#getProgress()} call will 
+   * return the last progress value set using 
+   * {@link FakeProgressive#setProgress(float)}.
+   */
+  static class FakeProgressive implements Progressive {
+    private float progress = 0F;
+    @Override
+    public float getProgress() {
+      return progress;
+    }
+    
+    void setProgress(float progress) {
+      this.progress = progress;
+    }
+  }
+  
+  /**
+   * A dummy reporter for {@link LoadJob.ResourceUsageMatcherRunner}.
+   */
+  private static class DummyReporter extends StatusReporter {
+    private Progressive progress;
+    
+    DummyReporter(Progressive progress) {
+      this.progress = progress;
+    }
+    
+    @Override
+    public org.apache.hadoop.mapreduce.Counter getCounter(Enum<?> name) {
+      return null;
+    }
+    
+    @Override
+    public org.apache.hadoop.mapreduce.Counter getCounter(String group,
+                                                          String name) {
+      return null;
+    }
+    
+    @Override
+    public void progress() {
+    }
+    
+    @Override
+    public float getProgress() {
+      return progress.getProgress();
+    }
+    
+    @Override
+    public void setStatus(String status) {
+    }
+  }
+  
+  // Extends ResourceUsageMatcherRunner for testing.
+  @SuppressWarnings("unchecked")
+  private static class FakeResourceUsageMatcherRunner 
+  extends ResourceUsageMatcherRunner {
+    FakeResourceUsageMatcherRunner(TaskInputOutputContext context, 
+                                   ResourceUsageMetrics metrics) {
+      super(context, metrics);
+    }
+    
+    // test ResourceUsageMatcherRunner
+    void test() throws Exception {
+      super.match();
+    }
+  }
+  
+  /**
+   * Test {@link LoadJob.ResourceUsageMatcherRunner}.
+   */
+  @Test
+  @SuppressWarnings("unchecked")
+  public void testResourceUsageMatcherRunner() throws Exception {
+    Configuration conf = new Configuration();
+    FakeProgressive progress = new FakeProgressive();
+    
+    // set the resource calculator plugin
+    conf.setClass(TaskTracker.TT_RESOURCE_CALCULATOR_PLUGIN,
+                  DummyResourceCalculatorPlugin.class, 
+                  ResourceCalculatorPlugin.class);
+    // set the resources
+    // set the resource implementation class
+    conf.setClass(ResourceUsageMatcher.RESOURCE_USAGE_EMULATION_PLUGINS, 
+                  TestResourceUsageEmulatorPlugin.class, 
+                  ResourceUsageEmulatorPlugin.class);
+    
+    long currentTime = System.currentTimeMillis();
+    
+    // initialize the matcher class
+    TaskAttemptID id = new TaskAttemptID("test", 1, true, 1, 1);
+    StatusReporter reporter = new DummyReporter(progress);
+    TaskInputOutputContext context = 
+      new MapContext(conf, id, null, null, null, reporter, null);
+    FakeResourceUsageMatcherRunner matcher = 
+      new FakeResourceUsageMatcherRunner(context, null);
+    
+    // check if the matcher initialized the plugin
+    String identifier = TestResourceUsageEmulatorPlugin.DEFAULT_IDENTIFIER;
+    long initTime = 
+      TestResourceUsageEmulatorPlugin.testInitialization(identifier, conf);
+    assertTrue("ResourceUsageMatcherRunner failed to initialize the"
+               + " configured plugin", initTime > currentTime);
+    
+    // check the progress
+    assertEquals("Progress mismatch in ResourceUsageMatcherRunner", 
+                 0, progress.getProgress(), 0D);
+    
+    // call match() and check progress
+    progress.setProgress(0.01f);
+    currentTime = System.currentTimeMillis();
+    matcher.test();
+    long emulateTime = 
+      TestResourceUsageEmulatorPlugin.testEmulation(identifier, conf);
+    assertTrue("ProgressBasedResourceUsageMatcher failed to load and emulate"
+               + " the configured plugin", emulateTime > currentTime);
+  }
+  
+  /**
+   * Test {@link CumulativeCpuUsageEmulatorPlugin}'s core CPU usage emulation 
+   * engine.
+   */
+  @Test
+  public void testCpuUsageEmulator() throws IOException {
+    // test CpuUsageEmulator calibration with fake resource calculator plugin
+    long target = 100000L; // 100 secs
+    int unitUsage = 50;
+    FakeCpuUsageEmulatorCore fakeCpuEmulator = new FakeCpuUsageEmulatorCore();
+    fakeCpuEmulator.setUnitUsage(unitUsage);
+    FakeResourceUsageMonitor fakeMonitor = 
+      new FakeResourceUsageMonitor(fakeCpuEmulator);
+    
+    // calibrate for 100ms
+    fakeCpuEmulator.calibrate(fakeMonitor, target);
+    
+    // by default, CpuUsageEmulator.calibrate() will consume 100ms of CPU usage
+    assertEquals("Fake calibration failed", 
+                 100, fakeMonitor.getCumulativeCpuTime());
+    assertEquals("Fake calibration failed", 
+                 100, fakeCpuEmulator.getCpuUsage());
+    // by default, CpuUsageEmulator.performUnitComputation() will be called 
+    // twice
+    assertEquals("Fake calibration failed", 
+                 2, fakeCpuEmulator.getNumCalls());
+  }
+  
+  /**
+   * This is a dummy class that fakes CPU usage.
+   */
+  private static class FakeCpuUsageEmulatorCore 
+  extends DefaultCpuUsageEmulator {
+    private int numCalls = 0;
+    private int unitUsage = 1;
+    private int cpuUsage = 0;
+    
+    @Override
+    protected void performUnitComputation() {
+      ++numCalls;
+      cpuUsage += unitUsage;
+    }
+    
+    int getNumCalls() {
+      return numCalls;
+    }
+    
+    int getCpuUsage() {
+      return cpuUsage;
+    }
+    
+    void reset() {
+      numCalls = 0;
+      cpuUsage = 0;
+    }
+    
+    void setUnitUsage(int unitUsage) {
+      this.unitUsage = unitUsage;
+    }
+  }
+  
+  // Creates a ResourceUsageMetrics object from the target usage
+  static ResourceUsageMetrics createMetrics(long target) {
+    ResourceUsageMetrics metrics = new ResourceUsageMetrics();
+    metrics.setCumulativeCpuUsage(target);
+    metrics.setVirtualMemoryUsage(target);
+    metrics.setPhysicalMemoryUsage(target);
+    metrics.setHeapUsage(target);
+    return metrics;
+  }
+  
+  /**
+   * Test {@link CumulativeCpuUsageEmulatorPlugin}.
+   */
+  @Test
+  public void testCumulativeCpuUsageEmulatorPlugin() throws Exception {
+    Configuration conf = new Configuration();
+    long targetCpuUsage = 1000L;
+    int unitCpuUsage = 50;
+    
+    // fake progress indicator
+    FakeProgressive fakeProgress = new FakeProgressive();
+    
+    // fake cpu usage generator
+    FakeCpuUsageEmulatorCore fakeCore = new FakeCpuUsageEmulatorCore();
+    fakeCore.setUnitUsage(unitCpuUsage);
+    
+    // a cumulative cpu usage emulator with fake core
+    CumulativeCpuUsageEmulatorPlugin cpuPlugin = 
+      new CumulativeCpuUsageEmulatorPlugin(fakeCore);
+    
+    // test with invalid or missing resource usage value
+    ResourceUsageMetrics invalidUsage = createMetrics(0);
+    cpuPlugin.initialize(conf, invalidUsage, null, null);
+    
+    // test if disabled cpu emulation plugin's emulate() call is a no-operation
+    // this will test if the emulation plugin is disabled or not
+    int numCallsPre = fakeCore.getNumCalls();
+    long cpuUsagePre = fakeCore.getCpuUsage();
+    cpuPlugin.emulate();
+    int numCallsPost = fakeCore.getNumCalls();
+    long cpuUsagePost = fakeCore.getCpuUsage();
+    
+    //  test if no calls are made cpu usage emulator core
+    assertEquals("Disabled cumulative CPU usage emulation plugin works!", 
+                 numCallsPre, numCallsPost);
+    
+    //  test if no calls are made cpu usage emulator core
+    assertEquals("Disabled cumulative CPU usage emulation plugin works!", 
+                 cpuUsagePre, cpuUsagePost);
+    
+    // test with valid resource usage value
+    ResourceUsageMetrics metrics = createMetrics(targetCpuUsage);
+    
+    // fake monitor
+    ResourceCalculatorPlugin monitor = new FakeResourceUsageMonitor(fakeCore);
+    
+    // test with default emulation interval
+    testEmulationAccuracy(conf, fakeCore, monitor, metrics, cpuPlugin, 
+                          targetCpuUsage, targetCpuUsage / unitCpuUsage);
+    
+    // test with custom value for emulation interval of 20%
+    conf.setFloat(CumulativeCpuUsageEmulatorPlugin.CPU_EMULATION_PROGRESS_INTERVAL,
+                  0.2F);
+    testEmulationAccuracy(conf, fakeCore, monitor, metrics, cpuPlugin, 
+                          targetCpuUsage, targetCpuUsage / unitCpuUsage);
+    
+    // test if emulation interval boundary is respected (unit usage = 1)
+    //  test the case where the current progress is less than threshold
+    fakeProgress = new FakeProgressive(); // initialize
+    fakeCore.reset();
+    fakeCore.setUnitUsage(1);
+    conf.setFloat(CumulativeCpuUsageEmulatorPlugin.CPU_EMULATION_PROGRESS_INTERVAL,
+                  0.25F);
+    cpuPlugin.initialize(conf, metrics, monitor, fakeProgress);
+    // take a snapshot after the initialization
+    long initCpuUsage = monitor.getCumulativeCpuTime();
+    long initNumCalls = fakeCore.getNumCalls();
+    // test with 0 progress
+    testEmulationBoundary(0F, fakeCore, fakeProgress, cpuPlugin, initCpuUsage, 
+                          initNumCalls, "[no-op, 0 progress]");
+    // test with 24% progress
+    testEmulationBoundary(0.24F, fakeCore, fakeProgress, cpuPlugin, 
+                          initCpuUsage, initNumCalls, "[no-op, 24% progress]");
+    // test with 25% progress
+    //  target = 1000ms, target emulation at 25% = 250ms, 
+    //  weighed target = 1000 * 0.25^4 (we are using progress^4 as the weight)
+    //                 ~ 4
+    //  but current usage = init-usage = 100, hence expected = 100
+    testEmulationBoundary(0.25F, fakeCore, fakeProgress, cpuPlugin, 
+                          initCpuUsage, initNumCalls, "[op, 25% progress]");
+    
+    // test with 80% progress
+    //  target = 1000ms, target emulation at 80% = 800ms, 
+    //  weighed target = 1000 * 0.25^4 (we are using progress^4 as the weight)
+    //                 ~ 410
+    //  current-usage = init-usage = 100, hence expected-usage = 410
+    testEmulationBoundary(0.80F, fakeCore, fakeProgress, cpuPlugin, 410, 410, 
+                          "[op, 80% progress]");
+    
+    // now test if the final call with 100% progress ramps up the CPU usage
+    testEmulationBoundary(1F, fakeCore, fakeProgress, cpuPlugin, targetCpuUsage,
+                          targetCpuUsage, "[op, 100% progress]");
+    
+    // test if emulation interval boundary is respected (unit usage = 50)
+    //  test the case where the current progress is less than threshold
+    fakeProgress = new FakeProgressive(); // initialize
+    fakeCore.reset();
+    fakeCore.setUnitUsage(unitCpuUsage);
+    conf.setFloat(CumulativeCpuUsageEmulatorPlugin.CPU_EMULATION_PROGRESS_INTERVAL,
+                  0.40F);
+    cpuPlugin.initialize(conf, metrics, monitor, fakeProgress);
+    // take a snapshot after the initialization
+    initCpuUsage = monitor.getCumulativeCpuTime();
+    initNumCalls = fakeCore.getNumCalls();
+    // test with 0 progress
+    testEmulationBoundary(0F, fakeCore, fakeProgress, cpuPlugin, initCpuUsage, 
+                          initNumCalls, "[no-op, 0 progress]");
+    // test with 39% progress
+    testEmulationBoundary(0.39F, fakeCore, fakeProgress, cpuPlugin, 
+                          initCpuUsage, initNumCalls, "[no-op, 39% progress]");
+    // test with 40% progress
+    //  target = 1000ms, target emulation at 40% = 4000ms, 
+    //  weighed target = 1000 * 0.40^4 (we are using progress^4 as the weight)
+    //                 ~ 26
+    // current-usage = init-usage = 100, hence expected-usage = 100
+    testEmulationBoundary(0.40F, fakeCore, fakeProgress, cpuPlugin, 
+                          initCpuUsage, initNumCalls, "[op, 40% progress]");
+    
+    // test with 90% progress
+    //  target = 1000ms, target emulation at 90% = 900ms, 
+    //  weighed target = 1000 * 0.90^4 (we are using progress^4 as the weight)
+    //                 ~ 657
+    //  current-usage = init-usage = 100, hence expected-usage = 657 but 
+    //  the fake-core increases in steps of 50, hence final target = 700
+    testEmulationBoundary(0.90F, fakeCore, fakeProgress, cpuPlugin, 700, 
+                          700 / unitCpuUsage, "[op, 90% progress]");
+    
+    // now test if the final call with 100% progress ramps up the CPU usage
+    testEmulationBoundary(1F, fakeCore, fakeProgress, cpuPlugin, targetCpuUsage,
+                          targetCpuUsage / unitCpuUsage, "[op, 100% progress]");
+  }
+  
+  // test whether the CPU usage emulator achieves the desired target using
+  // desired calls to the underling core engine.
+  private static void testEmulationAccuracy(Configuration conf, 
+                        FakeCpuUsageEmulatorCore fakeCore,
+                        ResourceCalculatorPlugin monitor,
+                        ResourceUsageMetrics metrics,
+                        CumulativeCpuUsageEmulatorPlugin cpuPlugin,
+                        long expectedTotalCpuUsage, long expectedTotalNumCalls) 
+  throws Exception {
+    FakeProgressive fakeProgress = new FakeProgressive();
+    fakeCore.reset();
+    cpuPlugin.initialize(conf, metrics, monitor, fakeProgress);
+    int numLoops = 0;
+    while (fakeProgress.getProgress() < 1) {
+      ++numLoops;
+      float progress = (float)numLoops / 100;
+      fakeProgress.setProgress(progress);
+      cpuPlugin.emulate();
+    }
+    
+    // test if the resource plugin shows the expected invocations
+    assertEquals("Cumulative cpu usage emulator plugin failed (num calls)!", 
+                 expectedTotalNumCalls, fakeCore.getNumCalls(), 0L);
+    // test if the resource plugin shows the expected usage
+    assertEquals("Cumulative cpu usage emulator plugin failed (total usage)!", 
+                 expectedTotalCpuUsage, fakeCore.getCpuUsage(), 0L);
+  }
+  
+  // tests if the CPU usage emulation plugin emulates only at the expected
+  // progress gaps
+  private static void testEmulationBoundary(float progress, 
+      FakeCpuUsageEmulatorCore fakeCore, FakeProgressive fakeProgress, 
+      CumulativeCpuUsageEmulatorPlugin cpuPlugin, long expectedTotalCpuUsage, 
+      long expectedTotalNumCalls, String info) throws Exception {
+    fakeProgress.setProgress(progress);
+    cpuPlugin.emulate();
+    
+    assertEquals("Emulation interval test for cpu usage failed " + info + "!", 
+                 expectedTotalCpuUsage, fakeCore.getCpuUsage(), 0L);
+    assertEquals("Emulation interval test for num calls failed " + info + "!", 
+                 expectedTotalNumCalls, fakeCore.getNumCalls(), 0L);
+  }
+}

+ 109 - 33
src/contrib/gridmix/src/test/org/apache/hadoop/mapred/gridmix/TestUserResolve.java

@@ -33,23 +33,31 @@ import org.apache.hadoop.security.UserGroupInformation;
 
 
 public class TestUserResolve {
 public class TestUserResolve {
 
 
-  static Path userlist;
+  private static Path rootDir = null;
+  private static Configuration conf = null;
+  private static FileSystem fs = null;
 
 
   @BeforeClass
   @BeforeClass
-  public static void writeUserList() throws IOException {
-    final Configuration conf = new Configuration();
-    final FileSystem fs = FileSystem.getLocal(conf);
-    final Path wd = new Path(new Path(
-          System.getProperty("test.build.data", "/tmp")).makeQualified(fs),
-        "gridmixUserResolve");
-    userlist = new Path(wd, "users");
+  public static void createRootDir() throws IOException {
+    conf = new Configuration();
+    fs = FileSystem.getLocal(conf);
+    rootDir = new Path(new Path(System.getProperty("test.build.data", "/tmp"))
+                  .makeQualified(fs), "gridmixUserResolve");
+  }
+
+  /**
+   * Creates users file with the content as the String usersFileContent.
+   * @param usersFilePath    the path to the file that is to be created
+   * @param usersFileContent Content of users file
+   * @throws IOException
+   */
+  private static void writeUserList(Path usersFilePath, String usersFileContent)
+  throws IOException {
+
     FSDataOutputStream out = null;
     FSDataOutputStream out = null;
     try {
     try {
-      out = fs.create(userlist, true);
-      out.writeBytes("user0,groupA,groupB,groupC\n");
-      out.writeBytes("user1,groupA,groupC\n");
-      out.writeBytes("user2,groupB\n");
-      out.writeBytes("user3,groupA,groupB,groupC\n");
+      out = fs.create(usersFilePath, true);
+      out.writeBytes(usersFileContent);
     } finally {
     } finally {
       if (out != null) {
       if (out != null) {
         out.close();
         out.close();
@@ -57,42 +65,110 @@ public class TestUserResolve {
     }
     }
   }
   }
 
 
-  @Test
-  public void testRoundRobinResolver() throws Exception {
-    final Configuration conf = new Configuration();
-    final UserResolver rslv = new RoundRobinUserResolver();
-
+  /**
+   * Validate RoundRobinUserResolver's behavior for bad user resource file.
+   * RoundRobinUserResolver.setTargetUsers() should throw proper Exception for
+   * the cases like
+   * <li> non existent user resource file and
+   * <li> empty user resource file
+   *
+   * @param rslv              The RoundRobinUserResolver object
+   * @param userRsrc          users file
+   * @param expectedErrorMsg  expected error message
+   */
+  private void validateBadUsersFile(UserResolver rslv, URI userRsrc,
+      String expectedErrorMsg) {
     boolean fail = false;
     boolean fail = false;
     try {
     try {
-      rslv.setTargetUsers(null, conf);
+      rslv.setTargetUsers(userRsrc, conf);
     } catch (IOException e) {
     } catch (IOException e) {
+      assertTrue("Exception message from RoundRobinUserResolver is wrong",
+          e.getMessage().equals(expectedErrorMsg));
       fail = true;
       fail = true;
     }
     }
     assertTrue("User list required for RoundRobinUserResolver", fail);
     assertTrue("User list required for RoundRobinUserResolver", fail);
+  }
+
+  /**
+   * Validate the behavior of {@link RoundRobinUserResolver} for different
+   * user resource files like
+   * <li> Empty user resource file
+   * <li> Non existent user resource file
+   * <li> User resource file with valid content
+   * @throws Exception
+   */
+  @Test
+  public void testRoundRobinResolver() throws Exception {
+
+    final UserResolver rslv = new RoundRobinUserResolver();
+    Path usersFilePath = new Path(rootDir, "users");
+    URI userRsrc = new URI(usersFilePath.toString());
+
+    // Check if the error message is as expected for non existent
+    // user resource file.
+    fs.delete(usersFilePath, false);
+    String expectedErrorMsg = "File " + userRsrc + " does not exist.";
+    validateBadUsersFile(rslv, userRsrc, expectedErrorMsg);
+
+    // Check if the error message is as expected for empty user resource file
+    writeUserList(usersFilePath, "");// creates empty users file
+    expectedErrorMsg =
+      RoundRobinUserResolver.buildEmptyUsersErrorMsg(userRsrc);
+    validateBadUsersFile(rslv, userRsrc, expectedErrorMsg);
 
 
-    rslv.setTargetUsers(new URI(userlist.toString()), conf);
-    UserGroupInformation ugi1;
-    assertEquals("user0", 
-        rslv.getTargetUgi((ugi1 = 
-          UserGroupInformation.createRemoteUser("hfre0"))).getUserName());
-    assertEquals("user1", rslv.getTargetUgi(UserGroupInformation.createRemoteUser("hfre1")).getUserName());
-    assertEquals("user2", rslv.getTargetUgi(UserGroupInformation.createRemoteUser("hfre2")).getUserName());
+    // Create user resource file with valid content like older users list file
+    // with usernames and groups
+    writeUserList(usersFilePath,
+    "user0,groupA,groupB,groupC\nuser1,groupA,groupC\n");
+    validateValidUsersFile(rslv, userRsrc);
+
+    // Create user resource file with valid content with
+    // usernames with groups and without groups
+    writeUserList(usersFilePath, "user0,groupA,groupB\nuser1,");
+    validateValidUsersFile(rslv, userRsrc);
+
+    // Create user resource file with valid content with
+    // usernames without groups
+    writeUserList(usersFilePath, "user0\nuser1");
+    validateValidUsersFile(rslv, userRsrc);
+  }
+
+  // Validate RoundRobinUserResolver for the case of
+  // user resource file with valid content.
+  private void validateValidUsersFile(UserResolver rslv, URI userRsrc)
+      throws IOException {
+    assertTrue(rslv.setTargetUsers(userRsrc, conf));
+    UserGroupInformation ugi1 = UserGroupInformation.createRemoteUser("hfre0");
     assertEquals("user0", rslv.getTargetUgi(ugi1).getUserName());
     assertEquals("user0", rslv.getTargetUgi(ugi1).getUserName());
-    assertEquals("user3", rslv.getTargetUgi(UserGroupInformation.createRemoteUser("hfre3")).getUserName());
+    assertEquals("user1",
+        rslv.getTargetUgi(UserGroupInformation.createRemoteUser("hfre1"))
+            .getUserName());
+    assertEquals("user0",
+        rslv.getTargetUgi(UserGroupInformation.createRemoteUser("hfre2"))
+            .getUserName());
     assertEquals("user0", rslv.getTargetUgi(ugi1).getUserName());
     assertEquals("user0", rslv.getTargetUgi(ugi1).getUserName());
+    assertEquals("user1",
+        rslv.getTargetUgi(UserGroupInformation.createRemoteUser("hfre3"))
+            .getUserName());
+
+    // Verify if same user comes again, its mapped user name should be
+    // correct even though UGI is constructed again.
+    assertEquals("user0", rslv.getTargetUgi(
+        UserGroupInformation.createRemoteUser("hfre0")).getUserName());
+    assertEquals("user0",
+        rslv.getTargetUgi(UserGroupInformation.createRemoteUser("hfre5"))
+        .getUserName());
+    assertEquals("user0",
+        rslv.getTargetUgi(UserGroupInformation.createRemoteUser("hfre0"))
+        .getUserName());
   }
   }
 
 
   @Test
   @Test
   public void testSubmitterResolver() throws Exception {
   public void testSubmitterResolver() throws Exception {
-    final Configuration conf = new Configuration();
     final UserResolver rslv = new SubmitterUserResolver();
     final UserResolver rslv = new SubmitterUserResolver();
-    rslv.setTargetUsers(null, conf);
+    assertFalse(rslv.needsTargetUsersList());
     UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
     UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
     assertEquals(ugi, rslv.getTargetUgi((UserGroupInformation)null));
     assertEquals(ugi, rslv.getTargetUgi((UserGroupInformation)null));
-    System.out.println(" Submitter current user " + ugi);
-    System.out.println(
-      " Target ugi " + rslv.getTargetUgi(
-        (UserGroupInformation) null));
   }
   }
 
 
 }
 }

+ 9 - 0
src/core/org/apache/hadoop/util/Progress.java

@@ -96,6 +96,7 @@ public class Progress {
     return node.getInternal();
     return node.getInternal();
   }
   }
 
 
+  
   /** Computes progress in this node. */
   /** Computes progress in this node. */
   private synchronized float getInternal() {
   private synchronized float getInternal() {
     int phaseCount = phases.size();
     int phaseCount = phases.size();
@@ -108,6 +109,14 @@ public class Progress {
     }
     }
   }
   }
 
 
+  /**
+   * Returns progress in this node. get() would give overall progress of the
+   * root node(not just given current node).
+   */
+  public synchronized float getProgress() {
+    return getInternal();
+  }
+
   public synchronized void setStatus(String status) {
   public synchronized void setStatus(String status) {
     this.status = status;
     this.status = status;
   }
   }

+ 757 - 139
src/docs/src/documentation/content/xdocs/gridmix.xml

@@ -15,150 +15,768 @@
   See the License for the specific language governing permissions and
   See the License for the specific language governing permissions and
   limitations under the License.
   limitations under the License.
 -->
 -->
-
 <!DOCTYPE document PUBLIC "-//APACHE//DTD Documentation V2.0//EN" "http://forrest.apache.org/dtd/document-v20.dtd">
 <!DOCTYPE document PUBLIC "-//APACHE//DTD Documentation V2.0//EN" "http://forrest.apache.org/dtd/document-v20.dtd">
-
 <document>
 <document>
+  <header>
+    <title>GridMix</title>
+  </header>
+  <body>
+    <section id="overview">
+      <title>Overview</title>
+      <p>GridMix is a benchmark for Hadoop clusters. It submits a mix of
+      synthetic jobs, modeling a profile mined from production loads.</p>
+      <p>There exist three versions of the GridMix tool. This document
+      discusses the third (checked into <code>src/contrib</code>), distinct
+      from the two checked into the <code>src/benchmarks</code> sub-directory.
+      While the first two versions of the tool included stripped-down versions
+      of common jobs, both were principally saturation tools for stressing the
+      framework at scale. In support of a broader range of deployments and
+      finer-tuned job mixes, this version of the tool will attempt to model
+      the resource profiles of production jobs to identify bottlenecks, guide
+      development, and serve as a replacement for the existing GridMix
+      benchmarks.</p>
+      <p>To run GridMix, you need a MapReduce job trace describing the job mix
+      for a given cluster. Such traces are typically generated by Rumen (see
+      Rumen documentation). GridMix also requires input data from which the
+      synthetic jobs will be reading bytes. The input data need not be in any
+      particular format, as the synthetic jobs are currently binary readers.
+      If you are running on a new cluster, an optional step generating input
+      data may precede the run.</p>
+      <p>In order to emulate the load of production jobs from a given cluster
+      on the same or another cluster, follow these steps:</p>
+      <ol>
+	<li>Locate the job history files on the production cluster. This
+	location is specified by the
+	<code>mapred.job.tracker.history.completed.location</code>
+	configuration property of the cluster.</li>
+	<li>Run Rumen to build a job trace in JSON format for all or select
+	jobs.</li>
+	<li>Use GridMix with the job trace on the benchmark cluster.</li>
+      </ol>
+      <p>Jobs submitted by GridMix have names of the form
+      &quot;<code>GRIDMIXnnnnnn</code>&quot;, where
+      &quot;<code>nnnnnn</code>&quot; is a sequence number padded with leading
+      zeroes.</p>
+    </section>
+    <section id="usage">
+      <title>Usage</title>
+      <p>Basic command-line usage without configuration parameters:</p>
+      <source>
+org.apache.hadoop.mapred.gridmix.Gridmix [-generate &lt;size&gt;] [-users &lt;users-list&gt;] &lt;iopath&gt; &lt;trace&gt;
+      </source>
+      <p>Basic command-line usage with configuration parameters:</p>
+      <source>
+org.apache.hadoop.mapred.gridmix.Gridmix \
+  -Dgridmix.client.submit.threads=10 -Dgridmix.output.directory=foo \
+  [-generate &lt;size&gt;] [-users &lt;users-list&gt;] &lt;iopath&gt; &lt;trace&gt;
+      </source>
+      <note>
+	Configuration parameters like
+	<code>-Dgridmix.client.submit.threads=10</code> and
+	<code>-Dgridmix.output.directory=foo</code> as given above should
+	be used <em>before</em> other GridMix parameters.
+      </note>
+      <p>The <code>&lt;iopath&gt;</code> parameter is the working directory for
+      GridMix. Note that this can either be on the local file-system
+      or on HDFS, but it is highly recommended that it be the same as that for
+      the original job mix so that GridMix puts the same load on the local
+      file-system and HDFS respectively.</p>
+      <p>The <code>-generate</code> option is used to generate input data and
+      Distributed Cache files for the synthetic jobs. It accepts standard units
+      of size suffixes, e.g. <code>100g</code> will generate
+      100 * 2<sup>30</sup> bytes as input data.
+      <code>&lt;iopath&gt;/input</code> is the destination directory for
+      generated input data and/or the directory from which input data will be
+      read. HDFS-based Distributed Cache files are generated under the
+      distributed cache directory <code>&lt;iopath&gt;/distributedCache</code>.
+      If some of the needed Distributed Cache files are already existing in the
+      distributed cache directory, then only the remaining non-existing
+      Distributed Cache files are generated when <code>-generate</code> option
+      is specified.</p>
+      <p>The <code>-users</code> option is used to point to a users-list
+      file (see <a href="#usersqueues">Emulating Users and Queues</a>).</p>
+      <p>The <code>&lt;trace&gt;</code> parameter is a path to a job trace
+      generated by Rumen. This trace can be compressed (it must be readable
+      using one of the compression codecs supported by the cluster) or
+      uncompressed. Use &quot;-&quot; as the value of this parameter if you
+      want to pass an <em>uncompressed</em> trace via the standard
+      input-stream of GridMix.</p>
+      <p>The class <code>org.apache.hadoop.mapred.gridmix.Gridmix</code> can
+      be found in the JAR
+      <code>contrib/gridmix/hadoop-gridmix-$VERSION.jar</code> inside your
+      Hadoop installation, where <code>$VERSION</code> corresponds to the
+      version of Hadoop installed. A simple way of ensuring that this class
+      and all its dependencies are loaded correctly is to use the
+      <code>hadoop</code> wrapper script in Hadoop:</p>
+      <source>
+hadoop jar &lt;gridmix-jar&gt; org.apache.hadoop.mapred.gridmix.Gridmix \
+  [-generate &lt;size&gt;] [-users &lt;users-list&gt;] &lt;iopath&gt; &lt;trace&gt;
+      </source>
+      <p>The supported configuration parameters are explained in the
+      following sections.</p>
+    </section>
+    <section id="cfgparams">
+      <title>General Configuration Parameters</title>
+      <p/>
+      <table>
+        <tr>
+          <th>Parameter</th>
+          <th>Description</th>
+        </tr>
+        <tr>
+          <td>
+            <code>gridmix.output.directory</code>
+          </td>
+          <td>The directory into which output will be written. If specified,
+	  <code>iopath</code> will be relative to this parameter. The
+	  submitting user must have read/write access to this directory. The
+	  user should also be mindful of any quota issues that may arise
+	  during a run. The default is &quot;<code>gridmix</code>&quot;.</td>
+        </tr>
+        <tr>
+          <td>
+            <code>gridmix.client.submit.threads</code>
+          </td>
+          <td>The number of threads submitting jobs to the cluster. This
+	  also controls how many splits will be loaded into memory at a given
+	  time, pending the submit time in the trace. Splits are pre-generated
+	  to hit submission deadlines, so particularly dense traces may want
+	  more submitting threads. However, storing splits in memory is
+	  reasonably expensive, so you should raise this cautiously. The
+	  default is 1 for the SERIAL job-submission policy (see
+	  <a href="#policies">Job Submission Policies</a>) and one more than
+	  the number of processors on the client machine for the other
+	  policies.</td>
+        </tr>
+	<tr>
+	  <td>
+	    <code>gridmix.submit.multiplier</code>
+	  </td>
+	  <td>The multiplier to accelerate or decelerate the submission of
+	  jobs. The time separating two jobs is multiplied by this factor.
+	  The default value is 1.0. This is a crude mechanism to size
+	  a job trace to a cluster.</td>
+	</tr>
+        <tr>
+          <td>
+            <code>gridmix.client.pending.queue.depth</code>
+          </td>
+          <td>The depth of the queue of job descriptions awaiting split
+	  generation. The jobs read from the trace occupy a queue of this
+	  depth before being processed by the submission threads. It is
+	  unusual to configure this. The default is 5.</td>
+        </tr>
+	<tr>
+	  <td>
+	    <code>gridmix.gen.blocksize</code>
+	  </td>
+	  <td>The block-size of generated data. The default value is 256
+	  MiB.</td>
+	</tr>
+	<tr>
+	  <td>
+	    <code>gridmix.gen.bytes.per.file</code>
+	  </td>
+	  <td>The maximum bytes written per file. The default value is 1
+	  GiB.</td>
+	</tr>
+        <tr>
+          <td>
+            <code>gridmix.min.file.size</code>
+          </td>
+          <td>The minimum size of the input files. The default limit is 128
+	  MiB. Tweak this parameter if you see an error-message like
+	  &quot;Found no satisfactory file&quot; while testing GridMix with
+	  a relatively-small input data-set.</td>
+        </tr>
+        <tr>
+          <td>
+            <code>gridmix.max.total.scan</code>
+          </td>
+          <td>The maximum size of the input files. The default limit is 100
+	  TiB.</td>
+        </tr>
+        <tr>
+          <td>
+            <code>gridmix.task.jvm-options.enable</code>
+          </td>
+          <td>Enables Gridmix to configure the simulated task's max heap 
+              options using the values obtained from the original task (i.e via
+              trace).
+          </td>
+        </tr>
+      </table>
+    </section>
+    <section id="jobtypes">
+      <title>Job Types</title>
+      <p>GridMix takes as input a job trace, essentially a stream of
+      JSON-encoded job descriptions. For each job description, the submission
+      client obtains the original job submission time and for each task in
+      that job, the byte and record counts read and written. Given this data,
+      it constructs a synthetic job with the same byte and record patterns as
+      recorded in the trace. It constructs jobs of two types:</p>
+      <table>
+        <tr>
+          <th>Job Type</th>
+          <th>Description</th>
+        </tr>
+        <tr>
+          <td>
+            <code>LOADJOB</code>
+          </td>
+          <td>A synthetic job that emulates the workload mentioned in Rumen
+	  trace. In the current version we are supporting I/O. It reproduces
+	  the I/O workload on the benchmark cluster. It does so by embedding
+	  the detailed I/O information for every map and reduce task, such as
+	  the number of bytes and records read and written, into each
+	  job's input splits. The map tasks further relay the I/O patterns of
+	  reduce tasks through the intermediate map output data.</td>
+        </tr>
+        <tr>
+          <td>
+            <code>SLEEPJOB</code>
+          </td>
+	  <td>A synthetic job where each task does <em>nothing</em> but sleep
+	  for a certain duration as observed in the production trace. The
+	  scalability of the Job Tracker is often limited by how many
+	  heartbeats it can handle every second. (Heartbeats are periodic
+	  messages sent from Task Trackers to update their status and grab new
+	  tasks from the Job Tracker.) Since a benchmark cluster is typically
+	  a fraction in size of a production cluster, the heartbeat traffic
+	  generated by the slave nodes is well below the level of the
+	  production cluster. One possible solution is to run multiple Task
+	  Trackers on each slave node. This leads to the obvious problem that
+	  the I/O workload generated by the synthetic jobs would thrash the
+	  slave nodes. Hence the need for such a job.</td>
+        </tr>
+      </table>
+      <p>The following configuration parameters affect the job type:</p>
+      <table>
+        <tr>
+          <th>Parameter</th>
+          <th>Description</th>
+        </tr>
+        <tr>
+          <td>
+            <code>gridmix.job.type</code>
+          </td>
+          <td>The value for this key can be one of LOADJOB or SLEEPJOB. The
+	  default value is LOADJOB.</td>
+        </tr>
+        <tr>
+          <td>
+            <code>gridmix.key.fraction</code>
+          </td>
+          <td>For a LOADJOB type of job, the fraction of a record used for
+	  the data for the key. The default value is 0.1.</td>
+        </tr>
+        <tr>
+          <td>
+            <code>gridmix.sleep.maptask-only</code>
+          </td>
+          <td>For a SLEEPJOB type of job, whether to ignore the reduce
+	  tasks for the job. The default is <code>false</code>.</td>
+        </tr>
+        <tr>
+          <td>
+            <code>gridmix.sleep.fake-locations</code>
+          </td>
+          <td>For a SLEEPJOB type of job, the number of fake locations
+	  for map tasks for the job. The default is 0.</td>
+        </tr>
+        <tr>
+          <td>
+            <code>gridmix.sleep.max-map-time</code>
+          </td>
+          <td>For a SLEEPJOB type of job, the maximum runtime for map
+	  tasks for the job in milliseconds. The default is unlimited.</td>
+        </tr>
+        <tr>
+          <td>
+            <code>gridmix.sleep.max-reduce-time</code>
+          </td>
+          <td>For a SLEEPJOB type of job, the maximum runtime for reduce
+	  tasks for the job in milliseconds. The default is unlimited.</td>
+        </tr>
+      </table>
+    </section>
+    <section id="policies">
+      <title>Job Submission Policies</title>
+      <p>GridMix controls the rate of job submission. This control can be
+      based on the trace information or can be based on statistics it gathers
+      from the Job Tracker. Based on the submission policies users define,
+      GridMix uses the respective algorithm to control the job submission.
+      There are currently three types of policies:</p>
+      <table>
+        <tr>
+          <th>Job Submission Policy</th>
+          <th>Description</th>
+        </tr>
+        <tr>
+          <td>
+            <code>STRESS</code>
+          </td>
+          <td>Keep submitting jobs so that the cluster remains under stress.
+	  In this mode we control the rate of job submission by monitoring
+	  the real-time load of the cluster so that we can maintain a stable
+	  stress level of workload on the cluster. Based on the statistics we
+	  gather we define if a cluster is <em>underloaded</em> or
+	  <em>overloaded</em>. We consider a cluster <em>underloaded</em> if
+	  and only if the following three conditions are true:
+	  <ol>
+	    <li>the number of pending and running jobs are under a threshold
+	    TJ</li>
+	    <li>the number of pending and running maps are under threshold
+	    TM</li>
+	    <li>the number of pending and running reduces are under threshold
+	    TR</li>
+	  </ol>
+          The thresholds TJ, TM and TR are proportional to the size of the
+	  cluster and map, reduce slots capacities respectively. In case of a
+	  cluster being <em>overloaded</em>, we throttle the job submission.
+	  In the actual calculation we also weigh each running task with its
+	  remaining work - namely, a 90% complete task is only counted as 0.1
+	  in calculation. Finally, to avoid a very large job blocking other
+	  jobs, we limit the number of pending/waiting tasks each job can
+	  contribute.</td>
+        </tr>
+        <tr>
+          <td>
+            <code>REPLAY</code>
+          </td>
+          <td>In this mode we replay the job traces faithfully. This mode
+	  exactly follows the time-intervals given in the actual job
+	  trace.</td>
+        </tr>
+        <tr>
+          <td>
+            <code>SERIAL</code>
+          </td>
+          <td>In this mode we submit the next job only once the job submitted
+	  earlier is completed.</td>
+        </tr>
+      </table>
+      <p>The following configuration parameters affect the job submission
+      policy:</p>
+      <table>
+        <tr>
+          <th>Parameter</th>
+          <th>Description</th>
+        </tr>
+        <tr>
+          <td>
+            <code>gridmix.job-submission.policy</code>
+          </td>
+          <td>The value for this key would be one of the three: STRESS, REPLAY
+	  or SERIAL. In most of the cases the value of key would be STRESS or
+	  REPLAY. The default value is STRESS.</td>
+        </tr>
+        <tr>
+          <td>
+            <code>gridmix.throttle.jobs-to-tracker-ratio</code>
+          </td>
+          <td>In STRESS mode, the minimum ratio of running jobs to Task
+	  Trackers in a cluster for the cluster to be considered
+	  <em>overloaded</em>. This is the threshold TJ referred to earlier.
+	  The default is 1.0.</td>
+        </tr>
+        <tr>
+          <td>
+            <code>gridmix.throttle.maps.task-to-slot-ratio</code>
+          </td>
+          <td>In STRESS mode, the minimum ratio of pending and running map
+	  tasks (i.e. incomplete map tasks) to the number of map slots for
+	  a cluster for the cluster to be considered <em>overloaded</em>.
+	  This is the threshold TM referred to earlier. Running map tasks are
+	  counted partially. For example, a 40% complete map task is counted
+	  as 0.6 map tasks. The default is 2.0.</td>
+        </tr>
+        <tr>
+          <td>
+            <code>gridmix.throttle.reduces.task-to-slot-ratio</code>
+          </td>
+          <td>In STRESS mode, the minimum ratio of pending and running reduce
+	  tasks (i.e. incomplete reduce tasks) to the number of reduce slots
+	  for a cluster for the cluster to be considered <em>overloaded</em>.
+	  This is the threshold TR referred to earlier. Running reduce tasks
+	  are counted partially. For example, a 30% complete reduce task is
+	  counted as 0.7 reduce tasks. The default is 2.5.</td>
+        </tr>
+        <tr>
+          <td>
+            <code>gridmix.throttle.maps.max-slot-share-per-job</code>
+          </td>
+          <td>In STRESS mode, the maximum share of a cluster's map-slots
+	  capacity that can be counted toward a job's incomplete map tasks in
+	  overload calculation. The default is 0.1.</td>
+        </tr>
+        <tr>
+          <td>
+            <code>gridmix.throttle.reducess.max-slot-share-per-job</code>
+          </td>
+          <td>In STRESS mode, the maximum share of a cluster's reduce-slots
+	  capacity that can be counted toward a job's incomplete reduce tasks
+	  in overload calculation. The default is 0.1.</td>
+        </tr>
+      </table>
+    </section>
+    <section id="usersqueues">
+      <title>Emulating Users and Queues</title>
+      <p>Typical production clusters are often shared with different users and
+      the cluster capacity is divided among different departments through job
+      queues. Ensuring fairness among jobs from all users, honoring queue
+      capacity allocation policies and avoiding an ill-behaving job from
+      taking over the cluster adds significant complexity in Hadoop software.
+      To be able to sufficiently test and discover bugs in these areas,
+      GridMix must emulate the contentions of jobs from different users and/or
+      submitted to different queues.</p>
+      <p>Emulating multiple queues is easy - we simply set up the benchmark
+      cluster with the same queue configuration as the production cluster and
+      we configure synthetic jobs so that they get submitted to the same queue
+      as recorded in the trace. However, not all users shown in the trace have
+      accounts on the benchmark cluster. Instead, we set up a number of testing
+      user accounts and associate each unique user in the trace to testing
+      users in a round-robin fashion.</p>
+      <p>The following configuration parameters affect the emulation of users
+      and queues:</p>
+      <table>
+        <tr>
+          <th>Parameter</th>
+          <th>Description</th>
+        </tr>
+        <tr>
+          <td>
+            <code>gridmix.job-submission.use-queue-in-trace</code>
+          </td>
+          <td>When set to <code>true</code> it uses exactly the same set of
+	  queues as those mentioned in the trace. The default value is
+	  <code>false</code>.</td>
+        </tr>
+        <tr>
+          <td>
+            <code>gridmix.job-submission.default-queue</code>
+          </td>
+          <td>Specifies the default queue to which all the jobs would be
+	  submitted. If this parameter is not specified, GridMix uses the
+	  default queue defined for the submitting user on the cluster.</td>
+        </tr>
+        <tr>
+          <td>
+            <code>gridmix.user.resolve.class</code>
+          </td>
+          <td>Specifies which <code>UserResolver</code> implementation to use.
+	  We currently have three implementations:
+	  <ol>
+	    <li><code>org.apache.hadoop.mapred.gridmix.EchoUserResolver</code>
+	    - submits a job as the user who submitted the original job. All
+	    the users of the production cluster identified in the job trace
+	    must also have accounts on the benchmark cluster in this case.</li>
+	    <li><code>org.apache.hadoop.mapred.gridmix.SubmitterUserResolver</code>
+	    - submits all the jobs as current GridMix user. In this case we
+	    simply map all the users in the trace to the current GridMix user
+	    and submit the job.</li>
+	    <li><code>org.apache.hadoop.mapred.gridmix.RoundRobinUserResolver</code>
+	    - maps trace users to test users in a round-robin fashion. In
+	    this case we set up a number of testing user accounts and
+	    associate each unique user in the trace to testing users in a
+	    round-robin fashion.</li>
+	  </ol>
+	  The default is
+	  <code>org.apache.hadoop.mapred.gridmix.SubmitterUserResolver</code>.</td>
+        </tr>
+      </table>
+      <p>If the parameter <code>gridmix.user.resolve.class</code> is set to
+      <code>org.apache.hadoop.mapred.gridmix.RoundRobinUserResolver</code>,
+      we need to define a users-list file with a list of test users.
+      This is specified using the <code>-users</code> option to GridMix.</p>
+      <note>
+      Specifying a users-list file using the <code>-users</code> option is
+      mandatory when using the round-robin user-resolver. Other user-resolvers
+      ignore this option.
+      </note>
+      <p>A users-list file has one user per line, each line of the format:</p>
+      <source>
+      &lt;username&gt;
+      </source>
+      <p>For example:</p>
+      <source>
+      user1
+      user2
+      user3
+      </source>
+      <p>In the above example we have defined three users <code>user1</code>,
+      <code>user2</code> and <code>user3</code>.
+      Now we would associate each unique user in the trace to the above users
+      defined in round-robin fashion. For example, if trace's users are
+      <code>tuser1</code>, <code>tuser2</code>, <code>tuser3</code>,
+      <code>tuser4</code> and <code>tuser5</code>, then the mappings would
+      be:</p>
+      <source>
+      tuser1 -&gt; user1
+      tuser2 -&gt; user2
+      tuser3 -&gt; user3
+      tuser4 -&gt; user1
+      tuser5 -&gt; user2
+      </source>
+      <p>For backward compatibility reasons, each line of users-list file can
+      contain username followed by groupnames in the form username[,group]*.
+      The groupnames will be ignored by Gridmix.
+      </p>
+    </section>
 
 
-<header>
-  <title>Gridmix</title>
-</header>
-
-<body>
-
-  <section>
-  <title>Overview</title>
-
-  <p>Gridmix is a benchmark for live clusters. It submits a mix of synthetic
-  jobs, modeling a profile mined from production loads.</p>
-
-  <p>There exist three versions of the Gridmix tool. This document discusses
-  the third (checked into contrib), distinct from the two checked into the
-  benchmarks subdirectory. While the first two versions of the tool included
-  stripped-down versions of common jobs, both were principally saturation
-  tools for stressing the framework at scale. In support of a broader range of
-  deployments and finer-tuned job mixes, this version of the tool will attempt
-  to model the resource profiles of production jobs to identify bottlenecks,
-  guide development, and serve as a replacement for the existing gridmix
-  benchmarks.</p>
-
-  </section>
-
-  <section id="usage">
-
-  <title>Usage</title>
-
-  <p>To run Gridmix, one requires a job trace describing the job mix for a
-  given cluster. Such traces are typically genenerated by Rumen (see related
-  documentation). Gridmix also requires input data from which the synthetic
-  jobs will draw bytes. The input data need not be in any particular format,
-  as the synthetic jobs are currently binary readers. If one is running on a
-  new cluster, an optional step generating input data may precede the run.</p>
-
-  <p>Basic command line usage:</p>
-<source>
-
-bin/mapred org.apache.hadoop.mapred.gridmix.Gridmix [-generate &lt;MiB&gt;] &lt;iopath&gt; &lt;trace&gt;
-</source>
-
-  <p>The <code>-generate</code> parameter accepts standard units, e.g.
-  <code>100g</code> will generate 100 * 2<sup>30</sup> bytes. The
-  &lt;iopath&gt; parameter is the destination directory for generated and/or
-  the directory from which input data will be read. The &lt;trace&gt;
-  parameter is a path to a job trace. The following configuration parameters
-  are also accepted in the standard idiom, before other Gridmix
-  parameters.</p>
-
-  <section>
-  <title>Configuration parameters</title>
-  <p></p>
-  <table>
-    <tr><th> Parameter </th><th> Description </th><th> Notes </th></tr>
-    <tr><td><code>gridmix.output.directory</code></td>
-        <td>The directory into which output will be written. If specified, the
-        <code>iopath</code> will be relative to this parameter.</td>
-        <td>The submitting user must have read/write access to this
-        directory. The user should also be mindful of any quota issues that
-        may arise during a run.</td></tr>
-    <tr><td><code>gridmix.client.submit.threads</code></td>
-        <td>The number of threads submitting jobs to the cluster. This also
-        controls how many splits will be loaded into memory at a given time,
-        pending the submit time in the trace.</td>
-        <td>Splits are pregenerated to hit submission deadlines, so
-        particularly dense traces may want more submitting threads. However,
-        storing splits in memory is reasonably expensive, so one should raise
-        this cautiously.</td></tr>
-    <tr><td><code>gridmix.client.pending.queue.depth</code></td>
-        <td>The depth of the queue of job descriptions awaiting split
-        generation.</td>
-        <td>The jobs read from the trace occupy a queue of this depth before
-        being processed by the submission threads. It is unusual to configure
-        this.</td></tr>
-    <tr><td><code>gridmix.min.key.length</code></td>
-        <td>The key size for jobs submitted to the cluster.</td>
-        <td>While this is clearly a job-specific, even task-specific property,
-        no data on key length is currently available. Since the intermediate
-        data are random, memcomparable data, not even the sort is likely
-        affected. It exists as a tunable as no default value is appropriate,
-        but future versions will likely replace it with trace data.</td></tr>
-  </table>
-
+  <section id="distributedcacheload">
+  <title>Emulating Distributed Cache Load</title>
+    <p>Gridmix emulates Distributed Cache load by default for LOADJOB type of
+    jobs. This is done by precreating the needed Distributed Cache files for all
+    the simulated jobs as part of a separate MapReduce job.</p>
+    <p>Emulation of Distributed Cache load in gridmix simulated jobs can be
+    disabled by configuring the property
+    <code>gridmix.distributed-cache-emulation.enable</code> to
+    <code>false</code>.
+    But generation of Distributed Cache data by gridmix is driven by
+    <code>-generate</code> option and is independent of this configuration
+    property.</p>
+    <p>Both generation of Distributed Cache files and emulation of
+    Distributed Cache load are disabled if:</p>
+    <ul>
+    <li>input trace comes from the standard input-stream instead of file, or</li>
+    <li><code>&lt;iopath&gt;</code> specified is on local file-system, or</li>
+    <li>any of the ascendant directories of the distributed cache directory
+    i.e. <code>&lt;iopath&gt;/distributedCache</code> (including the distributed
+    cache directory) doesn't have execute permission for others.</li>
+    </ul>
   </section>
   </section>
-</section>
-
-<section id="assumptions">
-
-  <title>Simplifying Assumptions</title>
-
-  <p>Gridmix will be developed in stages, incorporating feedback and patches
-  from the community. Currently, its intent is to evaluate Map/Reduce and HDFS
-  performance and not the layers on top of them (i.e. the extensive lib and
-  subproject space). Given these two limitations, the following
-  characteristics of job load are not currently captured in job traces and
-  cannot be accurately reproduced in Gridmix.</p>
-
-  <table>
-  <tr><th>Property</th><th>Notes</th></tr>
-  <tr><td>CPU usage</td><td>We have no data for per-task CPU usage, so we
-  cannot attempt even an approximation. Gridmix tasks are never CPU bound
-  independent of I/O, though this surely happens in practice.</td></tr>
-  <tr><td>Filesystem properties</td><td>No attempt is made to match block
-  sizes, namespace hierarchies, or any property of input, intermediate, or
-  output data other than the bytes/records consumed and emitted from a given
-  task. This implies that some of the most heavily used parts of the system-
-  the compression libraries, text processing, streaming, etc.- cannot be
-  meaningfully tested with the current implementation.</td></tr>
-  <tr><td>I/O rates</td><td>The rate at which records are consumed/emitted is
-  assumed to be limited only by the speed of the reader/writer and constant
-  throughout the task.</td></tr>
-  <tr><td>Memory profile</td><td>No data on tasks' memory usage over time is
-  available, though the max heap size is retained.</td></tr>
-  <tr><td>Skew</td><td>The records consumed and emitted to/from a given task
-  are assumed to follow observed averages, i.e. records will be more regular
-  than may be seen in the wild. Each map also generates a proportional
-  percentage of data for each reduce, so a job with unbalanced input will be
-  flattened.</td></tr>
-  <tr><td>Job failure</td><td>User code is assumed to be correct.</td></tr>
-  <tr><td>Job independence</td><td>The output or outcome of one job does not
-  affect when or whether a subsequent job will run.</td></tr>
-  </table>
-
-</section>
-
-<section>
-
-  <title>Appendix</title>
-
-  <p>Issues tracking the implementations of <a
-  href="https://issues.apache.org/jira/browse/HADOOP-2369">gridmix1</a>, <a
-  href="https://issues.apache.org/jira/browse/HADOOP-3770">gridmix2</a>, and
-  <a href="https://issues.apache.org/jira/browse/MAPREDUCE-776">gridmix3</a>.
-  Other issues tracking the development of Gridmix can be found by searching
-  the Map/Reduce <a
-  href="https://issues.apache.org/jira/browse/MAPREDUCE">JIRA</a></p>
 
 
-</section>
+    <section id="simulatedjobconf">
+      <title>Configuration of Simulated Jobs</title>
+      <p> Gridmix3 sets some configuration properties in the simulated Jobs
+      submitted by it so that they can be mapped back to the corresponding Job
+      in the input Job trace. These configuration parameters include:
+      </p>
+      <table>
+        <tr>
+          <th>Parameter</th>
+          <th>Description</th>
+        </tr>
+        <tr>
+          <td>
+            <code>gridmix.job.original-job-id</code>
+          </td>
+          <td> The job id of the original cluster's job corresponding to this
+          simulated job.
+          </td>
+        </tr>
+        <tr>
+          <td>
+            <code>gridmix.job.original-job-name</code>
+          </td>
+          <td> The job name of the original cluster's job corresponding to this
+          simulated job.
+          </td>
+        </tr>
+      </table>
+    </section>
 
 
-</body>
+  <section id="compression-emulation">
+      <title>Emulating Compression/Decompression</title>
+      <p>MapReduce supports data compression and decompression. 
+         Input to a MapReduce job can be compressed. Similarly, output of Map
+         and Reduce tasks can also be compressed. Compression/Decompression 
+         emulation in GridMix is important because emulating 
+         compression/decompression will effect the CPU and Memory usage of the 
+         task. A task emulating compression/decompression will affect other 
+         tasks and daemons running on the same node.
+       </p>
+       <p>Compression emulation is enabled if 
+         <code>gridmix.compression-emulation.enable</code> is set to
+         <code>true</code>. By default compression emulation is enabled for 
+         jobs of type <em>LOADJOB</em>. With compression emulation enabled, 
+         GridMix will now generate compressed text data with a constant 
+         compression ratio. Hence a simulated GridMix job will now emulate 
+         compression/decompression using compressible text data (having a 
+         constant compression ratio), irrespective of the compression ratio 
+         observed in the actual job.
+      </p>
+      <p>A typical MapReduce Job deals with data compression/decompression in 
+         the following phases </p>
+      <ul>
+        <li><code>Job input data decompression: </code> GridMix generates 
+            compressible input data when compression emulation is enabled. 
+            Based on the original job's configuration, a simulated GridMix job 
+            will use a decompressor to read the compressed input data. 
+            Currently, GridMix uses
+            <code>mapreduce.input.fileinputformat.inputdir</code> to determine 
+            if the original job used compressed input data or 
+            not. If the original job's input files are uncompressed then the 
+            simulated job will read the compressed input file without using a 
+            decompressor. 
+        </li>
+        <li><code>Intermediate data compression and decompression: </code>
+            If the original job has map output compression enabled then GridMix 
+            too will enable map output compression for the simulated job. 
+            Accordingly, the reducers will use a decompressor to read the map 
+            output data.
+        </li>
+        <li><code>Job output data compression: </code>
+            If the original job's output is compressed then GridMix 
+            too will enable job output compression for the simulated job. 
+        </li>
+      </ul>
+       
+      <p>The following configuration parameters affect compression emulation
+      </p>
+      <table>
+        <tr>
+          <th>Parameter</th>
+          <th>Description</th>
+        </tr>
+        <tr>
+          <td>gridmix.compression-emulation.enable</td>
+          <td>Enables compression emulation in simulated GridMix jobs. 
+              Default is true.</td>
+        </tr>
+      </table>
+      
+      <p>With compression emulation turned on, GridMix will generate compressed
+         input data. Hence the total size of the input 
+         data will be lesser than the expected size. Set 
+         <code>gridmix.min.file.size</code> to a smaller value (roughly 10% of
+         <code>gridmix.gen.bytes.per.file</code>) for enabling GridMix to 
+         correctly emulate compression.
+      </p>
+    </section>
 
 
+    <section id="highram-emulation">
+      <title>Emulating High-Ram jobs</title>
+      <p>MapReduce allows users to define a job as a High-Ram job. Tasks from a 
+         High-Ram job can occupy multiple slots on the task-trackers. 
+         Task-tracker assigns fixed virtual memory for each slot. Tasks from 
+         High-Ram jobs can occupy multiple slots and thus can use up more 
+         virtual memory as compared to a default task.
+      </p>
+      <p>Emulating this behavior is important because of the following reasons
+      </p>
+     <ul>
+       <li>Impact on scheduler:  Scheduling of tasks from High-Ram jobs 
+           impacts the scheduling behavior as it might result into slot 
+           reservation and slot/resource utilization.
+       </li>
+       <li>Impact on the node : Since High-Ram tasks occupy multiple slots,
+           trackers do some bookkeeping for allocating extra resources for 
+           these tasks. Thus this becomes a precursor for memory emulation
+           where tasks with high memory requirements needs to be considered
+           as a High-Ram task.
+       </li>
+     </ul>
+     <p>High-Ram feature emulation can be disabled by setting  
+        <code>gridmix.highram-emulation.enable</code> to
+        <code>false</code>.
+     </p>
+    </section>
+    
+    <section id="resource-usage-emulation">
+      <title>Emulating resource usages</title>
+      <p>Usages of resources like CPU, physical memory, virtual memory, JVM heap
+         etc are recorded by MapReduce using its task counters. This information
+         is used by GridMix to emulate the resource usages in the simulated 
+         tasks. Emulating resource usages will help GridMix exert similar load 
+         on the test cluster as seen in the actual cluster.
+      </p>
+      <p>MapReduce tasks use up resources during its entire lifetime. GridMix
+         also tries to mimic this behavior by spanning resource usage emulation
+         across the entire lifetime of the simulated task. Each resource to be
+         emulated should have an <em>emulator</em> associated with it.
+         Each such <em>emulator</em> should implement the 
+         <code>org.apache.hadoop.mapred.gridmix.emulators.resourceusage
+         .ResourceUsageEmulatorPlugin</code> interface. Resource 
+         <em>emulators</em> in GridMix are <em>plugins</em> that can be 
+         configured (plugged in or out) before every run. GridMix users can 
+         configure multiple emulator <em>plugins</em> by passing a comma 
+         separated list of <em>emulators</em> as a value for the 
+         <code>gridmix.emulators.resource-usage.plugins</code> parameter. 
+      </p>
+      <p>List of <em>emulators</em> shipped with GridMix:
+      </p>
+     <ul>
+       <li>Cumulative CPU usage <em>emulator</em>: 
+           GridMix uses the cumulative CPU usage value published by Rumen 
+           and makes sure that the total cumulative CPU usage of the simulated 
+           task is close to the value published by Rumen. GridMix can be 
+           configured to emulate cumulative CPU usage by adding 
+           <code>org.apache.hadoop.mapred.gridmix.emulators.resourceusage
+           .CumulativeCpuUsageEmulatorPlugin</code> to the list of emulator 
+           <em>plugins</em> configured for the 
+           <code>gridmix.emulators.resource-usage.plugins</code> parameter.
+           CPU usage emulator is designed in such a way that
+           it only emulates at specific progress boundaries of the task. This 
+           interval can be configured using 
+           <code>gridmix.emulators.resource-usage.cpu.emulation-interval</code>.
+           The default value for this parameter is <code>0.1</code> i.e 
+           <code>10%</code>.
+       </li>
+       <li>Total heap usage <em>emulator</em>: 
+           GridMix uses the total heap usage value published by Rumen 
+           and makes sure that the total heap usage of the simulated 
+           task is close to the value published by Rumen. GridMix can be 
+           configured to emulate total heap usage by adding 
+           <code>org.apache.hadoop.mapred.gridmix.emulators.resourceusage
+           .TotalHeapUsageEmulatorPlugin</code> to the list of emulator 
+           <em>plugins</em> configured for the 
+           <code>gridmix.emulators.resource-usage.plugins</code> parameter.
+           Heap usage emulator is designed in such a way that
+           it only emulates at specific progress boundaries of the task. This 
+           interval can be configured using 
+           <code>gridmix.emulators.resource-usage.heap.emulation-interval
+           </code>. The default value for this parameter is <code>0.1</code> 
+           i.e <code>10%</code> progress interval.
+</li>
+     </ul>
+     <p>Note that GridMix will emulate resource usages only for jobs of type 
+        <em>LOADJOB</em>.
+     </p>
+    </section>
+    
+    <section id="assumptions">
+      <title>Simplifying Assumptions</title>
+      <p>GridMix will be developed in stages, incorporating feedback and
+      patches from the community. Currently its intent is to evaluate
+      MapReduce and HDFS performance and not the layers on top of them (i.e.
+      the extensive lib and sub-project space). Given these two limitations,
+      the following characteristics of job load are not currently captured in
+      job traces and cannot be accurately reproduced in GridMix:</p>
+      <ul>
+	<li><em>Filesystem Properties</em> - No attempt is made to match block
+	sizes, namespace hierarchies, or any property of input, intermediate
+	or output data other than the bytes/records consumed and emitted from
+	a given task. This implies that some of the most heavily-used parts of
+	the system - text processing, streaming, etc. - cannot be meaningfully tested 
+	with the current implementation.</li>
+	<li><em>I/O Rates</em> - The rate at which records are
+	consumed/emitted is assumed to be limited only by the speed of the
+	reader/writer and constant throughout the task.</li>
+	<li><em>Memory Profile</em> - No data on tasks' memory usage over time
+	is available, though the max heap-size is retained.</li>
+	<li><em>Skew</em> - The records consumed and emitted to/from a given
+	task are assumed to follow observed averages, i.e. records will be
+	more regular than may be seen in the wild. Each map also generates
+	a proportional percentage of data for each reduce, so a job with
+	unbalanced input will be flattened.</li>
+	<li><em>Job Failure</em> - User code is assumed to be correct.</li>
+	<li><em>Job Independence</em> - The output or outcome of one job does
+	not affect when or whether a subsequent job will run.</li>
+      </ul>
+    </section>
+    <section id="appendix">
+      <title>Appendix</title>
+      <p>Issues tracking the original implementations of <a
+      href="https://issues.apache.org/jira/browse/HADOOP-2369">GridMix1</a>,
+      <a href="https://issues.apache.org/jira/browse/HADOOP-3770">GridMix2</a>,
+      and <a
+      href="https://issues.apache.org/jira/browse/MAPREDUCE-776">GridMix3</a>
+      can be found on the Apache Hadoop MapReduce JIRA. Other issues tracking
+      the current development of GridMix can be found by searching <a
+      href="https://issues.apache.org/jira/browse/MAPREDUCE/component/12313086">the
+      Apache Hadoop MapReduce JIRA</a></p>
+    </section>
+  </body>
 </document>
 </document>

+ 454 - 0
src/docs/src/documentation/content/xdocs/rumen.xml

@@ -0,0 +1,454 @@
+<?xml version="1.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.
+-->
+
+<!DOCTYPE document PUBLIC "-//APACHE//DTD Documentation V2.0//EN" "http://forrest.apache.org/dtd/document-v20.dtd">
+
+<document>
+
+<header>
+  <title>Rumen</title>
+</header>
+
+<body>
+  <!--
+    Overview [What is Rumen and why is it needed?]
+  -->
+  <section id="overview">
+    <title>Overview</title>
+    
+    <p><em>Rumen</em> is a data extraction and analysis tool built for
+       <em>Apache Hadoop</em>. <em>Rumen</em> mines <em>JobHistory</em> logs to 
+       extract meaningful data and stores it in an easily-parsed, condensed 
+       format or <em>digest</em>. The raw trace data from MapReduce logs are 
+       often insufficient for simulation, emulation, and benchmarking, as these 
+       tools often attempt to measure conditions that did not occur in the 
+       source data. For example, if a task ran locally in the raw trace data 
+       but a simulation of the scheduler elects to run that task on a remote 
+       rack, the simulator requires a runtime its input cannot provide. 
+       To fill in these gaps, Rumen performs a statistical analysis of the 
+       digest to estimate the variables the trace doesn't supply. Rumen traces 
+       drive both Gridmix (a benchmark of Hadoop MapReduce clusters) and Mumak 
+       (a simulator for the JobTracker).
+    </p>
+
+    <!--
+      Why is Rumen needed?
+    --> 
+    <section>     
+      <title>Motivation</title>
+      
+      <ul>
+        <li>Extracting meaningful data from <em>JobHistory</em> logs is a common
+            task for any tool built to work on <em>MapReduce</em>. It 
+            is tedious to write a custom tool which is so tightly coupled with 
+            the <em>MapReduce</em> framework. Hence there is a need for a 
+            built-in tool for performing framework level task of log parsing and
+            analysis. Such a tool would insulate external systems depending on 
+            job history against the changes made to the job history format.
+        </li>
+        <li>Performing statistical analysis of various attributes of a 
+            <em>MapReduce Job</em> such as <em>task runtimes, task failures 
+            etc</em> is another common task that the benchmarking 
+            and simulation tools might need. <em>Rumen</em> generates 
+            <a href="http://en.wikipedia.org/wiki/Cumulative_distribution_function">
+              <em>Cumulative Distribution Functions (CDF)</em>
+            </a> for the Map/Reduce task runtimes. 
+            Runtime CDF can be used for extrapolating the task runtime of 
+            incomplete, missing and synthetic tasks. Similarly CDF is also 
+            computed for the total number of successful tasks for every attempt.
+            
+        </li>
+      </ul>
+    </section>
+
+    <!--
+      Basic high level view of components
+    -->
+    <section>  
+      <title>Components</title>
+      
+      <p><em>Rumen</em> consists of 2 components</p>
+      
+      <ul>
+        <li><em>Trace Builder</em> : 
+            Converts <em>JobHistory</em> logs into an easily-parsed format.
+            Currently <code>TraceBuilder</code> outputs the trace in 
+            <a href="http://www.json.org/"><em>JSON</em></a> 
+            format.   
+        </li>
+        <li><em>Folder </em>: 
+            A utility to scale the input trace. A trace obtained from
+            <em>TraceBuilder</em> simply summarizes the jobs in the 
+            input folders and files. The time-span within which all the jobs in 
+            a given trace finish can be considered as the trace runtime. 
+            <em>Folder</em> can be used to scale the runtime of a trace.
+            Decreasing the trace runtime might involve dropping some jobs from 
+            the input trace and scaling down the runtime of remaining jobs. 
+            Increasing the trace runtime might involve adding some dummy jobs to
+            the resulting trace and scaling up the runtime of individual jobs.
+       </li>
+                 
+      </ul>
+      <p></p><p></p><p></p>
+    </section>
+  </section>    
+
+  <!--
+    Usage [How to run Rumen? What are the various configuration parameters?]
+  -->
+  <section id="usage">
+    <title>How to use <em>Rumen</em>?</title>
+    
+    <p>Converting <em>JobHistory</em> logs into a desired job-trace consists of 
+       2 steps</p>
+    <ol>
+      <li>Extracting information into an intermediate format</li>
+      <li>Adjusting the job-trace obtained from the intermediate trace to 
+          have the desired properties.</li>
+    </ol>
+       
+    <note>Extracting information from <em>JobHistory</em> logs is a one time
+          operation. This so called <em>Gold Trace</em> can be reused to
+          generate traces with desired values of properties such as 
+          <code>output-duration</code>, <code>concentration</code> etc.
+    </note>
+       
+    <p><em>Rumen</em> provides 2 basic commands</p>
+     <ul>
+       <li><code>TraceBuilder</code></li>
+       <li><code>Folder</code></li>
+     </ul>
+       
+    <p>Firstly, we need to generate the <em>Gold Trace</em>. Hence the first 
+       step is to run <code>TraceBuilder</code> on a job-history folder. 
+       The output of the <code>TraceBuilder</code> is a job-trace file (and an 
+       optional cluster-topology file). In case we want to scale the output, we 
+       can use the <code>Folder</code> utility to fold the current trace to the 
+       desired length. The remaining part of this section explains these 
+       utilities in detail.
+    </p>
+    
+    <note>Examples in this section assumes that certain libraries are present 
+          in the java CLASSPATH. See <em>Section-3.2</em> for more details.
+    </note>
+    <!--
+     TraceBuilder command
+    -->
+    <section>
+      <title>Trace Builder</title>
+      
+      <p><code>Command:</code></p>
+      <source>java org.apache.hadoop.tools.rumen.TraceBuilder [options] &lt;jobtrace-output&gt; &lt;topology-output&gt; &lt;inputs&gt;</source>
+
+      <p>This command invokes the <code>TraceBuilder</code> utility of
+         <em>Rumen</em>. It converts the JobHistory files into a series of JSON
+         objects and writes them into the <code>&lt;jobtrace-output&gt;</code>
+         file. It also extracts the cluster layout (topology) and writes it in
+         the<code>&lt;topology-output&gt;</code> file.
+         <code>&lt;inputs&gt;</code> represents a space-separated list of
+         JobHistory files and folders.
+      </p>
+         
+         <note>1) Input and output to <code>TraceBuilder</code> is expected to
+               be a fully qualified FileSystem path. So use '<em>file://</em>' 
+               to specify files on the <code>local</code> FileSystem and 
+               '<em>hdfs://</em>' to specify files on HDFS. Since input files or
+               folder are FileSystem paths, it means that they can be globbed.
+               This can be useful while specifying multiple file paths using
+               regular expressions.
+         </note>
+         <note>
+               2) By default, TraceBuilder does not recursively scan the input
+               folder for job history files. Only the files that are directly
+               placed under the input folder will be considered for generating
+               the trace. To add all the files under the input directory by
+               recursively scanning the input directory, use ‘-recursive’
+               option.
+         </note>
+      
+      <p>Cluster topology is used as follows :</p>
+      <ul>
+        <li>To reconstruct the splits and make sure that the 
+            distances/latencies seen in the actual run are modeled correctly.
+        </li>
+        <li>To extrapolate splits information for tasks with missing splits
+            details or synthetically generated tasks.
+        </li>
+      </ul>
+      
+      <p><code>Options :</code></p>
+      <table>
+        <tr>
+          <th> Parameter</th>
+          <th> Description</th>
+          <th> Notes </th>
+        </tr>
+        <tr>
+          <td><code>-demuxer</code></td>
+          <td>Used to read the jobhistory files. The default is 
+              <code>DefaultInputDemuxer</code>.</td>
+          <td>Demuxer decides how the input file maps to jobhistory file(s). 
+              Job history logs and job configuration files are typically small 
+              files, and can be more effectively stored when embedded in some
+              container file format like SequenceFile or TFile. To support such 
+              usage cases, one can specify a customized Demuxer class that can 
+              extract individual job history logs and job configuration files 
+              from the source files.
+          </td>
+        </tr>
+        <tr>
+           <td><code>-recursive</code></td>
+           <td>Recursively traverse input paths for job history logs.</td>
+           <td>This option should be used to inform the TraceBuilder to
+           recursively scan the input paths and process all the files under it.
+           Note that, by default, only the history logs that are directly under
+           the input folder are considered for generating the trace.
+           </td>
+        </tr>
+      </table>
+      
+      <section>
+        <title>Example</title>
+        <source>java org.apache.hadoop.tools.rumen.TraceBuilder file:///home/user/job-trace.json file:///home/user/topology.output file:///home/user/logs/history/done</source>
+        <p></p>
+        <p>This will analyze all the jobs in 
+         <code>/home/user/logs/history/done</code> stored on the 
+         <code>local</code> FileSystem and output the jobtraces in 
+         <code>/home/user/job-trace.json</code> along with topology 
+         information in <code>/home/user/topology.output</code>.
+        </p>
+      </section>
+      <p></p><p></p><p></p><p></p><p></p><p></p>
+    </section>
+
+  <!--
+   Folder command
+  -->
+  <section>
+      <title>Folder</title>
+      
+      <p><code>Command</code>:</p>
+      <source>java org.apache.hadoop.tools.rumen.Folder [options] [input] [output]</source>
+      
+      <note>Input and output to <code>Folder</code> is expected to be a fully 
+            qualified FileSystem path. So use '<em>file://</em>' to specify 
+            files on the <code>local</code> FileSystem and '<em>hdfs://</em>' to
+            specify files on HDFS.
+         </note>
+      
+      <p>This command invokes the <code>Folder</code> utility of 
+         <em>Rumen</em>. Folding essentially means that the output duration of 
+         the resulting trace is fixed and job timelines are adjusted 
+         to respect the final output duration. 
+      </p>
+      
+      <p></p>
+      <p><code>Options :</code></p>
+      <table>
+        <tr>
+          <th> Parameter</th>
+          <th> Description</th>
+          <th> Notes </th>
+        </tr>
+        <tr>
+          <td><code>-input-cycle</code></td>
+          <td>Defines the basic unit of time for the folding operation. There is
+              no default value for <code>input-cycle</code>. 
+              <strong>Input cycle must be provided</strong>.
+          </td>
+          <td>'<code>-input-cycle 10m</code>' 
+              implies that the whole trace run will be now sliced at a 10min 
+              interval. Basic operations will be done on the 10m chunks. Note 
+              that <em>Rumen</em> understands various time units like 
+              <em>m(min), h(hour), d(days) etc</em>.
+          </td>
+        </tr>
+        <tr>
+          <td><code>-output-duration</code></td>
+          <td>This parameter defines the final runtime of the trace. 
+              Default value if <strong>1 hour</strong>.
+          </td>
+          <td>'<code>-output-duration 30m</code>' 
+              implies that the resulting trace will have a max runtime of 
+              30mins. All the jobs in the input trace file will be folded and 
+              scaled to fit this window.
+          </td>
+        </tr>
+        <tr>
+          <td><code>-concentration</code></td>
+          <td>Set the concentration of the resulting trace. Default value is 
+              <strong>1</strong>.
+          </td>
+          <td>If the total runtime of the resulting trace is less than the total
+              runtime of the input trace, then the resulting trace would contain
+              lesser number of jobs as compared to the input trace. This 
+              essentially means that the output is diluted. To increase the 
+              density of jobs, set the concentration to a higher value.</td>
+        </tr>
+        <tr>
+          <td><code>-debug</code></td>
+          <td>Run the Folder in debug mode. By default it is set to 
+              <strong>false</strong>.</td>
+          <td>In debug mode, the Folder will print additional statements for 
+              debugging. Also the intermediate files generated in the scratch 
+              directory will not be cleaned up.
+          </td>
+        </tr>
+        <tr>
+          <td><code>-seed</code></td>
+          <td>Initial seed to the Random Number Generator. By default, a Random 
+              Number Generator is used to generate a seed and the seed value is
+              reported back to the user for future use.
+          </td>
+          <td>If an initial seed is passed, then the <code>Random Number 
+              Generator</code> will generate the random numbers in the same 
+              sequence i.e the sequence of random numbers remains same if the 
+              same seed is used. Folder uses Random Number Generator to decide 
+              whether or not to emit the job. 
+          </td>
+        </tr>
+        <tr>
+          <td><code>-temp-directory</code></td>
+          <td>Temporary directory for the Folder. By default the <strong>output
+              folder's parent directory</strong> is used as the scratch space.
+          </td>
+          <td>This is the scratch space used by Folder.  All the 
+              temporary files are cleaned up in the end unless the Folder is run
+              in <code>debug</code> mode.</td>
+        </tr>
+        <tr>
+          <td><code>-skew-buffer-length</code></td>
+          <td>Enables <em>Folder</em> to tolerate skewed jobs.
+              The default buffer length is <strong>0</strong>.</td>
+          <td>'<code>-skew-buffer-length 100</code>' 
+              indicates that if the jobs appear out of order within a window 
+              size of 100, then they will be emitted in-order by the folder. 
+              If a job appears out-of-order outside this window, then the Folder
+              will bail out provided <code>-allow-missorting</code> is not set.
+              <em>Folder</em> reports the maximum skew size seen in the 
+              input trace for future use.
+          </td>
+        </tr>
+        <tr>
+          <td><code>-allow-missorting</code></td>
+          <td>Enables <em>Folder</em> to tolerate out-of-order jobs. By default 
+              mis-sorting is not allowed.
+          </td>
+          <td>If mis-sorting is allowed, then the <em>Folder</em> will ignore 
+              out-of-order jobs that cannot be deskewed using a skew buffer of
+              size specified using <code>-skew-buffer-length</code>. If 
+              mis-sorting is not allowed, then the Folder will bail out if the
+              skew buffer is incapable of tolerating the skew.
+          </td>
+        </tr>
+      </table>
+      
+      <section>
+      <title>Examples</title>
+      <section>
+        <title>Folding an input trace with 10 hours of total runtime to 
+               generate an output trace with 1 hour of total runtime</title>
+        <source>java org.apache.hadoop.tools.rumen.Folder -output-duration 1h  -input-cycle 20m  file:///home/user/job-trace.json file:///home/user/job-trace-1hr.json</source>
+        <p></p>
+        <p>If the folded jobs are out of order then the command
+          will bail out. 
+        </p>
+        <p>
+        
+        </p>
+      </section>
+      
+      <section>
+        <title>Folding an input trace with 10 hours of total runtime to 
+               generate an output trace with 1 hour of total runtime and 
+               tolerate some skewness
+        </title>
+        <source>java org.apache.hadoop.tools.rumen.Folder -output-duration 1h -input-cycle 20m  -allow-missorting -skew-buffer-length 100 file:///home/user/job-trace.json file:///home/user/job-trace-1hr.json</source>
+        <p></p>
+        <p>If the folded jobs are out of order, then atmost
+          100 jobs will be de-skewed. If the 101<sup>st</sup> job is 
+          <em>out-of-order</em>, then the command will bail out.
+        </p>
+      </section>
+      <section>
+        <title>Folding an input trace with 10 hours of total runtime to 
+               generate an output trace with 1 hour of total runtime in debug 
+               mode
+        </title>
+        <source>java org.apache.hadoop.tools.rumen.Folder -output-duration 1h -input-cycle 20m  -debug -temp-directory file:///tmp/debug file:///home/user/job-trace.json file:///home/user/job-trace-1hr.json</source>
+        <p></p>
+        <p>This will fold the 10hr job-trace file 
+           <code>file:///home/user/job-trace.json</code> to finish within 1hr 
+           and use <code>file:///tmp/debug</code> as the temporary directory. 
+           The intermediate files in the temporary directory will not be cleaned
+           up.
+        </p>
+      </section>
+      
+      <section>
+        <title>Folding an input trace with 10 hours of total runtime to 
+               generate an output trace with 1 hour of total runtime with custom
+               concentration.
+        </title>
+        <source>java org.apache.hadoop.tools.rumen.Folder -output-duration 1h -input-cycle 20m  -concentration 2  file:///home/user/job-trace.json file:///home/user/job-trace-1hr.json</source>
+        <p></p>
+        <p>This will fold the 10hr job-trace file 
+           <code>file:///home/user/job-trace.json</code> to finish within 1hr 
+           with concentration of 2. <code>Example-2.3.2</code> will retain 10% 
+           of the jobs. With <em>concentration</em> as 2, 20% of the total input 
+           jobs will be retained.
+        </p>
+      </section>
+    </section>
+    </section>
+    <p></p><p></p><p></p>
+  </section>
+  
+  <!--
+    Appendix [Resources i.e ppts, jiras, definition etc]
+  -->
+  <section>
+    <title>Appendix</title>
+    
+    <section>
+      <title>Resources</title>
+      <p><a href="https://issues.apache.org/jira/browse/MAPREDUCE-751">MAPREDUCE-751</a> is the main JIRA that introduced <em>Rumen</em> to <em>MapReduce</em>. 
+         Look at the MapReduce <a href="https://issues.apache.org/jira/browse/MAPREDUCE/component/12313617">rumen-component</a> for further details.</p>
+    </section>
+    
+    <section>
+     <title>Dependencies</title>
+    <p><em>Rumen</em> expects certain library <em>JARs</em> to be present in 
+         the <em>CLASSPATH</em>. 
+              The required libraries are </p>
+      <ul>
+        <li><code>Hadoop MapReduce Tools</code> (<code>hadoop-mapred-tools-{hadoop-version}.jar</code>)</li>
+        <li><code>Hadoop Common</code> (<code>hadoop-common-{hadoop-version}.jar</code>)</li>
+        <li><code>Apache Commons Logging</code> (<code>commons-logging-1.1.1.jar</code>)</li>
+        <li><code>Apache Commons CLI</code> (<code>commons-cli-1.2.jar</code>)</li>
+        <li><code>Jackson Mapper</code> (<code>jackson-mapper-asl-1.4.2.jar</code>)</li>
+        <li><code>Jackson Core</code> (<code>jackson-core-asl-1.4.2.jar</code>)</li>
+      </ul>
+      
+      <note>One simple way to run Rumen is to use '$HADOOP_HOME/bin/hadoop jar' 
+              option  to run it.
+      </note>
+    </section>
+  </section>
+</body>
+</document>

+ 1 - 0
src/docs/src/documentation/content/xdocs/site.xml

@@ -49,6 +49,7 @@ See http://forrest.apache.org/docs/linking.html for more info.
     <vaidya         label="Vaidya"  href="vaidya.html"/>
     <vaidya         label="Vaidya"  href="vaidya.html"/>
     <archives     label="Hadoop Archives" href="hadoop_archives.html"/>
     <archives     label="Hadoop Archives" href="hadoop_archives.html"/>
     <gridmix       label="Gridmix"  href="gridmix.html"/>
     <gridmix       label="Gridmix"  href="gridmix.html"/>
+    <Rumen          label="Rumen"     href="rumen.html"/>
     <cap_scheduler  label="Capacity Scheduler" href="capacity_scheduler.html"/>
     <cap_scheduler  label="Capacity Scheduler" href="capacity_scheduler.html"/>
     <fair_scheduler    label="Fair Scheduler"  href="fair_scheduler.html"/>
     <fair_scheduler    label="Fair Scheduler"  href="fair_scheduler.html"/>
     <cap_scheduler  label="Hod Scheduler"  href="hod_scheduler.html"/>
     <cap_scheduler  label="Hod Scheduler"  href="hod_scheduler.html"/>

+ 2 - 2
src/mapred/org/apache/hadoop/mapred/JobConf.java

@@ -156,10 +156,10 @@ public class JobConf extends Configuration {
    */
    */
   public static final String DEFAULT_QUEUE_NAME = "default";
   public static final String DEFAULT_QUEUE_NAME = "default";
   
   
-  static final String MAPRED_JOB_MAP_MEMORY_MB_PROPERTY =
+  public static final String MAPRED_JOB_MAP_MEMORY_MB_PROPERTY =
       "mapred.job.map.memory.mb";
       "mapred.job.map.memory.mb";
 
 
-  static final String MAPRED_JOB_REDUCE_MEMORY_MB_PROPERTY =
+  public static final String MAPRED_JOB_REDUCE_MEMORY_MB_PROPERTY =
       "mapred.job.reduce.memory.mb";
       "mapred.job.reduce.memory.mb";
 
 
   static final String MR_ACLS_ENABLED = "mapred.acls.enabled";
   static final String MR_ACLS_ENABLED = "mapred.acls.enabled";

+ 2 - 2
src/mapred/org/apache/hadoop/mapred/JobTracker.java

@@ -4480,9 +4480,9 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
   public static final String MAPRED_CLUSTER_REDUCE_MEMORY_MB_PROPERTY =
   public static final String MAPRED_CLUSTER_REDUCE_MEMORY_MB_PROPERTY =
       "mapred.cluster.reduce.memory.mb";
       "mapred.cluster.reduce.memory.mb";
 
 
-  static final String MAPRED_CLUSTER_MAX_MAP_MEMORY_MB_PROPERTY =
+  public static final String MAPRED_CLUSTER_MAX_MAP_MEMORY_MB_PROPERTY =
       "mapred.cluster.max.map.memory.mb";
       "mapred.cluster.max.map.memory.mb";
-  static final String MAPRED_CLUSTER_MAX_REDUCE_MEMORY_MB_PROPERTY =
+  public static final String MAPRED_CLUSTER_MAX_REDUCE_MEMORY_MB_PROPERTY =
       "mapred.cluster.max.reduce.memory.mb";
       "mapred.cluster.max.reduce.memory.mb";
 
 
   /* 
   /* 

+ 10 - 0
src/mapred/org/apache/hadoop/mapred/Reporter.java

@@ -61,6 +61,10 @@ public interface Reporter extends Progressable {
       public InputSplit getInputSplit() throws UnsupportedOperationException {
       public InputSplit getInputSplit() throws UnsupportedOperationException {
         throw new UnsupportedOperationException("NULL reporter has no input");
         throw new UnsupportedOperationException("NULL reporter has no input");
       }
       }
+      @Override
+      public float getProgress() {
+        return 0;
+      }
     };
     };
 
 
   /**
   /**
@@ -117,4 +121,10 @@ public interface Reporter extends Progressable {
    */
    */
   public abstract InputSplit getInputSplit() 
   public abstract InputSplit getInputSplit() 
     throws UnsupportedOperationException;
     throws UnsupportedOperationException;
+  
+  /**
+   * Get the progress of the task. Progress is represented as a number between
+   * 0 and 1 (inclusive).
+   */
+  public float getProgress();
 }
 }

+ 5 - 0
src/mapred/org/apache/hadoop/mapred/Task.java

@@ -586,6 +586,11 @@ abstract public class Task implements Writable, Configurable {
       // indicate that progress update needs to be sent
       // indicate that progress update needs to be sent
       setProgressFlag();
       setProgressFlag();
     }
     }
+    
+    public float getProgress() {
+      return taskProgress.getProgress();
+    };
+    
     public void progress() {
     public void progress() {
       // indicate that progress update needs to be sent
       // indicate that progress update needs to be sent
       setProgressFlag();
       setProgressFlag();

+ 6 - 0
src/mapred/org/apache/hadoop/mapreduce/StatusReporter.java

@@ -21,5 +21,11 @@ public abstract class StatusReporter {
   public abstract Counter getCounter(Enum<?> name);
   public abstract Counter getCounter(Enum<?> name);
   public abstract Counter getCounter(String group, String name);
   public abstract Counter getCounter(String group, String name);
   public abstract void progress();
   public abstract void progress();
+  /**
+   * Get the current progress.
+   * @return a number between 0.0 and 1.0 (inclusive) indicating the attempt's 
+   * progress.
+   */
+  public abstract float getProgress();
   public abstract void setStatus(String status);
   public abstract void setStatus(String status);
 }
 }

+ 4 - 0
src/mapred/org/apache/hadoop/mapreduce/TaskInputOutputContext.java

@@ -93,6 +93,10 @@ public abstract class TaskInputOutputContext<KEYIN,VALUEIN,KEYOUT,VALUEOUT>
     reporter.progress();
     reporter.progress();
   }
   }
 
 
+  public float getProgress() {
+    return reporter.getProgress();
+  }
+
   @Override
   @Override
   public void setStatus(String status) {
   public void setStatus(String status) {
     reporter.setStatus(status);
     reporter.setStatus(status);

+ 4 - 0
src/mapred/org/apache/hadoop/mapreduce/lib/map/MultithreadedMapper.java

@@ -235,6 +235,10 @@ public class MultithreadedMapper<K1, V1, K2, V2>
       outer.setStatus(status);
       outer.setStatus(status);
     }
     }
     
     
+    @Override
+    public float getProgress() {
+      return outer.getProgress();
+    }
   }
   }
 
 
   private class MapRunner extends Thread {
   private class MapRunner extends Thread {

+ 11 - 2
src/test/org/apache/hadoop/mapred/UtilsForTests.java

@@ -608,6 +608,16 @@ public class UtilsForTests {
   public static RunningJob runJob(JobConf conf, Path inDir, Path outDir, 
   public static RunningJob runJob(JobConf conf, Path inDir, Path outDir, 
                                   int numMaps, int numReds) throws IOException {
                                   int numMaps, int numReds) throws IOException {
 
 
+    String input = "The quick brown fox\n" + "has many silly\n"
+                   + "red fox sox\n";
+    
+    // submit the job and wait for it to complete
+    return runJob(conf, inDir, outDir, numMaps, numReds, input);
+  }
+  
+  // Start a job with the specified input and return its RunningJob object
+  static RunningJob runJob(JobConf conf, Path inDir, Path outDir, int numMaps, 
+                           int numReds, String input) throws IOException {
     FileSystem fs = FileSystem.get(conf);
     FileSystem fs = FileSystem.get(conf);
     if (fs.exists(outDir)) {
     if (fs.exists(outDir)) {
       fs.delete(outDir, true);
       fs.delete(outDir, true);
@@ -615,8 +625,7 @@ public class UtilsForTests {
     if (!fs.exists(inDir)) {
     if (!fs.exists(inDir)) {
       fs.mkdirs(inDir);
       fs.mkdirs(inDir);
     }
     }
-    String input = "The quick brown fox\n" + "has many silly\n"
-        + "red fox sox\n";
+    
     for (int i = 0; i < numMaps; ++i) {
     for (int i = 0; i < numMaps; ++i) {
       DataOutputStream file = fs.create(new Path(inDir, "part-" + i));
       DataOutputStream file = fs.create(new Path(inDir, "part-" + i));
       file.writeBytes(input);
       file.writeBytes(input);

+ 4 - 0
src/test/org/apache/hadoop/mapreduce/MapReduceTestUtil.java

@@ -285,6 +285,10 @@ public class MapReduceTestUtil {
       }
       }
       public void progress() {
       public void progress() {
       }
       }
+      @Override
+      public float getProgress() {
+        return 0;
+      }
       public Counter getCounter(Enum<?> name) {
       public Counter getCounter(Enum<?> name) {
         return new Counters().findCounter(name);
         return new Counters().findCounter(name);
       }
       }

+ 404 - 52
src/test/org/apache/hadoop/tools/rumen/TestRumenJobTraces.java

@@ -19,6 +19,7 @@
 package org.apache.hadoop.tools.rumen;
 package org.apache.hadoop.tools.rumen;
 
 
 import java.io.BufferedInputStream;
 import java.io.BufferedInputStream;
+import java.io.DataOutputStream;
 import java.io.IOException;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.io.OutputStream;
@@ -46,6 +47,7 @@ import org.apache.hadoop.mapreduce.JobID;
 import org.apache.hadoop.mapreduce.TaskAttemptID;
 import org.apache.hadoop.mapreduce.TaskAttemptID;
 import org.apache.hadoop.mapreduce.TaskID;
 import org.apache.hadoop.mapreduce.TaskID;
 import org.apache.hadoop.mapreduce.TaskType;
 import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.hadoop.tools.rumen.TraceBuilder.MyOptions;
 import org.apache.hadoop.util.LineReader;
 import org.apache.hadoop.util.LineReader;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.hadoop.util.ToolRunner;
@@ -363,6 +365,160 @@ public class TestRumenJobTraces {
   }
   }
 
 
   /**
   /**
+   * Check if processing of input arguments is as expected by passing globbed
+   * input path
+   * <li> without -recursive option and
+   * <li> with -recursive option.
+   */
+  @Test
+  public void testProcessInputArgument() throws Exception {
+    final Configuration conf = new Configuration();
+    final FileSystem lfs = FileSystem.getLocal(conf);
+
+    // define the test's root temporary directory
+    final Path rootTempDir =
+      new Path(System.getProperty("test.build.data", "/tmp"))
+          .makeQualified(lfs.getUri(), lfs.getWorkingDirectory());
+    // define the test's root input directory
+    Path testRootInputDir = new Path(rootTempDir, "TestProcessInputArgument");
+    // define the nested input directory
+    Path nestedInputDir = new Path(testRootInputDir, "1/2/3/4");
+    // define the globbed version of the nested input directory
+    Path globbedInputNestedDir =
+      lfs.makeQualified(new Path(testRootInputDir, "*/*/*/*/*"));
+    try {
+      lfs.delete(nestedInputDir, true);
+
+      List<String> recursiveInputPaths = new ArrayList<String>();
+      List<String> nonRecursiveInputPaths = new ArrayList<String>();
+      // Create input files under the given path with multiple levels of
+      // sub directories
+      createHistoryLogsHierarchy(nestedInputDir, lfs, recursiveInputPaths,
+          nonRecursiveInputPaths);
+
+      // Check the case of globbed input path and without -recursive option
+      List<Path> inputs = MyOptions.processInputArgument(
+                              globbedInputNestedDir.toString(), conf, false);
+      validateHistoryLogPaths(inputs, nonRecursiveInputPaths);
+   // Check the case of globbed input path and with -recursive option
+      inputs = MyOptions.processInputArgument(
+                   globbedInputNestedDir.toString(), conf, true);
+      validateHistoryLogPaths(inputs, recursiveInputPaths);
+
+    } finally {
+      lfs.delete(testRootInputDir, true);
+    }
+  }
+
+  /**
+   * Validate if the input history log paths are as expected.
+   * @param inputs  the resultant input paths to be validated
+   * @param expectedHistoryFileNames  the expected input history logs
+   * @throws IOException
+   */
+  private void validateHistoryLogPaths(List<Path> inputs,
+      List<String> expectedHistoryFileNames) throws IOException {
+
+    System.out.println("\nExpected history files are:");
+    for (String historyFile : expectedHistoryFileNames) {
+      System.out.println(historyFile);
+    }
+    System.out.println("\nResultant history files are:");
+    List<String> historyLogs = new ArrayList<String>();
+    for (Path p : inputs) {
+      historyLogs.add(p.toUri().getPath());
+      System.out.println(p.toUri().getPath());
+    }
+
+    assertEquals("Number of history logs found is different from the expected.",
+        expectedHistoryFileNames.size(), inputs.size());
+
+    // Verify if all the history logs are expected ones and they are in the
+    // expected order
+    assertTrue("Some of the history log files do not match the expected.",
+        historyLogs.equals(expectedHistoryFileNames));
+  }
+
+  /**
+   * Create history logs under the given path with multiple levels of
+   * sub directories as shown below.
+   * <br>
+   * Create a file, an empty subdirectory and a nonempty subdirectory
+   * &lt;historyDir&gt; under the given input path.
+   * <br>
+   * The subdirectory &lt;historyDir&gt; contains the following dir structure:
+   * <br>
+   * <br>&lt;historyDir&gt;/historyFile1.txt
+   * <br>&lt;historyDir&gt;/historyFile1.gz
+   * <br>&lt;historyDir&gt;/subDir1/historyFile2.txt
+   * <br>&lt;historyDir&gt;/subDir1/historyFile2.gz
+   * <br>&lt;historyDir&gt;/subDir2/historyFile3.txt
+   * <br>&lt;historyDir&gt;/subDir2/historyFile3.gz
+   * <br>&lt;historyDir&gt;/subDir1/subDir11/historyFile4.txt
+   * <br>&lt;historyDir&gt;/subDir1/subDir11/historyFile4.gz
+   * <br>&lt;historyDir&gt;/subDir2/subDir21/
+   * <br>
+   * Create the lists of input paths that should be processed by TraceBuilder
+   * for recursive case and non-recursive case.
+   * @param nestedInputDir the input history logs directory where history files
+   *                       with nested subdirectories are created
+   * @param fs         FileSystem of the input paths
+   * @param recursiveInputPaths input paths for recursive case
+   * @param nonRecursiveInputPaths input paths for non-recursive case
+   * @throws IOException
+   */
+  private void createHistoryLogsHierarchy(Path nestedInputDir, FileSystem fs,
+      List<String> recursiveInputPaths, List<String> nonRecursiveInputPaths)
+  throws IOException {
+    List<Path> dirs = new ArrayList<Path>();
+    // define a file in the nested test input directory
+    Path inputPath1 = new Path(nestedInputDir, "historyFile.txt");
+    // define an empty sub-folder in the nested test input directory
+    Path emptyDir = new Path(nestedInputDir, "emptyDir");
+    // define a nonempty sub-folder in the nested test input directory
+    Path historyDir = new Path(nestedInputDir, "historyDir");
+
+    fs.mkdirs(nestedInputDir);
+    // Create an empty input file
+    fs.createNewFile(inputPath1);
+    // Create empty subdir
+    fs.mkdirs(emptyDir);// let us not create any files under this dir
+
+    fs.mkdirs(historyDir);
+    dirs.add(historyDir);
+
+    Path subDir1 = new Path(historyDir, "subDir1");
+    fs.mkdirs(subDir1);
+    dirs.add(subDir1);
+    Path subDir2 = new Path(historyDir, "subDir2");
+    fs.mkdirs(subDir2);
+    dirs.add(subDir2);
+
+    Path subDir11 = new Path(subDir1, "subDir11");
+    fs.mkdirs(subDir11);
+    dirs.add(subDir11);
+    Path subDir21 = new Path(subDir2, "subDir21");
+    fs.mkdirs(subDir21);// let us not create any files under this dir
+
+    int i = 0;
+    for (Path dir : dirs) {
+      i++;
+      Path gzPath = new Path(dir, "historyFile" + i + ".gz");
+      Path txtPath = new Path(dir, "historyFile" + i + ".txt");
+      fs.createNewFile(txtPath);
+      fs.createNewFile(gzPath);
+      recursiveInputPaths.add(gzPath.toUri().getPath());
+      recursiveInputPaths.add(txtPath.toUri().getPath());
+      if (i == 1) {
+        nonRecursiveInputPaths.add(gzPath.toUri().getPath());
+        nonRecursiveInputPaths.add(txtPath.toUri().getPath());
+      }
+    }
+    recursiveInputPaths.add(inputPath1.toUri().getPath());
+    nonRecursiveInputPaths.add(inputPath1.toUri().getPath());
+  }
+
+    /**
    * Test if {@link CurrentJHParser} can read events from current JH files.
    * Test if {@link CurrentJHParser} can read events from current JH files.
    */
    */
   @Test
   @Test
@@ -426,7 +582,7 @@ public class TestRumenJobTraces {
 
 
       // Test if the JobHistoryParserFactory can detect the parser correctly
       // Test if the JobHistoryParserFactory can detect the parser correctly
       parser = JobHistoryParserFactory.getParser(ris);
       parser = JobHistoryParserFactory.getParser(ris);
-        
+
       HistoryEvent e;
       HistoryEvent e;
       while ((e = parser.nextEvent()) != null) {
       while ((e = parser.nextEvent()) != null) {
         String eventString = e.getEventType().toString();
         String eventString = e.getEventType().toString();
@@ -470,71 +626,267 @@ public class TestRumenJobTraces {
     }
     }
   }
   }
   
   
-  @Test
-  public void testJobConfigurationParser() throws Exception {
-    String[] list1 =
-        { "mapred.job.queue.name", "mapreduce.job.name",
-            "mapred.child.java.opts" };
+    /**
+     * Test if the {@link JobConfigurationParser} can correctly extract out 
+     * key-value pairs from the job configuration.
+     */
+    @Test
+    public void testJobConfigurationParsing() throws Exception {
+      final FileSystem lfs = FileSystem.getLocal(new Configuration());
+  
+      final Path rootTempDir =
+          new Path(System.getProperty("test.build.data", "/tmp")).makeQualified(
+              lfs.getUri(), lfs.getWorkingDirectory());
+  
+      final Path tempDir = new Path(rootTempDir, "TestJobConfigurationParser");
+      lfs.delete(tempDir, true);
+  
+      // Add some configuration parameters to the conf
+      JobConf jConf = new JobConf(false);
+      String key = "test.data";
+      String value = "hello world";
+      jConf.set(key, value);
+      
+      // create the job conf file
+      Path jobConfPath = new Path(tempDir.toString(), "job.xml");
+      lfs.delete(jobConfPath, false);
+      DataOutputStream jobConfStream = lfs.create(jobConfPath);
+      jConf.writeXml(jobConfStream);
+      jobConfStream.close();
+      
+      // now read the job conf file using the job configuration parser
+      Properties properties = 
+        JobConfigurationParser.parse(lfs.open(jobConfPath));
+      
+      // check if the required parameter is loaded
+      assertEquals("Total number of extracted properties (" + properties.size() 
+                   + ") doesn't match the expected size of 1 ["
+                   + "JobConfigurationParser]",
+                   1, properties.size());
+      // check if the key is present in the extracted configuration
+      assertTrue("Key " + key + " is missing in the configuration extracted "
+                 + "[JobConfigurationParser]",
+                 properties.keySet().contains(key));
+      // check if the desired property has the correct value
+      assertEquals("JobConfigurationParser couldn't recover the parameters"
+                   + " correctly",
+                  value, properties.get(key));
+      
+      // Test ZombieJob
+      LoggedJob job = new LoggedJob();
+      job.setJobProperties(properties);
+      
+      ZombieJob zjob = new ZombieJob(job, null);
+      Configuration zconf = zjob.getJobConf();
+      // check if the required parameter is loaded
+      assertEquals("ZombieJob couldn't recover the parameters correctly", 
+                   value, zconf.get(key));
+    }
 
 
-    String[] list2 = { "mapred.job.queue.name", "mapred.child.java.opts" };
+    @Test
+    public void testJobConfigurationParser() throws Exception {
 
 
-    List<String> interested1 = new ArrayList<String>();
-    for (String interested : list1) {
-      interested1.add(interested);
+      // Validate parser with old mapred config properties from
+      // sample-conf-file.xml
+      validateJobConfParser("sample-conf.file.xml");
     }
     }
 
 
-    List<String> interested2 = new ArrayList<String>();
-    for (String interested : list2) {
-      interested2.add(interested);
-    }
+    private void validateJobConfParser(String confFile) throws Exception {
 
 
-    JobConfigurationParser jcp1 = new JobConfigurationParser(interested1);
-    JobConfigurationParser jcp2 = new JobConfigurationParser(interested2);
+      final Configuration conf = new Configuration();
+      final FileSystem lfs = FileSystem.getLocal(conf);
 
 
-    final Configuration conf = new Configuration();
-    final FileSystem lfs = FileSystem.getLocal(conf);
+      @SuppressWarnings("deprecation")
+      final Path rootInputDir =
+          new Path(System.getProperty("test.tools.input.dir", ""))
+              .makeQualified(lfs);
 
 
-    @SuppressWarnings("deprecation")
-    final Path rootInputDir =
-        new Path(System.getProperty("test.tools.input.dir", ""))
-            .makeQualified(lfs);
+      final Path rootInputPath = new Path(rootInputDir, "rumen/small-trace-test");
 
 
-    final Path rootInputPath = new Path(rootInputDir, "rumen/small-trace-test");
+      final Path inputPath = new Path(rootInputPath, confFile);
 
 
-    final Path inputPath = new Path(rootInputPath, "sample-conf.file.xml");
+      InputStream inputConfStream =
+          new PossiblyDecompressedInputStream(inputPath, conf);
 
 
-    InputStream inputConfStream =
-        new PossiblyDecompressedInputStream(inputPath, conf);
+      try {
+        Properties props = JobConfigurationParser.parse(inputConfStream);
+        inputConfStream.close();
+
+        // Make sure that parser puts the interested properties into props1 and
+        // props2 as defined by list1 and list2.
+        assertEquals("Config property for job queue name is not "
+            + " extracted properly.", "TheQueue",
+            JobBuilder.extract(props, JobConfPropertyNames.QUEUE_NAMES
+            .getCandidates(), null));
+        assertEquals("Config property for job name is not "
+            + " extracted properly.", "MyMRJob",
+            JobBuilder.extract(props, JobConfPropertyNames.JOB_NAMES
+            .getCandidates(), null));
+
+        validateChildJavaOpts(props);
 
 
-    try {
-      Properties props1 = jcp1.parse(inputConfStream);
-      inputConfStream.close();
-
-      inputConfStream = new PossiblyDecompressedInputStream(inputPath, conf);
-      Properties props2 = jcp2.parse(inputConfStream);
-
-      assertEquals("testJobConfigurationParser: wrong number of properties", 3,
-          props1.size());
-      assertEquals("testJobConfigurationParser: wrong number of properties", 2,
-          props2.size());
-
-      assertEquals("prop test 1", "TheQueue", props1
-          .get("mapred.job.queue.name"));
-      assertEquals("prop test 2", "job_0001", props1.get("mapreduce.job.name"));
-      assertEquals("prop test 3",
-          "-server -Xmx640m -Djava.net.preferIPv4Stack=true", props1
-              .get("mapred.child.java.opts"));
-      assertEquals("prop test 4", "TheQueue", props2
-          .get("mapred.job.queue.name"));
-      assertEquals("prop test 5",
-          "-server -Xmx640m -Djava.net.preferIPv4Stack=true", props2
-              .get("mapred.child.java.opts"));
+      } finally {
+        inputConfStream.close();
+      }
+    }
+    
+    // Validate child java opts in properties.
+    private void validateChildJavaOpts(Properties props) {
+      // if old property mapred.child.java.opts is set, then extraction of all
+      // the following 3 properties should give that value.
+      assertEquals("mapred.child.java.opts is not extracted properly.",
+          "-server -Xmx640m -Djava.net.preferIPv4Stack=true",
+          JobBuilder.extract(props, JobConfPropertyNames.TASK_JAVA_OPTS_S
+          .getCandidates(), null));
+      assertEquals("New config property " + JobConf.MAPRED_MAP_TASK_JAVA_OPTS
+          + " is not extracted properly when the old config property "
+          + "mapred.child.java.opts is set.",
+          "-server -Xmx640m -Djava.net.preferIPv4Stack=true",
+          JobBuilder.extract(props, JobConfPropertyNames.MAP_JAVA_OPTS_S
+          .getCandidates(), null));
+      assertEquals("New config property " + JobConf.MAPRED_REDUCE_TASK_JAVA_OPTS
+          + " is not extracted properly when the old config property "
+          + "mapred.child.java.opts is set.",
+          "-server -Xmx640m -Djava.net.preferIPv4Stack=true",
+          JobBuilder.extract(props, JobConfPropertyNames.REDUCE_JAVA_OPTS_S
+          .getCandidates(), null));
+    }
 
 
-    } finally {
-      inputConfStream.close();
+  /**
+   * Test {@link ResourceUsageMetrics}.
+   */
+  @Test
+  public void testResourceUsageMetrics() throws Exception {
+    final long cpuUsage = 100;
+    final long pMemUsage = 200;
+    final long vMemUsage = 300;
+    final long heapUsage = 400;
+    
+    // test ResourceUsageMetrics's setters
+    ResourceUsageMetrics metrics = new ResourceUsageMetrics();
+    metrics.setCumulativeCpuUsage(cpuUsage);
+    metrics.setPhysicalMemoryUsage(pMemUsage);
+    metrics.setVirtualMemoryUsage(vMemUsage);
+    metrics.setHeapUsage(heapUsage);
+    // test cpu usage value
+    assertEquals("Cpu usage values mismatch via set", cpuUsage, 
+                 metrics.getCumulativeCpuUsage());
+    // test pMem usage value
+    assertEquals("Physical memory usage values mismatch via set", pMemUsage, 
+                 metrics.getPhysicalMemoryUsage());
+    // test vMem usage value
+    assertEquals("Virtual memory usage values mismatch via set", vMemUsage, 
+                 metrics.getVirtualMemoryUsage());
+    // test heap usage value
+    assertEquals("Heap usage values mismatch via set", heapUsage, 
+                 metrics.getHeapUsage());
+    
+    // test deepCompare() (pass case)
+    testResourceUsageMetricViaDeepCompare(metrics, cpuUsage, vMemUsage, 
+                                          pMemUsage, heapUsage, true);
+    
+    // test deepCompare (fail case)
+    // test cpu usage mismatch
+    testResourceUsageMetricViaDeepCompare(metrics, 0, vMemUsage, pMemUsage, 
+                                          heapUsage, false);
+    // test pMem usage mismatch
+    testResourceUsageMetricViaDeepCompare(metrics, cpuUsage, vMemUsage, 0, 
+                                          heapUsage, false);
+    // test vMem usage mismatch
+    testResourceUsageMetricViaDeepCompare(metrics, cpuUsage, 0, pMemUsage, 
+                                          heapUsage, false);
+    // test heap usage mismatch
+    testResourceUsageMetricViaDeepCompare(metrics, cpuUsage, vMemUsage, 
+                                          pMemUsage, 0, false);
+    
+    // define a metric with a fixed value of size()
+    ResourceUsageMetrics metrics2 = new ResourceUsageMetrics() {
+      @Override
+      public int size() {
+        return -1;
+      }
+    };
+    metrics2.setCumulativeCpuUsage(cpuUsage);
+    metrics2.setPhysicalMemoryUsage(pMemUsage);
+    metrics2.setVirtualMemoryUsage(vMemUsage);
+    metrics2.setHeapUsage(heapUsage);
+    
+    // test with size mismatch
+    testResourceUsageMetricViaDeepCompare(metrics2, cpuUsage, vMemUsage, 
+                                          pMemUsage, heapUsage, false);
+  }
+  
+  // test ResourceUsageMetric's deepCompare() method
+  private static void testResourceUsageMetricViaDeepCompare(
+                        ResourceUsageMetrics metrics, long cpuUsage, 
+                        long vMemUsage, long pMemUsage, long heapUsage,
+                        boolean shouldPass) {
+    ResourceUsageMetrics testMetrics = new ResourceUsageMetrics();
+    testMetrics.setCumulativeCpuUsage(cpuUsage);
+    testMetrics.setPhysicalMemoryUsage(pMemUsage);
+    testMetrics.setVirtualMemoryUsage(vMemUsage);
+    testMetrics.setHeapUsage(heapUsage);
+    
+    Boolean passed = null;
+    try {
+      metrics.deepCompare(testMetrics, new TreePath(null, "<root>"));
+      passed = true;
+    } catch (DeepInequalityException die) {
+      passed = false;
     }
     }
+    
+    assertEquals("ResourceUsageMetrics deepCompare() failed!", 
+                 shouldPass, passed);
   }
   }
-
+  
+  /**
+   * Testing {@link ResourceUsageMetrics} using {@link HadoopLogsAnalyzer}.
+   */
+  @Test
+  @SuppressWarnings("deprecation")
+  public void testResourceUsageMetricsWithHadoopLogsAnalyzer() 
+  throws IOException {
+    Configuration conf = new Configuration();
+    // get the input trace file
+    Path rootInputDir =
+      new Path(System.getProperty("test.tools.input.dir", ""));
+    Path rootInputSubFolder = new Path(rootInputDir, "rumen/small-trace-test");
+    Path traceFile = new Path(rootInputSubFolder, "v20-resource-usage-log.gz");
+    
+    FileSystem lfs = FileSystem.getLocal(conf);
+    
+    // define the root test directory
+    Path rootTempDir =
+        new Path(System.getProperty("test.build.data", "/tmp"));
+
+    // define output directory
+    Path outputDir = 
+      new Path(rootTempDir, "testResourceUsageMetricsWithHadoopLogsAnalyzer");
+    lfs.delete(outputDir, true);
+    lfs.deleteOnExit(outputDir);
+    
+    // run HadoopLogsAnalyzer
+    HadoopLogsAnalyzer analyzer = new HadoopLogsAnalyzer();
+    analyzer.setConf(conf);
+    Path traceOutput = new Path(outputDir, "trace.json");
+    analyzer.run(new String[] {"-write-job-trace", traceOutput.toString(), 
+                               "-v1", traceFile.toString()});
+    
+    // test HadoopLogsAnalyzer's output w.r.t ResourceUsageMetrics
+    //  get the logged job
+    JsonObjectMapperParser<LoggedJob> traceParser =
+      new JsonObjectMapperParser<LoggedJob>(traceOutput, LoggedJob.class, 
+                                            conf);
+    
+    //  get the logged job from the output trace file
+    LoggedJob job = traceParser.getNext();
+    LoggedTaskAttempt attempt = job.getMapTasks().get(0).getAttempts().get(0);
+    ResourceUsageMetrics metrics = attempt.getResourceUsageMetrics();
+    
+    //  test via deepCompare()
+    testResourceUsageMetricViaDeepCompare(metrics, 200, 100, 75, 50, true);
+  }
+  
   @Test
   @Test
   public void testTopologyBuilder() throws Exception {
   public void testTopologyBuilder() throws Exception {
     final TopologyBuilder subject = new TopologyBuilder();
     final TopologyBuilder subject = new TopologyBuilder();

二进制
src/test/tools/data/rumen/small-trace-test/counters-test-trace.json.gz


二进制
src/test/tools/data/rumen/small-trace-test/dispatch-trace-output.json.gz


二进制
src/test/tools/data/rumen/small-trace-test/job-tracker-logs-trace-output.gz


+ 4 - 1
src/test/tools/data/rumen/small-trace-test/sample-conf.file.xml

@@ -19,11 +19,14 @@
  */
  */
 -->
 -->
 <configuration>
 <configuration>
+<!--
+Old mapred config properties
+-->
    <property>
    <property>
       <name>mapred.job.queue.name</name><value>TheQueue</value>
       <name>mapred.job.queue.name</name><value>TheQueue</value>
    </property>
    </property>
    <property>
    <property>
-      <name>mapreduce.job.name</name><value>job_0001</value>
+      <name>mapred.job.name</name><value>MyMRJob</value>
    </property>
    </property>
    <property>
    <property>
       <name>maproduce.uninteresting.property</name><value>abcdef</value>
       <name>maproduce.uninteresting.property</name><value>abcdef</value>

+ 3 - 0
src/test/tools/data/rumen/small-trace-test/truncated-trace-output

@@ -3,6 +3,9 @@
   "user" : "hadoopqa",
   "user" : "hadoopqa",
   "jobName" : null,
   "jobName" : null,
   "jobID" : "job_200904211745_0002",
   "jobID" : "job_200904211745_0002",
+   "jobProperties" : {
+    "mapred.child.java.opts" : "-server -Xmx640m -Djava.net.preferIPv4Stack=true"
+  },
   "mapTasks" : [ {
   "mapTasks" : [ {
     "startTime" : 1240336753705,
     "startTime" : 1240336753705,
     "attempts" : [ {
     "attempts" : [ {

二进制
src/test/tools/data/rumen/small-trace-test/v20-resource-usage-log.gz


+ 2 - 2
src/tools/org/apache/hadoop/tools/rumen/ClusterStory.java

@@ -54,13 +54,13 @@ public interface ClusterStory {
   /**
   /**
    * Get {@link MachineNode} by its host name.
    * Get {@link MachineNode} by its host name.
    * 
    * 
-   * @return The {@line MachineNode} with the same name. Or null if not found.
+   * @return The {@link MachineNode} with the same name. Or null if not found.
    */
    */
   public MachineNode getMachineByName(String name);
   public MachineNode getMachineByName(String name);
   
   
   /**
   /**
    * Get {@link RackNode} by its name.
    * Get {@link RackNode} by its name.
-   * @return The {@line RackNode} with the same name. Or null if not found.
+   * @return The {@link RackNode} with the same name. Or null if not found.
    */
    */
   public RackNode getRackByName(String name);
   public RackNode getRackByName(String name);
 
 

+ 1 - 1
src/tools/org/apache/hadoop/tools/rumen/DeskewedJobTraceReader.java

@@ -72,7 +72,7 @@ public class DeskewedJobTraceReader implements Closeable {
    * 
    * 
    * @param reader
    * @param reader
    *          the {@link JobTraceReader} that's being protected
    *          the {@link JobTraceReader} that's being protected
-   * @param skewBufferSize
+   * @param skewBufferLength
    *          [the number of late jobs that can preced a later out-of-order
    *          [the number of late jobs that can preced a later out-of-order
    *          earlier job
    *          earlier job
    * @throws IOException
    * @throws IOException

+ 34 - 0
src/tools/org/apache/hadoop/tools/rumen/HadoopLogsAnalyzer.java

@@ -1208,6 +1208,38 @@ public class HadoopLogsAnalyzer extends Configured implements Tool {
         attempt.spilledRecords = val;
         attempt.spilledRecords = val;
       }
       }
     }, counterString, "SPILLED_RECORDS");
     }, counterString, "SPILLED_RECORDS");
+    
+    // incorporate CPU usage
+    incorporateCounter(new SetField(attempt2) {
+      @Override
+      void set(long val) {
+        attempt.getResourceUsageMetrics().setCumulativeCpuUsage(val);
+      }
+    }, counterString, "CPU_MILLISECONDS");
+    
+    // incorporate virtual memory usage
+    incorporateCounter(new SetField(attempt2) {
+      @Override
+      void set(long val) {
+        attempt.getResourceUsageMetrics().setVirtualMemoryUsage(val);
+      }
+    }, counterString, "VIRTUAL_MEMORY_BYTES");
+    
+    // incorporate physical memory usage
+    incorporateCounter(new SetField(attempt2) {
+      @Override
+      void set(long val) {
+        attempt.getResourceUsageMetrics().setPhysicalMemoryUsage(val);
+      }
+    }, counterString, "PHYSICAL_MEMORY_BYTES");
+    
+    // incorporate heap usage
+    incorporateCounter(new SetField(attempt2) {
+      @Override
+      void set(long val) {
+        attempt.getResourceUsageMetrics().setHeapUsage(val);
+      }
+    }, counterString, "COMMITTED_HEAP_BYTES");
   }
   }
 
 
   private ParsedHost getAndRecordParsedHost(String hostName) {
   private ParsedHost getAndRecordParsedHost(String hostName) {
@@ -1594,6 +1626,8 @@ public class HadoopLogsAnalyzer extends Configured implements Tool {
       jobBeingTraced.setJobMapMB(jobconf.jobMapMB);
       jobBeingTraced.setJobMapMB(jobconf.jobMapMB);
       jobBeingTraced.setJobReduceMB(jobconf.jobReduceMB);
       jobBeingTraced.setJobReduceMB(jobconf.jobReduceMB);
 
 
+      jobBeingTraced.setJobProperties(jobconf.properties);
+      
       jobconf = null;
       jobconf = null;
 
 
       finalizeJob();
       finalizeJob();

+ 10 - 2
src/tools/org/apache/hadoop/tools/rumen/JobBuilder.java

@@ -74,6 +74,8 @@ public class JobBuilder {
   private static final Pattern heapPattern =
   private static final Pattern heapPattern =
       Pattern.compile("-Xmx([0-9]+[kKmMgGtT])");
       Pattern.compile("-Xmx([0-9]+[kKmMgGtT])");
 
 
+  private Properties jobConfigurationParameters = null;
+
   public JobBuilder(String jobID) {
   public JobBuilder(String jobID) {
     this.jobID = jobID;
     this.jobID = jobID;
   }
   }
@@ -142,7 +144,7 @@ public class JobBuilder {
           "JobBuilder.process(HistoryEvent): unknown event type");
           "JobBuilder.process(HistoryEvent): unknown event type");
   }
   }
 
 
-  private String extract(Properties conf, String[] names, String defaultValue) {
+  static String extract(Properties conf, String[] names, String defaultValue) {
     for (String name : names) {
     for (String name : names) {
       String result = conf.getProperty(name);
       String result = conf.getProperty(name);
 
 
@@ -206,6 +208,7 @@ public class JobBuilder {
           "JobBuilder.process(Properties conf) called after LoggedJob built");
           "JobBuilder.process(Properties conf) called after LoggedJob built");
     }
     }
 
 
+    //TODO remove this once the deprecate APIs in LoggedJob are removed
     result.setQueue(extract(conf, JobConfPropertyNames.QUEUE_NAMES
     result.setQueue(extract(conf, JobConfPropertyNames.QUEUE_NAMES
         .getCandidates(), "default"));
         .getCandidates(), "default"));
     result.setJobName(extract(conf, JobConfPropertyNames.JOB_NAMES
     result.setJobName(extract(conf, JobConfPropertyNames.JOB_NAMES
@@ -217,6 +220,8 @@ public class JobBuilder {
         JobConfPropertyNames.MAP_JAVA_OPTS_S.getCandidates()));
         JobConfPropertyNames.MAP_JAVA_OPTS_S.getCandidates()));
     maybeSetJobReduceMB(extractMegabytes(conf,
     maybeSetJobReduceMB(extractMegabytes(conf,
         JobConfPropertyNames.REDUCE_JAVA_OPTS_S.getCandidates()));
         JobConfPropertyNames.REDUCE_JAVA_OPTS_S.getCandidates()));
+        
+    this.jobConfigurationParameters = conf;
   }
   }
 
 
   /**
   /**
@@ -226,9 +231,12 @@ public class JobBuilder {
    * @return Parsed {@link LoggedJob} object.
    * @return Parsed {@link LoggedJob} object.
    */
    */
   public LoggedJob build() {
   public LoggedJob build() {
-    // The main job here is to build CDFs
+    // The main job here is to build CDFs and manage the conf
     finalized = true;
     finalized = true;
 
 
+    // set the conf
+    result.setJobProperties(jobConfigurationParameters);
+    
     // initialize all the per-job statistics gathering places
     // initialize all the per-job statistics gathering places
     Histogram[] successfulMapAttemptTimes =
     Histogram[] successfulMapAttemptTimes =
         new Histogram[ParsedHost.numberOfDistances() + 1];
         new Histogram[ParsedHost.numberOfDistances() + 1];

+ 3 - 18
src/tools/org/apache/hadoop/tools/rumen/JobConfigurationParser.java

@@ -17,13 +17,9 @@
  */
  */
 package org.apache.hadoop.tools.rumen;
 package org.apache.hadoop.tools.rumen;
 
 
-import java.io.BufferedInputStream;
 import java.io.IOException;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStream;
-import java.util.HashSet;
-import java.util.List;
 import java.util.Properties;
 import java.util.Properties;
-import java.util.Set;
 
 
 import javax.xml.parsers.DocumentBuilder;
 import javax.xml.parsers.DocumentBuilder;
 import javax.xml.parsers.DocumentBuilderFactory;
 import javax.xml.parsers.DocumentBuilderFactory;
@@ -38,22 +34,11 @@ import org.xml.sax.SAXException;
 
 
 /**
 /**
  * {@link JobConfigurationParser} parses the job configuration xml file, and
  * {@link JobConfigurationParser} parses the job configuration xml file, and
- * extracts various framework specific properties. It parses the file using a
+ * extracts configuration properties. It parses the file using a
  * stream-parser and thus is more memory efficient. [This optimization may be
  * stream-parser and thus is more memory efficient. [This optimization may be
  * postponed for a future release]
  * postponed for a future release]
  */
  */
 public class JobConfigurationParser {
 public class JobConfigurationParser {
-  final private Set<String> interested;
-
-  /**
-   * Constructor
-   * 
-   * @param interested
-   *          properties we should extract from the job configuration xml.
-   */
-  public JobConfigurationParser(List<String> interested) {
-    this.interested = new HashSet<String>(interested);
-  }
 
 
   /**
   /**
    * Parse the job configuration file (as an input stream) and return a
    * Parse the job configuration file (as an input stream) and return a
@@ -66,7 +51,7 @@ public class JobConfigurationParser {
    *         configuration xml.
    *         configuration xml.
    * @throws IOException
    * @throws IOException
    */
    */
-  Properties parse(InputStream input) throws IOException {
+  static Properties parse(InputStream input) throws IOException {
     Properties result = new Properties();
     Properties result = new Properties();
 
 
     try {
     try {
@@ -117,7 +102,7 @@ public class JobConfigurationParser {
           }
           }
         }
         }
 
 
-        if (interested.contains(attr) && value != null) {
+        if (attr != null && value != null) {
           result.put(attr, value);
           result.put(attr, value);
         }
         }
       }
       }

+ 1 - 1
src/tools/org/apache/hadoop/tools/rumen/JobHistoryParserFactory.java

@@ -38,7 +38,7 @@ public class JobHistoryParserFactory {
     throw new IOException("No suitable parser.");
     throw new IOException("No suitable parser.");
   }
   }
 
 
-  enum VersionDetector {
+  public enum VersionDetector {
     Hadoop20() {
     Hadoop20() {
 
 
       @Override
       @Override

+ 51 - 0
src/tools/org/apache/hadoop/tools/rumen/LoggedJob.java

@@ -22,6 +22,8 @@ package org.apache.hadoop.tools.rumen;
 
 
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.List;
+import java.util.Map;
+import java.util.Properties;
 import java.util.Set;
 import java.util.Set;
 import java.util.TreeSet;
 import java.util.TreeSet;
 
 
@@ -92,6 +94,8 @@ public class LoggedJob implements DeepCompare {
   double[] mapperTriesToSucceed;
   double[] mapperTriesToSucceed;
   double failedMapperFraction; // !!!!!
   double failedMapperFraction; // !!!!!
 
 
+  private Properties jobProperties = new Properties();
+  
   LoggedJob() {
   LoggedJob() {
 
 
   }
   }
@@ -102,6 +106,20 @@ public class LoggedJob implements DeepCompare {
     setJobID(jobID);
     setJobID(jobID);
   }
   }
 
 
+  /**
+   * Set the configuration properties of the job.
+   */
+  void setJobProperties(Properties conf) {
+    this.jobProperties = conf;
+  }
+  
+  /**
+   * Get the configuration properties of the job.
+   */
+  public Properties getJobProperties() {
+    return jobProperties;
+  }
+  
   void adjustTimes(long adjustment) {
   void adjustTimes(long adjustment) {
     submitTime += adjustment;
     submitTime += adjustment;
     launchTime += adjustment;
     launchTime += adjustment;
@@ -537,6 +555,35 @@ public class LoggedJob implements DeepCompare {
     }
     }
   }
   }
 
 
+  private void compareJobProperties(Properties prop1, Properties prop2,
+                                    TreePath loc, String eltname) 
+  throws DeepInequalityException {
+    if (prop1 == null && prop2 == null) {
+      return;
+    }
+
+    if (prop1 == null || prop2 == null) {
+      throw new DeepInequalityException(eltname + " miscompared [null]", 
+                                        new TreePath(loc, eltname));
+    }
+
+    if (prop1.size() != prop2.size()) {
+      throw new DeepInequalityException(eltname + " miscompared [size]", 
+                                        new TreePath(loc, eltname));
+    }
+    
+    for (Map.Entry<Object, Object> entry : prop1.entrySet()) {
+      Object v1 = entry.getValue();
+      Object v2 = prop2.get(entry.getKey());
+      if (v1 == null || v2 == null || !v1.equals(v2)) {
+        throw new DeepInequalityException(
+          eltname + " miscompared for value of key : " 
+            + entry.getKey().toString(), 
+          new TreePath(loc, eltname));
+      }
+    }
+  }
+  
   public void deepCompare(DeepCompare comparand, TreePath loc)
   public void deepCompare(DeepCompare comparand, TreePath loc)
       throws DeepInequalityException {
       throws DeepInequalityException {
     if (!(comparand instanceof LoggedJob)) {
     if (!(comparand instanceof LoggedJob)) {
@@ -600,5 +647,9 @@ public class LoggedJob implements DeepCompare {
     compare1(clusterReduceMB, other.clusterReduceMB, loc, "clusterReduceMB");
     compare1(clusterReduceMB, other.clusterReduceMB, loc, "clusterReduceMB");
     compare1(jobMapMB, other.jobMapMB, loc, "jobMapMB");
     compare1(jobMapMB, other.jobMapMB, loc, "jobMapMB");
     compare1(jobReduceMB, other.jobReduceMB, loc, "jobReduceMB");
     compare1(jobReduceMB, other.jobReduceMB, loc, "jobReduceMB");
+
+    // compare the job configuration parameters
+    compareJobProperties(jobProperties, other.getJobProperties(), loc, 
+                         "JobProperties");
   }
   }
 }
 }

+ 45 - 0
src/tools/org/apache/hadoop/tools/rumen/LoggedTaskAttempt.java

@@ -64,6 +64,9 @@ public class LoggedTaskAttempt implements DeepCompare {
 
 
   LoggedLocation location;
   LoggedLocation location;
 
 
+  // Initialize to default object for backward compatibility
+  ResourceUsageMetrics metrics = new ResourceUsageMetrics();
+  
   LoggedTaskAttempt() {
   LoggedTaskAttempt() {
     super();
     super();
   }
   }
@@ -349,8 +352,50 @@ public class LoggedTaskAttempt implements DeepCompare {
         attempt.spilledRecords = val;
         attempt.spilledRecords = val;
       }
       }
     }, counters, "SPILLED_RECORDS");
     }, counters, "SPILLED_RECORDS");
+    
+    // incorporate CPU usage
+    incorporateCounter(new SetField(this) {
+      @Override
+      void set(long val) {
+        metrics.setCumulativeCpuUsage(val);
+      }
+    }, counters, "CPU_MILLISECONDS");
+    
+    // incorporate virtual memory usage
+    incorporateCounter(new SetField(this) {
+      @Override
+      void set(long val) {
+        metrics.setVirtualMemoryUsage(val);
+      }
+    }, counters, "VIRTUAL_MEMORY_BYTES");
+    
+    // incorporate physical memory usage
+    incorporateCounter(new SetField(this) {
+      @Override
+      void set(long val) {
+        metrics.setPhysicalMemoryUsage(val);
+      }
+    }, counters, "PHYSICAL_MEMORY_BYTES");
+    
+    // incorporate heap usage
+    incorporateCounter(new SetField(this) {
+      @Override
+      void set(long val) {
+        metrics.setHeapUsage(val);
+      }
+    }, counters, "COMMITTED_HEAP_BYTES");
   }
   }
 
 
+  // Get the resource usage metrics
+  public ResourceUsageMetrics getResourceUsageMetrics() {
+    return metrics;
+  }
+  
+  // Set the resource usage metrics
+  void setResourceUsageMetrics(ResourceUsageMetrics metrics) {
+    this.metrics = metrics;
+  }
+  
   private static String canonicalizeCounterName(String nonCanonicalName) {
   private static String canonicalizeCounterName(String nonCanonicalName) {
     String result = nonCanonicalName.toLowerCase();
     String result = nonCanonicalName.toLowerCase();
 
 

+ 1 - 1
src/tools/org/apache/hadoop/tools/rumen/Node.java

@@ -24,7 +24,7 @@ import java.util.TreeSet;
 
 
 /**
 /**
  * {@link Node} represents a node in the cluster topology. A node can be a
  * {@link Node} represents a node in the cluster topology. A node can be a
- * {@MachineNode}, or a {@link RackNode}, etc.
+ * {@link MachineNode}, or a {@link RackNode}, etc.
  */
  */
 public class Node implements Comparable<Node> {
 public class Node implements Comparable<Node> {
   private static final SortedSet<Node> EMPTY_SET = 
   private static final SortedSet<Node> EMPTY_SET = 

+ 5 - 0
src/tools/org/apache/hadoop/tools/rumen/ParsedConfigFile.java

@@ -17,6 +17,7 @@
  */
  */
 package org.apache.hadoop.tools.rumen;
 package org.apache.hadoop.tools.rumen;
 
 
+import java.util.Properties;
 import java.util.regex.Pattern;
 import java.util.regex.Pattern;
 import java.util.regex.Matcher;
 import java.util.regex.Matcher;
 
 
@@ -55,6 +56,8 @@ class ParsedConfigFile {
   final String jobID;
   final String jobID;
 
 
   final boolean valid;
   final boolean valid;
+  
+  final Properties properties = new Properties();
 
 
   private int maybeGetIntValue(String propName, String attr, String value,
   private int maybeGetIntValue(String propName, String attr, String value,
       int oldValue) {
       int oldValue) {
@@ -143,6 +146,8 @@ class ParsedConfigFile {
                 "true".equals(((Text) field.getFirstChild()).getData());
                 "true".equals(((Text) field.getFirstChild()).getData());
           }
           }
         }
         }
+        
+        properties.setProperty(attr, value);
 
 
         if ("mapred.child.java.opts".equals(attr) && value != null) {
         if ("mapred.child.java.opts".equals(attr) && value != null) {
           Matcher matcher = heapPattern.matcher(value);
           Matcher matcher = heapPattern.matcher(value);

+ 160 - 0
src/tools/org/apache/hadoop/tools/rumen/ResourceUsageMetrics.java

@@ -0,0 +1,160 @@
+/**
+ * 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.tools.rumen;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableUtils;
+
+/**
+ * Captures the resource usage metrics.
+ */
+public class ResourceUsageMetrics implements Writable, DeepCompare  {
+  private long cumulativeCpuUsage;
+  private long virtualMemoryUsage;
+  private long physicalMemoryUsage;
+  private long heapUsage;
+  
+  public ResourceUsageMetrics() {
+  }
+  
+  /**
+   * Get the cumulative CPU usage.
+   */
+  public long getCumulativeCpuUsage() {
+    return cumulativeCpuUsage;
+  }
+  
+  /**
+   * Set the cumulative CPU usage.
+   */
+  public void setCumulativeCpuUsage(long usage) {
+    cumulativeCpuUsage = usage;
+  }
+  
+  /**
+   * Get the virtual memory usage.
+   */
+  public long getVirtualMemoryUsage() {
+    return virtualMemoryUsage;
+  }
+  
+  /**
+   * Set the virtual memory usage.
+   */
+  public void setVirtualMemoryUsage(long usage) {
+    virtualMemoryUsage = usage;
+  }
+  
+  /**
+   * Get the physical memory usage.
+   */
+  public long getPhysicalMemoryUsage() {
+    return physicalMemoryUsage;
+  }
+  
+  /**
+   * Set the physical memory usage.
+   */
+  public void setPhysicalMemoryUsage(long usage) {
+    physicalMemoryUsage = usage;
+  }
+  
+  /**
+   * Get the total heap usage.
+   */
+  public long getHeapUsage() {
+    return heapUsage;
+  }
+  
+  /**
+   * Set the total heap usage.
+   */
+  public void setHeapUsage(long usage) {
+    heapUsage = usage;
+  }
+  
+  /**
+   * Returns the size of the serialized data
+   */
+  public int size() {
+    int size = 0;
+    size += WritableUtils.getVIntSize(cumulativeCpuUsage);   // long #1
+    size += WritableUtils.getVIntSize(virtualMemoryUsage);   // long #2
+    size += WritableUtils.getVIntSize(physicalMemoryUsage);  // long #3
+    size += WritableUtils.getVIntSize(heapUsage);            // long #4
+    return size;
+  }
+  
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    cumulativeCpuUsage = WritableUtils.readVLong(in);  // long #1
+    virtualMemoryUsage = WritableUtils.readVLong(in);  // long #2
+    physicalMemoryUsage = WritableUtils.readVLong(in); // long #3
+    heapUsage = WritableUtils.readVLong(in);           // long #4
+  }
+  
+  @Override
+  public void write(DataOutput out) throws IOException {
+    //TODO Write resources version no too
+    WritableUtils.writeVLong(out, cumulativeCpuUsage);  // long #1
+    WritableUtils.writeVLong(out, virtualMemoryUsage);  // long #2
+    WritableUtils.writeVLong(out, physicalMemoryUsage); // long #3
+    WritableUtils.writeVLong(out, heapUsage);           // long #4
+  }
+
+  private static void compareMetric(long m1, long m2, TreePath loc) 
+  throws DeepInequalityException {
+    if (m1 != m2) {
+      throw new DeepInequalityException("Value miscompared:" + loc.toString(), 
+                                        loc);
+    }
+  }
+  
+  private static void compareSize(ResourceUsageMetrics m1, 
+                                  ResourceUsageMetrics m2, TreePath loc) 
+  throws DeepInequalityException {
+    if (m1.size() != m2.size()) {
+      throw new DeepInequalityException("Size miscompared: " + loc.toString(), 
+                                        loc);
+    }
+  }
+  
+  @Override
+  public void deepCompare(DeepCompare other, TreePath loc)
+      throws DeepInequalityException {
+    if (!(other instanceof ResourceUsageMetrics)) {
+      throw new DeepInequalityException("Comparand has wrong type", loc);
+    }
+
+    ResourceUsageMetrics metrics2 = (ResourceUsageMetrics) other;
+    compareMetric(getCumulativeCpuUsage(), metrics2.getCumulativeCpuUsage(), 
+                  new TreePath(loc, "cumulativeCpu"));
+    compareMetric(getVirtualMemoryUsage(), metrics2.getVirtualMemoryUsage(), 
+                  new TreePath(loc, "virtualMemory"));
+    compareMetric(getPhysicalMemoryUsage(), metrics2.getPhysicalMemoryUsage(), 
+                  new TreePath(loc, "physicalMemory"));
+    compareMetric(getHeapUsage(), metrics2.getHeapUsage(), 
+                  new TreePath(loc, "heapUsage"));
+    compareSize(this, metrics2, new TreePath(loc, "size"));
+  }
+}
+

+ 1 - 2
src/tools/org/apache/hadoop/tools/rumen/TaskAttemptInfo.java

@@ -17,7 +17,6 @@
  */
  */
 package org.apache.hadoop.tools.rumen;
 package org.apache.hadoop.tools.rumen;
 
 
-import org.apache.hadoop.mapred.TaskStatus;
 import org.apache.hadoop.mapred.TaskStatus.State;
 import org.apache.hadoop.mapred.TaskStatus.State;
 
 
 /**
 /**
@@ -38,7 +37,7 @@ public abstract class TaskAttemptInfo {
   }
   }
 
 
   /**
   /**
-   * Get the final {@link TaskStatus.State} of the task-attempt.
+   * Get the final {@link State} of the task-attempt.
    * 
    * 
    * @return the final <code>State</code> of the task-attempt
    * @return the final <code>State</code> of the task-attempt
    */
    */

+ 14 - 0
src/tools/org/apache/hadoop/tools/rumen/TaskInfo.java

@@ -23,14 +23,22 @@ public class TaskInfo {
   private final long bytesOut;
   private final long bytesOut;
   private final int recsOut;
   private final int recsOut;
   private final long maxMemory;
   private final long maxMemory;
+  private final ResourceUsageMetrics metrics;
 
 
   public TaskInfo(long bytesIn, int recsIn, long bytesOut, int recsOut,
   public TaskInfo(long bytesIn, int recsIn, long bytesOut, int recsOut,
       long maxMemory) {
       long maxMemory) {
+    this(bytesIn, recsIn, bytesOut, recsOut, maxMemory, 
+         new ResourceUsageMetrics());
+  }
+  
+  public TaskInfo(long bytesIn, int recsIn, long bytesOut, int recsOut,
+                  long maxMemory, ResourceUsageMetrics metrics) {
     this.bytesIn = bytesIn;
     this.bytesIn = bytesIn;
     this.recsIn = recsIn;
     this.recsIn = recsIn;
     this.bytesOut = bytesOut;
     this.bytesOut = bytesOut;
     this.recsOut = recsOut;
     this.recsOut = recsOut;
     this.maxMemory = maxMemory;
     this.maxMemory = maxMemory;
+    this.metrics = metrics;
   }
   }
 
 
   /**
   /**
@@ -70,4 +78,10 @@ public class TaskInfo {
     return maxMemory;
     return maxMemory;
   }
   }
 
 
+  /**
+   * @return Resource usage metrics
+   */
+  public ResourceUsageMetrics getResourceUsageMetrics() {
+    return metrics;
+  }
 }
 }

+ 111 - 36
src/tools/org/apache/hadoop/tools/rumen/TraceBuilder.java

@@ -22,6 +22,7 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStream;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Arrays;
+import java.util.Comparator;
 import java.util.LinkedList;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.List;
 import java.util.Properties;
 import java.util.Properties;
@@ -35,6 +36,7 @@ import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.mapred.JobHistory;
 import org.apache.hadoop.mapred.JobHistory;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.Tool;
@@ -49,7 +51,6 @@ public class TraceBuilder extends Configured implements Tool {
   static final int RUN_METHOD_FAILED_EXIT_CODE = 3;
   static final int RUN_METHOD_FAILED_EXIT_CODE = 3;
 
 
   TopologyBuilder topologyBuilder = new TopologyBuilder();
   TopologyBuilder topologyBuilder = new TopologyBuilder();
-  JobConfigurationParser jobConfParser;
   Outputter<LoggedJob> traceWriter;
   Outputter<LoggedJob> traceWriter;
   Outputter<LoggedNetworkTopology> topologyWriter;
   Outputter<LoggedNetworkTopology> topologyWriter;
 
 
@@ -67,48 +68,136 @@ public class TraceBuilder extends Configured implements Tool {
         IOException, ClassNotFoundException {
         IOException, ClassNotFoundException {
       int switchTop = 0;
       int switchTop = 0;
 
 
+      // to determine if the input paths should be recursively scanned or not
+      boolean doRecursiveTraversal = false;
+
       while (args[switchTop].startsWith("-")) {
       while (args[switchTop].startsWith("-")) {
         if (args[switchTop].equalsIgnoreCase("-demuxer")) {
         if (args[switchTop].equalsIgnoreCase("-demuxer")) {
           inputDemuxerClass =
           inputDemuxerClass =
-              Class.forName(args[++switchTop]).asSubclass(InputDemuxer.class);
-
-          ++switchTop;
+            Class.forName(args[++switchTop]).asSubclass(InputDemuxer.class);
+        } else if (args[switchTop].equalsIgnoreCase("-recursive")) {
+          doRecursiveTraversal = true;
         }
         }
+        ++switchTop;
       }
       }
 
 
       traceOutput = new Path(args[0 + switchTop]);
       traceOutput = new Path(args[0 + switchTop]);
       topologyOutput = new Path(args[1 + switchTop]);
       topologyOutput = new Path(args[1 + switchTop]);
 
 
       for (int i = 2 + switchTop; i < args.length; ++i) {
       for (int i = 2 + switchTop; i < args.length; ++i) {
+        inputs.addAll(processInputArgument(
+            args[i], conf, doRecursiveTraversal));
+      }
+    }
 
 
-        Path thisPath = new Path(args[i]);
-
-        FileSystem fs = thisPath.getFileSystem(conf);
-        if (fs.getFileStatus(thisPath).isDir()) {
-          FileStatus[] statuses = fs.listStatus(thisPath);
-
-          List<String> dirNames = new ArrayList<String>();
+    /**
+     * Compare the history file names, not the full paths.
+     * Job history file name format is such that doing lexicographic sort on the
+     * history file names should result in the order of jobs' submission times.
+     */
+    private static class HistoryLogsComparator
+    implements Comparator<FileStatus> {
+      @Override
+      public int compare(FileStatus file1, FileStatus file2) {
+        return file1.getPath().getName().compareTo(
+            file2.getPath().getName());
+      }
+    }
 
 
-          for (FileStatus s : statuses) {
-            if (s.isDir()) continue;
-            String name = s.getPath().getName();
+    private static class InputFilter implements PathFilter {
+      public boolean accept(Path path) {
+        return !(path.getName().endsWith(".crc")
+                 || path.getName().startsWith("."));
+      }
+    }
 
 
-            if (!(name.endsWith(".crc") || name.startsWith("."))) {
-              dirNames.add(name);
+    /**
+     * List files (possibly recursively) and get their statuses.
+     * @param path The path of the file/dir for which ls is to be done
+     * @param fs FileSystem of the path
+     * @param filter the user-supplied path filter
+     * @return the list of file statuses under the given path
+     */
+    static List<FileStatus> listFiles(Path path, FileSystem fs,
+        PathFilter filter, boolean isRecursive) throws IOException {
+      List<FileStatus> list = new ArrayList<FileStatus>();
+      FileStatus[] statuses = fs.listStatus(path, filter);
+      if (statuses != null) {
+        for (FileStatus status : statuses) {
+          if (status.isDir()) {
+            if (isRecursive) {
+              list.addAll(listFiles(status.getPath(), fs, filter, isRecursive));
             }
             }
+          } else {
+            list.add(status);
           }
           }
+        }
+      }
+      return list;
+    }
+
+    /**
+     * Processes the input file/folder argument. If the input is a file,
+     * then it is directly considered for further processing by TraceBuilder.
+     * If the input is a folder, then all the history logs in the
+     * input folder are considered for further processing.
+     *
+     * If isRecursive is true, then the input path is recursively scanned
+     * for job history logs for further processing by TraceBuilder.
+     *
+     * NOTE: If the input represents a globbed path, then it is first flattened
+     *       and then the individual paths represented by the globbed input
+     *       path are considered for further processing.
+     *
+     * @param input        input path, possibly globbed
+     * @param conf         configuration
+     * @param isRecursive  whether to recursively traverse the input paths to
+     *                     find history logs
+     * @return the input history log files' paths
+     * @throws FileNotFoundException
+     * @throws IOException
+     */
+    static List<Path> processInputArgument(String input, Configuration conf,
+        boolean isRecursive) throws FileNotFoundException, IOException {
+      Path inPath = new Path(input);
+      FileSystem fs = inPath.getFileSystem(conf);
+      FileStatus[] inStatuses = fs.globStatus(inPath);
+
+      List<Path> inputPaths = new LinkedList<Path>();
+      if (inStatuses == null || inStatuses.length == 0) {
+        return inputPaths;
+      }
 
 
-          String[] sortableNames = dirNames.toArray(new String[1]);
+      for (FileStatus inStatus : inStatuses) {
+        Path thisPath = inStatus.getPath();
+        if (inStatus.isDir()) {
 
 
-          Arrays.sort(sortableNames);
+          // Find list of files in this path(recursively if -recursive option
+              // is specified).
+          List<FileStatus> historyLogs = new ArrayList<FileStatus>();
 
 
-          for (String dirName : sortableNames) {
-            inputs.add(new Path(thisPath, dirName));
+          List<FileStatus> statuses = listFiles(thisPath, fs, new InputFilter(),
+              isRecursive);
+          for (FileStatus child : statuses) {
+            historyLogs.add(child);
+          }
+          if (historyLogs.size() > 0) {
+            // Add the sorted history log file names in this path to the
+            // inputPaths list
+            FileStatus[] sortableNames =
+              historyLogs.toArray(new FileStatus[historyLogs.size()]);
+            Arrays.sort(sortableNames, new HistoryLogsComparator());
+
+            for (FileStatus historyLog : sortableNames) {
+              inputPaths.add(historyLog.getPath());
+            }
           }
           }
         } else {
         } else {
-          inputs.add(thisPath);
+          inputPaths.add(thisPath);
         }
         }
       }
       }
+
+      return inputPaths;
     }
     }
   }
   }
 
 
@@ -169,25 +258,11 @@ public class TraceBuilder extends Configured implements Tool {
     return jobId != null;
     return jobId != null;
   }
   }
 
 
-  private void addInterestedProperties(List<String> interestedProperties,
-      String[] names) {
-    for (String name : names) {
-      interestedProperties.add(name);
-    }
-  }
 
 
   @SuppressWarnings("unchecked")
   @SuppressWarnings("unchecked")
   @Override
   @Override
   public int run(String[] args) throws Exception {
   public int run(String[] args) throws Exception {
     MyOptions options = new MyOptions(args, getConf());
     MyOptions options = new MyOptions(args, getConf());
-    List<String> interestedProperties = new ArrayList<String>();
-    {
-      for (JobConfPropertyNames candidateSet : JobConfPropertyNames.values()) {
-        addInterestedProperties(interestedProperties, candidateSet
-            .getCandidates());
-      }
-    }
-    jobConfParser = new JobConfigurationParser(interestedProperties);
     traceWriter = options.clazzTraceOutputter.newInstance();
     traceWriter = options.clazzTraceOutputter.newInstance();
     traceWriter.init(options.traceOutput, getConf());
     traceWriter.init(options.traceOutput, getConf());
     topologyWriter = new DefaultOutputter<LoggedNetworkTopology>();
     topologyWriter = new DefaultOutputter<LoggedNetworkTopology>();
@@ -232,7 +307,7 @@ public class TraceBuilder extends Configured implements Tool {
               }
               }
 
 
               if (isJobConfXml(filePair.first(), ris)) {
               if (isJobConfXml(filePair.first(), ris)) {
-                processJobConf(jobConfParser.parse(ris.rewind()), jobBuilder);
+            	processJobConf(JobConfigurationParser.parse(ris.rewind()), jobBuilder);
               } else {
               } else {
                 parser = JobHistoryParserFactory.getParser(ris);
                 parser = JobHistoryParserFactory.getParser(ris);
                 if (parser == null) {
                 if (parser == null) {

+ 18 - 2
src/tools/org/apache/hadoop/tools/rumen/ZombieJob.java

@@ -120,8 +120,20 @@ public class ZombieJob implements JobStory {
   @Override
   @Override
   public synchronized JobConf getJobConf() {
   public synchronized JobConf getJobConf() {
     if (jobConf == null) {
     if (jobConf == null) {
-      // TODO : add more to jobConf ?
       jobConf = new JobConf();
       jobConf = new JobConf();
+      
+      // Add parameters from the configuration in the job trace
+      //
+      // The reason why the job configuration parameters, as seen in the jobconf
+      // file, are added first because the specialized values obtained from 
+      // Rumen should override the job conf values.
+      //
+      for (Map.Entry<Object, Object> entry : job.getJobProperties().entrySet()) {
+        jobConf.set(entry.getKey().toString(), entry.getValue().toString());
+      }
+      
+      //TODO Eliminate parameters that are already copied from the job's 
+      // configuration file.
       jobConf.setJobName(getName());
       jobConf.setJobName(getName());
       jobConf.setUser(getUser());
       jobConf.setUser(getUser());
       jobConf.setNumMapTasks(getNumberMaps());
       jobConf.setNumMapTasks(getNumberMaps());
@@ -622,6 +634,7 @@ public class ZombieJob implements JobStory {
     long outputBytes = -1;
     long outputBytes = -1;
     long outputRecords = -1;
     long outputRecords = -1;
     long heapMegabytes = -1;
     long heapMegabytes = -1;
+    ResourceUsageMetrics metrics = new ResourceUsageMetrics();
 
 
     Values type = loggedTask.getTaskType();
     Values type = loggedTask.getTaskType();
     if ((type != Values.MAP) && (type != Values.REDUCE)) {
     if ((type != Values.MAP) && (type != Values.REDUCE)) {
@@ -656,12 +669,15 @@ public class ZombieJob implements JobStory {
             (job.getJobReduceMB() > 0) ? job.getJobReduceMB() : job
             (job.getJobReduceMB() > 0) ? job.getJobReduceMB() : job
                 .getHeapMegabytes();
                 .getHeapMegabytes();
       }
       }
+      // set the resource usage metrics
+      metrics = attempt.getResourceUsageMetrics();
       break;
       break;
     }
     }
 
 
     TaskInfo taskInfo =
     TaskInfo taskInfo =
         new TaskInfo(inputBytes, (int) inputRecords, outputBytes,
         new TaskInfo(inputBytes, (int) inputRecords, outputBytes,
-            (int) outputRecords, (int) heapMegabytes);
+            (int) outputRecords, (int) heapMegabytes,
+            metrics);
     return taskInfo;
     return taskInfo;
   }
   }
 
 

+ 377 - 0
src/tools/org/apache/hadoop/tools/rumen/package-info.java

@@ -0,0 +1,377 @@
+/*
+ * 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.
+ */
+
+/** Rumen is a data extraction and analysis tool built for 
+ * <a href="http://hadoop.apache.org/">Apache Hadoop</a>. Rumen mines job history
+ * logs to extract meaningful data and stores it into an easily-parsed format.
+ * 
+ * The default output format of Rumen is <a href="http://www.json.org">JSON</a>.
+ * Rumen uses the <a href="http://jackson.codehaus.org/">Jackson</a> library to 
+ * create JSON objects.
+ * <br><br>
+ * 
+ * The following classes can be used to programmatically invoke Rumen:
+ * <ol>
+ *   <li>
+ *    {@link org.apache.hadoop.tools.rumen.JobConfigurationParser}<br>
+ *      A parser to parse and filter out interesting properties from job 
+ *      configuration.
+ *      
+ *      <br><br>
+ *      <i>Sample code</i>:
+ *      <pre>
+ *      <code>
+ *        // An example to parse and filter out job name
+ *        
+ *        String conf_filename = .. // assume the job configuration filename here
+ *        
+ *        // construct a list of interesting properties
+ *        List<String> interestedProperties = new ArrayList<String>();
+ *        interestedProperties.add("mapreduce.job.name");
+ *        
+ *        JobConfigurationParser jcp = 
+ *          new JobConfigurationParser(interestedProperties);
+ *
+ *        InputStream in = new FileInputStream(conf_filename);
+ *        Properties parsedProperties = jcp.parse(in);
+ *     </code>
+ *     </pre>
+ *     Some of the commonly used interesting properties are enumerated in 
+ *     {@link org.apache.hadoop.tools.rumen.JobConfPropertyNames}. <br><br>
+ *     
+ *     <b>Note:</b>
+ *        A single instance of {@link org.apache.hadoop.tools.rumen.JobConfigurationParser} 
+ *        can be used to parse multiple job configuration files. 
+ *     
+ *   </li>
+ *   <li>
+ *    {@link org.apache.hadoop.tools.rumen.JobHistoryParser} <br>
+ *      A parser that parses job history files. It is an interface and actual 
+ *      implementations are defined as Enum in 
+ *      {@link org.apache.hadoop.tools.rumen.JobHistoryParserFactory}. Note that
+ *      {@link org.apache.hadoop.tools.rumen.RewindableInputStream}<br>
+ *      is a wrapper class around {@link java.io.InputStream} to make the input 
+ *      stream rewindable.
+ *      
+ *      <br>
+ *      <i>Sample code</i>:
+ *      <pre>
+ *      <code>
+ *        // An example to parse a current job history file i.e a job history 
+ *        // file for which the version is known
+ *        
+ *        String filename = .. // assume the job history filename here
+ *        
+ *        InputStream in = new FileInputStream(filename);
+ *        
+ *        HistoryEvent event = null;
+ *        
+ *        JobHistoryParser parser = new CurrentJHParser(in);
+ *        
+ *        event = parser.nextEvent();
+ *        // process all the events
+ *        while (event != null) {
+ *          // ... process all event
+ *          event = parser.nextEvent();
+ *        }
+ *        
+ *        // close the parser and the underlying stream
+ *        parser.close();
+ *      </code>
+ *      </pre>
+ *      
+ *      {@link org.apache.hadoop.tools.rumen.JobHistoryParserFactory} provides a 
+ *      {@link org.apache.hadoop.tools.rumen.JobHistoryParserFactory#getParser(org.apache.hadoop.tools.rumen.RewindableInputStream)}
+ *      API to get a parser for parsing the job history file. Note that this
+ *      API can be used if the job history version is unknown.<br><br>
+ *      <i>Sample code</i>:
+ *      <pre>
+ *      <code>
+ *        // An example to parse a job history for which the version is not 
+ *        // known i.e using JobHistoryParserFactory.getParser()
+ *        
+ *        String filename = .. // assume the job history filename here
+ *        
+ *        InputStream in = new FileInputStream(filename);
+ *        RewindableInputStream ris = new RewindableInputStream(in);
+ *        
+ *        // JobHistoryParserFactory will check and return a parser that can
+ *        // parse the file
+ *        JobHistoryParser parser = JobHistoryParserFactory.getParser(ris);
+ *        
+ *        // now use the parser to parse the events
+ *        HistoryEvent event = parser.nextEvent();
+ *        while (event != null) {
+ *          // ... process the event
+ *          event = parser.nextEvent();
+ *        }
+ *        
+ *        parser.close();
+ *      </code>
+ *      </pre>
+ *      <b>Note:</b>
+ *        Create one instance to parse a job history log and close it after use.
+ *  </li>
+ *  <li>
+ *    {@link org.apache.hadoop.tools.rumen.TopologyBuilder}<br>
+ *      Builds the cluster topology based on the job history events. Every 
+ *      job history file consists of events. Each event can be represented using
+ *      {@link org.apache.hadoop.mapreduce.jobhistory.HistoryEvent}. 
+ *      These events can be passed to {@link org.apache.hadoop.tools.rumen.TopologyBuilder} using 
+ *      {@link org.apache.hadoop.tools.rumen.TopologyBuilder#process(org.apache.hadoop.mapreduce.jobhistory.HistoryEvent)}.
+ *      A cluster topology can be represented using {@link org.apache.hadoop.tools.rumen.LoggedNetworkTopology}.
+ *      Once all the job history events are processed, the cluster 
+ *      topology can be obtained using {@link org.apache.hadoop.tools.rumen.TopologyBuilder#build()}.
+ *      
+ *      <br><br>
+ *      <i>Sample code</i>:
+ *      <pre>
+ *      <code>
+ *        // Building topology for a job history file represented using 
+ *        // 'filename' and the corresponding configuration file represented 
+ *        // using 'conf_filename'
+ *        String filename = .. // assume the job history filename here
+ *        String conf_filename = .. // assume the job configuration filename here
+ *        
+ *        InputStream jobConfInputStream = new FileInputStream(filename);
+ *        InputStream jobHistoryInputStream = new FileInputStream(conf_filename);
+ *        
+ *        TopologyBuilder tb = new TopologyBuilder();
+ *        
+ *        // construct a list of interesting properties
+ *        List<String> interestingProperties = new ArrayList<Strng>();
+ *        // add the interesting properties here
+ *        interestingProperties.add("mapreduce.job.name");
+ *        
+ *        JobConfigurationParser jcp = 
+ *          new JobConfigurationParser(interestingProperties);
+ *        
+ *        // parse the configuration file
+ *        tb.process(jcp.parse(jobConfInputStream));
+ *        
+ *        // read the job history file and pass it to the 
+ *        // TopologyBuilder.
+ *        JobHistoryParser parser = new CurrentJHParser(jobHistoryInputStream);
+ *        HistoryEvent e;
+ *        
+ *        // read and process all the job history events
+ *        while ((e = parser.nextEvent()) != null) {
+ *          tb.process(e);
+ *        }
+ *        
+ *        LoggedNetworkTopology topology = tb.build();
+ *      </code>
+ *      </pre>
+ *  </li>
+ *  <li>
+ *    {@link org.apache.hadoop.tools.rumen.JobBuilder}<br>
+ *      Summarizes a job history file.
+ *      {@link org.apache.hadoop.tools.rumen.TraceBuilder} provides  
+ *      {@link org.apache.hadoop.tools.rumen.TraceBuilder#extractJobID(String)} 
+ *      API for extracting job id from job history or job configuration files
+ *      which can be used for instantiating {@link org.apache.hadoop.tools.rumen.JobBuilder}. 
+ *      {@link org.apache.hadoop.tools.rumen.JobBuilder} generates a 
+ *      {@link org.apache.hadoop.tools.rumen.LoggedJob} object via 
+ *      {@link org.apache.hadoop.tools.rumen.JobBuilder#build()}. 
+ *      See {@link org.apache.hadoop.tools.rumen.LoggedJob} for more details.
+ *      
+ *      <br><br>
+ *      <i>Sample code</i>:
+ *      <pre>
+ *      <code>
+ *        // An example to summarize a current job history file 'filename'
+ *        // and the corresponding configuration file 'conf_filename'
+ *        
+ *        String filename = .. // assume the job history filename here
+ *        String conf_filename = .. // assume the job configuration filename here
+ *        
+ *        InputStream jobConfInputStream = new FileInputStream(job_filename);
+ *        InputStream jobHistoryInputStream = new FileInputStream(conf_filename);
+ *        
+ *        String jobID = TraceBuilder.extractJobID(job_filename);
+ *        JobBuilder jb = new JobBuilder(jobID);
+ *        
+ *        // construct a list of interesting properties
+ *        List<String> interestingProperties = new ArrayList<Strng>();
+ *        // add the interesting properties here
+ *        interestingProperties.add("mapreduce.job.name");
+ *        
+ *        JobConfigurationParser jcp = 
+ *          new JobConfigurationParser(interestingProperties);
+ *        
+ *        // parse the configuration file
+ *        jb.process(jcp.parse(jobConfInputStream));
+ *        
+ *        // parse the job history file
+ *        JobHistoryParser parser = new CurrentJHParser(jobHistoryInputStream);
+ *        try {
+ *          HistoryEvent e;
+ *          // read and process all the job history events
+ *          while ((e = parser.nextEvent()) != null) {
+ *            jobBuilder.process(e);
+ *          }
+ *        } finally {
+ *          parser.close();
+ *        }
+ *        
+ *        LoggedJob job = jb.build();
+ *      </code>
+ *      </pre>
+ *     <b>Note:</b>
+ *       The order of parsing the job configuration file or job history file is 
+ *       not important. Create one instance to parse the history file and job 
+ *       configuration.
+ *   </li>
+ *   <li>
+ *    {@link org.apache.hadoop.tools.rumen.DefaultOutputter}<br>
+ *      Implements {@link org.apache.hadoop.tools.rumen.Outputter} and writes 
+ *      JSON object in text format to the output file. 
+ *      {@link org.apache.hadoop.tools.rumen.DefaultOutputter} can be 
+ *      initialized with the output filename.
+ *      
+ *      <br><br>
+ *      <i>Sample code</i>:  
+ *      <pre>
+ *      <code>
+ *        // An example to summarize a current job history file represented by
+ *        // 'filename' and the configuration filename represented using 
+ *        // 'conf_filename'. Also output the job summary to 'out.json' along 
+ *        // with the cluster topology to 'topology.json'.
+ *        
+ *        String filename = .. // assume the job history filename here
+ *        String conf_filename = .. // assume the job configuration filename here
+ *        
+ *        Configuration conf = new Configuration();
+ *        DefaultOutputter do = new DefaultOutputter();
+ *        do.init("out.json", conf);
+ *        
+ *        InputStream jobConfInputStream = new FileInputStream(filename);
+ *        InputStream jobHistoryInputStream = new FileInputStream(conf_filename);
+ *        
+ *        // extract the job-id from the filename
+ *        String jobID = TraceBuilder.extractJobID(filename);
+ *        JobBuilder jb = new JobBuilder(jobID);
+ *        TopologyBuilder tb = new TopologyBuilder();
+ *        
+ *        // construct a list of interesting properties
+ *        List<String> interestingProperties = new ArrayList<Strng>();
+ *        // add the interesting properties here
+ *        interestingProperties.add("mapreduce.job.name");
+ *        
+ *        JobConfigurationParser jcp =
+ *          new JobConfigurationParser(interestingProperties);
+ *          
+ *        // parse the configuration file
+ *        tb.process(jcp.parse(jobConfInputStream));
+ *        
+ *        // read the job history file and pass it to the
+ *        // TopologyBuilder.
+ *        JobHistoryParser parser = new CurrentJHParser(jobHistoryInputStream);
+ *        HistoryEvent e;
+ *        while ((e = parser.nextEvent()) != null) {
+ *          jb.process(e);
+ *          tb.process(e);
+ *        }
+ *        
+ *        LoggedJob j = jb.build();
+ *        
+ *        // serialize the job summary in json (text) format
+ *        do.output(j);
+ *        
+ *        // close
+ *        do.close();
+ *        
+ *        do.init("topology.json", conf);
+ *        
+ *        // get the job summary using TopologyBuilder
+ *        LoggedNetworkTopology topology = topologyBuilder.build();
+ *        
+ *        // serialize the cluster topology in json (text) format
+ *        do.output(topology);
+ *        
+ *        // close
+ *        do.close();
+ *      </code>
+ *      </pre>
+ *   </li>
+ *   <li>
+ *    {@link org.apache.hadoop.tools.rumen.JobTraceReader}<br>
+ *      A reader for reading {@link org.apache.hadoop.tools.rumen.LoggedJob} serialized using 
+ *      {@link org.apache.hadoop.tools.rumen.DefaultOutputter}. {@link org.apache.hadoop.tools.rumen.LoggedJob} 
+ *      provides various APIs for extracting job details. Following are the most
+ *      commonly used ones
+ *        <ul>
+ *          <li>{@link org.apache.hadoop.tools.rumen.LoggedJob#getMapTasks()} : Get the map tasks</li>
+ *          <li>{@link org.apache.hadoop.tools.rumen.LoggedJob#getReduceTasks()} : Get the reduce tasks</li>
+ *          <li>{@link org.apache.hadoop.tools.rumen.LoggedJob#getOtherTasks()} : Get the setup/cleanup tasks</li>
+ *          <li>{@link org.apache.hadoop.tools.rumen.LoggedJob#getOutcome()} : Get the job's outcome</li>
+ *          <li>{@link org.apache.hadoop.tools.rumen.LoggedJob#getSubmitTime()} : Get the job's submit time</li>
+ *          <li>{@link org.apache.hadoop.tools.rumen.LoggedJob#getFinishTime()} : Get the job's finish time</li>
+ *        </ul>
+ *        
+ *      <br><br>
+ *      <i>Sample code</i>:
+ *      <pre>
+ *      <code>
+ *        // An example to read job summary from a trace file 'out.json'.
+ *        JobTraceReader reader = new JobTracerReader("out.json");
+ *        LoggedJob job = reader.getNext();
+ *        while (job != null) {
+ *          // .... process job level information
+ *          for (LoggedTask task : job.getMapTasks()) {
+ *            // process all the map tasks in the job
+ *            for (LoggedTaskAttempt attempt : task.getAttempts()) {
+ *              // process all the map task attempts in the job
+ *            }
+ *          }
+ *          
+ *          // get the next job
+ *          job = reader.getNext();
+ *        }
+ *        reader.close();
+ *      </code>
+ *      </pre>         
+ *   </li>
+ *   <li>
+ *    {@link org.apache.hadoop.tools.rumen.ClusterTopologyReader}<br>
+ *      A reader to read {@link org.apache.hadoop.tools.rumen.LoggedNetworkTopology} serialized using 
+ *      {@link org.apache.hadoop.tools.rumen.DefaultOutputter}. {@link org.apache.hadoop.tools.rumen.ClusterTopologyReader} can be 
+ *      initialized using the serialized topology filename. 
+ *      {@link org.apache.hadoop.tools.rumen.ClusterTopologyReader#get()} can
+ *      be used to get the 
+ *      {@link org.apache.hadoop.tools.rumen.LoggedNetworkTopology}. 
+ *      
+ *      <br><br>
+ *      <i>Sample code</i>:
+ *      <pre>
+ *      <code>
+ *        // An example to read the cluster topology from a topology output file
+ *        // 'topology.json'
+ *        ClusterTopologyReader reader = new ClusterTopologyReader("topology.json");
+ *        LoggedNetworkTopology topology  = reader.get();
+ *        for (LoggedNetworkTopology t : topology.getChildren()) {
+ *          // process the cluster topology
+ *        }
+ *        reader.close();
+ *      </code>
+ *      </pre>
+ *   </li>
+ * </ol>     
+ */
+
+package org.apache.hadoop.tools.rumen;