فهرست منبع

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. BIN
      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. BIN
      src/test/tools/data/rumen/small-trace-test/counters-test-trace.json.gz
  65. BIN
      src/test/tools/data/rumen/small-trace-test/dispatch-trace-output.json.gz
  66. BIN
      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. BIN
      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
 
+    MAPREDUCE-3118. Backport Gridmix and Rumen features to 
+                    branch-0.20-security (Ravi Gummadi via amarrk)
   BUG FIXES
 
     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="${hdfs.src.dir}"/>        	
     	<packageset dir="${examples.dir}"/>
+    	<packageset dir="${tools.src}"/>
 
     	<packageset dir="src/contrib/streaming/src/java"/>
     	<packageset dir="src/contrib/data_join/src/java"/>
@@ -1371,6 +1372,8 @@
        <packageset dir="src/core"/>
        <packageset dir="src/mapred"/>
        <packageset dir="src/tools"/>
+       <packageset dir="${tools.src}"/>
+       <packageset dir="${tools.src}"/>
        <classpath >
          <path refid="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.test" location="${root}/src/test"/>
+  <property name="src.test.data" location="${root}/src/test/data"/>
   <!-- Property added for contrib system tests -->
   <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="build.test" value="${build.test}"/>
+      <sysproperty key="src.test.data" value="${src.test.data}"/>
       <sysproperty key="contrib.name" value="${name}"/>
       
       <!-- 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 long accBytes = 0L;
   private long accRecords = 0L;
+  private int unspilledBytes = 0;
+  private int minSpilledBytes = 0;
 
   /**
    * @param targetBytes Expected byte count.
@@ -48,6 +50,14 @@ class AvgRecordFactory extends RecordFactory {
    */
   public AvgRecordFactory(long targetBytes, long targetRecords,
       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.targetRecords = targetRecords <= 0 && this.targetBytes >= 0
       ? Math.max(1,
@@ -58,6 +68,7 @@ class AvgRecordFactory extends RecordFactory {
     avgrec = (int) Math.min(Integer.MAX_VALUE, tmp + 1);
     keyLen = Math.max(1,
         (int)(tmp * Math.min(1.0f, conf.getFloat(GRIDMIX_KEY_FRC, 0.1f))));
+    this.minSpilledBytes = minSpilledBytes;
   }
 
   @Override
@@ -67,14 +78,33 @@ class AvgRecordFactory extends RecordFactory {
     }
     final int reclen = accRecords++ >= step ? avgrec - 1 : avgrec;
     final int len = (int) Math.min(targetBytes - accBytes, reclen);
+    
+    unspilledBytes += len;
+    
     // len != reclen?
     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 {
-      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;
   }
 

+ 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.net.URI;
-import java.util.Collections;
-import java.util.List;
-import java.util.ArrayList;
 
 import org.apache.hadoop.conf.Configuration;
 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.LogFactory;
 
@@ -50,4 +44,14 @@ public class EchoUserResolver implements UserResolver {
       UserGroupInformation 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 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.io.IOUtils;
 
@@ -34,7 +32,7 @@ class FileQueue extends InputStream {
 
   private int idx = -1;
   private long curlen = -1L;
-  private FSDataInputStream input;
+  private InputStream input;
   private final byte[] z = new byte[1];
   private final Path[] paths;
   private final long[] lengths;
@@ -64,9 +62,9 @@ class FileQueue extends InputStream {
     idx = (idx + 1) % paths.length;
     curlen = lengths[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

+ 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 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.fs.PathFilter;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.io.BytesWritable;
 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.JobClient;
 import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.Utils;
 import org.apache.hadoop.mapreduce.InputFormat;
 import org.apache.hadoop.mapreduce.InputSplit;
 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.output.FileOutputFormat;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.StringUtils;
 
 // TODO can replace with form of GridmixJob
 class GenerateData extends GridmixJob {
@@ -86,14 +90,103 @@ class GenerateData extends GridmixJob {
    * Replication of generated data.
    */
   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)
       throws IOException {
-    super(conf, 0L, "GRIDMIX_GENDATA");
+    super(conf, 0L, JOB_NAME);
     job.getConfiguration().setLong(GRIDMIX_GEN_BYTES, genbytes);
     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
   public Job call() throws IOException, InterruptedException,
                            ClassNotFoundException {
@@ -101,6 +194,18 @@ class GenerateData extends GridmixJob {
     ugi.doAs( new PrivilegedExceptionAction <Job>() {
        public Job run() throws IOException, ClassNotFoundException,
                                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.setNumReduceTasks(0);
         job.setMapOutputKeyClass(NullWritable.class);
@@ -113,12 +218,15 @@ class GenerateData extends GridmixJob {
         } 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 GenDataMapper
       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.Path;
 import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.mapred.gridmix.GenerateData.DataStatistics;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
+import org.apache.hadoop.tools.rumen.JobStoryProducer;
 import org.apache.hadoop.tools.rumen.ZombieJobProducer;
 
 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";
 
+  /**
+   * 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
   private JobFactory factory;
   private JobSubmitter submitter;
   private JobMonitor monitor;
   private Statistics statistics;
+  private Summarizer summarizer;
 
   // Shutdown hook
   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
    */
-  protected void writeInputData(long genbytes, Path ioPath)
+  protected void writeInputData(long genbytes, Path inputDir)
       throws IOException, InterruptedException {
     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) +
         " 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
     TimeUnit.SECONDS.sleep(10);
     try {
-      genData.getJob().waitForCompletion(false);
+      job.getJob().waitForCompletion(false);
     } catch (ClassNotFoundException 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.
    * @param conf Configuration data, no keys specific to this context
    * @param traceIn Either a Path to the trace data or &quot;-&quot; for
    *                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 startFlag Semaphore for starting job trace pipeline
    */
@@ -155,8 +238,8 @@ public class Gridmix extends Configured implements Tool {
       Path scratchDir, CountDownLatch startFlag, UserResolver userResolver)
       throws IOException {
     try {
-      GridmixJobSubmissionPolicy policy = GridmixJobSubmissionPolicy.getPolicy(
-        conf, GridmixJobSubmissionPolicy.STRESS);
+      Path inputDir = getGridmixInputDataPath(ioPath);
+      GridmixJobSubmissionPolicy policy = getJobSubmissionPolicy(conf);
       LOG.info(" Submission policy is " + policy.name());
       statistics = new Statistics(conf, policy.getPollingInterval(), startFlag);
       monitor = createJobMonitor(statistics);
@@ -167,16 +250,24 @@ public class Gridmix extends Configured implements Tool {
         monitor, conf.getInt(
           GRIDMIX_SUB_THR, noOfSubmitterThreads), conf.getInt(
           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) {
         statistics.addJobStatsListeners(factory);
       } else {
         statistics.addClusterStatsObservers(factory);
       }
-      
+
+      // add the gridmix run summarizer to the statistics
+      statistics.addJobStatsListeners(summarizer.getExecutionSummarizer());
+      statistics.addClusterStatsObservers(summarizer.getClusterSummarizer());
+
       monitor.start();
       submitter.start();
     }catch(Exception e) {
@@ -201,9 +292,8 @@ public class Gridmix extends Configured implements Tool {
     throws IOException {
     return GridmixJobSubmissionPolicy.getPolicy(
       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 {
@@ -217,6 +307,10 @@ public class Gridmix extends Configured implements Tool {
         return runJob(conf,argv);
       }
     });
+    
+    // print the run summary
+    System.out.print("\n\n");
+    System.out.println(summarizer.toString());
     return val; 
   }
 
@@ -232,6 +326,9 @@ public class Gridmix extends Configured implements Tool {
       printUsage(System.err);
       return 1;
     }
+    
+    // Should gridmix generate distributed cache data ?
+    boolean generate = false;
     long genbytes = -1L;
     String traceIn = null;
     Path ioPath = null;
@@ -243,6 +340,7 @@ public class Gridmix extends Configured implements Tool {
       for (int i = 0; i < argv.length - 2; ++i) {
         if ("-generate".equals(argv[i])) {
           genbytes = StringUtils.TraditionalBinaryPrefix.string2long(argv[++i]);
+          generate = true;
         } else if ("-users".equals(argv[i])) {
           userRsrc = new URI(argv[++i]);
         } else {
@@ -250,9 +348,22 @@ public class Gridmix extends Configured implements Tool {
           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]);
       traceIn = argv[argv.length - 1];
     } catch (Exception e) {
@@ -260,17 +371,46 @@ public class Gridmix extends Configured implements Tool {
       printUsage(System.err);
       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,
-      UserResolver userResolver) throws IOException, InterruptedException {
+      UserResolver userResolver, boolean generate)
+      throws IOException, InterruptedException {
+    DataStatistics stats = null;
     InputStream trace = null;
+    final FileSystem inputFs = ioPath.getFileSystem(conf);
+    ioPath = ioPath.makeQualified(inputFs);
+
     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"));
-      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.
       Runtime.getRuntime().addShutdownHook(sdh);
       CountDownLatch startFlag = new CountDownLatch(1);
@@ -278,12 +418,30 @@ public class Gridmix extends Configured implements Tool {
         // Create, start job submission threads
         startThreads(conf, traceIn, ioPath, scratchDir, startFlag,
             userResolver);
+
+        Path inputDir = getGridmixInputDataPath(ioPath);
+
         // Write input data if specified
         if (genbytes > 0) {
-          writeInputData(genbytes, ioPath);
+          writeInputData(genbytes, inputDir);
         }
+
+        // publish the data statistics
+        stats = GenerateData.publishDataStatistics(inputDir, genbytes, conf);
+
         // scan input dir contents
         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();
         statistics.start();
       } catch (Throwable e) {
@@ -313,11 +471,73 @@ public class Gridmix extends Configured implements Tool {
 
       }
     } finally {
+      if (factory != null) {
+        summarizer.finalize(factory, traceIn, genbytes, userResolver, stats, 
+                            conf);
+      }
       IOUtils.cleanup(LOG, trace);
     }
     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
    * 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 {
     int res = -1;
     try {
-      res = ToolRunner.run(new Configuration(), new Gridmix(), argv);
+      res = ToolRunner.run(new Configuration(), new Gridmix(argv), argv);
     } finally {
       System.exit(res);
     }
@@ -416,6 +636,11 @@ public class Gridmix extends Configured implements Tool {
     ToolRunner.printGenericCommandUsage(out);
     out.println("Usage: gridmix [-generate <MiB>] [-users URI] [-Dname=value ...] <iopath> <trace>");
     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("   General parameters:");
     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;
 
+import java.io.DataOutputStream;
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Formatter;
 import java.util.List;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.Delayed;
 import java.util.concurrent.TimeUnit;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
 import java.security.PrivilegedExceptionAction;
 
 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.io.DataInputBuffer;
 import org.apache.hadoop.io.RawComparator;
 import org.apache.hadoop.io.WritableComparator;
 import org.apache.hadoop.io.WritableUtils;
 import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.JobTracker;
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.JobContext;
@@ -53,16 +56,17 @@ import org.apache.commons.logging.LogFactory;
  */
 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);
 
   private static final ThreadLocal<Formatter> nameFormat =
     new ThreadLocal<Formatter>() {
       @Override
       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);
       }
     };
@@ -80,6 +84,14 @@ abstract class GridmixJob implements Callable<Job>, Delayed {
       "gridmix.job-submission.use-queue-in-trace";
   protected static final String GRIDMIX_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) {
     if (queue != null)
@@ -93,22 +105,56 @@ abstract class GridmixJob implements Callable<Job>, Delayed {
     this.jobdesc = jobdesc;
     this.seq = seq;
 
-    ((StringBuilder)nameFormat.get().out()).setLength(JOBNAME.length());
+    ((StringBuilder)nameFormat.get().out()).setLength(JOB_NAME_PREFIX.length());
     try {
       job = this.ugi.doAs(new PrivilegedExceptionAction<Job>() {
         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().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)) {
             setJobQueue(ret, jobdesc.getQueueName());
           } else {
             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;
         }
       });
@@ -120,6 +166,185 @@ abstract class GridmixJob implements Callable<Job>, Delayed {
         submissionMillis, TimeUnit.MILLISECONDS);
     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(
     final Configuration conf, long submissionMillis, final String name)
@@ -289,13 +514,18 @@ abstract class GridmixJob implements Callable<Job>, Delayed {
         TaskAttemptContext job) throws IOException {
 
       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>() {
         @Override
         public void write(K ignored, GridmixRecord value)
             throws IOException {
-          value.writeRandom(fileOut, value.getSize());
+          // Let the Gridmix record fill itself.
+          value.write(fileOut);
         }
         @Override
         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.WritableUtils;
 import org.apache.hadoop.io.WritableComparator;
+import org.apache.hadoop.tools.rumen.ResourceUsageMetrics;
 
 class GridmixKey extends GridmixRecord {
   static final byte REDUCE_SPEC = 0;
@@ -115,6 +116,22 @@ class GridmixKey extends GridmixRecord {
     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() {
     return type;
   }
@@ -195,18 +212,35 @@ class GridmixKey extends GridmixRecord {
     long rec_in;
     long rec_out;
     long bytes_out;
+    private ResourceUsageMetrics metrics = null;
+    private int sizeOfResourceUsageMetrics = 0;
     public Spec() { }
 
     public void set(Spec other) {
       rec_in = other.rec_in;
       bytes_out = other.bytes_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() {
       return WritableUtils.getVIntSize(rec_in) +
              WritableUtils.getVIntSize(rec_out) +
-             WritableUtils.getVIntSize(bytes_out);
+             WritableUtils.getVIntSize(bytes_out) +
+             WritableUtils.getVIntSize(sizeOfResourceUsageMetrics) +
+             sizeOfResourceUsageMetrics;
     }
 
     @Override
@@ -214,6 +248,11 @@ class GridmixKey extends GridmixRecord {
       rec_in = WritableUtils.readVLong(in);
       rec_out = WritableUtils.readVLong(in);
       bytes_out = WritableUtils.readVLong(in);
+      sizeOfResourceUsageMetrics =  WritableUtils.readVInt(in);
+      if (sizeOfResourceUsageMetrics > 0) {
+        metrics = new ResourceUsageMetrics();
+        metrics.readFields(in);
+      }
     }
 
     @Override
@@ -221,6 +260,10 @@ class GridmixKey extends GridmixRecord {
       WritableUtils.writeVLong(out, rec_in);
       WritableUtils.writeVLong(out, rec_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.WritableComparator;
 import org.apache.hadoop.io.WritableUtils;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 
 class GridmixRecord implements WritableComparable<GridmixRecord> {
 
@@ -39,6 +40,10 @@ class GridmixRecord implements WritableComparable<GridmixRecord> {
   private final DataOutputBuffer dob =
     new DataOutputBuffer(Long.SIZE / Byte.SIZE);
   private byte[] literal = dob.getData();
+  private boolean compressible = false;
+  private float compressionRatio = 
+    CompressionEmulationUtil.DEFAULT_COMPRESSION_RATIO;
+  private RandomTextDataGenerator rtg = null;
 
   GridmixRecord() {
     this(1, 0L);
@@ -57,6 +62,19 @@ class GridmixRecord implements WritableComparable<GridmixRecord> {
     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) {
     this.size = Math.max(1, size);
     try {
@@ -79,6 +97,39 @@ class GridmixRecord implements WritableComparable<GridmixRecord> {
     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 {
     long tmp = seed;
     out.writeLong(tmp);
@@ -120,8 +171,13 @@ class GridmixRecord implements WritableComparable<GridmixRecord> {
     WritableUtils.writeVInt(out, size);
     final int payload = size - WritableUtils.getVIntSize(size);
     if (payload > Long.SIZE / Byte.SIZE) {
-      writeRandom(out, payload);
+      if (compressible) {
+        writeRandomText(out, payload);
+      } else {
+        writeRandom(out, payload);
+      }
     } else if (payload > 0) {
+      //TODO What is compressible is turned on? LOG is a bad idea!
       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.Map.Entry;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.FileStatus;
 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.LogFactory;
@@ -42,6 +45,7 @@ class InputStriper {
   long currentStart;
   FileStatus current;
   final List<FileStatus> files = new ArrayList<FileStatus>();
+  final Configuration conf = new Configuration();
 
   /**
    * @param inputDir Pool from which files are requested.
@@ -91,7 +95,15 @@ class InputStriper {
       }
       currentStart += 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());
         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;
 
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.ClusterStatus;
 import org.apache.hadoop.mapred.JobClient;
 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.tools.rumen.JobStory;
 
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.Random;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
 public enum JobCreator {
 
   LOADJOB {
     @Override
     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);
-    }},
+    }
+
+    @Override
+    public boolean canEmulateDistCacheLoad() {
+      return true;
+    }
+  },
 
   SLEEPJOB {
     private String[] hosts;
@@ -72,12 +83,30 @@ public enum JobCreator {
       }
       return new SleepJob(conf, submissionMillis, jobdesc, outRoot, ugi, seq,
           numLocations, hosts);
-    }};
+    }
+
+    @Override
+    public boolean canEmulateDistCacheLoad() {
+      return false;
+    }
+  };
 
   public static final String GRIDMIX_JOB_TYPE = "gridmix.job.type";
   public static final String SLEEPJOB_RANDOM_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(
     final Configuration conf, long submissionMillis, final JobStory jobdesc,
     Path outRoot, UserGroupInformation ugi, final int seq) throws IOException;
@@ -86,4 +115,21 @@ public enum JobCreator {
     Configuration conf, JobCreator 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 ReentrantLock lock = new ReentrantLock(true);
   protected final JobCreator jobCreator;
+  protected int numJobsInTrace = 0;
 
   /**
    * 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) {
       super(info.getInputBytes(), info.getInputRecords(),
             info.getOutputBytes(), info.getOutputRecords(),
-            info.getTaskMemory());
+            info.getTaskMemory(), info.getResourceUsageMetrics());
     }
     public long 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() ;
 
+  //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 {
-    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) {
         @Override
         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.
    * TODO: Cleaner solution for this problem
    * @param 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);
   }
 

+ 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());
       } catch(Exception e) {
         //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());
       } finally {
         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.fs.Path;
 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.InputSplit;
 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.Reducer;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskInputOutputContext;
 import org.apache.hadoop.mapreduce.TaskType;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.util.ResourceCalculatorPlugin;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.tools.rumen.JobStory;
+import org.apache.hadoop.tools.rumen.ResourceUsageMetrics;
 import org.apache.hadoop.tools.rumen.TaskInfo;
 
 import java.io.IOException;
@@ -83,6 +89,106 @@ class LoadJob extends GridmixJob {
     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
       extends Mapper<NullWritable,GridmixRecord,GridmixKey,GridmixRecord> {
 
@@ -95,6 +201,9 @@ class LoadJob extends GridmixJob {
     private final GridmixKey key = new GridmixKey();
     private final GridmixRecord val = new GridmixRecord();
 
+    private ResourceUsageMatcherRunner matcher = null;
+    private StatusReporter reporter = null;
+    
     @Override
     protected void setup(Context ctxt)
         throws IOException, InterruptedException {
@@ -104,6 +213,20 @@ class LoadJob extends GridmixJob {
       final long[] reduceBytes = split.getOutputBytes();
       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;
       final int nReduces = ctxt.getNumReduceTasks();
       if (nReduces > 0) {
@@ -114,17 +237,30 @@ class LoadJob extends GridmixJob {
           if (i == id) {
             spec.bytes_out = split.getReduceBytes(idx);
             spec.rec_out = split.getReduceRecords(idx);
+            spec.setResourceUsageSpecification(
+                   split.getReduceResourceUsageMetrics(idx));
             ++idx;
             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(
-              new AvgRecordFactory(reduceBytes[i], reduceRecords[i], conf),
+              new AvgRecordFactory(mapOutputBytes, reduceRecords[i], conf, 
+                                   5*1024),
               i, reduceRecords[i], spec, conf));
           totalRecords += reduceRecords[i];
         }
       } 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];
       }
       final long splitRecords = split.getInputRecords();
@@ -134,6 +270,13 @@ class LoadJob extends GridmixJob {
         : splitRecords;
       ratio = totalRecords / (1.0 * inputRecords);
       acc = 0.0;
+      
+      matcher = new ResourceUsageMatcherRunner(ctxt, 
+                      split.getMapResourceUsageMetrics());
+      
+      // start the status reporter thread
+      reporter = new StatusReporter(ctxt);
+      reporter.start();
     }
 
     @Override
@@ -151,6 +294,13 @@ class LoadJob extends GridmixJob {
         }
         context.write(key, val);
         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)) {
           context.write(key, val);
           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 RecordFactory factory;
 
+    private ResourceUsageMatcherRunner matcher = null;
+    private StatusReporter reporter = null;
+    
     @Override
     protected void setup(Context context)
         throws IOException, InterruptedException {
@@ -187,20 +350,48 @@ class LoadJob extends GridmixJob {
       long outBytes = 0L;
       long outRecords = 0L;
       long inRecords = 0L;
+      ResourceUsageMetrics metrics = new ResourceUsageMetrics();
       for (GridmixRecord ignored : context.getValues()) {
         final GridmixKey spec = context.getCurrentKey();
         inRecords += spec.getReduceInputRecords();
         outBytes += spec.getReduceOutputBytes();
         outRecords += spec.getReduceOutputRecords();
+        if (spec.getReduceResourceUsageMetrics() != null) {
+          metrics = spec.getReduceResourceUsageMetrics();
+        }
       }
       if (0 == outRecords && inRecords > 0) {
         LOG.info("Spec output bytes w/o records. Using input record count");
         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 =
-        new AvgRecordFactory(outBytes, outRecords, context.getConfiguration());
+        new AvgRecordFactory(outBytes, outRecords, 
+                             context.getConfiguration(), 5*1024);
       ratio = outRecords / (1.0 * inRecords);
       acc = 0.0;
+      
+      matcher = new ResourceUsageMatcherRunner(context, metrics);
+      
+      // start the status reporter thread
+      reporter = new StatusReporter(context);
+      reporter.start();
     }
     @Override
     protected void reduce(GridmixKey key, Iterable<GridmixRecord> values,
@@ -210,6 +401,13 @@ class LoadJob extends GridmixJob {
         while (acc >= 1.0 && factory.next(null, val)) {
           context.write(NullWritable.get(), val);
           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)) {
         context.write(NullWritable.get(), val);
         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 long[] specBytes = new long[nSpec];
       final long[] specRecords = new long[nSpec];
+      final ResourceUsageMetrics[] metrics = new ResourceUsageMetrics[nSpec];
       for (int j = 0; j < nSpec; ++j) {
         final TaskInfo info =
           jobdesc.getTaskInfo(TaskType.REDUCE, i + j * maps);
         specBytes[j] = info.getOutputBytes();
         specRecords[j] = info.getOutputRecords();
+        metrics[j] = info.getResourceUsageMetrics();
         if (LOG.isDebugEnabled()) {
           LOG.debug(String.format("SPEC(%d) %d -> %d %d %d", id(), i,
               i + j * maps, info.getOutputRecords(), info.getOutputBytes()));
@@ -326,7 +533,8 @@ class LoadJob extends GridmixJob {
               info.getInputBytes(), 3), maps, i,
             info.getInputBytes(), info.getInputRecords(),
             info.getOutputBytes(), info.getOutputRecords(),
-            reduceByteRatio, reduceRecordRatio, specBytes, specRecords));
+            reduceByteRatio, reduceRecordRatio, specBytes, specRecords,
+            info.getResourceUsageMetrics(), metrics));
     }
     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 org.apache.hadoop.io.WritableUtils;
+import org.apache.hadoop.tools.rumen.ResourceUsageMetrics;
 
 class LoadSplit extends CombineFileSplit {
   private int id;
@@ -39,6 +40,9 @@ class LoadSplit extends CombineFileSplit {
   private long[] reduceOutputBytes = new long[0];
   private long[] reduceOutputRecords = new long[0];
 
+  private ResourceUsageMetrics mapMetrics;
+  private ResourceUsageMetrics[] reduceMetrics;
+
   LoadSplit() {
     super();
   }
@@ -46,7 +50,9 @@ class LoadSplit extends CombineFileSplit {
   public LoadSplit(CombineFileSplit cfsplit, int maps, int id,
       long inputBytes, long inputRecords, long outputBytes,
       long outputRecords, double[] reduceBytes, double[] reduceRecords,
-      long[] reduceOutputBytes, long[] reduceOutputRecords)
+      long[] reduceOutputBytes, long[] reduceOutputRecords,
+      ResourceUsageMetrics metrics,
+      ResourceUsageMetrics[] rMetrics)
       throws IOException {
     super(cfsplit);
     this.id = id;
@@ -60,6 +66,8 @@ class LoadSplit extends CombineFileSplit {
     nSpec = reduceOutputBytes.length;
     this.reduceOutputBytes = reduceOutputBytes;
     this.reduceOutputRecords = reduceOutputRecords;
+    this.mapMetrics = metrics;
+    this.reduceMetrics = rMetrics;
   }
 
   public int getId() {
@@ -97,6 +105,15 @@ class LoadSplit extends CombineFileSplit {
   public long getReduceRecords(int i) {
     return reduceOutputRecords[i];
   }
+  
+  public ResourceUsageMetrics getMapResourceUsageMetrics() {
+    return mapMetrics;
+  }
+  
+  public ResourceUsageMetrics getReduceResourceUsageMetrics(int i) {
+    return reduceMetrics[i];
+  }
+  
   @Override
   public void write(DataOutput out) throws IOException {
     super.write(out);
@@ -116,6 +133,12 @@ class LoadSplit extends CombineFileSplit {
       WritableUtils.writeVLong(out, reduceOutputBytes[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
   public void readFields(DataInput in) throws IOException {
@@ -144,5 +167,13 @@ class LoadSplit extends CombineFileSplit {
       reduceOutputBytes[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 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(
       URI userUri, Configuration conf) throws IOException {
@@ -54,64 +60,78 @@ public class RoundRobinUserResolver implements UserResolver {
     final Path userloc = new Path(userUri.toString());
     final Text rawUgi = new Text();
     final FileSystem fs = userloc.getFileSystem(conf);
-    final ArrayList<UserGroupInformation> ret = new ArrayList();
+    final ArrayList<UserGroupInformation> ugiList =
+        new ArrayList<UserGroupInformation>();
 
     LineReader in = null;
     try {
-      final ArrayList<String> groups = new ArrayList();
       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(",");
-        if (e <= 0) {
+        if (rawUgi.getLength() == 0 || e == 0) {
           throw new IOException("Missing username: " + rawUgi);
         }
+        if (e == -1) {
+          e = rawUgi.getLength();
+        }
         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 {
       if (in != null) {
         in.close();
       }
     }
-    return ret;
+    return ugiList;
   }
 
   @Override
   public synchronized boolean setTargetUsers(URI userloc, Configuration conf)
       throws IOException {
+    uidx = 0;
     users = parseUserList(userloc, conf);
     if (users.size() == 0) {
-      throw new IOException("Empty user list");
+      throw new IOException(buildEmptyUsersErrorMsg(userloc));
     }
-    usercache.keySet().retainAll(users);
+    usercache.clear();
     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
   public synchronized UserGroupInformation getTargetUgi(
       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);
   }
 
+  @Override
+  protected boolean canEmulateCompression() {
+    return false;
+  }
+  
   @Override
   public Job call()
     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 {
           jobCompleted.await(jtPollingInterval, TimeUnit.MILLISECONDS);
         } 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;
         } finally {
           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;
 
-  public SubmitterUserResolver() {
+  public SubmitterUserResolver() throws IOException {
     LOG.info(" Current user resolver is SubmitterUserResolver ");
+    ugi = UserGroupInformation.getLoginUser();
   }
 
   public synchronized boolean setTargetUsers(URI userdesc, Configuration conf)
       throws IOException {
-    ugi = UserGroupInformation.getLoginUser();
     return false;
   }
 
@@ -47,4 +47,13 @@ public class SubmitterUserResolver implements UserResolver {
     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.net.URI;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-
 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.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.
  */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
 public interface UserResolver {
 
   /**
    * Configure the user map given the URI and configuration. The resolver's
    * 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}
-   * 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.
    * @return true if the resource provided was used in building the list of
    * target users
@@ -55,4 +53,13 @@ public interface UserResolver {
    */
   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;
+  }
+}

BIN
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();
       r.setSeed(seed);
       id = seq.getAndIncrement();
-      name = String.format("MOCKJOB%05d", id);
+      name = String.format("MOCKJOB%06d", id);
       this.conf = conf;
       LOG.info(name + " (" + seed + ")");
       submitTime = timestamp.addAndGet(
@@ -209,9 +209,14 @@ public class DebugJobProducer implements JobStoryProducer {
 
    @Override
    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
@@ -285,7 +290,7 @@ public class DebugJobProducer implements JobStoryProducer {
 
     @Override
     public org.apache.hadoop.mapred.JobConf getJobConf() {
-      throw new UnsupportedOperationException();
+      return new JobConf(conf);
     }
 
     @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);
       final int min = WritableUtils.getVIntSize(in_rec)
                     + WritableUtils.getVIntSize(out_rec)
-                    + WritableUtils.getVIntSize(out_bytes);
+                    + WritableUtils.getVIntSize(out_bytes)
+                    + WritableUtils.getVIntSize(0);
       assertEquals(min + 2, a.fixedBytes()); // meta + vint min
       final int size = r.nextInt(1024) + a.fixedBytes() + 1;
       setSerialize(a, r.nextLong(), size, out);
@@ -207,7 +208,7 @@ public class TestGridmixRecord {
 
   @Test
   public void testKeySpec() throws Exception {
-    final int min = 5;
+    final int min = 6;
     final int max = 300;
     final GridmixKey a = 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.fs.ContentSummary;
 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.permission.FsPermission;
 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.TaskType;
 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.util.ToolRunner;
 import org.apache.log4j.Level;
@@ -41,13 +44,16 @@ import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
+import java.io.InputStream;
 import java.io.IOException;
+import java.text.DecimalFormat;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashMap;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.CountDownLatch;
 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_OUTPUT_BYTES;
@@ -103,17 +109,10 @@ public class TestGridmixSubmission {
         GridmixTestUtils.mrCluster.createJobConf());
       for (Job job : succeeded) {
         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 out = new Path("/gridmix").makeQualified(GridmixTestUtils.dfs);
           final ContentSummary generated = GridmixTestUtils.dfs.getContentSummary(in);
@@ -123,37 +122,55 @@ public class TestGridmixSubmission {
           FileStatus[] outstat = GridmixTestUtils.dfs.listStatus(out);
           assertEquals("Mismatched job count", NJOBS, outstat.length);
           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 {
-          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 nReds = spec.getNumberReduces();
 
         // TODO Blocked by MAPREDUCE-118
         if (true) return;
         // TODO
-        System.out.println(jobname + ": " + nMaps + "/" + nReds);
+        System.out.println(jobName + ": " + nMaps + "/" + nReds);
         final TaskReport[] mReports =
           client.getMapTaskReports(JobID.downgrade(job.getJobID()));
         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,
           final TaskReport[] runTasks,
           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
   public void testReplaySubmit() throws Exception {
     policy = GridmixJobSubmissionPolicy.REPLAY;
     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 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
   public void testStressSubmit() throws Exception {
     policy = GridmixJobSubmissionPolicy.STRESS;
     System.out.println(" Stress started at " + System.currentTimeMillis());
-    doSubmission(false);
+    doSubmission(false, false);
     System.out.println(" Stress ended at " + System.currentTimeMillis());
   }
 
@@ -346,7 +474,7 @@ public class TestGridmixSubmission {
     policy = GridmixJobSubmissionPolicy.STRESS;
     System.out.println(
       " Stress with default q started at " + System.currentTimeMillis());
-    doSubmission(true);
+    doSubmission(true, false);
     System.out.println(
       " Stress with default q ended at " + System.currentTimeMillis());
   }
@@ -355,26 +483,39 @@ public class TestGridmixSubmission {
   public void testSerialSubmit() throws Exception {
     policy = GridmixJobSubmissionPolicy.SERIAL;
     System.out.println("Serial started at " + System.currentTimeMillis());
-    doSubmission(false);
+    doSubmission(false, false);
     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 out = GridmixTestUtils.DEST.makeQualified(GridmixTestUtils.dfs);
     final Path root = new Path("/user");
     Configuration conf = null;
+
     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);
       if (useDefaultQueue) {
         conf.setBoolean(GridmixJob.GRIDMIX_USE_QUEUE_IN_TRACE, false);
@@ -382,13 +523,13 @@ public class TestGridmixSubmission {
       } else {
         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) {
        e.printStackTrace();
      } 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 {
 
-  static Path userlist;
+  private static Path rootDir = null;
+  private static Configuration conf = null;
+  private static FileSystem fs = null;
 
   @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;
     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 {
       if (out != null) {
         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;
     try {
-      rslv.setTargetUsers(null, conf);
+      rslv.setTargetUsers(userRsrc, conf);
     } catch (IOException e) {
+      assertTrue("Exception message from RoundRobinUserResolver is wrong",
+          e.getMessage().equals(expectedErrorMsg));
       fail = true;
     }
     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("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("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
   public void testSubmitterResolver() throws Exception {
-    final Configuration conf = new Configuration();
     final UserResolver rslv = new SubmitterUserResolver();
-    rslv.setTargetUsers(null, conf);
+    assertFalse(rslv.needsTargetUsersList());
     UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
     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();
   }
 
+  
   /** Computes progress in this node. */
   private synchronized float getInternal() {
     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) {
     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
   limitations under the License.
 -->
-
 <!DOCTYPE document PUBLIC "-//APACHE//DTD Documentation V2.0//EN" "http://forrest.apache.org/dtd/document-v20.dtd">
-
 <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 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>

+ 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"/>
     <archives     label="Hadoop Archives" href="hadoop_archives.html"/>
     <gridmix       label="Gridmix"  href="gridmix.html"/>
+    <Rumen          label="Rumen"     href="rumen.html"/>
     <cap_scheduler  label="Capacity Scheduler" href="capacity_scheduler.html"/>
     <fair_scheduler    label="Fair Scheduler"  href="fair_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";
   
-  static final String MAPRED_JOB_MAP_MEMORY_MB_PROPERTY =
+  public static final String MAPRED_JOB_MAP_MEMORY_MB_PROPERTY =
       "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";
 
   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 =
       "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";
-  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";
 
   /* 

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

@@ -61,6 +61,10 @@ public interface Reporter extends Progressable {
       public InputSplit getInputSplit() throws UnsupportedOperationException {
         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() 
     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
       setProgressFlag();
     }
+    
+    public float getProgress() {
+      return taskProgress.getProgress();
+    };
+    
     public void progress() {
       // indicate that progress update needs to be sent
       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(String group, String name);
   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);
 }

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

@@ -93,6 +93,10 @@ public abstract class TaskInputOutputContext<KEYIN,VALUEIN,KEYOUT,VALUEOUT>
     reporter.progress();
   }
 
+  public float getProgress() {
+    return reporter.getProgress();
+  }
+
   @Override
   public void setStatus(String 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);
     }
     
+    @Override
+    public float getProgress() {
+      return outer.getProgress();
+    }
   }
 
   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, 
                                   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);
     if (fs.exists(outDir)) {
       fs.delete(outDir, true);
@@ -615,8 +625,7 @@ public class UtilsForTests {
     if (!fs.exists(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) {
       DataOutputStream file = fs.create(new Path(inDir, "part-" + i));
       file.writeBytes(input);

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

@@ -285,6 +285,10 @@ public class MapReduceTestUtil {
       }
       public void progress() {
       }
+      @Override
+      public float getProgress() {
+        return 0;
+      }
       public Counter getCounter(Enum<?> 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;
 
 import java.io.BufferedInputStream;
+import java.io.DataOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
 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.TaskID;
 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.Tool;
 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
@@ -426,7 +582,7 @@ public class TestRumenJobTraces {
 
       // Test if the JobHistoryParserFactory can detect the parser correctly
       parser = JobHistoryParserFactory.getParser(ris);
-        
+
       HistoryEvent e;
       while ((e = parser.nextEvent()) != null) {
         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
   public void testTopologyBuilder() throws Exception {
     final TopologyBuilder subject = new TopologyBuilder();

BIN
src/test/tools/data/rumen/small-trace-test/counters-test-trace.json.gz


BIN
src/test/tools/data/rumen/small-trace-test/dispatch-trace-output.json.gz


BIN
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>
+<!--
+Old mapred config properties
+-->
    <property>
       <name>mapred.job.queue.name</name><value>TheQueue</value>
    </property>
    <property>
-      <name>mapreduce.job.name</name><value>job_0001</value>
+      <name>mapred.job.name</name><value>MyMRJob</value>
    </property>
    <property>
       <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",
   "jobName" : null,
   "jobID" : "job_200904211745_0002",
+   "jobProperties" : {
+    "mapred.child.java.opts" : "-server -Xmx640m -Djava.net.preferIPv4Stack=true"
+  },
   "mapTasks" : [ {
     "startTime" : 1240336753705,
     "attempts" : [ {

BIN
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.
    * 
-   * @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);
   
   /**
    * 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);
 

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

@@ -72,7 +72,7 @@ public class DeskewedJobTraceReader implements Closeable {
    * 
    * @param reader
    *          the {@link JobTraceReader} that's being protected
-   * @param skewBufferSize
+   * @param skewBufferLength
    *          [the number of late jobs that can preced a later out-of-order
    *          earlier job
    * @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;
       }
     }, 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) {
@@ -1594,6 +1626,8 @@ public class HadoopLogsAnalyzer extends Configured implements Tool {
       jobBeingTraced.setJobMapMB(jobconf.jobMapMB);
       jobBeingTraced.setJobReduceMB(jobconf.jobReduceMB);
 
+      jobBeingTraced.setJobProperties(jobconf.properties);
+      
       jobconf = null;
 
       finalizeJob();

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

@@ -74,6 +74,8 @@ public class JobBuilder {
   private static final Pattern heapPattern =
       Pattern.compile("-Xmx([0-9]+[kKmMgGtT])");
 
+  private Properties jobConfigurationParameters = null;
+
   public JobBuilder(String jobID) {
     this.jobID = jobID;
   }
@@ -142,7 +144,7 @@ public class JobBuilder {
           "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) {
       String result = conf.getProperty(name);
 
@@ -206,6 +208,7 @@ public class JobBuilder {
           "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
         .getCandidates(), "default"));
     result.setJobName(extract(conf, JobConfPropertyNames.JOB_NAMES
@@ -217,6 +220,8 @@ public class JobBuilder {
         JobConfPropertyNames.MAP_JAVA_OPTS_S.getCandidates()));
     maybeSetJobReduceMB(extractMegabytes(conf,
         JobConfPropertyNames.REDUCE_JAVA_OPTS_S.getCandidates()));
+        
+    this.jobConfigurationParameters = conf;
   }
 
   /**
@@ -226,9 +231,12 @@ public class JobBuilder {
    * @return Parsed {@link LoggedJob} object.
    */
   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;
 
+    // set the conf
+    result.setJobProperties(jobConfigurationParameters);
+    
     // initialize all the per-job statistics gathering places
     Histogram[] successfulMapAttemptTimes =
         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;
 
-import java.io.BufferedInputStream;
 import java.io.IOException;
 import java.io.InputStream;
-import java.util.HashSet;
-import java.util.List;
 import java.util.Properties;
-import java.util.Set;
 
 import javax.xml.parsers.DocumentBuilder;
 import javax.xml.parsers.DocumentBuilderFactory;
@@ -38,22 +34,11 @@ import org.xml.sax.SAXException;
 
 /**
  * {@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
  * postponed for a future release]
  */
 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
@@ -66,7 +51,7 @@ public class JobConfigurationParser {
    *         configuration xml.
    * @throws IOException
    */
-  Properties parse(InputStream input) throws IOException {
+  static Properties parse(InputStream input) throws IOException {
     Properties result = new Properties();
 
     try {
@@ -117,7 +102,7 @@ public class JobConfigurationParser {
           }
         }
 
-        if (interested.contains(attr) && value != null) {
+        if (attr != null && value != null) {
           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.");
   }
 
-  enum VersionDetector {
+  public enum VersionDetector {
     Hadoop20() {
 
       @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.List;
+import java.util.Map;
+import java.util.Properties;
 import java.util.Set;
 import java.util.TreeSet;
 
@@ -92,6 +94,8 @@ public class LoggedJob implements DeepCompare {
   double[] mapperTriesToSucceed;
   double failedMapperFraction; // !!!!!
 
+  private Properties jobProperties = new Properties();
+  
   LoggedJob() {
 
   }
@@ -102,6 +106,20 @@ public class LoggedJob implements DeepCompare {
     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) {
     submitTime += 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)
       throws DeepInequalityException {
     if (!(comparand instanceof LoggedJob)) {
@@ -600,5 +647,9 @@ public class LoggedJob implements DeepCompare {
     compare1(clusterReduceMB, other.clusterReduceMB, loc, "clusterReduceMB");
     compare1(jobMapMB, other.jobMapMB, loc, "jobMapMB");
     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;
 
+  // Initialize to default object for backward compatibility
+  ResourceUsageMetrics metrics = new ResourceUsageMetrics();
+  
   LoggedTaskAttempt() {
     super();
   }
@@ -349,8 +352,50 @@ public class LoggedTaskAttempt implements DeepCompare {
         attempt.spilledRecords = val;
       }
     }, 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) {
     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
- * {@MachineNode}, or a {@link RackNode}, etc.
+ * {@link MachineNode}, or a {@link RackNode}, etc.
  */
 public class Node implements Comparable<Node> {
   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;
 
+import java.util.Properties;
 import java.util.regex.Pattern;
 import java.util.regex.Matcher;
 
@@ -55,6 +56,8 @@ class ParsedConfigFile {
   final String jobID;
 
   final boolean valid;
+  
+  final Properties properties = new Properties();
 
   private int maybeGetIntValue(String propName, String attr, String value,
       int oldValue) {
@@ -143,6 +146,8 @@ class ParsedConfigFile {
                 "true".equals(((Text) field.getFirstChild()).getData());
           }
         }
+        
+        properties.setProperty(attr, value);
 
         if ("mapred.child.java.opts".equals(attr) && value != null) {
           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;
 
-import org.apache.hadoop.mapred.TaskStatus;
 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
    */

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

@@ -23,14 +23,22 @@ public class TaskInfo {
   private final long bytesOut;
   private final int recsOut;
   private final long maxMemory;
+  private final ResourceUsageMetrics metrics;
 
   public TaskInfo(long bytesIn, int recsIn, long bytesOut, int recsOut,
       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.recsIn = recsIn;
     this.bytesOut = bytesOut;
     this.recsOut = recsOut;
     this.maxMemory = maxMemory;
+    this.metrics = metrics;
   }
 
   /**
@@ -70,4 +78,10 @@ public class TaskInfo {
     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.util.ArrayList;
 import java.util.Arrays;
+import java.util.Comparator;
 import java.util.LinkedList;
 import java.util.List;
 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.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.mapred.JobHistory;
 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;
 
   TopologyBuilder topologyBuilder = new TopologyBuilder();
-  JobConfigurationParser jobConfParser;
   Outputter<LoggedJob> traceWriter;
   Outputter<LoggedNetworkTopology> topologyWriter;
 
@@ -67,48 +68,136 @@ public class TraceBuilder extends Configured implements Tool {
         IOException, ClassNotFoundException {
       int switchTop = 0;
 
+      // to determine if the input paths should be recursively scanned or not
+      boolean doRecursiveTraversal = false;
+
       while (args[switchTop].startsWith("-")) {
         if (args[switchTop].equalsIgnoreCase("-demuxer")) {
           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]);
       topologyOutput = new Path(args[1 + switchTop]);
 
       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 {
-          inputs.add(thisPath);
+          inputPaths.add(thisPath);
         }
       }
+
+      return inputPaths;
     }
   }
 
@@ -169,25 +258,11 @@ public class TraceBuilder extends Configured implements Tool {
     return jobId != null;
   }
 
-  private void addInterestedProperties(List<String> interestedProperties,
-      String[] names) {
-    for (String name : names) {
-      interestedProperties.add(name);
-    }
-  }
 
   @SuppressWarnings("unchecked")
   @Override
   public int run(String[] args) throws Exception {
     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.init(options.traceOutput, getConf());
     topologyWriter = new DefaultOutputter<LoggedNetworkTopology>();
@@ -232,7 +307,7 @@ public class TraceBuilder extends Configured implements Tool {
               }
 
               if (isJobConfXml(filePair.first(), ris)) {
-                processJobConf(jobConfParser.parse(ris.rewind()), jobBuilder);
+            	processJobConf(JobConfigurationParser.parse(ris.rewind()), jobBuilder);
               } else {
                 parser = JobHistoryParserFactory.getParser(ris);
                 if (parser == null) {

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

@@ -120,8 +120,20 @@ public class ZombieJob implements JobStory {
   @Override
   public synchronized JobConf getJobConf() {
     if (jobConf == null) {
-      // TODO : add more to 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.setUser(getUser());
       jobConf.setNumMapTasks(getNumberMaps());
@@ -622,6 +634,7 @@ public class ZombieJob implements JobStory {
     long outputBytes = -1;
     long outputRecords = -1;
     long heapMegabytes = -1;
+    ResourceUsageMetrics metrics = new ResourceUsageMetrics();
 
     Values type = loggedTask.getTaskType();
     if ((type != Values.MAP) && (type != Values.REDUCE)) {
@@ -656,12 +669,15 @@ public class ZombieJob implements JobStory {
             (job.getJobReduceMB() > 0) ? job.getJobReduceMB() : job
                 .getHeapMegabytes();
       }
+      // set the resource usage metrics
+      metrics = attempt.getResourceUsageMetrics();
       break;
     }
 
     TaskInfo taskInfo =
         new TaskInfo(inputBytes, (int) inputRecords, outputBytes,
-            (int) outputRecords, (int) heapMegabytes);
+            (int) outputRecords, (int) heapMegabytes,
+            metrics);
     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;