浏览代码

commit ab3374db39ac9f87def8db69b6e1c55609a310f2
Author: Chris Douglas <cdouglas@apache.org>
Date: Tue Dec 15 20:19:19 2009 -0800

MAPREDUCE:1124 from https://issues.apache.org/jira/secure/attachment/12427971/M1124-y20-1.patch

+++ b/YAHOO-CHANGES.txt
+ MAPREDUCE-1124. Import Gridmix3 and Rumen. (cdouglas)
+


git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-0.20-security-patches@1077079 13f79535-47bb-0310-9956-ffa450edef68

Owen O'Malley 14 年之前
父节点
当前提交
75864bfe3c
共有 90 个文件被更改,包括 29679 次插入3 次删除
  1. 3 1
      build.xml
  2. 1 0
      src/contrib/build-contrib.xml
  3. 1 0
      src/contrib/build.xml
  4. 22 0
      src/contrib/gridmix/README
  5. 23 0
      src/contrib/gridmix/build.xml
  6. 97 0
      src/contrib/gridmix/ivy.xml
  7. 22 0
      src/contrib/gridmix/ivy/libraries.properties
  8. 91 0
      src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/AvgRecordFactory.java
  9. 196 0
      src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/CombineFileSplit.java
  10. 369 0
      src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/FilePool.java
  11. 104 0
      src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/FileQueue.java
  12. 293 0
      src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/GenerateData.java
  13. 351 0
      src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/Gridmix.java
  14. 523 0
      src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/GridmixJob.java
  15. 258 0
      src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/GridmixKey.java
  16. 215 0
      src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/GridmixRecord.java
  17. 147 0
      src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/GridmixSplit.java
  18. 126 0
      src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/InputStriper.java
  19. 110 0
      src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/IntermediateRecordFactory.java
  20. 272 0
      src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/JobFactory.java
  21. 243 0
      src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/JobMonitor.java
  22. 177 0
      src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/JobSubmitter.java
  23. 85 0
      src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/ReadRecordFactory.java
  24. 40 0
      src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/RecordFactory.java
  25. 277 0
      src/contrib/gridmix/src/test/org/apache/hadoop/mapred/gridmix/DebugJobFactory.java
  26. 188 0
      src/contrib/gridmix/src/test/org/apache/hadoop/mapred/gridmix/TestFilePool.java
  27. 142 0
      src/contrib/gridmix/src/test/org/apache/hadoop/mapred/gridmix/TestFileQueue.java
  28. 277 0
      src/contrib/gridmix/src/test/org/apache/hadoop/mapred/gridmix/TestGridmixRecord.java
  29. 322 0
      src/contrib/gridmix/src/test/org/apache/hadoop/mapred/gridmix/TestGridmixSubmission.java
  30. 79 0
      src/contrib/gridmix/src/test/org/apache/hadoop/mapred/gridmix/TestRecordFactory.java
  31. 1 1
      src/mapred/org/apache/hadoop/mapred/Task.java
  32. 1 1
      src/mapred/org/apache/hadoop/mapred/TaskStatus.java
  33. 54 0
      src/test/org/apache/hadoop/tools/rumen/HistogramRawTestData.java
  34. 155 0
      src/test/org/apache/hadoop/tools/rumen/TestHistograms.java
  35. 123 0
      src/test/org/apache/hadoop/tools/rumen/TestPiecewiseLinearInterpolation.java
  36. 125 0
      src/test/org/apache/hadoop/tools/rumen/TestRumenJobTraces.java
  37. 338 0
      src/test/org/apache/hadoop/tools/rumen/TestZombieJob.java
  38. 15 0
      src/test/tools/data/rumen/histogram-tests/gold-minimal.json
  39. 15 0
      src/test/tools/data/rumen/histogram-tests/gold-one-value-many-repeats.json
  40. 15 0
      src/test/tools/data/rumen/histogram-tests/gold-only-one-value.json
  41. 15 0
      src/test/tools/data/rumen/histogram-tests/gold-three-values.json
  42. 17 0
      src/test/tools/data/rumen/histogram-tests/input-minimal.json
  43. 76 0
      src/test/tools/data/rumen/histogram-tests/input-one-value-many-repeats.json
  44. 13 0
      src/test/tools/data/rumen/histogram-tests/input-only-one-value.json
  45. 15 0
      src/test/tools/data/rumen/histogram-tests/input-three-values.json
  46. 1693 0
      src/test/tools/data/rumen/small-trace-test/job-tracker-logs-topology-output
  47. 二进制
      src/test/tools/data/rumen/small-trace-test/job-tracker-logs-trace-output.gz
  48. 二进制
      src/test/tools/data/rumen/small-trace-test/sample-job-tracker-logs.gz
  49. 110 0
      src/test/tools/data/rumen/small-trace-test/truncated-job-tracker-log
  50. 343 0
      src/test/tools/data/rumen/small-trace-test/truncated-topology-output
  51. 1407 0
      src/test/tools/data/rumen/small-trace-test/truncated-trace-output
  52. 1693 0
      src/test/tools/data/rumen/zombie/input-topology.json
  53. 11364 0
      src/test/tools/data/rumen/zombie/input-trace.json
  54. 185 0
      src/tools/org/apache/hadoop/tools/rumen/AbstractClusterStory.java
  55. 68 0
      src/tools/org/apache/hadoop/tools/rumen/CDFPiecewiseLinearRandomGenerator.java
  56. 89 0
      src/tools/org/apache/hadoop/tools/rumen/CDFRandomGenerator.java
  57. 81 0
      src/tools/org/apache/hadoop/tools/rumen/ClusterStory.java
  58. 81 0
      src/tools/org/apache/hadoop/tools/rumen/ClusterTopologyReader.java
  59. 45 0
      src/tools/org/apache/hadoop/tools/rumen/DeepCompare.java
  60. 61 0
      src/tools/org/apache/hadoop/tools/rumen/DeepInequalityException.java
  61. 1814 0
      src/tools/org/apache/hadoop/tools/rumen/HadoopLogsAnalyzer.java
  62. 164 0
      src/tools/org/apache/hadoop/tools/rumen/Histogram.java
  63. 118 0
      src/tools/org/apache/hadoop/tools/rumen/JobStory.java
  64. 33 0
      src/tools/org/apache/hadoop/tools/rumen/JobStoryProducer.java
  65. 51 0
      src/tools/org/apache/hadoop/tools/rumen/JobTraceReader.java
  66. 116 0
      src/tools/org/apache/hadoop/tools/rumen/JsonObjectMapperParser.java
  67. 73 0
      src/tools/org/apache/hadoop/tools/rumen/LogRecordType.java
  68. 144 0
      src/tools/org/apache/hadoop/tools/rumen/LoggedDiscreteCDF.java
  69. 586 0
      src/tools/org/apache/hadoop/tools/rumen/LoggedJob.java
  70. 98 0
      src/tools/org/apache/hadoop/tools/rumen/LoggedLocation.java
  71. 168 0
      src/tools/org/apache/hadoop/tools/rumen/LoggedNetworkTopology.java
  72. 101 0
      src/tools/org/apache/hadoop/tools/rumen/LoggedSingleRelativeRanking.java
  73. 266 0
      src/tools/org/apache/hadoop/tools/rumen/LoggedTask.java
  74. 344 0
      src/tools/org/apache/hadoop/tools/rumen/LoggedTaskAttempt.java
  75. 205 0
      src/tools/org/apache/hadoop/tools/rumen/MachineNode.java
  76. 48 0
      src/tools/org/apache/hadoop/tools/rumen/MapTaskAttemptInfo.java
  77. 148 0
      src/tools/org/apache/hadoop/tools/rumen/Node.java
  78. 38 0
      src/tools/org/apache/hadoop/tools/rumen/Pair.java
  79. 197 0
      src/tools/org/apache/hadoop/tools/rumen/ParsedConfigFile.java
  80. 126 0
      src/tools/org/apache/hadoop/tools/rumen/ParsedHost.java
  81. 117 0
      src/tools/org/apache/hadoop/tools/rumen/ParsedLine.java
  82. 49 0
      src/tools/org/apache/hadoop/tools/rumen/Pre21JobHistoryConstants.java
  83. 48 0
      src/tools/org/apache/hadoop/tools/rumen/RackNode.java
  84. 71 0
      src/tools/org/apache/hadoop/tools/rumen/ReduceTaskAttemptInfo.java
  85. 64 0
      src/tools/org/apache/hadoop/tools/rumen/TaskAttemptInfo.java
  86. 73 0
      src/tools/org/apache/hadoop/tools/rumen/TaskInfo.java
  87. 57 0
      src/tools/org/apache/hadoop/tools/rumen/TreePath.java
  88. 149 0
      src/tools/org/apache/hadoop/tools/rumen/ZombieCluster.java
  89. 880 0
      src/tools/org/apache/hadoop/tools/rumen/ZombieJob.java
  90. 81 0
      src/tools/org/apache/hadoop/tools/rumen/ZombieJobProducer.java

+ 3 - 1
build.xml

@@ -109,6 +109,7 @@
   <property name="test.junit.printsummary" value="yes" />
   <property name="test.junit.haltonfailure" value="no" />
   <property name="test.junit.maxmemory" value="512m" />
+  <property name="test.tools.input.dir" value="${basedir}/src/test/tools/data"/>
 
   <property name="test.libhdfs.conf.dir" value="${c++.libhdfs.src}/tests/conf"/>
   <property name="test.libhdfs.dir" value="${test.build.dir}/libhdfs"/>
@@ -508,7 +509,7 @@
   	description="Compile core only">
   </target>
 
-  <target name="compile-contrib" depends="compile-core,compile-c++-libhdfs">
+  <target name="compile-contrib" depends="compile-core,tools-jar,compile-c++-libhdfs">
      <subant target="compile">
         <property name="version" value="${version}"/>
         <fileset file="${contrib.dir}/build.xml"/>
@@ -734,6 +735,7 @@
       dir="${basedir}" timeout="${test.timeout}"
       errorProperty="tests.failed" failureProperty="tests.failed">
       <sysproperty key="test.build.data" value="${test.build.data}"/>
+      <sysproperty key="test.tools.input.dir" value="${test.tools.input.dir}"/>
       <sysproperty key="test.cache.data" value="${test.cache.data}"/>    	
       <sysproperty key="test.debug.data" value="${test.debug.data}"/>
       <sysproperty key="hadoop.log.dir" value="${test.log.dir}"/>

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

@@ -80,6 +80,7 @@
   <!-- the normal classpath -->
   <path id="contrib-classpath">
     <pathelement location="${build.classes}"/>
+    <pathelement location="${hadoop.root}/build/tools"/>
     <fileset refid="lib.jars"/>
     <pathelement location="${hadoop.root}/build/classes"/>
     <fileset dir="${hadoop.root}/lib">

+ 1 - 0
src/contrib/build.xml

@@ -50,6 +50,7 @@
       <fileset dir="." includes="streaming/build.xml"/>
       <fileset dir="." includes="fairscheduler/build.xml"/>
       <fileset dir="." includes="capacity-scheduler/build.xml"/>
+      <fileset dir="." includes="gridmix/build.xml"/>
     </subant>
   </target>
   

+ 22 - 0
src/contrib/gridmix/README

@@ -0,0 +1,22 @@
+# 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.
+
+This project implements the third version of Gridmix, a benchmark for live
+clusters. Given a description of jobs (a "trace") annotated with information
+about I/O, memory, etc. a synthetic mix of jobs will be generated and submitted
+to the cluster.
+
+Documentation of usage and configuration properties in forrest is available in
+src/docs/src/documentation/content/xdocs/gridmix.xml

+ 23 - 0
src/contrib/gridmix/build.xml

@@ -0,0 +1,23 @@
+<?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.
+-->
+
+<project name="gridmix" default="jar">
+
+  <import file="../build-contrib.xml"/>
+
+</project>

+ 97 - 0
src/contrib/gridmix/ivy.xml

@@ -0,0 +1,97 @@
+<?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.
+-->
+<ivy-module version="1.0">
+  <info organisation="org.apache.hadoop" module="${ant.project.name}">
+    <license name="Apache 2.0"/>
+    <description>Rumen</description>
+  </info>
+  <configurations defaultconfmapping="default">
+    <!--these match the Maven configurations-->
+    <conf name="default" extends="master,runtime"/>
+    <conf name="master" description="contains the artifact but no dependencies"/>
+    <conf name="runtime" description="runtime but not the artifact" />
+
+    <conf name="common" visibility="private" extends="runtime"
+      description="artifacts needed to compile/test the application"/>
+  </configurations>
+
+  <publications>
+    <!--get the artifact from our module name-->
+    <artifact conf="master"/>
+  </publications>
+  <dependencies>
+    <dependency org="commons-logging"
+      name="commons-logging"
+      rev="${commons-logging.version}"
+      conf="common->default"/>
+    <dependency org="log4j"
+      name="log4j"
+      rev="${log4j.version}"
+      conf="common->master"/>
+    <dependency org="junit"
+      name="junit"
+      rev="${junit.version}"
+      conf="common->default"/>
+
+    <!-- necessary for Mini*Clusters -->
+    <dependency org="commons-httpclient"
+      name="commons-httpclient"
+      rev="${commons-httpclient.version}"
+      conf="common->master"/>
+    <dependency org="commons-codec"
+      name="commons-codec"
+      rev="${commons-codec.version}"
+      conf="common->default"/>
+    <dependency org="commons-net"
+      name="commons-net"
+      rev="${commons-net.version}"
+      conf="common->default"/>
+    <dependency org="org.mortbay.jetty"
+      name="jetty"
+      rev="${jetty.version}"
+      conf="common->master"/>
+    <dependency org="org.mortbay.jetty"
+      name="jetty-util"
+      rev="${jetty-util.version}"
+      conf="common->master"/>
+    <dependency org="org.mortbay.jetty"
+      name="jsp-api-2.1"
+      rev="${jetty.version}"
+      conf="common->master"/>
+    <dependency org="org.mortbay.jetty"
+      name="jsp-2.1"
+      rev="${jetty.version}"
+      conf="common->master"/>
+    <dependency org="org.mortbay.jetty"
+      name="servlet-api-2.5"
+      rev="${servlet-api-2.5.version}"
+      conf="common->master"/>
+    <dependency org="commons-cli"
+      name="commons-cli"
+      rev="${commons-cli.version}"
+      conf="common->default"/>
+    <dependency org="org.codehaus.jackson"
+      name="jackson-mapper-asl"
+      rev="${jackson.version}"
+      conf="common->default"/>
+    <dependency org="org.codehaus.jackson"
+      name="jackson-core-asl"
+      rev="${jackson.version}"
+      conf="common->default"/>
+  </dependencies>
+</ivy-module>

+ 22 - 0
src/contrib/gridmix/ivy/libraries.properties

@@ -0,0 +1,22 @@
+#   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.
+
+#This properties file lists the versions of the various artifacts used by streaming.
+#It drives ivy and the generation of a maven POM
+
+#Please list the dependencies name with version if they are different from the ones
+#listed in the global libraries.properties file (in alphabetical order)
+
+jackson.version=1.0.1

+ 91 - 0
src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/AvgRecordFactory.java

@@ -0,0 +1,91 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.mapred.gridmix;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * Given byte and record targets, emit roughly equal-sized records satisfying
+ * the contract.
+ */
+class AvgRecordFactory extends RecordFactory {
+
+  /**
+   * Percentage of record for key data.
+   */
+  public static final String GRIDMIX_KEY_FRC = "gridmix.key.fraction";
+
+
+  private final long targetBytes;
+  private final long targetRecords;
+  private final long step;
+  private final int avgrec;
+  private final int keyLen;
+  private long accBytes = 0L;
+  private long accRecords = 0L;
+
+  /**
+   * @param targetBytes Expected byte count.
+   * @param targetRecords Expected record count.
+   * @param conf Used to resolve edge cases @see #GRIDMIX_KEY_FRC
+   */
+  public AvgRecordFactory(long targetBytes, long targetRecords,
+      Configuration conf) {
+    this.targetBytes = targetBytes;
+    this.targetRecords = targetRecords <= 0 && this.targetBytes >= 0
+      ? Math.max(1,
+          this.targetBytes / conf.getInt("gridmix.missing.rec.size", 64 * 1024))
+      : targetRecords;
+    final long tmp = this.targetBytes / this.targetRecords;
+    step = this.targetBytes - this.targetRecords * tmp;
+    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))));
+  }
+
+  @Override
+  public boolean next(GridmixKey key, GridmixRecord val) throws IOException {
+    if (accBytes >= targetBytes) {
+      return false;
+    }
+    final int reclen = accRecords++ >= step ? avgrec - 1 : avgrec;
+    final int len = (int) Math.min(targetBytes - accBytes, reclen);
+    // len != reclen?
+    if (key != null) {
+      key.setSize(keyLen);
+      val.setSize(len - key.getSize());
+    } else {
+      val.setSize(len);
+    }
+    accBytes += len;
+    return true;
+  }
+
+  @Override
+  public float getProgress() throws IOException {
+    return Math.min(1.0f, accBytes / ((float)targetBytes));
+  }
+
+  @Override
+  public void close() throws IOException {
+    // noop
+  }
+
+}

+ 196 - 0
src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/CombineFileSplit.java

@@ -0,0 +1,196 @@
+/**
+ * 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.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.RecordReader;
+
+/**
+ * A sub-collection of input files. 
+ * 
+ * Unlike {@link FileSplit}, CombineFileSplit class does not represent 
+ * a split of a file, but a split of input files into smaller sets. 
+ * A split may contain blocks from different file but all 
+ * the blocks in the same split are probably local to some rack <br> 
+ * CombineFileSplit can be used to implement {@link RecordReader}'s, 
+ * with reading one record per file.
+ * 
+ * @see FileSplit
+ * @see CombineFileInputFormat 
+ */
+public class CombineFileSplit extends InputSplit implements Writable {
+
+  private Path[] paths;
+  private long[] startoffset;
+  private long[] lengths;
+  private String[] locations;
+  private long totLength;
+
+  /**
+   * default constructor
+   */
+  public CombineFileSplit() {}
+  public CombineFileSplit(Path[] files, long[] start, 
+                          long[] lengths, String[] locations) {
+    initSplit(files, start, lengths, locations);
+  }
+
+  public CombineFileSplit(Path[] files, long[] lengths) {
+    long[] startoffset = new long[files.length];
+    for (int i = 0; i < startoffset.length; i++) {
+      startoffset[i] = 0;
+    }
+    String[] locations = new String[files.length];
+    for (int i = 0; i < locations.length; i++) {
+      locations[i] = "";
+    }
+    initSplit(files, startoffset, lengths, locations);
+  }
+  
+  private void initSplit(Path[] files, long[] start, 
+                         long[] lengths, String[] locations) {
+    this.startoffset = start;
+    this.lengths = lengths;
+    this.paths = files;
+    this.totLength = 0;
+    this.locations = locations;
+    for(long length : lengths) {
+      totLength += length;
+    }
+  }
+
+  /**
+   * Copy constructor
+   */
+  public CombineFileSplit(CombineFileSplit old) throws IOException {
+    this(old.getPaths(), old.getStartOffsets(),
+         old.getLengths(), old.getLocations());
+  }
+
+  public long getLength() {
+    return totLength;
+  }
+
+  /** Returns an array containing the start offsets of the files in the split*/ 
+  public long[] getStartOffsets() {
+    return startoffset;
+  }
+  
+  /** Returns an array containing the lengths of the files in the split*/ 
+  public long[] getLengths() {
+    return lengths;
+  }
+
+  /** Returns the start offset of the i<sup>th</sup> Path */
+  public long getOffset(int i) {
+    return startoffset[i];
+  }
+  
+  /** Returns the length of the i<sup>th</sup> Path */
+  public long getLength(int i) {
+    return lengths[i];
+  }
+  
+  /** Returns the number of Paths in the split */
+  public int getNumPaths() {
+    return paths.length;
+  }
+
+  /** Returns the i<sup>th</sup> Path */
+  public Path getPath(int i) {
+    return paths[i];
+  }
+  
+  /** Returns all the Paths in the split */
+  public Path[] getPaths() {
+    return paths;
+  }
+
+  /** Returns all the Paths where this input-split resides */
+  public String[] getLocations() throws IOException {
+    return locations;
+  }
+
+  public void readFields(DataInput in) throws IOException {
+    totLength = in.readLong();
+    int arrLength = in.readInt();
+    lengths = new long[arrLength];
+    for(int i=0; i<arrLength;i++) {
+      lengths[i] = in.readLong();
+    }
+    int filesLength = in.readInt();
+    paths = new Path[filesLength];
+    for(int i=0; i<filesLength;i++) {
+      paths[i] = new Path(Text.readString(in));
+    }
+    arrLength = in.readInt();
+    startoffset = new long[arrLength];
+    for(int i=0; i<arrLength;i++) {
+      startoffset[i] = in.readLong();
+    }
+  }
+
+  public void write(DataOutput out) throws IOException {
+    out.writeLong(totLength);
+    out.writeInt(lengths.length);
+    for(long length : lengths) {
+      out.writeLong(length);
+    }
+    out.writeInt(paths.length);
+    for(Path p : paths) {
+      Text.writeString(out, p.toString());
+    }
+    out.writeInt(startoffset.length);
+    for(long length : startoffset) {
+      out.writeLong(length);
+    }
+  }
+  
+  @Override
+ public String toString() {
+    StringBuffer sb = new StringBuffer();
+    for (int i = 0; i < paths.length; i++) {
+      if (i == 0 ) {
+        sb.append("Paths:");
+      }
+      sb.append(paths[i].toUri().getPath() + ":" + startoffset[i] +
+                "+" + lengths[i]);
+      if (i < paths.length -1) {
+        sb.append(",");
+      }
+    }
+    if (locations != null) {
+      String locs = "";
+      StringBuffer locsb = new StringBuffer();
+      for (int i = 0; i < locations.length; i++) {
+        locsb.append(locations[i] + ":");
+      }
+      locs = locsb.toString();
+      sb.append(" Locations:" + locs + "; ");
+    }
+    return sb.toString();
+  }
+}

+ 369 - 0
src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/FilePool.java

@@ -0,0 +1,369 @@
+/**
+ * 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.util.Arrays;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Random;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+/**
+ * Class for caching a pool of input data to be used by synthetic jobs for
+ * simulating read traffic.
+ */
+class FilePool {
+
+  public static final Log LOG = LogFactory.getLog(FilePool.class);
+
+  /**
+   * The minimum file size added to the pool. Default 128MiB.
+   */
+  public static final String GRIDMIX_MIN_FILE = "gridmix.min.file.size";
+
+  /**
+   * The maximum size for files added to the pool. Defualts to 100TiB.
+   */
+  public static final String GRIDMIX_MAX_TOTAL = "gridmix.max.total.scan";
+
+  private Node root;
+  private final Path path;
+  private final FileSystem fs;
+  private final Configuration conf;
+  private final ReadWriteLock updateLock;
+
+  /**
+   * Initialize a filepool under the path provided, but do not populate the
+   * cache.
+   */
+  public FilePool(Configuration conf, Path input) throws IOException {
+    root = null;
+    this.conf = conf;
+    this.path = input;
+    this.fs = path.getFileSystem(conf);
+    updateLock = new ReentrantReadWriteLock();
+  }
+
+  /**
+   * Gather a collection of files at least as large as minSize.
+   * @return The total size of files returned.
+   */
+  public long getInputFiles(long minSize, Collection<FileStatus> files)
+      throws IOException {
+    updateLock.readLock().lock();
+    try {
+      return root.selectFiles(minSize, files);
+    } finally {
+      updateLock.readLock().unlock();
+    }
+  }
+
+  /**
+   * (Re)generate cache of input FileStatus objects.
+   */
+  public void refresh() throws IOException {
+    updateLock.writeLock().lock();
+    try {
+      root = new InnerDesc(fs, fs.getFileStatus(path),
+        new MinFileFilter(conf.getLong(GRIDMIX_MIN_FILE, 128 * 1024 * 1024),
+                          conf.getLong(GRIDMIX_MAX_TOTAL, 100L * (1L << 40))));
+      if (0 == root.getSize()) {
+        throw new IOException("Found no satisfactory file in " + path);
+      }
+    } finally {
+      updateLock.writeLock().unlock();
+    }
+  }
+
+  /**
+   * Get a set of locations for the given file.
+   */
+  public BlockLocation[] locationsFor(FileStatus stat, long start, long len)
+      throws IOException {
+    // TODO cache
+    return fs.getFileBlockLocations(stat, start, len);
+  }
+
+  static abstract class Node {
+
+    protected final static Random rand = new Random();
+
+    /**
+     * Total size of files and directories under the current node.
+     */
+    abstract long getSize();
+
+    /**
+     * Return a set of files whose cumulative size is at least
+     * <tt>targetSize</tt>.
+     * TODO Clearly size is not the only criterion, e.g. refresh from
+     * generated data without including running task output, tolerance
+     * for permission issues, etc.
+     */
+    abstract long selectFiles(long targetSize, Collection<FileStatus> files)
+        throws IOException;
+  }
+
+  interface IndexMapper {
+    int get(int pos);
+    void swap(int a, int b);
+  }
+
+  /**
+   * A sparse index mapping table - useful when we want to
+   * non-destructively permute a small fraction of a large array.
+   */
+  static class SparseIndexMapper implements IndexMapper {
+    Map<Integer, Integer> mapping = new HashMap<Integer, Integer>();
+
+    public int get(int pos) {
+      Integer mapped = mapping.get(pos);
+      if (mapped == null) return pos;
+      return mapped;
+    }
+
+    public void swap(int a, int b) {
+      int valA = get(a);
+      int valB = get(b);
+      if (b == valA) {
+        mapping.remove(b);
+      } else {
+        mapping.put(b, valA);
+      }
+      if (a == valB) {
+        mapping.remove(a);
+      } else {
+        mapping.put(a, valB);
+      }
+    }
+  }
+
+  /**
+   * A dense index mapping table - useful when we want to
+   * non-destructively permute a large fraction of an array.
+   */
+  static class DenseIndexMapper implements IndexMapper {
+    int[] mapping;
+
+    DenseIndexMapper(int size) {
+      mapping = new int[size];
+      for (int i=0; i<size; ++i) {
+        mapping[i] = i;
+      }
+    }
+
+    public int get(int pos) {
+      if ( (pos < 0) || (pos>=mapping.length) ) {
+        throw new IndexOutOfBoundsException();
+      }
+      return mapping[pos];
+    }
+
+    public void swap(int a, int b) {
+      int valA = get(a);
+      int valB = get(b);
+      mapping[a]=valB;
+      mapping[b]=valA;
+    }
+  }
+
+  /**
+   * Files in current directory of this Node.
+   */
+  static class LeafDesc extends Node {
+    final long size;
+    final ArrayList<FileStatus> curdir;
+
+    LeafDesc(ArrayList<FileStatus> curdir, long size) {
+      this.size = size;
+      this.curdir = curdir;
+    }
+
+    @Override
+    public long getSize() {
+      return size;
+    }
+
+    @Override
+    public long selectFiles(long targetSize, Collection<FileStatus> files)
+        throws IOException {
+      if (targetSize >= getSize()) {
+        files.addAll(curdir);
+        return getSize();
+      }
+
+      IndexMapper mapping;
+      if ((curdir.size() < 200) || ((double) targetSize / getSize() > 0.5)) {
+        mapping = new DenseIndexMapper(curdir.size());
+      } else {
+        mapping = new SparseIndexMapper();
+      }
+
+      ArrayList<Integer> selected = new ArrayList<Integer>();
+      long ret = 0L;
+      int poolSize = curdir.size();
+      do {
+        int pos = rand.nextInt(poolSize);
+        int index = mapping.get(pos);
+        selected.add(index);
+        ret += curdir.get(index).getLen();
+        mapping.swap(pos, --poolSize);
+      } while (ret < targetSize);
+
+      for (Integer i : selected) {
+        files.add(curdir.get(i));
+      }
+
+      return ret;
+    }
+  }
+
+  /**
+   * A subdirectory of the current Node.
+   */
+  static class InnerDesc extends Node {
+    final long size;
+    final double[] dist;
+    final Node[] subdir;
+
+    private static final Comparator<Node> nodeComparator =
+      new Comparator<Node>() {
+          public int compare(Node n1, Node n2) {
+            return n1.getSize() < n2.getSize() ? -1
+                 : n1.getSize() > n2.getSize() ? 1 : 0;
+          }
+    };
+
+    InnerDesc(final FileSystem fs, FileStatus thisDir, MinFileFilter filter)
+        throws IOException {
+      long fileSum = 0L;
+      final ArrayList<FileStatus> curFiles = new ArrayList<FileStatus>();
+      final ArrayList<FileStatus> curDirs = new ArrayList<FileStatus>();
+      for (FileStatus stat : fs.listStatus(thisDir.getPath())) {
+        if (stat.isDir()) {
+          curDirs.add(stat);
+        } else if (filter.accept(stat)) {
+          curFiles.add(stat);
+          fileSum += stat.getLen();
+        }
+      }
+      ArrayList<Node> subdirList = new ArrayList<Node>();
+      if (!curFiles.isEmpty()) {
+        subdirList.add(new LeafDesc(curFiles, fileSum));
+      }
+      for (Iterator<FileStatus> i = curDirs.iterator();
+          !filter.done() && i.hasNext();) {
+        // add subdirectories
+        final Node d = new InnerDesc(fs, i.next(), filter);
+        final long dSize = d.getSize();
+        if (dSize > 0) {
+          fileSum += dSize;
+          subdirList.add(d);
+        }
+      }
+      size = fileSum;
+      LOG.debug(size + " bytes in " + thisDir.getPath());
+      subdir = subdirList.toArray(new Node[subdirList.size()]);
+      Arrays.sort(subdir, nodeComparator);
+      dist = new double[subdir.length];
+      for (int i = dist.length - 1; i > 0; --i) {
+        fileSum -= subdir[i].getSize();
+        dist[i] = fileSum / (1.0 * size);
+      }
+    }
+
+    @Override
+    public long getSize() {
+      return size;
+    }
+
+    @Override
+    public long selectFiles(long targetSize, Collection<FileStatus> files)
+        throws IOException {
+      long ret = 0L;
+      if (targetSize >= getSize()) {
+        // request larger than all subdirs; add everything
+        for (Node n : subdir) {
+          long added = n.selectFiles(targetSize, files);
+          ret += added;
+          targetSize -= added;
+        }
+        return ret;
+      }
+
+      // can satisfy request in proper subset of contents
+      // select random set, weighted by size
+      final HashSet<Node> sub = new HashSet<Node>();
+      do {
+        assert sub.size() < subdir.length;
+        final double r = rand.nextDouble();
+        int pos = Math.abs(Arrays.binarySearch(dist, r) + 1) - 1;
+        while (sub.contains(subdir[pos])) {
+          pos = (pos + 1) % subdir.length;
+        }
+        long added = subdir[pos].selectFiles(targetSize, files);
+        ret += added;
+        targetSize -= added;
+        sub.add(subdir[pos]);
+      } while (targetSize > 0);
+      return ret;
+    }
+  }
+
+  /**
+   * Filter enforcing the minFile/maxTotal parameters of the scan.
+   */
+  private static class MinFileFilter {
+
+    private long totalScan;
+    private final long minFileSize;
+
+    public MinFileFilter(long minFileSize, long totalScan) {
+      this.minFileSize = minFileSize;
+      this.totalScan = totalScan;
+    }
+    public boolean done() {
+      return totalScan <= 0;
+    }
+    public boolean accept(FileStatus stat) {
+      final boolean done = done();
+      if (!done && stat.getLen() >= minFileSize) {
+        totalScan -= stat.getLen();
+        return true;
+      }
+      return false;
+    }
+  }
+
+}

+ 104 - 0
src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/FileQueue.java

@@ -0,0 +1,104 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.mapred.gridmix;
+
+import 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;
+
+/**
+ * Given a {@link org.apache.hadoop.mapreduce.lib.input.CombineFileSplit},
+ * circularly read through each input source.
+ */
+class FileQueue extends InputStream {
+
+  private int idx = -1;
+  private long curlen = -1L;
+  private FSDataInputStream input;
+  private final byte[] z = new byte[1];
+  private final Path[] paths;
+  private final long[] lengths;
+  private final long[] startoffset;
+  private final Configuration conf;
+
+  /**
+   * @param split Description of input sources.
+   * @param conf Used to resolve FileSystem instances.
+   */
+  public FileQueue(CombineFileSplit split, Configuration conf)
+      throws IOException {
+    this.conf = conf;
+    paths = split.getPaths();
+    startoffset = split.getStartOffsets();
+    lengths = split.getLengths();
+    nextSource();
+  }
+
+  protected void nextSource() throws IOException {
+    if (0 == paths.length) {
+      return;
+    }
+    if (input != null) {
+      input.close();
+    }
+    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]);
+  }
+
+  @Override
+  public int read() throws IOException {
+    final int tmp = read(z);
+    return tmp == -1 ? -1 : (0xFF & z[0]);
+  }
+
+  @Override
+  public int read(byte[] b) throws IOException {
+    return read(b, 0, b.length);
+  }
+
+  @Override
+  public int read(byte[] b, int off, int len) throws IOException {
+    int kvread = 0;
+    while (kvread < len) {
+      if (curlen <= 0) {
+        nextSource();
+        continue;
+      }
+      final int srcRead = (int) Math.min(len - kvread, curlen);
+      IOUtils.readFully(input, b, kvread, srcRead);
+      curlen -= srcRead;
+      kvread += srcRead;
+    }
+    return kvread;
+  }
+
+  @Override
+  public void close() throws IOException {
+    input.close();
+  }
+
+}

+ 293 - 0
src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/GenerateData.java

@@ -0,0 +1,293 @@
+/**
+ * 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.io.DataInput;
+import java.io.DataOutput;
+import java.io.OutputStream;
+import java.util.Arrays;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Text;
+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.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.RecordWriter;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+
+// TODO can replace with form of GridmixJob
+class GenerateData extends GridmixJob {
+
+  /**
+   * Total bytes to write.
+   */
+  public static final String GRIDMIX_GEN_BYTES = "gridmix.gen.bytes";
+
+  /**
+   * Maximum size per file written.
+   */
+  public static final String GRIDMIX_GEN_CHUNK = "gridmix.gen.bytes.per.file";
+
+  /**
+   * Size of writes to output file.
+   */
+  public static final String GRIDMIX_VAL_BYTES = "gendata.val.bytes";
+
+  /**
+   * Status reporting interval, in megabytes.
+   */
+  public static final String GRIDMIX_GEN_INTERVAL = "gendata.interval.mb";
+
+  public GenerateData(Configuration conf, Path outdir, long genbytes)
+      throws IOException {
+    super(conf, 0L, "GRIDMIX_GENDATA");
+    job.getConfiguration().setLong(GRIDMIX_GEN_BYTES, genbytes);
+    FileOutputFormat.setOutputPath(job, outdir);
+  }
+
+  @Override
+  public Job call() throws IOException, InterruptedException,
+                           ClassNotFoundException {
+    job.setMapperClass(GenDataMapper.class);
+    job.setNumReduceTasks(0);
+    job.setMapOutputKeyClass(NullWritable.class);
+    job.setMapOutputValueClass(BytesWritable.class);
+    job.setInputFormatClass(GenDataFormat.class);
+    job.setOutputFormatClass(RawBytesOutputFormat.class);
+    job.setJarByClass(GenerateData.class);
+    FileInputFormat.addInputPath(job, new Path("ignored"));
+    job.submit();
+    return job;
+  }
+
+  public static class GenDataMapper
+      extends Mapper<NullWritable,LongWritable,NullWritable,BytesWritable> {
+
+    private BytesWritable val;
+    private final Random r = new Random();
+
+    @Override
+    protected void setup(Context context)
+        throws IOException, InterruptedException {
+      val = new BytesWritable(new byte[
+          context.getConfiguration().getInt(GRIDMIX_VAL_BYTES, 1024 * 1024)]);
+    }
+
+    @Override
+    public void map(NullWritable key, LongWritable value, Context context)
+        throws IOException, InterruptedException {
+      for (long bytes = value.get(); bytes > 0; bytes -= val.getLength()) {
+        r.nextBytes(val.getBytes());
+        val.setSize((int)Math.min(val.getLength(), bytes));
+        context.write(key, val);
+      }
+    }
+
+  }
+
+  static class GenDataFormat extends InputFormat<NullWritable,LongWritable> {
+
+    @Override
+    public List<InputSplit> getSplits(JobContext jobCtxt) throws IOException {
+      final JobClient client =
+        new JobClient(new JobConf(jobCtxt.getConfiguration()));
+      ClusterStatus stat = client.getClusterStatus(true);
+      final long toGen =
+        jobCtxt.getConfiguration().getLong(GRIDMIX_GEN_BYTES, -1);
+      if (toGen < 0) {
+        throw new IOException("Invalid/missing generation bytes: " + toGen);
+      }
+      final int nTrackers = stat.getTaskTrackers();
+      final long bytesPerTracker = toGen / nTrackers;
+      final ArrayList<InputSplit> splits = new ArrayList<InputSplit>(nTrackers);
+      final Pattern trackerPattern = Pattern.compile("tracker_([^:]*):.*");
+      final Matcher m = trackerPattern.matcher("");
+      for (String tracker : stat.getActiveTrackerNames()) {
+        m.reset(tracker);
+        if (!m.find()) {
+          System.err.println("Skipping node: " + tracker);
+          continue;
+        }
+        final String name = m.group(1);
+        splits.add(new GenSplit(bytesPerTracker, new String[] { name }));
+      }
+      return splits;
+    }
+
+    @Override
+    public RecordReader<NullWritable,LongWritable> createRecordReader(
+        InputSplit split, final TaskAttemptContext taskContext)
+        throws IOException {
+      return new RecordReader<NullWritable,LongWritable>() {
+        long written = 0L;
+        long write = 0L;
+        long RINTERVAL;
+        long toWrite;
+        final NullWritable key = NullWritable.get();
+        final LongWritable val = new LongWritable();
+
+        @Override
+        public void initialize(InputSplit split, TaskAttemptContext ctxt)
+            throws IOException, InterruptedException {
+          toWrite = split.getLength();
+          RINTERVAL = ctxt.getConfiguration().getInt(
+              GRIDMIX_GEN_INTERVAL, 10) << 20;
+        }
+        @Override
+        public boolean nextKeyValue() throws IOException {
+          written += write;
+          write = Math.min(toWrite - written, RINTERVAL);
+          val.set(write);
+          return written < toWrite;
+        }
+        @Override
+        public float getProgress() throws IOException {
+          return written / ((float)toWrite);
+        }
+        @Override
+        public NullWritable getCurrentKey() { return key; }
+        @Override
+        public LongWritable getCurrentValue() { return val; }
+        @Override
+        public void close() throws IOException {
+          taskContext.setStatus("Wrote " + toWrite);
+        }
+      };
+    }
+  }
+
+  static class GenSplit extends InputSplit implements Writable {
+    private long bytes;
+    private int nLoc;
+    private String[] locations;
+
+    public GenSplit() { }
+    public GenSplit(long bytes, String[] locations) {
+      this(bytes, locations.length, locations);
+    }
+    public GenSplit(long bytes, int nLoc, String[] locations) {
+      this.bytes = bytes;
+      this.nLoc = nLoc;
+      this.locations = Arrays.copyOf(locations, nLoc);
+    }
+    @Override
+    public long getLength() {
+      return bytes;
+    }
+    @Override
+    public String[] getLocations() {
+      return locations;
+    }
+    @Override
+    public void readFields(DataInput in) throws IOException {
+      bytes = in.readLong();
+      nLoc = in.readInt();
+      if (null == locations || locations.length < nLoc) {
+        locations = new String[nLoc];
+      }
+      for (int i = 0; i < nLoc; ++i) {
+        locations[i] = Text.readString(in);
+      }
+    }
+    @Override
+    public void write(DataOutput out) throws IOException {
+      out.writeLong(bytes);
+      out.writeInt(nLoc);
+      for (int i = 0; i < nLoc; ++i) {
+        Text.writeString(out, locations[i]);
+      }
+    }
+  }
+
+  static class RawBytesOutputFormat
+      extends FileOutputFormat<NullWritable,BytesWritable> {
+
+    @Override
+    public RecordWriter<NullWritable,BytesWritable> getRecordWriter(
+        TaskAttemptContext job) throws IOException {
+
+      return new ChunkWriter(getDefaultWorkFile(job, ""),
+          job.getConfiguration());
+    }
+
+    static class ChunkWriter extends RecordWriter<NullWritable,BytesWritable> {
+      private final Path outDir;
+      private final FileSystem fs;
+      private final long maxFileBytes;
+
+      private long accFileBytes = 0L;
+      private long fileIdx = -1L;
+      private OutputStream fileOut = null;
+
+      public ChunkWriter(Path outDir, Configuration conf) throws IOException {
+        this.outDir = outDir;
+        fs = outDir.getFileSystem(conf);
+        maxFileBytes = conf.getLong(GRIDMIX_GEN_CHUNK, 1L << 30);
+        nextDestination();
+      }
+      private void nextDestination() throws IOException {
+        if (fileOut != null) {
+          fileOut.close();
+        }
+        fileOut = fs.create(new Path(outDir, "segment-" + (++fileIdx)), false);
+        accFileBytes = 0L;
+      }
+      @Override
+      public void write(NullWritable key, BytesWritable value)
+          throws IOException {
+        int written = 0;
+        final int total = value.getLength();
+        while (written < total) {
+          final int write = (int)
+            Math.min(total - written, maxFileBytes - accFileBytes);
+          fileOut.write(value.getBytes(), written, write);
+          written += write;
+          accFileBytes += write;
+          if (accFileBytes >= maxFileBytes) {
+            nextDestination();
+          }
+        }
+      }
+      @Override
+      public void close(TaskAttemptContext ctxt) throws IOException {
+        fileOut.close();
+      }
+    }
+  }
+
+}

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

@@ -0,0 +1,351 @@
+/**
+ * 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.io.InputStream;
+import java.io.PrintStream;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+/**
+ * Driver class for the Gridmix3 benchmark. Gridmix accepts a timestamped
+ * stream (trace) of job/task descriptions. For each job in the trace, the
+ * client will submit a corresponding, synthetic job to the target cluster at
+ * the rate in the original trace. The intent is to provide a benchmark that
+ * can be configured and extended to closely match the measured resource
+ * profile of actual, production loads.
+ */
+public class Gridmix extends Configured implements Tool {
+
+  public static final Log LOG = LogFactory.getLog(Gridmix.class);
+
+  /**
+   * Output (scratch) directory for submitted jobs. Relative paths are
+   * resolved against the path provided as input and absolute paths remain
+   * independent of it. The default is &quot;gridmix&quot;.
+   */
+  public static final String GRIDMIX_OUT_DIR = "gridmix.output.directory";
+
+  /**
+   * Number of submitting threads at the client and upper bound for
+   * in-memory split data. Submitting threads precompute InputSplits for
+   * submitted jobs. This limits the number of splits held in memory waiting
+   * for submission and also permits parallel computation of split data.
+   */
+  public static final String GRIDMIX_SUB_THR = "gridmix.client.submit.threads";
+
+  /**
+   * The depth of the queue of job descriptions. Before splits are computed,
+   * a queue of pending descriptions is stored in memoory. This parameter
+   * limits the depth of that queue.
+   */
+  public static final String GRIDMIX_QUE_DEP =
+    "gridmix.client.pending.queue.depth";
+
+  /**
+   * Multiplier to accelerate or decelerate job submission. As a crude means of
+   * sizing a job trace to a cluster, the time separating two jobs is
+   * multiplied by this factor.
+   */
+  public static final String GRIDMIX_SUB_MUL = "gridmix.submit.multiplier";
+
+  // Submit data structures
+  private JobFactory factory;
+  private JobSubmitter submitter;
+  private JobMonitor monitor;
+
+  // Shutdown hook
+  private final Shutdown sdh = new Shutdown();
+
+  /**
+   * Write random bytes at the path provided.
+   * @see org.apache.hadoop.mapred.gridmix.GenerateData
+   */
+  protected void writeInputData(long genbytes, Path ioPath)
+      throws IOException, InterruptedException {
+    final Configuration conf = getConf();
+    final GridmixJob genData = new GenerateData(conf, ioPath, genbytes);
+    submitter.add(genData);
+    LOG.info("Generating " + StringUtils.humanReadableInt(genbytes) +
+        " of test data...");
+    // TODO add listeners, use for job dependencies
+    TimeUnit.SECONDS.sleep(10);
+    try {
+      genData.getJob().waitForCompletion(false);
+    } catch (ClassNotFoundException e) {
+      throw new IOException("Internal error", e);
+    }
+    if (!genData.getJob().isSuccessful()) {
+      throw new IOException("Data generation failed!");
+    }
+    LOG.info("Done.");
+  }
+
+  protected InputStream createInputStream(String in) throws IOException {
+    if ("-".equals(in)) {
+      return System.in;
+    }
+    final Path pin = new Path(in);
+    return pin.getFileSystem(getConf()).open(pin);
+  }
+
+  /**
+   * 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 scratchDir Path into which job output is written
+   * @param startFlag Semaphore for starting job trace pipeline
+   */
+  private void startThreads(Configuration conf, String traceIn, Path ioPath,
+      Path scratchDir, CountDownLatch startFlag) throws IOException {
+    monitor = createJobMonitor();
+    submitter = createJobSubmitter(monitor,
+        conf.getInt(GRIDMIX_SUB_THR,
+          Runtime.getRuntime().availableProcessors() + 1),
+        conf.getInt(GRIDMIX_QUE_DEP, 5),
+        new FilePool(conf, ioPath));
+    factory = createJobFactory(submitter, traceIn, scratchDir, conf, startFlag);
+    monitor.start();
+    submitter.start();
+    factory.start();
+  }
+
+  protected JobMonitor createJobMonitor() throws IOException {
+    return new JobMonitor();
+  }
+
+  protected JobSubmitter createJobSubmitter(JobMonitor monitor, int threads,
+      int queueDepth, FilePool pool) throws IOException {
+    return new JobSubmitter(monitor, threads, queueDepth, pool);
+  }
+
+  protected JobFactory createJobFactory(JobSubmitter submitter, String traceIn,
+      Path scratchDir, Configuration conf, CountDownLatch startFlag)
+      throws IOException {
+    return new JobFactory(submitter, createInputStream(traceIn), scratchDir,
+        conf, startFlag);
+  }
+
+  public int run(String[] argv) throws IOException, InterruptedException {
+    if (argv.length < 2) {
+      printUsage(System.err);
+      return 1;
+    }
+    long genbytes = 0;
+    String traceIn = null;
+    Path ioPath = null;
+    try {
+      int i = 0;
+      genbytes = "-generate".equals(argv[i++])
+        ? StringUtils.TraditionalBinaryPrefix.string2long(argv[i++])
+        : --i;
+      ioPath = new Path(argv[i++]);
+      traceIn = argv[i++];
+      if (i != argv.length) {
+        printUsage(System.err);
+        return 1;
+      }
+    } catch (Exception e) {
+      printUsage(System.err);
+      return 1;
+    }
+    InputStream trace = null;
+    try {
+      final Configuration conf = getConf();
+      Path scratchDir = new Path(ioPath, conf.get(GRIDMIX_OUT_DIR, "gridmix"));
+      // add shutdown hook for SIGINT, etc.
+      Runtime.getRuntime().addShutdownHook(sdh);
+      CountDownLatch startFlag = new CountDownLatch(1);
+      try {
+        // Create, start job submission threads
+        startThreads(conf, traceIn, ioPath, scratchDir, startFlag);
+        // Write input data if specified
+        if (genbytes > 0) {
+          writeInputData(genbytes, ioPath);
+        }
+        // scan input dir contents
+        submitter.refreshFilePool();
+      } catch (Throwable e) {
+        LOG.error("Startup failed", e);
+        if (factory != null) factory.abort(); // abort pipeline
+      } finally {
+        // signal for factory to start; sets start time
+        startFlag.countDown();
+      }
+
+      if (factory != null) {
+        // wait for input exhaustion
+        factory.join(Long.MAX_VALUE);
+        final Throwable badTraceException = factory.error();
+        if (null != badTraceException) {
+          LOG.error("Error in trace", badTraceException);
+          throw new IOException("Error in trace", badTraceException);
+        }
+        // wait for pending tasks to be submitted
+        submitter.shutdown();
+        submitter.join(Long.MAX_VALUE);
+        // wait for running tasks to complete
+        monitor.shutdown();
+        monitor.join(Long.MAX_VALUE);
+      }
+    } finally {
+      IOUtils.cleanup(LOG, trace);
+    }
+    return 0;
+  }
+
+  /**
+   * Handles orderly shutdown by requesting that each component in the
+   * pipeline abort its progress, waiting for each to exit and killing
+   * any jobs still running on the cluster.
+   */
+  class Shutdown extends Thread {
+
+    static final long FAC_SLEEP = 1000;
+    static final long SUB_SLEEP = 4000;
+    static final long MON_SLEEP = 15000;
+
+    private void killComponent(Component<?> component, long maxwait) {
+      if (component == null) {
+        return;
+      }
+      component.abort();
+      try {
+        component.join(maxwait);
+      } catch (InterruptedException e) {
+        LOG.warn("Interrupted waiting for " + component);
+      }
+
+    }
+
+    @Override
+    public void run() {
+      LOG.info("Exiting...");
+      try {
+        killComponent(factory, FAC_SLEEP);   // read no more tasks
+        killComponent(submitter, SUB_SLEEP); // submit no more tasks
+        killComponent(monitor, MON_SLEEP);   // process remaining jobs here
+      } finally {
+        if (monitor == null) {
+          return;
+        }
+        List<Job> remainingJobs = monitor.getRemainingJobs();
+        if (remainingJobs.isEmpty()) {
+          return;
+        }
+        LOG.info("Killing running jobs...");
+        for (Job job : remainingJobs) {
+          try {
+            if (!job.isComplete()) {
+              job.killJob();
+              LOG.info("Killed " + job.getJobName() + " (" +
+                  job.getJobID() + ")");
+            } else {
+              if (job.isSuccessful()) {
+                monitor.onSuccess(job);
+              } else {
+                monitor.onFailure(job);
+              }
+            }
+          } catch (IOException e) {
+            LOG.warn("Failure killing " + job.getJobName(), e);
+          } catch (Exception e) {
+            LOG.error("Unexcpected exception", e);
+          }
+        }
+        LOG.info("Done.");
+      }
+    }
+
+  }
+
+  public static void main(String[] argv) throws Exception {
+    int res = -1;
+    try {
+      res = ToolRunner.run(new Configuration(), new Gridmix(), argv);
+    } finally {
+      System.exit(res);
+    }
+  }
+
+  protected void printUsage(PrintStream out) {
+    ToolRunner.printGenericCommandUsage(out);
+    out.println("Usage: gridmix [-generate <MiB>] <iopath> <trace>");
+    out.println("  e.g. gridmix -generate 100m foo -");
+    out.println("Configuration parameters:");
+    out.printf("       %-40s : Output directory\n", GRIDMIX_OUT_DIR);
+    out.printf("       %-40s : Submitting threads\n", GRIDMIX_SUB_THR);
+    out.printf("       %-40s : Queued job desc\n", GRIDMIX_QUE_DEP);
+    out.printf("       %-40s : Key fraction of rec\n",
+        AvgRecordFactory.GRIDMIX_KEY_FRC);
+  }
+
+  /**
+   * Components in the pipeline must support the following operations for
+   * orderly startup and shutdown.
+   */
+  interface Component<T> {
+
+    /**
+     * Accept an item into this component from an upstream component. If
+     * shutdown or abort have been called, this may fail, depending on the
+     * semantics for the component.
+     */
+    void add(T item) throws InterruptedException;
+
+    /**
+     * Attempt to start the service.
+     */
+    void start();
+
+    /**
+     * Wait until the service completes. It is assumed that either a
+     * {@link #shutdown} or {@link #abort} has been requested.
+     */
+    void join(long millis) throws InterruptedException;
+
+    /**
+     * Shut down gracefully, finishing all pending work. Reject new requests.
+     */
+    void shutdown();
+
+    /**
+     * Shut down immediately, aborting any work in progress and discarding
+     * all pending work. It is legal to store pending work for another
+     * thread to process.
+     */
+    void abort();
+  }
+
+}

+ 523 - 0
src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/GridmixJob.java

@@ -0,0 +1,523 @@
+/**
+ * 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.util.ArrayList;
+import java.util.Formatter;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Delayed;
+import java.util.concurrent.TimeUnit;
+
+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.NullWritable;
+import org.apache.hadoop.io.RawComparator;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableComparator;
+import org.apache.hadoop.io.WritableUtils;
+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.Partitioner;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.hadoop.mapreduce.Reducer;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.tools.rumen.JobStory;
+import org.apache.hadoop.tools.rumen.TaskInfo;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+/**
+ * Synthetic job generated from a trace description.
+ */
+class GridmixJob implements Callable<Job>, Delayed {
+
+  public static final String JOBNAME = "GRIDMIX";
+  public static final String ORIGNAME = "gridmix.job.name.original";
+  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);
+        return new Formatter(sb);
+      }
+    };
+
+  private final int seq;
+  private final Path outdir;
+  protected final Job job;
+  private final JobStory jobdesc;
+  private final long submissionTimeNanos;
+
+  public GridmixJob(Configuration conf, long submissionMillis,
+      JobStory jobdesc, Path outRoot, int seq) throws IOException {
+    ((StringBuilder)nameFormat.get().out()).setLength(JOBNAME.length());
+    job = new Job(conf, nameFormat.get().format("%05d", seq).toString());
+    submissionTimeNanos = TimeUnit.NANOSECONDS.convert(
+        submissionMillis, TimeUnit.MILLISECONDS);
+    this.jobdesc = jobdesc;
+    this.seq = seq;
+    outdir = new Path(outRoot, "" + seq);
+  }
+
+  protected GridmixJob(Configuration conf, long submissionMillis, String name)
+      throws IOException {
+    job = new Job(conf, name);
+    submissionTimeNanos = TimeUnit.NANOSECONDS.convert(
+        submissionMillis, TimeUnit.MILLISECONDS);
+    jobdesc = null;
+    outdir = null;
+    seq = -1;
+  }
+
+  public String toString() {
+    return job.getJobName();
+  }
+
+  public long getDelay(TimeUnit unit) {
+    return unit.convert(submissionTimeNanos - System.nanoTime(),
+        TimeUnit.NANOSECONDS);
+  }
+
+  @Override
+  public int compareTo(Delayed other) {
+    if (this == other) {
+      return 0;
+    }
+    if (other instanceof GridmixJob) {
+      final long otherNanos = ((GridmixJob)other).submissionTimeNanos;
+      if (otherNanos < submissionTimeNanos) {
+        return 1;
+      }
+      if (otherNanos > submissionTimeNanos) {
+        return -1;
+      }
+      return id() - ((GridmixJob)other).id();
+    }
+    final long diff =
+      getDelay(TimeUnit.NANOSECONDS) - other.getDelay(TimeUnit.NANOSECONDS);
+    return 0 == diff ? 0 : (diff > 0 ? 1 : -1);
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (this == other) {
+      return true;
+    }
+    // not possible unless job is cloned; all jobs should be unique
+    return other instanceof GridmixJob && id() == ((GridmixJob)other).id();
+  }
+
+  @Override
+  public int hashCode() {
+    return id();
+  }
+
+  int id() {
+    return seq;
+  }
+
+  Job getJob() {
+    return job;
+  }
+
+  JobStory getJobDesc() {
+    return jobdesc;
+  }
+
+  public Job call() throws IOException, InterruptedException,
+                           ClassNotFoundException {
+    job.setMapperClass(GridmixMapper.class);
+    job.setReducerClass(GridmixReducer.class);
+    job.setNumReduceTasks(jobdesc.getNumberReduces());
+    job.setMapOutputKeyClass(GridmixKey.class);
+    job.setMapOutputValueClass(GridmixRecord.class);
+    job.setSortComparatorClass(GridmixKey.Comparator.class);
+    job.setGroupingComparatorClass(SpecGroupingComparator.class);
+    job.setInputFormatClass(GridmixInputFormat.class);
+    job.setOutputFormatClass(RawBytesOutputFormat.class);
+    job.setPartitionerClass(DraftPartitioner.class);
+    job.setJarByClass(GridmixJob.class);
+    job.getConfiguration().setInt("gridmix.job.seq", seq);
+    job.getConfiguration().set(ORIGNAME, null == jobdesc.getJobID()
+        ? "<unknown>" : jobdesc.getJobID().toString());
+    job.getConfiguration().setBoolean("mapred.used.genericoptionsparser", true);
+    FileInputFormat.addInputPath(job, new Path("ignored"));
+    FileOutputFormat.setOutputPath(job, outdir);
+    job.submit();
+    return job;
+  }
+
+  public static class DraftPartitioner<V> extends Partitioner<GridmixKey,V> {
+    public int getPartition(GridmixKey key, V value, int numReduceTasks) {
+      return key.getPartition();
+    }
+  }
+
+  public static class SpecGroupingComparator
+      implements RawComparator<GridmixKey> {
+    private final DataInputBuffer di = new DataInputBuffer();
+    private final byte[] reset = di.getData();
+    @Override
+    public int compare(GridmixKey g1, GridmixKey g2) {
+      final byte t1 = g1.getType();
+      final byte t2 = g2.getType();
+      if (t1 == GridmixKey.REDUCE_SPEC ||
+          t2 == GridmixKey.REDUCE_SPEC) {
+        return t1 - t2;
+      }
+      assert t1 == GridmixKey.DATA;
+      assert t2 == GridmixKey.DATA;
+      return g1.compareTo(g2);
+    }
+    @Override
+    public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+      try {
+        final int ret;
+        di.reset(b1, s1, l1);
+        final int x1 = WritableUtils.readVInt(di);
+        di.reset(b2, s2, l2);
+        final int x2 = WritableUtils.readVInt(di);
+        final int t1 = b1[s1 + x1];
+        final int t2 = b2[s2 + x2];
+        if (t1 == GridmixKey.REDUCE_SPEC ||
+            t2 == GridmixKey.REDUCE_SPEC) {
+          ret = t1 - t2;
+        } else {
+          assert t1 == GridmixKey.DATA;
+          assert t2 == GridmixKey.DATA;
+          ret =
+            WritableComparator.compareBytes(b1, s1, x1, b2, s2, x2);
+        }
+        di.reset(reset, 0, 0);
+        return ret;
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      }
+    }
+  }
+
+  public static class GridmixMapper
+      extends Mapper<NullWritable,GridmixRecord,GridmixKey,GridmixRecord> {
+
+    private double acc;
+    private double ratio;
+    private final ArrayList<RecordFactory> reduces =
+      new ArrayList<RecordFactory>();
+    private final Random r = new Random();
+
+    private final GridmixKey key = new GridmixKey();
+    private final GridmixRecord val = new GridmixRecord();
+
+    @Override
+    protected void setup(Context ctxt)
+        throws IOException, InterruptedException {
+      final Configuration conf = ctxt.getConfiguration();
+      final GridmixSplit split = (GridmixSplit) ctxt.getInputSplit();
+      final int maps = split.getMapCount();
+      final long[] reduceBytes = split.getOutputBytes();
+      final long[] reduceRecords = split.getOutputRecords();
+
+      long totalRecords = 0L;
+      final int nReduces = ctxt.getNumReduceTasks();
+      if (nReduces > 0) {
+        int idx = 0;
+        int id = split.getId();
+        for (int i = 0; i < nReduces; ++i) {
+          final GridmixKey.Spec spec = new GridmixKey.Spec();
+          if (i == id) {
+            spec.bytes_out = split.getReduceBytes(idx);
+            spec.rec_out = split.getReduceRecords(idx);
+            ++idx;
+            id += maps;
+          }
+          reduces.add(new IntermediateRecordFactory(
+              new AvgRecordFactory(reduceBytes[i], reduceRecords[i], conf),
+              i, reduceRecords[i], spec, conf));
+          totalRecords += reduceRecords[i];
+        }
+      } else {
+        reduces.add(new AvgRecordFactory(reduceBytes[0], reduceRecords[0],
+              conf));
+        totalRecords = reduceRecords[0];
+      }
+      final long splitRecords = split.getInputRecords();
+      final long inputRecords = splitRecords <= 0 && split.getLength() >= 0
+        ? Math.max(1,
+          split.getLength() / conf.getInt("gridmix.missing.rec.size", 64*1024))
+        : splitRecords;
+      ratio = totalRecords / (1.0 * inputRecords);
+      acc = 0.0;
+    }
+
+    @Override
+    public void map(NullWritable ignored, GridmixRecord rec,
+        Context context) throws IOException, InterruptedException {
+      acc += ratio;
+      while (acc >= 1.0 && !reduces.isEmpty()) {
+        key.setSeed(r.nextLong());
+        val.setSeed(r.nextLong());
+        final int idx = r.nextInt(reduces.size());
+        final RecordFactory f = reduces.get(idx);
+        if (!f.next(key, val)) {
+          reduces.remove(idx);
+          continue;
+        }
+        context.write(key, val);
+        acc -= 1.0;
+      }
+    }
+
+    @Override
+    public void cleanup(Context context)
+        throws IOException, InterruptedException {
+      for (RecordFactory factory : reduces) {
+        key.setSeed(r.nextLong());
+        while (factory.next(key, val)) {
+          context.write(key, val);
+          key.setSeed(r.nextLong());
+        }
+      }
+    }
+  }
+
+  public static class GridmixReducer
+      extends Reducer<GridmixKey,GridmixRecord,NullWritable,GridmixRecord> {
+
+    private final Random r = new Random();
+    private final GridmixRecord val = new GridmixRecord();
+
+    private double acc;
+    private double ratio;
+    private RecordFactory factory;
+
+    @Override
+    protected void setup(Context context)
+        throws IOException, InterruptedException {
+      if (!context.nextKey() ||
+           context.getCurrentKey().getType() != GridmixKey.REDUCE_SPEC) {
+        throw new IOException("Missing reduce spec");
+      }
+      long outBytes = 0L;
+      long outRecords = 0L;
+      long inRecords = 0L;
+      for (GridmixRecord ignored : context.getValues()) {
+        final GridmixKey spec = context.getCurrentKey();
+        inRecords += spec.getReduceInputRecords();
+        outBytes += spec.getReduceOutputBytes();
+        outRecords += spec.getReduceOutputRecords();
+      }
+      if (0 == outRecords && inRecords > 0) {
+        LOG.info("Spec output bytes w/o records. Using input record count");
+        outRecords = inRecords;
+      }
+      factory =
+        new AvgRecordFactory(outBytes, outRecords, context.getConfiguration());
+      ratio = outRecords / (1.0 * inRecords);
+      acc = 0.0;
+    }
+    @Override
+    protected void reduce(GridmixKey key, Iterable<GridmixRecord> values,
+        Context context) throws IOException, InterruptedException {
+      for (GridmixRecord ignored : values) {
+        acc += ratio;
+        while (acc >= 1.0 && factory.next(null, val)) {
+          context.write(NullWritable.get(), val);
+          acc -= 1.0;
+        }
+      }
+    }
+    @Override
+    protected void cleanup(Context context)
+        throws IOException, InterruptedException {
+      val.setSeed(r.nextLong());
+      while (factory.next(null, val)) {
+        context.write(NullWritable.get(), val);
+        val.setSeed(r.nextLong());
+      }
+    }
+  }
+
+  static class GridmixRecordReader
+      extends RecordReader<NullWritable,GridmixRecord> {
+
+    private RecordFactory factory;
+    private final Random r = new Random();
+    private final GridmixRecord val = new GridmixRecord();
+
+    public GridmixRecordReader() { }
+
+    @Override
+    public void initialize(InputSplit genericSplit, TaskAttemptContext ctxt)
+            throws IOException, InterruptedException {
+      final GridmixSplit split = (GridmixSplit)genericSplit;
+      final Configuration conf = ctxt.getConfiguration();
+      factory = new ReadRecordFactory(split.getLength(),
+          split.getInputRecords(), new FileQueue(split, conf), conf);
+    }
+
+    @Override
+    public boolean nextKeyValue() throws IOException {
+      val.setSeed(r.nextLong());
+      return factory.next(null, val);
+    }
+    @Override
+    public float getProgress() throws IOException {
+      return factory.getProgress();
+    }
+    @Override
+    public NullWritable getCurrentKey() {
+      return NullWritable.get();
+    }
+    @Override
+    public GridmixRecord getCurrentValue() {
+      return val;
+    }
+    @Override
+    public void close() throws IOException {
+      factory.close();
+    }
+  }
+
+  static class GridmixInputFormat
+      extends InputFormat<NullWritable,GridmixRecord> {
+
+    @Override
+    public List<InputSplit> getSplits(JobContext jobCtxt) throws IOException {
+      return pullDescription(jobCtxt.getConfiguration().getInt(
+            "gridmix.job.seq", -1));
+    }
+    @Override
+    public RecordReader<NullWritable,GridmixRecord> createRecordReader(
+        InputSplit split, final TaskAttemptContext taskContext)
+        throws IOException {
+      return new GridmixRecordReader();
+    }
+  }
+
+  static class RawBytesOutputFormat<K>
+      extends FileOutputFormat<K,GridmixRecord> {
+
+    @Override
+    public RecordWriter<K,GridmixRecord> getRecordWriter(
+        TaskAttemptContext job) throws IOException {
+
+      Path file = getDefaultWorkFile(job, "");
+      FileSystem fs = file.getFileSystem(job.getConfiguration());
+      final FSDataOutputStream fileOut = fs.create(file, false);
+      return new RecordWriter<K,GridmixRecord>() {
+        @Override
+        public void write(K ignored, GridmixRecord value)
+            throws IOException {
+          value.writeRandom(fileOut, value.getSize());
+        }
+        @Override
+        public void close(TaskAttemptContext ctxt) throws IOException {
+          fileOut.close();
+        }
+      };
+    }
+  }
+
+  // TODO replace with ThreadLocal submitter?
+  private static final ConcurrentHashMap<Integer,List<InputSplit>> descCache =
+    new ConcurrentHashMap<Integer,List<InputSplit>>();
+
+  static void pushDescription(int seq, List<InputSplit> splits) {
+    if (null != descCache.putIfAbsent(seq, splits)) {
+      throw new IllegalArgumentException("Description exists for id " + seq);
+    }
+  }
+
+  static List<InputSplit> pullDescription(int seq) {
+    return descCache.remove(seq);
+  }
+
+  // not nesc when TL
+  static void clearAll() {
+    descCache.clear();
+  }
+
+  void buildSplits(FilePool inputDir) throws IOException {
+    long mapInputBytesTotal = 0L;
+    long mapOutputBytesTotal = 0L;
+    long mapOutputRecordsTotal = 0L;
+    final JobStory jobdesc = getJobDesc();
+    if (null == jobdesc) {
+      return;
+    }
+    final int maps = jobdesc.getNumberMaps();
+    final int reds = jobdesc.getNumberReduces();
+    for (int i = 0; i < maps; ++i) {
+      final TaskInfo info = jobdesc.getTaskInfo(TaskType.MAP, i);
+      mapInputBytesTotal += info.getInputBytes();
+      mapOutputBytesTotal += info.getOutputBytes();
+      mapOutputRecordsTotal += info.getOutputRecords();
+    }
+    final double[] reduceRecordRatio = new double[reds];
+    final double[] reduceByteRatio = new double[reds];
+    for (int i = 0; i < reds; ++i) {
+      final TaskInfo info = jobdesc.getTaskInfo(TaskType.REDUCE, i);
+      reduceByteRatio[i] = info.getInputBytes() / (1.0 * mapOutputBytesTotal);
+      reduceRecordRatio[i] =
+        info.getInputRecords() / (1.0 * mapOutputRecordsTotal);
+    }
+    final InputStriper striper = new InputStriper(inputDir, mapInputBytesTotal);
+    final List<InputSplit> splits = new ArrayList<InputSplit>();
+    for (int i = 0; i < maps; ++i) {
+      final int nSpec = reds / maps + ((reds % maps) > i ? 1 : 0);
+      final long[] specBytes = new long[nSpec];
+      final long[] specRecords = new long[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();
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(String.format("SPEC(%d) %d -> %d %d %d", id(), i,
+              i + j * maps, info.getOutputRecords(), info.getOutputBytes()));
+        }
+      }
+      final TaskInfo info = jobdesc.getTaskInfo(TaskType.MAP, i);
+      splits.add(new GridmixSplit(striper.splitFor(inputDir,
+              info.getInputBytes(), 3), maps, i,
+            info.getInputBytes(), info.getInputRecords(),
+            info.getOutputBytes(), info.getOutputRecords(),
+            reduceByteRatio, reduceRecordRatio, specBytes, specRecords));
+    }
+    pushDescription(id(), splits);
+  }
+
+}

+ 258 - 0
src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/GridmixKey.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;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.io.DataInputBuffer;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableUtils;
+import org.apache.hadoop.io.WritableComparator;
+
+class GridmixKey extends GridmixRecord {
+  static final byte REDUCE_SPEC = 0;
+  static final byte DATA = 1;
+
+  static final int META_BYTES = 1;
+
+  private byte type;
+  private int partition; // NOT serialized
+  private Spec spec = new Spec();
+
+  GridmixKey() {
+    this(DATA, 1, 0L);
+  }
+  GridmixKey(byte type, int size, long seed) {
+    super(size, seed);
+    this.type = type;
+    // setting type may change pcnt random bytes
+    setSize(size);
+  }
+
+  @Override
+  public int getSize() {
+    switch (type) {
+      case REDUCE_SPEC:
+        return super.getSize() + spec.getSize() + META_BYTES;
+      case DATA:
+        return super.getSize() + META_BYTES;
+      default:
+        throw new IllegalStateException("Invalid type: " + type);
+    }
+  }
+
+  @Override
+  public void setSize(int size) {
+    switch (type) {
+      case REDUCE_SPEC:
+        super.setSize(size - (META_BYTES + spec.getSize()));
+        break;
+      case DATA:
+        super.setSize(size - META_BYTES);
+        break;
+      default:
+        throw new IllegalStateException("Invalid type: " + type);
+    }
+  }
+
+  /**
+   * Partition is not serialized.
+   */
+  public int getPartition() {
+    return partition;
+  }
+  public void setPartition(int partition) {
+    this.partition = partition;
+  }
+
+  public long getReduceInputRecords() {
+    assert REDUCE_SPEC == getType();
+    return spec.rec_in;
+  }
+  public void setReduceInputRecords(long rec_in) {
+    assert REDUCE_SPEC == getType();
+    final int origSize = getSize();
+    spec.rec_in = rec_in;
+    setSize(origSize);
+  }
+
+  public long getReduceOutputRecords() {
+    assert REDUCE_SPEC == getType();
+    return spec.rec_out;
+  }
+  public void setReduceOutputRecords(long rec_out) {
+    assert REDUCE_SPEC == getType();
+    final int origSize = getSize();
+    spec.rec_out = rec_out;
+    setSize(origSize);
+  }
+
+  public long getReduceOutputBytes() {
+    assert REDUCE_SPEC == getType();
+    return spec.bytes_out;
+  };
+  public void setReduceOutputBytes(long b_out) {
+    assert REDUCE_SPEC == getType();
+    final int origSize = getSize();
+    spec.bytes_out = b_out;
+    setSize(origSize);
+  }
+
+  public byte getType() {
+    return type;
+  }
+  public void setType(byte type) throws IOException {
+    final int origSize = getSize();
+    switch (type) {
+      case REDUCE_SPEC:
+      case DATA:
+        this.type = type;
+        break;
+      default:
+        throw new IOException("Invalid type: " + type);
+    }
+    setSize(origSize);
+  }
+
+  public void setSpec(Spec spec) {
+    assert REDUCE_SPEC == getType();
+    final int origSize = getSize();
+    this.spec.set(spec);
+    setSize(origSize);
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    super.readFields(in);
+    setType(in.readByte());
+    if (REDUCE_SPEC == getType()) {
+      spec.readFields(in);
+    }
+  }
+  @Override
+  public void write(DataOutput out) throws IOException {
+    super.write(out);
+    final byte t = getType();
+    out.writeByte(t);
+    if (REDUCE_SPEC == t) {
+      spec.write(out);
+    }
+  }
+  int fixedBytes() {
+    return super.fixedBytes() +
+      (REDUCE_SPEC == getType() ? spec.getSize() : 0) + META_BYTES;
+  }
+  @Override
+  public int compareTo(GridmixRecord other) {
+    final GridmixKey o = (GridmixKey) other;
+    final byte t1 = getType();
+    final byte t2 = o.getType();
+    if (t1 != t2) {
+      return t1 - t2;
+    }
+    return super.compareTo(other);
+  }
+
+  /**
+   * Note that while the spec is not explicitly included, changing the spec
+   * may change its size, which will affect equality.
+   */
+  @Override
+  public boolean equals(Object other) {
+    if (this == other) {
+      return true;
+    }
+    if (other != null && other.getClass() == getClass()) {
+      final GridmixKey o = ((GridmixKey)other);
+      return getType() == o.getType() && super.equals(o);
+    }
+    return false;
+  }
+
+  @Override
+  public int hashCode() {
+    return super.hashCode() ^ getType();
+  }
+
+  public static class Spec implements Writable {
+    long rec_in;
+    long rec_out;
+    long bytes_out;
+    public Spec() { }
+
+    public void set(Spec other) {
+      rec_in = other.rec_in;
+      bytes_out = other.bytes_out;
+      rec_out = other.rec_out;
+    }
+
+    public int getSize() {
+      return WritableUtils.getVIntSize(rec_in) +
+             WritableUtils.getVIntSize(rec_out) +
+             WritableUtils.getVIntSize(bytes_out);
+    }
+
+    @Override
+    public void readFields(DataInput in) throws IOException {
+      rec_in = WritableUtils.readVLong(in);
+      rec_out = WritableUtils.readVLong(in);
+      bytes_out = WritableUtils.readVLong(in);
+    }
+
+    @Override
+    public void write(DataOutput out) throws IOException {
+      WritableUtils.writeVLong(out, rec_in);
+      WritableUtils.writeVLong(out, rec_out);
+      WritableUtils.writeVLong(out, bytes_out);
+    }
+  }
+
+  public static class Comparator extends GridmixRecord.Comparator {
+
+    private final DataInputBuffer di = new DataInputBuffer();
+    private final byte[] reset = di.getData();
+
+    public Comparator() {
+      super(GridmixKey.class);
+    }
+
+    @Override
+    public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+      try {
+        di.reset(b1, s1, l1);
+        final int x1 = WritableUtils.readVInt(di);
+        di.reset(b2, s2, l2);
+        final int x2 = WritableUtils.readVInt(di);
+        final int ret = (b1[s1 + x1] != b2[s2 + x2])
+          ? b1[s1 + x1] - b2[s2 + x2]
+          : super.compare(b1, s1, x1, b2, s2, x2);
+        di.reset(reset, 0, 0);
+        return ret;
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      }
+    }
+
+    static {
+      WritableComparator.define(GridmixKey.class, new Comparator());
+    }
+  }
+}
+

+ 215 - 0
src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/GridmixRecord.java

@@ -0,0 +1,215 @@
+/**
+ * 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.DataInput;
+import java.io.DataOutput;
+import java.io.EOFException;
+import java.io.IOException;
+import java.util.Arrays;
+
+import org.apache.hadoop.io.DataInputBuffer;
+import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.io.WritableComparator;
+import org.apache.hadoop.io.WritableUtils;
+
+class GridmixRecord implements WritableComparable<GridmixRecord> {
+
+  private static final int FIXED_BYTES = 1;
+  private int size = -1;
+  private long seed;
+  private final DataInputBuffer dib =
+    new DataInputBuffer();
+  private final DataOutputBuffer dob =
+    new DataOutputBuffer(Long.SIZE / Byte.SIZE);
+  private byte[] literal = dob.getData();
+
+  GridmixRecord() {
+    this(1, 0L);
+  }
+
+  GridmixRecord(int size, long seed) {
+    this.seed = seed;
+    setSizeInternal(size);
+  }
+
+  public int getSize() {
+    return size;
+  }
+
+  public void setSize(int size) {
+    setSizeInternal(size);
+  }
+
+  private void setSizeInternal(int size) {
+    this.size = Math.max(1, size);
+    try {
+      seed = maskSeed(seed, this.size);
+      dob.reset();
+      dob.writeLong(seed);
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  public final void setSeed(long seed) {
+    this.seed = seed;
+  }
+
+  /** Marsaglia, 2003. */
+  long nextRand(long x) {
+    x ^= (x << 13);
+    x ^= (x >>> 7);
+    return (x ^= (x << 17));
+  }
+
+  public void writeRandom(DataOutput out, final int size) throws IOException {
+    long tmp = seed;
+    out.writeLong(tmp);
+    int i = size - (Long.SIZE / Byte.SIZE);
+    while (i > Long.SIZE / Byte.SIZE - 1) {
+      tmp = nextRand(tmp);
+      out.writeLong(tmp);
+      i -= Long.SIZE / Byte.SIZE;
+    }
+    for (tmp = nextRand(tmp); i > 0; --i) {
+      out.writeByte((int)(tmp & 0xFF));
+      tmp >>>= Byte.SIZE;
+    }
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    size = WritableUtils.readVInt(in);
+    int payload = size - WritableUtils.getVIntSize(size);
+    if (payload > Long.SIZE / Byte.SIZE) {
+      seed = in.readLong();
+      payload -= Long.SIZE / Byte.SIZE;
+    } else {
+      Arrays.fill(literal, (byte)0);
+      in.readFully(literal, 0, payload);
+      dib.reset(literal, 0, literal.length);
+      seed = dib.readLong();
+      payload = 0;
+    }
+    final int vBytes = in.skipBytes(payload);
+    if (vBytes != payload) {
+      throw new EOFException("Expected " + payload + ", read " + vBytes);
+    }
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    // data bytes including vint encoding
+    WritableUtils.writeVInt(out, size);
+    final int payload = size - WritableUtils.getVIntSize(size);
+    if (payload > Long.SIZE / Byte.SIZE) {
+      writeRandom(out, payload);
+    } else if (payload > 0) {
+      out.write(literal, 0, payload);
+    }
+  }
+
+  @Override
+  public int compareTo(GridmixRecord other) {
+    return compareSeed(other.seed,
+        Math.max(0, other.getSize() - other.fixedBytes()));
+  }
+
+  int fixedBytes() {
+    // min vint size
+    return FIXED_BYTES;
+  }
+
+  private static long maskSeed(long sd, int sz) {
+    // Don't use fixedBytes here; subclasses will set intended random len
+    if (sz <= FIXED_BYTES) {
+      sd = 0L;
+    } else if (sz < Long.SIZE / Byte.SIZE + FIXED_BYTES) {
+      final int tmp = sz - FIXED_BYTES;
+      final long mask = (1L << (Byte.SIZE * tmp)) - 1;
+      sd &= mask << (Byte.SIZE * (Long.SIZE / Byte.SIZE - tmp));
+    }
+    return sd;
+  }
+
+  int compareSeed(long jSeed, int jSize) {
+    final int iSize = Math.max(0, getSize() - fixedBytes());
+    final int seedLen = Math.min(iSize, jSize) + FIXED_BYTES;
+    jSeed = maskSeed(jSeed, seedLen);
+    long iSeed = maskSeed(seed, seedLen);
+    final int cmplen = Math.min(iSize, jSize);
+    for (int i = 0; i < cmplen; i += Byte.SIZE) {
+      final int k = cmplen - i;
+      for (long j = Long.SIZE - Byte.SIZE;
+          j >= Math.max(0, Long.SIZE / Byte.SIZE - k) * Byte.SIZE;
+          j -= Byte.SIZE) {
+        final int xi = (int)((iSeed >>> j) & 0xFFL);
+        final int xj = (int)((jSeed >>> j) & 0xFFL);
+        if (xi != xj) {
+          return xi - xj;
+        }
+      }
+      iSeed = nextRand(iSeed);
+      jSeed = nextRand(jSeed);
+    }
+    return iSize - jSize;
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (this == other) {
+      return true;
+    }
+    if (other != null && other.getClass() == getClass()) {
+      final GridmixRecord o = ((GridmixRecord)other);
+      return getSize() == o.getSize() && seed == o.seed;
+    }
+    return false;
+  }
+
+  @Override
+  public int hashCode() {
+    return (int)(seed * getSize());
+  }
+
+  public static class Comparator extends WritableComparator {
+
+    public Comparator() {
+      super(GridmixRecord.class);
+    }
+
+    public Comparator(Class<? extends WritableComparable<?>> sub) {
+      super(sub);
+    }
+
+    public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+      int n1 = WritableUtils.decodeVIntSize(b1[s1]);
+      int n2 = WritableUtils.decodeVIntSize(b2[s2]);
+      n1 -= WritableUtils.getVIntSize(n1);
+      n2 -= WritableUtils.getVIntSize(n2);
+      return compareBytes(b1, s1+n1, l1-n1, b2, s2+n2, l2-n2);
+    }
+
+    static {
+      WritableComparator.define(GridmixRecord.class, new Comparator());
+    }
+  }
+
+}

+ 147 - 0
src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/GridmixSplit.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.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.io.WritableUtils;
+
+class GridmixSplit extends CombineFileSplit {
+  private int id;
+  private int nSpec;
+  private int maps;
+  private int reduces;
+  private long inputRecords;
+  private long outputBytes;
+  private long outputRecords;
+  private long maxMemory;
+  private double[] reduceBytes = new double[0];
+  private double[] reduceRecords = new double[0];
+
+  // Spec for reduces id mod this
+  private long[] reduceOutputBytes = new long[0];
+  private long[] reduceOutputRecords = new long[0];
+
+  GridmixSplit() {
+    super();
+  }
+
+  public GridmixSplit(CombineFileSplit cfsplit, int maps, int id,
+      long inputBytes, long inputRecords, long outputBytes,
+      long outputRecords, double[] reduceBytes, double[] reduceRecords,
+      long[] reduceOutputBytes, long[] reduceOutputRecords)
+      throws IOException {
+    super(cfsplit);
+    this.id = id;
+    this.maps = maps;
+    reduces = reduceBytes.length;
+    this.inputRecords = inputRecords;
+    this.outputBytes = outputBytes;
+    this.outputRecords = outputRecords;
+    this.reduceBytes = reduceBytes;
+    this.reduceRecords = reduceRecords;
+    nSpec = reduceOutputBytes.length;
+    this.reduceOutputBytes = reduceOutputBytes;
+    this.reduceOutputRecords = reduceOutputRecords;
+  }
+  public int getId() {
+    return id;
+  }
+  public int getMapCount() {
+    return maps;
+  }
+  public long getInputRecords() {
+    return inputRecords;
+  }
+  public long[] getOutputBytes() {
+    if (0 == reduces) {
+      return new long[] { outputBytes };
+    }
+    final long[] ret = new long[reduces];
+    for (int i = 0; i < reduces; ++i) {
+      ret[i] = Math.round(outputBytes * reduceBytes[i]);
+    }
+    return ret;
+  }
+  public long[] getOutputRecords() {
+    if (0 == reduces) {
+      return new long[] { outputRecords };
+    }
+    final long[] ret = new long[reduces];
+    for (int i = 0; i < reduces; ++i) {
+      ret[i] = Math.round(outputRecords * reduceRecords[i]);
+    }
+    return ret;
+  }
+  public long getReduceBytes(int i) {
+    return reduceOutputBytes[i];
+  }
+  public long getReduceRecords(int i) {
+    return reduceOutputRecords[i];
+  }
+  @Override
+  public void write(DataOutput out) throws IOException {
+    super.write(out);
+    WritableUtils.writeVInt(out, id);
+    WritableUtils.writeVInt(out, maps);
+    WritableUtils.writeVLong(out, inputRecords);
+    WritableUtils.writeVLong(out, outputBytes);
+    WritableUtils.writeVLong(out, outputRecords);
+    WritableUtils.writeVLong(out, maxMemory);
+    WritableUtils.writeVInt(out, reduces);
+    for (int i = 0; i < reduces; ++i) {
+      out.writeDouble(reduceBytes[i]);
+      out.writeDouble(reduceRecords[i]);
+    }
+    WritableUtils.writeVInt(out, nSpec);
+    for (int i = 0; i < nSpec; ++i) {
+      WritableUtils.writeVLong(out, reduceOutputBytes[i]);
+      WritableUtils.writeVLong(out, reduceOutputRecords[i]);
+    }
+  }
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    super.readFields(in);
+    id = WritableUtils.readVInt(in);
+    maps = WritableUtils.readVInt(in);
+    inputRecords = WritableUtils.readVLong(in);
+    outputBytes = WritableUtils.readVLong(in);
+    outputRecords = WritableUtils.readVLong(in);
+    maxMemory = WritableUtils.readVLong(in);
+    reduces = WritableUtils.readVInt(in);
+    if (reduceBytes.length < reduces) {
+      reduceBytes = new double[reduces];
+      reduceRecords = new double[reduces];
+    }
+    for (int i = 0; i < reduces; ++i) {
+      reduceBytes[i] = in.readDouble();
+      reduceRecords[i] = in.readDouble();
+    }
+    nSpec = WritableUtils.readVInt(in);
+    if (reduceOutputBytes.length < nSpec) {
+      reduceOutputBytes = new long[nSpec];
+      reduceOutputRecords = new long[nSpec];
+    }
+    for (int i = 0; i < nSpec; ++i) {
+      reduceOutputBytes[i] = WritableUtils.readVLong(in);
+      reduceOutputRecords[i] = WritableUtils.readVLong(in);
+    }
+  }
+}

+ 126 - 0
src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/InputStriper.java

@@ -0,0 +1,126 @@
+/**
+ * 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.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map.Entry;
+
+import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+/**
+ * Given a {@link #FilePool}, obtain a set of files capable of satisfying
+ * a full set of splits, then iterate over each source to fill the request.
+ */
+class InputStriper {
+  public static final Log LOG = LogFactory.getLog(InputStriper.class);
+  int idx;
+  long currentStart;
+  FileStatus current;
+  final List<FileStatus> files = new ArrayList<FileStatus>();
+
+  /**
+   * @param inputDir Pool from which files are requested.
+   * @param mapBytes Sum of all expected split requests.
+   */
+  InputStriper(FilePool inputDir, long mapBytes)
+      throws IOException {
+    final long inputBytes = inputDir.getInputFiles(mapBytes, files);
+    if (mapBytes > inputBytes) {
+      LOG.warn("Using " + inputBytes + "/" + mapBytes + " bytes");
+    }
+    if (files.isEmpty() && mapBytes > 0) {
+      throw new IOException("Failed to satisfy request for " + mapBytes);
+    }
+    current = files.isEmpty() ? null : files.get(0);
+  }
+
+  /**
+   * @param inputDir Pool used to resolve block locations.
+   * @param bytes Target byte count
+   * @param nLocs Number of block locations per split.
+   * @return A set of files satisfying the byte count, with locations weighted
+   *         to the dominating proportion of input bytes.
+   */
+  CombineFileSplit splitFor(FilePool inputDir, long bytes, int nLocs)
+      throws IOException {
+    final ArrayList<Path> paths = new ArrayList<Path>();
+    final ArrayList<Long> start = new ArrayList<Long>();
+    final ArrayList<Long> length = new ArrayList<Long>();
+    final HashMap<String,Double> sb = new HashMap<String,Double>();
+    do {
+      paths.add(current.getPath());
+      start.add(currentStart);
+      final long fromFile = Math.min(bytes, current.getLen() - currentStart);
+      length.add(fromFile);
+      for (BlockLocation loc :
+          inputDir.locationsFor(current, currentStart, fromFile)) {
+        final double tedium = loc.getLength() / (1.0 * bytes);
+        for (String l : loc.getHosts()) {
+          Double j = sb.get(l);
+          if (null == j) {
+            sb.put(l, tedium);
+          } else {
+            sb.put(l, j.doubleValue() + tedium);
+          }
+        }
+      }
+      currentStart += fromFile;
+      bytes -= fromFile;
+      if (current.getLen() - currentStart == 0) {
+        current = files.get(++idx % files.size());
+        currentStart = 0;
+      }
+    } while (bytes > 0);
+    final ArrayList<Entry<String,Double>> sort =
+      new ArrayList<Entry<String,Double>>(sb.entrySet());
+    Collections.sort(sort, hostRank);
+    final String[] hosts = new String[Math.min(nLocs, sort.size())];
+    for (int i = 0; i < nLocs && i < sort.size(); ++i) {
+      hosts[i] = sort.get(i).getKey();
+    }
+    return new CombineFileSplit(paths.toArray(new Path[0]),
+        toLongArray(start), toLongArray(length), hosts);
+  }
+
+  private long[] toLongArray(final ArrayList<Long> sigh) {
+    final long[] ret = new long[sigh.size()];
+    for (int i = 0; i < ret.length; ++i) {
+      ret[i] = sigh.get(i);
+    }
+    return ret;
+  }
+
+  static final Comparator<Entry<String,Double>> hostRank =
+    new Comparator<Entry<String,Double>>() {
+      public int compare(Entry<String,Double> a, Entry<String,Double> b) {
+          final double va = a.getValue();
+          final double vb = b.getValue();
+          return va > vb ? -1 : va < vb ? 1 : 0;
+        }
+    };
+}

+ 110 - 0
src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/IntermediateRecordFactory.java

@@ -0,0 +1,110 @@
+/**
+ * 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;
+
+/**
+ * Factory passing reduce specification as its last record.
+ */
+class IntermediateRecordFactory extends RecordFactory {
+
+  private final GridmixKey.Spec spec;
+  private final RecordFactory factory;
+  private final int partition;
+  private final long targetRecords;
+  private boolean done = false;
+  private long accRecords = 0L;
+
+  /**
+   * @param targetBytes Expected byte count.
+   * @param targetRecords Expected record count; will emit spec records after
+   *                      this boundary is passed.
+   * @param partition Reduce to which records are emitted.
+   * @param spec Specification to emit.
+   * @param conf Unused.
+   */
+  public IntermediateRecordFactory(long targetBytes, long targetRecords,
+      int partition, GridmixKey.Spec spec, Configuration conf) {
+    this(new AvgRecordFactory(targetBytes, targetRecords, conf), partition,
+        targetRecords, spec, conf);
+  }
+
+  /**
+   * @param factory Factory from which byte/record counts are obtained.
+   * @param partition Reduce to which records are emitted.
+   * @param targetRecords Expected record count; will emit spec records after
+   *                      this boundary is passed.
+   * @param spec Specification to emit.
+   * @param conf Unused.
+   */
+  public IntermediateRecordFactory(RecordFactory factory, int partition,
+      long targetRecords, GridmixKey.Spec spec, Configuration conf) {
+    this.spec = spec;
+    this.factory = factory;
+    this.partition = partition;
+    this.targetRecords = targetRecords;
+  }
+
+  @Override
+  public boolean next(GridmixKey key, GridmixRecord val) throws IOException {
+    assert key != null;
+    final boolean rslt = factory.next(key, val);
+    ++accRecords;
+    if (rslt) {
+      if (accRecords < targetRecords) {
+        key.setType(GridmixKey.DATA);
+      } else {
+        final int orig = key.getSize();
+        key.setType(GridmixKey.REDUCE_SPEC);
+        spec.rec_in = accRecords;
+        key.setSpec(spec);
+        val.setSize(val.getSize() - (key.getSize() - orig));
+        // reset counters
+        accRecords = 0L;
+        spec.bytes_out = 0L;
+        spec.rec_out = 0L;
+        done = true;
+      }
+    } else if (!done) {
+      // ensure spec emitted
+      key.setType(GridmixKey.REDUCE_SPEC);
+      key.setPartition(partition);
+      key.setSize(0);
+      val.setSize(0);
+      spec.rec_in = 0L;
+      key.setSpec(spec);
+      done = true;
+      return true;
+    }
+    key.setPartition(partition);
+    return rslt;
+  }
+
+  @Override
+  public float getProgress() throws IOException {
+    return factory.getProgress();
+  }
+
+  @Override
+  public void close() throws IOException {
+    factory.close();
+  }
+}

+ 272 - 0
src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/JobFactory.java

@@ -0,0 +1,272 @@
+/**
+ * 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.io.InputStream;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.JobID;
+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.Pre21JobHistoryConstants.Values;
+import org.apache.hadoop.tools.rumen.Pre21JobHistoryConstants;
+import org.apache.hadoop.tools.rumen.TaskAttemptInfo;
+import org.apache.hadoop.tools.rumen.TaskInfo;
+import org.apache.hadoop.tools.rumen.ZombieJobProducer;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+
+/**
+ * Component reading job traces generated by Rumen. Each job in the trace is
+ * assigned a sequence number and given a submission time relative to the
+ * job that preceded it. Jobs are enqueued in the JobSubmitter provided at
+ * construction.
+ * @see org.apache.hadoop.tools.rumen.HadoopLogsAnalyzer
+ */
+class JobFactory implements Gridmix.Component<Void> {
+
+  public static final Log LOG = LogFactory.getLog(JobFactory.class);
+
+  private final Path scratch;
+  private final float rateFactor;
+  private final Configuration conf;
+  private final ReaderThread rThread;
+  private final AtomicInteger sequence;
+  private final JobSubmitter submitter;
+  private final CountDownLatch startFlag;
+  private volatile IOException error = null;
+  protected final JobStoryProducer jobProducer;
+
+  /**
+   * Creating a new instance does not start the thread.
+   * @param submitter Component to which deserialized jobs are passed
+   * @param jobTrace Stream of job traces with which to construct a
+   *                 {@link org.apache.hadoop.tools.rumen.ZombieJobProducer}
+   * @param scratch Directory into which to write output from simulated jobs
+   * @param conf Config passed to all jobs to be submitted
+   * @param startFlag Latch released from main to start pipeline
+   */
+  public JobFactory(JobSubmitter submitter, InputStream jobTrace,
+      Path scratch, Configuration conf, CountDownLatch startFlag)
+      throws IOException {
+    this(submitter, new ZombieJobProducer(jobTrace, null), scratch, conf,
+        startFlag);
+  }
+
+  /**
+   * Constructor permitting JobStoryProducer to be mocked.
+   * @param submitter Component to which deserialized jobs are passed
+   * @param jobProducer Producer generating JobStory objects.
+   * @param scratch Directory into which to write output from simulated jobs
+   * @param conf Config passed to all jobs to be submitted
+   * @param startFlag Latch released from main to start pipeline
+   */
+  protected JobFactory(JobSubmitter submitter, JobStoryProducer jobProducer,
+      Path scratch, Configuration conf, CountDownLatch startFlag) {
+    sequence = new AtomicInteger(0);
+    this.scratch = scratch;
+    this.rateFactor = conf.getFloat(Gridmix.GRIDMIX_SUB_MUL, 1.0f);
+    this.jobProducer = jobProducer;
+    this.conf = new Configuration(conf);
+    this.submitter = submitter;
+    this.startFlag = startFlag;
+    this.rThread = new ReaderThread();
+  }
+
+  static class MinTaskInfo extends TaskInfo {
+    public MinTaskInfo(TaskInfo info) {
+      super(info.getInputBytes(), info.getInputRecords(),
+            info.getOutputBytes(), info.getOutputRecords(),
+            info.getTaskMemory());
+    }
+    public long getInputBytes() {
+      return Math.max(0, super.getInputBytes());
+    }
+    public int getInputRecords() {
+      return Math.max(0, super.getInputRecords());
+    }
+    public long getOutputBytes() {
+      return Math.max(0, super.getOutputBytes());
+    }
+    public int getOutputRecords() {
+      return Math.max(0, super.getOutputRecords());
+    }
+    public long getTaskMemory() {
+      return Math.max(0, super.getTaskMemory());
+    }
+  }
+
+  static class FilterJobStory implements JobStory {
+
+    protected final JobStory job;
+
+    public FilterJobStory(JobStory job) {
+      this.job = job;
+    }
+    public JobConf getJobConf() { return job.getJobConf(); }
+    public String getName() { return job.getName(); }
+    public JobID getJobID() { return job.getJobID(); }
+    public String getUser() { return job.getUser(); }
+    public long getSubmissionTime() { return job.getSubmissionTime(); }
+    public InputSplit[] getInputSplits() { return job.getInputSplits(); }
+    public int getNumberMaps() { return job.getNumberMaps(); }
+    public int getNumberReduces() { return job.getNumberReduces(); }
+    public TaskInfo getTaskInfo(TaskType taskType, int taskNumber) {
+      return job.getTaskInfo(taskType, taskNumber);
+    }
+    public TaskAttemptInfo getTaskAttemptInfo(TaskType taskType, int taskNumber,
+        int taskAttemptNumber) {
+      return job.getTaskAttemptInfo(taskType, taskNumber, taskAttemptNumber);
+    }
+    public TaskAttemptInfo getMapTaskAttemptInfoAdjusted(
+        int taskNumber, int taskAttemptNumber, int locality) {
+      return job.getMapTaskAttemptInfoAdjusted(
+          taskNumber, taskAttemptNumber, locality);
+    }
+    public Values getOutcome() {
+      return job.getOutcome();
+    }
+  }
+
+  /**
+   * Worker thread responsible for reading descriptions, assigning sequence
+   * numbers, and normalizing time.
+   */
+  private class ReaderThread extends Thread {
+
+    public ReaderThread() {
+      super("GridmixJobFactory");
+    }
+
+    private JobStory getNextJobFiltered() throws IOException {
+      JobStory job;
+      do {
+        job = jobProducer.getNextJob();
+      } while (job != null
+          && (job.getOutcome() != Pre21JobHistoryConstants.Values.SUCCESS ||
+              job.getSubmissionTime() < 0));
+      return null == job ? null : new FilterJobStory(job) {
+          @Override
+          public TaskInfo getTaskInfo(TaskType taskType, int taskNumber) {
+            return new MinTaskInfo(this.job.getTaskInfo(taskType, taskNumber));
+          }
+        };
+    }
+
+    @Override
+    public void run() {
+      try {
+        startFlag.await();
+        if (Thread.currentThread().isInterrupted()) {
+          return;
+        }
+        final long initTime = TimeUnit.MILLISECONDS.convert(
+            System.nanoTime(), TimeUnit.NANOSECONDS);
+        LOG.debug("START @ " + initTime);
+        long first = -1;
+        long last = -1;
+        while (!Thread.currentThread().isInterrupted()) {
+          try {
+            final JobStory job = getNextJobFiltered();
+            if (null == job) {
+              return;
+            }
+            if (first < 0) {
+              first = job.getSubmissionTime();
+            }
+            final long current = job.getSubmissionTime();
+            if (current < last) {
+              LOG.warn("Job " + job.getJobID() + " out of order");
+              continue;
+            }
+            last = current;
+            submitter.add(new GridmixJob(conf, initTime +
+                  Math.round(rateFactor * (current - first)),
+                job, scratch, sequence.getAndIncrement()));
+          } catch (IOException e) {
+            JobFactory.this.error = e;
+            return;
+          }
+        }
+      } catch (InterruptedException e) {
+        // exit thread; ignore any jobs remaining in the trace
+        return;
+      } finally {
+        IOUtils.cleanup(null, jobProducer);
+      }
+    }
+  }
+
+  /**
+   * Obtain the error that caused the thread to exit unexpectedly.
+   */
+  public IOException error() {
+    return error;
+  }
+
+  /**
+   * Add is disabled.
+   * @throws UnsupportedOperationException
+   */
+  public void add(Void ignored) {
+    throw new UnsupportedOperationException(getClass().getName() +
+        " is at the start of the pipeline and accepts no events");
+  }
+
+  /**
+   * Start the reader thread, wait for latch if necessary.
+   */
+  public void start() {
+    rThread.start();
+  }
+
+  /**
+   * Wait for the reader thread to exhaust the job trace.
+   */
+  public void join(long millis) throws InterruptedException {
+    rThread.join(millis);
+  }
+
+  /**
+   * Interrupt the reader thread.
+   */
+  public void shutdown() {
+    rThread.interrupt();
+  }
+
+  /**
+   * Interrupt the reader thread. This requires no special consideration, as
+   * the thread has no pending work queue.
+   */
+  public void abort() {
+    // Currently no special work
+    rThread.interrupt();
+  }
+
+}

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

@@ -0,0 +1,243 @@
+/**
+ * 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.nio.channels.ClosedByInterruptException;
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Queue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import org.apache.hadoop.mapreduce.Job;
+
+/**
+ * Component accepting submitted, running jobs and responsible for
+ * monitoring jobs for success and failure. Once a job is submitted, it is
+ * polled for status until complete. If a job is complete, then the monitor
+ * thread returns immediately to the queue. If not, the monitor will sleep
+ * for some duration.
+ */
+class JobMonitor implements Gridmix.Component<Job> {
+
+  public static final Log LOG = LogFactory.getLog(JobMonitor.class);
+
+  private final Queue<Job> mJobs;
+  private final MonitorThread mThread;
+  private final BlockingQueue<Job> runningJobs;
+  private final long pollDelayMillis;
+  private boolean graceful = false;
+  private boolean shutdown = false;
+
+  /**
+   * Create a JobMonitor with a default polling interval of 5s.
+   */
+  public JobMonitor() {
+    this(5, TimeUnit.SECONDS);
+  }
+
+  /**
+   * Create a JobMonitor that sleeps for the specified duration after
+   * polling a still-running job.
+   * @param pollDelay Delay after polling a running job
+   * @param unit Time unit for pollDelaySec (rounded to milliseconds)
+   */
+  public JobMonitor(int pollDelay, TimeUnit unit) {
+    mThread = new MonitorThread();
+    runningJobs = new LinkedBlockingQueue<Job>();
+    mJobs = new LinkedList<Job>();
+    this.pollDelayMillis = TimeUnit.MILLISECONDS.convert(pollDelay, unit);
+  }
+
+  /**
+   * Add a job to the polling queue.
+   */
+  public void add(Job job) throws InterruptedException {
+    runningJobs.put(job);
+  }
+
+  /**
+   * Temporary hook for recording job success.
+   */
+  protected void onSuccess(Job job) {
+    LOG.info(job.getJobName() + " (" + job.getJobID() + ")" + " success");
+  }
+
+  /**
+   * Temporary hook for recording job failure.
+   */
+  protected void onFailure(Job job) {
+    LOG.info(job.getJobName() + " (" + job.getJobID() + ")" + " failure");
+  }
+
+  /**
+   * If shutdown before all jobs have completed, any still-running jobs
+   * may be extracted from the component.
+   * @throws IllegalStateException If monitoring thread is still running.
+   * @return Any jobs submitted and not known to have completed.
+   */
+  List<Job> getRemainingJobs() {
+    if (mThread.isAlive()) {
+      LOG.warn("Internal error: Polling running monitor for jobs");
+    }
+    synchronized (mJobs) {
+      return new ArrayList<Job>(mJobs);
+    }
+  }
+
+  /**
+   * Monitoring thread pulling running jobs from the component and into
+   * a queue to be polled for status.
+   */
+  private class MonitorThread extends Thread {
+
+    public MonitorThread() {
+      super("GridmixJobMonitor");
+    }
+
+    /**
+     * Check a job for success or failure.
+     */
+    public void process(Job job) throws IOException, InterruptedException {
+      if (job.isSuccessful()) {
+        onSuccess(job);
+      } else {
+        onFailure(job);
+      }
+    }
+
+    @Override
+    public void run() {
+      boolean graceful;
+      boolean shutdown;
+      while (true) {
+        try {
+          synchronized (mJobs) {
+            graceful = JobMonitor.this.graceful;
+            shutdown = JobMonitor.this.shutdown;
+            runningJobs.drainTo(mJobs);
+          }
+
+          // shutdown conditions; either shutdown requested and all jobs
+          // have completed or abort requested and there are recently
+          // submitted jobs not in the monitored set
+          if (shutdown) {
+            if (!graceful) {
+              while (!runningJobs.isEmpty()) {
+                synchronized (mJobs) {
+                  runningJobs.drainTo(mJobs);
+                }
+              }
+              break;
+            } else if (mJobs.isEmpty()) {
+              break;
+            }
+          }
+          while (!mJobs.isEmpty()) {
+            Job job;
+            synchronized (mJobs) {
+              job = mJobs.poll();
+            }
+            try {
+              if (job.isComplete()) {
+                process(job);
+                continue;
+              }
+            } catch (IOException e) {
+              if (e.getCause() instanceof ClosedByInterruptException) {
+                // Job doesn't throw InterruptedException, but RPC socket layer
+                // is blocking and may throw a wrapped Exception if this thread
+                // is interrupted. Since the lower level cleared the flag,
+                // reset it here
+                Thread.currentThread().interrupt();
+              } else {
+                LOG.warn("Lost job " + (null == job.getJobName()
+                     ? "<unknown>" : job.getJobName()), e);
+                continue;
+              }
+            }
+            synchronized (mJobs) {
+              if (!mJobs.offer(job)) {
+                LOG.error("Lost job " + (null == job.getJobName()
+                     ? "<unknown>" : job.getJobName())); // should never
+                                                         // happen
+              }
+            }
+            break;
+          }
+          try {
+            TimeUnit.MILLISECONDS.sleep(pollDelayMillis);
+          } catch (InterruptedException e) {
+            shutdown = true;
+            continue;
+          }
+        } catch (Throwable e) {
+          LOG.warn("Unexpected exception: ", e);
+        }
+      }
+    }
+  }
+
+  /**
+   * Start the internal, monitoring thread.
+   */
+  public void start() {
+    mThread.start();
+  }
+
+  /**
+   * Wait for the monitor to halt, assuming shutdown or abort have been
+   * called. Note that, since submission may be sporatic, this will hang
+   * if no form of shutdown has been requested.
+   */
+  public void join(long millis) throws InterruptedException {
+    mThread.join(millis);
+  }
+
+  /**
+   * Drain all submitted jobs to a queue and stop the monitoring thread.
+   * Upstream submitter is assumed dead.
+   */
+  public void abort() {
+    synchronized (mJobs) {
+      graceful = false;
+      shutdown = true;
+    }
+    mThread.interrupt();
+  }
+
+  /**
+   * When all monitored jobs have completed, stop the monitoring thread.
+   * Upstream submitter is assumed dead.
+   */
+  public void shutdown() {
+    synchronized (mJobs) {
+      graceful = true;
+      shutdown = true;
+    }
+    mThread.interrupt();
+  }
+}
+
+

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

@@ -0,0 +1,177 @@
+/**
+ * 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.nio.channels.ClosedByInterruptException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+/**
+ * Component accepting deserialized job traces, computing split data, and
+ * submitting to the cluster on deadline. Each job added from an upstream
+ * factory must be submitted to the cluster by the deadline recorded on it.
+ * Once submitted, jobs must be added to a downstream component for
+ * monitoring.
+ */
+class JobSubmitter implements Gridmix.Component<GridmixJob> {
+
+  public static final Log LOG = LogFactory.getLog(JobSubmitter.class);
+
+  final Semaphore sem;
+  private final FilePool inputDir;
+  private final JobMonitor monitor;
+  private final ExecutorService sched;
+  private volatile boolean shutdown = false;
+
+  /**
+   * Initialize the submission component with downstream monitor and pool of
+   * files from which split data may be read.
+   * @param monitor Monitor component to which jobs should be passed
+   * @param threads Number of submission threads
+   *   See {@link Gridmix#GRIDMIX_SUB_THR}.
+   * @param queueDepth Max depth of pending work queue
+   *   See {@link Gridmix#GRIDMIX_QUE_DEP}.
+   * @param inputDir Set of files from which split data may be mined for
+   *   synthetic jobs.
+   */
+  public JobSubmitter(JobMonitor monitor, int threads, int queueDepth,
+      FilePool inputDir) {
+    sem = new Semaphore(queueDepth);
+    sched = new ThreadPoolExecutor(threads, threads, 0L,
+        TimeUnit.MILLISECONDS, new LinkedBlockingQueue<Runnable>());
+    this.inputDir = inputDir;
+    this.monitor = monitor;
+  }
+
+  /**
+   * Runnable wrapping a job to be submitted to the cluster.
+   */
+  private class SubmitTask implements Runnable {
+
+    final GridmixJob job;
+    public SubmitTask(GridmixJob job) {
+      this.job = job;
+    }
+    public void run() {
+      try {
+        // pre-compute split information
+        try {
+          job.buildSplits(inputDir);
+        } catch (IOException e) {
+          LOG.warn("Failed to submit " + job.getJob().getJobName(), e);
+          return;
+        }
+        // Sleep until deadline
+        long nsDelay = job.getDelay(TimeUnit.NANOSECONDS);
+        while (nsDelay > 0) {
+          TimeUnit.NANOSECONDS.sleep(nsDelay);
+          nsDelay = job.getDelay(TimeUnit.NANOSECONDS);
+        }
+        try {
+          // submit job
+          monitor.add(job.call());
+          LOG.debug("SUBMIT " + job + "@" + System.currentTimeMillis() +
+              " (" + job.getJob().getJobID() + ")");
+        } catch (IOException e) {
+          LOG.warn("Failed to submit " + job.getJob().getJobName(), e);
+          if (e.getCause() instanceof ClosedByInterruptException) {
+            throw new InterruptedException("Failed to submit " +
+                job.getJob().getJobName());
+          }
+        } catch (ClassNotFoundException e) {
+          LOG.warn("Failed to submit " + job.getJob().getJobName(), e);
+        }
+      } catch (InterruptedException e) {
+        // abort execution, remove splits if nesc
+        // TODO release ThdLoc
+        GridmixJob.pullDescription(job.id());
+        Thread.currentThread().interrupt();
+        return;
+      } finally {
+        sem.release();
+      }
+    }
+  }
+
+  /**
+   * Enqueue the job to be submitted per the deadline associated with it.
+   */
+  public void add(final GridmixJob job) throws InterruptedException {
+    final boolean addToQueue = !shutdown;
+    if (addToQueue) {
+      final SubmitTask task = new SubmitTask(job);
+      sem.acquire();
+      try {
+        sched.execute(task);
+      } catch (RejectedExecutionException e) {
+        sem.release();
+      }
+    }
+  }
+
+  /**
+   * (Re)scan the set of input files from which splits are derived.
+   */
+  public void refreshFilePool() throws IOException {
+    inputDir.refresh();
+  }
+
+  /**
+   * Does nothing, as the threadpool is already initialized and waiting for
+   * work from the upstream factory.
+   */
+  public void start() { }
+
+  /**
+   * Continue running until all queued jobs have been submitted to the
+   * cluster.
+   */
+  public void join(long millis) throws InterruptedException {
+    if (!shutdown) {
+      throw new IllegalStateException("Cannot wait for active submit thread");
+    }
+    sched.awaitTermination(millis, TimeUnit.MILLISECONDS);
+  }
+
+  /**
+   * Finish all jobs pending submission, but do not accept new work.
+   */
+  public void shutdown() {
+    // complete pending tasks, but accept no new tasks
+    shutdown = true;
+    sched.shutdown();
+  }
+
+  /**
+   * Discard pending work, including precomputed work waiting to be
+   * submitted.
+   */
+  public void abort() {
+    //pendingJobs.clear();
+    shutdown = true;
+    sched.shutdownNow();
+  }
+}

+ 85 - 0
src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/ReadRecordFactory.java

@@ -0,0 +1,85 @@
+/**
+ * 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.io.InputStream;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.IOUtils;
+
+/**
+ * For every record consumed, read key + val bytes from the stream provided.
+ */
+class ReadRecordFactory extends RecordFactory {
+
+  /**
+   * Size of internal, scratch buffer to read from internal stream.
+   */
+  public static final String GRIDMIX_READ_BUF_SIZE = "gridmix.read.buffer.size";
+
+  private final byte[] buf;
+  private final InputStream src;
+  private final RecordFactory factory;
+
+  /**
+   * @param targetBytes Expected byte count.
+   * @param targetRecords Expected record count.
+   * @param src Stream to read bytes.
+   * @param conf Used to establish read buffer size. @see #GRIDMIX_READ_BUF_SIZE
+   */
+  public ReadRecordFactory(long targetBytes, long targetRecords,
+      InputStream src, Configuration conf) {
+    this(new AvgRecordFactory(targetBytes, targetRecords, conf), src, conf);
+  }
+
+  /**
+   * @param factory Factory to draw record sizes.
+   * @param src Stream to read bytes.
+   * @param conf Used to establish read buffer size. @see #GRIDMIX_READ_BUF_SIZE
+   */
+  public ReadRecordFactory(RecordFactory factory, InputStream src,
+      Configuration conf) {
+    this.src = src;
+    this.factory = factory;
+    buf = new byte[conf.getInt(GRIDMIX_READ_BUF_SIZE, 64 * 1024)];
+  }
+
+  @Override
+  public boolean next(GridmixKey key, GridmixRecord val) throws IOException {
+    if (!factory.next(key, val)) {
+      return false;
+    }
+    for (int len = (null == key ? 0 : key.getSize()) + val.getSize();
+         len > 0; len -= buf.length) {
+      IOUtils.readFully(src, buf, 0, Math.min(buf.length, len));
+    }
+    return true;
+  }
+
+  @Override
+  public float getProgress() throws IOException {
+    return factory.getProgress();
+  }
+
+  @Override
+  public void close() throws IOException {
+    IOUtils.cleanup(null, src);
+    factory.close();
+  }
+}

+ 40 - 0
src/contrib/gridmix/src/java/org/apache/hadoop/mapred/gridmix/RecordFactory.java

@@ -0,0 +1,40 @@
+/**
+ * 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.Closeable;
+import java.io.IOException;
+
+/**
+ * Interface for producing records as inputs and outputs to tasks.
+ */
+abstract class RecordFactory implements Closeable {
+
+  /**
+   * Transform the given record or perform some operation.
+   * @return true if the record should be emitted.
+   */
+  public abstract boolean next(GridmixKey key, GridmixRecord val)
+    throws IOException;
+
+  /**
+   * Estimate of exhausted record capacity.
+   */
+  public abstract float getProgress() throws IOException;
+
+}

+ 277 - 0
src/contrib/gridmix/src/test/org/apache/hadoop/mapred/gridmix/DebugJobFactory.java

@@ -0,0 +1,277 @@
+/**
+ * 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.util.ArrayList;
+import java.util.Arrays;
+import java.util.Random;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.JobID;
+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.TaskAttemptInfo;
+import org.apache.hadoop.tools.rumen.TaskInfo;
+import org.apache.hadoop.tools.rumen.Pre21JobHistoryConstants.Values;;
+
+
+/**
+ * Component generating random job traces for testing on a single node.
+ */
+class DebugJobFactory extends JobFactory {
+
+  public DebugJobFactory(JobSubmitter submitter, Path scratch, int numJobs,
+      Configuration conf, CountDownLatch startFlag) throws IOException {
+    super(submitter, new DebugJobProducer(numJobs, conf), scratch, conf,
+        startFlag);
+  }
+
+  ArrayList<JobStory> getSubmitted() {
+    return ((DebugJobProducer)jobProducer).submitted;
+  }
+
+  private static class DebugJobProducer implements JobStoryProducer {
+    final ArrayList<JobStory> submitted;
+    private final Configuration conf;
+    private final AtomicInteger numJobs;
+
+    public DebugJobProducer(int numJobs, Configuration conf) {
+      super();
+      this.conf = conf;
+      this.numJobs = new AtomicInteger(numJobs);
+      this.submitted = new ArrayList<JobStory>();
+    }
+
+    @Override
+    public JobStory getNextJob() throws IOException {
+      if (numJobs.getAndDecrement() > 0) {
+        final MockJob ret = new MockJob(conf);
+        submitted.add(ret);
+        return ret;
+      }
+      return null;
+    }
+
+    @Override
+    public void close() { }
+  }
+
+  static double[] getDistr(Random r, double mindist, int size) {
+    assert 0.0 <= mindist && mindist <= 1.0;
+    final double min = mindist / size;
+    final double rem = 1.0 - min * size;
+    final double[] tmp = new double[size];
+    for (int i = 0; i < tmp.length - 1; ++i) {
+      tmp[i] = r.nextDouble() * rem;
+    }
+    tmp[tmp.length - 1] = rem;
+    Arrays.sort(tmp);
+
+    final double[] ret = new double[size];
+    ret[0] = tmp[0] + min;
+    for (int i = 1; i < size; ++i) {
+      ret[i] = tmp[i] - tmp[i-1] + min;
+    }
+    return ret;
+  }
+
+  /**
+   * Generate random task data for a synthetic job.
+   */
+  static class MockJob implements JobStory {
+
+    static final int MIN_REC = 1 << 14;
+    static final int MIN_BYTES = 1 << 20;
+    static final int VAR_REC = 1 << 14;
+    static final int VAR_BYTES = 4 << 20;
+    static final int MAX_MAP = 5;
+    static final int MAX_RED = 3;
+
+    static void initDist(Random r, double min, int[] recs, long[] bytes,
+        long tot_recs, long tot_bytes) {
+      final double[] recs_dist = getDistr(r, min, recs.length);
+      final double[] bytes_dist = getDistr(r, min, recs.length);
+      long totalbytes = 0L;
+      int totalrecs = 0;
+      for (int i = 0; i < recs.length; ++i) {
+        recs[i] = (int) Math.round(tot_recs * recs_dist[i]);
+        bytes[i] = Math.round(tot_bytes * bytes_dist[i]);
+        totalrecs += recs[i];
+        totalbytes += bytes[i];
+      }
+      // Add/remove excess
+      recs[0] += totalrecs - tot_recs;
+      bytes[0] += totalbytes - tot_bytes;
+      if (LOG.isInfoEnabled()) {
+        LOG.info("DIST: " + Arrays.toString(recs) + " " +
+            tot_recs + "/" + totalrecs + " " +
+            Arrays.toString(bytes) + " " + tot_bytes + "/" + totalbytes);
+      }
+    }
+
+    private static final AtomicInteger seq = new AtomicInteger(0);
+    // set timestamps in the past
+    private static final AtomicLong timestamp =
+      new AtomicLong(System.currentTimeMillis() -
+        TimeUnit.MILLISECONDS.convert(60, TimeUnit.DAYS));
+
+    private final int id;
+    private final String name;
+    private final int[] m_recsIn, m_recsOut, r_recsIn, r_recsOut;
+    private final long[] m_bytesIn, m_bytesOut, r_bytesIn, r_bytesOut;
+    private final long submitTime;
+
+    public MockJob(Configuration conf) {
+      final Random r = new Random();
+      final long seed = r.nextLong();
+      r.setSeed(seed);
+      id = seq.getAndIncrement();
+      name = String.format("MOCKJOB%05d", id);
+      LOG.info(name + " (" + seed + ")");
+      submitTime = timestamp.addAndGet(TimeUnit.MILLISECONDS.convert(
+            r.nextInt(10), TimeUnit.SECONDS));
+
+      m_recsIn = new int[r.nextInt(MAX_MAP) + 1];
+      m_bytesIn = new long[m_recsIn.length];
+      m_recsOut = new int[m_recsIn.length];
+      m_bytesOut = new long[m_recsIn.length];
+
+      r_recsIn = new int[r.nextInt(MAX_RED) + 1];
+      r_bytesIn = new long[r_recsIn.length];
+      r_recsOut = new int[r_recsIn.length];
+      r_bytesOut = new long[r_recsIn.length];
+
+      // map input
+      final long map_recs = r.nextInt(VAR_REC) + MIN_REC;
+      final long map_bytes = r.nextInt(VAR_BYTES) + MIN_BYTES;
+      initDist(r, 0.5, m_recsIn, m_bytesIn, map_recs, map_bytes);
+
+      // shuffle
+      final long shuffle_recs = r.nextInt(VAR_REC) + MIN_REC;
+      final long shuffle_bytes = r.nextInt(VAR_BYTES) + MIN_BYTES;
+      initDist(r, 0.4, m_recsOut, m_bytesOut, shuffle_recs, shuffle_bytes);
+      initDist(r, 0.8, r_recsIn, r_bytesIn, shuffle_recs, shuffle_bytes);
+
+      // reduce output
+      final long red_recs = r.nextInt(VAR_REC) + MIN_REC;
+      final long red_bytes = r.nextInt(VAR_BYTES) + MIN_BYTES;
+      initDist(r, 0.4, r_recsOut, r_bytesOut, red_recs, red_bytes);
+
+      if (LOG.isDebugEnabled()) {
+        int iMapBTotal = 0, oMapBTotal = 0, iRedBTotal = 0, oRedBTotal = 0;
+        int iMapRTotal = 0, oMapRTotal = 0, iRedRTotal = 0, oRedRTotal = 0;
+        for (int i = 0; i < m_recsIn.length; ++i) {
+          iMapRTotal += m_recsIn[i];
+          iMapBTotal += m_bytesIn[i];
+          oMapRTotal += m_recsOut[i];
+          oMapBTotal += m_bytesOut[i];
+        }
+        for (int i = 0; i < r_recsIn.length; ++i) {
+          iRedRTotal += r_recsIn[i];
+          iRedBTotal += r_bytesIn[i];
+          oRedRTotal += r_recsOut[i];
+          oRedBTotal += r_bytesOut[i];
+        }
+        LOG.debug(String.format("%s: M (%03d) %6d/%10d -> %6d/%10d" +
+                                   " R (%03d) %6d/%10d -> %6d/%10d @%d", name,
+            m_bytesIn.length, iMapRTotal, iMapBTotal, oMapRTotal, oMapBTotal,
+            r_bytesIn.length, iRedRTotal, iRedBTotal, oRedRTotal, oRedBTotal,
+            submitTime));
+      }
+    }
+
+    @Override
+    public String getName() {
+      return name;
+    }
+
+    @Override
+    public String getUser() {
+      return "FOOBAR";
+    }
+
+    @Override
+    public JobID getJobID() {
+      return new JobID("job_mock_" + name, id);
+    }
+
+    @Override
+    public Values getOutcome() {
+      return Values.SUCCESS;
+    }
+
+    @Override
+    public long getSubmissionTime() {
+      return submitTime;
+    }
+
+    @Override
+    public int getNumberMaps() {
+      return m_bytesIn.length;
+    }
+
+    @Override
+    public int getNumberReduces() {
+      return r_bytesIn.length;
+    }
+
+    @Override
+    public TaskInfo getTaskInfo(TaskType taskType, int taskNumber) {
+      switch (taskType) {
+        case MAP:
+          return new TaskInfo(m_bytesIn[taskNumber], m_recsIn[taskNumber],
+              m_bytesOut[taskNumber], m_recsOut[taskNumber], -1);
+        case REDUCE:
+          return new TaskInfo(r_bytesIn[taskNumber], r_recsIn[taskNumber],
+              r_bytesOut[taskNumber], r_recsOut[taskNumber], -1);
+        default:
+          throw new IllegalArgumentException("Not interested");
+      }
+    }
+
+    @Override
+    public InputSplit[] getInputSplits() {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public TaskAttemptInfo getTaskAttemptInfo(TaskType taskType,
+        int taskNumber, int taskAttemptNumber) {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public TaskAttemptInfo getMapTaskAttemptInfoAdjusted(int taskNumber,
+        int taskAttemptNumber, int locality) {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public org.apache.hadoop.mapred.JobConf getJobConf() {
+      throw new UnsupportedOperationException();
+    }
+  }
+}

+ 188 - 0
src/contrib/gridmix/src/test/org/apache/hadoop/mapred/gridmix/TestFilePool.java

@@ -0,0 +1,188 @@
+/**
+ * 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.io.OutputStream;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Random;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import static org.junit.Assert.*;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+public class TestFilePool {
+
+  static final Log LOG = LogFactory.getLog(TestFileQueue.class);
+  static final int NFILES = 26;
+  static final Path base = getBaseDir();
+
+  static Path getBaseDir() {
+    try {
+      final Configuration conf = new Configuration();
+      final FileSystem fs = FileSystem.getLocal(conf).getRaw();
+      return new Path(System.getProperty("test.build.data", "/tmp"),
+          "testFilePool").makeQualified(fs);
+    } catch (IOException e) {
+      fail();
+    }
+    return null;
+  }
+
+  @BeforeClass
+  public static void setup() throws IOException {
+    final Configuration conf = new Configuration();
+    final FileSystem fs = FileSystem.getLocal(conf).getRaw();
+    fs.delete(base, true);
+    final Random r = new Random();
+    final long seed = r.nextLong();
+    r.setSeed(seed);
+    LOG.info("seed: " + seed);
+    fs.mkdirs(base);
+    for (int i = 0; i < NFILES; ++i) {
+      Path file = base;
+      for (double d = 0.6; d > 0.0; d *= 0.8) {
+        if (r.nextDouble() < d) {
+          file = new Path(base, Integer.toString(r.nextInt(3)));
+          continue;
+        }
+        break;
+      }
+      OutputStream out = null;
+      try {
+        out = fs.create(new Path(file, "" + (char)('A' + i)));
+        final byte[] b = new byte[1024];
+        Arrays.fill(b, (byte)('A' + i));
+        for (int len = ((i % 13) + 1) * 1024; len > 0; len -= 1024) {
+          out.write(b);
+        }
+      } finally {
+        if (out != null) {
+          out.close();
+        }
+      }
+    }
+  }
+
+  @AfterClass
+  public static void cleanup() throws IOException {
+    final Configuration conf = new Configuration();
+    final FileSystem fs = FileSystem.getLocal(conf).getRaw();
+    fs.delete(base, true);
+  }
+
+  @Test
+  public void testUnsuitable() throws Exception {
+    try {
+      final Configuration conf = new Configuration();
+      // all files 13k or less
+      conf.setLong(FilePool.GRIDMIX_MIN_FILE, 14 * 1024);
+      final FilePool pool = new FilePool(conf, base);
+      pool.refresh();
+    } catch (IOException e) {
+      return;
+    }
+    fail();
+  }
+
+  @Test
+  public void testPool() throws Exception {
+    final Random r = new Random();
+    final Configuration conf = new Configuration();
+    conf.setLong(FilePool.GRIDMIX_MIN_FILE, 3 * 1024);
+    final FilePool pool = new FilePool(conf, base);
+    pool.refresh();
+    final ArrayList<FileStatus> files = new ArrayList<FileStatus>();
+
+    // ensure 1k, 2k files excluded
+    final int expectedPoolSize = (NFILES / 2 * (NFILES / 2 + 1) - 6) * 1024;
+    assertEquals(expectedPoolSize, pool.getInputFiles(Long.MAX_VALUE, files));
+    assertEquals(NFILES - 4, files.size());
+
+    // exact match
+    files.clear();
+    assertEquals(expectedPoolSize, pool.getInputFiles(expectedPoolSize, files));
+
+    // match random within 12k
+    files.clear();
+    final long rand = r.nextInt(expectedPoolSize);
+    assertTrue("Missed: " + rand,
+        (NFILES / 2) * 1024 > rand - pool.getInputFiles(rand, files));
+
+    // all files
+    conf.setLong(FilePool.GRIDMIX_MIN_FILE, 0);
+    pool.refresh();
+    files.clear();
+    assertEquals((NFILES / 2 * (NFILES / 2 + 1)) * 1024,
+        pool.getInputFiles(Long.MAX_VALUE, files));
+  }
+
+  void checkSplitEq(FileSystem fs, CombineFileSplit split, long bytes)
+      throws Exception {
+    long splitBytes = 0L;
+    HashSet<Path> uniq = new HashSet<Path>();
+    for (int i = 0; i < split.getNumPaths(); ++i) {
+      splitBytes += split.getLength(i);
+      assertTrue(
+          split.getLength(i) <= fs.getFileStatus(split.getPath(i)).getLen());
+      assertFalse(uniq.contains(split.getPath(i)));
+      uniq.add(split.getPath(i));
+    }
+    assertEquals(bytes, splitBytes);
+  }
+
+  @Test
+  public void testStriper() throws Exception {
+    final Random r = new Random();
+    final Configuration conf = new Configuration();
+    final FileSystem fs = FileSystem.getLocal(conf).getRaw();
+    conf.setLong(FilePool.GRIDMIX_MIN_FILE, 3 * 1024);
+    final FilePool pool = new FilePool(conf, base) {
+      @Override
+      public BlockLocation[] locationsFor(FileStatus stat, long start, long len)
+          throws IOException {
+        return new BlockLocation[] { new BlockLocation() };
+      }
+    };
+    pool.refresh();
+
+    final int expectedPoolSize = (NFILES / 2 * (NFILES / 2 + 1) - 6) * 1024;
+    final InputStriper striper = new InputStriper(pool, expectedPoolSize);
+    int last = 0;
+    for (int i = 0; i < expectedPoolSize;
+        last = Math.min(expectedPoolSize - i, r.nextInt(expectedPoolSize))) {
+      checkSplitEq(fs, striper.splitFor(pool, last, 0), last);
+      i += last;
+    }
+    final InputStriper striper2 = new InputStriper(pool, expectedPoolSize);
+    checkSplitEq(fs, striper2.splitFor(pool, expectedPoolSize, 0),
+        expectedPoolSize);
+  }
+
+}

+ 142 - 0
src/contrib/gridmix/src/test/org/apache/hadoop/mapred/gridmix/TestFileQueue.java

@@ -0,0 +1,142 @@
+/**
+ * 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.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.Arrays;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import static org.junit.Assert.*;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+public class TestFileQueue {
+
+  static final Log LOG = LogFactory.getLog(TestFileQueue.class);
+  static final int NFILES = 4;
+  static final int BLOCK = 256;
+  static final Path[] paths = new Path[NFILES];
+  static final String[] loc = new String[NFILES];
+  static final long[] start = new long[NFILES];
+  static final long[] len = new long[NFILES];
+
+  @BeforeClass
+  public static void setup() throws IOException {
+    final Configuration conf = new Configuration();
+    final FileSystem fs = FileSystem.getLocal(conf).getRaw();
+    final Path p = new Path(System.getProperty("test.build.data", "/tmp"),
+        "testFileQueue").makeQualified(fs);
+    fs.delete(p, true);
+    final byte[] b = new byte[BLOCK];
+    for (int i = 0; i < NFILES; ++i) {
+      Arrays.fill(b, (byte)('A' + i));
+      paths[i] = new Path(p, "" + (char)('A' + i));
+      OutputStream f = null;
+      try {
+        f = fs.create(paths[i]);
+        f.write(b);
+      } finally {
+        if (f != null) {
+          f.close();
+        }
+      }
+    }
+  }
+
+  @AfterClass
+  public static void cleanup() throws IOException {
+    final Configuration conf = new Configuration();
+    final FileSystem fs = FileSystem.getLocal(conf).getRaw();
+    final Path p = new Path(System.getProperty("test.build.data", "/tmp"),
+        "testFileQueue").makeQualified(fs);
+    fs.delete(p, true);
+  }
+
+  static ByteArrayOutputStream fillVerif() throws IOException {
+    final byte[] b = new byte[BLOCK];
+    final ByteArrayOutputStream out = new ByteArrayOutputStream();
+    for (int i = 0; i < NFILES; ++i) {
+      Arrays.fill(b, (byte)('A' + i));
+      out.write(b, 0, (int)len[i]);
+    }
+    return out;
+  }
+
+  @Test
+  public void testRepeat() throws Exception {
+    final Configuration conf = new Configuration();
+    Arrays.fill(loc, "");
+    Arrays.fill(start, 0L);
+    Arrays.fill(len, BLOCK);
+
+    final ByteArrayOutputStream out = fillVerif();
+    final FileQueue q =
+      new FileQueue(new CombineFileSplit(paths, start, len, loc), conf);
+    final byte[] verif = out.toByteArray();
+    final byte[] check = new byte[2 * NFILES * BLOCK];
+    q.read(check, 0, NFILES * BLOCK);
+    assertArrayEquals(verif, Arrays.copyOf(check, NFILES * BLOCK));
+
+    final byte[] verif2 = new byte[2 * NFILES * BLOCK];
+    System.arraycopy(verif, 0, verif2, 0, verif.length);
+    System.arraycopy(verif, 0, verif2, verif.length, verif.length);
+    q.read(check, 0, 2 * NFILES * BLOCK);
+    assertArrayEquals(verif2, check);
+
+  }
+
+  @Test
+  public void testUneven() throws Exception {
+    final Configuration conf = new Configuration();
+    Arrays.fill(loc, "");
+    Arrays.fill(start, 0L);
+    Arrays.fill(len, BLOCK);
+
+    final int B2 = BLOCK / 2;
+    for (int i = 0; i < NFILES; i += 2) {
+      start[i] += B2;
+      len[i] -= B2;
+    }
+    final FileQueue q =
+      new FileQueue(new CombineFileSplit(paths, start, len, loc), conf);
+    final ByteArrayOutputStream out = fillVerif();
+    final byte[] verif = out.toByteArray();
+    final byte[] check = new byte[NFILES / 2 * BLOCK + NFILES / 2 * B2];
+    q.read(check, 0, verif.length);
+    assertArrayEquals(verif, Arrays.copyOf(check, verif.length));
+    q.read(check, 0, verif.length);
+    assertArrayEquals(verif, Arrays.copyOf(check, verif.length));
+  }
+
+  @Test
+  public void testEmpty() throws Exception {
+    final Configuration conf = new Configuration();
+    // verify OK if unused
+    final FileQueue q = new FileQueue(new CombineFileSplit(
+          new Path[0], new long[0], new long[0], new String[0]), conf);
+  }
+
+}

+ 277 - 0
src/contrib/gridmix/src/test/org/apache/hadoop/mapred/gridmix/TestGridmixRecord.java

@@ -0,0 +1,277 @@
+/**
+ * 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.util.Arrays;
+import java.util.Random;
+
+import org.junit.Test;
+import static org.junit.Assert.*;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import org.apache.hadoop.io.DataInputBuffer;
+import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.io.WritableComparator;
+import org.apache.hadoop.io.WritableUtils;
+
+public class TestGridmixRecord {
+  private static final Log LOG = LogFactory.getLog(TestGridmixRecord.class);
+
+  static void lengthTest(GridmixRecord x, GridmixRecord y, int min,
+      int max) throws Exception {
+    final Random r = new Random();
+    final long seed = r.nextLong();
+    r.setSeed(seed);
+    LOG.info("length: " + seed);
+    final DataInputBuffer in = new DataInputBuffer();
+    final DataOutputBuffer out1 = new DataOutputBuffer();
+    final DataOutputBuffer out2 = new DataOutputBuffer();
+    for (int i = min; i < max; ++i) {
+      setSerialize(x, r.nextLong(), i, out1);
+      // check write
+      assertEquals(i, out1.getLength());
+      // write to stream
+      x.write(out2);
+      // check read
+      in.reset(out1.getData(), 0, out1.getLength());
+      y.readFields(in);
+      assertEquals(i, x.getSize());
+      assertEquals(i, y.getSize());
+    }
+    // check stream read
+    in.reset(out2.getData(), 0, out2.getLength());
+    for (int i = min; i < max; ++i) {
+      y.readFields(in);
+      assertEquals(i, y.getSize());
+    }
+  }
+
+  static void randomReplayTest(GridmixRecord x, GridmixRecord y, int min,
+      int max) throws Exception {
+    final Random r = new Random();
+    final long seed = r.nextLong();
+    r.setSeed(seed);
+    LOG.info("randReplay: " + seed);
+    final DataOutputBuffer out1 = new DataOutputBuffer();
+    for (int i = min; i < max; ++i) {
+      final int s = out1.getLength();
+      x.setSeed(r.nextLong());
+      x.setSize(i);
+      x.write(out1);
+      assertEquals(i, out1.getLength() - s);
+    }
+    final DataInputBuffer in = new DataInputBuffer();
+    in.reset(out1.getData(), 0, out1.getLength());
+    final DataOutputBuffer out2 = new DataOutputBuffer();
+    // deserialize written records, write to separate buffer
+    for (int i = min; i < max; ++i) {
+      final int s = in.getPosition();
+      y.readFields(in);
+      assertEquals(i, in.getPosition() - s);
+      y.write(out2);
+    }
+    // verify written contents match
+    assertEquals(out1.getLength(), out2.getLength());
+    // assumes that writes will grow buffer deterministically
+    assertEquals("Bad test", out1.getData().length, out2.getData().length);
+    assertArrayEquals(out1.getData(), out2.getData());
+  }
+
+  static void eqSeedTest(GridmixRecord x, GridmixRecord y, int max)
+      throws Exception {
+    final Random r = new Random();
+    final long s = r.nextLong();
+    r.setSeed(s);
+    LOG.info("eqSeed: " + s);
+    assertEquals(x.fixedBytes(), y.fixedBytes());
+    final int min = x.fixedBytes() + 1;
+    final DataOutputBuffer out1 = new DataOutputBuffer();
+    final DataOutputBuffer out2 = new DataOutputBuffer();
+    for (int i = min; i < max; ++i) {
+      final long seed = r.nextLong();
+      setSerialize(x, seed, i, out1);
+      setSerialize(y, seed, i, out2);
+      assertEquals(x, y);
+      assertEquals(x.hashCode(), y.hashCode());
+
+      // verify written contents match
+      assertEquals(out1.getLength(), out2.getLength());
+      // assumes that writes will grow buffer deterministically
+      assertEquals("Bad test", out1.getData().length, out2.getData().length);
+      assertArrayEquals(out1.getData(), out2.getData());
+    }
+  }
+
+  static void binSortTest(GridmixRecord x, GridmixRecord y, int min,
+      int max, WritableComparator cmp) throws Exception {
+    final Random r = new Random();
+    final long s = r.nextLong();
+    r.setSeed(s);
+    LOG.info("sort: " + s);
+    final DataOutputBuffer out1 = new DataOutputBuffer();
+    final DataOutputBuffer out2 = new DataOutputBuffer();
+    for (int i = min; i < max; ++i) {
+      final long seed1 = r.nextLong();
+      setSerialize(x, seed1, i, out1);
+      assertEquals(0, x.compareSeed(seed1, Math.max(0, i - x.fixedBytes())));
+
+      final long seed2 = r.nextLong();
+      setSerialize(y, seed2, i, out2);
+      assertEquals(0, y.compareSeed(seed2, Math.max(0, i - x.fixedBytes())));
+
+      // for eq sized records, ensure byte cmp where req
+      final int chk = WritableComparator.compareBytes(
+          out1.getData(), 0, out1.getLength(),
+          out2.getData(), 0, out2.getLength());
+      assertEquals(chk, x.compareTo(y));
+      assertEquals(chk, cmp.compare(
+            out1.getData(), 0, out1.getLength(),
+            out2.getData(), 0, out2.getLength()));
+      // write second copy, compare eq
+      final int s1 = out1.getLength();
+      x.write(out1);
+      assertEquals(0, cmp.compare(out1.getData(), 0, s1,
+            out1.getData(), s1, out1.getLength() - s1));
+      final int s2 = out2.getLength();
+      y.write(out2);
+      assertEquals(0, cmp.compare(out2.getData(), 0, s2,
+            out2.getData(), s2, out2.getLength() - s2));
+      assertEquals(chk, cmp.compare(out1.getData(), 0, s1,
+            out2.getData(), s2, out2.getLength() - s2));
+    }
+  }
+
+  static void checkSpec(GridmixKey a, GridmixKey b) throws Exception {
+    final Random r = new Random();
+    final long s = r.nextLong();
+    r.setSeed(s);
+    LOG.info("spec: " + s);
+    final DataInputBuffer in = new DataInputBuffer();
+    final DataOutputBuffer out = new DataOutputBuffer();
+    a.setType(GridmixKey.REDUCE_SPEC);
+    b.setType(GridmixKey.REDUCE_SPEC);
+    for (int i = 0; i < 100; ++i) {
+      final int in_rec = r.nextInt(Integer.MAX_VALUE);
+      a.setReduceInputRecords(in_rec);
+      final int out_rec = r.nextInt(Integer.MAX_VALUE);
+      a.setReduceOutputRecords(out_rec);
+      final int out_bytes = r.nextInt(Integer.MAX_VALUE);
+      a.setReduceOutputBytes(out_bytes);
+      final int min = WritableUtils.getVIntSize(in_rec)
+                    + WritableUtils.getVIntSize(out_rec)
+                    + WritableUtils.getVIntSize(out_bytes);
+      assertEquals(min + 2, a.fixedBytes()); // meta + vint min
+      final int size = r.nextInt(1024) + a.fixedBytes() + 1;
+      setSerialize(a, r.nextLong(), size, out);
+      assertEquals(size, out.getLength());
+      assertTrue(a.equals(a));
+      assertEquals(0, a.compareTo(a));
+
+      in.reset(out.getData(), 0, out.getLength());
+
+      b.readFields(in);
+      assertEquals(size, b.getSize());
+      assertEquals(in_rec, b.getReduceInputRecords());
+      assertEquals(out_rec, b.getReduceOutputRecords());
+      assertEquals(out_bytes, b.getReduceOutputBytes());
+      assertTrue(a.equals(b));
+      assertEquals(0, a.compareTo(b));
+      assertEquals(a.hashCode(), b.hashCode());
+    }
+  }
+
+  static void setSerialize(GridmixRecord x, long seed, int size,
+      DataOutputBuffer out) throws IOException {
+    x.setSeed(seed);
+    x.setSize(size);
+    out.reset();
+    x.write(out);
+  }
+
+  @Test
+  public void testKeySpec() throws Exception {
+    final int min = 5;
+    final int max = 300;
+    final GridmixKey a = new GridmixKey(GridmixKey.REDUCE_SPEC, 1, 0L);
+    final GridmixKey b = new GridmixKey(GridmixKey.REDUCE_SPEC, 1, 0L);
+    lengthTest(a, b, min, max);
+    randomReplayTest(a, b, min, max);
+    binSortTest(a, b, min, max, new GridmixKey.Comparator());
+    // 2 fixed GR bytes, 1 type, 3 spec
+    eqSeedTest(a, b, max);
+    checkSpec(a, b);
+  }
+
+  @Test
+  public void testKeyData() throws Exception {
+    final int min = 2;
+    final int max = 300;
+    final GridmixKey a = new GridmixKey(GridmixKey.DATA, 1, 0L);
+    final GridmixKey b = new GridmixKey(GridmixKey.DATA, 1, 0L);
+    lengthTest(a, b, min, max);
+    randomReplayTest(a, b, min, max);
+    binSortTest(a, b, min, max, new GridmixKey.Comparator());
+    // 2 fixed GR bytes, 1 type
+    eqSeedTest(a, b, 300);
+  }
+
+  @Test
+  public void testBaseRecord() throws Exception {
+    final int min = 1;
+    final int max = 300;
+    final GridmixRecord a = new GridmixRecord();
+    final GridmixRecord b = new GridmixRecord();
+    lengthTest(a, b, min, max);
+    randomReplayTest(a, b, min, max);
+    binSortTest(a, b, min, max, new GridmixRecord.Comparator());
+    // 2 fixed GR bytes
+    eqSeedTest(a, b, 300);
+  }
+
+  public static void main(String[] argv) throws Exception {
+    boolean fail = false;
+    final TestGridmixRecord test = new TestGridmixRecord();
+    try { test.testKeySpec(); } catch (Exception e) {
+      fail = true;
+      e.printStackTrace();
+    }
+    try {test.testKeyData(); } catch (Exception e) {
+      fail = true;
+      e.printStackTrace();
+    }
+    try {test.testBaseRecord(); } catch (Exception e) {
+      fail = true;
+      e.printStackTrace();
+    }
+    System.exit(fail ? -1 : 0);
+  }
+
+  static void printDebug(GridmixRecord a, GridmixRecord b) throws IOException {
+    DataOutputBuffer out = new DataOutputBuffer();
+    a.write(out);
+    System.out.println("A " +
+        Arrays.toString(Arrays.copyOf(out.getData(), out.getLength())));
+    out.reset();
+    b.write(out);
+    System.out.println("B " +
+        Arrays.toString(Arrays.copyOf(out.getData(), out.getLength())));
+  }
+
+}

+ 322 - 0
src/contrib/gridmix/src/test/org/apache/hadoop/mapred/gridmix/TestGridmixSubmission.java

@@ -0,0 +1,322 @@
+/**
+ * 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.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 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.Path;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.mapred.Counters;
+import org.apache.hadoop.mapred.JobClient;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.JobID;
+import org.apache.hadoop.mapred.MiniMRCluster;
+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.TaskInfo;
+import org.apache.hadoop.util.ToolRunner;
+import static org.apache.hadoop.mapred.Task.Counter.*;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import static org.junit.Assert.*;
+
+import org.apache.commons.logging.LogFactory;
+import org.apache.commons.logging.impl.Log4JLogger;
+import org.apache.log4j.Level;
+
+public class TestGridmixSubmission {
+  {
+    ((Log4JLogger)LogFactory.getLog("org.apache.hadoop.mapred.gridmix")
+        ).getLogger().setLevel(Level.DEBUG);
+  }
+
+  private static FileSystem dfs = null;
+  private static MiniDFSCluster dfsCluster = null;
+  private static MiniMRCluster mrCluster = null;
+
+  private static final int NJOBS = 2;
+  private static final long GENDATA = 50; // in megabytes
+  private static final int GENSLOP = 100 * 1024; // +/- 100k for logs
+
+  @BeforeClass
+  public static void initCluster() throws IOException {
+    Configuration conf = new Configuration();
+    dfsCluster = new MiniDFSCluster(conf, 3, true, null);
+    dfs = dfsCluster.getFileSystem();
+    mrCluster = new MiniMRCluster(3, dfs.getUri().toString(), 1, null, null,
+        new JobConf(conf));
+  }
+
+  @AfterClass
+  public static void shutdownCluster() throws IOException {
+    if (mrCluster != null) {
+      mrCluster.shutdown();
+    }
+    if (dfsCluster != null) {
+      dfsCluster.shutdown();
+    }
+  }
+
+  static class TestMonitor extends JobMonitor {
+
+    static final long SLOPBYTES = 1024;
+    private final int expected;
+    private final BlockingQueue<Job> retiredJobs;
+
+    public TestMonitor(int expected) {
+      super();
+      this.expected = expected;
+      retiredJobs = new LinkedBlockingQueue<Job>();
+    }
+
+    public void verify(ArrayList<JobStory> submitted) throws Exception {
+      final ArrayList<Job> succeeded = new ArrayList<Job>();
+      assertEquals("Bad job count", expected, retiredJobs.drainTo(succeeded));
+      final HashMap<String,JobStory> sub = new HashMap<String,JobStory>();
+      for (JobStory spec : submitted) {
+        sub.put(spec.getName(), spec);
+      }
+      final JobClient client = new JobClient(mrCluster.createJobConf());
+      for (Job job : succeeded) {
+        final String jobname = job.getJobName();
+        if ("GRIDMIX_GENDATA".equals(jobname)) {
+          final Path in = new Path("foo").makeQualified(dfs);
+          final Path out = new Path("/gridmix").makeQualified(dfs);
+          final ContentSummary generated = dfs.getContentSummary(in);
+          assertTrue("Mismatched data gen", // +/- 100k for logs
+              (GENDATA << 20) < generated.getLength() + GENSLOP ||
+              (GENDATA << 20) > generated.getLength() - GENSLOP);
+          FileStatus[] outstat = dfs.listStatus(out);
+          assertEquals("Mismatched job count", NJOBS, outstat.length);
+          continue;
+        }
+        final JobStory spec =
+          sub.get(job.getJobName().replace("GRIDMIX", "MOCKJOB"));
+        assertNotNull("No spec for " + job.getJobName(), spec);
+        assertNotNull("No counters for " + job.getJobName(), job.getCounters());
+
+        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);
+        final TaskReport[] mReports =
+          client.getMapTaskReports(JobID.downgrade(job.getJobID()));
+        assertEquals("Mismatched map count", nMaps, mReports.length);
+        check(TaskType.MAP, job, spec, mReports,
+            0, 0, SLOPBYTES, nReds);
+
+        final TaskReport[] rReports =
+          client.getReduceTaskReports(JobID.downgrade(job.getJobID()));
+        assertEquals("Mismatched reduce count", nReds, rReports.length);
+        check(TaskType.REDUCE, job, spec, rReports,
+            nMaps * SLOPBYTES, 2 * nMaps, 0, 0);
+      }
+    }
+
+    public void check(final TaskType type, Job job, JobStory spec,
+          final TaskReport[] runTasks,
+          long extraInputBytes, int extraInputRecords,
+          long extraOutputBytes, int extraOutputRecords) throws Exception {
+
+      long[] runInputRecords = new long[runTasks.length];
+      long[] runInputBytes = new long[runTasks.length];
+      long[] runOutputRecords = new long[runTasks.length];
+      long[] runOutputBytes = new long[runTasks.length];
+      long[] specInputRecords = new long[runTasks.length];
+      long[] specInputBytes = new long[runTasks.length];
+      long[] specOutputRecords = new long[runTasks.length];
+      long[] specOutputBytes = new long[runTasks.length];
+
+      for (int i = 0; i < runTasks.length; ++i) {
+        final TaskInfo specInfo;
+        final Counters counters = runTasks[i].getCounters();
+        switch (type) {
+          case MAP:
+             runInputBytes[i] = counters.findCounter("FileSystemCounters",
+                 "HDFS_BYTES_READ").getValue();
+             runInputRecords[i] =
+               (int)counters.findCounter(MAP_INPUT_RECORDS).getValue();
+             runOutputBytes[i] =
+               counters.findCounter(MAP_OUTPUT_BYTES).getValue();
+             runOutputRecords[i] =
+               (int)counters.findCounter(MAP_OUTPUT_RECORDS).getValue();
+
+            specInfo = spec.getTaskInfo(TaskType.MAP, i);
+            specInputRecords[i] = specInfo.getInputRecords();
+            specInputBytes[i] = specInfo.getInputBytes();
+            specOutputRecords[i] = specInfo.getOutputRecords();
+            specOutputBytes[i] = specInfo.getOutputBytes();
+            System.out.printf(type + " SPEC: %9d -> %9d :: %5d -> %5d\n",
+                 specInputBytes[i], specOutputBytes[i],
+                 specInputRecords[i], specOutputRecords[i]);
+            System.out.printf(type + " RUN:  %9d -> %9d :: %5d -> %5d\n",
+                 runInputBytes[i], runOutputBytes[i],
+                 runInputRecords[i], runOutputRecords[i]);
+            break;
+          case REDUCE:
+            runInputBytes[i] = 0;
+            runInputRecords[i] =
+              (int)counters.findCounter(REDUCE_INPUT_RECORDS).getValue();
+            runOutputBytes[i] =
+              counters.findCounter("FileSystemCounters",
+                  "HDFS_BYTES_WRITTEN").getValue();
+            runOutputRecords[i] =
+              (int)counters.findCounter(REDUCE_OUTPUT_RECORDS).getValue();
+
+
+            specInfo = spec.getTaskInfo(TaskType.REDUCE, i);
+            // There is no reliable counter for reduce input bytes. The
+            // variable-length encoding of intermediate records and other noise
+            // make this quantity difficult to estimate. The shuffle and spec
+            // input bytes are included in debug output for reference, but are
+            // not checked
+            specInputBytes[i] = 0;
+            specInputRecords[i] = specInfo.getInputRecords();
+            specOutputRecords[i] = specInfo.getOutputRecords();
+            specOutputBytes[i] = specInfo.getOutputBytes();
+            System.out.printf(type + " SPEC: (%9d) -> %9d :: %5d -> %5d\n",
+                 specInfo.getInputBytes(), specOutputBytes[i],
+                 specInputRecords[i], specOutputRecords[i]);
+            System.out.printf(type + " RUN:  (%9d) -> %9d :: %5d -> %5d\n",
+                 counters.findCounter(REDUCE_SHUFFLE_BYTES).getValue(),
+                 runOutputBytes[i], runInputRecords[i], runOutputRecords[i]);
+            break;
+          default:
+            specInfo = null;
+            fail("Unexpected type: " + type);
+        }
+      }
+
+      // Check input bytes
+      Arrays.sort(specInputBytes);
+      Arrays.sort(runInputBytes);
+      for (int i = 0; i < runTasks.length; ++i) {
+        assertTrue("Mismatched " + type + " input bytes " +
+            specInputBytes[i] + "/" + runInputBytes[i],
+            eqPlusMinus(runInputBytes[i], specInputBytes[i], extraInputBytes));
+      }
+
+      // Check input records
+      Arrays.sort(specInputRecords);
+      Arrays.sort(runInputRecords);
+      for (int i = 0; i < runTasks.length; ++i) {
+        assertTrue("Mismatched " + type + " input records " +
+            specInputRecords[i] + "/" + runInputRecords[i],
+            eqPlusMinus(runInputRecords[i], specInputRecords[i],
+              extraInputRecords));
+      }
+
+      // Check output bytes
+      Arrays.sort(specOutputBytes);
+      Arrays.sort(runOutputBytes);
+      for (int i = 0; i < runTasks.length; ++i) {
+        assertTrue("Mismatched " + type + " output bytes " +
+            specOutputBytes[i] + "/" + runOutputBytes[i],
+            eqPlusMinus(runOutputBytes[i], specOutputBytes[i],
+              extraOutputBytes));
+      }
+
+      // Check output records
+      Arrays.sort(specOutputRecords);
+      Arrays.sort(runOutputRecords);
+      for (int i = 0; i < runTasks.length; ++i) {
+        assertTrue("Mismatched " + type + " output records " +
+            specOutputRecords[i] + "/" + runOutputRecords[i],
+            eqPlusMinus(runOutputRecords[i], specOutputRecords[i],
+              extraOutputRecords));
+      }
+
+    }
+
+    private static boolean eqPlusMinus(long a, long b, long x) {
+      final long diff = Math.abs(a - b);
+      return diff <= x;
+    }
+
+    @Override
+    protected void onSuccess(Job job) {
+      retiredJobs.add(job);
+    }
+    @Override
+    protected void onFailure(Job job) {
+      fail("Job failure: " + job);
+    }
+  }
+
+  static class DebugGridmix extends Gridmix {
+
+    private DebugJobFactory factory;
+    private TestMonitor monitor;
+
+    public void checkMonitor() throws Exception {
+      monitor.verify(factory.getSubmitted());
+    }
+
+    @Override
+    protected JobMonitor createJobMonitor() {
+      monitor = new TestMonitor(NJOBS + 1); // include data generation job
+      return monitor;
+    }
+
+    @Override
+    protected JobFactory createJobFactory(JobSubmitter submitter,
+        String traceIn, Path scratchDir, Configuration conf,
+        CountDownLatch startFlag) throws IOException {
+      factory =
+        new DebugJobFactory(submitter, scratchDir, NJOBS, conf, startFlag);
+      return factory;
+    }
+  }
+
+  @Test
+  public void testSubmit() throws Exception {
+    final Path in = new Path("foo").makeQualified(dfs);
+    final Path out = new Path("/gridmix").makeQualified(dfs);
+    final String[] argv = {
+      "-D" + FilePool.GRIDMIX_MIN_FILE + "=0",
+      "-D" + Gridmix.GRIDMIX_OUT_DIR + "=" + out,
+      "-generate", String.valueOf(GENDATA) + "m",
+      in.toString(),
+      "-" // ignored by DebugGridmix
+    };
+    DebugGridmix client = new DebugGridmix();
+    final Configuration conf = mrCluster.createJobConf();
+    //conf.setInt(Gridmix.GRIDMIX_KEY_LEN, 2);
+    int res = ToolRunner.run(conf, client, argv);
+    assertEquals("Client exited with nonzero status", 0, res);
+    client.checkMonitor();
+  }
+
+}

+ 79 - 0
src/contrib/gridmix/src/test/org/apache/hadoop/mapred/gridmix/TestRecordFactory.java

@@ -0,0 +1,79 @@
+
+/**
+ * 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.Random;
+
+import org.junit.Test;
+import static org.junit.Assert.*;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.DataOutputBuffer;
+
+public class TestRecordFactory {
+  private static final Log LOG = LogFactory.getLog(TestRecordFactory.class);
+
+  public static void testFactory(long targetBytes, long targetRecs)
+      throws Exception {
+    final Configuration conf = new Configuration();
+    final GridmixKey key = new GridmixKey();
+    final GridmixRecord val = new GridmixRecord();
+    LOG.info("Target bytes/records: " + targetBytes + "/" + targetRecs);
+    final RecordFactory f = new AvgRecordFactory(targetBytes, targetRecs, conf);
+    targetRecs = targetRecs <= 0 && targetBytes >= 0
+      ? Math.max(1,
+          targetBytes / conf.getInt("gridmix.missing.rec.size", 64 * 1024))
+      : targetRecs;
+
+    long records = 0L;
+    final DataOutputBuffer out = new DataOutputBuffer();
+    while (f.next(key, val)) {
+      ++records;
+      key.write(out);
+      val.write(out);
+    }
+    assertEquals(targetRecs, records);
+    assertEquals(targetBytes, out.getLength());
+  }
+
+  @Test
+  public void testRandom() throws Exception {
+    final Random r = new Random();
+    final long targetBytes = r.nextInt(1 << 20) + 3 * (1 << 14);
+    final long targetRecs = r.nextInt(1 << 14);
+    testFactory(targetBytes, targetRecs);
+  }
+
+  @Test
+  public void testAvg() throws Exception {
+    final Random r = new Random();
+    final long avgsize = r.nextInt(1 << 10) + 1;
+    final long targetRecs = r.nextInt(1 << 14);
+    testFactory(targetRecs * avgsize, targetRecs);
+  }
+
+  @Test
+  public void testZero() throws Exception {
+    final Random r = new Random();
+    final long targetBytes = r.nextInt(1 << 20);
+    testFactory(targetBytes, 0);
+  }
+}

+ 1 - 1
src/mapred/org/apache/hadoop/mapred/Task.java

@@ -63,7 +63,7 @@ abstract public class Task implements Writable, Configurable {
     LogFactory.getLog(Task.class);
 
   // Counters used by Task subclasses
-  protected static enum Counter { 
+  public static enum Counter { 
     MAP_INPUT_RECORDS, 
     MAP_OUTPUT_RECORDS,
     MAP_SKIPPED_RECORDS,

+ 1 - 1
src/mapred/org/apache/hadoop/mapred/TaskStatus.java

@@ -33,7 +33,7 @@ import org.apache.hadoop.util.StringUtils;
  * not intended to be a comprehensive piece of data.
  *
  **************************************************/
-abstract class TaskStatus implements Writable, Cloneable {
+public abstract class TaskStatus implements Writable, Cloneable {
   static final Log LOG =
     LogFactory.getLog(TaskStatus.class.getName());
   

+ 54 - 0
src/test/org/apache/hadoop/tools/rumen/HistogramRawTestData.java

@@ -0,0 +1,54 @@
+/**
+ * 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.util.ArrayList;
+import java.util.List;
+
+class HistogramRawTestData {
+  List<Long> data = new ArrayList<Long>();
+
+  List<Integer> percentiles = new ArrayList<Integer>();
+
+  int scale;
+
+  public List<Integer> getPercentiles() {
+    return percentiles;
+  }
+
+  public void setPercentiles(List<Integer> percentiles) {
+    this.percentiles = percentiles;
+  }
+
+  public int getScale() {
+    return scale;
+  }
+
+  public void setScale(int scale) {
+    this.scale = scale;
+  }
+
+  public List<Long> getData() {
+    return data;
+  }
+
+  public void setData(List<Long> data) {
+    this.data = data;
+  }
+}

+ 155 - 0
src/test/org/apache/hadoop/tools/rumen/TestHistograms.java

@@ -0,0 +1,155 @@
+/**
+ * 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.IOException;
+
+import java.util.List;
+
+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.codehaus.jackson.JsonEncoding;
+import org.codehaus.jackson.JsonGenerator;
+import org.codehaus.jackson.JsonFactory;
+import org.codehaus.jackson.map.ObjectMapper;
+
+import org.junit.Test;
+import static org.junit.Assert.*;
+
+public class TestHistograms {
+
+  /**
+   * @throws IOException
+   * 
+   *           There should be files in the directory named by
+   *           ${test.build.data}/rumen/histogram-test .
+   * 
+   *           There will be pairs of files, inputXxx.json and goldXxx.json .
+   * 
+   *           We read the input file as a HistogramRawTestData in json. Then we
+   *           create a Histogram using the data field, and then a
+   *           LoggedDiscreteCDF using the percentiles and scale field. Finally,
+   *           we read the corresponding goldXxx.json as a LoggedDiscreteCDF and
+   *           deepCompare them.
+   */
+  @Test
+  public void testHistograms() throws IOException {
+    final Configuration conf = new Configuration();
+    final FileSystem lfs = FileSystem.getLocal(conf);
+    final Path rootInputDir = new Path(
+        System.getProperty("test.tools.input.dir", "")).makeQualified(lfs);
+    final Path rootInputFile = new Path(rootInputDir, "rumen/histogram-tests");
+
+
+    FileStatus[] tests = lfs.listStatus(rootInputFile);
+
+    for (int i = 0; i < tests.length; ++i) {
+      Path filePath = tests[i].getPath();
+      String fileName = filePath.getName();
+      if (fileName.startsWith("input")) {
+        String testName = fileName.substring("input".length());
+        Path goldFilePath = new Path(rootInputFile, "gold"+testName);
+        assertTrue("Gold file dies not exist", lfs.exists(goldFilePath));
+        LoggedDiscreteCDF newResult = histogramFileToCDF(filePath, lfs);
+        System.out.println("Testing a Histogram for " + fileName);
+        FSDataInputStream goldStream = lfs.open(goldFilePath);
+        JsonObjectMapperParser<LoggedDiscreteCDF> parser = new JsonObjectMapperParser<LoggedDiscreteCDF>(
+            goldStream, LoggedDiscreteCDF.class); 
+        try {
+          LoggedDiscreteCDF dcdf = parser.getNext();
+          dcdf.deepCompare(newResult, new TreePath(null, "<root>"));
+        } catch (DeepInequalityException e) {
+          fail(e.path.toString());
+        }
+        finally {
+            parser.close();
+        }
+      }
+    }
+  }
+
+  private static LoggedDiscreteCDF histogramFileToCDF(Path path, FileSystem fs)
+      throws IOException {
+    FSDataInputStream dataStream = fs.open(path);
+    JsonObjectMapperParser<HistogramRawTestData> parser = new JsonObjectMapperParser<HistogramRawTestData>(
+        dataStream, HistogramRawTestData.class);
+    HistogramRawTestData data;
+    try {
+      data = parser.getNext();
+    } finally {
+      parser.close();
+    }
+    
+    Histogram hist = new Histogram();
+    List<Long> measurements = data.getData();
+    List<Long> typeProbeData = new HistogramRawTestData().getData();
+
+    assertTrue(
+        "The data attribute of a jackson-reconstructed HistogramRawTestData "
+            + " should be a " + typeProbeData.getClass().getName()
+            + ", like a virgin HistogramRawTestData, but it's a "
+            + measurements.getClass().getName(),
+        measurements.getClass() == typeProbeData.getClass());
+
+    for (int j = 0; j < measurements.size(); ++j) {
+      hist.enter(measurements.get(j));
+    }
+
+    LoggedDiscreteCDF result = new LoggedDiscreteCDF();
+    int[] percentiles = new int[data.getPercentiles().size()];
+
+    for (int j = 0; j < data.getPercentiles().size(); ++j) {
+      percentiles[j] = data.getPercentiles().get(j);
+    }
+
+    result.setCDF(hist, percentiles, data.getScale());
+    return result;
+  }
+  
+  public static void main(String[] args) throws IOException {
+    final Configuration conf = new Configuration();
+    final FileSystem lfs = FileSystem.getLocal(conf);
+
+    for (String arg : args) {
+      Path filePath = new Path(arg).makeQualified(lfs);
+      String fileName = filePath.getName();
+      if (fileName.startsWith("input")) {
+        LoggedDiscreteCDF newResult = histogramFileToCDF(filePath, lfs);
+        String testName = fileName.substring("input".length());
+        Path goldFilePath = new Path(filePath.getParent(), "gold"+testName);
+
+        ObjectMapper mapper = new ObjectMapper();
+        JsonFactory factory = mapper.getJsonFactory();
+        FSDataOutputStream ostream = lfs.create(goldFilePath, true);
+        JsonGenerator gen = factory.createJsonGenerator(ostream,
+            JsonEncoding.UTF8);
+        gen.useDefaultPrettyPrinter();
+        
+        gen.writeObject(newResult);
+        
+        gen.close();
+      } else {
+        System.err.println("Input file not started with \"input\". File "+fileName+" skipped.");
+      }
+    }
+  }
+}

+ 123 - 0
src/test/org/apache/hadoop/tools/rumen/TestPiecewiseLinearInterpolation.java

@@ -0,0 +1,123 @@
+/**
+ * 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.util.ArrayList;
+
+import org.junit.Test;
+import static org.junit.Assert.*;
+
+public class TestPiecewiseLinearInterpolation {
+
+  static private double maximumRelativeError = 0.002D;
+
+  static private LoggedSingleRelativeRanking makeRR(double ranking, long datum) {
+    LoggedSingleRelativeRanking result = new LoggedSingleRelativeRanking();
+
+    result.setDatum(datum);
+    result.setRelativeRanking(ranking);
+
+    return result;
+  }
+
+  @Test
+  public void testOneRun() {
+    LoggedDiscreteCDF input = new LoggedDiscreteCDF();
+
+    input.setMinimum(100000L);
+    input.setMaximum(1100000L);
+
+    ArrayList<LoggedSingleRelativeRanking> rankings = new ArrayList<LoggedSingleRelativeRanking>();
+
+    rankings.add(makeRR(0.1, 200000L));
+    rankings.add(makeRR(0.5, 800000L));
+    rankings.add(makeRR(0.9, 1000000L));
+
+    input.setRankings(rankings);
+    input.setNumberValues(3);
+
+    CDFRandomGenerator gen = new CDFPiecewiseLinearRandomGenerator(input);
+    Histogram values = new Histogram();
+
+    for (int i = 0; i < 1000000; ++i) {
+      long value = gen.randomValue();
+      values.enter(value);
+    }
+
+    /*
+     * Now we build a percentiles CDF, and compute the sum of the squares of the
+     * actual percentiles vrs. the predicted percentiles
+     */
+    int[] percentiles = new int[99];
+
+    for (int i = 0; i < 99; ++i) {
+      percentiles[i] = i + 1;
+    }
+
+    long[] result = values.getCDF(100, percentiles);
+    long sumErrorSquares = 0L;
+
+    for (int i = 0; i < 10; ++i) {
+      long error = result[i] - (10000L * i + 100000L);
+      System.out.println("element " + i + ", got " + result[i] + ", expected "
+          + (10000L * i + 100000L) + ", error = " + error);
+      sumErrorSquares += error * error;
+    }
+
+    for (int i = 10; i < 50; ++i) {
+      long error = result[i] - (15000L * i + 50000L);
+      System.out.println("element " + i + ", got " + result[i] + ", expected "
+          + (15000L * i + 50000L) + ", error = " + error);
+      sumErrorSquares += error * error;
+    }
+
+    for (int i = 50; i < 90; ++i) {
+      long error = result[i] - (5000L * i + 550000L);
+      System.out.println("element " + i + ", got " + result[i] + ", expected "
+          + (5000L * i + 550000L) + ", error = " + error);
+      sumErrorSquares += error * error;
+    }
+
+    for (int i = 90; i <= 100; ++i) {
+      long error = result[i] - (10000L * i + 100000L);
+      System.out.println("element " + i + ", got " + result[i] + ", expected "
+          + (10000L * i + 100000L) + ", error = " + error);
+      sumErrorSquares += error * error;
+    }
+
+    // normalize the error
+    double realSumErrorSquares = (double) sumErrorSquares;
+
+    double normalizedError = realSumErrorSquares / 100
+        / rankings.get(1).getDatum() / rankings.get(1).getDatum();
+    double RMSNormalizedError = Math.sqrt(normalizedError);
+
+    System.out.println("sumErrorSquares = " + sumErrorSquares);
+
+    System.out.println("normalizedError: " + normalizedError
+        + ", RMSNormalizedError: " + RMSNormalizedError);
+
+    System.out.println("Cumulative error is " + RMSNormalizedError);
+
+    assertTrue("The RMS relative error per bucket, " + RMSNormalizedError
+        + ", exceeds our tolerance of " + maximumRelativeError,
+        RMSNormalizedError <= maximumRelativeError);
+
+  }
+}

+ 125 - 0
src/test/org/apache/hadoop/tools/rumen/TestRumenJobTraces.java

@@ -0,0 +1,125 @@
+/**
+ * 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.IOException;
+import java.io.InputStream;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.util.ToolRunner;
+
+import org.junit.Test;
+import static org.junit.Assert.*;
+
+public class TestRumenJobTraces {
+  @Test
+  public void testSmallTrace() throws Exception {
+    performSingleTest("sample-job-tracker-logs.gz",
+        "job-tracker-logs-topology-output", "job-tracker-logs-trace-output.gz");
+  }
+
+  @Test
+  public void testTruncatedTask() throws Exception {
+    performSingleTest("truncated-job-tracker-log", "truncated-topology-output",
+        "truncated-trace-output");
+  }
+
+  private void performSingleTest(String jtLogName, String goldTopology,
+      String goldTrace) throws Exception {
+    final Configuration conf = new Configuration();
+    final FileSystem lfs = FileSystem.getLocal(conf);
+
+    final Path rootInputDir =
+        new Path(System.getProperty("test.tools.input.dir", ""))
+            .makeQualified(lfs);
+    final Path rootTempDir =
+        new Path(System.getProperty("test.build.data", "/tmp"))
+            .makeQualified(lfs);
+
+    final Path rootInputFile = new Path(rootInputDir, "rumen/small-trace-test");
+    final Path tempDir = new Path(rootTempDir, "TestRumenJobTraces");
+    lfs.delete(tempDir, true);
+
+    final Path topologyFile = new Path(tempDir, jtLogName + "-topology.json");
+    final Path traceFile = new Path(tempDir, jtLogName + "-trace.json");
+
+    final Path inputFile = new Path(rootInputFile, jtLogName);
+
+    System.out.println("topology result file = " + topologyFile);
+    System.out.println("trace result file = " + traceFile);
+
+    String[] args = new String[6];
+
+    args[0] = "-v1";
+
+    args[1] = "-write-topology";
+    args[2] = topologyFile.toString();
+
+    args[3] = "-write-job-trace";
+    args[4] = traceFile.toString();
+
+    args[5] = inputFile.toString();
+
+    final Path topologyGoldFile = new Path(rootInputFile, goldTopology);
+    final Path traceGoldFile = new Path(rootInputFile, goldTrace);
+
+    HadoopLogsAnalyzer analyzer = new HadoopLogsAnalyzer();
+    int result = ToolRunner.run(analyzer, args);
+    assertEquals("Non-zero exit", 0, result);
+
+    TestRumenJobTraces
+        .<LoggedNetworkTopology> jsonFileMatchesGold(lfs, topologyFile,
+            topologyGoldFile, LoggedNetworkTopology.class, "topology");
+    TestRumenJobTraces.<LoggedJob> jsonFileMatchesGold(lfs, traceFile,
+        traceGoldFile, LoggedJob.class, "trace");
+  }
+
+  static private <T extends DeepCompare> void jsonFileMatchesGold(
+      FileSystem lfs, Path result, Path gold, Class<? extends T> clazz,
+      String fileDescription) throws IOException {
+    JsonObjectMapperParser<T> goldParser =
+        new JsonObjectMapperParser<T>(gold, clazz, new Configuration());
+    InputStream resultStream = lfs.open(result);
+    JsonObjectMapperParser<T> resultParser =
+        new JsonObjectMapperParser<T>(resultStream, clazz);
+    try {
+      while (true) {
+        DeepCompare goldJob = goldParser.getNext();
+        DeepCompare resultJob = resultParser.getNext();
+        if ((goldJob == null) || (resultJob == null)) {
+          assertTrue(goldJob == resultJob);
+          break;
+        }
+
+        try {
+          resultJob.deepCompare(goldJob, new TreePath(null, "<root>"));
+        } catch (DeepInequalityException e) {
+          String error = e.path.toString();
+
+          assertFalse(fileDescription + " mismatches: " + error, true);
+        }
+      }
+    } finally {
+      IOUtils.cleanup(null, goldParser, resultParser);
+    }
+  }
+}

+ 338 - 0
src/test/org/apache/hadoop/tools/rumen/TestZombieJob.java

@@ -0,0 +1,338 @@
+/**
+ * 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.util.List;
+import java.util.ArrayList;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.TaskStatus.State;
+import org.apache.hadoop.mapreduce.TaskType;
+
+import org.junit.Before;
+import org.junit.Test;
+import static org.junit.Assert.*;
+
+public class TestZombieJob {
+  final double epsilon = 0.01;
+  private final int[] attemptTimesPercentiles = new int[] { 10, 50, 90 };
+  private long[] succeededCDF = new long[] { 5268, 5268, 5268, 5268, 5268 };
+  private long[] failedCDF = new long[] { 18592, 18592, 18592, 18592, 18592 };
+  private double[] expectedPs = new double[] { 0.000001, 0.18707660239708182,
+      0.0013027618551328818, 2.605523710265763E-4 };
+
+  private final long[] mapTaskCounts = new long[] { 7838525L, 342277L, 100228L,
+      1564L, 1234L };
+  private final long[] reduceTaskCounts = new long[] { 4405338L, 139391L,
+      1514383L, 139391, 1234L };
+
+  List<LoggedJob> loggedJobs = new ArrayList<LoggedJob>();
+  List<JobStory> jobStories = new ArrayList<JobStory>();
+
+  @Before
+  public void setUp() throws Exception {
+    final Configuration conf = new Configuration();
+    final FileSystem lfs = FileSystem.getLocal(conf);
+
+    final Path rootInputDir = new Path(
+        System.getProperty("test.tools.input.dir", "")).makeQualified(lfs);
+    final Path rootInputFile = new Path(rootInputDir, "rumen/zombie");
+
+    ZombieJobProducer parser = new ZombieJobProducer(new Path(rootInputFile,
+        "input-trace.json"), new ZombieCluster(new Path(rootInputFile,
+        "input-topology.json"), null, conf), conf);
+
+    JobStory job = null;
+    for (int i = 0; i < 4; i++) {
+      job = parser.getNextJob();
+      ZombieJob zJob = (ZombieJob) job;
+      LoggedJob loggedJob = zJob.getLoggedJob();
+      System.out.println(i + ":" + job.getNumberMaps() + "m, "
+          + job.getNumberReduces() + "r");
+      System.out
+          .println(loggedJob.getOutcome() + ", " + loggedJob.getJobtype());
+
+      System.out.println("Input Splits -- " + job.getInputSplits().length
+          + ", " + job.getNumberMaps());
+
+      System.out.println("Successful Map CDF -------");
+      for (LoggedDiscreteCDF cdf : loggedJob.getSuccessfulMapAttemptCDFs()) {
+        System.out.println(cdf.getNumberValues() + ": " + cdf.getMinimum()
+            + "--" + cdf.getMaximum());
+        for (LoggedSingleRelativeRanking ranking : cdf.getRankings()) {
+          System.out.println("   " + ranking.getRelativeRanking() + ":"
+              + ranking.getDatum());
+        }
+      }
+      System.out.println("Failed Map CDF -----------");
+      for (LoggedDiscreteCDF cdf : loggedJob.getFailedMapAttemptCDFs()) {
+        System.out.println(cdf.getNumberValues() + ": " + cdf.getMinimum()
+            + "--" + cdf.getMaximum());
+        for (LoggedSingleRelativeRanking ranking : cdf.getRankings()) {
+          System.out.println("   " + ranking.getRelativeRanking() + ":"
+              + ranking.getDatum());
+        }
+      }
+      System.out.println("Successful Reduce CDF ----");
+      LoggedDiscreteCDF cdf = loggedJob.getSuccessfulReduceAttemptCDF();
+      System.out.println(cdf.getNumberValues() + ": " + cdf.getMinimum() + "--"
+          + cdf.getMaximum());
+      for (LoggedSingleRelativeRanking ranking : cdf.getRankings()) {
+        System.out.println("   " + ranking.getRelativeRanking() + ":"
+            + ranking.getDatum());
+      }
+      System.out.println("Failed Reduce CDF --------");
+      cdf = loggedJob.getFailedReduceAttemptCDF();
+      System.out.println(cdf.getNumberValues() + ": " + cdf.getMinimum() + "--"
+          + cdf.getMaximum());
+      for (LoggedSingleRelativeRanking ranking : cdf.getRankings()) {
+        System.out.println("   " + ranking.getRelativeRanking() + ":"
+            + ranking.getDatum());
+      }
+      System.out.print("map attempts to success -- ");
+      for (double p : loggedJob.getMapperTriesToSucceed()) {
+        System.out.print(p + ", ");
+      }
+      System.out.println();
+      System.out.println("===============");
+
+      loggedJobs.add(loggedJob);
+      jobStories.add(job);
+    }
+  }
+
+  @Test
+  public void testFirstJob() {
+    // 20th job seems reasonable: "totalMaps":329,"totalReduces":101
+    // successful map: 80 node-local, 196 rack-local, 53 rack-remote, 2 unknown
+    // failed map: 0-0-0-1
+    // successful reduce: 99 failed reduce: 13
+    // map attempts to success -- 0.9969879518072289, 0.0030120481927710845,
+    JobStory job = jobStories.get(0);
+    assertEquals(1, job.getNumberMaps());
+    assertEquals(1, job.getNumberReduces());
+
+    // get splits
+
+    TaskAttemptInfo taInfo = null;
+    long expectedRuntime = 2423;
+    // get a succeeded map task attempt, expect the exact same task attempt
+    taInfo = job.getMapTaskAttemptInfoAdjusted(14, 0, 1);
+    assertEquals(expectedRuntime, taInfo.getRuntime());
+    assertEquals(State.SUCCEEDED, taInfo.getRunState());
+
+    // get a succeeded map attempt, but reschedule with different locality.
+    taInfo = job.getMapTaskAttemptInfoAdjusted(14, 0, 2);
+    assertEquals(State.SUCCEEDED, taInfo.getRunState());
+    taInfo = job.getMapTaskAttemptInfoAdjusted(14, 0, 0);
+    assertEquals(State.SUCCEEDED, taInfo.getRunState());
+
+    expectedRuntime = 97502;
+    // get a succeeded reduce task attempt, expect the exact same task attempt
+    taInfo = job.getTaskAttemptInfo(TaskType.REDUCE, 14, 0);
+    assertEquals(State.SUCCEEDED, taInfo.getRunState());
+
+    // get a failed reduce task attempt, expect the exact same task attempt
+    taInfo = job.getTaskAttemptInfo(TaskType.REDUCE, 14, 0);
+    assertEquals(State.SUCCEEDED, taInfo.getRunState());
+
+    // get a non-exist reduce task attempt, expect a made-up task attempt
+    // TODO fill in test case
+  }
+
+  @Test
+  public void testSecondJob() {
+    // 7th job has many failed tasks.
+    // 3204 m, 0 r
+    // successful maps 497-586-23-1, failed maps 0-0-0-2714
+    // map attempts to success -- 0.8113600833767587, 0.18707660239708182,
+    // 0.0013027618551328818, 2.605523710265763E-4,
+    JobStory job = jobStories.get(1);
+    assertEquals(20, job.getNumberMaps());
+    assertEquals(1, job.getNumberReduces());
+
+    TaskAttemptInfo taInfo = null;
+    // get a succeeded map task attempt
+    taInfo = job.getMapTaskAttemptInfoAdjusted(17, 1, 1);
+    assertEquals(State.SUCCEEDED, taInfo.getRunState());
+
+    // get a succeeded map task attempt, with different locality
+    taInfo = job.getMapTaskAttemptInfoAdjusted(17, 1, 2);
+    assertEquals(State.SUCCEEDED, taInfo.getRunState());
+    taInfo = job.getMapTaskAttemptInfoAdjusted(17, 1, 0);
+    assertEquals(State.SUCCEEDED, taInfo.getRunState());
+
+    // get a failed map task attempt
+    taInfo = job.getMapTaskAttemptInfoAdjusted(14, 0, 1);
+    assertEquals(1927, taInfo.getRuntime());
+    assertEquals(State.SUCCEEDED, taInfo.getRunState());
+
+    // get a failed map task attempt, with different locality
+    // TODO: this test does not make sense here, because I don't have
+    // available data set.
+  }
+
+  @Test
+  public void testFourthJob() {
+    // 7th job has many failed tasks.
+    // 3204 m, 0 r
+    // successful maps 497-586-23-1, failed maps 0-0-0-2714
+    // map attempts to success -- 0.8113600833767587, 0.18707660239708182,
+    // 0.0013027618551328818, 2.605523710265763E-4,
+    JobStory job = jobStories.get(3);
+    assertEquals(131, job.getNumberMaps());
+    assertEquals(47, job.getNumberReduces());
+
+    TaskAttemptInfo taInfo = null;
+    // get a succeeded map task attempt
+    long runtime = 5268;
+    taInfo = job.getMapTaskAttemptInfoAdjusted(113, 1, 1);
+    assertEquals(State.SUCCEEDED, taInfo.getRunState());
+    assertEquals(runtime, taInfo.getRuntime());
+
+    // get a succeeded map task attempt, with different locality
+    taInfo = job.getMapTaskAttemptInfoAdjusted(113, 1, 2);
+    assertEquals(State.SUCCEEDED, taInfo.getRunState());
+    assertEquals(runtime, taInfo.getRuntime() / 2);
+    taInfo = job.getMapTaskAttemptInfoAdjusted(113, 1, 0);
+    assertEquals(State.SUCCEEDED, taInfo.getRunState());
+    assertEquals((long) (runtime / 1.5), taInfo.getRuntime());
+
+    // get a failed map task attempt
+    taInfo = job.getMapTaskAttemptInfoAdjusted(113, 0, 1);
+    assertEquals(18592, taInfo.getRuntime());
+    assertEquals(State.FAILED, taInfo.getRunState());
+  }
+
+  @Test
+  public void testRecordIOInfo() {
+    JobStory job = jobStories.get(3);
+
+    TaskInfo mapTask = job.getTaskInfo(TaskType.MAP, 113);
+
+    TaskInfo reduceTask = job.getTaskInfo(TaskType.REDUCE, 0);
+
+    assertEquals(mapTaskCounts[0], mapTask.getInputBytes());
+    assertEquals(mapTaskCounts[1], mapTask.getInputRecords());
+    assertEquals(mapTaskCounts[2], mapTask.getOutputBytes());
+    assertEquals(mapTaskCounts[3], mapTask.getOutputRecords());
+    assertEquals(mapTaskCounts[4], mapTask.getTaskMemory());
+
+    assertEquals(reduceTaskCounts[0], reduceTask.getInputBytes());
+    assertEquals(reduceTaskCounts[1], reduceTask.getInputRecords());
+    assertEquals(reduceTaskCounts[2], reduceTask.getOutputBytes());
+    assertEquals(reduceTaskCounts[3], reduceTask.getOutputRecords());
+    assertEquals(reduceTaskCounts[4], reduceTask.getTaskMemory());
+  }
+
+  @Test
+  public void testMakeUpInfo() {
+    // get many non-exist tasks
+    // total 3204 map tasks, 3300 is a non-exist task.
+    checkMakeUpTask(jobStories.get(3), 113, 1);
+  }
+
+  private void checkMakeUpTask(JobStory job, int taskNumber, int locality) {
+    TaskAttemptInfo taInfo = null;
+
+    Histogram sampleSucceeded = new Histogram();
+    Histogram sampleFailed = new Histogram();
+    List<Integer> sampleAttempts = new ArrayList<Integer>();
+    for (int i = 0; i < 100000; i++) {
+      int attemptId = 0;
+      while (true) {
+        taInfo = job.getMapTaskAttemptInfoAdjusted(taskNumber, attemptId, 1);
+        if (taInfo.getRunState() == State.SUCCEEDED) {
+          sampleSucceeded.enter(taInfo.getRuntime());
+          break;
+        }
+        sampleFailed.enter(taInfo.getRuntime());
+        attemptId++;
+      }
+      sampleAttempts.add(attemptId);
+    }
+
+    // check state distribution
+    int[] countTries = new int[] { 0, 0, 0, 0 };
+    for (int attempts : sampleAttempts) {
+      assertTrue(attempts < 4);
+      countTries[attempts]++;
+    }
+    /*
+     * System.out.print("Generated map attempts to success -- "); for (int
+     * count: countTries) { System.out.print((double)count/sampleAttempts.size()
+     * + ", "); } System.out.println(); System.out.println("===============");
+     */
+    for (int i = 0; i < 4; i++) {
+      int count = countTries[i];
+      double p = (double) count / sampleAttempts.size();
+      assertTrue(expectedPs[i] - p < epsilon);
+    }
+
+    // check succeeded attempts runtime distribution
+    long[] expectedCDF = succeededCDF;
+    LoggedDiscreteCDF cdf = new LoggedDiscreteCDF();
+    cdf.setCDF(sampleSucceeded, attemptTimesPercentiles, 100);
+    /*
+     * System.out.println("generated succeeded map runtime distribution");
+     * System.out.println(cdf.getNumberValues() + ": " + cdf.getMinimum() + "--"
+     * + cdf.getMaximum()); for (LoggedSingleRelativeRanking ranking:
+     * cdf.getRankings()) { System.out.println("   " +
+     * ranking.getRelativeRanking() + ":" + ranking.getDatum()); }
+     */
+    assertRuntimeEqual(cdf.getMinimum(), expectedCDF[0]);
+    assertRuntimeEqual(cdf.getMaximum(), expectedCDF[4]);
+    for (int i = 0; i < 3; i++) {
+      LoggedSingleRelativeRanking ranking = cdf.getRankings().get(i);
+      assertRuntimeEqual(expectedCDF[i + 1], ranking.getDatum());
+    }
+
+    // check failed attempts runtime distribution
+    expectedCDF = failedCDF;
+    cdf = new LoggedDiscreteCDF();
+    cdf.setCDF(sampleFailed, attemptTimesPercentiles, 100);
+
+    System.out.println("generated failed map runtime distribution");
+    System.out.println(cdf.getNumberValues() + ": " + cdf.getMinimum() + "--"
+        + cdf.getMaximum());
+    for (LoggedSingleRelativeRanking ranking : cdf.getRankings()) {
+      System.out.println("   " + ranking.getRelativeRanking() + ":"
+          + ranking.getDatum());
+    }
+    assertRuntimeEqual(cdf.getMinimum(), expectedCDF[0]);
+    assertRuntimeEqual(cdf.getMaximum(), expectedCDF[4]);
+    for (int i = 0; i < 3; i++) {
+      LoggedSingleRelativeRanking ranking = cdf.getRankings().get(i);
+      assertRuntimeEqual(expectedCDF[i + 1], ranking.getDatum());
+    }
+  }
+
+  private void assertRuntimeEqual(long expected, long generated) {
+    if (expected == 0) {
+      assertTrue(generated > -1000 && generated < 1000);
+    } else {
+      long epsilon = Math.max(expected / 10, 5000);
+      assertTrue(expected - generated > -epsilon);
+      assertTrue(expected - generated < epsilon);
+    }
+  }
+
+}

+ 15 - 0
src/test/tools/data/rumen/histogram-tests/gold-minimal.json

@@ -0,0 +1,15 @@
+{
+  "minimum" : 12345,
+  "rankings" : [ {
+    "relativeRanking" : 0.25,
+    "datum" : 12345
+  }, {
+    "relativeRanking" : 0.5,
+    "datum" : 2345678901
+  }, {
+    "relativeRanking" : 0.75,
+    "datum" : 2345678902
+  } ],
+  "maximum" : 23456789012,
+  "numberValues" : 5
+}

+ 15 - 0
src/test/tools/data/rumen/histogram-tests/gold-one-value-many-repeats.json

@@ -0,0 +1,15 @@
+{
+  "minimum" : 23456789012,
+  "rankings" : [ {
+    "relativeRanking" : 0.25,
+    "datum" : 23456789012
+  }, {
+    "relativeRanking" : 0.5,
+    "datum" : 23456789012
+  }, {
+    "relativeRanking" : 0.75,
+    "datum" : 23456789012
+  } ],
+  "maximum" : 23456789012,
+  "numberValues" : 64
+}

+ 15 - 0
src/test/tools/data/rumen/histogram-tests/gold-only-one-value.json

@@ -0,0 +1,15 @@
+{
+  "minimum" : 23456789012,
+  "rankings" : [ {
+    "relativeRanking" : 0.25,
+    "datum" : 23456789012
+  }, {
+    "relativeRanking" : 0.5,
+    "datum" : 23456789012
+  }, {
+    "relativeRanking" : 0.75,
+    "datum" : 23456789012
+  } ],
+  "maximum" : 23456789012,
+  "numberValues" : 1
+}

+ 15 - 0
src/test/tools/data/rumen/histogram-tests/gold-three-values.json

@@ -0,0 +1,15 @@
+{
+  "minimum" : 1,
+  "rankings" : [ {
+    "relativeRanking" : 0.25,
+    "datum" : 1
+  }, {
+    "relativeRanking" : 0.5,
+    "datum" : 1
+  }, {
+    "relativeRanking" : 0.75,
+    "datum" : 23456789012
+  } ],
+  "maximum" : 234567890123,
+  "numberValues" : 3
+}

+ 17 - 0
src/test/tools/data/rumen/histogram-tests/input-minimal.json

@@ -0,0 +1,17 @@
+{
+	"data" :
+		[
+			12345,
+			2345678901,
+			23456789012,
+			2345678902,
+			23456789012
+		],
+	"percentiles" :
+		[
+			25,
+			50,
+			75
+		],
+	"scale" : 100
+}

+ 76 - 0
src/test/tools/data/rumen/histogram-tests/input-one-value-many-repeats.json

@@ -0,0 +1,76 @@
+{
+	"data" :
+		[
+			23456789012,
+			23456789012,
+			23456789012,
+			23456789012,
+			23456789012,
+			23456789012,
+			23456789012,
+			23456789012,
+			23456789012,
+			23456789012,
+			23456789012,
+			23456789012,
+			23456789012,
+			23456789012,
+			23456789012,
+			23456789012,
+			23456789012,
+			23456789012,
+			23456789012,
+			23456789012,
+			23456789012,
+			23456789012,
+			23456789012,
+			23456789012,
+			23456789012,
+			23456789012,
+			23456789012,
+			23456789012,
+			23456789012,
+			23456789012,
+			23456789012,
+			23456789012,
+			23456789012,
+			23456789012,
+			23456789012,
+			23456789012,
+			23456789012,
+			23456789012,
+			23456789012,
+			23456789012,
+			23456789012,
+			23456789012,
+			23456789012,
+			23456789012,
+			23456789012,
+			23456789012,
+			23456789012,
+			23456789012,
+			23456789012,
+			23456789012,
+			23456789012,
+			23456789012,
+			23456789012,
+			23456789012,
+			23456789012,
+			23456789012,
+			23456789012,
+			23456789012,
+			23456789012,
+			23456789012,
+			23456789012,
+			23456789012,
+			23456789012,
+			23456789012
+		],
+	"percentiles" :
+		[
+			25,
+			50,
+			75
+		],
+	"scale" : 100
+}

+ 13 - 0
src/test/tools/data/rumen/histogram-tests/input-only-one-value.json

@@ -0,0 +1,13 @@
+{
+	"data" :
+		[
+			23456789012
+		],
+	"percentiles" :
+		[
+			25,
+			50,
+			75
+		],
+	"scale" : 100
+}

+ 15 - 0
src/test/tools/data/rumen/histogram-tests/input-three-values.json

@@ -0,0 +1,15 @@
+{
+	"data" :
+		[
+			1,
+			23456789012,
+			234567890123
+		],
+	"percentiles" :
+		[
+			25,
+			50,
+			75
+		],
+	"scale" : 100
+}

+ 1693 - 0
src/test/tools/data/rumen/small-trace-test/job-tracker-logs-topology-output

@@ -0,0 +1,1693 @@
+{
+  "name" : "<root>",
+  "children" : [ {
+    "name" : "194\\.6\\.133\\.192",
+    "children" : [ {
+      "name" : "cluster50213\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50226\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50228\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50217\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50214\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50231\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50232\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50204\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50206\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50203\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50205\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50210\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50208\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50218\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50225\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "192\\.30\\.117\\.128",
+    "children" : [ {
+      "name" : "cluster1209\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1205\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1235\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1239\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1200\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1227\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1212\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1223\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1217\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1207\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1228\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1218\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1221\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1215\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1226\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1236\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "194\\.6\\.134\\.64",
+    "children" : [ {
+      "name" : "cluster50317\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50292\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50291\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50294\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50285\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50300\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50281\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50311\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50297\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50319\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50286\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50307\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50296\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50315\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50316\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "194\\.6\\.129\\.128",
+    "children" : [ {
+      "name" : "cluster1859\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1877\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1871\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1876\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1854\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1841\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1858\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1843\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1857\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1842\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1872\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1869\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1853\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1846\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1867\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "194\\.6\\.130\\.64",
+    "children" : [ {
+      "name" : "cluster1976\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1969\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1961\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1963\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1968\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1979\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1967\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1989\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1970\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1999\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "192\\.30\\.116\\.192",
+    "children" : [ {
+      "name" : "cluster1150\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1127\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1139\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1154\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1138\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1137\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1130\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1151\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1131\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1141\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1124\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1158\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1140\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1144\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1136\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1157\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1143\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "194\\.6\\.128\\.0",
+    "children" : [ {
+      "name" : "cluster1592\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1567\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1594\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1586\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1561\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1585\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1562\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1581\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1566\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1598\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1568\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1560\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1574\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1573\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1583\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1579\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "194\\.6\\.134\\.192",
+    "children" : [ {
+      "name" : "cluster50364\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50372\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50365\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50377\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50368\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50396\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50375\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50389\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50382\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "192\\.30\\.127\\.192",
+    "children" : [ {
+      "name" : "cluster1533\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1531\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1557\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1555\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1534\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1553\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1550\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1540\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1538\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1520\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1559\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1535\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1525\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1529\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1551\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "194\\.6\\.133\\.64",
+    "children" : [ {
+      "name" : "cluster50124\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50159\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50144\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50145\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50133\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50120\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50130\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50142\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50147\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50156\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50125\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50141\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50152\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "192\\.30\\.117\\.192",
+    "children" : [ {
+      "name" : "cluster1250\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1276\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1248\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1246\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1251\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1259\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1261\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1260\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1243\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1256\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1272\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1274\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1245\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1249\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "192\\.30\\.116\\.64",
+    "children" : [ {
+      "name" : "cluster1041\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1075\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1042\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1078\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1072\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1053\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1056\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1064\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1055\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1070\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1061\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1059\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1040\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "194\\.6\\.133\\.128",
+    "children" : [ {
+      "name" : "cluster50171\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50195\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50161\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50191\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50174\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50185\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50177\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50166\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50170\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50179\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "192\\.30\\.126\\.0",
+    "children" : [ {
+      "name" : "cluster1283\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1299\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1281\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1288\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1302\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1294\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1289\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1315\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1305\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1316\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "192\\.30\\.116\\.128",
+    "children" : [ {
+      "name" : "cluster1107\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1118\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1080\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1093\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1102\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1104\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1097\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1087\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1095\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1110\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "194\\.6\\.135\\.64",
+    "children" : [ {
+      "name" : "cluster3071\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster3079\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster3068\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster3057\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster3058\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster3070\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster3054\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster3077\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster3049\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster3063\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster3075\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster3065\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster3076\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster3061\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster3073\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster3055\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "194\\.6\\.128\\.128",
+    "children" : [ {
+      "name" : "cluster50468\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50476\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50440\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50473\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50477\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50460\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50475\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50459\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50447\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50464\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50441\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50444\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "194\\.6\\.135\\.128",
+    "children" : [ {
+      "name" : "cluster3097\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster3089\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster3111\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster3093\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster3099\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster3106\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster3108\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster3112\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster3085\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster3094\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster3103\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster3098\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster3082\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster3104\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster3114\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "194\\.6\\.128\\.192",
+    "children" : [ {
+      "name" : "cluster50485\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50493\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50510\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50494\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50484\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50481\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50490\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50501\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50478\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50491\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50505\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50488\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50509\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50513\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "194\\.6\\.133\\.0",
+    "children" : [ {
+      "name" : "cluster50085\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50117\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50113\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50101\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50108\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50090\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "192\\.30\\.63\\.0",
+    "children" : [ {
+      "name" : "cluster1789\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1777\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1785\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1770\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1793\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1779\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1788\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1776\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1773\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1798\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1762\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1772\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1778\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1782\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1774\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1781\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1760\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1796\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1775\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1768\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1786\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1771\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "194\\.6\\.130\\.0",
+    "children" : [ {
+      "name" : "cluster1959\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1957\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1931\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1920\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1938\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1925\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1932\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1927\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1933\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1930\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1928\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1924\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1953\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1936\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "192\\.30\\.62\\.128",
+    "children" : [ {
+      "name" : "cluster1717\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1708\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1707\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1690\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1714\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1683\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1703\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1702\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1694\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1700\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1711\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1713\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1718\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "192\\.30\\.126\\.192",
+    "children" : [ {
+      "name" : "cluster1418\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1429\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1420\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1412\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1400\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1415\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1437\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1405\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1427\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "192\\.30\\.126\\.64",
+    "children" : [ {
+      "name" : "cluster1334\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1332\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1346\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1350\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1328\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1333\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1321\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1358\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1357\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1356\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "194\\.6\\.129\\.192",
+    "children" : [ {
+      "name" : "cluster1914\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1883\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1896\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1911\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1913\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1915\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1903\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1906\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1900\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1891\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1889\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1907\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "192\\.30\\.63\\.192",
+    "children" : [ {
+      "name" : "cluster1006\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1035\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1018\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1026\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1020\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1021\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1027\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1031\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1036\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1032\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1029\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1004\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1011\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1008\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1025\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1002\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1030\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1019\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1017\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1028\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "192\\.30\\.127\\.64",
+    "children" : [ {
+      "name" : "cluster1445\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1470\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1449\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1462\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1450\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1454\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1466\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1465\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1474\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1444\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1448\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1463\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1457\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1447\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1455\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1442\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1479\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1467\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1446\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "194\\.6\\.132\\.128",
+    "children" : [ {
+      "name" : "cluster50034\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50011\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50023\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50025\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50021\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "194\\.6\\.129\\.64",
+    "children" : [ {
+      "name" : "cluster1800\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1809\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1816\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1819\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1813\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1806\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1803\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1835\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1822\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1807\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1823\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1832\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "194\\.6\\.134\\.128",
+    "children" : [ {
+      "name" : "cluster50359\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50326\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50348\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50325\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50342\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50352\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "192\\.30\\.126\\.128",
+    "children" : [ {
+      "name" : "cluster1383\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1378\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1393\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1395\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1396\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1373\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1388\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1379\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1370\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1368\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1371\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1377\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1369\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "192\\.30\\.62\\.64",
+    "children" : [ {
+      "name" : "cluster1643\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1660\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1652\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1672\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1654\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1648\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1657\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1655\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1641\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1669\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1662\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1649\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1666\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1678\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1650\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1679\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "192\\.30\\.127\\.128",
+    "children" : [ {
+      "name" : "cluster1482\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1517\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1491\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1498\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1490\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1504\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1515\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1480\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1518\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1493\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1503\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "194\\.6\\.129\\.0",
+    "children" : [ {
+      "name" : "cluster50520\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50539\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50530\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50526\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50543\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "192\\.30\\.62\\.192",
+    "children" : [ {
+      "name" : "cluster50407\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50409\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50423\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50427\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50429\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50416\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50420\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50418\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50411\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50425\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "194\\.6\\.134\\.0",
+    "children" : [ {
+      "name" : "cluster50275\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50254\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50272\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50274\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50245\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50276\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50243\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50252\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50263\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50279\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50273\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50261\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50260\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "194\\.6\\.128\\.64",
+    "children" : [ {
+      "name" : "cluster1639\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1615\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1628\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1635\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1611\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1607\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1629\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1623\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1633\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1610\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1632\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1614\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1636\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1600\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1626\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1602\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "194\\.6\\.132\\.192",
+    "children" : [ {
+      "name" : "cluster50047\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50055\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50051\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50059\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50050\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50076\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50077\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50046\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50053\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50057\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50072\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50044\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50043\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50058\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "192\\.30\\.117\\.64",
+    "children" : [ {
+      "name" : "cluster1193\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1175\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1185\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1171\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1174\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1167\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1180\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "192\\.30\\.63\\.64",
+    "children" : [ {
+      "name" : "cluster1755\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1757\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1725\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1727\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1736\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1722\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1752\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1759\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1758\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1732\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1743\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1731\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1733\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1751\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  } ]
+}

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


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


+ 110 - 0
src/test/tools/data/rumen/small-trace-test/truncated-job-tracker-log

@@ -0,0 +1,110 @@
+!!FILE=cluster-jt1.red.ygrid.megatron.com_1240335959557_job_200904211745_0002_hadoopqa_word+count!!
+Meta VERSION="1" .
+Job JOBID="job_200904211745_0002" JOBNAME="word count" USER="hadoopqa" SUBMIT_TIME="1240335962848" JOBCONF="hdfs://cluster-nn1\.secondleveldomain\.com/mapredsystem/hadoop/mapredsystem/job_200904211745_0002/job\.xml" .
+Job JOBID="job_200904211745_0002" JOB_PRIORITY="NORMAL" .
+Job JOBID="job_200904211745_0002" LAUNCH_TIME="1240335964437" TOTAL_MAPS="20" TOTAL_REDUCES="1" JOB_STATUS="PREP" .
+Task TASKID="task_200904211745_0002_m_000021" TASK_TYPE="SETUP" START_TIME="1240336739508" SPLITS="" .
+MapAttempt TASK_TYPE="SETUP" TASKID="task_200904211745_0002_m_000021" TASK_ATTEMPT_ID="attempt_200904211745_0002_m_000021_0" START_TIME="1240336739565" TRACKER_NAME="tracker_cluster1028\.secondleveldomain\.com:localhost/127\.0\.0\.1:52187" HTTP_PORT="50060" .
+MapAttempt TASK_TYPE="SETUP" TASKID="task_200904211745_0002_m_000021" TASK_ATTEMPT_ID="attempt_200904211745_0002_m_000021_0" TASK_STATUS="SUCCESS" FINISH_TIME="1240336742217" HOSTNAME="/192\.30\.63\.192/cluster1028\.secondleveldomain\.com" STATE_STRING="setup" COUNTERS="{(org\.apache\.hadoop\.mapred\.Task$Counter)(Map-Reduce Framework)[(SPILLED_RECORDS)(Spilled Records)(0)]}" .
+Task TASKID="task_200904211745_0002_m_000021" TASK_TYPE="SETUP" TASK_STATUS="SUCCESS" FINISH_TIME="1240336752812" COUNTERS="{(org\.apache\.hadoop\.mapred\.Task$Counter)(Map-Reduce Framework)[(SPILLED_RECORDS)(Spilled Records)(0)]}" .
+Job JOBID="job_200904211745_0002" JOB_STATUS="RUNNING" .
+Task TASKID="task_200904211745_0002_m_000010" TASK_TYPE="MAP" START_TIME="1240336753705" SPLITS="/192\.30\.126\.128/cluster1369\.secondleveldomain\.com,/194\.6\.129\.128/cluster1854\.secondleveldomain\.com,/194\.6\.129\.128/cluster1872\.secondleveldomain\.com" .
+Task TASKID="task_200904211745_0002_m_000017" TASK_TYPE="MAP" START_TIME="1240336753750" SPLITS="/194\.6\.129\.64/cluster1803\.secondleveldomain\.com,/194\.6\.130\.0/cluster1930\.secondleveldomain\.com,/194\.6\.130\.0/cluster1932\.secondleveldomain\.com" .
+Task TASKID="task_200904211745_0002_m_000019" TASK_TYPE="MAP" START_TIME="1240336753796" SPLITS="/194\.6\.128\.192/cluster50481\.secondleveldomain\.com,/194\.6\.128\.192/cluster50505\.secondleveldomain\.com,/194\.6\.134\.128/cluster50359\.secondleveldomain\.com" .
+Task TASKID="task_200904211745_0002_m_000005" TASK_TYPE="MAP" START_TIME="1240336753840" SPLITS="/192\.30\.117\.192/cluster1245\.secondleveldomain\.com,/192\.30\.117\.192/cluster1261\.secondleveldomain\.com,/194\.6\.132\.128/cluster50021\.secondleveldomain\.com" .
+Task TASKID="task_200904211745_0002_m_000000" TASK_TYPE="MAP" START_TIME="1240336753888" SPLITS="/192\.30\.117\.128/cluster1236\.secondleveldomain\.com,/194\.6\.129\.192/cluster1889\.secondleveldomain\.com,/194\.6\.129\.192/cluster1911\.secondleveldomain\.com" .
+Task TASKID="task_200904211745_0002_m_000009" TASK_TYPE="MAP" START_TIME="1240336753938" SPLITS="/192\.30\.117\.128/cluster1227\.secondleveldomain\.com,/192\.30\.117\.192/cluster1259\.secondleveldomain\.com,/192\.30\.117\.192/cluster1260\.secondleveldomain\.com" .
+Task TASKID="task_200904211745_0002_m_000002" TASK_TYPE="MAP" START_TIME="1240336753987" SPLITS="/192\.30\.126\.64/cluster1357\.secondleveldomain\.com,/192\.30\.127\.64/cluster1450\.secondleveldomain\.com,/192\.30\.127\.64/cluster1457\.secondleveldomain\.com" .
+Task TASKID="task_200904211745_0002_m_000004" TASK_TYPE="MAP" START_TIME="1240336754030" SPLITS="/192\.30\.126\.0/cluster1294\.secondleveldomain\.com,/192\.30\.126\.0/cluster1288\.secondleveldomain\.com,/194\.6\.129\.128/cluster1876\.secondleveldomain\.com" .
+Task TASKID="task_200904211745_0002_m_000007" TASK_TYPE="MAP" START_TIME="1240336754077" SPLITS="/192\.30\.127\.64/cluster1466\.secondleveldomain\.com,/194\.6\.133\.192/cluster50218\.secondleveldomain\.com,/194\.6\.133\.192/cluster50232\.secondleveldomain\.com" .
+Task TASKID="task_200904211745_0002_m_000013" TASK_TYPE="MAP" START_TIME="1240336754124" SPLITS="/194\.6\.134\.64/cluster50286\.secondleveldomain\.com,/194\.6\.135\.64/cluster3071\.secondleveldomain\.com,/194\.6\.135\.64/cluster3049\.secondleveldomain\.com" .
+Task TASKID="task_200904211745_0002_m_000012" TASK_TYPE="MAP" START_TIME="1240336754176" SPLITS="/194\.6\.132\.192/cluster50057\.secondleveldomain\.com,/194\.6\.135\.128/cluster3112\.secondleveldomain\.com,/194\.6\.135\.128/cluster3082\.secondleveldomain\.com" .
+Task TASKID="task_200904211745_0002_m_000001" TASK_TYPE="MAP" START_TIME="1240336754223" SPLITS="/192\.30\.116\.128/cluster1080\.secondleveldomain\.com,/192\.30\.116\.128/cluster1097\.secondleveldomain\.com,/194\.6\.129\.0/cluster50543\.secondleveldomain\.com" .
+Task TASKID="task_200904211745_0002_m_000015" TASK_TYPE="MAP" START_TIME="1240336754270" SPLITS="/192\.30\.126\.192/cluster1412\.secondleveldomain\.com,/192\.30\.62\.192/cluster50427\.secondleveldomain\.com,/192\.30\.62\.192/cluster50411\.secondleveldomain\.com" .
+Task TASKID="task_200904211745_0002_m_000016" TASK_TYPE="MAP" START_TIME="1240336754319" SPLITS="/192\.30\.126\.128/cluster1393\.secondleveldomain\.com,/194\.6\.133\.64/cluster50130\.secondleveldomain\.com,/194\.6\.133\.64/cluster50141\.secondleveldomain\.com" .
+Task TASKID="task_200904211745_0002_m_000018" TASK_TYPE="MAP" START_TIME="1240336754366" SPLITS="/192\.30\.117\.128/cluster1223\.secondleveldomain\.com,/192\.30\.117\.128/cluster1200\.secondleveldomain\.com,/194\.6\.133\.64/cluster50152\.secondleveldomain\.com" .
+Task TASKID="task_200904211745_0002_m_000003" TASK_TYPE="MAP" START_TIME="1240336754409" SPLITS="/192\.30\.63\.64/cluster1733\.secondleveldomain\.com,/194\.6\.128\.64/cluster1607\.secondleveldomain\.com,/194\.6\.128\.64/cluster1639\.secondleveldomain\.com" .
+Task TASKID="task_200904211745_0002_m_000006" TASK_TYPE="MAP" START_TIME="1240336754452" SPLITS="/192\.30\.116\.64/cluster1064\.secondleveldomain\.com,/194\.6\.128\.192/cluster50510\.secondleveldomain\.com,/194\.6\.128\.192/cluster50478\.secondleveldomain\.com" .
+Task TASKID="task_200904211745_0002_m_000014" TASK_TYPE="MAP" START_TIME="1240336754500" SPLITS="/192\.30\.116\.64/cluster1059\.secondleveldomain\.com,/194\.6\.132\.192/cluster50053\.secondleveldomain\.com,/194\.6\.132\.192/cluster50050\.secondleveldomain\.com" .
+Task TASKID="task_200904211745_0002_m_000008" TASK_TYPE="MAP" START_TIME="1240336754548" SPLITS="/192\.30\.116\.192/cluster1157\.secondleveldomain\.com,/192\.30\.62\.128/cluster1718\.secondleveldomain\.com,/192\.30\.62\.128/cluster1694\.secondleveldomain\.com" .
+Task TASKID="task_200904211745_0002_m_000011" TASK_TYPE="MAP" START_TIME="1240336754596" SPLITS="/192\.30\.116\.192/cluster1143\.secondleveldomain\.com,/192\.30\.63\.192/cluster1004\.secondleveldomain\.com,/192\.30\.63\.192/cluster1020\.secondleveldomain\.com" .
+MapAttempt TASK_TYPE="MAP" TASKID="task_200904211745_0002_m_000015" TASK_ATTEMPT_ID="attempt_200904211745_0002_m_000015_0" START_TIME="1240336762622" TRACKER_NAME="tracker_cluster3104\.secondleveldomain\.com:localhost/127\.0\.0\.1:48449" HTTP_PORT="50060" .
+MapAttempt TASK_TYPE="MAP" TASKID="task_200904211745_0002_m_000015" TASK_ATTEMPT_ID="attempt_200904211745_0002_m_000015_0" TASK_STATUS="SUCCESS" FINISH_TIME="1240336765834" HOSTNAME="/194\.6\.135\.128/cluster3104\.secondleveldomain\.com" STATE_STRING="" COUNTERS="{(FileSystemCounters)(FileSystemCounters)[(HDFS_BYTES_READ)(HDFS_BYTES_READ)(53639)][(FILE_BYTES_WRITTEN)(FILE_BYTES_WRITTEN)(37170)]}{(org\.apache\.hadoop\.mapred\.Task$Counter)(Map-Reduce Framework)[(COMBINE_OUTPUT_RECORDS)(Combine output records)(5315)][(MAP_INPUT_RECORDS)(Map input records)(3601)][(SPILLED_RECORDS)(Spilled Records)(5315)][(MAP_OUTPUT_BYTES)(Map output bytes)(247925)][(COMBINE_INPUT_RECORDS)(Combine input records)(26425)][(MAP_OUTPUT_RECORDS)(Map output records)(26425)]}" .
+Task TASKID="task_200904211745_0002_m_000015" TASK_TYPE="MAP" TASK_STATUS="SUCCESS" FINISH_TIME="1240336774468" COUNTERS="{(FileSystemCounters)(FileSystemCounters)[(HDFS_BYTES_READ)(HDFS_BYTES_READ)(53639)][(FILE_BYTES_WRITTEN)(FILE_BYTES_WRITTEN)(37170)]}{(org\.apache\.hadoop\.mapred\.Task$Counter)(Map-Reduce Framework)[(COMBINE_OUTPUT_RECORDS)(Combine output records)(5315)][(MAP_INPUT_RECORDS)(Map input records)(3601)][(SPILLED_RECORDS)(Spilled Records)(5315)][(MAP_OUTPUT_BYTES)(Map output bytes)(247925)][(COMBINE_INPUT_RECORDS)(Combine input records)(26425)][(MAP_OUTPUT_RECORDS)(Map output records)(26425)]}" .
+MapAttempt TASK_TYPE="MAP" TASKID="task_200904211745_0002_m_000001" TASK_ATTEMPT_ID="attempt_200904211745_0002_m_000001_0" START_TIME="1240336754801" TRACKER_NAME="tracker_cluster1660\.secondleveldomain\.com:localhost/127\.0\.0\.1:40006" HTTP_PORT="50060" .
+MapAttempt TASK_TYPE="MAP" TASKID="task_200904211745_0002_m_000001" TASK_ATTEMPT_ID="attempt_200904211745_0002_m_000001_0" TASK_STATUS="SUCCESS" FINISH_TIME="1240336758231" HOSTNAME="/192\.30\.62\.64/cluster1660\.secondleveldomain\.com" STATE_STRING="" COUNTERS="{(FileSystemCounters)(FileSystemCounters)[(HDFS_BYTES_READ)(HDFS_BYTES_READ)(148286)][(FILE_BYTES_WRITTEN)(FILE_BYTES_WRITTEN)(37170)]}{(org\.apache\.hadoop\.mapred\.Task$Counter)(Map-Reduce Framework)[(COMBINE_OUTPUT_RECORDS)(Combine output records)(5315)][(MAP_INPUT_RECORDS)(Map input records)(3601)][(SPILLED_RECORDS)(Spilled Records)(5315)][(MAP_OUTPUT_BYTES)(Map output bytes)(247925)][(COMBINE_INPUT_RECORDS)(Combine input records)(26425)][(MAP_OUTPUT_RECORDS)(Map output records)(26425)]}" .
+Task TASKID="task_200904211745_0002_m_000001" TASK_TYPE="MAP" TASK_STATUS="SUCCESS" FINISH_TIME="1240336774476" COUNTERS="{(FileSystemCounters)(FileSystemCounters)[(HDFS_BYTES_READ)(HDFS_BYTES_READ)(148286)][(FILE_BYTES_WRITTEN)(FILE_BYTES_WRITTEN)(37170)]}{(org\.apache\.hadoop\.mapred\.Task$Counter)(Map-Reduce Framework)[(COMBINE_OUTPUT_RECORDS)(Combine output records)(5315)][(MAP_INPUT_RECORDS)(Map input records)(3601)][(SPILLED_RECORDS)(Spilled Records)(5315)][(MAP_OUTPUT_BYTES)(Map output bytes)(247925)][(COMBINE_INPUT_RECORDS)(Combine input records)(26425)][(MAP_OUTPUT_RECORDS)(Map output records)(26425)]}" .
+MapAttempt TASK_TYPE="MAP" TASKID="task_200904211745_0002_m_000012" TASK_ATTEMPT_ID="attempt_200904211745_0002_m_000012_0" START_TIME="1240336768390" TRACKER_NAME="tracker_cluster3097\.secondleveldomain\.com:localhost/127\.0\.0\.1:32840" HTTP_PORT="50060" .
+MapAttempt TASK_TYPE="MAP" TASKID="task_200904211745_0002_m_000012" TASK_ATTEMPT_ID="attempt_200904211745_0002_m_000012_0" TASK_STATUS="SUCCESS" FINISH_TIME="1240336771627" HOSTNAME="/194\.6\.135\.128/cluster3097\.secondleveldomain\.com" STATE_STRING="" COUNTERS="{(FileSystemCounters)(FileSystemCounters)[(HDFS_BYTES_READ)(HDFS_BYTES_READ)(53639)][(FILE_BYTES_WRITTEN)(FILE_BYTES_WRITTEN)(37170)]}{(org\.apache\.hadoop\.mapred\.Task$Counter)(Map-Reduce Framework)[(COMBINE_OUTPUT_RECORDS)(Combine output records)(5315)][(MAP_INPUT_RECORDS)(Map input records)(3601)][(SPILLED_RECORDS)(Spilled Records)(5315)][(MAP_OUTPUT_BYTES)(Map output bytes)(247925)][(COMBINE_INPUT_RECORDS)(Combine input records)(26425)][(MAP_OUTPUT_RECORDS)(Map output records)(26425)]}" .
+Task TASKID="task_200904211745_0002_m_000012" TASK_TYPE="MAP" TASK_STATUS="SUCCESS" FINISH_TIME="1240336774482" COUNTERS="{(FileSystemCounters)(FileSystemCounters)[(HDFS_BYTES_READ)(HDFS_BYTES_READ)(53639)][(FILE_BYTES_WRITTEN)(FILE_BYTES_WRITTEN)(37170)]}{(org\.apache\.hadoop\.mapred\.Task$Counter)(Map-Reduce Framework)[(COMBINE_OUTPUT_RECORDS)(Combine output records)(5315)][(MAP_INPUT_RECORDS)(Map input records)(3601)][(SPILLED_RECORDS)(Spilled Records)(5315)][(MAP_OUTPUT_BYTES)(Map output bytes)(247925)][(COMBINE_INPUT_RECORDS)(Combine input records)(26425)][(MAP_OUTPUT_RECORDS)(Map output records)(26425)]}" .
+Task TASKID="task_200904211745_0002_r_000000" TASK_TYPE="REDUCE" START_TIME="1240336774548" SPLITS="" .
+MapAttempt TASK_TYPE="MAP" TASKID="task_200904211745_0002_m_000011" TASK_ATTEMPT_ID="attempt_200904211745_0002_m_000011_0" START_TIME="1240336744261" TRACKER_NAME="tracker_cluster3098\.secondleveldomain\.com:localhost/127\.0\.0\.1:53110" HTTP_PORT="50060" .
+MapAttempt TASK_TYPE="MAP" TASKID="task_200904211745_0002_m_000011" TASK_ATTEMPT_ID="attempt_200904211745_0002_m_000011_0" TASK_STATUS="SUCCESS" FINISH_TIME="1240336747517" HOSTNAME="/194\.6\.135\.128/cluster3098\.secondleveldomain\.com" STATE_STRING="" COUNTERS="{(FileSystemCounters)(FileSystemCounters)[(HDFS_BYTES_READ)(HDFS_BYTES_READ)(53639)][(FILE_BYTES_WRITTEN)(FILE_BYTES_WRITTEN)(37170)]}{(org\.apache\.hadoop\.mapred\.Task$Counter)(Map-Reduce Framework)[(COMBINE_OUTPUT_RECORDS)(Combine output records)(5315)][(MAP_INPUT_RECORDS)(Map input records)(3601)][(SPILLED_RECORDS)(Spilled Records)(5315)][(MAP_OUTPUT_BYTES)(Map output bytes)(247925)][(COMBINE_INPUT_RECORDS)(Combine input records)(26425)][(MAP_OUTPUT_RECORDS)(Map output records)(26425)]}" .
+Task TASKID="task_200904211745_0002_m_000011" TASK_TYPE="MAP" TASK_STATUS="SUCCESS" FINISH_TIME="1240336776031" COUNTERS="{(FileSystemCounters)(FileSystemCounters)[(HDFS_BYTES_READ)(HDFS_BYTES_READ)(53639)][(FILE_BYTES_WRITTEN)(FILE_BYTES_WRITTEN)(37170)]}{(org\.apache\.hadoop\.mapred\.Task$Counter)(Map-Reduce Framework)[(COMBINE_OUTPUT_RECORDS)(Combine output records)(5315)][(MAP_INPUT_RECORDS)(Map input records)(3601)][(SPILLED_RECORDS)(Spilled Records)(5315)][(MAP_OUTPUT_BYTES)(Map output bytes)(247925)][(COMBINE_INPUT_RECORDS)(Combine input records)(26425)][(MAP_OUTPUT_RECORDS)(Map output records)(26425)]}" .
+MapAttempt TASK_TYPE="MAP" TASKID="task_200904211745_0002_m_000003" TASK_ATTEMPT_ID="attempt_200904211745_0002_m_000003_0" START_TIME="1240336755433" TRACKER_NAME="tracker_cluster1551\.secondleveldomain\.com:localhost/127\.0\.0\.1:46404" HTTP_PORT="50060" .
+MapAttempt TASK_TYPE="MAP" TASKID="task_200904211745_0002_m_000003" TASK_ATTEMPT_ID="attempt_200904211745_0002_m_000003_0" TASK_STATUS="SUCCESS" FINISH_TIME="1240336758966" HOSTNAME="/192\.30\.127\.192/cluster1551\.secondleveldomain\.com" STATE_STRING="" COUNTERS="{(FileSystemCounters)(FileSystemCounters)[(HDFS_BYTES_READ)(HDFS_BYTES_READ)(148286)][(FILE_BYTES_WRITTEN)(FILE_BYTES_WRITTEN)(37170)]}{(org\.apache\.hadoop\.mapred\.Task$Counter)(Map-Reduce Framework)[(COMBINE_OUTPUT_RECORDS)(Combine output records)(5315)][(MAP_INPUT_RECORDS)(Map input records)(3601)][(SPILLED_RECORDS)(Spilled Records)(5315)][(MAP_OUTPUT_BYTES)(Map output bytes)(247925)][(COMBINE_INPUT_RECORDS)(Combine input records)(26425)][(MAP_OUTPUT_RECORDS)(Map output records)(26425)]}" .
+Task TASKID="task_200904211745_0002_m_000003" TASK_TYPE="MAP" TASK_STATUS="SUCCESS" FINISH_TIME="1240336816560" COUNTERS="{(FileSystemCounters)(FileSystemCounters)[(HDFS_BYTES_READ)(HDFS_BYTES_READ)(148286)][(FILE_BYTES_WRITTEN)(FILE_BYTES_WRITTEN)(37170)]}{(org\.apache\.hadoop\.mapred\.Task$Counter)(Map-Reduce Framework)[(COMBINE_OUTPUT_RECORDS)(Combine output records)(5315)][(MAP_INPUT_RECORDS)(Map input records)(3601)][(SPILLED_RECORDS)(Spilled Records)(5315)][(MAP_OUTPUT_BYTES)(Map output bytes)(247925)][(COMBINE_INPUT_RECORDS)(Combine input records)(26425)][(MAP_OUTPUT_RECORDS)(Map output records)(26425)]}" .
+MapAttempt TASK_TYPE="MAP" TASKID="task_200904211745_0002_m_000006" TASK_ATTEMPT_ID="attempt_200904211745_0002_m_000006_0" START_TIME="1240336747936" TRACKER_NAME="tracker_cluster1405\.secondleveldomain\.com:localhost/127\.0\.0\.1:35101" HTTP_PORT="50060" .
+MapAttempt TASK_TYPE="MAP" TASKID="task_200904211745_0002_m_000006" TASK_ATTEMPT_ID="attempt_200904211745_0002_m_000006_0" TASK_STATUS="SUCCESS" FINISH_TIME="1240336751464" HOSTNAME="/192\.30\.126\.192/cluster1405\.secondleveldomain\.com" STATE_STRING="" COUNTERS="{(FileSystemCounters)(FileSystemCounters)[(HDFS_BYTES_READ)(HDFS_BYTES_READ)(148286)][(FILE_BYTES_WRITTEN)(FILE_BYTES_WRITTEN)(37170)]}{(org\.apache\.hadoop\.mapred\.Task$Counter)(Map-Reduce Framework)[(COMBINE_OUTPUT_RECORDS)(Combine output records)(5315)][(MAP_INPUT_RECORDS)(Map input records)(3601)][(SPILLED_RECORDS)(Spilled Records)(5315)][(MAP_OUTPUT_BYTES)(Map output bytes)(247925)][(COMBINE_INPUT_RECORDS)(Combine input records)(26425)][(MAP_OUTPUT_RECORDS)(Map output records)(26425)]}" .
+Task TASKID="task_200904211745_0002_m_000006" TASK_TYPE="MAP" TASK_STATUS="SUCCESS" FINISH_TIME="1240336817330" COUNTERS="{(FileSystemCounters)(FileSystemCounters)[(HDFS_BYTES_READ)(HDFS_BYTES_READ)(148286)][(FILE_BYTES_WRITTEN)(FILE_BYTES_WRITTEN)(37170)]}{(org\.apache\.hadoop\.mapred\.Task$Counter)(Map-Reduce Framework)[(COMBINE_OUTPUT_RECORDS)(Combine output records)(5315)][(MAP_INPUT_RECORDS)(Map input records)(3601)][(SPILLED_RECORDS)(Spilled Records)(5315)][(MAP_OUTPUT_BYTES)(Map output bytes)(247925)][(COMBINE_INPUT_RECORDS)(Combine input records)(26425)][(MAP_OUTPUT_RECORDS)(Map output records)(26425)]}" .
+MapAttempt TASK_TYPE="MAP" TASKID="task_200904211745_0002_m_000016" TASK_ATTEMPT_ID="attempt_200904211745_0002_m_000016_0" START_TIME="1240336759528" TRACKER_NAME="tracker_cluster1867\.secondleveldomain\.com:localhost/127\.0\.0\.1:43031" HTTP_PORT="50060" .
+MapAttempt TASK_TYPE="MAP" TASKID="task_200904211745_0002_m_000016" TASK_ATTEMPT_ID="attempt_200904211745_0002_m_000016_0" TASK_STATUS="SUCCESS" FINISH_TIME="1240336764456" HOSTNAME="/194\.6\.129\.128/cluster1867\.secondleveldomain\.com" STATE_STRING="" COUNTERS="{(FileSystemCounters)(FileSystemCounters)[(HDFS_BYTES_READ)(HDFS_BYTES_READ)(53639)][(FILE_BYTES_WRITTEN)(FILE_BYTES_WRITTEN)(37170)]}{(org\.apache\.hadoop\.mapred\.Task$Counter)(Map-Reduce Framework)[(COMBINE_OUTPUT_RECORDS)(Combine output records)(5315)][(MAP_INPUT_RECORDS)(Map input records)(3601)][(SPILLED_RECORDS)(Spilled Records)(5315)][(MAP_OUTPUT_BYTES)(Map output bytes)(247925)][(COMBINE_INPUT_RECORDS)(Combine input records)(26425)][(MAP_OUTPUT_RECORDS)(Map output records)(26425)]}" .
+Task TASKID="task_200904211745_0002_m_000016" TASK_TYPE="MAP" TASK_STATUS="SUCCESS" FINISH_TIME="1240336817903" COUNTERS="{(FileSystemCounters)(FileSystemCounters)[(HDFS_BYTES_READ)(HDFS_BYTES_READ)(53639)][(FILE_BYTES_WRITTEN)(FILE_BYTES_WRITTEN)(37170)]}{(org\.apache\.hadoop\.mapred\.Task$Counter)(Map-Reduce Framework)[(COMBINE_OUTPUT_RECORDS)(Combine output records)(5315)][(MAP_INPUT_RECORDS)(Map input records)(3601)][(SPILLED_RECORDS)(Spilled Records)(5315)][(MAP_OUTPUT_BYTES)(Map output bytes)(247925)][(COMBINE_INPUT_RECORDS)(Combine input records)(26425)][(MAP_OUTPUT_RECORDS)(Map output records)(26425)]}" .
+MapAttempt TASK_TYPE="MAP" TASKID="task_200904211745_0002_m_000019" TASK_ATTEMPT_ID="attempt_200904211745_0002_m_000019_0" START_TIME="1240336766565" TRACKER_NAME="tracker_cluster1771\.secondleveldomain\.com:localhost/127\.0\.0\.1:49430" HTTP_PORT="50060" .
+MapAttempt TASK_TYPE="MAP" TASKID="task_200904211745_0002_m_000019" TASK_ATTEMPT_ID="attempt_200904211745_0002_m_000019_0" TASK_STATUS="SUCCESS" FINISH_TIME="1240336770163" HOSTNAME="/192\.30\.63\.0/cluster1771\.secondleveldomain\.com" STATE_STRING="" COUNTERS="{(FileSystemCounters)(FileSystemCounters)[(HDFS_BYTES_READ)(HDFS_BYTES_READ)(53639)][(FILE_BYTES_WRITTEN)(FILE_BYTES_WRITTEN)(37170)]}{(org\.apache\.hadoop\.mapred\.Task$Counter)(Map-Reduce Framework)[(COMBINE_OUTPUT_RECORDS)(Combine output records)(5315)][(MAP_INPUT_RECORDS)(Map input records)(3601)][(SPILLED_RECORDS)(Spilled Records)(5315)][(MAP_OUTPUT_BYTES)(Map output bytes)(247925)][(COMBINE_INPUT_RECORDS)(Combine input records)(26425)][(MAP_OUTPUT_RECORDS)(Map output records)(26425)]}" .
+Task TASKID="task_200904211745_0002_m_000019" TASK_TYPE="MAP" TASK_STATUS="SUCCESS" FINISH_TIME="1240336818106" COUNTERS="{(FileSystemCounters)(FileSystemCounters)[(HDFS_BYTES_READ)(HDFS_BYTES_READ)(53639)][(FILE_BYTES_WRITTEN)(FILE_BYTES_WRITTEN)(37170)]}{(org\.apache\.hadoop\.mapred\.Task$Counter)(Map-Reduce Framework)[(COMBINE_OUTPUT_RECORDS)(Combine output records)(5315)][(MAP_INPUT_RECORDS)(Map input records)(3601)][(SPILLED_RECORDS)(Spilled Records)(5315)][(MAP_OUTPUT_BYTES)(Map output bytes)(247925)][(COMBINE_INPUT_RECORDS)(Combine input records)(26425)][(MAP_OUTPUT_RECORDS)(Map output records)(26425)]}" .
+MapAttempt TASK_TYPE="MAP" TASKID="task_200904211745_0002_m_000010" TASK_ATTEMPT_ID="attempt_200904211745_0002_m_000010_0" START_TIME="1240336647215" TRACKER_NAME="tracker_cluster1396\.secondleveldomain\.com:localhost/127\.0\.0\.1:46109" HTTP_PORT="50060" .
+MapAttempt TASK_TYPE="MAP" TASKID="task_200904211745_0002_m_000010" TASK_ATTEMPT_ID="attempt_200904211745_0002_m_000010_0" TASK_STATUS="SUCCESS" FINISH_TIME="1240336651127" HOSTNAME="/192\.30\.126\.128/cluster1396\.secondleveldomain\.com" STATE_STRING="" COUNTERS="{(FileSystemCounters)(FileSystemCounters)[(HDFS_BYTES_READ)(HDFS_BYTES_READ)(53639)][(FILE_BYTES_WRITTEN)(FILE_BYTES_WRITTEN)(37170)]}{(org\.apache\.hadoop\.mapred\.Task$Counter)(Map-Reduce Framework)[(COMBINE_OUTPUT_RECORDS)(Combine output records)(5315)][(MAP_INPUT_RECORDS)(Map input records)(3601)][(SPILLED_RECORDS)(Spilled Records)(5315)][(MAP_OUTPUT_BYTES)(Map output bytes)(247925)][(COMBINE_INPUT_RECORDS)(Combine input records)(26425)][(MAP_OUTPUT_RECORDS)(Map output records)(26425)]}" .
+Task TASKID="task_200904211745_0002_m_000010" TASK_TY .
+MapAttempt TASK_TYPE="MAP" TASKID="task_200904211745_0002_m_000017" TASK_ATTEMPT_ID="attempt_200904211745_0002_m_000017_0" START_TIME="1240336752204" TRACKER_NAME="tracker_cluster1553\.secondleveldomain\.com:localhost/127\.0\.0\.1:33829" HTTP_PORT="50060" .
+MapAttempt TASK_TYPE="MAP" TASKID="task_200904211745_0002_m_000017" TASK_ATTEMPT_ID="attempt_200904211745_0002_m_000017_0" TASK_STATUS="SUCCESS" FINISH_TIME="1240336755959" HOSTNAME="/192\.30\.127\.192/cluster1553\.secondleveldomain\.com" STATE_STRING="" COUNTERS="{(FileSystemCounters)(FileSystemCounters)[(HDFS_BYTES_READ)(HDFS_BYTES_READ)(53639)][(FILE_BYTES_WRITTEN)(FILE_BYTES_WRITTEN)(37170)]}{(org\.apache\.hadoop\.mapred\.Task$Counter)(Map-Reduce Framework)[(COMBINE_OUTPUT_RECORDS)(Combine output records)(5315)][(MAP_INPUT_RECORDS)(Map input records)(3601)][(SPILLED_RECORDS)(Spilled Records)(5315)][(MAP_OUTPUT_BYTES)(Map output bytes)(247925)][(COMBINE_INPUT_RECORDS)(Combine input records)(26425)][(MAP_OUTPUT_RECORDS)(Map output records)(26425)]}" .
+Task TASKID="task_200904211745_0002_m_000017" TASK_TYPE="MAP" TASK_STATUS="SUCCESS" FINISH_TIME="1240336818110" COUNTERS="{(FileSystemCounters)(FileSystemCounters)[(HDFS_BYTES_READ)(HDFS_BYTES_READ)(53639)][(FILE_BYTES_WRITTEN)(FILE_BYTES_WRITTEN)(37170)]}{(org\.apache\.hadoop\.mapred\.Task$Counter)(Map-Reduce Framework)[(COMBINE_OUTPUT_RECORDS)(Combine output records)(5315)][(MAP_INPUT_RECORDS)(Map input records)(3601)][(SPILLED_RECORDS)(Spilled Records)(5315)][(MAP_OUTPUT_BYTES)(Map output bytes)(247925)][(COMBINE_INPUT_RECORDS)(Combine input records)(26425)][(MAP_OUTPUT_RECORDS)(Map output records)(26425)]}" .
+MapAttempt TASK_TYPE="MAP" TASKID="task_200904211745_0002_m_000000" TASK_ATTEMPT_ID="attempt_200904211745_0002_m_000000_0" START_TIME="1240336755247" TRACKER_NAME="tracker_cluster1218\.secondleveldomain\.com:localhost/127\.0\.0\.1:37882" HTTP_PORT="50060" .
+MapAttempt TASK_TYPE="MAP" TASKID="task_200904211745_0002_m_000000" TASK_ATTEMPT_ID="attempt_200904211745_0002_m_000000_0" TASK_STATUS="SUCCESS" FINISH_TIME="1240336763432" HOSTNAME="/192\.30\.117\.128/cluster1218\.secondleveldomain\.com" STATE_STRING="" COUNTERS="{(FileSystemCounters)(FileSystemCounters)[(HDFS_BYTES_READ)(HDFS_BYTES_READ)(148286)][(FILE_BYTES_WRITTEN)(FILE_BYTES_WRITTEN)(37170)]}{(org\.apache\.hadoop\.mapred\.Task$Counter)(Map-Reduce Framework)[(COMBINE_OUTPUT_RECORDS)(Combine output records)(5315)][(MAP_INPUT_RECORDS)(Map input records)(3601)][(SPILLED_RECORDS)(Spilled Records)(5315)][(MAP_OUTPUT_BYTES)(Map output bytes)(247925)][(COMBINE_INPUT_RECORDS)(Combine input records)(26425)][(MAP_OUTPUT_RECORDS)(Map output records)(26425)]}" .
+Task TASKID="task_200904211745_0002_m_000000" TASK_TYPE="MAP" TASK_STATUS="SUCCESS" FINISH_TIME="1240336818113" COUNTERS="{(FileSystemCounters)(FileSystemCounters)[(HDFS_BYTES_READ)(HDFS_BYTES_READ)(148286)][(FILE_BYTES_WRITTEN)(FILE_BYTES_WRITTEN)(37170)]}{(org\.apache\.hadoop\.mapred\.Task$Counter)(Map-Reduce Framework)[(COMBINE_OUTPUT_RECORDS)(Combine output records)(5315)][(MAP_INPUT_RECORDS)(Map input records)(3601)][(SPILLED_RECORDS)(Spilled Records)(5315)][(MAP_OUTPUT_BYTES)(Map output bytes)(247925)][(COMBINE_INPUT_RECORDS)(Combine input records)(26425)][(MAP_OUTPUT_RECORDS)(Map output records)(26425)]}" .
+MapAttempt TASK_TYPE="MAP" TASKID="task_200904211745_0002_m_000005" TASK_ATTEMPT_ID="attempt_200904211745_0002_m_000005_0" START_TIME="1240336765887" TRACKER_NAME="tracker_cluster1779\.secondleveldomain\.com:localhost/127\.0\.0\.1:57465" HTTP_PORT="50060" .
+MapAttempt TASK_TYPE="MAP" TASKID="task_200904211745_0002_m_000005" TASK_ATTEMPT_ID="attempt_200904211745_0002_m_000005_0" TASK_STATUS="SUCCESS" FINISH_TIME="1240336785565" HOSTNAME="/192\.30\.63\.0/cluster1779\.secondleveldomain\.com" STATE_STRING="" COUNTERS="{(FileSystemCounters)(FileSystemCounters)[(HDFS_BYTES_READ)(HDFS_BYTES_READ)(148286)][(FILE_BYTES_WRITTEN)(FILE_BYTES_WRITTEN)(37170)]}{(org\.apache\.hadoop\.mapred\.Task$Counter)(Map-Reduce Framework)[(COMBINE_OUTPUT_RECORDS)(Combine output records)(5315)][(MAP_INPUT_RECORDS)(Map input records)(3601)][(SPILLED_RECORDS)(Spilled Records)(5315)][(MAP_OUTPUT_BYTES)(Map output bytes)(247925)][(COMBINE_INPUT_RECORDS)(Combine input records)(26425)][(MAP_OUTPUT_RECORDS)(Map output records)(26425)]}" .
+Task TASKID="task_200904211745_0002_m_000005" TASK_TYPE="MAP" TASK_STATUS="SUCCESS" FINISH_TIME="1240336818114" COUNTERS="{(FileSystemCounters)(FileSystemCounters)[(HDFS_BYTES_READ)(HDFS_BYTES_READ)(148286)][(FILE_BYTES_WRITTEN)(FILE_BYTES_WRITTEN)(37170)]}{(org\.apache\.hadoop\.mapred\.Task$Counter)(Map-Reduce Framework)[(COMBINE_OUTPUT_RECORDS)(Combine output records)(5315)][(MAP_INPUT_RECORDS)(Map input records)(3601)][(SPILLED_RECORDS)(Spilled Records)(5315)][(MAP_OUTPUT_BYTES)(Map output bytes)(247925)][(COMBINE_INPUT_RECORDS)(Combine input records)(26425)][(MAP_OUTPUT_RECORDS)(Map output records)(26425)]}" .
+MapAttempt TASK_TYPE="MAP" TASKID="task_200904211745_0002_m_000002" TASK_ATTEMPT_ID="attempt_200904211745_0002_m_000002_0" START_TIME="1240336754665" TRACKER_NAME="tracker_cluster1670\.secondleveldomain\.com:localhost/127\.0\.0\.1:47698" HTTP_PORT="50060" .
+MapAttempt TASK_TYPE="MAP" TASKID="task_200904211745_0002_m_000002" TASK_ATTEMPT_ID="attempt_200904211745_0002_m_000002_0" TASK_STATUS="FAILED" FINISH_TIME="1240336777673" HOSTNAME="cluster1670\.secondleveldomain\.com" ERROR="java\.io\.IOException: Task process exit with nonzero status of 15\.
+	at org\.apache\.hadoop\.mapred\.TaskRunner\.run(TaskRunner\.java:424)
+,java\.io\.IOException: Task process exit with nonzero status of 15\.
+	at org\.apache\.hadoop\.mapred\.TaskRunner\.run(TaskRunner\.java:424)
+" .
+MapAttempt TASK_TYPE="MAP" TASKID="task_200904211745_0002_m_000009" TASK_ATTEMPT_ID="attempt_200904211745_0002_m_000009_0" START_TIME="1240336758229" TRACKER_NAME="tracker_cluster1586\.secondleveldomain\.com:localhost/127\.0\.0\.1:38422" HTTP_PORT="50060" .
+MapAttempt TASK_TYPE="MAP" TASKID="task_200904211745_0002_m_000009" TASK_ATTEMPT_ID="attempt_200904211745_0002_m_000009_0" TASK_STATUS="SUCCESS" FINISH_TIME="1240336761612" HOSTNAME="/194\.6\.128\.0/cluster1586\.secondleveldomain\.com" STATE_STRING="" COUNTERS="{(FileSystemCounters)(FileSystemCounters)[(HDFS_BYTES_READ)(HDFS_BYTES_READ)(148286)][(FILE_BYTES_WRITTEN)(FILE_BYTES_WRITTEN)(37170)]}{(org\.apache\.hadoop\.mapred\.Task$Counter)(Map-Reduce Framework)[(COMBINE_OUTPUT_RECORDS)(Combine output records)(5315)][(MAP_INPUT_RECORDS)(Map input records)(3601)][(SPILLED_RECORDS)(Spilled Records)(5315)][(MAP_OUTPUT_BYTES)(Map output bytes)(247925)][(COMBINE_INPUT_RECORDS)(Combine input records)(26425)][(MAP_OUTPUT_RECORDS)(Map output records)(26425)]}" .
+Task TASKID="task_200904211745_0002_m_000009" TASK_TYPE="MAP" TASK_STATUS="SUCCESS" FINISH_TIME="1240336818116" COUNTERS="{(FileSystemCounters)(FileSystemCounters)[(HDFS_BYTES_READ)(HDFS_BYTES_READ)(148286)][(FILE_BYTES_WRITTEN)(FILE_BYTES_WRITTEN)(37170)]}{(org\.apache\.hadoop\.mapred\.Task$Counter)(Map-Reduce Framework)[(COMBINE_OUTPUT_RECORDS)(Combine output records)(5315)][(MAP_INPUT_RECORDS)(Map input records)(3601)][(SPILLED_RECORDS)(Spilled Records)(5315)][(MAP_OUTPUT_BYTES)(Map output bytes)(247925)][(COMBINE_INPUT_RECORDS)(Combine input records)(26425)][(MAP_OUTPUT_RECORDS)(Map output records)(26425)]}" .
+MapAttempt TASK_TYPE="MAP" TASKID="task_200904211745_0002_m_000004" TASK_ATTEMPT_ID="attempt_200904211745_0002_m_000004_0" START_TIME="1240336757380" TRACKER_NAME="tracker_cluster1869\.secondleveldomain\.com:localhost/127\.0\.0\.1:40050" HTTP_PORT="50060" .
+MapAttempt TASK_TYPE="MAP" TASKID="task_200904211745_0002_m_000004" TASK_ATTEMPT_ID="attempt_200904211745_0002_m_000004_0" TASK_STATUS="SUCCESS" FINISH_TIME="1240336763176" HOSTNAME="/194\.6\.129\.128/cluster1869\.secondleveldomain\.com" STATE_STRING="" COUNTERS="{(FileSystemCounters)(FileSystemCounters)[(HDFS_BYTES_READ)(HDFS_BYTES_READ)(148286)][(FILE_BYTES_WRITTEN)(FILE_BYTES_WRITTEN)(37170)]}{(org\.apache\.hadoop\.mapred\.Task$Counter)(Map-Reduce Framework)[(COMBINE_OUTPUT_RECORDS)(Combine output records)(5315)][(MAP_INPUT_RECORDS)(Map input records)(3601)][(SPILLED_RECORDS)(Spilled Records)(5315)][(MAP_OUTPUT_BYTES)(Map output bytes)(247925)][(COMBINE_INPUT_RECORDS)(Combine input records)(26425)][(MAP_OUTPUT_RECORDS)(Map output records)(26425)]}" .
+Task TASKID="task_200904211745_0002_m_000004" TASK_TYPE="MAP" TASK_STATUS="SUCCESS" FINISH_TIME="1240336818118" COUNTERS="{(FileSystemCounters)(FileSystemCounters)[(HDFS_BYTES_READ)(HDFS_BYTES_READ)(148286)][(FILE_BYTES_WRITTEN)(FILE_BYTES_WRITTEN)(37170)]}{(org\.apache\.hadoop\.mapred\.Task$Counter)(Map-Reduce Framework)[(COMBINE_OUTPUT_RECORDS)(Combine output records)(5315)][(MAP_INPUT_RECORDS)(Map input records)(3601)][(SPILLED_RECORDS)(Spilled Records)(5315)][(MAP_OUTPUT_BYTES)(Map output bytes)(247925)][(COMBINE_INPUT_RECORDS)(Combine input records)(26425)][(MAP_OUTPUT_RECORDS)(Map output records)(26425)]}" .
+MapAttempt TASK_TYPE="MAP" TASKID="task_200904211745_0002_m_000007" TASK_ATTEMPT_ID="attempt_200904211745_0002_m_000007_0" START_TIME="1240336763994" TRACKER_NAME="tracker_cluster1770\.secondleveldomain\.com:localhost/127\.0\.0\.1:52486" HTTP_PORT="50060" .
+MapAttempt TASK_TYPE="MAP" TASKID="task_200904211745_0002_m_000007" TASK_ATTEMPT_ID="attempt_200904211745_0002_m_000007_0" TASK_STATUS="SUCCESS" FINISH_TIME="1240336767750" HOSTNAME="/192\.30\.63\.0/cluster1770\.secondleveldomain\.com" STATE_STRING="" COUNTERS="{(FileSystemCounters)(FileSystemCounters)[(HDFS_BYTES_READ)(HDFS_BYTES_READ)(148286)][(FILE_BYTES_WRITTEN)(FILE_BYTES_WRITTEN)(37170)]}{(org\.apache\.hadoop\.mapred\.Task$Counter)(Map-Reduce Framework)[(COMBINE_OUTPUT_RECORDS)(Combine output records)(5315)][(MAP_INPUT_RECORDS)(Map input records)(3601)][(SPILLED_RECORDS)(Spilled Records)(5315)][(MAP_OUTPUT_BYTES)(Map output bytes)(247925)][(COMBINE_INPUT_RECORDS)(Combine input records)(26425)][(MAP_OUTPUT_RECORDS)(Map output records)(26425)]}" .
+Task TASKID="task_200904211745_0002_m_000007" TASK_TYPE="MAP" TASK_STATUS="SUCCESS" FINISH_TIME="1240336818119" COUNTERS="{(FileSystemCounters)(FileSystemCounters)[(HDFS_BYTES_READ)(HDFS_BYTES_READ)(148286)][(FILE_BYTES_WRITTEN)(FILE_BYTES_WRITTEN)(37170)]}{(org\.apache\.hadoop\.mapred\.Task$Counter)(Map-Reduce Framework)[(COMBINE_OUTPUT_RECORDS)(Combine output records)(5315)][(MAP_INPUT_RECORDS)(Map input records)(3601)][(SPILLED_RECORDS)(Spilled Records)(5315)][(MAP_OUTPUT_BYTES)(Map output bytes)(247925)][(COMBINE_INPUT_RECORDS)(Combine input records)(26425)][(MAP_OUTPUT_RECORDS)(Map output records)(26425)]}" .
+MapAttempt TASK_TYPE="MAP" TASKID="task_200904211745_0002_m_000013" TASK_ATTEMPT_ID="attempt_200904211745_0002_m_000013_0" START_TIME="1240336758341" TRACKER_NAME="tracker_cluster1816\.secondleveldomain\.com:localhost/127\.0\.0\.1:41947" HTTP_PORT="50060" .
+MapAttempt TASK_TYPE="MAP" TASKID="task_200904211745_0002_m_000013" TASK_ATTEMPT_ID="attempt_200904211745_0002_m_000013_0" TASK_STATUS="SUCCESS" FINISH_TIME="1240336762025" HOSTNAME="/194\.6\.129\.64/cluster1816\.secondleveldomain\.com" STATE_STRING="" COUNTERS="{(FileSystemCounters)(FileSystemCounters)[(HDFS_BYTES_READ)(HDFS_BYTES_READ)(53639)][(FILE_BYTES_WRITTEN)(FILE_BYTES_WRITTEN)(37170)]}{(org\.apache\.hadoop\.mapred\.Task$Counter)(Map-Reduce Framework)[(COMBINE_OUTPUT_RECORDS)(Combine output records)(5315)][(MAP_INPUT_RECORDS)(Map input records)(3601)][(SPILLED_RECORDS)(Spilled Records)(5315)][(MAP_OUTPUT_BYTES)(Map output bytes)(247925)][(COMBINE_INPUT_RECORDS)(Combine input records)(26425)][(MAP_OUTPUT_RECORDS)(Map output records)(26425)]}" .
+Task TASKID="task_200904211745_0002_m_000013" TASK_TYPE="MAP" TASK_STATUS="SUCCESS" FINISH_TIME="1240336818120" COUNTERS="{(FileSystemCounters)(FileSystemCounters)[(HDFS_BYTES_READ)(HDFS_BYTES_READ)(53639)][(FILE_BYTES_WRITTEN)(FILE_BYTES_WRITTEN)(37170)]}{(org\.apache\.hadoop\.mapred\.Task$Counter)(Map-Reduce Framework)[(COMBINE_OUTPUT_RECORDS)(Combine output records)(5315)][(MAP_INPUT_RECORDS)(Map input records)(3601)][(SPILLED_RECORDS)(Spilled Records)(5315)][(MAP_OUTPUT_BYTES)(Map output bytes)(247925)][(COMBINE_INPUT_RECORDS)(Combine input records)(26425)][(MAP_OUTPUT_RECORDS)(Map output records)(26425)]}" .
+MapAttempt TASK_TYPE="MAP" TASKID="task_200904211745_0002_m_000018" TASK_ATTEMPT_ID="attempt_200904211745_0002_m_000018_0" START_TIME="1240336759909" TRACKER_NAME="tracker_cluster1649\.secondleveldomain\.com:localhost/127\.0\.0\.1:36608" HTTP_PORT="50060" .
+MapAttempt TASK_TYPE="MAP" TASKID="task_200904211745_0002_m_000018" TASK_ATTEMPT_ID="attempt_200904211745_0002_m_000018_0" TASK_STATUS="SUCCESS" FINISH_TIME="1240336763727" HOSTNAME="/192\.30\.62\.64/cluster1649\.secondleveldomain\.com" STATE_STRING="" COUNTERS="{(FileSystemCounters)(FileSystemCounters)[(HDFS_BYTES_READ)(HDFS_BYTES_READ)(53639)][(FILE_BYTES_WRITTEN)(FILE_BYTES_WRITTEN)(37170)]}{(org\.apache\.hadoop\.mapred\.Task$Counter)(Map-Reduce Framework)[(COMBINE_OUTPUT_RECORDS)(Combine output records)(5315)][(MAP_INPUT_RECORDS)(Map input records)(3601)][(SPILLED_RECORDS)(Spilled Records)(5315)][(MAP_OUTPUT_BYTES)(Map output bytes)(247925)][(COMBINE_INPUT_RECORDS)(Combine input records)(26425)][(MAP_OUTPUT_RECORDS)(Map output records)(26425)]}" .
+Task TASKID="task_200904211745_0002_m_000018" TASK_TYPE="MAP" TASK_STATUS="SUCCESS" FINISH_TIME="1240336818420" COUNTERS="{(FileSystemCounters)(FileSystemCounters)[(HDFS_BYTES_READ)(HDFS_BYTES_READ)(53639)][(FILE_BYTES_WRITTEN)(FILE_BYTES_WRITTEN)(37170)]}{(org\.apache\.hadoop\.mapred\.Task$Counter)(Map-Reduce Framework)[(COMBINE_OUTPUT_RECORDS)(Combine output records)(5315)][(MAP_INPUT_RECORDS)(Map input records)(3601)][(SPILLED_RECORDS)(Spilled Records)(5315)][(MAP_OUTPUT_BYTES)(Map output bytes)(247925)][(COMBINE_INPUT_RECORDS)(Combine input records)(26425)][(MAP_OUTPUT_RECORDS)(Map output records)(26425)]}" .
+MapAttempt TASK_TYPE="MAP" TASKID="task_200904211745_0002_m_000014" TASK_ATTEMPT_ID="attempt_200904211745_0002_m_000014_0" START_TIME="1240336756161" TRACKER_NAME="tracker_cluster1928\.secondleveldomain\.com:localhost/127\.0\.0\.1:58972" HTTP_PORT="50060" .
+MapAttempt TASK_TYPE="MAP" TASKID="task_200904211745_0002_m_000014" TASK_ATTEMPT_ID="attempt_200904211745_0002_m_000014_0" TASK_STATUS="SUCCESS" FINISH_TIME="1240336760016" HOSTNAME="/194\.6\.130\.0/cluster1928\.secondleveldomain\.com" STATE_STRING="" COUNTERS="{(FileSystemCounters)(FileSystemCounters)[(HDFS_BYTES_READ)(HDFS_BYTES_READ)(53639)][(FILE_BYTES_WRITTEN)(FILE_BYTES_WRITTEN)(37170)]}{(org\.apache\.hadoop\.mapred\.Task$Counter)(Map-Reduce Framework)[(COMBINE_OUTPUT_RECORDS)(Combine output records)(5315)][(MAP_INPUT_RECORDS)(Map input records)(3601)][(SPILLED_RECORDS)(Spilled Records)(5315)][(MAP_OUTPUT_BYTES)(Map output bytes)(247925)][(COMBINE_INPUT_RECORDS)(Combine input records)(26425)][(MAP_OUTPUT_RECORDS)(Map output records)(26425)]}" .
+Task TASKID="task_200904211745_0002_m_000014" TASK_TYPE="MAP" TASK_STATUS="SUCCESS" FINISH_TIME="1240336824060" COUNTERS="{(FileSystemCounters)(FileSystemCounters)[(HDFS_BYTES_READ)(HDFS_BYTES_READ)(53639)][(FILE_BYTES_WRITTEN)(FILE_BYTES_WRITTEN)(37170)]}{(org\.apache\.hadoop\.mapred\.Task$Counter)(Map-Reduce Framework)[(COMBINE_OUTPUT_RECORDS)(Combine output records)(5315)][(MAP_INPUT_RECORDS)(Map input records)(3601)][(SPILLED_RECORDS)(Spilled Records)(5315)][(MAP_OUTPUT_BYTES)(Map output bytes)(247925)][(COMBINE_INPUT_RECORDS)(Combine input records)(26425)][(MAP_OUTPUT_RECORDS)(Map output records)(26425)]}" .
+MapAttempt TASK_TYPE="MAP" TASKID="task_200904211745_0002_m_000008" TASK_ATTEMPT_ID="attempt_200904211745_0002_m_000008_0" START_TIME="1240336758220" TRACKER_NAME="tracker_cluster1846\.secondleveldomain\.com:localhost/127\.0\.0\.1:44127" HTTP_PORT="50060" .
+MapAttempt TASK_TYPE="MAP" TASKID="task_200904211745_0002_m_000008" TASK_ATTEMPT_ID="attempt_200904211745_0002_m_000008_0" TASK_STATUS="SUCCESS" FINISH_TIME="1240336762903" HOSTNAME="/194\.6\.129\.128/cluster1846\.secondleveldomain\.com" STATE_STRING="" COUNTERS="{(FileSystemCounters)(FileSystemCounters)[(HDFS_BYTES_READ)(HDFS_BYTES_READ)(148286)][(FILE_BYTES_WRITTEN)(FILE_BYTES_WRITTEN)(37170)]}{(org\.apache\.hadoop\.mapred\.Task$Counter)(Map-Reduce Framework)[(COMBINE_OUTPUT_RECORDS)(Combine output records)(5315)][(MAP_INPUT_RECORDS)(Map input records)(3601)][(SPILLED_RECORDS)(Spilled Records)(5315)][(MAP_OUTPUT_BYTES)(Map output bytes)(247925)][(COMBINE_INPUT_RECORDS)(Combine input records)(26425)][(MAP_OUTPUT_RECORDS)(Map output records)(26425)]}" .
+Task TASKID="task_200904211745_0002_m_000008" TASK_TYPE="MAP" TASK_STATUS="SUCCESS" FINISH_TIME="1240336824556" COUNTERS="{(FileSystemCounters)(FileSystemCounters)[(HDFS_BYTES_READ)(HDFS_BYTES_READ)(148286)][(FILE_BYTES_WRITTEN)(FILE_BYTES_WRITTEN)(37170)]}{(org\.apache\.hadoop\.mapred\.Task$Counter)(Map-Reduce Framework)[(COMBINE_OUTPUT_RECORDS)(Combine output records)(5315)][(MAP_INPUT_RECORDS)(Map input records)(3601)][(SPILLED_RECORDS)(Spilled Records)(5315)][(MAP_OUTPUT_BYTES)(Map output bytes)(247925)][(COMBINE_INPUT_RECORDS)(Combine input records)(26425)][(MAP_OUTPUT_RECORDS)(Map output records)(26425)]}" .
+MapAttempt TASK_TYPE="MAP" TASKID="task_200904211745_0002_m_000002" TASK_ATTEMPT_ID="attempt_200904211745_0002_m_000002_1" START_TIME="1240336821839" TRACKER_NAME="tracker_cluster1586\.secondleveldomain\.com:localhost/127\.0\.0\.1:38422" HTTP_PORT="50060" .
+MapAttempt TASK_TYPE="MAP" TASKID="task_200904211745_0002_m_000002" TASK_ATTEMPT_ID="attempt_200904211745_0002_m_000002_1" TASK_STATUS="SUCCESS" FINISH_TIME="1240336824652" HOSTNAME="/194\.6\.128\.0/cluster1586\.secondleveldomain\.com" STATE_STRING="" COUNTERS="{(FileSystemCounters)(FileSystemCounters)[(HDFS_BYTES_READ)(HDFS_BYTES_READ)(148286)][(FILE_BYTES_WRITTEN)(FILE_BYTES_WRITTEN)(37170)]}{(org\.apache\.hadoop\.mapred\.Task$Counter)(Map-Reduce Framework)[(COMBINE_OUTPUT_RECORDS)(Combine output records)(5315)][(MAP_INPUT_RECORDS)(Map input records)(3601)][(SPILLED_RECORDS)(Spilled Records)(5315)][(MAP_OUTPUT_BYTES)(Map output bytes)(247925)][(COMBINE_INPUT_RECORDS)(Combine input records)(26425)][(MAP_OUTPUT_RECORDS)(Map output records)(26425)]}" .
+Task TASKID="task_200904211745_0002_m_000002" TASK_TYPE="MAP" TASK_STATUS="SUCCESS" FINISH_TIME="1240336842768" COUNTERS="{(FileSystemCounters)(FileSystemCounters)[(HDFS_BYTES_READ)(HDFS_BYTES_READ)(148286)][(FILE_BYTES_WRITTEN)(FILE_BYTES_WRITTEN)(37170)]}{(org\.apache\.hadoop\.mapred\.Task$Counter)(Map-Reduce Framework)[(COMBINE_OUTPUT_RECORDS)(Combine output records)(5315)][(MAP_INPUT_RECORDS)(Map input records)(3601)][(SPILLED_RECORDS)(Spilled Records)(5315)][(MAP_OUTPUT_BYTES)(Map output bytes)(247925)][(COMBINE_INPUT_RECORDS)(Combine input records)(26425)][(MAP_OUTPUT_RECORDS)(Map output records)(26425)]}" .
+ReduceAttempt TASK_TYPE="REDUCE" TASKID="task_200904211745_0002_r_000000" TASK_ATTEMPT_ID="attempt_200904211745_0002_r_000000_0" START_TIME="1240336786769" TRACKER_NAME="tracker_cluster1771\.secondleveldomain\.com:localhost/127\.0\.0\.1:49430" HTTP_PORT="50060" .
+ReduceAttempt TASK_TYPE="REDUCE" TASKID="task_200904211745_0002_r_000000" TASK_ATTEMPT_ID="attempt_200904211745_0002_r_000000_0" TASK_STATUS="SUCCESS" SHUFFLE_FINISHED="1240336859759" SORT_FINISHED="1240336860092" FINISH_TIME="1240336870553" HOSTNAME="/192\.30\.63\.0/cluster1771\.secondleveldomain\.com" STATE_STRING="reduce > reduce" COUNTERS="{(FileSystemCounters)(FileSystemCounters)[(FILE_BYTES_READ)(FILE_BYTES_READ)(71200)][(FILE_BYTES_WRITTEN)(FILE_BYTES_WRITTEN)(71200)][(HDFS_BYTES_WRITTEN)(HDFS_BYTES_WRITTEN)(56630)]}{(org\.apache\.hadoop\.mapred\.Task$Counter)(Map-Reduce Framework)[(REDUCE_INPUT_GROUPS)(Reduce input groups)(0)][(COMBINE_OUTPUT_RECORDS)(Combine output records)(0)][(REDUCE_SHUFFLE_BYTES)(Reduce shuffle bytes)(705622)][(REDUCE_OUTPUT_RECORDS)(Reduce output records)(0)][(SPILLED_RECORDS)(Spilled Records)(106300)][(COMBINE_INPUT_RECORDS)(Combine input records)(0)][(REDUCE_INPUT_RECORDS)(Reduce input records)(106300)]}" .
+Task TASKID="task_200904211745_0002_r_000000" TASK_TYPE="REDUCE" TASK_STATUS="SUCCESS" FINISH_TIME="1240336873648" COUNTERS="{(FileSystemCounters)(FileSystemCounters)[(FILE_BYTES_READ)(FILE_BYTES_READ)(71200)][(FILE_BYTES_WRITTEN)(FILE_BYTES_WRITTEN)(71200)][(HDFS_BYTES_WRITTEN)(HDFS_BYTES_WRITTEN)(56630)]}{(org\.apache\.hadoop\.mapred\.Task$Counter)(Map-Reduce Framework)[(REDUCE_INPUT_GROUPS)(Reduce input groups)(0)][(COMBINE_OUTPUT_RECORDS)(Combine output records)(0)][(REDUCE_SHUFFLE_BYTES)(Reduce shuffle bytes)(705622)][(REDUCE_OUTPUT_RECORDS)(Reduce output records)(0)][(SPILLED_RECORDS)(Spilled Records)(106300)][(COMBINE_INPUT_RECORDS)(Combine input records)(0)][(REDUCE_INPUT_RECORDS)(Reduce input records)(106300)]}" .
+Task TASKID="task_200904211745_0002_m_000020" TASK_TYPE="CLEANUP" START_TIME="1240336873651" SPLITS="" .
+MapAttempt TASK_TYPE="CLEANUP" TASKID="task_200904211745_0002_m_000020" TASK_ATTEMPT_ID="attempt_200904211745_0002_m_000020_0" START_TIME="1240336885885" TRACKER_NAME="tracker_cluster1771\.secondleveldomain\.com:localhost/127\.0\.0\.1:49430" HTTP_PORT="50060" .
+MapAttempt TASK_TYPE="CLEANUP" TASKID="task_200904211745_0002_m_000020" TASK_ATTEMPT_ID="attempt_200904211745_0002_m_000020_0" TASK_STATUS="SUCCESS" FINISH_TIME="1240336887642" HOSTNAME="/192\.30\.63\.0/cluster1771\.secondleveldomain\.com" STATE_STRING="cleanup" COUNTERS="{(org\.apache\.hadoop\.mapred\.Task$Counter)(Map-Reduce Framework)[(SPILLED_RECORDS)(Spilled Records)(0)]}" .
+Task TASKID="task_200904211745_0002_m_000020" TASK_TYPE="CLEANUP" TASK_STATUS="SUCCESS" FINISH_TIME="1240336889658" COUNTERS="{(org\.apache\.hadoop\.mapred\.Task$Counter)(Map-Reduce Framework)[(SPILLED_RECORDS)(Spilled Records)(0)]}" .
+Job JOBID="job_200904211745_0002" FINISH_TIME="1240336889659" JOB_STATUS="SUCCESS" FINISHED_MAPS="20" FINISHED_REDUCES="1" FAILED_MAPS="1" FAILED_REDUCES="0" COUNTERS="{(org\.apache\.hadoop\.mapred\.JobInProgress$Counter)(Job Counters )[(TOTAL_LAUNCHED_REDUCES)(Launched reduce tasks)(1)][(RACK_LOCAL_MAPS)(Rack-local map tasks)(4)][(TOTAL_LAUNCHED_MAPS)(Launched map tasks)(21)]}{(FileSystemCounters)(FileSystemCounters)[(FILE_BYTES_READ)(FILE_BYTES_READ)(71200)][(HDFS_BYTES_READ)(HDFS_BYTES_READ)(2019250)][(FILE_BYTES_WRITTEN)(FILE_BYTES_WRITTEN)(814600)][(HDFS_BYTES_WRITTEN)(HDFS_BYTES_WRITTEN)(56630)]}{(org\.apache\.hadoop\.mapred\.Task$Counter)(Map-Reduce Framework)[(REDUCE_INPUT_GROUPS)(Reduce input groups)(0)][(COMBINE_OUTPUT_RECORDS)(Combine output records)(106300)][(MAP_INPUT_RECORDS)(Map input records)(72020)][(REDUCE_SHUFFLE_BYTES)(Reduce shuffle bytes)(705622)][(REDUCE_OUTPUT_RECORDS)(Reduce output records)(0)][(SPILLED_RECORDS)(Spilled Records)(212600)][(MAP_OUTPUT_BYTES)(Map output bytes)(4958500)][(MAP_OUTPUT_RECORDS)(Map output records)(528500)][(COMBINE_INPUT_RECORDS)(Combine input records)(528500)][(REDUCE_INPUT_RECORDS)(Reduce input records)(106300)]}" .
+
+!!FILE=cluster-jt1.red.ygrid.megatron.com_1240335959557_job_200904211745_0002_conf.xml!!
+<?xml version="1.0" encoding="UTF-8" standalone="no"?><configuration>
+<property><name>mapred.child.java.opts</name><value>-server -Xmx640m -Djava.net.preferIPv4Stack=true</value></property>
+</configuration>

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

@@ -0,0 +1,343 @@
+{
+  "name" : "<root>",
+  "children" : [ {
+    "name" : "194\\.6\\.133\\.192",
+    "children" : [ {
+      "name" : "cluster50218\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50232\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "192\\.30\\.117\\.128",
+    "children" : [ {
+      "name" : "cluster1218\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1236\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1200\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1227\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1223\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "194\\.6\\.129\\.128",
+    "children" : [ {
+      "name" : "cluster1872\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1876\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1854\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1869\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1846\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1867\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "194\\.6\\.134\\.64",
+    "children" : [ {
+      "name" : "cluster50286\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "192\\.30\\.116\\.192",
+    "children" : [ {
+      "name" : "cluster1143\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1157\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "194\\.6\\.128\\.0",
+    "children" : [ {
+      "name" : "cluster1586\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "192\\.30\\.127\\.192",
+    "children" : [ {
+      "name" : "cluster1553\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1551\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "194\\.6\\.133\\.64",
+    "children" : [ {
+      "name" : "cluster50130\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50141\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50152\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "192\\.30\\.117\\.192",
+    "children" : [ {
+      "name" : "cluster1259\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1245\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1260\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1261\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "192\\.30\\.116\\.64",
+    "children" : [ {
+      "name" : "cluster1064\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1059\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "192\\.30\\.126\\.0",
+    "children" : [ {
+      "name" : "cluster1288\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1294\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "192\\.30\\.116\\.128",
+    "children" : [ {
+      "name" : "cluster1080\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1097\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "194\\.6\\.135\\.64",
+    "children" : [ {
+      "name" : "cluster3071\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster3049\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "194\\.6\\.135\\.128",
+    "children" : [ {
+      "name" : "cluster3097\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster3098\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster3082\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster3112\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster3104\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "194\\.6\\.128\\.192",
+    "children" : [ {
+      "name" : "cluster50510\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50478\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50505\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50481\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "194\\.6\\.130\\.0",
+    "children" : [ {
+      "name" : "cluster1930\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1928\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1932\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "192\\.30\\.63\\.0",
+    "children" : [ {
+      "name" : "cluster1770\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1779\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1771\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "192\\.30\\.62\\.128",
+    "children" : [ {
+      "name" : "cluster1694\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1718\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "192\\.30\\.126\\.192",
+    "children" : [ {
+      "name" : "cluster1412\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1405\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "192\\.30\\.126\\.64",
+    "children" : [ {
+      "name" : "cluster1357\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "194\\.6\\.129\\.192",
+    "children" : [ {
+      "name" : "cluster1911\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1889\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "192\\.30\\.63\\.192",
+    "children" : [ {
+      "name" : "cluster1004\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1020\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1028\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "194\\.6\\.132\\.128",
+    "children" : [ {
+      "name" : "cluster50021\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "192\\.30\\.127\\.64",
+    "children" : [ {
+      "name" : "cluster1457\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1450\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1466\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "194\\.6\\.129\\.64",
+    "children" : [ {
+      "name" : "cluster1816\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1803\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "194\\.6\\.134\\.128",
+    "children" : [ {
+      "name" : "cluster50359\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "192\\.30\\.126\\.128",
+    "children" : [ {
+      "name" : "cluster1393\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1396\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1369\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "192\\.30\\.62\\.64",
+    "children" : [ {
+      "name" : "cluster1660\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1649\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "194\\.6\\.129\\.0",
+    "children" : [ {
+      "name" : "cluster50543\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "194\\.6\\.128\\.64",
+    "children" : [ {
+      "name" : "cluster1639\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1607\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "192\\.30\\.62\\.192",
+    "children" : [ {
+      "name" : "cluster50427\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50411\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "194\\.6\\.132\\.192",
+    "children" : [ {
+      "name" : "cluster50053\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50057\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50050\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "192\\.30\\.63\\.64",
+    "children" : [ {
+      "name" : "cluster1733\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  } ]
+}

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

@@ -0,0 +1,1407 @@
+{
+  "priority" : "NORMAL",
+  "user" : "hadoopqa",
+  "jobName" : null,
+  "jobID" : "job_200904211745_0002",
+  "mapTasks" : [ {
+    "startTime" : 1240336753705,
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.126\\.128", "cluster1396\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.126\\.128/cluster1396\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336647215,
+      "finishTime" : 1240336651127,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0002_m_000010_0",
+      "hdfsBytesRead" : 53639,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 37170,
+      "mapInputRecords" : 3601,
+      "mapOutputBytes" : 247925,
+      "mapOutputRecords" : 26425,
+      "combineInputRecords" : 26425,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 5315,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.126\\.128", "cluster1369\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.129\\.128", "cluster1854\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.129\\.128", "cluster1872\\.secondleveldomain\\.com" ]
+    } ],
+    "finishTime" : -1,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0002_m_000010",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : null,
+    "taskType" : "MAP"
+  }, {
+    "startTime" : 1240336753750,
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.127\\.192", "cluster1553\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.127\\.192/cluster1553\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336752204,
+      "finishTime" : 1240336755959,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0002_m_000017_0",
+      "hdfsBytesRead" : 53639,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 37170,
+      "mapInputRecords" : 3601,
+      "mapOutputBytes" : 247925,
+      "mapOutputRecords" : 26425,
+      "combineInputRecords" : 26425,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 5315,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "194\\.6\\.129\\.64", "cluster1803\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.130\\.0", "cluster1930\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.130\\.0", "cluster1932\\.secondleveldomain\\.com" ]
+    } ],
+    "finishTime" : 1240336818110,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0002_m_000017",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "startTime" : 1240336753796,
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.63\\.0", "cluster1771\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.63\\.0/cluster1771\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336766565,
+      "finishTime" : 1240336770163,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0002_m_000019_0",
+      "hdfsBytesRead" : 53639,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 37170,
+      "mapInputRecords" : 3601,
+      "mapOutputBytes" : 247925,
+      "mapOutputRecords" : 26425,
+      "combineInputRecords" : 26425,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 5315,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "194\\.6\\.128\\.192", "cluster50481\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.128\\.192", "cluster50505\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.134\\.128", "cluster50359\\.secondleveldomain\\.com" ]
+    } ],
+    "finishTime" : 1240336818106,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0002_m_000019",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "startTime" : 1240336753840,
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.63\\.0", "cluster1779\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.63\\.0/cluster1779\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336765887,
+      "finishTime" : 1240336785565,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0002_m_000005_0",
+      "hdfsBytesRead" : 148286,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 37170,
+      "mapInputRecords" : 3601,
+      "mapOutputBytes" : 247925,
+      "mapOutputRecords" : 26425,
+      "combineInputRecords" : 26425,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 5315,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.117\\.192", "cluster1245\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.117\\.192", "cluster1261\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.132\\.128", "cluster50021\\.secondleveldomain\\.com" ]
+    } ],
+    "finishTime" : 1240336818114,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0002_m_000005",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "startTime" : 1240336753888,
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.117\\.128", "cluster1218\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.117\\.128/cluster1218\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336755247,
+      "finishTime" : 1240336763432,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0002_m_000000_0",
+      "hdfsBytesRead" : 148286,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 37170,
+      "mapInputRecords" : 3601,
+      "mapOutputBytes" : 247925,
+      "mapOutputRecords" : 26425,
+      "combineInputRecords" : 26425,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 5315,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.117\\.128", "cluster1236\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.129\\.192", "cluster1889\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.129\\.192", "cluster1911\\.secondleveldomain\\.com" ]
+    } ],
+    "finishTime" : 1240336818113,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0002_m_000000",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "startTime" : 1240336753938,
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.128\\.0", "cluster1586\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.128\\.0/cluster1586\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336758229,
+      "finishTime" : 1240336761612,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0002_m_000009_0",
+      "hdfsBytesRead" : 148286,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 37170,
+      "mapInputRecords" : 3601,
+      "mapOutputBytes" : 247925,
+      "mapOutputRecords" : 26425,
+      "combineInputRecords" : 26425,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 5315,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.117\\.128", "cluster1227\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.117\\.192", "cluster1259\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.117\\.192", "cluster1260\\.secondleveldomain\\.com" ]
+    } ],
+    "finishTime" : 1240336818116,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0002_m_000009",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "startTime" : 1240336753987,
+    "attempts" : [ {
+      "location" : null,
+      "hostName" : "cluster1670\\.secondleveldomain\\.com",
+      "result" : "FAILED",
+      "startTime" : 1240336754665,
+      "finishTime" : 1240336777673,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0002_m_000002_0",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : -1,
+      "mapInputBytes" : -1
+    }, {
+      "location" : {
+        "layers" : [ "194\\.6\\.128\\.0", "cluster1586\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.128\\.0/cluster1586\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336821839,
+      "finishTime" : 1240336824652,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0002_m_000002_1",
+      "hdfsBytesRead" : 148286,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 37170,
+      "mapInputRecords" : 3601,
+      "mapOutputBytes" : 247925,
+      "mapOutputRecords" : 26425,
+      "combineInputRecords" : 26425,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 5315,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.126\\.64", "cluster1357\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.127\\.64", "cluster1450\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.127\\.64", "cluster1457\\.secondleveldomain\\.com" ]
+    } ],
+    "finishTime" : 1240336842768,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0002_m_000002",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "startTime" : 1240336754030,
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.129\\.128", "cluster1869\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.129\\.128/cluster1869\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336757380,
+      "finishTime" : 1240336763176,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0002_m_000004_0",
+      "hdfsBytesRead" : 148286,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 37170,
+      "mapInputRecords" : 3601,
+      "mapOutputBytes" : 247925,
+      "mapOutputRecords" : 26425,
+      "combineInputRecords" : 26425,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 5315,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.126\\.0", "cluster1294\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.126\\.0", "cluster1288\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.129\\.128", "cluster1876\\.secondleveldomain\\.com" ]
+    } ],
+    "finishTime" : 1240336818118,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0002_m_000004",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "startTime" : 1240336754077,
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.63\\.0", "cluster1770\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.63\\.0/cluster1770\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336763994,
+      "finishTime" : 1240336767750,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0002_m_000007_0",
+      "hdfsBytesRead" : 148286,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 37170,
+      "mapInputRecords" : 3601,
+      "mapOutputBytes" : 247925,
+      "mapOutputRecords" : 26425,
+      "combineInputRecords" : 26425,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 5315,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.127\\.64", "cluster1466\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.133\\.192", "cluster50218\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.133\\.192", "cluster50232\\.secondleveldomain\\.com" ]
+    } ],
+    "finishTime" : 1240336818119,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0002_m_000007",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "startTime" : 1240336754124,
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.129\\.64", "cluster1816\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.129\\.64/cluster1816\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336758341,
+      "finishTime" : 1240336762025,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0002_m_000013_0",
+      "hdfsBytesRead" : 53639,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 37170,
+      "mapInputRecords" : 3601,
+      "mapOutputBytes" : 247925,
+      "mapOutputRecords" : 26425,
+      "combineInputRecords" : 26425,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 5315,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "194\\.6\\.134\\.64", "cluster50286\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.135\\.64", "cluster3071\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.135\\.64", "cluster3049\\.secondleveldomain\\.com" ]
+    } ],
+    "finishTime" : 1240336818120,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0002_m_000013",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "startTime" : 1240336754176,
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.135\\.128", "cluster3097\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.135\\.128/cluster3097\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336768390,
+      "finishTime" : 1240336771627,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0002_m_000012_0",
+      "hdfsBytesRead" : 53639,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 37170,
+      "mapInputRecords" : 3601,
+      "mapOutputBytes" : 247925,
+      "mapOutputRecords" : 26425,
+      "combineInputRecords" : 26425,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 5315,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "194\\.6\\.132\\.192", "cluster50057\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.135\\.128", "cluster3112\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.135\\.128", "cluster3082\\.secondleveldomain\\.com" ]
+    } ],
+    "finishTime" : 1240336774482,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0002_m_000012",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "startTime" : 1240336754223,
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.62\\.64", "cluster1660\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.62\\.64/cluster1660\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336754801,
+      "finishTime" : 1240336758231,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0002_m_000001_0",
+      "hdfsBytesRead" : 148286,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 37170,
+      "mapInputRecords" : 3601,
+      "mapOutputBytes" : 247925,
+      "mapOutputRecords" : 26425,
+      "combineInputRecords" : 26425,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 5315,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.116\\.128", "cluster1080\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.116\\.128", "cluster1097\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.129\\.0", "cluster50543\\.secondleveldomain\\.com" ]
+    } ],
+    "finishTime" : 1240336774476,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0002_m_000001",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "startTime" : 1240336754270,
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.135\\.128", "cluster3104\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.135\\.128/cluster3104\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336762622,
+      "finishTime" : 1240336765834,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0002_m_000015_0",
+      "hdfsBytesRead" : 53639,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 37170,
+      "mapInputRecords" : 3601,
+      "mapOutputBytes" : 247925,
+      "mapOutputRecords" : 26425,
+      "combineInputRecords" : 26425,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 5315,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.126\\.192", "cluster1412\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.62\\.192", "cluster50427\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.62\\.192", "cluster50411\\.secondleveldomain\\.com" ]
+    } ],
+    "finishTime" : 1240336774468,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0002_m_000015",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "startTime" : 1240336754319,
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.129\\.128", "cluster1867\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.129\\.128/cluster1867\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336759528,
+      "finishTime" : 1240336764456,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0002_m_000016_0",
+      "hdfsBytesRead" : 53639,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 37170,
+      "mapInputRecords" : 3601,
+      "mapOutputBytes" : 247925,
+      "mapOutputRecords" : 26425,
+      "combineInputRecords" : 26425,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 5315,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.126\\.128", "cluster1393\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.133\\.64", "cluster50130\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.133\\.64", "cluster50141\\.secondleveldomain\\.com" ]
+    } ],
+    "finishTime" : 1240336817903,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0002_m_000016",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "startTime" : 1240336754366,
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.62\\.64", "cluster1649\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.62\\.64/cluster1649\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336759909,
+      "finishTime" : 1240336763727,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0002_m_000018_0",
+      "hdfsBytesRead" : 53639,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 37170,
+      "mapInputRecords" : 3601,
+      "mapOutputBytes" : 247925,
+      "mapOutputRecords" : 26425,
+      "combineInputRecords" : 26425,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 5315,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.117\\.128", "cluster1223\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.117\\.128", "cluster1200\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.133\\.64", "cluster50152\\.secondleveldomain\\.com" ]
+    } ],
+    "finishTime" : 1240336818420,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0002_m_000018",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "startTime" : 1240336754409,
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.127\\.192", "cluster1551\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.127\\.192/cluster1551\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336755433,
+      "finishTime" : 1240336758966,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0002_m_000003_0",
+      "hdfsBytesRead" : 148286,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 37170,
+      "mapInputRecords" : 3601,
+      "mapOutputBytes" : 247925,
+      "mapOutputRecords" : 26425,
+      "combineInputRecords" : 26425,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 5315,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.63\\.64", "cluster1733\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.128\\.64", "cluster1607\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.128\\.64", "cluster1639\\.secondleveldomain\\.com" ]
+    } ],
+    "finishTime" : 1240336816560,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0002_m_000003",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "startTime" : 1240336754452,
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.126\\.192", "cluster1405\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.126\\.192/cluster1405\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336747936,
+      "finishTime" : 1240336751464,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0002_m_000006_0",
+      "hdfsBytesRead" : 148286,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 37170,
+      "mapInputRecords" : 3601,
+      "mapOutputBytes" : 247925,
+      "mapOutputRecords" : 26425,
+      "combineInputRecords" : 26425,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 5315,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.116\\.64", "cluster1064\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.128\\.192", "cluster50510\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.128\\.192", "cluster50478\\.secondleveldomain\\.com" ]
+    } ],
+    "finishTime" : 1240336817330,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0002_m_000006",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "startTime" : 1240336754500,
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.130\\.0", "cluster1928\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.130\\.0/cluster1928\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336756161,
+      "finishTime" : 1240336760016,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0002_m_000014_0",
+      "hdfsBytesRead" : 53639,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 37170,
+      "mapInputRecords" : 3601,
+      "mapOutputBytes" : 247925,
+      "mapOutputRecords" : 26425,
+      "combineInputRecords" : 26425,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 5315,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.116\\.64", "cluster1059\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.132\\.192", "cluster50053\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.132\\.192", "cluster50050\\.secondleveldomain\\.com" ]
+    } ],
+    "finishTime" : 1240336824060,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0002_m_000014",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "startTime" : 1240336754548,
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.129\\.128", "cluster1846\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.129\\.128/cluster1846\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336758220,
+      "finishTime" : 1240336762903,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0002_m_000008_0",
+      "hdfsBytesRead" : 148286,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 37170,
+      "mapInputRecords" : 3601,
+      "mapOutputBytes" : 247925,
+      "mapOutputRecords" : 26425,
+      "combineInputRecords" : 26425,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 5315,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.116\\.192", "cluster1157\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.62\\.128", "cluster1718\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.62\\.128", "cluster1694\\.secondleveldomain\\.com" ]
+    } ],
+    "finishTime" : 1240336824556,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0002_m_000008",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "startTime" : 1240336754596,
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.135\\.128", "cluster3098\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.135\\.128/cluster3098\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336744261,
+      "finishTime" : 1240336747517,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0002_m_000011_0",
+      "hdfsBytesRead" : 53639,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 37170,
+      "mapInputRecords" : 3601,
+      "mapOutputBytes" : 247925,
+      "mapOutputRecords" : 26425,
+      "combineInputRecords" : 26425,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 5315,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.116\\.192", "cluster1143\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.63\\.192", "cluster1004\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.63\\.192", "cluster1020\\.secondleveldomain\\.com" ]
+    } ],
+    "finishTime" : 1240336776031,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0002_m_000011",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  } ],
+  "reduceTasks" : [ {
+    "startTime" : 1240336774548,
+    "attempts" : [ {
+      "location" : null,
+      "hostName" : "/192\\.30\\.63\\.0/cluster1771\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336786769,
+      "finishTime" : 1240336870553,
+      "shuffleFinished" : 1240336859759,
+      "sortFinished" : 1240336860092,
+      "attemptID" : "attempt_200904211745_0002_r_000000_0",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : 56630,
+      "fileBytesRead" : 71200,
+      "fileBytesWritten" : 71200,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : 0,
+      "reduceInputRecords" : 106300,
+      "reduceShuffleBytes" : 705622,
+      "reduceOutputRecords" : 0,
+      "spilledRecords" : 106300,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ ],
+    "finishTime" : 1240336873648,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0002_r_000000",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "REDUCE"
+  } ],
+  "otherTasks" : [ {
+    "startTime" : 1240336739508,
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.63\\.192", "cluster1028\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.63\\.192/cluster1028\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336739565,
+      "finishTime" : 1240336742217,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0002_m_000021_0",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 0,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ ],
+    "finishTime" : 1240336752812,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0002_m_000021",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "SETUP"
+  }, {
+    "startTime" : 1240336873651,
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.63\\.0", "cluster1771\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.63\\.0/cluster1771\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336885885,
+      "finishTime" : 1240336887642,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0002_m_000020_0",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 0,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ ],
+    "finishTime" : 1240336889658,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0002_m_000020",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "CLEANUP"
+  } ],
+  "finishTime" : 1240336889659,
+  "computonsPerMapInputByte" : -1,
+  "computonsPerMapOutputByte" : -1,
+  "computonsPerReduceInputByte" : -1,
+  "computonsPerReduceOutputByte" : -1,
+  "submitTime" : 1240335962848,
+  "launchTime" : 1240335964437,
+  "heapMegabytes" : 640,
+  "totalMaps" : 20,
+  "totalReduces" : 1,
+  "outcome" : "SUCCESS",
+  "jobtype" : "JAVA",
+  "directDependantJobs" : [ ],
+  "successfulMapAttemptCDFs" : [ {
+    "maximum" : 9223372036854775807,
+    "minimum" : -9223372036854775808,
+    "rankings" : [ ],
+    "numberValues" : 0
+  }, {
+    "maximum" : 8185,
+    "minimum" : 3237,
+    "rankings" : [ {
+      "relativeRanking" : 0.05,
+      "datum" : 3237
+    }, {
+      "relativeRanking" : 0.1,
+      "datum" : 3237
+    }, {
+      "relativeRanking" : 0.15,
+      "datum" : 3237
+    }, {
+      "relativeRanking" : 0.2,
+      "datum" : 3237
+    }, {
+      "relativeRanking" : 0.25,
+      "datum" : 3237
+    }, {
+      "relativeRanking" : 0.3,
+      "datum" : 3237
+    }, {
+      "relativeRanking" : 0.35,
+      "datum" : 3237
+    }, {
+      "relativeRanking" : 0.4,
+      "datum" : 3237
+    }, {
+      "relativeRanking" : 0.45,
+      "datum" : 3237
+    }, {
+      "relativeRanking" : 0.5,
+      "datum" : 3912
+    }, {
+      "relativeRanking" : 0.55,
+      "datum" : 3912
+    }, {
+      "relativeRanking" : 0.6,
+      "datum" : 3912
+    }, {
+      "relativeRanking" : 0.65,
+      "datum" : 3912
+    }, {
+      "relativeRanking" : 0.7,
+      "datum" : 3912
+    }, {
+      "relativeRanking" : 0.75,
+      "datum" : 5796
+    }, {
+      "relativeRanking" : 0.8,
+      "datum" : 5796
+    }, {
+      "relativeRanking" : 0.85,
+      "datum" : 5796
+    }, {
+      "relativeRanking" : 0.9,
+      "datum" : 5796
+    }, {
+      "relativeRanking" : 0.95,
+      "datum" : 5796
+    } ],
+    "numberValues" : 4
+  }, {
+    "maximum" : 19678,
+    "minimum" : 2813,
+    "rankings" : [ {
+      "relativeRanking" : 0.05,
+      "datum" : 2813
+    }, {
+      "relativeRanking" : 0.1,
+      "datum" : 2813
+    }, {
+      "relativeRanking" : 0.15,
+      "datum" : 3212
+    }, {
+      "relativeRanking" : 0.2,
+      "datum" : 3256
+    }, {
+      "relativeRanking" : 0.25,
+      "datum" : 3383
+    }, {
+      "relativeRanking" : 0.3,
+      "datum" : 3383
+    }, {
+      "relativeRanking" : 0.35,
+      "datum" : 3430
+    }, {
+      "relativeRanking" : 0.4,
+      "datum" : 3528
+    }, {
+      "relativeRanking" : 0.45,
+      "datum" : 3533
+    }, {
+      "relativeRanking" : 0.5,
+      "datum" : 3598
+    }, {
+      "relativeRanking" : 0.55,
+      "datum" : 3598
+    }, {
+      "relativeRanking" : 0.6,
+      "datum" : 3684
+    }, {
+      "relativeRanking" : 0.65,
+      "datum" : 3755
+    }, {
+      "relativeRanking" : 0.7,
+      "datum" : 3756
+    }, {
+      "relativeRanking" : 0.75,
+      "datum" : 3818
+    }, {
+      "relativeRanking" : 0.8,
+      "datum" : 3818
+    }, {
+      "relativeRanking" : 0.85,
+      "datum" : 3855
+    }, {
+      "relativeRanking" : 0.9,
+      "datum" : 4683
+    }, {
+      "relativeRanking" : 0.95,
+      "datum" : 4928
+    } ],
+    "numberValues" : 16
+  }, {
+    "maximum" : 2652,
+    "minimum" : 1757,
+    "rankings" : [ {
+      "relativeRanking" : 0.05,
+      "datum" : 1757
+    }, {
+      "relativeRanking" : 0.1,
+      "datum" : 1757
+    }, {
+      "relativeRanking" : 0.15,
+      "datum" : 1757
+    }, {
+      "relativeRanking" : 0.2,
+      "datum" : 1757
+    }, {
+      "relativeRanking" : 0.25,
+      "datum" : 1757
+    }, {
+      "relativeRanking" : 0.3,
+      "datum" : 1757
+    }, {
+      "relativeRanking" : 0.35,
+      "datum" : 1757
+    }, {
+      "relativeRanking" : 0.4,
+      "datum" : 1757
+    }, {
+      "relativeRanking" : 0.45,
+      "datum" : 1757
+    }, {
+      "relativeRanking" : 0.5,
+      "datum" : 1757
+    }, {
+      "relativeRanking" : 0.55,
+      "datum" : 1757
+    }, {
+      "relativeRanking" : 0.6,
+      "datum" : 1757
+    }, {
+      "relativeRanking" : 0.65,
+      "datum" : 1757
+    }, {
+      "relativeRanking" : 0.7,
+      "datum" : 1757
+    }, {
+      "relativeRanking" : 0.75,
+      "datum" : 1757
+    }, {
+      "relativeRanking" : 0.8,
+      "datum" : 1757
+    }, {
+      "relativeRanking" : 0.85,
+      "datum" : 1757
+    }, {
+      "relativeRanking" : 0.9,
+      "datum" : 1757
+    }, {
+      "relativeRanking" : 0.95,
+      "datum" : 1757
+    } ],
+    "numberValues" : 2
+  } ],
+  "failedMapAttemptCDFs" : [ {
+    "maximum" : 9223372036854775807,
+    "minimum" : -9223372036854775808,
+    "rankings" : [ ],
+    "numberValues" : 0
+  }, {
+    "maximum" : 9223372036854775807,
+    "minimum" : -9223372036854775808,
+    "rankings" : [ ],
+    "numberValues" : 0
+  }, {
+    "maximum" : 9223372036854775807,
+    "minimum" : -9223372036854775808,
+    "rankings" : [ ],
+    "numberValues" : 0
+  }, {
+    "maximum" : 23008,
+    "minimum" : 23008,
+    "rankings" : [ {
+      "relativeRanking" : 0.05,
+      "datum" : 23008
+    }, {
+      "relativeRanking" : 0.1,
+      "datum" : 23008
+    }, {
+      "relativeRanking" : 0.15,
+      "datum" : 23008
+    }, {
+      "relativeRanking" : 0.2,
+      "datum" : 23008
+    }, {
+      "relativeRanking" : 0.25,
+      "datum" : 23008
+    }, {
+      "relativeRanking" : 0.3,
+      "datum" : 23008
+    }, {
+      "relativeRanking" : 0.35,
+      "datum" : 23008
+    }, {
+      "relativeRanking" : 0.4,
+      "datum" : 23008
+    }, {
+      "relativeRanking" : 0.45,
+      "datum" : 23008
+    }, {
+      "relativeRanking" : 0.5,
+      "datum" : 23008
+    }, {
+      "relativeRanking" : 0.55,
+      "datum" : 23008
+    }, {
+      "relativeRanking" : 0.6,
+      "datum" : 23008
+    }, {
+      "relativeRanking" : 0.65,
+      "datum" : 23008
+    }, {
+      "relativeRanking" : 0.7,
+      "datum" : 23008
+    }, {
+      "relativeRanking" : 0.75,
+      "datum" : 23008
+    }, {
+      "relativeRanking" : 0.8,
+      "datum" : 23008
+    }, {
+      "relativeRanking" : 0.85,
+      "datum" : 23008
+    }, {
+      "relativeRanking" : 0.9,
+      "datum" : 23008
+    }, {
+      "relativeRanking" : 0.95,
+      "datum" : 23008
+    } ],
+    "numberValues" : 1
+  } ],
+  "successfulReduceAttemptCDF" : {
+    "maximum" : 83784,
+    "minimum" : 83784,
+    "rankings" : [ {
+      "relativeRanking" : 0.05,
+      "datum" : 83784
+    }, {
+      "relativeRanking" : 0.1,
+      "datum" : 83784
+    }, {
+      "relativeRanking" : 0.15,
+      "datum" : 83784
+    }, {
+      "relativeRanking" : 0.2,
+      "datum" : 83784
+    }, {
+      "relativeRanking" : 0.25,
+      "datum" : 83784
+    }, {
+      "relativeRanking" : 0.3,
+      "datum" : 83784
+    }, {
+      "relativeRanking" : 0.35,
+      "datum" : 83784
+    }, {
+      "relativeRanking" : 0.4,
+      "datum" : 83784
+    }, {
+      "relativeRanking" : 0.45,
+      "datum" : 83784
+    }, {
+      "relativeRanking" : 0.5,
+      "datum" : 83784
+    }, {
+      "relativeRanking" : 0.55,
+      "datum" : 83784
+    }, {
+      "relativeRanking" : 0.6,
+      "datum" : 83784
+    }, {
+      "relativeRanking" : 0.65,
+      "datum" : 83784
+    }, {
+      "relativeRanking" : 0.7,
+      "datum" : 83784
+    }, {
+      "relativeRanking" : 0.75,
+      "datum" : 83784
+    }, {
+      "relativeRanking" : 0.8,
+      "datum" : 83784
+    }, {
+      "relativeRanking" : 0.85,
+      "datum" : 83784
+    }, {
+      "relativeRanking" : 0.9,
+      "datum" : 83784
+    }, {
+      "relativeRanking" : 0.95,
+      "datum" : 83784
+    } ],
+    "numberValues" : 1
+  },
+  "failedReduceAttemptCDF" : {
+    "maximum" : 9223372036854775807,
+    "minimum" : -9223372036854775808,
+    "rankings" : [ ],
+    "numberValues" : 0
+  },
+  "mapperTriesToSucceed" : [ 0.9565217391304348, 0.043478260869565216 ],
+  "failedMapperFraction" : 0.0,
+  "relativeTime" : 0,
+  "queue" : null,
+  "clusterMapMB" : -1,
+  "clusterReduceMB" : -1,
+  "jobMapMB" : -1,
+  "jobReduceMB" : -1
+}

+ 1693 - 0
src/test/tools/data/rumen/zombie/input-topology.json

@@ -0,0 +1,1693 @@
+{
+  "name" : "<root>",
+  "children" : [ {
+    "name" : "194\\.6\\.133\\.192",
+    "children" : [ {
+      "name" : "cluster50213\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50226\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50228\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50217\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50214\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50231\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50232\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50204\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50206\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50203\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50205\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50210\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50208\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50218\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50225\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "192\\.30\\.117\\.128",
+    "children" : [ {
+      "name" : "cluster1209\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1205\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1235\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1239\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1200\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1227\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1212\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1223\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1217\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1207\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1228\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1218\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1221\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1215\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1226\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1236\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "194\\.6\\.134\\.64",
+    "children" : [ {
+      "name" : "cluster50317\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50292\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50291\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50294\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50285\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50300\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50281\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50311\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50297\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50319\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50286\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50307\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50296\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50315\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50316\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "194\\.6\\.129\\.128",
+    "children" : [ {
+      "name" : "cluster1859\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1877\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1871\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1876\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1854\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1841\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1858\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1843\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1857\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1842\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1872\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1869\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1853\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1846\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1867\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "194\\.6\\.130\\.64",
+    "children" : [ {
+      "name" : "cluster1976\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1969\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1961\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1963\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1968\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1979\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1967\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1989\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1970\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1999\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "192\\.30\\.116\\.192",
+    "children" : [ {
+      "name" : "cluster1150\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1127\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1139\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1154\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1138\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1137\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1130\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1151\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1131\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1141\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1124\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1158\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1140\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1144\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1136\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1157\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1143\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "194\\.6\\.128\\.0",
+    "children" : [ {
+      "name" : "cluster1592\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1567\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1594\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1586\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1561\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1585\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1562\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1581\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1566\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1598\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1568\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1560\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1574\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1573\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1583\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1579\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "194\\.6\\.134\\.192",
+    "children" : [ {
+      "name" : "cluster50364\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50372\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50365\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50377\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50368\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50396\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50375\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50389\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50382\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "192\\.30\\.127\\.192",
+    "children" : [ {
+      "name" : "cluster1533\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1531\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1557\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1555\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1534\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1553\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1550\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1540\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1538\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1520\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1559\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1535\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1525\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1529\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1551\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "194\\.6\\.133\\.64",
+    "children" : [ {
+      "name" : "cluster50124\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50159\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50144\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50145\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50133\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50120\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50130\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50142\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50147\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50156\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50125\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50141\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50152\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "192\\.30\\.117\\.192",
+    "children" : [ {
+      "name" : "cluster1250\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1276\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1248\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1246\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1251\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1259\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1261\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1260\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1243\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1256\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1272\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1274\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1245\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1249\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "192\\.30\\.116\\.64",
+    "children" : [ {
+      "name" : "cluster1041\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1075\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1042\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1078\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1072\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1053\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1056\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1064\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1055\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1070\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1061\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1059\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1040\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "194\\.6\\.133\\.128",
+    "children" : [ {
+      "name" : "cluster50171\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50195\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50161\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50191\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50174\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50185\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50177\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50166\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50170\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50179\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "192\\.30\\.126\\.0",
+    "children" : [ {
+      "name" : "cluster1283\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1299\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1281\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1288\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1302\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1294\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1289\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1315\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1305\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1316\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "192\\.30\\.116\\.128",
+    "children" : [ {
+      "name" : "cluster1107\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1118\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1080\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1093\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1102\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1104\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1097\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1087\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1095\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1110\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "194\\.6\\.135\\.64",
+    "children" : [ {
+      "name" : "cluster3071\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster3079\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster3068\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster3057\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster3058\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster3070\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster3054\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster3077\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster3049\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster3063\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster3075\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster3065\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster3076\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster3061\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster3073\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster3055\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "194\\.6\\.128\\.128",
+    "children" : [ {
+      "name" : "cluster50468\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50476\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50440\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50473\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50477\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50460\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50475\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50459\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50447\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50464\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50441\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50444\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "194\\.6\\.135\\.128",
+    "children" : [ {
+      "name" : "cluster3097\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster3089\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster3111\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster3093\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster3099\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster3106\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster3108\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster3112\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster3085\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster3094\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster3103\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster3098\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster3082\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster3104\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster3114\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "194\\.6\\.128\\.192",
+    "children" : [ {
+      "name" : "cluster50485\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50493\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50510\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50494\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50484\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50481\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50490\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50501\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50478\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50491\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50505\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50488\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50509\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50513\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "194\\.6\\.133\\.0",
+    "children" : [ {
+      "name" : "cluster50085\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50117\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50113\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50101\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50108\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50090\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "192\\.30\\.63\\.0",
+    "children" : [ {
+      "name" : "cluster1789\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1777\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1785\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1770\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1793\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1779\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1788\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1776\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1773\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1798\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1762\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1772\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1778\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1782\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1774\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1781\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1760\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1796\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1775\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1768\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1786\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1771\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "194\\.6\\.130\\.0",
+    "children" : [ {
+      "name" : "cluster1959\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1957\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1931\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1920\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1938\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1925\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1932\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1927\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1933\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1930\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1928\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1924\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1953\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1936\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "192\\.30\\.62\\.128",
+    "children" : [ {
+      "name" : "cluster1717\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1708\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1707\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1690\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1714\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1683\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1703\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1702\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1694\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1700\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1711\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1713\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1718\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "192\\.30\\.126\\.192",
+    "children" : [ {
+      "name" : "cluster1418\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1429\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1420\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1412\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1400\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1415\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1437\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1405\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1427\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "192\\.30\\.126\\.64",
+    "children" : [ {
+      "name" : "cluster1334\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1332\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1346\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1350\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1328\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1333\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1321\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1358\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1357\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1356\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "194\\.6\\.129\\.192",
+    "children" : [ {
+      "name" : "cluster1914\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1883\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1896\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1911\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1913\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1915\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1903\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1906\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1900\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1891\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1889\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1907\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "192\\.30\\.63\\.192",
+    "children" : [ {
+      "name" : "cluster1006\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1035\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1018\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1026\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1020\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1021\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1027\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1031\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1036\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1032\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1029\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1004\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1011\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1008\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1025\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1002\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1030\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1019\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1017\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1028\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "192\\.30\\.127\\.64",
+    "children" : [ {
+      "name" : "cluster1445\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1470\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1449\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1462\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1450\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1454\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1466\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1465\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1474\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1444\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1448\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1463\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1457\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1447\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1455\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1442\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1479\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1467\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1446\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "194\\.6\\.132\\.128",
+    "children" : [ {
+      "name" : "cluster50034\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50011\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50023\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50025\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50021\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "194\\.6\\.129\\.64",
+    "children" : [ {
+      "name" : "cluster1800\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1809\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1816\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1819\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1813\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1806\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1803\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1835\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1822\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1807\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1823\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1832\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "194\\.6\\.134\\.128",
+    "children" : [ {
+      "name" : "cluster50359\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50326\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50348\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50325\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50342\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50352\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "192\\.30\\.126\\.128",
+    "children" : [ {
+      "name" : "cluster1383\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1378\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1393\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1395\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1396\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1373\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1388\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1379\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1370\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1368\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1371\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1377\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1369\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "192\\.30\\.62\\.64",
+    "children" : [ {
+      "name" : "cluster1643\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1660\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1652\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1672\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1654\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1648\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1657\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1655\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1641\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1669\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1662\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1649\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1666\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1678\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1650\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1679\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "192\\.30\\.127\\.128",
+    "children" : [ {
+      "name" : "cluster1482\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1517\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1491\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1498\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1490\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1504\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1515\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1480\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1518\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1493\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1503\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "194\\.6\\.129\\.0",
+    "children" : [ {
+      "name" : "cluster50520\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50539\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50530\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50526\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50543\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "192\\.30\\.62\\.192",
+    "children" : [ {
+      "name" : "cluster50407\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50409\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50423\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50427\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50429\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50416\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50420\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50418\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50411\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50425\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "194\\.6\\.134\\.0",
+    "children" : [ {
+      "name" : "cluster50275\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50254\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50272\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50274\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50245\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50276\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50243\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50252\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50263\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50279\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50273\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50261\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50260\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "194\\.6\\.128\\.64",
+    "children" : [ {
+      "name" : "cluster1639\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1615\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1628\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1635\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1611\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1607\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1629\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1623\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1633\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1610\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1632\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1614\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1636\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1600\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1626\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1602\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "194\\.6\\.132\\.192",
+    "children" : [ {
+      "name" : "cluster50047\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50055\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50051\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50059\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50050\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50076\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50077\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50046\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50053\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50057\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50072\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50044\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50043\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster50058\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "192\\.30\\.117\\.64",
+    "children" : [ {
+      "name" : "cluster1193\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1175\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1185\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1171\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1174\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1167\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1180\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  }, {
+    "name" : "192\\.30\\.63\\.64",
+    "children" : [ {
+      "name" : "cluster1755\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1757\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1725\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1727\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1736\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1722\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1752\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1759\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1758\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1732\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1743\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1731\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1733\\.secondleveldomain\\.com",
+      "children" : null
+    }, {
+      "name" : "cluster1751\\.secondleveldomain\\.com",
+      "children" : null
+    } ]
+  } ]
+}

+ 11364 - 0
src/test/tools/data/rumen/zombie/input-trace.json

@@ -0,0 +1,11364 @@
+{
+  "priority" : "NORMAL",
+  "jobID" : "job_200904211745_0001",
+  "mapTasks" : [ ],
+  "reduceTasks" : [ ],
+  "otherTasks" : [ {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.63\\.192", "cluster1020\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.63\\.192/cluster1020\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336740671,
+      "finishTime" : 1240336743094,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0001_m_000001_0",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 0,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ ],
+    "startTime" : 1240336739206,
+    "finishTime" : 1240336760537,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0001_m_000001",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "CLEANUP"
+  } ],
+  "finishTime" : 1240336760540,
+  "user" : "geek1",
+  "jobName" : null,
+  "computonsPerMapInputByte" : -1,
+  "computonsPerMapOutputByte" : -1,
+  "computonsPerReduceInputByte" : -1,
+  "computonsPerReduceOutputByte" : -1,
+  "submitTime" : 1240335960685,
+  "launchTime" : 1240335961050,
+  "heapMegabytes" : 1024,
+  "totalMaps" : 1,
+  "totalReduces" : 1,
+  "outcome" : "KILLED",
+  "jobtype" : "JAVA",
+  "directDependantJobs" : [ ],
+  "successfulMapAttemptCDFs" : [ {
+    "maximum" : 9223372036854775807,
+    "minimum" : -9223372036854775808,
+    "rankings" : [ ],
+    "numberValues" : 0
+  }, {
+    "maximum" : 9223372036854775807,
+    "minimum" : -9223372036854775808,
+    "rankings" : [ ],
+    "numberValues" : 0
+  }, {
+    "maximum" : 9223372036854775807,
+    "minimum" : -9223372036854775808,
+    "rankings" : [ ],
+    "numberValues" : 0
+  }, {
+    "maximum" : 2423,
+    "minimum" : 2423,
+    "rankings" : [ {
+      "relativeRanking" : 0.05,
+      "datum" : 2423
+    }, {
+      "relativeRanking" : 0.1,
+      "datum" : 2423
+    }, {
+      "relativeRanking" : 0.15,
+      "datum" : 2423
+    }, {
+      "relativeRanking" : 0.2,
+      "datum" : 2423
+    }, {
+      "relativeRanking" : 0.25,
+      "datum" : 2423
+    }, {
+      "relativeRanking" : 0.3,
+      "datum" : 2423
+    }, {
+      "relativeRanking" : 0.35,
+      "datum" : 2423
+    }, {
+      "relativeRanking" : 0.4,
+      "datum" : 2423
+    }, {
+      "relativeRanking" : 0.45,
+      "datum" : 2423
+    }, {
+      "relativeRanking" : 0.5,
+      "datum" : 2423
+    }, {
+      "relativeRanking" : 0.55,
+      "datum" : 2423
+    }, {
+      "relativeRanking" : 0.6,
+      "datum" : 2423
+    }, {
+      "relativeRanking" : 0.65,
+      "datum" : 2423
+    }, {
+      "relativeRanking" : 0.7,
+      "datum" : 2423
+    }, {
+      "relativeRanking" : 0.75,
+      "datum" : 2423
+    }, {
+      "relativeRanking" : 0.8,
+      "datum" : 2423
+    }, {
+      "relativeRanking" : 0.85,
+      "datum" : 2423
+    }, {
+      "relativeRanking" : 0.9,
+      "datum" : 2423
+    }, {
+      "relativeRanking" : 0.95,
+      "datum" : 2423
+    } ],
+    "numberValues" : 1
+  } ],
+  "failedMapAttemptCDFs" : [ {
+    "maximum" : 9223372036854775807,
+    "minimum" : -9223372036854775808,
+    "rankings" : [ ],
+    "numberValues" : 0
+  }, {
+    "maximum" : 9223372036854775807,
+    "minimum" : -9223372036854775808,
+    "rankings" : [ ],
+    "numberValues" : 0
+  }, {
+    "maximum" : 9223372036854775807,
+    "minimum" : -9223372036854775808,
+    "rankings" : [ ],
+    "numberValues" : 0
+  }, {
+    "maximum" : 9223372036854775807,
+    "minimum" : -9223372036854775808,
+    "rankings" : [ ],
+    "numberValues" : 0
+  } ],
+  "successfulReduceAttemptCDF" : {
+    "maximum" : 9223372036854775807,
+    "minimum" : -9223372036854775808,
+    "rankings" : [ ],
+    "numberValues" : 0
+  },
+  "failedReduceAttemptCDF" : {
+    "maximum" : 9223372036854775807,
+    "minimum" : -9223372036854775808,
+    "rankings" : [ ],
+    "numberValues" : 0
+  },
+  "mapperTriesToSucceed" : [ 1.0 ],
+  "failedMapperFraction" : 0.0,
+  "relativeTime" : 0,
+  "queue" : null,
+  "clusterMapMB" : -1,
+  "clusterReduceMB" : -1,
+  "jobMapMB" : -1,
+  "jobReduceMB" : -1
+}
+ {
+  "priority" : "NORMAL",
+  "jobID" : "job_200904211745_0002",
+  "mapTasks" : [ {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.126\\.128", "cluster1396\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.126\\.128/cluster1396\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336647215,
+      "finishTime" : 1240336651127,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0002_m_000010_0",
+      "hdfsBytesRead" : 53639,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 37170,
+      "mapInputRecords" : 3601,
+      "mapOutputBytes" : 247925,
+      "mapOutputRecords" : 26425,
+      "combineInputRecords" : 26425,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 5315,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.126\\.128", "cluster1369\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.129\\.128", "cluster1854\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.129\\.128", "cluster1872\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336753705,
+    "finishTime" : 1240336818108,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0002_m_000010",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.127\\.192", "cluster1553\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.127\\.192/cluster1553\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336752204,
+      "finishTime" : 1240336755959,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0002_m_000017_0",
+      "hdfsBytesRead" : 53639,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 37170,
+      "mapInputRecords" : 3601,
+      "mapOutputBytes" : 247925,
+      "mapOutputRecords" : 26425,
+      "combineInputRecords" : 26425,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 5315,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "194\\.6\\.129\\.64", "cluster1803\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.130\\.0", "cluster1930\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.130\\.0", "cluster1932\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336753750,
+    "finishTime" : 1240336818110,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0002_m_000017",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.63\\.0", "cluster1771\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.63\\.0/cluster1771\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336766565,
+      "finishTime" : 1240336770163,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0002_m_000019_0",
+      "hdfsBytesRead" : 53639,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 37170,
+      "mapInputRecords" : 3601,
+      "mapOutputBytes" : 247925,
+      "mapOutputRecords" : 26425,
+      "combineInputRecords" : 26425,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 5315,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "194\\.6\\.128\\.192", "cluster50481\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.128\\.192", "cluster50505\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.134\\.128", "cluster50359\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336753796,
+    "finishTime" : 1240336818106,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0002_m_000019",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.63\\.0", "cluster1779\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.63\\.0/cluster1779\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336765887,
+      "finishTime" : 1240336785565,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0002_m_000005_0",
+      "hdfsBytesRead" : 148286,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 37170,
+      "mapInputRecords" : 3601,
+      "mapOutputBytes" : 247925,
+      "mapOutputRecords" : 26425,
+      "combineInputRecords" : 26425,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 5315,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.117\\.192", "cluster1245\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.117\\.192", "cluster1261\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.132\\.128", "cluster50021\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336753840,
+    "finishTime" : 1240336818114,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0002_m_000005",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.117\\.128", "cluster1218\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.117\\.128/cluster1218\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336755247,
+      "finishTime" : 1240336763432,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0002_m_000000_0",
+      "hdfsBytesRead" : 148286,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 37170,
+      "mapInputRecords" : 3601,
+      "mapOutputBytes" : 247925,
+      "mapOutputRecords" : 26425,
+      "combineInputRecords" : 26425,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 5315,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.117\\.128", "cluster1236\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.129\\.192", "cluster1889\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.129\\.192", "cluster1911\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336753888,
+    "finishTime" : 1240336818113,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0002_m_000000",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.128\\.0", "cluster1586\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.128\\.0/cluster1586\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336758229,
+      "finishTime" : 1240336761612,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0002_m_000009_0",
+      "hdfsBytesRead" : 148286,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 37170,
+      "mapInputRecords" : 3601,
+      "mapOutputBytes" : 247925,
+      "mapOutputRecords" : 26425,
+      "combineInputRecords" : 26425,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 5315,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.117\\.128", "cluster1227\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.117\\.192", "cluster1259\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.117\\.192", "cluster1260\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336753938,
+    "finishTime" : 1240336818116,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0002_m_000009",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : null,
+      "hostName" : "cluster1670\\.secondleveldomain\\.com",
+      "result" : "FAILED",
+      "startTime" : 1240336754665,
+      "finishTime" : 1240336777673,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0002_m_000002_0",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : -1,
+      "mapInputBytes" : -1
+    }, {
+      "location" : {
+        "layers" : [ "194\\.6\\.128\\.0", "cluster1586\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.128\\.0/cluster1586\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336821839,
+      "finishTime" : 1240336824652,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0002_m_000002_1",
+      "hdfsBytesRead" : 148286,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 37170,
+      "mapInputRecords" : 3601,
+      "mapOutputBytes" : 247925,
+      "mapOutputRecords" : 26425,
+      "combineInputRecords" : 26425,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 5315,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.126\\.64", "cluster1357\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.127\\.64", "cluster1450\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.127\\.64", "cluster1457\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336753987,
+    "finishTime" : 1240336842768,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0002_m_000002",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.129\\.128", "cluster1869\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.129\\.128/cluster1869\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336757380,
+      "finishTime" : 1240336763176,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0002_m_000004_0",
+      "hdfsBytesRead" : 148286,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 37170,
+      "mapInputRecords" : 3601,
+      "mapOutputBytes" : 247925,
+      "mapOutputRecords" : 26425,
+      "combineInputRecords" : 26425,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 5315,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.126\\.0", "cluster1294\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.126\\.0", "cluster1288\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.129\\.128", "cluster1876\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336754030,
+    "finishTime" : 1240336818118,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0002_m_000004",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.63\\.0", "cluster1770\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.63\\.0/cluster1770\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336763994,
+      "finishTime" : 1240336767750,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0002_m_000007_0",
+      "hdfsBytesRead" : 148286,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 37170,
+      "mapInputRecords" : 3601,
+      "mapOutputBytes" : 247925,
+      "mapOutputRecords" : 26425,
+      "combineInputRecords" : 26425,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 5315,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.127\\.64", "cluster1466\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.133\\.192", "cluster50218\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.133\\.192", "cluster50232\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336754077,
+    "finishTime" : 1240336818119,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0002_m_000007",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.129\\.64", "cluster1816\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.129\\.64/cluster1816\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336758341,
+      "finishTime" : 1240336762025,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0002_m_000013_0",
+      "hdfsBytesRead" : 53639,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 37170,
+      "mapInputRecords" : 3601,
+      "mapOutputBytes" : 247925,
+      "mapOutputRecords" : 26425,
+      "combineInputRecords" : 26425,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 5315,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "194\\.6\\.134\\.64", "cluster50286\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.135\\.64", "cluster3071\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.135\\.64", "cluster3049\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336754124,
+    "finishTime" : 1240336818120,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0002_m_000013",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.135\\.128", "cluster3097\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.135\\.128/cluster3097\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336768390,
+      "finishTime" : 1240336771627,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0002_m_000012_0",
+      "hdfsBytesRead" : 53639,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 37170,
+      "mapInputRecords" : 3601,
+      "mapOutputBytes" : 247925,
+      "mapOutputRecords" : 26425,
+      "combineInputRecords" : 26425,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 5315,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "194\\.6\\.132\\.192", "cluster50057\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.135\\.128", "cluster3112\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.135\\.128", "cluster3082\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336754176,
+    "finishTime" : 1240336774482,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0002_m_000012",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.62\\.64", "cluster1660\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.62\\.64/cluster1660\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336754801,
+      "finishTime" : 1240336758231,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0002_m_000001_0",
+      "hdfsBytesRead" : 148286,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 37170,
+      "mapInputRecords" : 3601,
+      "mapOutputBytes" : 247925,
+      "mapOutputRecords" : 26425,
+      "combineInputRecords" : 26425,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 5315,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.116\\.128", "cluster1080\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.116\\.128", "cluster1097\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.129\\.0", "cluster50543\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336754223,
+    "finishTime" : 1240336774476,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0002_m_000001",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.135\\.128", "cluster3104\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.135\\.128/cluster3104\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336762622,
+      "finishTime" : 1240336765834,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0002_m_000015_0",
+      "hdfsBytesRead" : 53639,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 37170,
+      "mapInputRecords" : 3601,
+      "mapOutputBytes" : 247925,
+      "mapOutputRecords" : 26425,
+      "combineInputRecords" : 26425,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 5315,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.126\\.192", "cluster1412\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.62\\.192", "cluster50427\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.62\\.192", "cluster50411\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336754270,
+    "finishTime" : 1240336774468,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0002_m_000015",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.129\\.128", "cluster1867\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.129\\.128/cluster1867\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336759528,
+      "finishTime" : 1240336764456,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0002_m_000016_0",
+      "hdfsBytesRead" : 53639,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 37170,
+      "mapInputRecords" : 3601,
+      "mapOutputBytes" : 247925,
+      "mapOutputRecords" : 26425,
+      "combineInputRecords" : 26425,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 5315,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.126\\.128", "cluster1393\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.133\\.64", "cluster50130\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.133\\.64", "cluster50141\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336754319,
+    "finishTime" : 1240336817903,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0002_m_000016",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.62\\.64", "cluster1649\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.62\\.64/cluster1649\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336759909,
+      "finishTime" : 1240336763727,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0002_m_000018_0",
+      "hdfsBytesRead" : 53639,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 37170,
+      "mapInputRecords" : 3601,
+      "mapOutputBytes" : 247925,
+      "mapOutputRecords" : 26425,
+      "combineInputRecords" : 26425,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 5315,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.117\\.128", "cluster1223\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.117\\.128", "cluster1200\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.133\\.64", "cluster50152\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336754366,
+    "finishTime" : 1240336818420,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0002_m_000018",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.127\\.192", "cluster1551\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.127\\.192/cluster1551\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336755433,
+      "finishTime" : 1240336758966,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0002_m_000003_0",
+      "hdfsBytesRead" : 148286,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 37170,
+      "mapInputRecords" : 3601,
+      "mapOutputBytes" : 247925,
+      "mapOutputRecords" : 26425,
+      "combineInputRecords" : 26425,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 5315,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.63\\.64", "cluster1733\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.128\\.64", "cluster1607\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.128\\.64", "cluster1639\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336754409,
+    "finishTime" : 1240336816560,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0002_m_000003",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.126\\.192", "cluster1405\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.126\\.192/cluster1405\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336747936,
+      "finishTime" : 1240336751464,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0002_m_000006_0",
+      "hdfsBytesRead" : 148286,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 37170,
+      "mapInputRecords" : 3601,
+      "mapOutputBytes" : 247925,
+      "mapOutputRecords" : 26425,
+      "combineInputRecords" : 26425,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 5315,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.116\\.64", "cluster1064\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.128\\.192", "cluster50510\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.128\\.192", "cluster50478\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336754452,
+    "finishTime" : 1240336817330,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0002_m_000006",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.130\\.0", "cluster1928\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.130\\.0/cluster1928\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336756161,
+      "finishTime" : 1240336760016,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0002_m_000014_0",
+      "hdfsBytesRead" : 53639,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 37170,
+      "mapInputRecords" : 3601,
+      "mapOutputBytes" : 247925,
+      "mapOutputRecords" : 26425,
+      "combineInputRecords" : 26425,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 5315,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.116\\.64", "cluster1059\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.132\\.192", "cluster50053\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.132\\.192", "cluster50050\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336754500,
+    "finishTime" : 1240336824060,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0002_m_000014",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.129\\.128", "cluster1846\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.129\\.128/cluster1846\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336758220,
+      "finishTime" : 1240336762903,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0002_m_000008_0",
+      "hdfsBytesRead" : 148286,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 37170,
+      "mapInputRecords" : 3601,
+      "mapOutputBytes" : 247925,
+      "mapOutputRecords" : 26425,
+      "combineInputRecords" : 26425,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 5315,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.116\\.192", "cluster1157\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.62\\.128", "cluster1718\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.62\\.128", "cluster1694\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336754548,
+    "finishTime" : 1240336824556,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0002_m_000008",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.135\\.128", "cluster3098\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.135\\.128/cluster3098\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336744261,
+      "finishTime" : 1240336747517,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0002_m_000011_0",
+      "hdfsBytesRead" : 53639,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 37170,
+      "mapInputRecords" : 3601,
+      "mapOutputBytes" : 247925,
+      "mapOutputRecords" : 26425,
+      "combineInputRecords" : 26425,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 5315,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.116\\.192", "cluster1143\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.63\\.192", "cluster1004\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.63\\.192", "cluster1020\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336754596,
+    "finishTime" : 1240336776031,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0002_m_000011",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  } ],
+  "reduceTasks" : [ {
+    "attempts" : [ {
+      "location" : null,
+      "hostName" : "/192\\.30\\.63\\.0/cluster1771\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336786769,
+      "finishTime" : 1240336870553,
+      "shuffleFinished" : 1240336859759,
+      "sortFinished" : 1240336860092,
+      "attemptID" : "attempt_200904211745_0002_r_000000_0",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : 56630,
+      "fileBytesRead" : 71200,
+      "fileBytesWritten" : 71200,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : 0,
+      "reduceInputRecords" : 106300,
+      "reduceShuffleBytes" : 705622,
+      "reduceOutputRecords" : 0,
+      "spilledRecords" : 106300,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ ],
+    "startTime" : 1240336774548,
+    "finishTime" : 1240336873648,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0002_r_000000",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "REDUCE"
+  } ],
+  "otherTasks" : [ {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.63\\.192", "cluster1028\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.63\\.192/cluster1028\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336739565,
+      "finishTime" : 1240336742217,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0002_m_000021_0",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 0,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ ],
+    "startTime" : 1240336739508,
+    "finishTime" : 1240336752812,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0002_m_000021",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "SETUP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.63\\.0", "cluster1771\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.63\\.0/cluster1771\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336885885,
+      "finishTime" : 1240336887642,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0002_m_000020_0",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 0,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ ],
+    "startTime" : 1240336873651,
+    "finishTime" : 1240336889658,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0002_m_000020",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "CLEANUP"
+  } ],
+  "finishTime" : 1240336889659,
+  "user" : "hadoopqa",
+  "jobName" : null,
+  "computonsPerMapInputByte" : -1,
+  "computonsPerMapOutputByte" : -1,
+  "computonsPerReduceInputByte" : -1,
+  "computonsPerReduceOutputByte" : -1,
+  "submitTime" : 1240335962848,
+  "launchTime" : 1240335964437,
+  "heapMegabytes" : 640,
+  "totalMaps" : 20,
+  "totalReduces" : 1,
+  "outcome" : "SUCCESS",
+  "jobtype" : "JAVA",
+  "directDependantJobs" : [ ],
+  "successfulMapAttemptCDFs" : [ {
+    "maximum" : 9223372036854775807,
+    "minimum" : -9223372036854775808,
+    "rankings" : [ ],
+    "numberValues" : 0
+  }, {
+    "maximum" : 8185,
+    "minimum" : 3237,
+    "rankings" : [ {
+      "relativeRanking" : 0.05,
+      "datum" : 3237
+    }, {
+      "relativeRanking" : 0.1,
+      "datum" : 3237
+    }, {
+      "relativeRanking" : 0.15,
+      "datum" : 3237
+    }, {
+      "relativeRanking" : 0.2,
+      "datum" : 3237
+    }, {
+      "relativeRanking" : 0.25,
+      "datum" : 3237
+    }, {
+      "relativeRanking" : 0.3,
+      "datum" : 3237
+    }, {
+      "relativeRanking" : 0.35,
+      "datum" : 3237
+    }, {
+      "relativeRanking" : 0.4,
+      "datum" : 3237
+    }, {
+      "relativeRanking" : 0.45,
+      "datum" : 3237
+    }, {
+      "relativeRanking" : 0.5,
+      "datum" : 3912
+    }, {
+      "relativeRanking" : 0.55,
+      "datum" : 3912
+    }, {
+      "relativeRanking" : 0.6,
+      "datum" : 3912
+    }, {
+      "relativeRanking" : 0.65,
+      "datum" : 3912
+    }, {
+      "relativeRanking" : 0.7,
+      "datum" : 3912
+    }, {
+      "relativeRanking" : 0.75,
+      "datum" : 5796
+    }, {
+      "relativeRanking" : 0.8,
+      "datum" : 5796
+    }, {
+      "relativeRanking" : 0.85,
+      "datum" : 5796
+    }, {
+      "relativeRanking" : 0.9,
+      "datum" : 5796
+    }, {
+      "relativeRanking" : 0.95,
+      "datum" : 5796
+    } ],
+    "numberValues" : 4
+  }, {
+    "maximum" : 19678,
+    "minimum" : 2813,
+    "rankings" : [ {
+      "relativeRanking" : 0.05,
+      "datum" : 2813
+    }, {
+      "relativeRanking" : 0.1,
+      "datum" : 2813
+    }, {
+      "relativeRanking" : 0.15,
+      "datum" : 3212
+    }, {
+      "relativeRanking" : 0.2,
+      "datum" : 3256
+    }, {
+      "relativeRanking" : 0.25,
+      "datum" : 3383
+    }, {
+      "relativeRanking" : 0.3,
+      "datum" : 3383
+    }, {
+      "relativeRanking" : 0.35,
+      "datum" : 3430
+    }, {
+      "relativeRanking" : 0.4,
+      "datum" : 3528
+    }, {
+      "relativeRanking" : 0.45,
+      "datum" : 3533
+    }, {
+      "relativeRanking" : 0.5,
+      "datum" : 3598
+    }, {
+      "relativeRanking" : 0.55,
+      "datum" : 3598
+    }, {
+      "relativeRanking" : 0.6,
+      "datum" : 3684
+    }, {
+      "relativeRanking" : 0.65,
+      "datum" : 3755
+    }, {
+      "relativeRanking" : 0.7,
+      "datum" : 3756
+    }, {
+      "relativeRanking" : 0.75,
+      "datum" : 3818
+    }, {
+      "relativeRanking" : 0.8,
+      "datum" : 3818
+    }, {
+      "relativeRanking" : 0.85,
+      "datum" : 3855
+    }, {
+      "relativeRanking" : 0.9,
+      "datum" : 4683
+    }, {
+      "relativeRanking" : 0.95,
+      "datum" : 4928
+    } ],
+    "numberValues" : 16
+  }, {
+    "maximum" : 2652,
+    "minimum" : 1757,
+    "rankings" : [ {
+      "relativeRanking" : 0.05,
+      "datum" : 1757
+    }, {
+      "relativeRanking" : 0.1,
+      "datum" : 1757
+    }, {
+      "relativeRanking" : 0.15,
+      "datum" : 1757
+    }, {
+      "relativeRanking" : 0.2,
+      "datum" : 1757
+    }, {
+      "relativeRanking" : 0.25,
+      "datum" : 1757
+    }, {
+      "relativeRanking" : 0.3,
+      "datum" : 1757
+    }, {
+      "relativeRanking" : 0.35,
+      "datum" : 1757
+    }, {
+      "relativeRanking" : 0.4,
+      "datum" : 1757
+    }, {
+      "relativeRanking" : 0.45,
+      "datum" : 1757
+    }, {
+      "relativeRanking" : 0.5,
+      "datum" : 1757
+    }, {
+      "relativeRanking" : 0.55,
+      "datum" : 1757
+    }, {
+      "relativeRanking" : 0.6,
+      "datum" : 1757
+    }, {
+      "relativeRanking" : 0.65,
+      "datum" : 1757
+    }, {
+      "relativeRanking" : 0.7,
+      "datum" : 1757
+    }, {
+      "relativeRanking" : 0.75,
+      "datum" : 1757
+    }, {
+      "relativeRanking" : 0.8,
+      "datum" : 1757
+    }, {
+      "relativeRanking" : 0.85,
+      "datum" : 1757
+    }, {
+      "relativeRanking" : 0.9,
+      "datum" : 1757
+    }, {
+      "relativeRanking" : 0.95,
+      "datum" : 1757
+    } ],
+    "numberValues" : 2
+  } ],
+  "failedMapAttemptCDFs" : [ {
+    "maximum" : 9223372036854775807,
+    "minimum" : -9223372036854775808,
+    "rankings" : [ ],
+    "numberValues" : 0
+  }, {
+    "maximum" : 9223372036854775807,
+    "minimum" : -9223372036854775808,
+    "rankings" : [ ],
+    "numberValues" : 0
+  }, {
+    "maximum" : 9223372036854775807,
+    "minimum" : -9223372036854775808,
+    "rankings" : [ ],
+    "numberValues" : 0
+  }, {
+    "maximum" : 23008,
+    "minimum" : 23008,
+    "rankings" : [ {
+      "relativeRanking" : 0.05,
+      "datum" : 23008
+    }, {
+      "relativeRanking" : 0.1,
+      "datum" : 23008
+    }, {
+      "relativeRanking" : 0.15,
+      "datum" : 23008
+    }, {
+      "relativeRanking" : 0.2,
+      "datum" : 23008
+    }, {
+      "relativeRanking" : 0.25,
+      "datum" : 23008
+    }, {
+      "relativeRanking" : 0.3,
+      "datum" : 23008
+    }, {
+      "relativeRanking" : 0.35,
+      "datum" : 23008
+    }, {
+      "relativeRanking" : 0.4,
+      "datum" : 23008
+    }, {
+      "relativeRanking" : 0.45,
+      "datum" : 23008
+    }, {
+      "relativeRanking" : 0.5,
+      "datum" : 23008
+    }, {
+      "relativeRanking" : 0.55,
+      "datum" : 23008
+    }, {
+      "relativeRanking" : 0.6,
+      "datum" : 23008
+    }, {
+      "relativeRanking" : 0.65,
+      "datum" : 23008
+    }, {
+      "relativeRanking" : 0.7,
+      "datum" : 23008
+    }, {
+      "relativeRanking" : 0.75,
+      "datum" : 23008
+    }, {
+      "relativeRanking" : 0.8,
+      "datum" : 23008
+    }, {
+      "relativeRanking" : 0.85,
+      "datum" : 23008
+    }, {
+      "relativeRanking" : 0.9,
+      "datum" : 23008
+    }, {
+      "relativeRanking" : 0.95,
+      "datum" : 23008
+    } ],
+    "numberValues" : 1
+  } ],
+  "successfulReduceAttemptCDF" : {
+    "maximum" : 83784,
+    "minimum" : 83784,
+    "rankings" : [ {
+      "relativeRanking" : 0.05,
+      "datum" : 83784
+    }, {
+      "relativeRanking" : 0.1,
+      "datum" : 83784
+    }, {
+      "relativeRanking" : 0.15,
+      "datum" : 83784
+    }, {
+      "relativeRanking" : 0.2,
+      "datum" : 83784
+    }, {
+      "relativeRanking" : 0.25,
+      "datum" : 83784
+    }, {
+      "relativeRanking" : 0.3,
+      "datum" : 83784
+    }, {
+      "relativeRanking" : 0.35,
+      "datum" : 83784
+    }, {
+      "relativeRanking" : 0.4,
+      "datum" : 83784
+    }, {
+      "relativeRanking" : 0.45,
+      "datum" : 83784
+    }, {
+      "relativeRanking" : 0.5,
+      "datum" : 83784
+    }, {
+      "relativeRanking" : 0.55,
+      "datum" : 83784
+    }, {
+      "relativeRanking" : 0.6,
+      "datum" : 83784
+    }, {
+      "relativeRanking" : 0.65,
+      "datum" : 83784
+    }, {
+      "relativeRanking" : 0.7,
+      "datum" : 83784
+    }, {
+      "relativeRanking" : 0.75,
+      "datum" : 83784
+    }, {
+      "relativeRanking" : 0.8,
+      "datum" : 83784
+    }, {
+      "relativeRanking" : 0.85,
+      "datum" : 83784
+    }, {
+      "relativeRanking" : 0.9,
+      "datum" : 83784
+    }, {
+      "relativeRanking" : 0.95,
+      "datum" : 83784
+    } ],
+    "numberValues" : 1
+  },
+  "failedReduceAttemptCDF" : {
+    "maximum" : 9223372036854775807,
+    "minimum" : -9223372036854775808,
+    "rankings" : [ ],
+    "numberValues" : 0
+  },
+  "mapperTriesToSucceed" : [ 0.9565217391304348, 0.043478260869565216 ],
+  "failedMapperFraction" : 0.0,
+  "relativeTime" : 0,
+  "queue" : null,
+  "clusterMapMB" : -1,
+  "clusterReduceMB" : -1,
+  "jobMapMB" : -1,
+  "jobReduceMB" : -1
+}
+ {
+  "priority" : "NORMAL",
+  "jobID" : "job_200904211745_0003",
+  "mapTasks" : [ {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.128\\.192", "cluster50494\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.128\\.192/cluster50494\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336870159,
+      "finishTime" : 1240336876906,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0003_m_000000_0",
+      "hdfsBytesRead" : 40,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 120,
+      "mapInputRecords" : 6,
+      "mapOutputBytes" : 64,
+      "mapOutputRecords" : 6,
+      "combineInputRecords" : 6,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 6,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "194\\.6\\.129\\.64", "cluster1823\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.129\\.64", "cluster1800\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.130\\.64", "cluster1979\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336862787,
+    "finishTime" : 1240336878867,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0003_m_000000",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.134\\.192", "cluster50396\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.134\\.192/cluster50396\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336863528,
+      "finishTime" : 1240336878419,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0003_m_000002_0",
+      "hdfsBytesRead" : 29,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 100,
+      "mapInputRecords" : 4,
+      "mapOutputBytes" : 45,
+      "mapOutputRecords" : 4,
+      "combineInputRecords" : 4,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 4,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.127\\.64", "cluster1457\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.127\\.64", "cluster1463\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.133\\.64", "cluster50144\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336862891,
+    "finishTime" : 1240336878973,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0003_m_000002",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.62\\.192", "cluster50416\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.62\\.192/cluster50416\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336863725,
+      "finishTime" : 1240336868610,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0003_m_000001_0",
+      "hdfsBytesRead" : 36,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 109,
+      "mapInputRecords" : 5,
+      "mapOutputBytes" : 56,
+      "mapOutputRecords" : 5,
+      "combineInputRecords" : 5,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 5,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.126\\.64", "cluster1346\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.126\\.64", "cluster1333\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.128\\.128", "cluster50476\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336862992,
+    "finishTime" : 1240336879103,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0003_m_000001",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  } ],
+  "reduceTasks" : [ {
+    "attempts" : [ {
+      "location" : null,
+      "hostName" : "/194\\.6\\.133\\.192/cluster50225\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336879990,
+      "finishTime" : 1240336912326,
+      "shuffleFinished" : 1240336906008,
+      "sortFinished" : 1240336906089,
+      "attemptID" : "attempt_200904211745_0003_r_000000_0",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : 61,
+      "fileBytesRead" : 112,
+      "fileBytesWritten" : 112,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : 0,
+      "reduceInputRecords" : 15,
+      "reduceShuffleBytes" : 233,
+      "reduceOutputRecords" : 0,
+      "spilledRecords" : 15,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ ],
+    "startTime" : 1240336879303,
+    "finishTime" : 1240336928223,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0003_r_000000",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "REDUCE"
+  } ],
+  "otherTasks" : [ {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.134\\.0", "cluster50261\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.134\\.0/cluster50261\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336846902,
+      "finishTime" : 1240336858206,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0003_m_000004_0",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 0,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ ],
+    "startTime" : 1240336846190,
+    "finishTime" : 1240336862349,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0003_m_000004",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "SETUP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.133\\.192", "cluster50225\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.133\\.192/cluster50225\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336928353,
+      "finishTime" : 1240336930147,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0003_m_000003_0",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 0,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ ],
+    "startTime" : 1240336928224,
+    "finishTime" : 1240336944237,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0003_m_000003",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "CLEANUP"
+  } ],
+  "finishTime" : 1240336944238,
+  "user" : "geek2",
+  "jobName" : null,
+  "computonsPerMapInputByte" : -1,
+  "computonsPerMapOutputByte" : -1,
+  "computonsPerReduceInputByte" : -1,
+  "computonsPerReduceOutputByte" : -1,
+  "submitTime" : 1240336843916,
+  "launchTime" : 1240336844448,
+  "heapMegabytes" : 640,
+  "totalMaps" : 3,
+  "totalReduces" : 1,
+  "outcome" : "SUCCESS",
+  "jobtype" : "JAVA",
+  "directDependantJobs" : [ ],
+  "successfulMapAttemptCDFs" : [ {
+    "maximum" : 9223372036854775807,
+    "minimum" : -9223372036854775808,
+    "rankings" : [ ],
+    "numberValues" : 0
+  }, {
+    "maximum" : 9223372036854775807,
+    "minimum" : -9223372036854775808,
+    "rankings" : [ ],
+    "numberValues" : 0
+  }, {
+    "maximum" : 14891,
+    "minimum" : 4885,
+    "rankings" : [ {
+      "relativeRanking" : 0.05,
+      "datum" : 4885
+    }, {
+      "relativeRanking" : 0.1,
+      "datum" : 4885
+    }, {
+      "relativeRanking" : 0.15,
+      "datum" : 4885
+    }, {
+      "relativeRanking" : 0.2,
+      "datum" : 4885
+    }, {
+      "relativeRanking" : 0.25,
+      "datum" : 4885
+    }, {
+      "relativeRanking" : 0.3,
+      "datum" : 4885
+    }, {
+      "relativeRanking" : 0.35,
+      "datum" : 4885
+    }, {
+      "relativeRanking" : 0.4,
+      "datum" : 4885
+    }, {
+      "relativeRanking" : 0.45,
+      "datum" : 4885
+    }, {
+      "relativeRanking" : 0.5,
+      "datum" : 4885
+    }, {
+      "relativeRanking" : 0.55,
+      "datum" : 4885
+    }, {
+      "relativeRanking" : 0.6,
+      "datum" : 4885
+    }, {
+      "relativeRanking" : 0.65,
+      "datum" : 4885
+    }, {
+      "relativeRanking" : 0.7,
+      "datum" : 6747
+    }, {
+      "relativeRanking" : 0.75,
+      "datum" : 6747
+    }, {
+      "relativeRanking" : 0.8,
+      "datum" : 6747
+    }, {
+      "relativeRanking" : 0.85,
+      "datum" : 6747
+    }, {
+      "relativeRanking" : 0.9,
+      "datum" : 6747
+    }, {
+      "relativeRanking" : 0.95,
+      "datum" : 6747
+    } ],
+    "numberValues" : 3
+  }, {
+    "maximum" : 11304,
+    "minimum" : 1794,
+    "rankings" : [ {
+      "relativeRanking" : 0.05,
+      "datum" : 1794
+    }, {
+      "relativeRanking" : 0.1,
+      "datum" : 1794
+    }, {
+      "relativeRanking" : 0.15,
+      "datum" : 1794
+    }, {
+      "relativeRanking" : 0.2,
+      "datum" : 1794
+    }, {
+      "relativeRanking" : 0.25,
+      "datum" : 1794
+    }, {
+      "relativeRanking" : 0.3,
+      "datum" : 1794
+    }, {
+      "relativeRanking" : 0.35,
+      "datum" : 1794
+    }, {
+      "relativeRanking" : 0.4,
+      "datum" : 1794
+    }, {
+      "relativeRanking" : 0.45,
+      "datum" : 1794
+    }, {
+      "relativeRanking" : 0.5,
+      "datum" : 1794
+    }, {
+      "relativeRanking" : 0.55,
+      "datum" : 1794
+    }, {
+      "relativeRanking" : 0.6,
+      "datum" : 1794
+    }, {
+      "relativeRanking" : 0.65,
+      "datum" : 1794
+    }, {
+      "relativeRanking" : 0.7,
+      "datum" : 1794
+    }, {
+      "relativeRanking" : 0.75,
+      "datum" : 1794
+    }, {
+      "relativeRanking" : 0.8,
+      "datum" : 1794
+    }, {
+      "relativeRanking" : 0.85,
+      "datum" : 1794
+    }, {
+      "relativeRanking" : 0.9,
+      "datum" : 1794
+    }, {
+      "relativeRanking" : 0.95,
+      "datum" : 1794
+    } ],
+    "numberValues" : 2
+  } ],
+  "failedMapAttemptCDFs" : [ {
+    "maximum" : 9223372036854775807,
+    "minimum" : -9223372036854775808,
+    "rankings" : [ ],
+    "numberValues" : 0
+  }, {
+    "maximum" : 9223372036854775807,
+    "minimum" : -9223372036854775808,
+    "rankings" : [ ],
+    "numberValues" : 0
+  }, {
+    "maximum" : 9223372036854775807,
+    "minimum" : -9223372036854775808,
+    "rankings" : [ ],
+    "numberValues" : 0
+  }, {
+    "maximum" : 9223372036854775807,
+    "minimum" : -9223372036854775808,
+    "rankings" : [ ],
+    "numberValues" : 0
+  } ],
+  "successfulReduceAttemptCDF" : {
+    "maximum" : 32336,
+    "minimum" : 32336,
+    "rankings" : [ {
+      "relativeRanking" : 0.05,
+      "datum" : 32336
+    }, {
+      "relativeRanking" : 0.1,
+      "datum" : 32336
+    }, {
+      "relativeRanking" : 0.15,
+      "datum" : 32336
+    }, {
+      "relativeRanking" : 0.2,
+      "datum" : 32336
+    }, {
+      "relativeRanking" : 0.25,
+      "datum" : 32336
+    }, {
+      "relativeRanking" : 0.3,
+      "datum" : 32336
+    }, {
+      "relativeRanking" : 0.35,
+      "datum" : 32336
+    }, {
+      "relativeRanking" : 0.4,
+      "datum" : 32336
+    }, {
+      "relativeRanking" : 0.45,
+      "datum" : 32336
+    }, {
+      "relativeRanking" : 0.5,
+      "datum" : 32336
+    }, {
+      "relativeRanking" : 0.55,
+      "datum" : 32336
+    }, {
+      "relativeRanking" : 0.6,
+      "datum" : 32336
+    }, {
+      "relativeRanking" : 0.65,
+      "datum" : 32336
+    }, {
+      "relativeRanking" : 0.7,
+      "datum" : 32336
+    }, {
+      "relativeRanking" : 0.75,
+      "datum" : 32336
+    }, {
+      "relativeRanking" : 0.8,
+      "datum" : 32336
+    }, {
+      "relativeRanking" : 0.85,
+      "datum" : 32336
+    }, {
+      "relativeRanking" : 0.9,
+      "datum" : 32336
+    }, {
+      "relativeRanking" : 0.95,
+      "datum" : 32336
+    } ],
+    "numberValues" : 1
+  },
+  "failedReduceAttemptCDF" : {
+    "maximum" : 9223372036854775807,
+    "minimum" : -9223372036854775808,
+    "rankings" : [ ],
+    "numberValues" : 0
+  },
+  "mapperTriesToSucceed" : [ 1.0 ],
+  "failedMapperFraction" : 0.0,
+  "relativeTime" : 0,
+  "queue" : null,
+  "clusterMapMB" : -1,
+  "clusterReduceMB" : -1,
+  "jobMapMB" : -1,
+  "jobReduceMB" : -1
+}
+ {
+  "priority" : "NORMAL",
+  "jobID" : "job_200904211745_0004",
+  "mapTasks" : [ {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.62\\.128", "cluster1702\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.62\\.128/cluster1702\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336873318,
+      "finishTime" : 1240336916053,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000024_0",
+      "hdfsBytesRead" : 64208964,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2476698,
+      "mapInputRecords" : 4024803,
+      "mapOutputBytes" : 4143839,
+      "mapOutputRecords" : 64187,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 64187,
+      "mapInputBytes" : 249623354
+    }, {
+      "location" : null,
+      "hostName" : "cluster1687\\.secondleveldomain\\.com",
+      "result" : "KILLED",
+      "startTime" : 1240336915251,
+      "finishTime" : 1240336948507,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000024_1",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : -1,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.62\\.128", "cluster1683\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.63\\.192", "cluster1025\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.63\\.192", "cluster1030\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872259,
+    "finishTime" : 1240336920373,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000024",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.117\\.64", "cluster1185\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.117\\.64/cluster1185\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336875106,
+      "finishTime" : 1240336929081,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000007_0",
+      "hdfsBytesRead" : 67416667,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2601457,
+      "mapInputRecords" : 4254279,
+      "mapOutputBytes" : 4350539,
+      "mapOutputRecords" : 67759,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 67759,
+      "mapInputBytes" : 263667529
+    }, {
+      "location" : null,
+      "hostName" : "cluster1195\\.secondleveldomain\\.com",
+      "result" : "KILLED",
+      "startTime" : 1240336918908,
+      "finishTime" : 1240336988786,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000007_1",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : -1,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.117\\.64", "cluster1174\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.117\\.64", "cluster1171\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.128\\.64", "cluster1600\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872260,
+    "finishTime" : 1240336936418,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000007",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.117\\.128", "cluster1209\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.117\\.128/cluster1209\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336874012,
+      "finishTime" : 1240336912483,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000037_0",
+      "hdfsBytesRead" : 62083938,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2407514,
+      "mapInputRecords" : 3862317,
+      "mapOutputBytes" : 4057593,
+      "mapOutputRecords" : 62680,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 62680,
+      "mapInputBytes" : 241895332
+    }, {
+      "location" : null,
+      "hostName" : "cluster1227\\.secondleveldomain\\.com",
+      "result" : "KILLED",
+      "startTime" : 1240336915459,
+      "finishTime" : 1240336949436,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000037_1",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : -1,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.116\\.128", "cluster1107\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.117\\.128", "cluster1215\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.117\\.128", "cluster1226\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872265,
+    "finishTime" : 1240336920359,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000037",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.116\\.128", "cluster1110\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.116\\.128/cluster1110\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336871992,
+      "finishTime" : 1240336912072,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000015_0",
+      "hdfsBytesRead" : 65491754,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2477099,
+      "mapInputRecords" : 4163656,
+      "mapOutputBytes" : 4140319,
+      "mapOutputRecords" : 64248,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 64248,
+      "mapInputBytes" : 256585350
+    }, {
+      "location" : null,
+      "hostName" : "cluster1825\\.secondleveldomain\\.com",
+      "result" : "KILLED",
+      "startTime" : 1240336921125,
+      "finishTime" : 1240336954463,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000015_1",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : -1,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.116\\.128", "cluster1104\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.127\\.64", "cluster1465\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.127\\.64", "cluster1462\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872268,
+    "finishTime" : 1240336920434,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000015",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.63\\.64", "cluster1755\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.63\\.64/cluster1755\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336876311,
+      "finishTime" : 1240336942586,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000041_0",
+      "hdfsBytesRead" : 33589630,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 35182817,
+      "mapInputRecords" : 1398039,
+      "mapOutputBytes" : 74392390,
+      "mapOutputRecords" : 1217208,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 1217208,
+      "mapInputBytes" : 103351186
+    }, {
+      "location" : null,
+      "hostName" : "cluster1441\\.secondleveldomain\\.com",
+      "result" : "KILLED",
+      "startTime" : 1240336916237,
+      "finishTime" : 1240336981644,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000041_1",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : -1,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.126\\.64", "cluster1346\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.126\\.64", "cluster1332\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.63\\.64", "cluster1759\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872270,
+    "finishTime" : 1240336952476,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000041",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.127\\.192", "cluster1534\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.127\\.192/cluster1534\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336873710,
+      "finishTime" : 1240336886646,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000115_0",
+      "hdfsBytesRead" : 7802358,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 61768,
+      "mapInputRecords" : 338528,
+      "mapOutputBytes" : 117149,
+      "mapOutputRecords" : 1831,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 1831,
+      "mapInputBytes" : 25534769
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.116\\.64", "cluster1070\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.116\\.64", "cluster1041\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.127\\.192", "cluster1534\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872271,
+    "finishTime" : 1240336888399,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000115",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.62\\.64", "cluster1657\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.62\\.64/cluster1657\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336872974,
+      "finishTime" : 1240336912238,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000022_0",
+      "hdfsBytesRead" : 64534282,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2454984,
+      "mapInputRecords" : 4070141,
+      "mapOutputBytes" : 4111780,
+      "mapOutputRecords" : 64137,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 64137,
+      "mapInputBytes" : 257286422
+    }, {
+      "location" : null,
+      "hostName" : "cluster1973\\.secondleveldomain\\.com",
+      "result" : "KILLED",
+      "startTime" : 1240336917218,
+      "finishTime" : 1240336950542,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000022_1",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : -1,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.62\\.64", "cluster1657\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.63\\.0", "cluster1782\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.63\\.0", "cluster1762\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872273,
+    "finishTime" : 1240336920438,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000022",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.129\\.64", "cluster1835\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.129\\.64/cluster1835\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336874481,
+      "finishTime" : 1240336949882,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000045_0",
+      "hdfsBytesRead" : 30958862,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2445,
+      "mapInputRecords" : 2212648,
+      "mapOutputBytes" : 508,
+      "mapOutputRecords" : 9,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 9,
+      "mapInputBytes" : 134455678
+    }, {
+      "location" : null,
+      "hostName" : "cluster1804\\.secondleveldomain\\.com",
+      "result" : "KILLED",
+      "startTime" : 1240336917613,
+      "finishTime" : 1240336951416,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000045_1",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : -1,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "194\\.6\\.129\\.64", "cluster1819\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.129\\.64", "cluster1809\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.134\\.64", "cluster50316\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872275,
+    "finishTime" : 1240336952483,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000045",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.134\\.0", "cluster50273\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.134\\.0/cluster50273\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336872929,
+      "finishTime" : 1240336917687,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000000_0",
+      "hdfsBytesRead" : 70551688,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2705106,
+      "mapInputRecords" : 4456455,
+      "mapOutputBytes" : 4576941,
+      "mapOutputRecords" : 71582,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 71582,
+      "mapInputBytes" : 275178411
+    }, {
+      "location" : null,
+      "hostName" : "cluster50279\\.secondleveldomain\\.com",
+      "result" : "KILLED",
+      "startTime" : 1240336915037,
+      "finishTime" : 1240336948454,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000000_1",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : -1,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "194\\.6\\.133\\.128", "cluster50191\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.134\\.0", "cluster50252\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.134\\.0", "cluster50263\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872276,
+    "finishTime" : 1240336920429,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000000",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : null,
+      "hostName" : "cluster1484\\.secondleveldomain\\.com",
+      "result" : "FAILED",
+      "startTime" : 1240336873052,
+      "finishTime" : 1240336891468,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000026_0",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : -1,
+      "mapInputBytes" : -1
+    }, {
+      "location" : {
+        "layers" : [ "194\\.6\\.133\\.64", "cluster50159\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.133\\.64/cluster50159\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336905498,
+      "finishTime" : 1240336942729,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000026_1",
+      "hdfsBytesRead" : 63978817,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2435223,
+      "mapInputRecords" : 4013967,
+      "mapOutputBytes" : 4114611,
+      "mapOutputRecords" : 63929,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 63929,
+      "mapInputBytes" : 248841916
+    }, {
+      "location" : null,
+      "hostName" : "cluster50025\\.secondleveldomain\\.com",
+      "result" : "KILLED",
+      "startTime" : 1240336916380,
+      "finishTime" : 1240336958665,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000026_2",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : -1,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.127\\.128", "cluster1504\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.63\\.192", "cluster1035\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.63\\.192", "cluster1008\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872278,
+    "finishTime" : 1240336952813,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000026",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.133\\.64", "cluster50125\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.133\\.64/cluster50125\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336872976,
+      "finishTime" : 1240336915820,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000012_0",
+      "hdfsBytesRead" : 65683422,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2514927,
+      "mapInputRecords" : 4135021,
+      "mapOutputBytes" : 4224631,
+      "mapOutputRecords" : 66548,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 66548,
+      "mapInputBytes" : 255713201
+    }, {
+      "location" : null,
+      "hostName" : "cluster1283\\.secondleveldomain\\.com",
+      "result" : "KILLED",
+      "startTime" : 1240336914458,
+      "finishTime" : 1240336947756,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000012_1",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : -1,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.126\\.0", "cluster1299\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.126\\.0", "cluster1315\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.133\\.64", "cluster50147\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872299,
+    "finishTime" : 1240336920463,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000012",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.134\\.64", "cluster50285\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.134\\.64/cluster50285\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336872964,
+      "finishTime" : 1240336939111,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000006_0",
+      "hdfsBytesRead" : 67732698,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2596196,
+      "mapInputRecords" : 4299703,
+      "mapOutputBytes" : 4415200,
+      "mapOutputRecords" : 70202,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 70202,
+      "mapInputBytes" : 264286110
+    }, {
+      "location" : null,
+      "hostName" : "cluster50282\\.secondleveldomain\\.com",
+      "result" : "KILLED",
+      "startTime" : 1240336915054,
+      "finishTime" : 1240336955710,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000006_1",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : -1,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.63\\.0", "cluster1781\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.134\\.64", "cluster50307\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.134\\.64", "cluster50311\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872300,
+    "finishTime" : 1240336952474,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000006",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.133\\.192", "cluster50204\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.133\\.192/cluster50204\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336872970,
+      "finishTime" : 1240336919341,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000002_0",
+      "hdfsBytesRead" : 68694241,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2641160,
+      "mapInputRecords" : 4318855,
+      "mapOutputBytes" : 4523968,
+      "mapOutputRecords" : 71535,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 71535,
+      "mapInputBytes" : 268342920
+    }, {
+      "location" : null,
+      "hostName" : "cluster1894\\.secondleveldomain\\.com",
+      "result" : "KILLED",
+      "startTime" : 1240336915343,
+      "finishTime" : 1240336956992,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000002_1",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : -1,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "194\\.6\\.129\\.192", "cluster1896\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.133\\.192", "cluster50213\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.133\\.192", "cluster50228\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872301,
+    "finishTime" : 1240336920455,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000002",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.128\\.0", "cluster1568\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.128\\.0/cluster1568\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336918628,
+      "finishTime" : 1240336945487,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000060_1",
+      "hdfsBytesRead" : 30143641,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2581,
+      "mapInputRecords" : 2183668,
+      "mapOutputBytes" : 619,
+      "mapOutputRecords" : 9,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 9,
+      "mapInputBytes" : 128698969
+    }, {
+      "location" : null,
+      "hostName" : "cluster1560\\.secondleveldomain\\.com",
+      "result" : "KILLED",
+      "startTime" : 1240336873218,
+      "finishTime" : 1240336938166,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000060_0",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : -1,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "194\\.6\\.128\\.0", "cluster1581\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.128\\.0", "cluster1560\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.134\\.0", "cluster50276\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872374,
+    "finishTime" : 1240336946441,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000060",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.117\\.128", "cluster1207\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.117\\.128/cluster1207\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336867477,
+      "finishTime" : 1240336891579,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000068_0",
+      "hdfsBytesRead" : 29914824,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2149,
+      "mapInputRecords" : 2146323,
+      "mapOutputBytes" : 176,
+      "mapOutputRecords" : 3,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 3,
+      "mapInputBytes" : 127372829
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.117\\.128", "cluster1239\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.117\\.128", "cluster1217\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.127\\.64", "cluster1479\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872374,
+    "finishTime" : 1240336904545,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000068",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.129\\.192", "cluster1911\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.129\\.192/cluster1911\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336875928,
+      "finishTime" : 1240336914911,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000008_0",
+      "hdfsBytesRead" : 67080786,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2606104,
+      "mapInputRecords" : 4182439,
+      "mapOutputBytes" : 4393005,
+      "mapOutputRecords" : 68146,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 68146,
+      "mapInputBytes" : 260552004
+    }, {
+      "location" : null,
+      "hostName" : "cluster50001\\.secondleveldomain\\.com",
+      "result" : "KILLED",
+      "startTime" : 1240336914959,
+      "finishTime" : 1240336959329,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000008_1",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : -1,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "194\\.6\\.129\\.192", "cluster1914\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.129\\.192", "cluster1911\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.132\\.128", "cluster50011\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872375,
+    "finishTime" : 1240336920488,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000008",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.129\\.128", "cluster1858\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.129\\.128/cluster1858\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336874685,
+      "finishTime" : 1240336917858,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000010_0",
+      "hdfsBytesRead" : 66510009,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2504720,
+      "mapInputRecords" : 4229507,
+      "mapOutputBytes" : 4193214,
+      "mapOutputRecords" : 65150,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 65150,
+      "mapInputBytes" : 261348815
+    }, {
+      "location" : null,
+      "hostName" : "cluster50030\\.secondleveldomain\\.com",
+      "result" : "KILLED",
+      "startTime" : 1240336914970,
+      "finishTime" : 1240336948391,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000010_1",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : -1,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.117\\.64", "cluster1180\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.117\\.64", "cluster1171\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.129\\.128", "cluster1857\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872376,
+    "finishTime" : 1240336920531,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000010",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.63\\.192", "cluster1021\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.63\\.192/cluster1021\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336872982,
+      "finishTime" : 1240336912193,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000019_0",
+      "hdfsBytesRead" : 64941817,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2419356,
+      "mapInputRecords" : 4085619,
+      "mapOutputBytes" : 4074644,
+      "mapOutputRecords" : 63304,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 63304,
+      "mapInputBytes" : 254958753
+    }, {
+      "location" : null,
+      "hostName" : "cluster1576\\.secondleveldomain\\.com",
+      "result" : "KILLED",
+      "startTime" : 1240336919809,
+      "finishTime" : 1240336953416,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000019_1",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : -1,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.63\\.192", "cluster1006\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.63\\.192", "cluster1011\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.128\\.0", "cluster1573\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872376,
+    "finishTime" : 1240336920515,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000019",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.62\\.128", "cluster1700\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.62\\.128/cluster1700\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336869803,
+      "finishTime" : 1240336893577,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000047_0",
+      "hdfsBytesRead" : 30559058,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2349,
+      "mapInputRecords" : 2184357,
+      "mapOutputBytes" : 444,
+      "mapOutputRecords" : 8,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 8,
+      "mapInputBytes" : 129670187
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.62\\.128", "cluster1717\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.62\\.128", "cluster1713\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.134\\.192", "cluster50389\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872377,
+    "finishTime" : 1240336904552,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000047",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.127\\.64", "cluster1457\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.127\\.64/cluster1457\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336871305,
+      "finishTime" : 1240336910899,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000017_0",
+      "hdfsBytesRead" : 65262059,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2517094,
+      "mapInputRecords" : 4042279,
+      "mapOutputBytes" : 4239279,
+      "mapOutputRecords" : 66022,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 66022,
+      "mapInputBytes" : 252581214
+    }, {
+      "location" : null,
+      "hostName" : "cluster50539\\.secondleveldomain\\.com",
+      "result" : "KILLED",
+      "startTime" : 1240336915026,
+      "finishTime" : 1240336948436,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000017_1",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : -1,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.127\\.64", "cluster1454\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.127\\.64", "cluster1466\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.129\\.0", "cluster50539\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872377,
+    "finishTime" : 1240336920534,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000017",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.128\\.0", "cluster1598\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.128\\.0/cluster1598\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336862833,
+      "finishTime" : 1240336886002,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000081_0",
+      "hdfsBytesRead" : 29655169,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2492,
+      "mapInputRecords" : 2160857,
+      "mapOutputBytes" : 519,
+      "mapOutputRecords" : 6,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 6,
+      "mapInputBytes" : 128065621
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "194\\.6\\.128\\.0", "cluster1566\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.128\\.0", "cluster1598\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.128\\.64", "cluster1626\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872378,
+    "finishTime" : 1240336904475,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000081",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.128\\.0", "cluster1592\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.128\\.0/cluster1592\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336876693,
+      "finishTime" : 1240336916107,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000021_0",
+      "hdfsBytesRead" : 64599955,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2454477,
+      "mapInputRecords" : 4026318,
+      "mapOutputBytes" : 4137511,
+      "mapOutputRecords" : 63818,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 63818,
+      "mapInputBytes" : 251469382
+    }, {
+      "location" : null,
+      "hostName" : "cluster1564\\.secondleveldomain\\.com",
+      "result" : "KILLED",
+      "startTime" : 1240336917825,
+      "finishTime" : 1240336951212,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000021_1",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : -1,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.116\\.192", "cluster1127\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.116\\.192", "cluster1131\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.128\\.0", "cluster1562\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872379,
+    "finishTime" : 1240336920487,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000021",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.135\\.128", "cluster3106\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.135\\.128/cluster3106\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336879219,
+      "finishTime" : 1240336913954,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000003_0",
+      "hdfsBytesRead" : 68427487,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2623342,
+      "mapInputRecords" : 4252689,
+      "mapOutputBytes" : 4454505,
+      "mapOutputRecords" : 70042,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 70042,
+      "mapInputBytes" : 264536055
+    }, {
+      "location" : null,
+      "hostName" : "cluster1829\\.secondleveldomain\\.com",
+      "result" : "KILLED",
+      "startTime" : 1240336917410,
+      "finishTime" : 1240336950698,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000003_1",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : -1,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.62\\.64", "cluster1672\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.135\\.128", "cluster3089\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.135\\.128", "cluster3114\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872379,
+    "finishTime" : 1240336920465,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000003",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.126\\.64", "cluster1334\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.126\\.64/cluster1334\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336874371,
+      "finishTime" : 1240336916964,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000013_0",
+      "hdfsBytesRead" : 65647050,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2545518,
+      "mapInputRecords" : 4090204,
+      "mapOutputBytes" : 4275432,
+      "mapOutputRecords" : 66126,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 66126,
+      "mapInputBytes" : 255910823
+    }, {
+      "location" : null,
+      "hostName" : "cluster1410\\.secondleveldomain\\.com",
+      "result" : "KILLED",
+      "startTime" : 1240336914216,
+      "finishTime" : 1240336963861,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000013_1",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : -1,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.126\\.192", "cluster1415\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.126\\.192", "cluster1412\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.126\\.64", "cluster1332\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872380,
+    "finishTime" : 1240336920509,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000013",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.127\\.64", "cluster1442\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.127\\.64/cluster1442\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336871928,
+      "finishTime" : 1240336911159,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000033_0",
+      "hdfsBytesRead" : 62843496,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2438028,
+      "mapInputRecords" : 3993110,
+      "mapOutputBytes" : 4086319,
+      "mapOutputRecords" : 63785,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 63785,
+      "mapInputBytes" : 246369440
+    }, {
+      "location" : null,
+      "hostName" : "cluster1941\\.secondleveldomain\\.com",
+      "result" : "KILLED",
+      "startTime" : 1240336909373,
+      "finishTime" : 1240336966403,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000033_1",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : -1,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.126\\.64", "cluster1356\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.126\\.64", "cluster1350\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.127\\.64", "cluster1455\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872380,
+    "finishTime" : 1240336920485,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000033",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : null,
+      "hostName" : "cluster1751\\.secondleveldomain\\.com",
+      "result" : "FAILED",
+      "startTime" : 1240336873015,
+      "finishTime" : 1240336891599,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000046_0",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : -1,
+      "mapInputBytes" : -1
+    }, {
+      "location" : {
+        "layers" : [ "194\\.6\\.134\\.192", "cluster50368\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.134\\.192/cluster50368\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336905058,
+      "finishTime" : 1240336926560,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000046_1",
+      "hdfsBytesRead" : 30599610,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2489,
+      "mapInputRecords" : 2280773,
+      "mapOutputBytes" : 658,
+      "mapOutputRecords" : 12,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 12,
+      "mapInputBytes" : 133157243
+    }, {
+      "location" : null,
+      "hostName" : "cluster1247\\.secondleveldomain\\.com",
+      "result" : "KILLED",
+      "startTime" : 1240336916005,
+      "finishTime" : 1240336942980,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000046_2",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : -1,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.63\\.64", "cluster1757\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.63\\.64", "cluster1751\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.134\\.64", "cluster50292\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872381,
+    "finishTime" : 1240336936977,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000046",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.63\\.0", "cluster1777\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.63\\.0/cluster1777\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336878797,
+      "finishTime" : 1240336901414,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000059_0",
+      "hdfsBytesRead" : 30176905,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2160,
+      "mapInputRecords" : 2227855,
+      "mapOutputBytes" : 210,
+      "mapOutputRecords" : 4,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 4,
+      "mapInputBytes" : 130221301
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.62\\.192", "cluster50425\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.62\\.192", "cluster50407\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.63\\.0", "cluster1774\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872381,
+    "finishTime" : 1240336904493,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000059",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.63\\.192", "cluster1027\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.63\\.192/cluster1027\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336873401,
+      "finishTime" : 1240336909982,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000038_0",
+      "hdfsBytesRead" : 61899686,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2392428,
+      "mapInputRecords" : 3861737,
+      "mapOutputBytes" : 3999150,
+      "mapOutputRecords" : 61718,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 61718,
+      "mapInputBytes" : 240872637
+    }, {
+      "location" : null,
+      "hostName" : "cluster3117\\.secondleveldomain\\.com",
+      "result" : "KILLED",
+      "startTime" : 1240336917616,
+      "finishTime" : 1240336952203,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000038_1",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : -1,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.63\\.192", "cluster1002\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.63\\.192", "cluster1027\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.135\\.128", "cluster3093\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872382,
+    "finishTime" : 1240336920531,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000038",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.116\\.128", "cluster1093\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.116\\.128/cluster1093\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336877844,
+      "finishTime" : 1240336902594,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000066_0",
+      "hdfsBytesRead" : 29972278,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2311,
+      "mapInputRecords" : 2151755,
+      "mapOutputBytes" : 476,
+      "mapOutputRecords" : 8,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 8,
+      "mapInputBytes" : 127537268
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.116\\.128", "cluster1118\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.63\\.192", "cluster1026\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.63\\.192", "cluster1031\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872383,
+    "finishTime" : 1240336904501,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000066",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.128\\.64", "cluster1632\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.128\\.64/cluster1632\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336874600,
+      "finishTime" : 1240336886276,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000098_0",
+      "hdfsBytesRead" : 8241095,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 53850,
+      "mapInputRecords" : 359279,
+      "mapOutputBytes" : 97992,
+      "mapOutputRecords" : 1544,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 1544,
+      "mapInputBytes" : 26848771
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.63\\.64", "cluster1758\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.128\\.64", "cluster1614\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.128\\.64", "cluster1632\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872383,
+    "finishTime" : 1240336888503,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000098",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.63\\.0", "cluster1785\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.63\\.0/cluster1785\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336879910,
+      "finishTime" : 1240336910063,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000062_0",
+      "hdfsBytesRead" : 30075619,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2239,
+      "mapInputRecords" : 2131858,
+      "mapOutputBytes" : 300,
+      "mapOutputRecords" : 6,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 6,
+      "mapInputBytes" : 126857339
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.126\\.64", "cluster1357\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.63\\.0", "cluster1776\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.63\\.0", "cluster1773\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872384,
+    "finishTime" : 1240336904537,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000062",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.128\\.64", "cluster1626\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.128\\.64/cluster1626\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336876699,
+      "finishTime" : 1240336914881,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000025_0",
+      "hdfsBytesRead" : 64200239,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2529950,
+      "mapInputRecords" : 3996858,
+      "mapOutputBytes" : 4239266,
+      "mapOutputRecords" : 65959,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 65959,
+      "mapInputBytes" : 248767046
+    }, {
+      "location" : null,
+      "hostName" : "cluster1601\\.secondleveldomain\\.com",
+      "result" : "KILLED",
+      "startTime" : 1240336918879,
+      "finishTime" : 1240336952244,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000025_1",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : -1,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "194\\.6\\.128\\.64", "cluster1633\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.128\\.64", "cluster1611\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.134\\.0", "cluster50275\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872384,
+    "finishTime" : 1240336920528,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000025",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.128\\.64", "cluster1628\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.128\\.64/cluster1628\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336881912,
+      "finishTime" : 1240336906234,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000079_0",
+      "hdfsBytesRead" : 29704868,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2212,
+      "mapInputRecords" : 2170913,
+      "mapOutputBytes" : 222,
+      "mapOutputRecords" : 4,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 4,
+      "mapInputBytes" : 127927659
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.127\\.128", "cluster1490\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.127\\.128", "cluster1482\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.128\\.64", "cluster1636\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872385,
+    "finishTime" : 1240336904599,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000079",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.127\\.64", "cluster1470\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.127\\.64/cluster1470\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336871928,
+      "finishTime" : 1240336893040,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000088_0",
+      "hdfsBytesRead" : 8927130,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 63070,
+      "mapInputRecords" : 416911,
+      "mapOutputBytes" : 119856,
+      "mapOutputRecords" : 2051,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 2051,
+      "mapInputBytes" : 29933360
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.127\\.64", "cluster1474\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.127\\.64", "cluster1479\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.130\\.0", "cluster1927\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872385,
+    "finishTime" : 1240336904691,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000088",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.62\\.64", "cluster1662\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.62\\.64/cluster1662\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336880282,
+      "finishTime" : 1240336919548,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000011_0",
+      "hdfsBytesRead" : 66183568,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2597810,
+      "mapInputRecords" : 4149962,
+      "mapOutputBytes" : 4367215,
+      "mapOutputRecords" : 67906,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 67906,
+      "mapInputBytes" : 256962052
+    }, {
+      "location" : null,
+      "hostName" : "cluster50234\\.secondleveldomain\\.com",
+      "result" : "KILLED",
+      "startTime" : 1240336915110,
+      "finishTime" : 1240336948409,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000011_1",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : -1,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.62\\.64", "cluster1652\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.135\\.64", "cluster3065\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.135\\.64", "cluster3076\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872386,
+    "finishTime" : 1240336920520,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000011",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.128\\.64", "cluster1623\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.128\\.64/cluster1623\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336879066,
+      "finishTime" : 1240336901295,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000087_0",
+      "hdfsBytesRead" : 29290661,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2299,
+      "mapInputRecords" : 2093812,
+      "mapOutputBytes" : 307,
+      "mapOutputRecords" : 5,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 5,
+      "mapInputBytes" : 124249977
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.63\\.192", "cluster1008\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.63\\.192", "cluster1020\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.128\\.64", "cluster1602\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872387,
+    "finishTime" : 1240336904528,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000087",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.127\\.192", "cluster1535\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.127\\.192/cluster1535\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336874347,
+      "finishTime" : 1240336896706,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000057_0",
+      "hdfsBytesRead" : 30235345,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2182,
+      "mapInputRecords" : 2261814,
+      "mapOutputBytes" : 234,
+      "mapOutputRecords" : 2,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 2,
+      "mapInputBytes" : 131804880
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.127\\.192", "cluster1559\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.128\\.0", "cluster1594\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.128\\.0", "cluster1579\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872387,
+    "finishTime" : 1240336904515,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000057",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.128\\.0", "cluster1574\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.128\\.0/cluster1574\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336876540,
+      "finishTime" : 1240336915909,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000028_0",
+      "hdfsBytesRead" : 63494931,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2479449,
+      "mapInputRecords" : 3946537,
+      "mapOutputBytes" : 4204287,
+      "mapOutputRecords" : 65600,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 65600,
+      "mapInputBytes" : 246575765
+    }, {
+      "location" : null,
+      "hostName" : "cluster50205\\.secondleveldomain\\.com",
+      "result" : "KILLED",
+      "startTime" : 1240336915586,
+      "finishTime" : 1240336949032,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000028_1",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : -1,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "194\\.6\\.128\\.0", "cluster1561\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.133\\.192", "cluster50206\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.133\\.192", "cluster50217\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872388,
+    "finishTime" : 1240336920508,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000028",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.126\\.0", "cluster1302\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.126\\.0/cluster1302\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336875716,
+      "finishTime" : 1240336915224,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000014_0",
+      "hdfsBytesRead" : 65542507,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2519207,
+      "mapInputRecords" : 4123667,
+      "mapOutputBytes" : 4223635,
+      "mapOutputRecords" : 65390,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 65390,
+      "mapInputBytes" : 256189404
+    }, {
+      "location" : null,
+      "hostName" : "cluster50154\\.secondleveldomain\\.com",
+      "result" : "KILLED",
+      "startTime" : 1240336915022,
+      "finishTime" : 1240336954536,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000014_1",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : -1,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.126\\.0", "cluster1299\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.126\\.0", "cluster1289\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.133\\.64", "cluster50142\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872388,
+    "finishTime" : 1240336920547,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000014",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.126\\.128", "cluster1377\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.126\\.128/cluster1377\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336870160,
+      "finishTime" : 1240336910333,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000018_0",
+      "hdfsBytesRead" : 65179399,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2488791,
+      "mapInputRecords" : 4131630,
+      "mapOutputBytes" : 4155682,
+      "mapOutputRecords" : 64683,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 64683,
+      "mapInputBytes" : 255327265
+    }, {
+      "location" : null,
+      "hostName" : "cluster1364\\.secondleveldomain\\.com",
+      "result" : "KILLED",
+      "startTime" : 1240336911917,
+      "finishTime" : 1240336945214,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000018_1",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : -1,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.126\\.128", "cluster1383\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.126\\.128", "cluster1395\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.128\\.128", "cluster50441\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872389,
+    "finishTime" : 1240336920490,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000018",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.126\\.64", "cluster1357\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.126\\.64/cluster1357\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336866494,
+      "finishTime" : 1240336887708,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000064_0",
+      "hdfsBytesRead" : 29993789,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2363,
+      "mapInputRecords" : 2168531,
+      "mapOutputBytes" : 454,
+      "mapOutputRecords" : 8,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 8,
+      "mapInputBytes" : 128381652
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.126\\.64", "cluster1321\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.128\\.128", "cluster50475\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.128\\.128", "cluster50473\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872389,
+    "finishTime" : 1240336904527,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000064",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.126\\.128", "cluster1368\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.126\\.128/cluster1368\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336871607,
+      "finishTime" : 1240336898707,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000063_0",
+      "hdfsBytesRead" : 29998454,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2352,
+      "mapInputRecords" : 2182549,
+      "mapOutputBytes" : 428,
+      "mapOutputRecords" : 8,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 8,
+      "mapInputBytes" : 128497476
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.126\\.128", "cluster1371\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.129\\.64", "cluster1822\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.129\\.64", "cluster1832\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872390,
+    "finishTime" : 1240336904536,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000063",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.117\\.192", "cluster1251\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.117\\.192/cluster1251\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336872977,
+      "finishTime" : 1240336910764,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000023_0",
+      "hdfsBytesRead" : 64457911,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2454975,
+      "mapInputRecords" : 4073107,
+      "mapOutputBytes" : 4111625,
+      "mapOutputRecords" : 63981,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 63981,
+      "mapInputBytes" : 252476783
+    }, {
+      "location" : null,
+      "hostName" : "cluster1255\\.secondleveldomain\\.com",
+      "result" : "KILLED",
+      "startTime" : 1240336916099,
+      "finishTime" : 1240336969463,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000023_1",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : -1,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.117\\.192", "cluster1246\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.117\\.192", "cluster1274\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.62\\.192", "cluster50423\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872390,
+    "finishTime" : 1240336920495,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000023",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.126\\.0", "cluster1281\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.126\\.0/cluster1281\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336873567,
+      "finishTime" : 1240336910205,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000074_0",
+      "hdfsBytesRead" : 29798610,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 3066,
+      "mapInputRecords" : 2137289,
+      "mapOutputBytes" : 1604,
+      "mapOutputRecords" : 38,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 38,
+      "mapInputBytes" : 126797079
+    }, {
+      "location" : null,
+      "hostName" : "cluster1830\\.secondleveldomain\\.com",
+      "result" : "KILLED",
+      "startTime" : 1240336920116,
+      "finishTime" : 1240336954303,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000074_1",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : -1,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.126\\.0", "cluster1316\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.128\\.192", "cluster50493\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.128\\.192", "cluster50488\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872391,
+    "finishTime" : 1240336920505,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000074",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.126\\.192", "cluster1420\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.126\\.192/cluster1420\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336870902,
+      "finishTime" : 1240336911334,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000001_0",
+      "hdfsBytesRead" : 69761283,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2709488,
+      "mapInputRecords" : 4346205,
+      "mapOutputBytes" : 4543664,
+      "mapOutputRecords" : 70626,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 70626,
+      "mapInputBytes" : 270462297
+    }, {
+      "location" : null,
+      "hostName" : "cluster1850\\.secondleveldomain\\.com",
+      "result" : "KILLED",
+      "startTime" : 1240336917236,
+      "finishTime" : 1240336950666,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000001_1",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : -1,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.126\\.192", "cluster1405\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.126\\.192", "cluster1427\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.134\\.192", "cluster50365\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872391,
+    "finishTime" : 1240336920486,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000001",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.127\\.192", "cluster1540\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.127\\.192/cluster1540\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336871750,
+      "finishTime" : 1240336899790,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000061_0",
+      "hdfsBytesRead" : 30111997,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2348,
+      "mapInputRecords" : 2160034,
+      "mapOutputBytes" : 354,
+      "mapOutputRecords" : 6,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 6,
+      "mapInputBytes" : 128130156
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.116\\.192", "cluster1138\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.116\\.192", "cluster1137\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.127\\.192", "cluster1520\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872392,
+    "finishTime" : 1240336904500,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000061",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.127\\.128", "cluster1493\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.127\\.128/cluster1493\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336874481,
+      "finishTime" : 1240336923543,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000049_0",
+      "hdfsBytesRead" : 30486394,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2076,
+      "mapInputRecords" : 2239224,
+      "mapOutputBytes" : 102,
+      "mapOutputRecords" : 2,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 2,
+      "mapInputBytes" : 131723681
+    }, {
+      "location" : null,
+      "hostName" : "cluster1519\\.secondleveldomain\\.com",
+      "result" : "KILLED",
+      "startTime" : 1240336915105,
+      "finishTime" : 1240336939262,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000049_1",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : -1,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.127\\.128", "cluster1518\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.128\\.192", "cluster50501\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.128\\.192", "cluster50484\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872393,
+    "finishTime" : 1240336936512,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000049",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.129\\.128", "cluster1843\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.129\\.128/cluster1843\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336873996,
+      "finishTime" : 1240336930684,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000029_0",
+      "hdfsBytesRead" : 63373946,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2388744,
+      "mapInputRecords" : 4033224,
+      "mapOutputBytes" : 3991838,
+      "mapOutputRecords" : 62073,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 62073,
+      "mapInputBytes" : 248134254
+    }, {
+      "location" : null,
+      "hostName" : "cluster1872\\.secondleveldomain\\.com",
+      "result" : "KILLED",
+      "startTime" : 1240336916333,
+      "finishTime" : 1240336965543,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000029_1",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : -1,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.63\\.192", "cluster1035\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.63\\.192", "cluster1018\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.129\\.128", "cluster1853\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872393,
+    "finishTime" : 1240336936568,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000029",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.127\\.192", "cluster1555\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.127\\.192/cluster1555\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336875187,
+      "finishTime" : 1240336898191,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000075_0",
+      "hdfsBytesRead" : 29746839,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2502,
+      "mapInputRecords" : 2130992,
+      "mapOutputBytes" : 642,
+      "mapOutputRecords" : 13,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 13,
+      "mapInputBytes" : 126084263
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.127\\.192", "cluster1557\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.133\\.192", "cluster50214\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.133\\.192", "cluster50210\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872394,
+    "finishTime" : 1240336904493,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000075",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.117\\.128", "cluster1228\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.117\\.128/cluster1228\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336873902,
+      "finishTime" : 1240336914414,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000085_0",
+      "hdfsBytesRead" : 29405954,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2264,
+      "mapInputRecords" : 2125410,
+      "mapOutputBytes" : 307,
+      "mapOutputRecords" : 6,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 6,
+      "mapInputBytes" : 125216183
+    }, {
+      "location" : null,
+      "hostName" : "cluster1239\\.secondleveldomain\\.com",
+      "result" : "KILLED",
+      "startTime" : 1240336914422,
+      "finishTime" : 1240336947813,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000085_1",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : -1,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.117\\.128", "cluster1212\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.117\\.128", "cluster1205\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.63\\.64", "cluster1743\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872394,
+    "finishTime" : 1240336920525,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000085",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.117\\.64", "cluster1180\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.117\\.64/cluster1180\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336873955,
+      "finishTime" : 1240336917137,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000009_0",
+      "hdfsBytesRead" : 66872279,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2517067,
+      "mapInputRecords" : 4251406,
+      "mapOutputBytes" : 4217383,
+      "mapOutputRecords" : 65906,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 65906,
+      "mapInputBytes" : 261611387
+    }, {
+      "location" : null,
+      "hostName" : "cluster50545\\.secondleveldomain\\.com",
+      "result" : "KILLED",
+      "startTime" : 1240336916823,
+      "finishTime" : 1240336950112,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000009_1",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : -1,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.117\\.64", "cluster1180\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.117\\.64", "cluster1193\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.134\\.64", "cluster50281\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872395,
+    "finishTime" : 1240336920563,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000009",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.62\\.128", "cluster1690\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.62\\.128/cluster1690\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336874755,
+      "finishTime" : 1240336907982,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000078_0",
+      "hdfsBytesRead" : 29710276,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2244,
+      "mapInputRecords" : 2103351,
+      "mapOutputBytes" : 429,
+      "mapOutputRecords" : 8,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 8,
+      "mapInputBytes" : 125317251
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.62\\.128", "cluster1703\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.62\\.128", "cluster1694\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.63\\.0", "cluster1768\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872395,
+    "finishTime" : 1240336920560,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000078",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.128\\.64", "cluster1639\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.128\\.64/cluster1639\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336879086,
+      "finishTime" : 1240336891448,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000091_0",
+      "hdfsBytesRead" : 8632081,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 61030,
+      "mapInputRecords" : 415389,
+      "mapOutputBytes" : 125127,
+      "mapOutputRecords" : 2025,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 2025,
+      "mapInputBytes" : 30313509
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "194\\.6\\.128\\.64", "cluster1635\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.134\\.64", "cluster50317\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.134\\.64", "cluster50281\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872396,
+    "finishTime" : 1240336888496,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000091",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.117\\.192", "cluster1250\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.117\\.192/cluster1250\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336874153,
+      "finishTime" : 1240336895738,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000083_0",
+      "hdfsBytesRead" : 29568513,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2278,
+      "mapInputRecords" : 2122899,
+      "mapOutputBytes" : 302,
+      "mapOutputRecords" : 5,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 5,
+      "mapInputBytes" : 125860315
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.117\\.192", "cluster1250\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.130\\.0", "cluster1931\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.130\\.0", "cluster1957\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872396,
+    "finishTime" : 1240336904518,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000083",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.63\\.64", "cluster1727\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.63\\.64/cluster1727\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336879765,
+      "finishTime" : 1240336932976,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000043_0",
+      "hdfsBytesRead" : 32717815,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 34330747,
+      "mapInputRecords" : 1341156,
+      "mapOutputBytes" : 71461506,
+      "mapOutputRecords" : 1155088,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 1155088,
+      "mapInputBytes" : 99883113
+    }, {
+      "location" : null,
+      "hostName" : "cluster50135\\.secondleveldomain\\.com",
+      "result" : "KILLED",
+      "startTime" : 1240336915113,
+      "finishTime" : 1240336964459,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000043_1",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : -1,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.63\\.64", "cluster1752\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.63\\.64", "cluster1725\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.133\\.64", "cluster50156\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872397,
+    "finishTime" : 1240336936550,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000043",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.126\\.64", "cluster1356\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.126\\.64/cluster1356\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336872994,
+      "finishTime" : 1240336894558,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000069_0",
+      "hdfsBytesRead" : 29896312,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2490,
+      "mapInputRecords" : 2161066,
+      "mapOutputBytes" : 626,
+      "mapOutputRecords" : 9,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 9,
+      "mapInputBytes" : 127337881
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.126\\.64", "cluster1358\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.126\\.64", "cluster1346\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.127\\.128", "cluster1518\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872397,
+    "finishTime" : 1240336904506,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000069",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.126\\.192", "cluster1418\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.126\\.192/cluster1418\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336872526,
+      "finishTime" : 1240336920970,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000042_0",
+      "hdfsBytesRead" : 32920579,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 34480517,
+      "mapInputRecords" : 1354003,
+      "mapOutputBytes" : 72052350,
+      "mapOutputRecords" : 1167455,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 1167455,
+      "mapInputBytes" : 100609874
+    }, {
+      "location" : null,
+      "hostName" : "cluster50192\\.secondleveldomain\\.com",
+      "result" : "KILLED",
+      "startTime" : 1240336914080,
+      "finishTime" : 1240336963396,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000042_1",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : -1,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.126\\.192", "cluster1400\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.133\\.128", "cluster50195\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.133\\.128", "cluster50179\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872397,
+    "finishTime" : 1240336936525,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000042",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.129\\.64", "cluster1807\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.129\\.64/cluster1807\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336874347,
+      "finishTime" : 1240336897429,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000052_0",
+      "hdfsBytesRead" : 30352683,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2466,
+      "mapInputRecords" : 2223842,
+      "mapOutputBytes" : 728,
+      "mapOutputRecords" : 12,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 12,
+      "mapInputBytes" : 130818373
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "194\\.6\\.129\\.64", "cluster1832\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.133\\.192", "cluster50226\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.133\\.192", "cluster50205\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872398,
+    "finishTime" : 1240336904498,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000052",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.117\\.192", "cluster1249\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.117\\.192/cluster1249\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336873038,
+      "finishTime" : 1240336937714,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000031_0",
+      "hdfsBytesRead" : 63310753,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2411979,
+      "mapInputRecords" : 3945460,
+      "mapOutputBytes" : 4026410,
+      "mapOutputRecords" : 62373,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 62373,
+      "mapInputBytes" : 245235770
+    }, {
+      "location" : null,
+      "hostName" : "cluster50242\\.secondleveldomain\\.com",
+      "result" : "KILLED",
+      "startTime" : 1240336916045,
+      "finishTime" : 1240336953642,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000031_1",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : -1,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.117\\.192", "cluster1248\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.134\\.0", "cluster50274\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.134\\.0", "cluster50260\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872398,
+    "finishTime" : 1240336952565,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000031",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.128\\.0", "cluster1561\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.128\\.0/cluster1561\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336876300,
+      "finishTime" : 1240336927625,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000032_0",
+      "hdfsBytesRead" : 63245290,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2440937,
+      "mapInputRecords" : 3984125,
+      "mapOutputBytes" : 4070942,
+      "mapOutputRecords" : 62997,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 62997,
+      "mapInputBytes" : 247088467
+    }, {
+      "location" : null,
+      "hostName" : "cluster1585\\.secondleveldomain\\.com",
+      "result" : "KILLED",
+      "startTime" : 1240336915947,
+      "finishTime" : 1240336965244,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000032_1",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : -1,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "194\\.6\\.128\\.0", "cluster1585\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.135\\.64", "cluster3061\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.135\\.64", "cluster3079\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872399,
+    "finishTime" : 1240336936618,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000032",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.126\\.128", "cluster1395\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.126\\.128/cluster1395\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336872064,
+      "finishTime" : 1240336894757,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000077_0",
+      "hdfsBytesRead" : 29714767,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2099,
+      "mapInputRecords" : 2121327,
+      "mapOutputBytes" : 152,
+      "mapOutputRecords" : 3,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 3,
+      "mapInputBytes" : 126146844
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.126\\.128", "cluster1379\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.126\\.128", "cluster1373\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.135\\.128", "cluster3085\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872400,
+    "finishTime" : 1240336904494,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000077",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.129\\.128", "cluster1877\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.129\\.128/cluster1877\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336872985,
+      "finishTime" : 1240336894181,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000086_0",
+      "hdfsBytesRead" : 29331367,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2457,
+      "mapInputRecords" : 2098982,
+      "mapOutputBytes" : 513,
+      "mapOutputRecords" : 9,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 9,
+      "mapInputBytes" : 124536660
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "194\\.6\\.129\\.128", "cluster1859\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.129\\.128", "cluster1871\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.132\\.192", "cluster50072\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872400,
+    "finishTime" : 1240336904496,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000086",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.126\\.192", "cluster1437\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.126\\.192/cluster1437\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336873019,
+      "finishTime" : 1240336921372,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000080_0",
+      "hdfsBytesRead" : 29702894,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2351,
+      "mapInputRecords" : 2127301,
+      "mapOutputBytes" : 379,
+      "mapOutputRecords" : 5,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 5,
+      "mapInputBytes" : 126316979
+    }, {
+      "location" : null,
+      "hostName" : "cluster50007\\.secondleveldomain\\.com",
+      "result" : "KILLED",
+      "startTime" : 1240336924581,
+      "finishTime" : 1240336946151,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000080_1",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : -1,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.127\\.192", "cluster1533\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.127\\.192", "cluster1529\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.63\\.192", "cluster1036\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872401,
+    "finishTime" : 1240336936524,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000080",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.62\\.128", "cluster1711\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.62\\.128/cluster1711\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336876373,
+      "finishTime" : 1240336889533,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000099_0",
+      "hdfsBytesRead" : 8195626,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 53268,
+      "mapInputRecords" : 364433,
+      "mapOutputBytes" : 92731,
+      "mapOutputRecords" : 1473,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 1473,
+      "mapInputBytes" : 26915776
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.62\\.128", "cluster1708\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.62\\.128", "cluster1714\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.133\\.0", "cluster50085\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872401,
+    "finishTime" : 1240336888509,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000099",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.63\\.0", "cluster1775\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.63\\.0/cluster1775\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336874334,
+      "finishTime" : 1240336895935,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000065_0",
+      "hdfsBytesRead" : 29991531,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2256,
+      "mapInputRecords" : 2146107,
+      "mapOutputBytes" : 291,
+      "mapOutputRecords" : 4,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 4,
+      "mapInputBytes" : 127398601
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.117\\.64", "cluster1175\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.63\\.0", "cluster1762\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.63\\.0", "cluster1786\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872405,
+    "finishTime" : 1240336904543,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000065",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.116\\.64", "cluster1072\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.116\\.64/cluster1072\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336876623,
+      "finishTime" : 1240336920733,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000016_0",
+      "hdfsBytesRead" : 65311537,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2517895,
+      "mapInputRecords" : 4137686,
+      "mapOutputBytes" : 4245719,
+      "mapOutputRecords" : 65718,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 65718,
+      "mapInputBytes" : 257371314
+    }, {
+      "location" : null,
+      "hostName" : "cluster1524\\.secondleveldomain\\.com",
+      "result" : "KILLED",
+      "startTime" : 1240336913870,
+      "finishTime" : 1240336947267,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000016_1",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : -1,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.116\\.64", "cluster1053\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.116\\.64", "cluster1055\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.128\\.128", "cluster50459\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872405,
+    "finishTime" : 1240336920528,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000016",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.129\\.64", "cluster1806\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.129\\.64/cluster1806\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336875605,
+      "finishTime" : 1240336887030,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000100_0",
+      "hdfsBytesRead" : 8167681,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 54317,
+      "mapInputRecords" : 363347,
+      "mapOutputBytes" : 93315,
+      "mapOutputRecords" : 1533,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 1533,
+      "mapInputBytes" : 26787874
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.62\\.64", "cluster1669\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.62\\.64", "cluster1643\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.129\\.64", "cluster1816\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872406,
+    "finishTime" : 1240336888526,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000100",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.130\\.0", "cluster1920\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.130\\.0/cluster1920\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336875470,
+      "finishTime" : 1240336912179,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000040_0",
+      "hdfsBytesRead" : 60683975,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2370660,
+      "mapInputRecords" : 3782452,
+      "mapOutputBytes" : 3971981,
+      "mapOutputRecords" : 61525,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 61525,
+      "mapInputBytes" : 235446580
+    }, {
+      "location" : null,
+      "hostName" : "cluster1953\\.secondleveldomain\\.com",
+      "result" : "KILLED",
+      "startTime" : 1240336916862,
+      "finishTime" : 1240336950159,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000040_1",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : -1,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.117\\.64", "cluster1167\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.130\\.0", "cluster1924\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.130\\.0", "cluster1925\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872407,
+    "finishTime" : 1240336920542,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000040",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.127\\.64", "cluster1448\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.127\\.64/cluster1448\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336872975,
+      "finishTime" : 1240336883733,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000097_0",
+      "hdfsBytesRead" : 8245699,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 57116,
+      "mapInputRecords" : 361754,
+      "mapOutputBytes" : 109299,
+      "mapOutputRecords" : 1733,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 1733,
+      "mapInputBytes" : 27052310
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.127\\.128", "cluster1515\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.127\\.64", "cluster1449\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.127\\.64", "cluster1446\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872407,
+    "finishTime" : 1240336888495,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000097",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.128\\.64", "cluster1610\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.128\\.64/cluster1610\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336874243,
+      "finishTime" : 1240336886322,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000114_0",
+      "hdfsBytesRead" : 7815881,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 56450,
+      "mapInputRecords" : 338147,
+      "mapOutputBytes" : 101843,
+      "mapOutputRecords" : 1713,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 1713,
+      "mapInputBytes" : 25352645
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "194\\.6\\.128\\.64", "cluster1629\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.128\\.64", "cluster1615\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.130\\.64", "cluster1989\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872408,
+    "finishTime" : 1240336888543,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000114",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.135\\.64", "cluster3068\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.135\\.64/cluster3068\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336871157,
+      "finishTime" : 1240336903920,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000030_0",
+      "hdfsBytesRead" : 63316983,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2499856,
+      "mapInputRecords" : 3998244,
+      "mapOutputBytes" : 4194242,
+      "mapOutputRecords" : 65558,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 65558,
+      "mapInputBytes" : 247082392
+    }, {
+      "location" : null,
+      "hostName" : "cluster50114\\.secondleveldomain\\.com",
+      "result" : "KILLED",
+      "startTime" : 1240336915039,
+      "finishTime" : 1240336948328,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000030_1",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : -1,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "194\\.6\\.133\\.0", "cluster50101\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.135\\.64", "cluster3054\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.135\\.64", "cluster3070\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872408,
+    "finishTime" : 1240336920494,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000030",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.135\\.64", "cluster3073\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.135\\.64/cluster3073\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336873209,
+      "finishTime" : 1240336892920,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000048_0",
+      "hdfsBytesRead" : 30548276,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2470,
+      "mapInputRecords" : 2272297,
+      "mapOutputBytes" : 588,
+      "mapOutputRecords" : 8,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 8,
+      "mapInputBytes" : 133071879
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "194\\.6\\.133\\.192", "cluster50232\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.135\\.64", "cluster3077\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.135\\.64", "cluster3055\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872409,
+    "finishTime" : 1240336904497,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000048",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.63\\.0", "cluster1788\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.63\\.0/cluster1788\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336887602,
+      "finishTime" : 1240336909157,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000072_0",
+      "hdfsBytesRead" : 29828627,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2687,
+      "mapInputRecords" : 2132740,
+      "mapOutputBytes" : 935,
+      "mapOutputRecords" : 16,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 16,
+      "mapInputBytes" : 126719922
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.62\\.64", "cluster1650\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.62\\.64", "cluster1666\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.63\\.0", "cluster1760\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872409,
+    "finishTime" : 1240336904505,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000072",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.127\\.192", "cluster1538\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.127\\.192/cluster1538\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336874182,
+      "finishTime" : 1240336886198,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000105_0",
+      "hdfsBytesRead" : 7999674,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 56463,
+      "mapInputRecords" : 351714,
+      "mapOutputBytes" : 102216,
+      "mapOutputRecords" : 1667,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 1667,
+      "mapInputBytes" : 26137712
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.127\\.192", "cluster1550\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.127\\.192", "cluster1557\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.128\\.192", "cluster50490\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872410,
+    "finishTime" : 1240336888521,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000105",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.116\\.128", "cluster1095\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.116\\.128/cluster1095\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336873957,
+      "finishTime" : 1240336885026,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000095_0",
+      "hdfsBytesRead" : 8324595,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 60582,
+      "mapInputRecords" : 370323,
+      "mapOutputBytes" : 112110,
+      "mapOutputRecords" : 1745,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 1745,
+      "mapInputBytes" : 27631638
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.116\\.128", "cluster1097\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.132\\.192", "cluster50058\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.132\\.192", "cluster50047\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872410,
+    "finishTime" : 1240336888498,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000095",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.135\\.128", "cluster3085\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.135\\.128/cluster3085\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336883981,
+      "finishTime" : 1240336903185,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000058_0",
+      "hdfsBytesRead" : 30204544,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2544,
+      "mapInputRecords" : 2174809,
+      "mapOutputBytes" : 619,
+      "mapOutputRecords" : 12,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 12,
+      "mapInputBytes" : 128829948
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.63\\.192", "cluster1029\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.135\\.128", "cluster3111\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.135\\.128", "cluster3085\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872411,
+    "finishTime" : 1240336904451,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000058",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.63\\.64", "cluster1736\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.63\\.64/cluster1736\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336874390,
+      "finishTime" : 1240336885111,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000129_0",
+      "hdfsBytesRead" : 7596302,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 56324,
+      "mapInputRecords" : 329237,
+      "mapOutputBytes" : 101487,
+      "mapOutputRecords" : 1619,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 1619,
+      "mapInputBytes" : 24704627
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.63\\.0", "cluster1796\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.63\\.64", "cluster1736\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.63\\.64", "cluster1757\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872411,
+    "finishTime" : 1240336888504,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000129",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.135\\.64", "cluster3075\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.135\\.64/cluster3075\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336871519,
+      "finishTime" : 1240336890113,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000084_0",
+      "hdfsBytesRead" : 29527352,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2526,
+      "mapInputRecords" : 2107299,
+      "mapOutputBytes" : 542,
+      "mapOutputRecords" : 9,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 9,
+      "mapInputBytes" : 125356348
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "194\\.6\\.134\\.192", "cluster50375\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.134\\.192", "cluster50382\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.135\\.64", "cluster3076\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872412,
+    "finishTime" : 1240336904452,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000084",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.117\\.192", "cluster1276\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.117\\.192/cluster1276\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336873160,
+      "finishTime" : 1240336910195,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000034_0",
+      "hdfsBytesRead" : 62826152,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2442116,
+      "mapInputRecords" : 3912487,
+      "mapOutputBytes" : 4109087,
+      "mapOutputRecords" : 63793,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 63793,
+      "mapInputBytes" : 243815199
+    }, {
+      "location" : null,
+      "hostName" : "cluster50061\\.secondleveldomain\\.com",
+      "result" : "KILLED",
+      "startTime" : 1240336948611,
+      "finishTime" : 1240336986156,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000034_1",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : -1,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.117\\.192", "cluster1276\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.117\\.192", "cluster1243\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.132\\.192", "cluster50046\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872412,
+    "finishTime" : 1240336920545,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000034",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.63\\.64", "cluster1722\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.63\\.64/cluster1722\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336886217,
+      "finishTime" : 1240336899148,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000092_0",
+      "hdfsBytesRead" : 8571804,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 65830,
+      "mapInputRecords" : 382509,
+      "mapOutputBytes" : 153711,
+      "mapOutputRecords" : 2418,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 2418,
+      "mapInputBytes" : 28713382
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.117\\.128", "cluster1221\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.63\\.64", "cluster1731\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.63\\.64", "cluster1732\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872413,
+    "finishTime" : 1240336888549,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000092",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.135\\.128", "cluster3103\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.135\\.128/cluster3103\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336871792,
+      "finishTime" : 1240336904149,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000039_0",
+      "hdfsBytesRead" : 61493951,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2383004,
+      "mapInputRecords" : 3826526,
+      "mapOutputBytes" : 4014747,
+      "mapOutputRecords" : 62222,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 62222,
+      "mapInputBytes" : 238695445
+    }, {
+      "location" : null,
+      "hostName" : "cluster50119\\.secondleveldomain\\.com",
+      "result" : "KILLED",
+      "startTime" : 1240336914788,
+      "finishTime" : 1240336948132,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000039_1",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : -1,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "194\\.6\\.134\\.128", "cluster50348\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.134\\.128", "cluster50352\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.135\\.128", "cluster3094\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872415,
+    "finishTime" : 1240336920500,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000039",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.135\\.128", "cluster3099\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.135\\.128/cluster3099\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336879295,
+      "finishTime" : 1240336889446,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000094_0",
+      "hdfsBytesRead" : 8363130,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 61542,
+      "mapInputRecords" : 362353,
+      "mapOutputBytes" : 116919,
+      "mapOutputRecords" : 1812,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 1812,
+      "mapInputBytes" : 27305556
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.126\\.0", "cluster1316\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.126\\.0", "cluster1283\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.135\\.128", "cluster3097\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872416,
+    "finishTime" : 1240336888457,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000094",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.133\\.128", "cluster50185\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.133\\.128/cluster50185\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336874482,
+      "finishTime" : 1240336901837,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000112_0",
+      "hdfsBytesRead" : 7897873,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 59138,
+      "mapInputRecords" : 348461,
+      "mapOutputBytes" : 109578,
+      "mapOutputRecords" : 1815,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 1815,
+      "mapInputBytes" : 25836442
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "194\\.6\\.133\\.128", "cluster50174\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.133\\.128", "cluster50185\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.133\\.64", "cluster50120\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872418,
+    "finishTime" : 1240336904531,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000112",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.134\\.64", "cluster50294\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.134\\.64/cluster50294\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336873068,
+      "finishTime" : 1240336912576,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000020_0",
+      "hdfsBytesRead" : 64761324,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2512516,
+      "mapInputRecords" : 4036072,
+      "mapOutputBytes" : 4220545,
+      "mapOutputRecords" : 65595,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 65595,
+      "mapInputBytes" : 251142910
+    }, {
+      "location" : null,
+      "hostName" : "cluster50045\\.secondleveldomain\\.com",
+      "result" : "KILLED",
+      "startTime" : 1240336915002,
+      "finishTime" : 1240336950197,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000020_1",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : -1,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "194\\.6\\.132\\.192", "cluster50055\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.134\\.64", "cluster50316\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.134\\.64", "cluster50315\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872419,
+    "finishTime" : 1240336920577,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000020",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.127\\.64", "cluster1447\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.127\\.64/cluster1447\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336871738,
+      "finishTime" : 1240336885628,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000120_0",
+      "hdfsBytesRead" : 7756197,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 57141,
+      "mapInputRecords" : 332797,
+      "mapOutputBytes" : 105980,
+      "mapOutputRecords" : 1680,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 1680,
+      "mapInputBytes" : 25063052
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.127\\.128", "cluster1491\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.127\\.128", "cluster1480\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.127\\.64", "cluster1444\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872419,
+    "finishTime" : 1240336888501,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000120",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.134\\.64", "cluster50300\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.134\\.64/cluster50300\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336873077,
+      "finishTime" : 1240336890562,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000117_0",
+      "hdfsBytesRead" : 7779599,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 54489,
+      "mapInputRecords" : 342102,
+      "mapOutputBytes" : 99769,
+      "mapOutputRecords" : 1593,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 1593,
+      "mapInputBytes" : 25758088
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.117\\.192", "cluster1259\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.117\\.192", "cluster1256\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.127\\.192", "cluster1525\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872420,
+    "finishTime" : 1240336904516,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000117",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.130\\.64", "cluster1969\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.130\\.64/cluster1969\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336874912,
+      "finishTime" : 1240336912529,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000027_0",
+      "hdfsBytesRead" : 63744100,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2482609,
+      "mapInputRecords" : 3916909,
+      "mapOutputBytes" : 4146823,
+      "mapOutputRecords" : 64050,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 64050,
+      "mapInputBytes" : 245872068
+    }, {
+      "location" : null,
+      "hostName" : "cluster1990\\.secondleveldomain\\.com",
+      "result" : "KILLED",
+      "startTime" : 1240336919940,
+      "finishTime" : 1240336953269,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000027_1",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : -1,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.116\\.192", "cluster1154\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.130\\.64", "cluster1999\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.130\\.64", "cluster1967\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872420,
+    "finishTime" : 1240336920525,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000027",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.127\\.128", "cluster1498\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.127\\.128/cluster1498\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336876312,
+      "finishTime" : 1240336913198,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000056_0",
+      "hdfsBytesRead" : 30250367,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2417,
+      "mapInputRecords" : 2228688,
+      "mapOutputBytes" : 448,
+      "mapOutputRecords" : 9,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 9,
+      "mapInputBytes" : 130399482
+    }, {
+      "location" : null,
+      "hostName" : "cluster50040\\.secondleveldomain\\.com",
+      "result" : "KILLED",
+      "startTime" : 1240336915085,
+      "finishTime" : 1240336948419,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000056_1",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : -1,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.127\\.128", "cluster1517\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.132\\.192", "cluster50053\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.132\\.192", "cluster50057\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872421,
+    "finishTime" : 1240336921375,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000056",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.135\\.128", "cluster3108\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.135\\.128/cluster3108\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336744382,
+      "finishTime" : 1240336754327,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000118_0",
+      "hdfsBytesRead" : 7761361,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 54570,
+      "mapInputRecords" : 333787,
+      "mapOutputBytes" : 97693,
+      "mapOutputRecords" : 1550,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 1550,
+      "mapInputBytes" : 25134207
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.127\\.192", "cluster1550\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.127\\.192", "cluster1533\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.130\\.0", "cluster1928\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872421,
+    "finishTime" : 1240336888458,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000118",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.134\\.64", "cluster50296\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.134\\.64/cluster50296\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336873049,
+      "finishTime" : 1240336907869,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000053_0",
+      "hdfsBytesRead" : 30302479,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2191,
+      "mapInputRecords" : 2169494,
+      "mapOutputBytes" : 201,
+      "mapOutputRecords" : 4,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 4,
+      "mapInputBytes" : 129763826
+    }, {
+      "location" : null,
+      "hostName" : "cluster1937\\.secondleveldomain\\.com",
+      "result" : "KILLED",
+      "startTime" : 1240336915752,
+      "finishTime" : 1240336948962,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000053_1",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : -1,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.116\\.64", "cluster1075\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.116\\.64", "cluster1042\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.130\\.0", "cluster1938\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872422,
+    "finishTime" : 1240336920575,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000053",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.63\\.0", "cluster1768\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.63\\.0/cluster1768\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336886240,
+      "finishTime" : 1240336908239,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000082_0",
+      "hdfsBytesRead" : 29653562,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2207,
+      "mapInputRecords" : 2109744,
+      "mapOutputBytes" : 228,
+      "mapOutputRecords" : 4,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 4,
+      "mapInputBytes" : 125417788
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.63\\.0", "cluster1772\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.132\\.128", "cluster50025\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.132\\.128", "cluster50023\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872423,
+    "finishTime" : 1240336904530,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000082",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.133\\.0", "cluster50108\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.133\\.0/cluster50108\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336915063,
+      "finishTime" : 1240336930514,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000110_1",
+      "hdfsBytesRead" : 7921493,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 54235,
+      "mapInputRecords" : 349247,
+      "mapOutputBytes" : 110600,
+      "mapOutputRecords" : 1804,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 1804,
+      "mapInputBytes" : 26033361
+    }, {
+      "location" : null,
+      "hostName" : "cluster50097\\.secondleveldomain\\.com",
+      "result" : "KILLED",
+      "startTime" : 1240336873099,
+      "finishTime" : 1240336927622,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000110_0",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : -1,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.62\\.64", "cluster1652\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.133\\.0", "cluster50108\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.133\\.0", "cluster50113\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872423,
+    "finishTime" : 1240336930526,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000110",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.134\\.192", "cluster50382\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.134\\.192/cluster50382\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336873058,
+      "finishTime" : 1240336913430,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000005_0",
+      "hdfsBytesRead" : 68085382,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2636901,
+      "mapInputRecords" : 4335874,
+      "mapOutputBytes" : 4439823,
+      "mapOutputRecords" : 69636,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 69636,
+      "mapInputBytes" : 266679989
+    }, {
+      "location" : null,
+      "hostName" : "cluster50186\\.secondleveldomain\\.com",
+      "result" : "KILLED",
+      "startTime" : 1240336915168,
+      "finishTime" : 1240336948458,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000005_1",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : -1,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "194\\.6\\.133\\.128", "cluster50166\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.134\\.192", "cluster50372\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.134\\.192", "cluster50377\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872424,
+    "finishTime" : 1240336920570,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000005",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.134\\.192", "cluster50377\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.134\\.192/cluster50377\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336873119,
+      "finishTime" : 1240336889361,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000089_0",
+      "hdfsBytesRead" : 8799897,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 55966,
+      "mapInputRecords" : 414955,
+      "mapOutputBytes" : 103189,
+      "mapOutputRecords" : 1701,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 1701,
+      "mapInputBytes" : 29805669
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "194\\.6\\.133\\.192", "cluster50232\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.134\\.192", "cluster50365\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.134\\.192", "cluster50364\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872425,
+    "finishTime" : 1240336904514,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000089",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.134\\.128", "cluster50348\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.134\\.128/cluster50348\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336873048,
+      "finishTime" : 1240336886665,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000107_0",
+      "hdfsBytesRead" : 7976981,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 54786,
+      "mapInputRecords" : 382253,
+      "mapOutputBytes" : 99015,
+      "mapOutputRecords" : 1613,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 1613,
+      "mapInputBytes" : 27687269
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "194\\.6\\.132\\.192", "cluster50043\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.134\\.128", "cluster50326\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.134\\.128", "cluster50325\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872425,
+    "finishTime" : 1240336888490,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000107",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.128\\.128", "cluster50477\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.128\\.128/cluster50477\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336877770,
+      "finishTime" : 1240336917699,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000004_0",
+      "hdfsBytesRead" : 68248954,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2553348,
+      "mapInputRecords" : 4274813,
+      "mapOutputBytes" : 4305896,
+      "mapOutputRecords" : 66582,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 66582,
+      "mapInputBytes" : 265135117
+    }, {
+      "location" : null,
+      "hostName" : "cluster1164\\.secondleveldomain\\.com",
+      "result" : "KILLED",
+      "startTime" : 1240336916017,
+      "finishTime" : 1240336949262,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000004_1",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : -1,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "194\\.6\\.128\\.128", "cluster50447\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.128\\.128", "cluster50464\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.128\\.192", "cluster50491\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872426,
+    "finishTime" : 1240336920537,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000004",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.116\\.64", "cluster1078\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.116\\.64/cluster1078\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336873042,
+      "finishTime" : 1240336905504,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000073_0",
+      "hdfsBytesRead" : 29810727,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2406,
+      "mapInputRecords" : 2120499,
+      "mapOutputBytes" : 442,
+      "mapOutputRecords" : 8,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 8,
+      "mapInputBytes" : 126971404
+    }, {
+      "location" : null,
+      "hostName" : "cluster1301\\.secondleveldomain\\.com",
+      "result" : "KILLED",
+      "startTime" : 1240336915497,
+      "finishTime" : 1240336948840,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000073_1",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : -1,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.116\\.64", "cluster1061\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.116\\.64", "cluster1040\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.134\\.0", "cluster50245\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872427,
+    "finishTime" : 1240336920518,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000073",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.129\\.0", "cluster50520\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.129\\.0/cluster50520\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336874158,
+      "finishTime" : 1240336896172,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000054_0",
+      "hdfsBytesRead" : 30291933,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2467,
+      "mapInputRecords" : 2197164,
+      "mapOutputBytes" : 537,
+      "mapOutputRecords" : 9,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 9,
+      "mapInputBytes" : 130029656
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "194\\.6\\.129\\.0", "cluster50530\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.129\\.0", "cluster50526\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.130\\.64", "cluster1970\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872427,
+    "finishTime" : 1240336904495,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000054",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.134\\.64", "cluster50319\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.134\\.64/cluster50319\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336873105,
+      "finishTime" : 1240336899922,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000071_0",
+      "hdfsBytesRead" : 29867527,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2529,
+      "mapInputRecords" : 2168718,
+      "mapOutputBytes" : 639,
+      "mapOutputRecords" : 9,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 9,
+      "mapInputBytes" : 128006719
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.63\\.192", "cluster1032\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.63\\.192", "cluster1026\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.130\\.0", "cluster1936\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872428,
+    "finishTime" : 1240336904508,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000071",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.128\\.0", "cluster1583\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.128\\.0/cluster1583\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336878767,
+      "finishTime" : 1240336889771,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000116_0",
+      "hdfsBytesRead" : 7788625,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 55129,
+      "mapInputRecords" : 334841,
+      "mapOutputBytes" : 105424,
+      "mapOutputRecords" : 1675,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 1675,
+      "mapInputBytes" : 25214728
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "194\\.6\\.128\\.0", "cluster1567\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.128\\.0", "cluster1594\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.129\\.192", "cluster1915\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872429,
+    "finishTime" : 1240336888563,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000116",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.129\\.192", "cluster1883\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.129\\.192/cluster1883\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336875263,
+      "finishTime" : 1240336916121,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000035_0",
+      "hdfsBytesRead" : 62277748,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2394254,
+      "mapInputRecords" : 3851918,
+      "mapOutputBytes" : 4034362,
+      "mapOutputRecords" : 62292,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 62292,
+      "mapInputBytes" : 241639956
+    }, {
+      "location" : null,
+      "hostName" : "cluster1891\\.secondleveldomain\\.com",
+      "result" : "KILLED",
+      "startTime" : 1240336914750,
+      "finishTime" : 1240336948061,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000035_1",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : -1,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.63\\.192", "cluster1017\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.63\\.192", "cluster1019\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.129\\.192", "cluster1900\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872430,
+    "finishTime" : 1240336920630,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000035",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.134\\.64", "cluster50297\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.134\\.64/cluster50297\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336873087,
+      "finishTime" : 1240336887207,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000103_0",
+      "hdfsBytesRead" : 8054880,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 57203,
+      "mapInputRecords" : 352581,
+      "mapOutputBytes" : 105232,
+      "mapOutputRecords" : 1677,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 1677,
+      "mapInputBytes" : 26276597
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "194\\.6\\.128\\.192", "cluster50490\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.130\\.0", "cluster1959\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.130\\.0", "cluster1933\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872430,
+    "finishTime" : 1240336888512,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000103",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.133\\.192", "cluster50203\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.133\\.192/cluster50203\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336873054,
+      "finishTime" : 1240336940061,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000036_0",
+      "hdfsBytesRead" : 62256077,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2425449,
+      "mapInputRecords" : 3896871,
+      "mapOutputBytes" : 4079715,
+      "mapOutputRecords" : 63812,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 63812,
+      "mapInputBytes" : 242202529
+    }, {
+      "location" : null,
+      "hostName" : "cluster50170\\.secondleveldomain\\.com",
+      "result" : "KILLED",
+      "startTime" : 1240336915065,
+      "finishTime" : 1240336955862,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000036_1",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : -1,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "194\\.6\\.133\\.128", "cluster50170\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.133\\.192", "cluster50208\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.133\\.192", "cluster50210\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872431,
+    "finishTime" : 1240336952548,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000036",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.134\\.64", "cluster50291\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.134\\.64/cluster50291\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336873125,
+      "finishTime" : 1240336888240,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000125_0",
+      "hdfsBytesRead" : 7644012,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 56569,
+      "mapInputRecords" : 330028,
+      "mapOutputBytes" : 107007,
+      "mapOutputRecords" : 1684,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 1684,
+      "mapInputBytes" : 24814596
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.116\\.192", "cluster1144\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.116\\.192", "cluster1141\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.130\\.0", "cluster1953\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872431,
+    "finishTime" : 1240336888528,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000125",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.130\\.0", "cluster1936\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.130\\.0/cluster1936\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336873757,
+      "finishTime" : 1240336886908,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000101_0",
+      "hdfsBytesRead" : 8100684,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 64050,
+      "mapInputRecords" : 353384,
+      "mapOutputBytes" : 148877,
+      "mapOutputRecords" : 2652,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 2652,
+      "mapInputBytes" : 26767647
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.127\\.128", "cluster1503\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.130\\.64", "cluster1961\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.130\\.64", "cluster1963\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872433,
+    "finishTime" : 1240336888492,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000101",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.63\\.0", "cluster1778\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.63\\.0/cluster1778\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336873042,
+      "finishTime" : 1240336884233,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000119_0",
+      "hdfsBytesRead" : 7760781,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 54166,
+      "mapInputRecords" : 342134,
+      "mapOutputBytes" : 100849,
+      "mapOutputRecords" : 1633,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 1633,
+      "mapInputBytes" : 25522379
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.116\\.128", "cluster1107\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.63\\.0", "cluster1798\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.63\\.0", "cluster1789\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872435,
+    "finishTime" : 1240336888500,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000119",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.134\\.128", "cluster50342\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.134\\.128/cluster50342\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336873123,
+      "finishTime" : 1240336886459,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000130_0",
+      "hdfsBytesRead" : 7567997,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 58226,
+      "mapInputRecords" : 330387,
+      "mapOutputBytes" : 106187,
+      "mapOutputRecords" : 1681,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 1681,
+      "mapInputBytes" : 24831142
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.117\\.192", "cluster1272\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.62\\.128", "cluster1683\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.62\\.128", "cluster1707\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872440,
+    "finishTime" : 1240336888519,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000130",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.129\\.64", "cluster1813\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.129\\.64/cluster1813\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336875527,
+      "finishTime" : 1240336911619,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000050_0",
+      "hdfsBytesRead" : 30483937,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2986,
+      "mapInputRecords" : 2181690,
+      "mapOutputBytes" : 2645,
+      "mapOutputRecords" : 59,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 59,
+      "mapInputBytes" : 129508563
+    }, {
+      "location" : null,
+      "hostName" : "cluster50099\\.secondleveldomain\\.com",
+      "result" : "KILLED",
+      "startTime" : 1240336915651,
+      "finishTime" : 1240336949203,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000050_1",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : -1,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.116\\.192", "cluster1151\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.62\\.64", "cluster1679\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.62\\.64", "cluster1678\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872447,
+    "finishTime" : 1240336920621,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000050",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.129\\.192", "cluster1906\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.129\\.192/cluster1906\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336877854,
+      "finishTime" : 1240336915858,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000090_0",
+      "hdfsBytesRead" : 8744991,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 59096,
+      "mapInputRecords" : 400419,
+      "mapOutputBytes" : 114548,
+      "mapOutputRecords" : 1879,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 1879,
+      "mapInputBytes" : 28968801
+    }, {
+      "location" : null,
+      "hostName" : "cluster50041\\.secondleveldomain\\.com",
+      "result" : "KILLED",
+      "startTime" : 1240336915097,
+      "finishTime" : 1240336966677,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000090_1",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : -1,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "194\\.6\\.129\\.192", "cluster1891\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.129\\.192", "cluster1913\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.132\\.192", "cluster50051\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872493,
+    "finishTime" : 1240336920834,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000090",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.133\\.64", "cluster50124\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.133\\.64/cluster50124\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336915274,
+      "finishTime" : 1240336938869,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000067_1",
+      "hdfsBytesRead" : 29917139,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2179,
+      "mapInputRecords" : 2151889,
+      "mapOutputBytes" : 189,
+      "mapOutputRecords" : 4,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 4,
+      "mapInputBytes" : 127323156
+    }, {
+      "location" : null,
+      "hostName" : "cluster50475\\.secondleveldomain\\.com",
+      "result" : "KILLED",
+      "startTime" : 1240336877497,
+      "finishTime" : 1240336951548,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000067_0",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : -1,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "194\\.6\\.128\\.128", "cluster50444\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.133\\.64", "cluster50124\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.133\\.64", "cluster50156\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872506,
+    "finishTime" : 1240336946437,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000067",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.129\\.192", "cluster1903\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.129\\.192/cluster1903\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336877965,
+      "finishTime" : 1240336889156,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000109_0",
+      "hdfsBytesRead" : 7954082,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 59357,
+      "mapInputRecords" : 344387,
+      "mapOutputBytes" : 107739,
+      "mapOutputRecords" : 1775,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 1775,
+      "mapInputBytes" : 25743781
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "194\\.6\\.129\\.128", "cluster1841\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.129\\.128", "cluster1842\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.129\\.192", "cluster1907\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872544,
+    "finishTime" : 1240336888614,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000109",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.133\\.192", "cluster50231\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.133\\.192/cluster50231\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336874559,
+      "finishTime" : 1240336902529,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000051_0",
+      "hdfsBytesRead" : 30393039,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2309,
+      "mapInputRecords" : 2170141,
+      "mapOutputBytes" : 418,
+      "mapOutputRecords" : 9,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 9,
+      "mapInputBytes" : 128768431
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.116\\.192", "cluster1130\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.116\\.192", "cluster1158\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.133\\.192", "cluster50205\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872547,
+    "finishTime" : 1240336904655,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000051",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.133\\.64", "cluster50159\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.133\\.64/cluster50159\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336873969,
+      "finishTime" : 1240336897513,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000070_0",
+      "hdfsBytesRead" : 29891233,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2372,
+      "mapInputRecords" : 2181020,
+      "mapOutputBytes" : 492,
+      "mapOutputRecords" : 9,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 9,
+      "mapInputBytes" : 128287582
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "194\\.6\\.128\\.192", "cluster50484\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.128\\.192", "cluster50509\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.133\\.64", "cluster50133\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872651,
+    "finishTime" : 1240336904748,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000070",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.116\\.128", "cluster1102\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.116\\.128/cluster1102\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336876290,
+      "finishTime" : 1240336900685,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000126_0",
+      "hdfsBytesRead" : 7643780,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 50624,
+      "mapInputRecords" : 332992,
+      "mapOutputBytes" : 89335,
+      "mapOutputRecords" : 1413,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 1413,
+      "mapInputBytes" : 24892336
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.116\\.128", "cluster1087\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.62\\.192", "cluster50420\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.62\\.192", "cluster50429\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872665,
+    "finishTime" : 1240336905377,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000126",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.130\\.64", "cluster1976\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.130\\.64/cluster1976\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336916201,
+      "finishTime" : 1240336948646,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000055_1",
+      "hdfsBytesRead" : 30263792,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2383,
+      "mapInputRecords" : 2169198,
+      "mapOutputBytes" : 547,
+      "mapOutputRecords" : 7,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 7,
+      "mapInputBytes" : 128672563
+    }, {
+      "location" : null,
+      "hostName" : "cluster1735\\.secondleveldomain\\.com",
+      "result" : "KILLED",
+      "startTime" : 1240336884134,
+      "finishTime" : 1240336962153,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000055_0",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : -1,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.62\\.64", "cluster1669\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.132\\.192", "cluster50076\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.132\\.192", "cluster50077\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872669,
+    "finishTime" : 1240336962561,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000055",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.134\\.0", "cluster50252\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.134\\.0/cluster50252\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336873488,
+      "finishTime" : 1240336913959,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000108_0",
+      "hdfsBytesRead" : 7959315,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 55584,
+      "mapInputRecords" : 358754,
+      "mapOutputBytes" : 100438,
+      "mapOutputRecords" : 1658,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 1658,
+      "mapInputBytes" : 26607263
+    }, {
+      "location" : null,
+      "hostName" : "cluster1738\\.secondleveldomain\\.com",
+      "result" : "KILLED",
+      "startTime" : 1240336925060,
+      "finishTime" : 1240336958446,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000108_1",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : -1,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "194\\.6\\.128\\.128", "cluster50460\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.128\\.128", "cluster50440\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.134\\.0", "cluster50272\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872723,
+    "finishTime" : 1240336920897,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000108",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.132\\.128", "cluster50034\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.132\\.128/cluster50034\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336915103,
+      "finishTime" : 1240336927032,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000102_1",
+      "hdfsBytesRead" : 8093114,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 57157,
+      "mapInputRecords" : 352241,
+      "mapOutputBytes" : 105550,
+      "mapOutputRecords" : 1675,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 1675,
+      "mapInputBytes" : 26466064
+    }, {
+      "location" : null,
+      "hostName" : "cluster50303\\.secondleveldomain\\.com",
+      "result" : "KILLED",
+      "startTime" : 1240336873423,
+      "finishTime" : 1240336926231,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000102_0",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : -1,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.62\\.64", "cluster1655\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.62\\.64", "cluster1648\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.132\\.192", "cluster50059\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872762,
+    "finishTime" : 1240336930552,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000102",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.134\\.192", "cluster50368\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.134\\.192/cluster50368\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336873439,
+      "finishTime" : 1240336902317,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000106_0",
+      "hdfsBytesRead" : 7984742,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 61170,
+      "mapInputRecords" : 347188,
+      "mapOutputBytes" : 117181,
+      "mapOutputRecords" : 1839,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 1839,
+      "mapInputBytes" : 26617003
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.62\\.64", "cluster1678\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.62\\.64", "cluster1654\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.128\\.192", "cluster50485\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872831,
+    "finishTime" : 1240336904966,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000106",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.133\\.64", "cluster50145\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.133\\.64/cluster50145\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336874931,
+      "finishTime" : 1240336890399,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000121_0",
+      "hdfsBytesRead" : 7753496,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 54465,
+      "mapInputRecords" : 335167,
+      "mapOutputBytes" : 104220,
+      "mapOutputRecords" : 1650,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 1650,
+      "mapInputBytes" : 25170333
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.126\\.128", "cluster1388\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.126\\.128", "cluster1383\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.133\\.64", "cluster50133\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872869,
+    "finishTime" : 1240336904994,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000121",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.128\\.192", "cluster50513\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.128\\.192/cluster50513\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336874944,
+      "finishTime" : 1240336885951,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000123_0",
+      "hdfsBytesRead" : 7711498,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 57707,
+      "mapInputRecords" : 336739,
+      "mapOutputBytes" : 111288,
+      "mapOutputRecords" : 1762,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 1762,
+      "mapInputBytes" : 25234362
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.127\\.64", "cluster1474\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.127\\.64", "cluster1445\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.62\\.64", "cluster1654\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872873,
+    "finishTime" : 1240336888962,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000123",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : null,
+      "hostName" : "cluster50458\\.secondleveldomain\\.com",
+      "result" : "FAILED",
+      "startTime" : 1240336873610,
+      "finishTime" : 1240336892202,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000113_0",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : -1,
+      "mapInputBytes" : -1
+    }, {
+      "location" : {
+        "layers" : [ "192\\.30\\.116\\.192", "cluster1150\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.116\\.192/cluster1150\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336905590,
+      "finishTime" : 1240336916127,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000113_1",
+      "hdfsBytesRead" : 7838525,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 54305,
+      "mapInputRecords" : 342277,
+      "mapOutputBytes" : 100228,
+      "mapOutputRecords" : 1564,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 1564,
+      "mapInputBytes" : 25563399
+    }, {
+      "location" : null,
+      "hostName" : "cluster3063\\.secondleveldomain\\.com",
+      "result" : "FAILED",
+      "startTime" : 1240336937806,
+      "finishTime" : 1240336971856,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000113_2",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : -1,
+      "mapInputBytes" : -1
+    }, {
+      "location" : null,
+      "hostName" : "cluster3063\\.secondleveldomain\\.com",
+      "result" : "KILLED",
+      "startTime" : 1240336937806,
+      "finishTime" : 1240336971856,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000113_3",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : -1,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "194\\.6\\.128\\.128", "cluster50468\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.135\\.64", "cluster3058\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.135\\.64", "cluster3063\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336872938,
+    "finishTime" : 1240336921233,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000113",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.135\\.64", "cluster3077\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.135\\.64/cluster3077\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336883475,
+      "finishTime" : 1240336893425,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000128_0",
+      "hdfsBytesRead" : 7614230,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 57139,
+      "mapInputRecords" : 327184,
+      "mapOutputBytes" : 98814,
+      "mapOutputRecords" : 1560,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 1560,
+      "mapInputBytes" : 24696916
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "194\\.6\\.134\\.0", "cluster50243\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.134\\.0", "cluster50279\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.135\\.64", "cluster3057\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336873047,
+    "finishTime" : 1240336889130,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000128",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.126\\.192", "cluster1429\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.126\\.192/cluster1429\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336869816,
+      "finishTime" : 1240336880564,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000124_0",
+      "hdfsBytesRead" : 7699474,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 57466,
+      "mapInputRecords" : 335634,
+      "mapOutputBytes" : 114714,
+      "mapOutputRecords" : 1809,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 1809,
+      "mapInputBytes" : 25212522
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.116\\.192", "cluster1139\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.127\\.64", "cluster1467\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.127\\.64", "cluster1479\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336873048,
+    "finishTime" : 1240336889146,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000124",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.127\\.64", "cluster1466\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.127\\.64/cluster1466\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336869967,
+      "finishTime" : 1240336905157,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000093_0",
+      "hdfsBytesRead" : 8500796,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 61110,
+      "mapInputRecords" : 381639,
+      "mapOutputBytes" : 111387,
+      "mapOutputRecords" : 1871,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 1871,
+      "mapInputBytes" : 28132206
+    }, {
+      "location" : null,
+      "hostName" : "cluster50174\\.secondleveldomain\\.com",
+      "result" : "KILLED",
+      "startTime" : 1240336914921,
+      "finishTime" : 1240336926260,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000093_1",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : -1,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.116\\.192", "cluster1157\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.133\\.128", "cluster50174\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.133\\.128", "cluster50177\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336873049,
+    "finishTime" : 1240336921197,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000093",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.116\\.192", "cluster1150\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.116\\.192/cluster1150\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336873948,
+      "finishTime" : 1240336901907,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000044_0",
+      "hdfsBytesRead" : 31080028,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2324,
+      "mapInputRecords" : 2317829,
+      "mapOutputBytes" : 406,
+      "mapOutputRecords" : 8,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 8,
+      "mapInputBytes" : 136117012
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.116\\.192", "cluster1140\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.116\\.192", "cluster1124\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.132\\.192", "cluster50058\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336873050,
+    "finishTime" : 1240336905227,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000044",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.127\\.192", "cluster1531\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.127\\.192/cluster1531\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336874990,
+      "finishTime" : 1240336911553,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000104_0",
+      "hdfsBytesRead" : 8040935,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 55143,
+      "mapInputRecords" : 350537,
+      "mapOutputBytes" : 100554,
+      "mapOutputRecords" : 1616,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 1616,
+      "mapInputBytes" : 26367904
+    }, {
+      "location" : null,
+      "hostName" : "cluster3092\\.secondleveldomain\\.com",
+      "result" : "KILLED",
+      "startTime" : 1240336961643,
+      "finishTime" : 1240336971382,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000104_1",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : -1,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.117\\.128", "cluster1235\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.133\\.128", "cluster50171\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.133\\.128", "cluster50161\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336873051,
+    "finishTime" : 1240336921172,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000104",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.126\\.0", "cluster1305\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.126\\.0/cluster1305\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336874698,
+      "finishTime" : 1240336924089,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000076_0",
+      "hdfsBytesRead" : 29718365,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 2434,
+      "mapInputRecords" : 2122176,
+      "mapOutputBytes" : 514,
+      "mapOutputRecords" : 8,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 8,
+      "mapInputBytes" : 125964532
+    }, {
+      "location" : null,
+      "hostName" : "cluster1964\\.secondleveldomain\\.com",
+      "result" : "KILLED",
+      "startTime" : 1240336915897,
+      "finishTime" : 1240336940807,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000076_1",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : -1,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.116\\.64", "cluster1053\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.116\\.64", "cluster1056\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.130\\.64", "cluster1968\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336873052,
+    "finishTime" : 1240336937226,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000076",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.127\\.64", "cluster1454\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.127\\.64/cluster1454\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336874635,
+      "finishTime" : 1240336886965,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000096_0",
+      "hdfsBytesRead" : 8294959,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 57076,
+      "mapInputRecords" : 373384,
+      "mapOutputBytes" : 115997,
+      "mapOutputRecords" : 1854,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 1854,
+      "mapInputBytes" : 27404774
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.63\\.192", "cluster1028\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.63\\.192", "cluster1030\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.132\\.192", "cluster50044\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336873053,
+    "finishTime" : 1240336889184,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000096",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.134\\.0", "cluster50254\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.134\\.0/cluster50254\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336874658,
+      "finishTime" : 1240336888484,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000127_0",
+      "hdfsBytesRead" : 7627825,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 60903,
+      "mapInputRecords" : 330989,
+      "mapOutputBytes" : 120185,
+      "mapOutputRecords" : 1985,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 1985,
+      "mapInputBytes" : 24806195
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.116\\.192", "cluster1157\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.116\\.192", "cluster1136\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.62\\.192", "cluster50409\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336873054,
+    "finishTime" : 1240336889174,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000127",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.126\\.64", "cluster1328\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.126\\.64/cluster1328\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336874488,
+      "finishTime" : 1240336913539,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000122_0",
+      "hdfsBytesRead" : 7746591,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 55261,
+      "mapInputRecords" : 337548,
+      "mapOutputBytes" : 110325,
+      "mapOutputRecords" : 1750,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 1750,
+      "mapInputBytes" : 25159067
+    }, {
+      "location" : null,
+      "hostName" : "cluster1060\\.secondleveldomain\\.com",
+      "result" : "KILLED",
+      "startTime" : 1240336915247,
+      "finishTime" : 1240336927337,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000122_1",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : -1,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.126\\.128", "cluster1370\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.126\\.128", "cluster1378\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "192\\.30\\.63\\.0", "cluster1793\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336873055,
+    "finishTime" : 1240336921185,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000122",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  }, {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "192\\.30\\.62\\.64", "cluster1641\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/192\\.30\\.62\\.64/cluster1641\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336879658,
+      "finishTime" : 1240336890529,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000111_0",
+      "hdfsBytesRead" : 7914429,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : 58343,
+      "mapInputRecords" : 345958,
+      "mapOutputBytes" : 107374,
+      "mapOutputRecords" : 1748,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 1748,
+      "mapInputBytes" : 25806139
+    } ],
+    "preferredLocations" : [ {
+      "layers" : [ "192\\.30\\.62\\.192", "cluster50418\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.133\\.0", "cluster50090\\.secondleveldomain\\.com" ]
+    }, {
+      "layers" : [ "194\\.6\\.133\\.0", "cluster50117\\.secondleveldomain\\.com" ]
+    } ],
+    "startTime" : 1240336873056,
+    "finishTime" : 1240336889145,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000111",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "MAP"
+  } ],
+  "reduceTasks" : [ {
+    "attempts" : [ {
+      "location" : null,
+      "hostName" : "/194\\.6\\.134\\.64/cluster50303\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336888934,
+      "finishTime" : 1240336985264,
+      "shuffleFinished" : 1240336974089,
+      "sortFinished" : 1240336974613,
+      "attemptID" : "attempt_200904211745_0004_r_000000_0",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : 1514383,
+      "fileBytesRead" : 2632927,
+      "fileBytesWritten" : 2632927,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : 135445,
+      "reduceInputRecords" : 139391,
+      "reduceShuffleBytes" : 4405338,
+      "reduceOutputRecords" : 139391,
+      "spilledRecords" : 139391,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ ],
+    "startTime" : 1240336888851,
+    "finishTime" : 1240337000993,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_r_000000",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "REDUCE"
+  }, {
+    "attempts" : [ {
+      "location" : null,
+      "hostName" : "/194\\.6\\.134\\.0/cluster50252\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336888952,
+      "finishTime" : 1240336985501,
+      "shuffleFinished" : 1240336971396,
+      "sortFinished" : 1240336971892,
+      "attemptID" : "attempt_200904211745_0004_r_000001_0",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : 1499786,
+      "fileBytesRead" : 2600635,
+      "fileBytesWritten" : 2600635,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : 127806,
+      "reduceInputRecords" : 130935,
+      "reduceShuffleBytes" : 4415977,
+      "reduceOutputRecords" : 130935,
+      "spilledRecords" : 130935,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ ],
+    "startTime" : 1240336888869,
+    "finishTime" : 1240337000960,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_r_000001",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "REDUCE"
+  }, {
+    "attempts" : [ {
+      "location" : null,
+      "hostName" : "/194\\.6\\.134\\.192/cluster50368\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336889019,
+      "finishTime" : 1240336985309,
+      "shuffleFinished" : 1240336975164,
+      "sortFinished" : 1240336975677,
+      "attemptID" : "attempt_200904211745_0004_r_000002_0",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : 1530537,
+      "fileBytesRead" : 2654430,
+      "fileBytesWritten" : 2654430,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : 133326,
+      "reduceInputRecords" : 136708,
+      "reduceShuffleBytes" : 4499527,
+      "reduceOutputRecords" : 136708,
+      "spilledRecords" : 136708,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ ],
+    "startTime" : 1240336888941,
+    "finishTime" : 1240337001041,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_r_000002",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "REDUCE"
+  }, {
+    "attempts" : [ {
+      "location" : null,
+      "hostName" : "/194\\.6\\.128\\.192/cluster50513\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336890410,
+      "finishTime" : 1240336987320,
+      "shuffleFinished" : 1240336974158,
+      "sortFinished" : 1240336974678,
+      "attemptID" : "attempt_200904211745_0004_r_000003_0",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : 1499750,
+      "fileBytesRead" : 2604086,
+      "fileBytesWritten" : 2604086,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : 129054,
+      "reduceInputRecords" : 132376,
+      "reduceShuffleBytes" : 4422289,
+      "reduceOutputRecords" : 132376,
+      "spilledRecords" : 132376,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ ],
+    "startTime" : 1240336888963,
+    "finishTime" : 1240337001095,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_r_000003",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "REDUCE"
+  }, {
+    "attempts" : [ {
+      "location" : null,
+      "hostName" : "/194\\.6\\.133\\.64/cluster50145\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336890439,
+      "finishTime" : 1240336986603,
+      "shuffleFinished" : 1240336973537,
+      "sortFinished" : 1240336974021,
+      "attemptID" : "attempt_200904211745_0004_r_000004_0",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : 1463089,
+      "fileBytesRead" : 2534927,
+      "fileBytesWritten" : 2534927,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : 125284,
+      "reduceInputRecords" : 128270,
+      "reduceShuffleBytes" : 4292451,
+      "reduceOutputRecords" : 128270,
+      "spilledRecords" : 128270,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ ],
+    "startTime" : 1240336888984,
+    "finishTime" : 1240337001109,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_r_000004",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "REDUCE"
+  }, {
+    "attempts" : [ {
+      "location" : null,
+      "hostName" : "/192\\.30\\.63\\.64/cluster1735\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336898197,
+      "finishTime" : 1240336994614,
+      "shuffleFinished" : 1240336982632,
+      "sortFinished" : 1240336983604,
+      "attemptID" : "attempt_200904211745_0004_r_000005_0",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : 1524925,
+      "fileBytesRead" : 2651643,
+      "fileBytesWritten" : 2651643,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : 133361,
+      "reduceInputRecords" : 136557,
+      "reduceShuffleBytes" : 4483041,
+      "reduceOutputRecords" : 136557,
+      "spilledRecords" : 136557,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ ],
+    "startTime" : 1240336889084,
+    "finishTime" : 1240337001321,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_r_000005",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "REDUCE"
+  }, {
+    "attempts" : [ {
+      "location" : null,
+      "hostName" : "/194\\.6\\.133\\.64/cluster50132\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336889699,
+      "finishTime" : 1240336985779,
+      "shuffleFinished" : 1240336974653,
+      "sortFinished" : 1240336975212,
+      "attemptID" : "attempt_200904211745_0004_r_000006_0",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : 1490233,
+      "fileBytesRead" : 2582328,
+      "fileBytesWritten" : 2582328,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : 127906,
+      "reduceInputRecords" : 131571,
+      "reduceShuffleBytes" : 4383017,
+      "reduceOutputRecords" : 131571,
+      "spilledRecords" : 131571,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ ],
+    "startTime" : 1240336889089,
+    "finishTime" : 1240337001250,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_r_000006",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "REDUCE"
+  }, {
+    "attempts" : [ {
+      "location" : null,
+      "hostName" : "/194\\.6\\.133\\.192/cluster50235\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336889794,
+      "finishTime" : 1240336986114,
+      "shuffleFinished" : 1240336973868,
+      "sortFinished" : 1240336974400,
+      "attemptID" : "attempt_200904211745_0004_r_000007_0",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : 1505343,
+      "fileBytesRead" : 2610391,
+      "fileBytesWritten" : 2610391,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : 129022,
+      "reduceInputRecords" : 132145,
+      "reduceShuffleBytes" : 4444172,
+      "reduceOutputRecords" : 132145,
+      "spilledRecords" : 132145,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ ],
+    "startTime" : 1240336889090,
+    "finishTime" : 1240337001307,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_r_000007",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "REDUCE"
+  }, {
+    "attempts" : [ {
+      "location" : null,
+      "hostName" : "/194\\.6\\.128\\.192/cluster50491\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336892799,
+      "finishTime" : 1240336989074,
+      "shuffleFinished" : 1240336977913,
+      "sortFinished" : 1240336978491,
+      "attemptID" : "attempt_200904211745_0004_r_000008_0",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : 1504684,
+      "fileBytesRead" : 2608073,
+      "fileBytesWritten" : 2608073,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : 127971,
+      "reduceInputRecords" : 131200,
+      "reduceShuffleBytes" : 4441998,
+      "reduceOutputRecords" : 131200,
+      "spilledRecords" : 131200,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ ],
+    "startTime" : 1240336889091,
+    "finishTime" : 1240337001271,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_r_000008",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "REDUCE"
+  }, {
+    "attempts" : [ {
+      "location" : null,
+      "hostName" : "/192\\.30\\.62\\.64/cluster1679\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336889759,
+      "finishTime" : 1240336986187,
+      "shuffleFinished" : 1240336974015,
+      "sortFinished" : 1240336974545,
+      "attemptID" : "attempt_200904211745_0004_r_000009_0",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : 1496159,
+      "fileBytesRead" : 2593399,
+      "fileBytesWritten" : 2593399,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : 128913,
+      "reduceInputRecords" : 131980,
+      "reduceShuffleBytes" : 4397570,
+      "reduceOutputRecords" : 131980,
+      "spilledRecords" : 131980,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ ],
+    "startTime" : 1240336889094,
+    "finishTime" : 1240337001265,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_r_000009",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "REDUCE"
+  }, {
+    "attempts" : [ {
+      "location" : null,
+      "hostName" : "/194\\.6\\.132\\.128/cluster50024\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336889731,
+      "finishTime" : 1240336985667,
+      "shuffleFinished" : 1240336975405,
+      "sortFinished" : 1240336975928,
+      "attemptID" : "attempt_200904211745_0004_r_000010_0",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : 1516034,
+      "fileBytesRead" : 2633863,
+      "fileBytesWritten" : 2633863,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : 130278,
+      "reduceInputRecords" : 133696,
+      "reduceShuffleBytes" : 4454003,
+      "reduceOutputRecords" : 133696,
+      "spilledRecords" : 133696,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ ],
+    "startTime" : 1240336889095,
+    "finishTime" : 1240337001270,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_r_000010",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "REDUCE"
+  }, {
+    "attempts" : [ {
+      "location" : null,
+      "hostName" : "/194\\.6\\.133\\.192/cluster50223\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336889705,
+      "finishTime" : 1240336985456,
+      "shuffleFinished" : 1240336972242,
+      "sortFinished" : 1240336972740,
+      "attemptID" : "attempt_200904211745_0004_r_000011_0",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : 1493749,
+      "fileBytesRead" : 2585694,
+      "fileBytesWritten" : 2585694,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : 128462,
+      "reduceInputRecords" : 131413,
+      "reduceShuffleBytes" : 4380350,
+      "reduceOutputRecords" : 131413,
+      "spilledRecords" : 131413,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ ],
+    "startTime" : 1240336889096,
+    "finishTime" : 1240337001272,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_r_000011",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "REDUCE"
+  }, {
+    "attempts" : [ {
+      "location" : null,
+      "hostName" : "/192\\.30\\.126\\.0/cluster1314\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336890592,
+      "finishTime" : 1240336986175,
+      "shuffleFinished" : 1240336973921,
+      "sortFinished" : 1240336974467,
+      "attemptID" : "attempt_200904211745_0004_r_000012_0",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : 1542372,
+      "fileBytesRead" : 2681618,
+      "fileBytesWritten" : 2681618,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : 136804,
+      "reduceInputRecords" : 139932,
+      "reduceShuffleBytes" : 4537451,
+      "reduceOutputRecords" : 139932,
+      "spilledRecords" : 139932,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ ],
+    "startTime" : 1240336889100,
+    "finishTime" : 1240337001225,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_r_000012",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "REDUCE"
+  }, {
+    "attempts" : [ {
+      "location" : null,
+      "hostName" : "/192\\.30\\.127\\.64/cluster1440\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336890397,
+      "finishTime" : 1240336986407,
+      "shuffleFinished" : 1240336974415,
+      "sortFinished" : 1240336974913,
+      "attemptID" : "attempt_200904211745_0004_r_000013_0",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : 1518799,
+      "fileBytesRead" : 2631693,
+      "fileBytesWritten" : 2631693,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : 129644,
+      "reduceInputRecords" : 132582,
+      "reduceShuffleBytes" : 4464832,
+      "reduceOutputRecords" : 132582,
+      "spilledRecords" : 132582,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ ],
+    "startTime" : 1240336889101,
+    "finishTime" : 1240337001247,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_r_000013",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "REDUCE"
+  }, {
+    "attempts" : [ {
+      "location" : null,
+      "hostName" : "/192\\.30\\.62\\.64/cluster1647\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336889674,
+      "finishTime" : 1240336985701,
+      "shuffleFinished" : 1240336974501,
+      "sortFinished" : 1240336975018,
+      "attemptID" : "attempt_200904211745_0004_r_000014_0",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : 1529493,
+      "fileBytesRead" : 2660190,
+      "fileBytesWritten" : 2660190,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : 130930,
+      "reduceInputRecords" : 134318,
+      "reduceShuffleBytes" : 4508491,
+      "reduceOutputRecords" : 134318,
+      "spilledRecords" : 134318,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ ],
+    "startTime" : 1240336889106,
+    "finishTime" : 1240337001250,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_r_000014",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "REDUCE"
+  }, {
+    "attempts" : [ {
+      "location" : null,
+      "hostName" : "/192\\.30\\.126\\.0/cluster1295\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336890630,
+      "finishTime" : 1240336986643,
+      "shuffleFinished" : 1240336974515,
+      "sortFinished" : 1240336975047,
+      "attemptID" : "attempt_200904211745_0004_r_000015_0",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : 1488785,
+      "fileBytesRead" : 2587120,
+      "fileBytesWritten" : 2587120,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : 129374,
+      "reduceInputRecords" : 132757,
+      "reduceShuffleBytes" : 4397939,
+      "reduceOutputRecords" : 132757,
+      "spilledRecords" : 132757,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ ],
+    "startTime" : 1240336889107,
+    "finishTime" : 1240337001221,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_r_000015",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "REDUCE"
+  }, {
+    "attempts" : [ {
+      "location" : null,
+      "hostName" : "/194\\.6\\.135\\.64/cluster3077\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336899098,
+      "finishTime" : 1240336995768,
+      "shuffleFinished" : 1240336982067,
+      "sortFinished" : 1240336982475,
+      "attemptID" : "attempt_200904211745_0004_r_000016_0",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : 1482497,
+      "fileBytesRead" : 2568924,
+      "fileBytesWritten" : 2568924,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : 125907,
+      "reduceInputRecords" : 128797,
+      "reduceShuffleBytes" : 4350055,
+      "reduceOutputRecords" : 128797,
+      "spilledRecords" : 128797,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ ],
+    "startTime" : 1240336889131,
+    "finishTime" : 1240337001197,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_r_000016",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "REDUCE"
+  }, {
+    "attempts" : [ {
+      "location" : null,
+      "hostName" : "/192\\.30\\.62\\.64/cluster1641\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336895225,
+      "finishTime" : 1240336991501,
+      "shuffleFinished" : 1240336978367,
+      "sortFinished" : 1240336978877,
+      "attemptID" : "attempt_200904211745_0004_r_000017_0",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : 1501337,
+      "fileBytesRead" : 2604597,
+      "fileBytesWritten" : 2604597,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : 131288,
+      "reduceInputRecords" : 134365,
+      "reduceShuffleBytes" : 4400916,
+      "reduceOutputRecords" : 134365,
+      "spilledRecords" : 134365,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ ],
+    "startTime" : 1240336889145,
+    "finishTime" : 1240337001222,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_r_000017",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "REDUCE"
+  }, {
+    "attempts" : [ {
+      "location" : null,
+      "hostName" : "/192\\.30\\.126\\.192/cluster1429\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336885349,
+      "finishTime" : 1240336981576,
+      "shuffleFinished" : 1240336968527,
+      "sortFinished" : 1240336969054,
+      "attemptID" : "attempt_200904211745_0004_r_000018_0",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : 1512739,
+      "fileBytesRead" : 2623583,
+      "fileBytesWritten" : 2623583,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : 131750,
+      "reduceInputRecords" : 134780,
+      "reduceShuffleBytes" : 4448997,
+      "reduceOutputRecords" : 134780,
+      "spilledRecords" : 134780,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ ],
+    "startTime" : 1240336889147,
+    "finishTime" : 1240337001223,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_r_000018",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "REDUCE"
+  }, {
+    "attempts" : [ {
+      "location" : null,
+      "hostName" : "/192\\.30\\.127\\.192/cluster1531\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336890514,
+      "finishTime" : 1240336987216,
+      "shuffleFinished" : 1240336974189,
+      "sortFinished" : 1240336974682,
+      "attemptID" : "attempt_200904211745_0004_r_000019_0",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : 1497900,
+      "fileBytesRead" : 2595945,
+      "fileBytesWritten" : 2595945,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : 127094,
+      "reduceInputRecords" : 130020,
+      "reduceShuffleBytes" : 4421936,
+      "reduceOutputRecords" : 130020,
+      "spilledRecords" : 130020,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ ],
+    "startTime" : 1240336889152,
+    "finishTime" : 1240337001238,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_r_000019",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "REDUCE"
+  }, {
+    "attempts" : [ {
+      "location" : null,
+      "hostName" : "/192\\.30\\.126\\.64/cluster1328\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336890066,
+      "finishTime" : 1240336986478,
+      "shuffleFinished" : 1240336976422,
+      "sortFinished" : 1240336976934,
+      "attemptID" : "attempt_200904211745_0004_r_000020_0",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : 1527353,
+      "fileBytesRead" : 2649256,
+      "fileBytesWritten" : 2649256,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : 133593,
+      "reduceInputRecords" : 136882,
+      "reduceShuffleBytes" : 4474494,
+      "reduceOutputRecords" : 136882,
+      "spilledRecords" : 136882,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ ],
+    "startTime" : 1240336889157,
+    "finishTime" : 1240337001251,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_r_000020",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "REDUCE"
+  }, {
+    "attempts" : [ {
+      "location" : null,
+      "hostName" : "/194\\.6\\.134\\.0/cluster50254\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336890210,
+      "finishTime" : 1240336986556,
+      "shuffleFinished" : 1240336973579,
+      "sortFinished" : 1240336974104,
+      "attemptID" : "attempt_200904211745_0004_r_000021_0",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : 1513275,
+      "fileBytesRead" : 2630609,
+      "fileBytesWritten" : 2630609,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : 131516,
+      "reduceInputRecords" : 135124,
+      "reduceShuffleBytes" : 4466234,
+      "reduceOutputRecords" : 135124,
+      "spilledRecords" : 135124,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ ],
+    "startTime" : 1240336889174,
+    "finishTime" : 1240337001247,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_r_000021",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "REDUCE"
+  }, {
+    "attempts" : [ {
+      "location" : null,
+      "hostName" : "/192\\.30\\.127\\.64/cluster1466\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336885559,
+      "finishTime" : 1240336982206,
+      "shuffleFinished" : 1240336972182,
+      "sortFinished" : 1240336972682,
+      "attemptID" : "attempt_200904211745_0004_r_000022_0",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : 1502055,
+      "fileBytesRead" : 2608484,
+      "fileBytesWritten" : 2608484,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : 129784,
+      "reduceInputRecords" : 132724,
+      "reduceShuffleBytes" : 4414804,
+      "reduceOutputRecords" : 132724,
+      "spilledRecords" : 132724,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ ],
+    "startTime" : 1240336889179,
+    "finishTime" : 1240337001263,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_r_000022",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "REDUCE"
+  }, {
+    "attempts" : [ {
+      "location" : null,
+      "hostName" : "/192\\.30\\.127\\.64/cluster1454\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336890222,
+      "finishTime" : 1240336986523,
+      "shuffleFinished" : 1240336973420,
+      "sortFinished" : 1240336973931,
+      "attemptID" : "attempt_200904211745_0004_r_000023_0",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : 1468760,
+      "fileBytesRead" : 2543252,
+      "fileBytesWritten" : 2543252,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : 126859,
+      "reduceInputRecords" : 129871,
+      "reduceShuffleBytes" : 4296232,
+      "reduceOutputRecords" : 129871,
+      "spilledRecords" : 129871,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ ],
+    "startTime" : 1240336889185,
+    "finishTime" : 1240337001262,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_r_000023",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "REDUCE"
+  }, {
+    "attempts" : [ {
+      "location" : null,
+      "hostName" : "/192\\.30\\.126\\.0/cluster1305\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336890266,
+      "finishTime" : 1240336986394,
+      "shuffleFinished" : 1240336975908,
+      "sortFinished" : 1240336976433,
+      "attemptID" : "attempt_200904211745_0004_r_000024_0",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : 1527960,
+      "fileBytesRead" : 2649290,
+      "fileBytesWritten" : 2649290,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : 133015,
+      "reduceInputRecords" : 136281,
+      "reduceShuffleBytes" : 4485423,
+      "reduceOutputRecords" : 136281,
+      "spilledRecords" : 136281,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ ],
+    "startTime" : 1240336889196,
+    "finishTime" : 1240337001291,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_r_000024",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "REDUCE"
+  }, {
+    "attempts" : [ ],
+    "preferredLocations" : [ ],
+    "startTime" : 1240336889205,
+    "finishTime" : -1,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_r_000025",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : null,
+    "taskType" : "REDUCE"
+  }, {
+    "attempts" : [ {
+      "location" : null,
+      "hostName" : "/192\\.30\\.116\\.192/cluster1150\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336889529,
+      "finishTime" : 1240336986187,
+      "shuffleFinished" : 1240336973131,
+      "sortFinished" : 1240336973659,
+      "attemptID" : "attempt_200904211745_0004_r_000026_0",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : 1482330,
+      "fileBytesRead" : 2569201,
+      "fileBytesWritten" : 2569201,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : 125873,
+      "reduceInputRecords" : 129080,
+      "reduceShuffleBytes" : 4363921,
+      "reduceOutputRecords" : 129080,
+      "spilledRecords" : 129080,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ ],
+    "startTime" : 1240336889208,
+    "finishTime" : 1240337001309,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_r_000026",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "REDUCE"
+  }, {
+    "attempts" : [ {
+      "location" : null,
+      "hostName" : "/194\\.6\\.128\\.192/cluster50511\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336890681,
+      "finishTime" : 1240336986867,
+      "shuffleFinished" : 1240336975793,
+      "sortFinished" : 1240336976337,
+      "attemptID" : "attempt_200904211745_0004_r_000027_0",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : 1443497,
+      "fileBytesRead" : 2503238,
+      "fileBytesWritten" : 2503238,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : 122459,
+      "reduceInputRecords" : 125626,
+      "reduceShuffleBytes" : 4238108,
+      "reduceOutputRecords" : 125626,
+      "spilledRecords" : 125626,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ ],
+    "startTime" : 1240336889209,
+    "finishTime" : 1240337001366,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_r_000027",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "REDUCE"
+  }, {
+    "attempts" : [ {
+      "location" : null,
+      "hostName" : "/194\\.6\\.133\\.128/cluster50189\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336889839,
+      "finishTime" : 1240336986369,
+      "shuffleFinished" : 1240336976232,
+      "sortFinished" : 1240336976742,
+      "attemptID" : "attempt_200904211745_0004_r_000028_0",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : 1495322,
+      "fileBytesRead" : 2590588,
+      "fileBytesWritten" : 2590588,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : 0,
+      "reduceInputGroups" : 126998,
+      "reduceInputRecords" : 130037,
+      "reduceShuffleBytes" : 4405490,
+      "reduceOutputRecords" : 130037,
+      "spilledRecords" : 130037,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ ],
+    "startTime" : 1240336889210,
+    "finishTime" : -1,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_r_000028",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : null,
+    "taskType" : "REDUCE"
+  }, {
+    "attempts" : [ ],
+    "preferredLocations" : [ ],
+    "startTime" : 1240336889318,
+    "finishTime" : -1,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_r_000029",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : null,
+    "taskType" : "REDUCE"
+  }, {
+    "attempts" : [ ],
+    "preferredLocations" : [ ],
+    "startTime" : 1240336889364,
+    "finishTime" : -1,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_r_000030",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : null,
+    "taskType" : "REDUCE"
+  }, {
+    "attempts" : [ ],
+    "preferredLocations" : [ ],
+    "startTime" : 1240336889419,
+    "finishTime" : -1,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_r_000031",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : null,
+    "taskType" : "REDUCE"
+  }, {
+    "attempts" : [ ],
+    "preferredLocations" : [ ],
+    "startTime" : 1240336889425,
+    "finishTime" : -1,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_r_000032",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : null,
+    "taskType" : "REDUCE"
+  }, {
+    "attempts" : [ ],
+    "preferredLocations" : [ ],
+    "startTime" : 1240336889426,
+    "finishTime" : -1,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_r_000033",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : null,
+    "taskType" : "REDUCE"
+  }, {
+    "attempts" : [ ],
+    "preferredLocations" : [ ],
+    "startTime" : 1240336889427,
+    "finishTime" : -1,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_r_000034",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : null,
+    "taskType" : "REDUCE"
+  }, {
+    "attempts" : [ ],
+    "preferredLocations" : [ ],
+    "startTime" : 1240336889429,
+    "finishTime" : -1,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_r_000035",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : null,
+    "taskType" : "REDUCE"
+  }, {
+    "attempts" : [ ],
+    "preferredLocations" : [ ],
+    "startTime" : 1240336889429,
+    "finishTime" : -1,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_r_000036",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : null,
+    "taskType" : "REDUCE"
+  }, {
+    "attempts" : [ ],
+    "preferredLocations" : [ ],
+    "startTime" : 1240336889430,
+    "finishTime" : -1,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_r_000037",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : null,
+    "taskType" : "REDUCE"
+  }, {
+    "attempts" : [ ],
+    "preferredLocations" : [ ],
+    "startTime" : 1240336889430,
+    "finishTime" : -1,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_r_000038",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : null,
+    "taskType" : "REDUCE"
+  }, {
+    "attempts" : [ ],
+    "preferredLocations" : [ ],
+    "startTime" : 1240336889431,
+    "finishTime" : -1,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_r_000039",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : null,
+    "taskType" : "REDUCE"
+  }, {
+    "attempts" : [ ],
+    "preferredLocations" : [ ],
+    "startTime" : 1240336889431,
+    "finishTime" : -1,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_r_000040",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : null,
+    "taskType" : "REDUCE"
+  }, {
+    "attempts" : [ ],
+    "preferredLocations" : [ ],
+    "startTime" : 1240336889432,
+    "finishTime" : -1,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_r_000041",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : null,
+    "taskType" : "REDUCE"
+  }, {
+    "attempts" : [ ],
+    "preferredLocations" : [ ],
+    "startTime" : 1240336889432,
+    "finishTime" : -1,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_r_000042",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : null,
+    "taskType" : "REDUCE"
+  }, {
+    "attempts" : [ ],
+    "preferredLocations" : [ ],
+    "startTime" : 1240336889433,
+    "finishTime" : -1,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_r_000043",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : null,
+    "taskType" : "REDUCE"
+  }, {
+    "attempts" : [ ],
+    "preferredLocations" : [ ],
+    "startTime" : 1240336889433,
+    "finishTime" : -1,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_r_000044",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : null,
+    "taskType" : "REDUCE"
+  }, {
+    "attempts" : [ ],
+    "preferredLocations" : [ ],
+    "startTime" : 1240336889434,
+    "finishTime" : -1,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_r_000045",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : null,
+    "taskType" : "REDUCE"
+  }, {
+    "attempts" : [ ],
+    "preferredLocations" : [ ],
+    "startTime" : 1240336889434,
+    "finishTime" : -1,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_r_000046",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : null,
+    "taskType" : "REDUCE"
+  } ],
+  "otherTasks" : [ {
+    "attempts" : [ {
+      "location" : {
+        "layers" : [ "194\\.6\\.134\\.64", "cluster50316\\.secondleveldomain\\.com" ]
+      },
+      "hostName" : "/194\\.6\\.134\\.64/cluster50316\\.secondleveldomain\\.com",
+      "result" : "SUCCESS",
+      "startTime" : 1240336856225,
+      "finishTime" : 1240336858461,
+      "shuffleFinished" : -1,
+      "sortFinished" : -1,
+      "attemptID" : "attempt_200904211745_0004_m_000132_0",
+      "hdfsBytesRead" : -1,
+      "hdfsBytesWritten" : -1,
+      "fileBytesRead" : -1,
+      "fileBytesWritten" : -1,
+      "mapInputRecords" : -1,
+      "mapOutputBytes" : -1,
+      "mapOutputRecords" : -1,
+      "combineInputRecords" : -1,
+      "reduceInputGroups" : -1,
+      "reduceInputRecords" : -1,
+      "reduceShuffleBytes" : -1,
+      "reduceOutputRecords" : -1,
+      "spilledRecords" : 0,
+      "mapInputBytes" : -1
+    } ],
+    "preferredLocations" : [ ],
+    "startTime" : 1240336855651,
+    "finishTime" : 1240336871747,
+    "inputBytes" : -1,
+    "inputRecords" : -1,
+    "outputBytes" : -1,
+    "outputRecords" : -1,
+    "taskID" : "task_200904211745_0004_m_000132",
+    "numberMaps" : -1,
+    "numberReduces" : -1,
+    "taskStatus" : "SUCCESS",
+    "taskType" : "SETUP"
+  } ],
+  "finishTime" : -1,
+  "user" : "geek3",
+  "jobName" : null,
+  "computonsPerMapInputByte" : -1,
+  "computonsPerMapOutputByte" : -1,
+  "computonsPerReduceInputByte" : -1,
+  "computonsPerReduceOutputByte" : -1,
+  "submitTime" : 1240336853354,
+  "launchTime" : 1240336854289,
+  "heapMegabytes" : 1234,
+  "totalMaps" : 131,
+  "totalReduces" : 47,
+  "outcome" : "SUCCESS",
+  "jobtype" : "JAVA",
+  "directDependantJobs" : [ ],
+  "successfulMapAttemptCDFs" : [ {
+    "maximum" : 43182,
+    "minimum" : 10721,
+    "rankings" : [ {
+      "relativeRanking" : 0.05,
+      "datum" : 10721
+    }, {
+      "relativeRanking" : 0.1,
+      "datum" : 10721
+    }, {
+      "relativeRanking" : 0.15,
+      "datum" : 11676
+    }, {
+      "relativeRanking" : 0.2,
+      "datum" : 11676
+    }, {
+      "relativeRanking" : 0.25,
+      "datum" : 12936
+    }, {
+      "relativeRanking" : 0.3,
+      "datum" : 15451
+    }, {
+      "relativeRanking" : 0.35,
+      "datum" : 15451
+    }, {
+      "relativeRanking" : 0.4,
+      "datum" : 19204
+    }, {
+      "relativeRanking" : 0.45,
+      "datum" : 21585
+    }, {
+      "relativeRanking" : 0.5,
+      "datum" : 23169
+    }, {
+      "relativeRanking" : 0.55,
+      "datum" : 23169
+    }, {
+      "relativeRanking" : 0.6,
+      "datum" : 23595
+    }, {
+      "relativeRanking" : 0.65,
+      "datum" : 27355
+    }, {
+      "relativeRanking" : 0.7,
+      "datum" : 27355
+    }, {
+      "relativeRanking" : 0.75,
+      "datum" : 36581
+    }, {
+      "relativeRanking" : 0.8,
+      "datum" : 37035
+    }, {
+      "relativeRanking" : 0.85,
+      "datum" : 37035
+    }, {
+      "relativeRanking" : 0.9,
+      "datum" : 38983
+    }, {
+      "relativeRanking" : 0.95,
+      "datum" : 39264
+    } ],
+    "numberValues" : 14
+  }, {
+    "maximum" : 75401,
+    "minimum" : 9950,
+    "rankings" : [ {
+      "relativeRanking" : 0.05,
+      "datum" : 11004
+    }, {
+      "relativeRanking" : 0.1,
+      "datum" : 12016
+    }, {
+      "relativeRanking" : 0.15,
+      "datum" : 13160
+    }, {
+      "relativeRanking" : 0.2,
+      "datum" : 18594
+    }, {
+      "relativeRanking" : 0.25,
+      "datum" : 21555
+    }, {
+      "relativeRanking" : 0.3,
+      "datum" : 22014
+    }, {
+      "relativeRanking" : 0.35,
+      "datum" : 23004
+    }, {
+      "relativeRanking" : 0.4,
+      "datum" : 24102
+    }, {
+      "relativeRanking" : 0.45,
+      "datum" : 27100
+    }, {
+      "relativeRanking" : 0.5,
+      "datum" : 32357
+    }, {
+      "relativeRanking" : 0.55,
+      "datum" : 34735
+    }, {
+      "relativeRanking" : 0.6,
+      "datum" : 37787
+    }, {
+      "relativeRanking" : 0.65,
+      "datum" : 39211
+    }, {
+      "relativeRanking" : 0.7,
+      "datum" : 39508
+    }, {
+      "relativeRanking" : 0.75,
+      "datum" : 40173
+    }, {
+      "relativeRanking" : 0.8,
+      "datum" : 40512
+    }, {
+      "relativeRanking" : 0.85,
+      "datum" : 43173
+    }, {
+      "relativeRanking" : 0.9,
+      "datum" : 48444
+    }, {
+      "relativeRanking" : 0.95,
+      "datum" : 56688
+    } ],
+    "numberValues" : 92
+  }, {
+    "maximum" : 49391,
+    "minimum" : 9945,
+    "rankings" : [ {
+      "relativeRanking" : 0.05,
+      "datum" : 9945
+    }, {
+      "relativeRanking" : 0.1,
+      "datum" : 10537
+    }, {
+      "relativeRanking" : 0.15,
+      "datum" : 10748
+    }, {
+      "relativeRanking" : 0.2,
+      "datum" : 11007
+    }, {
+      "relativeRanking" : 0.25,
+      "datum" : 11929
+    }, {
+      "relativeRanking" : 0.3,
+      "datum" : 12330
+    }, {
+      "relativeRanking" : 0.35,
+      "datum" : 13151
+    }, {
+      "relativeRanking" : 0.4,
+      "datum" : 13826
+    }, {
+      "relativeRanking" : 0.45,
+      "datum" : 14120
+    }, {
+      "relativeRanking" : 0.5,
+      "datum" : 15115
+    }, {
+      "relativeRanking" : 0.55,
+      "datum" : 17485
+    }, {
+      "relativeRanking" : 0.6,
+      "datum" : 26817
+    }, {
+      "relativeRanking" : 0.65,
+      "datum" : 28878
+    }, {
+      "relativeRanking" : 0.7,
+      "datum" : 32445
+    }, {
+      "relativeRanking" : 0.75,
+      "datum" : 34820
+    }, {
+      "relativeRanking" : 0.8,
+      "datum" : 36092
+    }, {
+      "relativeRanking" : 0.85,
+      "datum" : 36563
+    }, {
+      "relativeRanking" : 0.9,
+      "datum" : 37231
+    }, {
+      "relativeRanking" : 0.95,
+      "datum" : 39051
+    } ],
+    "numberValues" : 25
+  }, {
+    "maximum" : 2236,
+    "minimum" : 2236,
+    "rankings" : [ {
+      "relativeRanking" : 0.05,
+      "datum" : 2236
+    }, {
+      "relativeRanking" : 0.1,
+      "datum" : 2236
+    }, {
+      "relativeRanking" : 0.15,
+      "datum" : 2236
+    }, {
+      "relativeRanking" : 0.2,
+      "datum" : 2236
+    }, {
+      "relativeRanking" : 0.25,
+      "datum" : 2236
+    }, {
+      "relativeRanking" : 0.3,
+      "datum" : 2236
+    }, {
+      "relativeRanking" : 0.35,
+      "datum" : 2236
+    }, {
+      "relativeRanking" : 0.4,
+      "datum" : 2236
+    }, {
+      "relativeRanking" : 0.45,
+      "datum" : 2236
+    }, {
+      "relativeRanking" : 0.5,
+      "datum" : 2236
+    }, {
+      "relativeRanking" : 0.55,
+      "datum" : 2236
+    }, {
+      "relativeRanking" : 0.6,
+      "datum" : 2236
+    }, {
+      "relativeRanking" : 0.65,
+      "datum" : 2236
+    }, {
+      "relativeRanking" : 0.7,
+      "datum" : 2236
+    }, {
+      "relativeRanking" : 0.75,
+      "datum" : 2236
+    }, {
+      "relativeRanking" : 0.8,
+      "datum" : 2236
+    }, {
+      "relativeRanking" : 0.85,
+      "datum" : 2236
+    }, {
+      "relativeRanking" : 0.9,
+      "datum" : 2236
+    }, {
+      "relativeRanking" : 0.95,
+      "datum" : 2236
+    } ],
+    "numberValues" : 1
+  } ],
+  "failedMapAttemptCDFs" : [ {
+    "maximum" : 9223372036854775807,
+    "minimum" : -9223372036854775808,
+    "rankings" : [ ],
+    "numberValues" : 0
+  }, {
+    "maximum" : 9223372036854775807,
+    "minimum" : -9223372036854775808,
+    "rankings" : [ ],
+    "numberValues" : 0
+  }, {
+    "maximum" : 9223372036854775807,
+    "minimum" : -9223372036854775808,
+    "rankings" : [ ],
+    "numberValues" : 0
+  }, {
+    "maximum" : 18592,
+    "minimum" : 18416,
+    "rankings" : [ {
+      "relativeRanking" : 0.05,
+      "datum" : 18416
+    }, {
+      "relativeRanking" : 0.1,
+      "datum" : 18416
+    }, {
+      "relativeRanking" : 0.15,
+      "datum" : 18416
+    }, {
+      "relativeRanking" : 0.2,
+      "datum" : 18416
+    }, {
+      "relativeRanking" : 0.25,
+      "datum" : 18416
+    }, {
+      "relativeRanking" : 0.3,
+      "datum" : 18416
+    }, {
+      "relativeRanking" : 0.35,
+      "datum" : 18416
+    }, {
+      "relativeRanking" : 0.4,
+      "datum" : 18416
+    }, {
+      "relativeRanking" : 0.45,
+      "datum" : 18416
+    }, {
+      "relativeRanking" : 0.5,
+      "datum" : 18416
+    }, {
+      "relativeRanking" : 0.55,
+      "datum" : 18416
+    }, {
+      "relativeRanking" : 0.6,
+      "datum" : 18416
+    }, {
+      "relativeRanking" : 0.65,
+      "datum" : 18416
+    }, {
+      "relativeRanking" : 0.7,
+      "datum" : 18584
+    }, {
+      "relativeRanking" : 0.75,
+      "datum" : 18584
+    }, {
+      "relativeRanking" : 0.8,
+      "datum" : 18584
+    }, {
+      "relativeRanking" : 0.85,
+      "datum" : 18584
+    }, {
+      "relativeRanking" : 0.9,
+      "datum" : 18584
+    }, {
+      "relativeRanking" : 0.95,
+      "datum" : 18584
+    } ],
+    "numberValues" : 3
+  } ],
+  "successfulReduceAttemptCDF" : {
+    "maximum" : 96910,
+    "minimum" : 95583,
+    "rankings" : [ {
+      "relativeRanking" : 0.05,
+      "datum" : 95583
+    }, {
+      "relativeRanking" : 0.1,
+      "datum" : 95751
+    }, {
+      "relativeRanking" : 0.15,
+      "datum" : 96010
+    }, {
+      "relativeRanking" : 0.2,
+      "datum" : 96013
+    }, {
+      "relativeRanking" : 0.25,
+      "datum" : 96080
+    }, {
+      "relativeRanking" : 0.3,
+      "datum" : 96128
+    }, {
+      "relativeRanking" : 0.35,
+      "datum" : 96164
+    }, {
+      "relativeRanking" : 0.4,
+      "datum" : 96227
+    }, {
+      "relativeRanking" : 0.45,
+      "datum" : 96275
+    }, {
+      "relativeRanking" : 0.5,
+      "datum" : 96290
+    }, {
+      "relativeRanking" : 0.55,
+      "datum" : 96301
+    }, {
+      "relativeRanking" : 0.6,
+      "datum" : 96320
+    }, {
+      "relativeRanking" : 0.65,
+      "datum" : 96346
+    }, {
+      "relativeRanking" : 0.7,
+      "datum" : 96412
+    }, {
+      "relativeRanking" : 0.75,
+      "datum" : 96428
+    }, {
+      "relativeRanking" : 0.8,
+      "datum" : 96530
+    }, {
+      "relativeRanking" : 0.85,
+      "datum" : 96549
+    }, {
+      "relativeRanking" : 0.9,
+      "datum" : 96658
+    }, {
+      "relativeRanking" : 0.95,
+      "datum" : 96670
+    } ],
+    "numberValues" : 28
+  },
+  "failedReduceAttemptCDF" : {
+    "maximum" : 9223372036854775807,
+    "minimum" : -9223372036854775808,
+    "rankings" : [ ],
+    "numberValues" : 0
+  },
+  "mapperTriesToSucceed" : [ 0.6567164179104478, 0.3283582089552239, 0.014925373134328358 ],
+  "failedMapperFraction" : 0.0,
+  "relativeTime" : 0,
+  "queue" : null,
+  "clusterMapMB" : -1,
+  "clusterReduceMB" : -1,
+  "jobMapMB" : -1,
+  "jobReduceMB" : -1
+}

+ 185 - 0
src/tools/org/apache/hadoop/tools/rumen/AbstractClusterStory.java

@@ -0,0 +1,185 @@
+/**
+ * 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.util.ArrayDeque;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Random;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
+
+/**
+ * {@link AbstractClusterStory} provides a partial implementation of
+ * {@link ClusterStory} by parsing the topology tree.
+ */
+public abstract class AbstractClusterStory implements ClusterStory {
+  protected Set<MachineNode> machineNodes;
+  protected Set<RackNode> rackNodes;
+  protected MachineNode[] mNodesFlattened;
+  protected Map<String, MachineNode> mNodeMap;
+  protected Map<String, RackNode> rNodeMap;
+  protected int maximumDistance = 0;
+  protected Random random;
+  
+  @Override
+  public Set<MachineNode> getMachines() {
+    parseTopologyTree();
+    return machineNodes;
+  }
+  
+  @Override
+  public synchronized Set<RackNode> getRacks() {
+    parseTopologyTree();    
+    return rackNodes;
+  }
+  
+  @Override
+  public synchronized MachineNode[] getRandomMachines(int expected) {
+    if (expected == 0) {
+      return new MachineNode[0];
+    }
+
+    parseTopologyTree();
+    int total = machineNodes.size();
+    int select = Math.min(expected, total);
+
+    if (mNodesFlattened == null) {
+      mNodesFlattened = machineNodes.toArray(new MachineNode[total]);
+      random = new Random();
+    }
+
+    MachineNode[] retval = new MachineNode[select];
+    int i = 0;
+    while ((i != select) && (total != i + select)) {
+      int index = random.nextInt(total - i);
+      MachineNode tmp = mNodesFlattened[index];
+      mNodesFlattened[index] = mNodesFlattened[total - i - 1];
+      mNodesFlattened[total - i - 1] = tmp;
+      ++i;
+    }
+    if (i == select) {
+      System.arraycopy(mNodesFlattened, total - i, retval, 0, select);
+    } else {
+      System.arraycopy(mNodesFlattened, 0, retval, 0, select);
+    }
+
+    return retval;
+  }
+  
+  protected synchronized void buildMachineNodeMap() {
+    if (mNodeMap == null) {
+      mNodeMap = new HashMap<String, MachineNode>(machineNodes.size());
+      for (MachineNode mn : machineNodes) {
+        mNodeMap.put(mn.getName(), mn);
+      }
+    }
+  }
+  
+  @Override
+  public MachineNode getMachineByName(String name) {
+    buildMachineNodeMap();
+    return mNodeMap.get(name);
+  }
+  
+  @Override
+  public int distance(Node a, Node b) {
+    int lvl_a = a.getLevel();
+    int lvl_b = b.getLevel();
+    int retval = 0;
+    if (lvl_a > lvl_b) {
+      retval = lvl_a-lvl_b;
+      for (int i=0; i<retval; ++i) {
+        a = a.getParent();
+      }
+    } else if (lvl_a < lvl_b) {
+      retval = lvl_b-lvl_a;
+      for (int i=0; i<retval; ++i) {
+        b = b.getParent();
+      }      
+    }
+    
+    while (a != b) {
+      a = a.getParent();
+      b = b.getParent();
+      ++retval;
+    }
+    
+    return retval;
+  }
+  
+  protected synchronized void buildRackNodeMap() {
+    if (rNodeMap == null) {
+      rNodeMap = new HashMap<String, RackNode>(rackNodes.size());
+      for (RackNode rn : rackNodes) {
+        rNodeMap.put(rn.getName(), rn);
+      }
+    }
+  }
+  
+  @Override
+  public RackNode getRackByName(String name) {
+    buildRackNodeMap();
+    return rNodeMap.get(name);
+  }
+  
+  @Override
+  public int getMaximumDistance() {
+    parseTopologyTree();
+    return maximumDistance;
+  }
+  
+  protected synchronized void parseTopologyTree() {
+    if (machineNodes == null) {
+      Node root = getClusterTopology();
+      SortedSet<MachineNode> mNodes = new TreeSet<MachineNode>();
+      SortedSet<RackNode> rNodes = new TreeSet<RackNode>();
+      // dfs search of the tree.
+      Deque<Node> unvisited = new ArrayDeque<Node>();
+      Deque<Integer> distUnvisited = new ArrayDeque<Integer>();
+      unvisited.add(root);
+      distUnvisited.add(0);
+      for (Node n = unvisited.poll(); n != null; n = unvisited.poll()) {
+        int distance = distUnvisited.poll();
+        if (n instanceof RackNode) {
+          rNodes.add((RackNode) n);
+          mNodes.addAll(((RackNode) n).getMachinesInRack());
+          if (distance + 1 > maximumDistance) {
+            maximumDistance = distance + 1;
+          }
+        } else if (n instanceof MachineNode) {
+          mNodes.add((MachineNode) n);
+          if (distance > maximumDistance) {
+            maximumDistance = distance;
+          }
+        } else {
+          for (Node child : n.getChildren()) {
+            unvisited.addFirst(child);
+            distUnvisited.addFirst(distance+1);
+          }
+        }
+      }
+
+      machineNodes = Collections.unmodifiableSortedSet(mNodes);
+      rackNodes = Collections.unmodifiableSortedSet(rNodes);
+    }
+  }
+}

+ 68 - 0
src/tools/org/apache/hadoop/tools/rumen/CDFPiecewiseLinearRandomGenerator.java

@@ -0,0 +1,68 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.tools.rumen;
+
+public class CDFPiecewiseLinearRandomGenerator extends CDFRandomGenerator {
+
+  /**
+   * @param cdf
+   *          builds a CDFRandomValue engine around this
+   *          {@link LoggedDiscreteCDF}, with a defaultly seeded RNG
+   */
+  public CDFPiecewiseLinearRandomGenerator(LoggedDiscreteCDF cdf) {
+    super(cdf);
+  }
+
+  /**
+   * @param cdf
+   *          builds a CDFRandomValue engine around this
+   *          {@link LoggedDiscreteCDF}, with an explicitly seeded RNG
+   * @param seed
+   *          the random number generator seed
+   */
+  public CDFPiecewiseLinearRandomGenerator(LoggedDiscreteCDF cdf, long seed) {
+    super(cdf, seed);
+  }
+
+  /**
+   * TODO This code assumes that the empirical minimum resp. maximum is the
+   * epistomological minimum resp. maximum. This is probably okay for the
+   * minimum, because that likely represents a task where everything went well,
+   * but for the maximum we may want to develop a way of extrapolating past the
+   * maximum.
+   */
+  @Override
+  public long valueAt(double probability) {
+    int rangeFloor = floorIndex(probability);
+
+    double segmentProbMin = getRankingAt(rangeFloor);
+    double segmentProbMax = getRankingAt(rangeFloor + 1);
+
+    long segmentMinValue = getDatumAt(rangeFloor);
+    long segmentMaxValue = getDatumAt(rangeFloor + 1);
+
+    // If this is zero, this object is based on an ill-formed cdf
+    double segmentProbRange = segmentProbMax - segmentProbMin;
+    long segmentDatumRange = segmentMaxValue - segmentMinValue;
+
+    long result = (long) ((probability - segmentProbMin) / segmentProbRange * segmentDatumRange)
+        + segmentMinValue;
+
+    return result;
+  }
+}

+ 89 - 0
src/tools/org/apache/hadoop/tools/rumen/CDFRandomGenerator.java

@@ -0,0 +1,89 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ * 
+ */
+package org.apache.hadoop.tools.rumen;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Random;
+
+/**
+ * An instance of this class generates random values that confirm to the
+ * embedded {@link LoggedDiscreteCDF} . The discrete CDF is a pointwise
+ * approximation of the "real" CDF. We therefore have a choice of interpolation
+ * rules.
+ * 
+ * A concrete subclass of this abstract class will implement valueAt(double)
+ * using a class-dependent interpolation rule.
+ * 
+ */
+public abstract class CDFRandomGenerator {
+  final double[] rankings;
+  final long[] values;
+
+  final Random random;
+
+  CDFRandomGenerator(LoggedDiscreteCDF cdf) {
+    this(cdf, new Random());
+  }
+
+  CDFRandomGenerator(LoggedDiscreteCDF cdf, long seed) {
+    this(cdf, new Random(seed));
+  }
+
+  private CDFRandomGenerator(LoggedDiscreteCDF cdf, Random random) {
+    this.random = random;
+    rankings = new double[cdf.getRankings().size() + 2];
+    values = new long[cdf.getRankings().size() + 2];
+    initializeTables(cdf);
+  }
+
+  protected final void initializeTables(LoggedDiscreteCDF cdf) {
+    rankings[0] = 0.0;
+    values[0] = cdf.getMinimum();
+    rankings[rankings.length - 1] = 1.0;
+    values[rankings.length - 1] = cdf.getMaximum();
+
+    List<LoggedSingleRelativeRanking> subjects = cdf.getRankings();
+
+    for (int i = 0; i < subjects.size(); ++i) {
+      rankings[i + 1] = subjects.get(i).getRelativeRanking();
+      values[i + 1] = subjects.get(i).getDatum();
+    }
+  }
+
+  protected int floorIndex(double probe) {
+    int result = Arrays.binarySearch(rankings, probe);
+
+    return Math.abs(result + 1) - 1;
+  }
+
+  protected double getRankingAt(int index) {
+    return rankings[index];
+  }
+
+  protected long getDatumAt(int index) {
+    return values[index];
+  }
+
+  public long randomValue() {
+    return valueAt(random.nextDouble());
+  }
+
+  public abstract long valueAt(double probability);
+}

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

@@ -0,0 +1,81 @@
+/**
+ * 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.util.Set;
+
+/**
+ * {@link ClusterStory} represents all configurations of a MapReduce cluster,
+ * including nodes, network topology, and slot configurations.
+ */
+public interface ClusterStory {
+  /**
+   * Get all machines of the cluster.
+   * @return A read-only set that contains all machines of the cluster.
+   */
+  public Set<MachineNode> getMachines();
+
+  /**
+   * Get all racks of the cluster.
+   * @return A read-only set that contains all racks of the cluster.
+   */
+  public Set<RackNode> getRacks();
+  
+  /**
+   * Get the cluster topology tree.
+   * @return The root node of the cluster topology tree.
+   */
+  public Node getClusterTopology();
+  
+  /**
+   * Select a random set of machines.
+   * @param expected The expected sample size.
+   * @return An array of up to expected number of {@link MachineNode}s.
+   */
+  public MachineNode[] getRandomMachines(int expected);
+
+  /**
+   * Get {@link MachineNode} by its host name.
+   * 
+   * @return The {@line 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.
+   */
+  public RackNode getRackByName(String name);
+
+  /**
+   * Determine the distance between two {@link Node}s. Currently, the distance
+   * is loosely defined as the length of the longer path for either a or b to
+   * reach their common ancestor.
+   * 
+   * @param a
+   * @param b
+   * @return The distance between {@link Node} a and {@link Node} b.
+   */
+  int distance(Node a, Node b);
+  
+  /**
+   * Get the maximum distance possible between any two nodes.
+   * @return the maximum distance possible between any two nodes.
+   */
+  int getMaximumDistance();
+}

+ 81 - 0
src/tools/org/apache/hadoop/tools/rumen/ClusterTopologyReader.java

@@ -0,0 +1,81 @@
+/**
+ * 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.IOException;
+import java.io.InputStream;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+
+/**
+ * Reading JSON-encoded cluster topology and produce the parsed
+ * {@link LoggedNetworkTopology} object.
+ */
+public class ClusterTopologyReader {
+  private LoggedNetworkTopology topology;
+
+  private void readTopology(JsonObjectMapperParser<LoggedNetworkTopology> parser)
+      throws IOException {
+    try {
+      topology = parser.getNext();
+      if (topology == null) {
+        throw new IOException(
+            "Input file does not contain valid topology data.");
+      }
+    } finally {
+      parser.close();
+    }
+  }
+
+  /**
+   * Constructor.
+   * 
+   * @param path
+   *          Path to the JSON-encoded topology file, possibly compressed.
+   * @param conf
+   * @throws IOException
+   */
+  public ClusterTopologyReader(Path path, Configuration conf)
+      throws IOException {
+    JsonObjectMapperParser<LoggedNetworkTopology> parser = new JsonObjectMapperParser<LoggedNetworkTopology>(
+        path, LoggedNetworkTopology.class, conf);
+    readTopology(parser);
+  }
+
+  /**
+   * Constructor.
+   * 
+   * @param input
+   *          The input stream for the JSON-encoded topology data.
+   */
+  public ClusterTopologyReader(InputStream input) throws IOException {
+    JsonObjectMapperParser<LoggedNetworkTopology> parser = new JsonObjectMapperParser<LoggedNetworkTopology>(
+        input, LoggedNetworkTopology.class);
+    readTopology(parser);
+  }
+
+  /**
+   * Get the {@link LoggedNetworkTopology} object.
+   * 
+   * @return The {@link LoggedNetworkTopology} object parsed from the input.
+   */
+  public LoggedNetworkTopology get() {
+    return topology;
+  }
+}

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

@@ -0,0 +1,45 @@
+/**
+ * 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;
+
+/**
+ * Classes that implement this interface can deep-compare [for equality only,
+ * not order] with another instance. They do a deep compare. If there is any
+ * semantically significant difference, an implementer throws an Exception to be
+ * thrown with a chain of causes describing the chain of field references and
+ * indices that get you to the miscompared point.
+ * 
+ */
+public interface DeepCompare {
+  /**
+   * @param other
+   *          the other comparand that's being compared to me
+   * @param myLocation
+   *          the path that got to me. In the root, myLocation is null. To
+   *          process the scalar {@code foo} field of the root we will make a
+   *          recursive call with a {@link TreePath} whose {@code fieldName} is
+   *          {@code "bar"} and whose {@code index} is -1 and whose {@code
+   *          parent} is {@code null}. To process the plural {@code bar} field
+   *          of the root we will make a recursive call with a {@link TreePath}
+   *          whose fieldName is {@code "foo"} and whose {@code index} is -1 and
+   *          whose {@code parent} is also {@code null}.
+   * @throws DeepInequalityException
+   */
+  public void deepCompare(DeepCompare other, TreePath myLocation)
+      throws DeepInequalityException;
+}

+ 61 - 0
src/tools/org/apache/hadoop/tools/rumen/DeepInequalityException.java

@@ -0,0 +1,61 @@
+/**
+ * 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;
+
+/**
+ * We use this exception class in the unit test, and we do a deep comparison
+ * when we run the
+ * 
+ */
+public class DeepInequalityException extends Exception {
+
+  static final long serialVersionUID = 1352469876;
+
+  final TreePath path;
+
+  /**
+   * @param message
+   *          an exception message
+   * @param path
+   *          the path that gets from the root to the inequality
+   * 
+   *          This is the constructor that I intend to have used for this
+   *          exception.
+   */
+  public DeepInequalityException(String message, TreePath path,
+      Throwable chainee) {
+    super(message, chainee);
+
+    this.path = path;
+  }
+
+  /**
+   * @param message
+   *          an exception message
+   * @param path
+   *          the path that gets from the root to the inequality
+   * 
+   *          This is the constructor that I intend to have used for this
+   *          exception.
+   */
+  public DeepInequalityException(String message, TreePath path) {
+    super(message);
+
+    this.path = path;
+  }
+}

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

@@ -0,0 +1,1814 @@
+/**
+ * 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.FileNotFoundException;
+import java.io.IOException;
+import java.io.EOFException;
+import java.io.PrintStream;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.StringTokenizer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.hadoop.util.LineReader;
+
+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.FSDataInputStream;
+
+import org.apache.hadoop.io.Text;
+
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.io.compress.CompressionCodecFactory;
+import org.apache.hadoop.io.compress.CodecPool;
+import org.apache.hadoop.io.compress.Decompressor;
+
+import org.codehaus.jackson.JsonEncoding;
+import org.codehaus.jackson.JsonFactory;
+import org.codehaus.jackson.JsonGenerator;
+import org.codehaus.jackson.JsonProcessingException;
+import org.codehaus.jackson.map.ObjectMapper;
+import org.codehaus.jackson.map.SerializationConfig;
+
+/**
+ * This is the main class for rumen log mining functionality.
+ * 
+ * It reads a directory of job tracker logs, and computes various information
+ * about it. See {@code usage()}, below.
+ * 
+ */
+public class HadoopLogsAnalyzer extends Configured implements Tool {
+
+  // output streams
+  private PrintStream statusOutput = System.out;
+  private PrintStream statisticalOutput = System.out;
+
+  private static PrintStream staticDebugOutput = System.err;
+
+  /**
+   * The number of splits a task can have, before we ignore them all.
+   */
+  private final static int MAXIMUM_PREFERRED_LOCATIONS = 25;
+
+  /**
+   * This element is to compensate for the fact that our percentiles engine
+   * rounds up for the expected sample count, so if the total number of readings
+   * is small enough we need to compensate slightly when aggregating the spread
+   * data from jobs with few reducers together with jobs with many reducers.
+   */
+  private static final long SMALL_SPREAD_COMPENSATION_THRESHOLD = 5L;
+
+  /**
+   * {@code MAXIMUM_CLOCK_SKEW} is the maximum plausible difference between the
+   * clocks of machines in the same cluster. This is important because an event
+   * that logically must follow a second event will be considered non-anomalous
+   * if it precedes that second event, provided they happen on different
+   * machines.
+   */
+  private static final long MAXIMUM_CLOCK_SKEW = 10000L;
+
+  /**
+   * The regular expression used to parse task attempt IDs in job tracker logs
+   */
+  private final static Pattern taskAttemptIDPattern = Pattern
+      .compile(".*_([0-9]+)");
+
+  private final static Pattern xmlFilePrefix = Pattern.compile("[ \t]*<");
+
+  private final static Pattern confFileHeader = Pattern.compile("_conf.xml!!");
+
+  private final Map<String, Pattern> counterPatterns = new HashMap<String, Pattern>();
+
+  /**
+   * The unpaired job config file. Currently only used to glean the {@code -Xmx}
+   * field of the JRE options
+   */
+  private ParsedConfigFile jobconf = null;
+
+  /**
+   * Set by {@code -omit-task-details}. If true, we <i>only</i> emit the job
+   * digest [statistical info], not the detailed job trace.
+   */
+  private boolean omitTaskDetails = false;
+
+  private JsonGenerator jobTraceGen = null;
+
+  private boolean prettyprintTrace = true;
+
+  private LoggedJob jobBeingTraced = null;
+
+  private Map<String, LoggedTask> tasksInCurrentJob;
+
+  private Map<String, LoggedTaskAttempt> attemptsInCurrentJob;
+
+  private Histogram[] successfulMapAttemptTimes;
+  private Histogram successfulReduceAttemptTimes;
+  private Histogram[] failedMapAttemptTimes;
+  private Histogram failedReduceAttemptTimes;
+  private Histogram successfulNthMapperAttempts;
+  private Histogram successfulNthReducerAttempts;
+  private Histogram mapperLocality;
+
+  static final private Log LOG = LogFactory.getLog(HadoopLogsAnalyzer.class);
+
+  private int[] attemptTimesPercentiles;
+
+  private JsonGenerator topologyGen = null;
+
+  private HashSet<ParsedHost> allHosts = new HashSet<ParsedHost>();
+
+  // number of ticks per second
+  private boolean collecting = false;
+
+  private long lineNumber = 0;
+
+  private String rereadableLine = null;
+
+  private String inputFilename;
+
+  private boolean inputIsDirectory = false;
+
+  private Path inputDirectoryPath = null;
+  private String[] inputDirectoryFiles = null;
+
+  private int inputDirectoryCursor = -1;
+
+  private LineReader input = null;
+  private CompressionCodec inputCodec = null;
+  private Decompressor inputDecompressor = null;
+  private Text inputLineText = new Text();
+
+  private boolean debug = false;
+
+  private int version = 0;
+
+  private int numberBuckets = 99;
+
+  private int spreadMin;
+
+  private int spreadMax;
+
+  private boolean spreading = false;
+  private boolean delays = false;
+  private boolean runtimes = false;
+
+  private boolean collectTaskTimes = false;
+
+  private LogRecordType canonicalJob = LogRecordType.intern("Job");
+  private LogRecordType canonicalMapAttempt = LogRecordType
+      .intern("MapAttempt");
+  private LogRecordType canonicalReduceAttempt = LogRecordType
+      .intern("ReduceAttempt");
+  private LogRecordType canonicalTask = LogRecordType.intern("Task");
+
+  private static Pattern streamingJobnamePattern = Pattern
+      .compile("streamjob\\d+.jar");
+
+  private HashSet<String> hostNames = new HashSet<String>();
+
+  private boolean fileFirstLine = true;
+  private String currentFileName = null;
+
+  // Here are the cumulative statistics.
+  enum JobOutcome {
+    SUCCESS, FAILURE, OVERALL
+  };
+
+  /**
+   * These rectangular arrays of {@link Histogram}s are indexed by the job type
+   * [java, streaming, pig or pipes] and then by the outcome [success or
+   * failure]
+   */
+  private Histogram runTimeDists[][];
+  private Histogram delayTimeDists[][];
+  private Histogram mapTimeSpreadDists[][];
+  private Histogram shuffleTimeSpreadDists[][];
+  private Histogram sortTimeSpreadDists[][];
+  private Histogram reduceTimeSpreadDists[][];
+
+  private Histogram mapTimeDists[][];
+  private Histogram shuffleTimeDists[][];
+  private Histogram sortTimeDists[][];
+  private Histogram reduceTimeDists[][];
+
+  private Map<String, Long> taskAttemptStartTimes;
+  private Map<String, Long> taskReduceAttemptShuffleEndTimes;
+  private Map<String, Long> taskReduceAttemptSortEndTimes;
+  private Map<String, Long> taskMapAttemptFinishTimes;
+  private Map<String, Long> taskReduceAttemptFinishTimes;
+
+  private long submitTimeCurrentJob;
+  private long launchTimeCurrentJob;
+
+  private String currentJobID;
+
+  // TODO this is currently not being set correctly. We should fix it.
+  // That only matters for statistics extraction.
+  private LoggedJob.JobType thisJobType;
+
+  private Histogram[][] newDistributionBlock() {
+    return newDistributionBlock(null);
+  }
+
+  private Histogram[][] newDistributionBlock(String blockname) {
+    Histogram[][] result = new Histogram[JobOutcome.values().length][];
+
+    for (int i = 0; i < JobOutcome.values().length; ++i) {
+      result[i] = new Histogram[LoggedJob.JobType.values().length];
+
+      for (int j = 0; j < LoggedJob.JobType.values().length; ++j) {
+        result[i][j] = blockname == null ? new Histogram() : new Histogram(
+            blockname);
+      }
+    }
+
+    return result;
+  }
+
+  private Histogram getDistribution(Histogram[][] block, JobOutcome outcome,
+      LoggedJob.JobType type) {
+    return block[outcome.ordinal()][type.ordinal()];
+  }
+
+  private void usage() {
+    statusOutput
+        .print("Usage: \n"
+            + "administrative subcommands:\n"
+            + "-v1                  specify version 1 of the jt logs\n"
+            + "-h or -help          print this message\n"
+            + "-d or -debug         print voluminous debug info during processing\n"
+            + "-collect-prefixes    collect the prefixes of log lines\n\n"
+            + "  job trace subcommands\n"
+            + "-write-job-trace     takes a filename.\n"
+            + "                     writes job trace in JSON to that filename\n"
+            + "-single-line-job-traces  omit prettyprinting of job trace\n"
+            + "-omit-task-details   leave out info about each task and attempt,\n"
+            + "                     so only statistical info is added to each job\n"
+            + "-write-topology      takes a filename.\n"
+            + "                     writes JSON file giving network topology\n"
+            + "-job-digest-spectra  takes a list of percentile points\n"
+            + "                     writes CDFs with min, max, and those percentiles\n\n"
+            + "subcommands for task statistical info\n"
+            + "-spreads             we have a mode where, for each job, we can\n"
+            + "                     develop the ratio of percentile B to percentile A\n"
+            + "                     of task run times.  Having developed that ratio,\n"
+            + "                     we can consider it to be a datum and we can\n"
+            + "                     build a CDF of those ratios.  -spreads turns\n"
+            + "                     this option on, and takes A and B\n"
+            + "-delays              tells us to gather and print CDFs for delays\n"
+            + "                     from job submit to job start\n"
+            + "-runtimes            prints CDFs of job wallclock times [launch\n"
+            + "                     to finish]\n"
+            + "-tasktimes           prints CDFs of job wallclock times [launch\n"
+            + "                     to finish]\n\n");
+  }
+
+  public HadoopLogsAnalyzer() {
+    super();
+  }
+
+  private boolean pathIsDirectory(Path p) throws IOException {
+    FileSystem fs = p.getFileSystem(getConf());
+    return fs.getFileStatus(p).isDir();
+  }
+
+  /**
+   * @param args
+   *          string arguments. See {@code usage()}
+   * @throws FileNotFoundException
+   * @throws IOException
+   */
+  private int initializeHadoopLogsAnalyzer(String[] args)
+      throws FileNotFoundException, IOException {
+    Path jobTraceFilename = null;
+    Path topologyFilename = null;
+    if (args.length == 0 || args[args.length - 1].charAt(0) == '-') {
+      throw new IllegalArgumentException("No input specified.");
+    } else {
+      inputFilename = args[args.length - 1];
+    }
+
+    for (int i = 0; i < args.length - (inputFilename == null ? 0 : 1); ++i) {
+      if ("-h".equals(args[i].toLowerCase())
+          || "-help".equals(args[i].toLowerCase())) {
+        usage();
+        return 0;
+      }
+
+      if ("-c".equals(args[i].toLowerCase())
+          || "-collect-prefixes".equals(args[i].toLowerCase())) {
+        collecting = true;
+        continue;
+      }
+
+      // these control the job digest
+      if ("-write-job-trace".equals(args[i].toLowerCase())) {
+        ++i;
+        jobTraceFilename = new Path(args[i]);
+        continue;
+      }
+
+      if ("-single-line-job-traces".equals(args[i].toLowerCase())) {
+        prettyprintTrace = false;
+        continue;
+      }
+
+      if ("-omit-task-details".equals(args[i].toLowerCase())) {
+        omitTaskDetails = true;
+        continue;
+      }
+
+      if ("-write-topology".equals(args[i].toLowerCase())) {
+        ++i;
+        topologyFilename = new Path(args[i]);
+        continue;
+      }
+
+      if ("-job-digest-spectra".equals(args[i].toLowerCase())) {
+        ArrayList<Integer> values = new ArrayList<Integer>();
+
+        ++i;
+
+        while (i < args.length && Character.isDigit(args[i].charAt(0))) {
+          values.add(Integer.parseInt(args[i]));
+          ++i;
+        }
+
+        if (values.size() == 0) {
+          throw new IllegalArgumentException("Empty -job-digest-spectra list");
+        }
+
+        attemptTimesPercentiles = new int[values.size()];
+
+        int lastValue = 0;
+
+        for (int j = 0; j < attemptTimesPercentiles.length; ++j) {
+          if (values.get(j) <= lastValue || values.get(j) >= 100) {
+            throw new IllegalArgumentException(
+                "Bad -job-digest-spectra percentiles list");
+          }
+          attemptTimesPercentiles[j] = values.get(j);
+        }
+
+        --i;
+        continue;
+      }
+
+      if ("-d".equals(args[i].toLowerCase())
+          || "-debug".equals(args[i].toLowerCase())) {
+        debug = true;
+        continue;
+      }
+
+      if ("-spreads".equals(args[i].toLowerCase())) {
+        int min = Integer.parseInt(args[i + 1]);
+        int max = Integer.parseInt(args[i + 2]);
+
+        if (min < max && min < 1000 && max < 1000) {
+          spreadMin = min;
+          spreadMax = max;
+          spreading = true;
+          i += 2;
+        }
+        continue;
+      }
+
+      // These control log-wide CDF outputs
+      if ("-delays".equals(args[i].toLowerCase())) {
+        delays = true;
+        continue;
+      }
+
+      if ("-runtimes".equals(args[i].toLowerCase())) {
+        runtimes = true;
+        continue;
+      }
+
+      if ("-tasktimes".equals(args[i].toLowerCase())) {
+        collectTaskTimes = true;
+        continue;
+      }
+
+      if ("-v1".equals(args[i].toLowerCase())) {
+        version = 1;
+        continue;
+      }
+
+      throw new IllegalArgumentException("Unrecognized argument: " + args[i]);
+    }
+
+    runTimeDists = newDistributionBlock();
+    delayTimeDists = newDistributionBlock();
+    mapTimeSpreadDists = newDistributionBlock("map-time-spreads");
+    shuffleTimeSpreadDists = newDistributionBlock();
+    sortTimeSpreadDists = newDistributionBlock();
+    reduceTimeSpreadDists = newDistributionBlock();
+
+    mapTimeDists = newDistributionBlock();
+    shuffleTimeDists = newDistributionBlock();
+    sortTimeDists = newDistributionBlock();
+    reduceTimeDists = newDistributionBlock();
+
+    taskAttemptStartTimes = new HashMap<String, Long>();
+    taskReduceAttemptShuffleEndTimes = new HashMap<String, Long>();
+    taskReduceAttemptSortEndTimes = new HashMap<String, Long>();
+    taskMapAttemptFinishTimes = new HashMap<String, Long>();
+    taskReduceAttemptFinishTimes = new HashMap<String, Long>();
+
+    final Path inputPath = new Path(inputFilename);
+
+    inputIsDirectory = pathIsDirectory(inputPath);
+
+    if (jobTraceFilename != null && attemptTimesPercentiles == null) {
+      attemptTimesPercentiles = new int[19];
+
+      for (int i = 0; i < 19; ++i) {
+        attemptTimesPercentiles[i] = (i + 1) * 5;
+      }
+    }
+
+    if (!inputIsDirectory) {
+      input = maybeUncompressedPath(inputPath);
+    } else {
+      inputDirectoryPath = inputPath;
+      FileSystem fs = inputPath.getFileSystem(getConf());
+      FileStatus[] statuses = fs.listStatus(inputPath);
+      inputDirectoryFiles = new String[statuses.length];
+
+      for (int i = 0; i < statuses.length; ++i) {
+        inputDirectoryFiles[i] = statuses[i].getPath().getName();
+      }
+
+      // filter out the .crc files, if any
+      int dropPoint = 0;
+
+      for (int i = 0; i < inputDirectoryFiles.length; ++i) {
+        String name = inputDirectoryFiles[i];
+
+        if (!(name.length() >= 4 && ".crc".equals(name
+            .substring(name.length() - 4)))) {
+          inputDirectoryFiles[dropPoint++] = name;
+        }
+      }
+
+      LOG.info("We dropped " + (inputDirectoryFiles.length - dropPoint)
+          + " crc files.");
+
+      String[] new_inputDirectoryFiles = new String[dropPoint];
+      System.arraycopy(inputDirectoryFiles, 0, new_inputDirectoryFiles, 0,
+          dropPoint);
+      inputDirectoryFiles = new_inputDirectoryFiles;
+
+      Arrays.sort(inputDirectoryFiles);
+
+      if (!setNextDirectoryInputStream()) {
+        throw new FileNotFoundException("Empty directory specified.");
+      }
+    }
+
+    if (jobTraceFilename != null) {
+      ObjectMapper jmapper = new ObjectMapper();
+      jmapper.configure(
+          SerializationConfig.Feature.CAN_OVERRIDE_ACCESS_MODIFIERS, true);
+      JsonFactory jfactory = jmapper.getJsonFactory();
+      FileSystem jobFS = jobTraceFilename.getFileSystem(getConf());
+      jobTraceGen = jfactory.createJsonGenerator(
+          jobFS.create(jobTraceFilename), JsonEncoding.UTF8);
+      if (prettyprintTrace) {
+        jobTraceGen.useDefaultPrettyPrinter();
+      }
+
+      if (topologyFilename != null) {
+        ObjectMapper tmapper = new ObjectMapper();
+        tmapper.configure(
+            SerializationConfig.Feature.CAN_OVERRIDE_ACCESS_MODIFIERS, true);
+        JsonFactory tfactory = tmapper.getJsonFactory();
+        FileSystem topoFS = topologyFilename.getFileSystem(getConf());
+        topologyGen = tfactory.createJsonGenerator(
+            topoFS.create(topologyFilename), JsonEncoding.UTF8);
+        topologyGen.useDefaultPrettyPrinter();
+      }
+    }
+
+    return 0;
+  }
+
+  private LineReader maybeUncompressedPath(Path p)
+      throws FileNotFoundException, IOException {
+    CompressionCodecFactory codecs = new CompressionCodecFactory(getConf());
+    inputCodec = codecs.getCodec(p);
+    FileSystem fs = p.getFileSystem(getConf());
+    FSDataInputStream fileIn = fs.open(p);
+
+    if (inputCodec == null) {
+      return new LineReader(fileIn, getConf());
+    } else {
+      inputDecompressor = CodecPool.getDecompressor(inputCodec);
+      return new LineReader(inputCodec.createInputStream(fileIn,
+          inputDecompressor), getConf());
+    }
+  }
+
+  private boolean setNextDirectoryInputStream() throws FileNotFoundException,
+      IOException {
+    if (input != null) {
+      input.close();
+      LOG.info("File closed: "+currentFileName);
+      input = null;
+    }
+
+    if (inputCodec != null) {
+      CodecPool.returnDecompressor(inputDecompressor);
+      inputDecompressor = null;
+      inputCodec = null;
+    }
+
+    ++inputDirectoryCursor;
+
+    if (inputDirectoryCursor >= inputDirectoryFiles.length) {
+      return false;
+    }
+
+    fileFirstLine = true;
+
+    currentFileName = inputDirectoryFiles[inputDirectoryCursor];
+
+    LOG.info("\nOpening file " + currentFileName
+        + "  *************************** .");
+    LOG
+        .info("This file, " + (inputDirectoryCursor + 1) + "/"
+            + inputDirectoryFiles.length + ", starts with line " + lineNumber
+            + ".");
+
+    input = maybeUncompressedPath(new Path(inputDirectoryPath, currentFileName));
+
+    return input != null;
+  }
+
+  private String readInputLine() throws IOException {
+    try {
+      if (input == null) {
+        return null;
+      }
+      inputLineText.clear();
+      if (input.readLine(inputLineText) == 0) {
+        return null;
+      }
+
+      return inputLineText.toString();
+    } catch (EOFException e) {
+      return null;
+    }
+
+  }
+
+  private String readCountedLine() throws IOException {
+    if (rereadableLine != null) {
+      String result = rereadableLine;
+      rereadableLine = null;
+      return result;
+    }
+
+    String result = readInputLine();
+
+    if (result != null) {
+      if (fileFirstLine && (result.equals("") || result.charAt(0) != '\f')) {
+        fileFirstLine = false;
+        rereadableLine = result;
+        return "\f!!FILE " + currentFileName + "!!\n";
+      }
+      fileFirstLine = false;
+      ++lineNumber;
+    } else if (inputIsDirectory && setNextDirectoryInputStream()) {
+      result = readCountedLine();
+    }
+
+    return result;
+  }
+
+  private void unreadCountedLine(String unreadee) {
+    if (rereadableLine == null) {
+      rereadableLine = unreadee;
+    }
+  }
+
+  private boolean apparentConfFileHeader(String header) {
+    return confFileHeader.matcher(header).find();
+  }
+
+  private boolean apparentXMLFileStart(String line) {
+    return xmlFilePrefix.matcher(line).lookingAt();
+  }
+
+  // This can return either the Pair of the !!file line and the XMLconf
+  // file, or null and an ordinary line. Returns just null if there's
+  // no more input.
+  private Pair<String, String> readBalancedLine() throws IOException {
+    String line = readCountedLine();
+
+    if (line == null) {
+      return null;
+    }
+
+    while (line.indexOf('\f') > 0) {
+      line = line.substring(line.indexOf('\f'));
+    }
+
+    if (line.length() != 0 && line.charAt(0) == '\f') {
+      String subjectLine = readCountedLine();
+
+      if (subjectLine != null && subjectLine.length() != 0
+          && apparentConfFileHeader(line) && apparentXMLFileStart(subjectLine)) {
+        StringBuilder sb = new StringBuilder();
+
+        while (subjectLine != null && subjectLine.indexOf('\f') > 0) {
+          subjectLine = subjectLine.substring(subjectLine.indexOf('\f'));
+        }
+
+        while (subjectLine != null
+            && (subjectLine.length() == 0 || subjectLine.charAt(0) != '\f')) {
+          sb.append(subjectLine);
+          subjectLine = readCountedLine();
+        }
+
+        if (subjectLine != null) {
+          unreadCountedLine(subjectLine);
+        }
+
+        return new Pair<String, String>(line, sb.toString());
+      }
+
+      // here we had a file line, but it introduced a log segment, not
+      // a conf file. We want to just ignore the file line.
+
+      return readBalancedLine();
+    }
+
+    String endlineString = (version == 0 ? " " : " .");
+
+    if (line.length() < endlineString.length()) {
+      return new Pair<String, String>(null, line);
+    }
+
+    if (!endlineString.equals(line.substring(line.length()
+        - endlineString.length()))) {
+      StringBuilder sb = new StringBuilder(line);
+
+      String addedLine;
+
+      do {
+        addedLine = readCountedLine();
+
+        if (addedLine == null) {
+          return new Pair<String, String>(null, sb.toString());
+        }
+
+        while (addedLine.indexOf('\f') > 0) {
+          addedLine = addedLine.substring(addedLine.indexOf('\f'));
+        }
+
+        if (addedLine.length() > 0 && addedLine.charAt(0) == '\f') {
+          unreadCountedLine(addedLine);
+          return new Pair<String, String>(null, sb.toString());
+        }
+
+        sb.append("\n");
+        sb.append(addedLine);
+      } while (!endlineString.equals(addedLine.substring(addedLine.length()
+          - endlineString.length())));
+
+      line = sb.toString();
+    }
+
+    return new Pair<String, String>(null, line);
+  }
+
+  private void incorporateSpread(Histogram taskTimes, Histogram[][] spreadTo,
+      JobOutcome outcome, LoggedJob.JobType jtype) {
+    if (!spreading) {
+      return;
+    }
+
+    if (taskTimes.getTotalCount() <= 1) {
+      return;
+    }
+
+    // there are some literals here that probably should be options
+    int[] endpoints = new int[2];
+
+    endpoints[0] = spreadMin;
+    endpoints[1] = spreadMax;
+
+    long[] endpointKeys = taskTimes.getCDF(1000, endpoints);
+
+    int smallResultOffset = (taskTimes.getTotalCount() < SMALL_SPREAD_COMPENSATION_THRESHOLD ? 1
+        : 0);
+
+    Histogram myTotal = spreadTo[outcome.ordinal()][jtype.ordinal()];
+
+    long dividend = endpointKeys[2 + smallResultOffset];
+    long divisor = endpointKeys[1 - smallResultOffset];
+
+    if (divisor > 0) {
+      long mytotalRatio = dividend * 1000000L / divisor;
+
+      myTotal.enter(mytotalRatio);
+    }
+  }
+
+  private void canonicalDistributionsEnter(Histogram[][] block,
+      JobOutcome outcome, LoggedJob.JobType type, long value) {
+    getDistribution(block, outcome, type).enter(value);
+    getDistribution(block, JobOutcome.OVERALL, type).enter(value);
+    getDistribution(block, outcome, LoggedJob.JobType.OVERALL).enter(value);
+    getDistribution(block, JobOutcome.OVERALL, LoggedJob.JobType.OVERALL)
+        .enter(value);
+  }
+
+  private void processJobLine(ParsedLine line) throws JsonProcessingException,
+      IOException {
+    try {
+      if (version == 0 || version == 1) {
+        // determine the job type if this is the declaration line
+        String jobID = line.get("JOBID");
+
+        String user = line.get("USER");
+
+        String jobPriority = line.get("JOB_PRIORITY");
+
+        String submitTime = line.get("SUBMIT_TIME");
+
+        String jobName = line.get("JOBNAME");
+
+        String launchTime = line.get("LAUNCH_TIME");
+
+        String finishTime = line.get("FINISH_TIME");
+
+        String status = line.get("JOB_STATUS");
+
+        String totalMaps = line.get("TOTAL_MAPS");
+
+        String totalReduces = line.get("TOTAL_REDUCES");
+
+        /*
+         * If the job appears new [the ID is different from the most recent one,
+         * if any] we make a new LoggedJob.
+         */
+        if (jobID != null
+            && jobTraceGen != null
+            && (jobBeingTraced == null || !jobID.equals(jobBeingTraced
+                .getJobID()))) {
+          // push out the old job if there is one, even though it did't get
+          // mated
+          // with a conf.
+
+          finalizeJob();
+
+          jobBeingTraced = new LoggedJob(jobID);
+
+          tasksInCurrentJob = new HashMap<String, LoggedTask>();
+          attemptsInCurrentJob = new HashMap<String, LoggedTaskAttempt>();
+
+          // initialize all the per-job statistics gathering places
+          successfulMapAttemptTimes = new Histogram[ParsedHost
+              .numberOfDistances() + 1];
+          for (int i = 0; i < successfulMapAttemptTimes.length; ++i) {
+            successfulMapAttemptTimes[i] = new Histogram();
+          }
+
+          successfulReduceAttemptTimes = new Histogram();
+          failedMapAttemptTimes = new Histogram[ParsedHost.numberOfDistances() + 1];
+          for (int i = 0; i < failedMapAttemptTimes.length; ++i) {
+            failedMapAttemptTimes[i] = new Histogram();
+          }
+
+          failedReduceAttemptTimes = new Histogram();
+          successfulNthMapperAttempts = new Histogram();
+          successfulNthReducerAttempts = new Histogram();
+          mapperLocality = new Histogram();
+        }
+
+        // here we fill in all the stuff the trace might need
+        if (jobBeingTraced != null) {
+          if (user != null) {
+            jobBeingTraced.setUser(user);
+          }
+
+          if (jobPriority != null) {
+            jobBeingTraced.setPriority(LoggedJob.JobPriority
+                .valueOf(jobPriority));
+          }
+
+          if (totalMaps != null) {
+            jobBeingTraced.setTotalMaps(Integer.parseInt(totalMaps));
+          }
+
+          if (totalReduces != null) {
+            jobBeingTraced.setTotalReduces(Integer.parseInt(totalReduces));
+          }
+
+          if (submitTime != null) {
+            jobBeingTraced.setSubmitTime(Long.parseLong(submitTime));
+          }
+
+          if (launchTime != null) {
+            jobBeingTraced.setLaunchTime(Long.parseLong(launchTime));
+          }
+
+          if (finishTime != null) {
+            jobBeingTraced.setFinishTime(Long.parseLong(finishTime));
+            if (status != null) {
+              jobBeingTraced.setOutcome(Pre21JobHistoryConstants.Values.valueOf(status));
+            }
+
+            maybeMateJobAndConf();
+          }
+        }
+
+        if (jobName != null) {
+          // we'll make it java unless the name parses out
+          Matcher m = streamingJobnamePattern.matcher(jobName);
+
+          thisJobType = LoggedJob.JobType.JAVA;
+
+          if (m.matches()) {
+            thisJobType = LoggedJob.JobType.STREAMING;
+          }
+        }
+        if (submitTime != null) {
+          submitTimeCurrentJob = Long.parseLong(submitTime);
+
+          currentJobID = jobID;
+
+          taskAttemptStartTimes = new HashMap<String, Long>();
+          taskReduceAttemptShuffleEndTimes = new HashMap<String, Long>();
+          taskReduceAttemptSortEndTimes = new HashMap<String, Long>();
+          taskMapAttemptFinishTimes = new HashMap<String, Long>();
+          taskReduceAttemptFinishTimes = new HashMap<String, Long>();
+
+          launchTimeCurrentJob = 0L;
+        } else if (launchTime != null && jobID != null
+            && currentJobID.equals(jobID)) {
+          launchTimeCurrentJob = Long.parseLong(launchTime);
+        } else if (finishTime != null && jobID != null
+            && currentJobID.equals(jobID)) {
+          long endTime = Long.parseLong(finishTime);
+
+          if (launchTimeCurrentJob != 0) {
+            String jobResultText = line.get("JOB_STATUS");
+
+            JobOutcome thisOutcome = ((jobResultText != null && "SUCCESS"
+                .equals(jobResultText)) ? JobOutcome.SUCCESS
+                : JobOutcome.FAILURE);
+
+            if (submitTimeCurrentJob != 0L) {
+              canonicalDistributionsEnter(delayTimeDists, thisOutcome,
+                  thisJobType, launchTimeCurrentJob - submitTimeCurrentJob);
+            }
+
+            if (launchTimeCurrentJob != 0L) {
+              canonicalDistributionsEnter(runTimeDists, thisOutcome,
+                  thisJobType, endTime - launchTimeCurrentJob);
+            }
+
+            // Now we process the hash tables with successful task attempts
+
+            Histogram currentJobMapTimes = new Histogram();
+            Histogram currentJobShuffleTimes = new Histogram();
+            Histogram currentJobSortTimes = new Histogram();
+            Histogram currentJobReduceTimes = new Histogram();
+
+            Iterator<Map.Entry<String, Long>> taskIter = taskAttemptStartTimes
+                .entrySet().iterator();
+
+            while (taskIter.hasNext()) {
+              Map.Entry<String, Long> entry = taskIter.next();
+
+              long startTime = entry.getValue();
+
+              // Map processing
+              Long mapEndTime = taskMapAttemptFinishTimes.get(entry.getKey());
+
+              if (mapEndTime != null) {
+                currentJobMapTimes.enter(mapEndTime - startTime);
+
+                canonicalDistributionsEnter(mapTimeDists, thisOutcome,
+                    thisJobType, mapEndTime - startTime);
+              }
+
+              // Reduce processing
+              Long shuffleEnd = taskReduceAttemptShuffleEndTimes.get(entry
+                  .getKey());
+              Long sortEnd = taskReduceAttemptSortEndTimes.get(entry.getKey());
+              Long reduceEnd = taskReduceAttemptFinishTimes.get(entry.getKey());
+
+              if (shuffleEnd != null && sortEnd != null && reduceEnd != null) {
+                currentJobShuffleTimes.enter(shuffleEnd - startTime);
+                currentJobSortTimes.enter(sortEnd - shuffleEnd);
+                currentJobReduceTimes.enter(reduceEnd - sortEnd);
+
+                canonicalDistributionsEnter(shuffleTimeDists, thisOutcome,
+                    thisJobType, shuffleEnd - startTime);
+                canonicalDistributionsEnter(sortTimeDists, thisOutcome,
+                    thisJobType, sortEnd - shuffleEnd);
+                canonicalDistributionsEnter(reduceTimeDists, thisOutcome,
+                    thisJobType, reduceEnd - sortEnd);
+              }
+            }
+
+            // Here we save out the task information
+            incorporateSpread(currentJobMapTimes, mapTimeSpreadDists,
+                thisOutcome, thisJobType);
+            incorporateSpread(currentJobShuffleTimes, shuffleTimeSpreadDists,
+                thisOutcome, thisJobType);
+            incorporateSpread(currentJobSortTimes, sortTimeSpreadDists,
+                thisOutcome, thisJobType);
+            incorporateSpread(currentJobReduceTimes, reduceTimeSpreadDists,
+                thisOutcome, thisJobType);
+          }
+        }
+      }
+    } catch (NumberFormatException e) {
+      LOG.warn(
+          "HadoopLogsAnalyzer.processJobLine: bad numerical format, at line "
+              + lineNumber + ".", e);
+    }
+  }
+
+  private void processTaskLine(ParsedLine line) {
+    if (jobBeingTraced != null) {
+      // these fields are in both the start and finish record
+      String taskID = line.get("TASKID");
+      String taskType = line.get("TASK_TYPE");
+
+      // this field is only in the start record
+      String startTime = line.get("START_TIME");
+
+      // these fields only exist or are only relevant in the finish record
+      String status = line.get("TASK_STATUS");
+      String finishTime = line.get("FINISH_TIME");
+
+      String splits = line.get("SPLITS");
+
+      LoggedTask task = tasksInCurrentJob.get(taskID);
+
+      boolean taskAlreadyLogged = task != null;
+
+      if (task == null) {
+        task = new LoggedTask();
+      }
+
+      if (splits != null) {
+        ArrayList<LoggedLocation> locations = null;
+
+        StringTokenizer tok = new StringTokenizer(splits, ",", false);
+
+        if (tok.countTokens() <= MAXIMUM_PREFERRED_LOCATIONS) {
+          locations = new ArrayList<LoggedLocation>();
+        }
+
+        while (tok.hasMoreTokens()) {
+          String nextSplit = tok.nextToken();
+
+          ParsedHost node = getAndRecordParsedHost(nextSplit);
+
+          if (locations != null && node != null) {
+            locations.add(node.makeLoggedLocation());
+          }
+        }
+
+        task.setPreferredLocations(locations);
+      }
+
+      task.setTaskID(taskID);
+
+      if (startTime != null) {
+        task.setStartTime(Long.parseLong(startTime));
+      }
+
+      if (finishTime != null) {
+        task.setFinishTime(Long.parseLong(finishTime));
+      }
+
+      Pre21JobHistoryConstants.Values typ;
+      Pre21JobHistoryConstants.Values stat;
+
+      try {
+        stat = status == null ? null : Pre21JobHistoryConstants.Values.valueOf(status);
+      } catch (IllegalArgumentException e) {
+        LOG.error("A task status you don't know about is \"" + status + "\".",
+            e);
+        stat = null;
+      }
+
+      task.setTaskStatus(stat);
+
+      try {
+        typ = taskType == null ? null : Pre21JobHistoryConstants.Values.valueOf(taskType);
+      } catch (IllegalArgumentException e) {
+        LOG.error("A task type you don't know about is \"" + taskType + "\".",
+            e);
+        typ = null;
+      }
+      
+      if (typ == null) {
+        return;
+      }
+
+      task.setTaskType(typ);
+
+      List<LoggedTask> vec = typ == Pre21JobHistoryConstants.Values.MAP ? jobBeingTraced
+          .getMapTasks() : typ == Pre21JobHistoryConstants.Values.REDUCE ? jobBeingTraced
+          .getReduceTasks() : jobBeingTraced.getOtherTasks();
+
+      if (!taskAlreadyLogged) {
+        vec.add(task);
+
+        tasksInCurrentJob.put(taskID, task);
+      }
+    }
+  }
+
+  private Pattern counterPattern(String counterName) {
+    Pattern result = counterPatterns.get(counterName);
+
+    if (result == null) {
+      String namePatternRegex = "\\[\\(" + counterName
+          + "\\)\\([^)]+\\)\\(([0-9]+)\\)\\]";
+      result = Pattern.compile(namePatternRegex);
+      counterPatterns.put(counterName, result);
+    }
+
+    return result;
+  }
+
+  private String parseCounter(String counterString, String counterName) {
+    if (counterString == null) {
+      return null;
+    }
+
+    Matcher mat = counterPattern(counterName).matcher(counterString);
+
+    if (mat.find()) {
+      return mat.group(1);
+    }
+
+    return null;
+  }
+
+  abstract class SetField {
+    LoggedTaskAttempt attempt;
+
+    SetField(LoggedTaskAttempt attempt) {
+      this.attempt = attempt;
+    }
+
+    abstract void set(long value);
+  }
+
+  private void incorporateCounter(SetField thunk, String counterString,
+      String counterName) {
+    String valueString = parseCounter(counterString, counterName);
+
+    if (valueString != null) {
+      thunk.set(Long.parseLong(valueString));
+    }
+  }
+
+  private void incorporateCounters(LoggedTaskAttempt attempt2,
+      String counterString) {
+    incorporateCounter(new SetField(attempt2) {
+      @Override
+      void set(long val) {
+        attempt.hdfsBytesRead = val;
+      }
+    }, counterString, "HDFS_BYTES_READ");
+    incorporateCounter(new SetField(attempt2) {
+      @Override
+      void set(long val) {
+        attempt.hdfsBytesWritten = val;
+      }
+    }, counterString, "HDFS_BYTES_WRITTEN");
+    incorporateCounter(new SetField(attempt2) {
+      @Override
+      void set(long val) {
+        attempt.fileBytesRead = val;
+      }
+    }, counterString, "FILE_BYTES_READ");
+    incorporateCounter(new SetField(attempt2) {
+      @Override
+      void set(long val) {
+        attempt.fileBytesWritten = val;
+      }
+    }, counterString, "FILE_BYTES_WRITTEN");
+    incorporateCounter(new SetField(attempt2) {
+      @Override
+      void set(long val) {
+        attempt.mapInputBytes = val;
+      }
+    }, counterString, "MAP_INPUT_BYTES");
+    incorporateCounter(new SetField(attempt2) {
+      @Override
+      void set(long val) {
+        attempt.mapInputRecords = val;
+      }
+    }, counterString, "MAP_INPUT_RECORDS");
+    incorporateCounter(new SetField(attempt2) {
+      @Override
+      void set(long val) {
+        attempt.mapOutputBytes = val;
+      }
+    }, counterString, "MAP_OUTPUT_BYTES");
+    incorporateCounter(new SetField(attempt2) {
+      @Override
+      void set(long val) {
+        attempt.mapOutputRecords = val;
+      }
+    }, counterString, "MAP_OUTPUT_RECORDS");
+    incorporateCounter(new SetField(attempt2) {
+      @Override
+      void set(long val) {
+        attempt.combineInputRecords = val;
+      }
+    }, counterString, "COMBINE_INPUT_RECORDS");
+    incorporateCounter(new SetField(attempt2) {
+      @Override
+      void set(long val) {
+        attempt.reduceInputGroups = val;
+      }
+    }, counterString, "REDUCE_INPUT_GROUPS");
+    incorporateCounter(new SetField(attempt2) {
+      @Override
+      void set(long val) {
+        attempt.reduceInputRecords = val;
+      }
+    }, counterString, "REDUCE_INPUT_RECORDS");
+    incorporateCounter(new SetField(attempt2) {
+      @Override
+      void set(long val) {
+        attempt.reduceShuffleBytes = val;
+      }
+    }, counterString, "REDUCE_SHUFFLE_BYTES");
+    incorporateCounter(new SetField(attempt2) {
+      @Override
+      void set(long val) {
+        attempt.reduceOutputRecords = val;
+      }
+    }, counterString, "REDUCE_OUTPUT_RECORDS");
+    incorporateCounter(new SetField(attempt2) {
+      @Override
+      void set(long val) {
+        attempt.spilledRecords = val;
+      }
+    }, counterString, "SPILLED_RECORDS");
+  }
+
+  private ParsedHost getAndRecordParsedHost(String hostName) {
+    ParsedHost result = ParsedHost.parse(hostName);
+
+    if (result != null && !allHosts.contains(result)) {
+      allHosts.add(result);
+    }
+
+    return result;
+  }
+
+  private void processMapAttemptLine(ParsedLine line) {
+    String attemptID = line.get("TASK_ATTEMPT_ID");
+
+    String taskID = line.get("TASKID");
+
+    String status = line.get("TASK_STATUS");
+
+    String attemptStartTime = line.get("START_TIME");
+    String attemptFinishTime = line.get("FINISH_TIME");
+
+    String hostName = line.get("HOSTNAME");
+
+    String counters = line.get("COUNTERS");
+
+    if (jobBeingTraced != null && taskID != null) {
+      LoggedTask task = tasksInCurrentJob.get(taskID);
+
+      if (task == null) {
+        task = new LoggedTask();
+
+        task.setTaskID(taskID);
+
+        jobBeingTraced.getMapTasks().add(task);
+
+        tasksInCurrentJob.put(taskID, task);
+      }
+
+      task.setTaskID(taskID);
+
+      LoggedTaskAttempt attempt = attemptsInCurrentJob.get(attemptID);
+
+      boolean attemptAlreadyExists = attempt != null;
+
+      if (attempt == null) {
+        attempt = new LoggedTaskAttempt();
+
+        attempt.setAttemptID(attemptID);
+      }
+
+      if (!attemptAlreadyExists) {
+        attemptsInCurrentJob.put(attemptID, attempt);
+        task.getAttempts().add(attempt);
+      }
+
+      Pre21JobHistoryConstants.Values stat = null;
+
+      try {
+        stat = status == null ? null : Pre21JobHistoryConstants.Values.valueOf(status);
+      } catch (IllegalArgumentException e) {
+        LOG.error("A map attempt status you don't know about is \"" + status
+            + "\".", e);
+        stat = null;
+      }
+
+      incorporateCounters(attempt, counters);
+
+      attempt.setResult(stat);
+
+      if (attemptStartTime != null) {
+        attempt.setStartTime(Long.parseLong(attemptStartTime));
+      }
+
+      if (attemptFinishTime != null) {
+        attempt.setFinishTime(Long.parseLong(attemptFinishTime));
+      }
+
+      int distance = Integer.MAX_VALUE;
+
+      if (hostName != null) {
+        attempt.setHostName(hostName);
+
+        ParsedHost host = null;
+
+        host = getAndRecordParsedHost(hostName);
+
+        if (host != null) {
+          attempt.setLocation(host.makeLoggedLocation());
+        }
+
+        ArrayList<LoggedLocation> locs = task.getPreferredLocations();
+
+        if (host != null && locs != null) {
+          for (LoggedLocation loc : locs) {
+            ParsedHost preferedLoc = new ParsedHost(loc);
+
+            distance = Math.min(distance, preferedLoc.distance(host));
+          }
+        }
+
+        mapperLocality.enter(distance);
+      }
+
+      distance = Math.min(distance, successfulMapAttemptTimes.length - 1);
+
+      if (attempt.getStartTime() > 0 && attempt.getFinishTime() > 0) {
+        long runtime = attempt.getFinishTime() - attempt.getStartTime();
+
+        if (stat == Pre21JobHistoryConstants.Values.SUCCESS) {
+          successfulMapAttemptTimes[distance].enter(runtime);
+        }
+
+        if (stat == Pre21JobHistoryConstants.Values.FAILED) {
+          failedMapAttemptTimes[distance].enter(runtime);
+        }
+      }
+
+      if (attemptID != null) {
+        Matcher matcher = taskAttemptIDPattern.matcher(attemptID);
+
+        if (matcher.matches()) {
+          String attemptNumberString = matcher.group(1);
+
+          if (attemptNumberString != null) {
+            int attemptNumber = Integer.parseInt(attemptNumberString);
+
+            successfulNthMapperAttempts.enter(attemptNumber);
+          }
+        }
+      }
+    }
+
+    try {
+      if (attemptStartTime != null) {
+        long startTimeValue = Long.parseLong(attemptStartTime);
+
+        if (startTimeValue != 0
+            && startTimeValue + MAXIMUM_CLOCK_SKEW >= launchTimeCurrentJob) {
+          taskAttemptStartTimes.put(attemptID, startTimeValue);
+        } else {
+          taskAttemptStartTimes.remove(attemptID);
+        }
+      } else if (status != null && attemptFinishTime != null) {
+        long finishTime = Long.parseLong(attemptFinishTime);
+
+        if (status.equals("SUCCESS")) {
+          taskMapAttemptFinishTimes.put(attemptID, finishTime);
+        }
+      }
+    } catch (NumberFormatException e) {
+      LOG.warn(
+          "HadoopLogsAnalyzer.processMapAttemptLine: bad numerical format, at line"
+              + lineNumber + ".", e);
+    }
+  }
+
+  private void processReduceAttemptLine(ParsedLine line) {
+    String attemptID = line.get("TASK_ATTEMPT_ID");
+
+    String taskID = line.get("TASKID");
+
+    String status = line.get("TASK_STATUS");
+
+    String attemptStartTime = line.get("START_TIME");
+    String attemptFinishTime = line.get("FINISH_TIME");
+    String attemptShuffleFinished = line.get("SHUFFLE_FINISHED");
+    String attemptSortFinished = line.get("SORT_FINISHED");
+
+    String counters = line.get("COUNTERS");
+
+    String hostName = line.get("HOSTNAME");
+
+    if (hostName != null && !hostNames.contains(hostName)) {
+      hostNames.add(hostName);
+    }
+
+    if (jobBeingTraced != null && taskID != null) {
+      LoggedTask task = tasksInCurrentJob.get(taskID);
+
+      if (task == null) {
+        task = new LoggedTask();
+
+        task.setTaskID(taskID);
+
+        jobBeingTraced.getReduceTasks().add(task);
+
+        tasksInCurrentJob.put(taskID, task);
+      }
+
+      task.setTaskID(taskID);
+
+      LoggedTaskAttempt attempt = attemptsInCurrentJob.get(attemptID);
+
+      boolean attemptAlreadyExists = attempt != null;
+
+      if (attempt == null) {
+        attempt = new LoggedTaskAttempt();
+
+        attempt.setAttemptID(attemptID);
+      }
+
+      if (!attemptAlreadyExists) {
+        attemptsInCurrentJob.put(attemptID, attempt);
+        task.getAttempts().add(attempt);
+      }
+
+      Pre21JobHistoryConstants.Values stat = null;
+
+      try {
+        stat = status == null ? null : Pre21JobHistoryConstants.Values.valueOf(status);
+      } catch (IllegalArgumentException e) {
+        LOG.warn("A map attempt status you don't know about is \"" + status
+            + "\".", e);
+        stat = null;
+      }
+
+      incorporateCounters(attempt, counters);
+
+      attempt.setResult(stat);
+
+      if (attemptStartTime != null) {
+        attempt.setStartTime(Long.parseLong(attemptStartTime));
+      }
+
+      if (attemptFinishTime != null) {
+        attempt.setFinishTime(Long.parseLong(attemptFinishTime));
+      }
+
+      if (attemptShuffleFinished != null) {
+        attempt.setShuffleFinished(Long.parseLong(attemptShuffleFinished));
+      }
+
+      if (attemptSortFinished != null) {
+        attempt.setSortFinished(Long.parseLong(attemptSortFinished));
+      }
+
+      if (attempt.getStartTime() > 0 && attempt.getFinishTime() > 0) {
+        long runtime = attempt.getFinishTime() - attempt.getStartTime();
+
+        if (stat == Pre21JobHistoryConstants.Values.SUCCESS) {
+          successfulReduceAttemptTimes.enter(runtime);
+        }
+
+        if (stat == Pre21JobHistoryConstants.Values.FAILED) {
+          failedReduceAttemptTimes.enter(runtime);
+        }
+      }
+      if (hostName != null) {
+        attempt.setHostName(hostName);
+      }
+
+      if (attemptID != null) {
+        Matcher matcher = taskAttemptIDPattern.matcher(attemptID);
+
+        if (matcher.matches()) {
+          String attemptNumberString = matcher.group(1);
+
+          if (attemptNumberString != null) {
+            int attemptNumber = Integer.parseInt(attemptNumberString);
+
+            successfulNthReducerAttempts.enter(attemptNumber);
+          }
+        }
+      }
+    }
+
+    try {
+      if (attemptStartTime != null) {
+        long startTimeValue = Long.parseLong(attemptStartTime);
+
+        if (startTimeValue != 0
+            && startTimeValue + MAXIMUM_CLOCK_SKEW >= launchTimeCurrentJob) {
+          taskAttemptStartTimes.put(attemptID, startTimeValue);
+        }
+      } else if (status != null && status.equals("SUCCESS")
+          && attemptFinishTime != null) {
+        long finishTime = Long.parseLong(attemptFinishTime);
+
+        taskReduceAttemptFinishTimes.put(attemptID, finishTime);
+
+        if (attemptShuffleFinished != null) {
+          taskReduceAttemptShuffleEndTimes.put(attemptID, Long
+              .parseLong(attemptShuffleFinished));
+        }
+
+        if (attemptSortFinished != null) {
+          taskReduceAttemptSortEndTimes.put(attemptID, Long
+              .parseLong(attemptSortFinished));
+        }
+      }
+    } catch (NumberFormatException e) {
+      LOG.error(
+          "HadoopLogsAnalyzer.processReduceAttemptLine: bad numerical format, at line"
+              + lineNumber + ".", e);
+    }
+  }
+
+  private void processParsedLine(ParsedLine line)
+      throws JsonProcessingException, IOException {
+    if (!collecting) {
+      // "Job", "MapAttempt", "ReduceAttempt", "Task"
+      LogRecordType myType = line.getType();
+
+      if (myType == canonicalJob) {
+        processJobLine(line);
+      } else if (myType == canonicalTask) {
+        processTaskLine(line);
+      } else if (myType == canonicalMapAttempt) {
+        processMapAttemptLine(line);
+      } else if (myType == canonicalReduceAttempt) {
+        processReduceAttemptLine(line);
+      } else {
+      }
+    }
+  }
+
+  private void printDistributionSet(String title, Histogram[][] distSet) {
+    statisticalOutput.print(title + "\n\n");
+
+    // print out buckets
+
+    for (int i = 0; i < JobOutcome.values().length; ++i) {
+      for (int j = 0; j < LoggedJob.JobType.values().length; ++j) {
+        JobOutcome thisOutcome = JobOutcome.values()[i];
+        LoggedJob.JobType thisType = LoggedJob.JobType.values()[j];
+
+        statisticalOutput.print("outcome = ");
+        statisticalOutput.print(thisOutcome.toString());
+        statisticalOutput.print(", and type = ");
+        statisticalOutput.print(thisType.toString());
+        statisticalOutput.print(".\n\n");
+
+        Histogram dist = distSet[i][j];
+
+        printSingleDistributionData(dist);
+      }
+    }
+  }
+
+  private void printSingleDistributionData(Histogram dist) {
+    int[] percentiles = new int[numberBuckets];
+
+    for (int k = 0; k < numberBuckets; ++k) {
+      percentiles[k] = k + 1;
+    }
+
+    long[] cdf = dist.getCDF(numberBuckets + 1, percentiles);
+
+    if (cdf == null) {
+      statisticalOutput.print("(No data)\n");
+    } else {
+      statisticalOutput.print("min:  ");
+      statisticalOutput.print(cdf[0]);
+      statisticalOutput.print("\n");
+
+      for (int k = 0; k < numberBuckets; ++k) {
+        statisticalOutput.print(percentiles[k]);
+        statisticalOutput.print("%   ");
+        statisticalOutput.print(cdf[k + 1]);
+        statisticalOutput.print("\n");
+      }
+
+      statisticalOutput.print("max:  ");
+      statisticalOutput.print(cdf[numberBuckets + 1]);
+      statisticalOutput.print("\n");
+    }
+  }
+
+  private void maybeMateJobAndConf() throws IOException {
+    if (jobBeingTraced != null && jobconf != null
+        && jobBeingTraced.getJobID().equals(jobconf.jobID)) {
+      jobBeingTraced.setHeapMegabytes(jobconf.heapMegabytes);
+
+      jobBeingTraced.setQueue(jobconf.queue);
+      jobBeingTraced.setJobName(jobconf.jobName);
+
+      jobBeingTraced.setClusterMapMB(jobconf.clusterMapMB);
+      jobBeingTraced.setClusterReduceMB(jobconf.clusterReduceMB);
+      jobBeingTraced.setJobMapMB(jobconf.jobMapMB);
+      jobBeingTraced.setJobReduceMB(jobconf.jobReduceMB);
+
+      jobconf = null;
+
+      finalizeJob();
+    }
+  }
+
+  private ArrayList<LoggedDiscreteCDF> mapCDFArrayList(Histogram[] data) {
+    ArrayList<LoggedDiscreteCDF> result = new ArrayList<LoggedDiscreteCDF>();
+
+    for (Histogram hist : data) {
+      LoggedDiscreteCDF discCDF = new LoggedDiscreteCDF();
+      discCDF.setCDF(hist, attemptTimesPercentiles, 100);
+      result.add(discCDF);
+    }
+
+    return result;
+  }
+
+  private void finalizeJob() throws IOException {
+    if (jobBeingTraced != null) {
+      if (omitTaskDetails) {
+        jobBeingTraced.setMapTasks(null);
+        jobBeingTraced.setReduceTasks(null);
+        jobBeingTraced.setOtherTasks(null);
+      }
+
+      // add digest info to the job
+      jobBeingTraced
+          .setSuccessfulMapAttemptCDFs(mapCDFArrayList(successfulMapAttemptTimes));
+      jobBeingTraced
+          .setFailedMapAttemptCDFs(mapCDFArrayList(failedMapAttemptTimes));
+
+      LoggedDiscreteCDF discCDF = new LoggedDiscreteCDF();
+      discCDF
+          .setCDF(successfulReduceAttemptTimes, attemptTimesPercentiles, 100);
+      jobBeingTraced.setSuccessfulReduceAttemptCDF(discCDF);
+
+      discCDF = new LoggedDiscreteCDF();
+      discCDF.setCDF(failedReduceAttemptTimes, attemptTimesPercentiles, 100);
+      jobBeingTraced.setFailedReduceAttemptCDF(discCDF);
+
+      long totalSuccessfulAttempts = 0L;
+      long maxTriesToSucceed = 0L;
+
+      for (Map.Entry<Long, Long> ent : successfulNthMapperAttempts) {
+        totalSuccessfulAttempts += ent.getValue();
+        maxTriesToSucceed = Math.max(maxTriesToSucceed, ent.getKey());
+      }
+
+      if (totalSuccessfulAttempts > 0L) {
+        double[] successAfterI = new double[(int) maxTriesToSucceed + 1];
+        for (int i = 0; i < successAfterI.length; ++i) {
+          successAfterI[i] = 0.0D;
+        }
+
+        for (Map.Entry<Long, Long> ent : successfulNthMapperAttempts) {
+          successAfterI[ent.getKey().intValue()] = ((double) ent.getValue())
+              / totalSuccessfulAttempts;
+        }
+        jobBeingTraced.setMapperTriesToSucceed(successAfterI);
+      } else {
+        jobBeingTraced.setMapperTriesToSucceed(null);
+      }
+
+      jobTraceGen.writeObject(jobBeingTraced);
+
+      jobTraceGen.writeRaw("\n");
+
+      jobBeingTraced = null;
+    }
+  }
+
+  public int run(String[] args) throws IOException {
+
+    int result = initializeHadoopLogsAnalyzer(args);
+
+    if (result != 0) {
+      return result;
+    }
+
+    return run();
+  }
+
+  int run() throws IOException {
+    Pair<String, String> line = readBalancedLine();
+
+    while (line != null) {
+      if (debug
+          && (lineNumber < 1000000L && lineNumber % 1000L == 0 || lineNumber % 1000000L == 0)) {
+        LOG.debug("" + lineNumber + " " + line.second());
+      }
+
+      if (line.first() == null) {
+        try {
+          // HACK ALERT!! It's possible for a Job end line to end a
+          // job for which we have a config file
+          // image [ a ParsedConfigFile ] in jobconf.
+          //
+          // processParsedLine handles this.
+
+          processParsedLine(new ParsedLine(line.second(), version));
+        } catch (StringIndexOutOfBoundsException e) {
+          LOG.warn("anomalous line #" + lineNumber + ":" + line, e);
+        }
+      } else {
+        jobconf = new ParsedConfigFile(line.first(), line.second());
+
+        if (jobconf.valid == false) {
+          jobconf = null;
+        }
+
+        maybeMateJobAndConf();
+      }
+
+      line = readBalancedLine();
+    }
+
+    finalizeJob();
+
+    if (collecting) {
+      String[] typeNames = LogRecordType.lineTypes();
+
+      for (int i = 0; i < typeNames.length; ++i) {
+        statisticalOutput.print(typeNames[i]);
+        statisticalOutput.print('\n');
+      }
+    } else {
+      if (delays) {
+        printDistributionSet("Job start delay spectrum:", delayTimeDists);
+      }
+
+      if (runtimes) {
+        printDistributionSet("Job run time spectrum:", runTimeDists);
+      }
+
+      if (spreading) {
+        String ratioDescription = "(" + spreadMax + "/1000 %ile) to ("
+            + spreadMin + "/1000 %ile) scaled by 1000000";
+
+        printDistributionSet(
+            "Map task success times " + ratioDescription + ":",
+            mapTimeSpreadDists);
+        printDistributionSet("Shuffle success times " + ratioDescription + ":",
+            shuffleTimeSpreadDists);
+        printDistributionSet("Sort success times " + ratioDescription + ":",
+            sortTimeSpreadDists);
+        printDistributionSet("Reduce success times " + ratioDescription + ":",
+            reduceTimeSpreadDists);
+      }
+
+      if (collectTaskTimes) {
+        printDistributionSet("Global map task success times:", mapTimeDists);
+        printDistributionSet("Global shuffle task success times:",
+            shuffleTimeDists);
+        printDistributionSet("Global sort task success times:", sortTimeDists);
+        printDistributionSet("Global reduce task success times:",
+            reduceTimeDists);
+      }
+    }
+
+    if (topologyGen != null) {
+      LoggedNetworkTopology topo = new LoggedNetworkTopology(allHosts,
+          "<root>", 0);
+      topologyGen.writeObject(topo);
+      topologyGen.close();
+    }
+
+    if (jobTraceGen != null) {
+      jobTraceGen.close();
+    }
+
+    if (input != null) {
+      input.close();
+      input = null;
+    }
+
+    if (inputCodec != null) {
+      CodecPool.returnDecompressor(inputDecompressor);
+      inputDecompressor = null;
+      inputCodec = null;
+    }
+
+    return 0;
+  }
+
+  /**
+   * @param args
+   * 
+   *          Last arg is the input file. That file can be a directory, in which
+   *          case you get all the files in sorted order. We will decompress
+   *          files whose nmes end in .gz .
+   * 
+   *          switches: -c collect line types.
+   * 
+   *          -d debug mode
+   * 
+   *          -delays print out the delays [interval between job submit time and
+   *          launch time]
+   * 
+   *          -runtimes print out the job runtimes
+   * 
+   *          -spreads print out the ratio of 10%ile and 90%ile, of both the
+   *          successful map task attempt run times and the the successful
+   *          reduce task attempt run times
+   * 
+   *          -tasktimes prints out individual task time distributions
+   * 
+   *          collects all the line types and prints the first example of each
+   *          one
+   */
+  public static void main(String[] args) {
+    try {
+      HadoopLogsAnalyzer analyzer = new HadoopLogsAnalyzer();
+
+      int result = ToolRunner.run(analyzer, args);
+
+      if (result == 0) {
+        return;
+      }
+
+      System.exit(result);
+    } catch (FileNotFoundException e) {
+      LOG.error("", e);
+      e.printStackTrace(staticDebugOutput);
+      System.exit(1);
+    } catch (IOException e) {
+      LOG.error("", e);
+      e.printStackTrace(staticDebugOutput);
+      System.exit(2);
+    } catch (Exception e) {
+      LOG.error("", e);
+      e.printStackTrace(staticDebugOutput);
+      System.exit(3);
+    }
+  }
+}

+ 164 - 0
src/tools/org/apache/hadoop/tools/rumen/Histogram.java

@@ -0,0 +1,164 @@
+/**
+ * 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.PrintStream;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.TreeMap;
+
+/**
+ * {@link Histogram} represents an ordered summary of a sequence of {@code long}
+ * s which can be queried to produce a discrete approximation of its cumulative
+ * distribution function
+ * 
+ */
+class Histogram implements Iterable<Map.Entry<Long, Long>> {
+  private TreeMap<Long, Long> content = new TreeMap<Long, Long>();
+
+  private String name;
+
+  private long totalCount;
+
+  public Histogram() {
+    this("(anonymous)");
+  }
+
+  public Histogram(String name) {
+    super();
+
+    this.name = name;
+
+    totalCount = 0L;
+  }
+
+  public void dump(PrintStream stream) {
+    stream.print("dumping Histogram " + name + ":\n");
+
+    Iterator<Map.Entry<Long, Long>> iter = iterator();
+
+    while (iter.hasNext()) {
+      Map.Entry<Long, Long> ent = iter.next();
+
+      stream.print("val/count pair: " + (long) ent.getKey() + ", "
+          + (long) ent.getValue() + "\n");
+    }
+
+    stream.print("*** end *** \n");
+  }
+
+  public Iterator<Map.Entry<Long, Long>> iterator() {
+    return content.entrySet().iterator();
+  }
+
+  public long get(long key) {
+    Long result = content.get(key);
+
+    return result == null ? 0 : result;
+  }
+
+  public long getTotalCount() {
+    return totalCount;
+  }
+
+  public void enter(long value) {
+    Long existingValue = content.get(value);
+
+    if (existingValue == null) {
+      content.put(value, 1L);
+    } else {
+      content.put(value, existingValue + 1L);
+    }
+
+    ++totalCount;
+  }
+
+  /**
+   * Produces a discrete approximation of the CDF. The user provides the points
+   * on the {@code Y} axis he wants, and we give the corresponding points on the
+   * {@code X} axis, plus the minimum and maximum from the data.
+   * 
+   * @param scale
+   *          the denominator applied to every element of buckets. For example,
+   *          if {@code scale} is {@code 1000}, a {@code buckets} element of 500
+   *          will specify the median in that output slot.
+   * @param buckets
+   *          an array of int, all less than scale and each strictly greater
+   *          than its predecessor if any. We don't check these requirements.
+   * @return a {@code long[]}, with two more elements than {@code buckets} has.
+   *         The first resp. last element is the minimum resp. maximum value
+   *         that was ever {@code enter}ed. The rest of the elements correspond
+   *         to the elements of {@code buckets} and carry the first element
+   *         whose rank is no less than {@code #content elements * scale /
+   *         bucket}.
+   * 
+   */
+  public long[] getCDF(int scale, int[] buckets) {
+    if (totalCount == 0) {
+      return null;
+    }
+
+    long[] result = new long[buckets.length + 2];
+
+    // fill in the min and the max
+    result[0] = content.firstEntry().getKey();
+
+    result[buckets.length + 1] = content.lastEntry().getKey();
+
+    Iterator<Map.Entry<Long, Long>> iter = content.entrySet().iterator();
+    long cumulativeCount = 0;
+    int bucketCursor = 0;
+
+    
+    // Loop invariant: the item at buckets[bucketCursor] can still be reached
+    // from iter, and the number of logged elements no longer available from
+    // iter is cumulativeCount.
+    // 
+    // cumulativeCount/totalCount is therefore strictly less than
+    // buckets[bucketCursor]/scale .
+     
+    while (iter.hasNext()) {
+      long targetCumulativeCount = buckets[bucketCursor] * totalCount / scale;
+
+      Map.Entry<Long, Long> elt = iter.next();
+
+      cumulativeCount += elt.getValue();
+
+      while (cumulativeCount >= targetCumulativeCount) {
+        result[bucketCursor + 1] = elt.getKey();
+
+        ++bucketCursor;
+
+        if (bucketCursor < buckets.length) {
+          targetCumulativeCount = buckets[bucketCursor] * totalCount / scale;
+        } else {
+          break;
+        }
+      }
+
+      if (bucketCursor == buckets.length) {
+        break;
+      }
+    }
+
+    return result;
+  }
+}

+ 118 - 0
src/tools/org/apache/hadoop/tools/rumen/JobStory.java

@@ -0,0 +1,118 @@
+/**
+ * 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 org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.JobID;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.hadoop.tools.rumen.Pre21JobHistoryConstants.Values;
+
+/**
+ * {@link JobStory} represents the runtime information available for a
+ * completed Map-Reduce job.
+ */
+public interface JobStory {
+  
+  /**
+   * Get the {@link JobConf} for the job.
+   * @return the <code>JobConf</code> for the job
+   */
+  public JobConf getJobConf();
+  
+  /**
+   * Get the job name.
+   * @return the job name
+   */
+  public String getName();
+  
+  /**
+   * Get the job ID
+   * @return the job ID
+   */
+  public JobID getJobID();
+  
+  /**
+   * Get the user who ran the job.
+   * @return the user who ran the job
+   */
+  public String getUser();
+  
+  /**
+   * Get the job submission time.
+   * @return the job submission time
+   */
+  public long getSubmissionTime();
+  
+  /**
+   * Get the number of maps in the {@link JobStory}.
+   * @return the number of maps in the <code>Job</code>
+   */
+  public int getNumberMaps();
+  
+  /**
+   * Get the number of reduce in the {@link JobStory}.
+   * @return the number of reduces in the <code>Job</code>
+   */
+  public int getNumberReduces();
+
+  /**
+   * Get the input splits for the job.
+   * @return the input splits for the job
+   */
+  public InputSplit[] getInputSplits();
+  
+  /**
+   * Get {@link TaskInfo} for a given task.
+   * @param taskType {@link TaskType} of the task
+   * @param taskNumber Partition number of the task
+   * @return the <code>TaskInfo</code> for the given task
+   */
+  public TaskInfo getTaskInfo(TaskType taskType, int taskNumber);
+  
+  /**
+   * Get {@link TaskAttemptInfo} for a given task-attempt, without regard to
+   * impact of locality (e.g. not needed to make scheduling decisions).
+   * @param taskType {@link TaskType} of the task-attempt
+   * @param taskNumber Partition number of the task-attempt
+   * @param taskAttemptNumber Attempt number of the task
+   * @return the <code>TaskAttemptInfo</code> for the given task-attempt
+   */
+  public TaskAttemptInfo getTaskAttemptInfo(TaskType taskType, 
+                                            int taskNumber, 
+                                            int taskAttemptNumber);
+  
+  /**
+   * Get {@link TaskAttemptInfo} for a given task-attempt, considering impact
+   * of locality.
+   * @param taskNumber Partition number of the task-attempt
+   * @param taskAttemptNumber Attempt number of the task
+   * @param locality Data locality of the task as scheduled in simulation
+   * @return the <code>TaskAttemptInfo</code> for the given task-attempt
+   */
+  public TaskAttemptInfo
+    getMapTaskAttemptInfoAdjusted(int taskNumber,
+                                  int taskAttemptNumber,
+                                  int locality);
+  
+  /**
+   * Get the outcome of the job execution.
+   * @return The outcome of the job execution.
+   */
+  public Values getOutcome();
+}

+ 33 - 0
src/tools/org/apache/hadoop/tools/rumen/JobStoryProducer.java

@@ -0,0 +1,33 @@
+/**
+ * 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.Closeable;
+import java.io.IOException;
+
+/**
+ * {@link JobStoryProducer} produces the sequence of {@link JobStory}'s.
+ */
+public interface JobStoryProducer extends Closeable {
+  /**
+   * Get the next job.
+   * @return The next job. Or null if no more job is available.
+   * @throws IOException 
+   */
+  JobStory getNextJob() throws IOException;
+}

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

@@ -0,0 +1,51 @@
+/**
+ * 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.IOException;
+import java.io.InputStream;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+
+/**
+ * Reading JSON-encoded job traces and produce {@link LoggedJob} instances.
+ */
+public class JobTraceReader extends JsonObjectMapperParser<LoggedJob> {
+  /**
+   * Constructor.
+   * 
+   * @param path
+   *          Path to the JSON trace file, possibly compressed.
+   * @param conf
+   * @throws IOException
+   */
+  public JobTraceReader(Path path, Configuration conf) throws IOException {
+    super(path, LoggedJob.class, conf);
+  }
+
+  /**
+   * Constructor.
+   * 
+   * @param input
+   *          The input stream for the JSON trace.
+   */
+  public JobTraceReader(InputStream input) throws IOException {
+    super(input, LoggedJob.class);
+  }
+}

+ 116 - 0
src/tools/org/apache/hadoop/tools/rumen/JsonObjectMapperParser.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.tools.rumen;
+
+import java.io.Closeable;
+import java.io.EOFException;
+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.compress.CodecPool;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.io.compress.CompressionCodecFactory;
+import org.apache.hadoop.io.compress.Decompressor;
+import org.codehaus.jackson.JsonParser;
+import org.codehaus.jackson.map.DeserializationConfig;
+import org.codehaus.jackson.map.ObjectMapper;
+
+/**
+ * A simple wrapper for parsing JSON-encoded data using ObjectMapper.
+ * @param <T> The (base) type of the object(s) to be parsed by this parser.
+ */
+class JsonObjectMapperParser<T> implements Closeable {
+  private final ObjectMapper mapper;
+  private final Class<? extends T> clazz;
+  private final JsonParser jsonParser;
+  private final Decompressor decompressor;
+
+  /**
+   * Constructor.
+   * 
+   * @param path 
+   *          Path to the JSON data file, possibly compressed.
+   * @param conf
+   * @throws IOException
+   */
+  public JsonObjectMapperParser(Path path, Class<? extends T> clazz,
+      Configuration conf) throws IOException {
+    mapper = new ObjectMapper();
+    mapper.configure(
+        DeserializationConfig.Feature.CAN_OVERRIDE_ACCESS_MODIFIERS, true);
+    this.clazz = clazz;
+    FileSystem fs = path.getFileSystem(conf);
+    CompressionCodec codec = new CompressionCodecFactory(conf).getCodec(path);
+    InputStream input;
+    if (codec == null) {
+      input = fs.open(path);
+      decompressor = null;
+    } else {
+      FSDataInputStream fsdis = fs.open(path);
+      decompressor = CodecPool.getDecompressor(codec);
+      input = codec.createInputStream(fsdis, decompressor);
+    }
+    jsonParser = mapper.getJsonFactory().createJsonParser(input);
+  }
+
+  /**
+   * Constructor.
+   * 
+   * @param input
+   *          The input stream for the JSON data.
+   */
+  public JsonObjectMapperParser(InputStream input, Class<? extends T> clazz)
+      throws IOException {
+    mapper = new ObjectMapper();
+    mapper.configure(
+        DeserializationConfig.Feature.CAN_OVERRIDE_ACCESS_MODIFIERS, true);
+    this.clazz = clazz;
+    decompressor = null;
+    jsonParser = mapper.getJsonFactory().createJsonParser(input);
+  }
+
+  /**
+   * Get the next object from the trace.
+   * 
+   * @return The next instance of the object. Or null if we reach the end of
+   *         stream.
+   * @throws IOException
+   */
+  public T getNext() throws IOException {
+    try {
+      return mapper.readValue(jsonParser, clazz);
+    } catch (EOFException e) {
+      return null;
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    try {
+      jsonParser.close();
+    } finally {
+      if (decompressor != null) {
+        CodecPool.returnDecompressor(decompressor);
+      }
+    }
+  }
+}

+ 73 - 0
src/tools/org/apache/hadoop/tools/rumen/LogRecordType.java

@@ -0,0 +1,73 @@
+/**
+ * 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.util.HashMap;
+import java.util.Map;
+import java.util.Iterator;
+
+class LogRecordType {
+  static Map<String, LogRecordType> internees = new HashMap<String, LogRecordType>();
+
+  final String name;
+
+  final int index;
+
+  private LogRecordType(String name) {
+    super();
+
+    this.name = name;
+
+    index = internees.size();
+  }
+
+  static LogRecordType intern(String typeName) {
+    LogRecordType result = internees.get(typeName);
+
+    if (result == null) {
+      result = new LogRecordType(typeName);
+
+      internees.put(typeName, result);
+    }
+
+    return result;
+  }
+
+  static LogRecordType internSoft(String typeName) {
+    return internees.get(typeName);
+  }
+
+  @Override
+  public String toString() {
+    return name;
+  }
+
+  static String[] lineTypes() {
+    Iterator<Map.Entry<String, LogRecordType>> iter = internees.entrySet()
+        .iterator();
+
+    String[] result = new String[internees.size()];
+
+    for (int i = 0; i < internees.size(); ++i) {
+      result[i] = iter.next().getKey();
+    }
+
+    return result;
+  }
+}

+ 144 - 0
src/tools/org/apache/hadoop/tools/rumen/LoggedDiscreteCDF.java

@@ -0,0 +1,144 @@
+/**
+ * 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.util.ArrayList;
+import java.util.List;
+
+/**
+ * A {@link LoggedDiscreteCDF} is a discrete approximation of a cumulative
+ * distribution function, with this class set up to meet the requirements of the
+ * Jackson JSON parser/generator.
+ * 
+ * All of the public methods are simply accessors for the instance variables we
+ * want to write out in the JSON files.
+ * 
+ */
+public class LoggedDiscreteCDF implements DeepCompare {
+  /**
+   * The number of values this CDF is built on
+   */
+  long numberValues = -1L;
+  /**
+   * The least {@code X} value
+   */
+  long minimum = Long.MIN_VALUE;
+  /**
+   * The coordinates of the bulk of the CDF
+   */
+  List<LoggedSingleRelativeRanking> rankings = new ArrayList<LoggedSingleRelativeRanking>();
+  /**
+   * The greatest {@code X} value
+   */
+  long maximum = Long.MAX_VALUE;
+
+  void setCDF(Histogram data, int[] steps, int modulus) {
+
+    numberValues = data.getTotalCount();
+    long[] CDF = data.getCDF(modulus, steps);
+
+    if (CDF != null) {
+      minimum = CDF[0];
+      maximum = CDF[CDF.length - 1];
+
+      rankings = new ArrayList<LoggedSingleRelativeRanking>();
+
+      for (int i = 1; i < CDF.length - 1; ++i) {
+        LoggedSingleRelativeRanking srr = new LoggedSingleRelativeRanking();
+
+        srr.setRelativeRanking(((double) steps[i - 1]) / modulus);
+        srr.setDatum(CDF[i]);
+
+        rankings.add(srr);
+      }
+    }
+  }
+
+  public long getMinimum() {
+    return minimum;
+  }
+
+  void setMinimum(long minimum) {
+    this.minimum = minimum;
+  }
+
+  public List<LoggedSingleRelativeRanking> getRankings() {
+    return rankings;
+  }
+
+  void setRankings(List<LoggedSingleRelativeRanking> rankings) {
+    this.rankings = rankings;
+  }
+
+  public long getMaximum() {
+    return maximum;
+  }
+
+  void setMaximum(long maximum) {
+    this.maximum = maximum;
+  }
+
+  public long getNumberValues() {
+    return numberValues;
+  }
+
+  void setNumberValues(long numberValues) {
+    this.numberValues = numberValues;
+  }
+
+  private void compare1(long c1, long c2, TreePath loc, String eltname)
+      throws DeepInequalityException {
+    if (c1 != c2) {
+      throw new DeepInequalityException(eltname + " miscompared", new TreePath(
+          loc, eltname));
+    }
+  }
+
+  private void compare1(List<LoggedSingleRelativeRanking> c1,
+      List<LoggedSingleRelativeRanking> c2, TreePath loc, String eltname)
+      throws DeepInequalityException {
+    if (c1 == null && c2 == null) {
+      return;
+    }
+
+    if (c1 == null || c2 == null || c1.size() != c2.size()) {
+      throw new DeepInequalityException(eltname + " miscompared", new TreePath(
+          loc, eltname));
+    }
+
+    for (int i = 0; i < c1.size(); ++i) {
+      c1.get(i).deepCompare(c2.get(i), new TreePath(loc, eltname, i));
+    }
+  }
+
+  public void deepCompare(DeepCompare comparand, TreePath loc)
+      throws DeepInequalityException {
+    if (!(comparand instanceof LoggedDiscreteCDF)) {
+      throw new DeepInequalityException("comparand has wrong type", loc);
+    }
+
+    LoggedDiscreteCDF other = (LoggedDiscreteCDF) comparand;
+
+    compare1(numberValues, other.numberValues, loc, "numberValues");
+
+    compare1(minimum, other.minimum, loc, "minimum");
+    compare1(maximum, other.maximum, loc, "maximum");
+
+    compare1(rankings, other.rankings, loc, "rankings");
+  }
+}

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

@@ -0,0 +1,586 @@
+/**
+ * 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.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import java.util.TreeSet;
+
+import org.codehaus.jackson.annotate.JsonAnySetter;
+
+/**
+ * A {@link LoggedDiscreteCDF} is a representation of an hadoop job, with the
+ * details of this class set up to meet the requirements of the Jackson JSON
+ * parser/generator.
+ * 
+ * All of the public methods are simply accessors for the instance variables we
+ * want to write out in the JSON files.
+ * 
+ */
+public class LoggedJob implements DeepCompare {
+  public enum JobType {
+    JAVA, PIG, STREAMING, PIPES, OVERALL
+  };
+
+  public enum JobPriority {
+    VERY_LOW, LOW, NORMAL, HIGH, VERY_HIGH
+  };
+
+  static private Set<String> alreadySeenAnySetterAttributes =
+      new TreeSet<String>();
+
+  String jobID;
+  String user;
+  long computonsPerMapInputByte = -1L;
+  long computonsPerMapOutputByte = -1L;
+  long computonsPerReduceInputByte = -1L;
+  long computonsPerReduceOutputByte = -1L;
+  long submitTime = -1L;
+  long launchTime = -1L;
+  long finishTime = -1L;
+
+  int heapMegabytes = -1;
+  int totalMaps = -1;
+  int totalReduces = -1;
+  Pre21JobHistoryConstants.Values outcome = null;
+  JobType jobtype = JobType.JAVA;
+  JobPriority priority = JobPriority.NORMAL;
+
+  List<String> directDependantJobs = new ArrayList<String>();
+  List<LoggedTask> mapTasks = new ArrayList<LoggedTask>();
+  List<LoggedTask> reduceTasks = new ArrayList<LoggedTask>();
+  List<LoggedTask> otherTasks = new ArrayList<LoggedTask>();
+
+  // There are CDFs for each level of locality -- most local first
+  ArrayList<LoggedDiscreteCDF> successfulMapAttemptCDFs;
+  // There are CDFs for each level of locality -- most local first
+  ArrayList<LoggedDiscreteCDF> failedMapAttemptCDFs;
+
+  LoggedDiscreteCDF successfulReduceAttemptCDF;
+  LoggedDiscreteCDF failedReduceAttemptCDF;
+
+  String queue = null;
+
+  String jobName = null;
+
+  int clusterMapMB = -1;
+  int clusterReduceMB = -1;
+  int jobMapMB = -1;
+  int jobReduceMB = -1;
+
+  long relativeTime = 0;
+
+  double[] mapperTriesToSucceed;
+  double failedMapperFraction; // !!!!!
+
+  LoggedJob() {
+
+  }
+
+  LoggedJob(String jobID) {
+    super();
+
+    setJobID(jobID);
+  }
+
+  @SuppressWarnings("unused")
+  // for input parameter ignored.
+  @JsonAnySetter
+  public void setUnknownAttribute(String attributeName, Object ignored) {
+    if (!alreadySeenAnySetterAttributes.contains(attributeName)) {
+      alreadySeenAnySetterAttributes.add(attributeName);
+      System.err.println("In LoggedJob, we saw the unknown attribute "
+          + attributeName + ".");
+    }
+  }
+
+  public String getUser() {
+    return user;
+  }
+
+  void setUser(String user) {
+    this.user = user;
+  }
+
+  public String getJobID() {
+    return jobID;
+  }
+
+  void setJobID(String jobID) {
+    this.jobID = jobID;
+  }
+
+  public JobPriority getPriority() {
+    return priority;
+  }
+
+  void setPriority(JobPriority priority) {
+    this.priority = priority;
+  }
+
+  public long getComputonsPerMapInputByte() {
+    return computonsPerMapInputByte;
+  }
+
+  void setComputonsPerMapInputByte(long computonsPerMapInputByte) {
+    this.computonsPerMapInputByte = computonsPerMapInputByte;
+  }
+
+  public long getComputonsPerMapOutputByte() {
+    return computonsPerMapOutputByte;
+  }
+
+  void setComputonsPerMapOutputByte(long computonsPerMapOutputByte) {
+    this.computonsPerMapOutputByte = computonsPerMapOutputByte;
+  }
+
+  public long getComputonsPerReduceInputByte() {
+    return computonsPerReduceInputByte;
+  }
+
+  void setComputonsPerReduceInputByte(long computonsPerReduceInputByte) {
+    this.computonsPerReduceInputByte = computonsPerReduceInputByte;
+  }
+
+  public long getComputonsPerReduceOutputByte() {
+    return computonsPerReduceOutputByte;
+  }
+
+  void setComputonsPerReduceOutputByte(long computonsPerReduceOutputByte) {
+    this.computonsPerReduceOutputByte = computonsPerReduceOutputByte; // !!!!!
+  }
+
+  public long getSubmitTime() {
+    return submitTime;
+  }
+
+  void setSubmitTime(long submitTime) {
+    this.submitTime = submitTime;
+  }
+
+  public long getLaunchTime() {
+    return launchTime;
+  }
+
+  void setLaunchTime(long startTime) {
+    this.launchTime = startTime;
+  }
+
+  public long getFinishTime() {
+    return finishTime;
+  }
+
+  void setFinishTime(long finishTime) {
+    this.finishTime = finishTime;
+  }
+
+  public int getHeapMegabytes() {
+    return heapMegabytes;
+  }
+
+  void setHeapMegabytes(int heapMegabytes) {
+    this.heapMegabytes = heapMegabytes;
+  }
+
+  public int getTotalMaps() {
+    return totalMaps;
+  }
+
+  void setTotalMaps(int totalMaps) {
+    this.totalMaps = totalMaps;
+  }
+
+  public int getTotalReduces() {
+    return totalReduces;
+  }
+
+  void setTotalReduces(int totalReduces) {
+    this.totalReduces = totalReduces;
+  }
+
+  public Pre21JobHistoryConstants.Values getOutcome() {
+    return outcome;
+  }
+
+  void setOutcome(Pre21JobHistoryConstants.Values outcome) {
+    this.outcome = outcome;
+  }
+
+  public JobType getJobtype() {
+    return jobtype;
+  }
+
+  void setJobtype(JobType jobtype) {
+    this.jobtype = jobtype;
+  }
+
+  public List<String> getDirectDependantJobs() {
+    return directDependantJobs;
+  }
+
+  void setDirectDependantJobs(List<String> directDependantJobs) {
+    this.directDependantJobs = directDependantJobs;
+  }
+
+  public List<LoggedTask> getMapTasks() {
+    return mapTasks;
+  }
+
+  void setMapTasks(List<LoggedTask> mapTasks) {
+    this.mapTasks = mapTasks;
+  }
+
+  public List<LoggedTask> getReduceTasks() {
+    return reduceTasks;
+  }
+
+  void setReduceTasks(List<LoggedTask> reduceTasks) {
+    this.reduceTasks = reduceTasks;
+  }
+
+  public List<LoggedTask> getOtherTasks() {
+    return otherTasks;
+  }
+
+  void setOtherTasks(List<LoggedTask> otherTasks) {
+    this.otherTasks = otherTasks;
+  }
+
+  public ArrayList<LoggedDiscreteCDF> getSuccessfulMapAttemptCDFs() {
+    return successfulMapAttemptCDFs;
+  }
+
+  void setSuccessfulMapAttemptCDFs(
+      ArrayList<LoggedDiscreteCDF> successfulMapAttemptCDFs) {
+    this.successfulMapAttemptCDFs = successfulMapAttemptCDFs;
+  }
+
+  public ArrayList<LoggedDiscreteCDF> getFailedMapAttemptCDFs() {
+    return failedMapAttemptCDFs;
+  }
+
+  void setFailedMapAttemptCDFs(ArrayList<LoggedDiscreteCDF> failedMapAttemptCDFs) {
+    this.failedMapAttemptCDFs = failedMapAttemptCDFs;
+  }
+
+  public LoggedDiscreteCDF getSuccessfulReduceAttemptCDF() {
+    return successfulReduceAttemptCDF;
+  }
+
+  void setSuccessfulReduceAttemptCDF(
+      LoggedDiscreteCDF successfulReduceAttemptCDF) {
+    this.successfulReduceAttemptCDF = successfulReduceAttemptCDF;
+  }
+
+  public LoggedDiscreteCDF getFailedReduceAttemptCDF() {
+    return failedReduceAttemptCDF;
+  }
+
+  void setFailedReduceAttemptCDF(LoggedDiscreteCDF failedReduceAttemptCDF) {
+    this.failedReduceAttemptCDF = failedReduceAttemptCDF;
+  }
+
+  public double[] getMapperTriesToSucceed() {
+    return mapperTriesToSucceed;
+  }
+
+  void setMapperTriesToSucceed(double[] mapperTriesToSucceed) {
+    this.mapperTriesToSucceed = mapperTriesToSucceed;
+  }
+
+  public double getFailedMapperFraction() {
+    return failedMapperFraction;
+  }
+
+  void setFailedMapperFraction(double failedMapperFraction) {
+    this.failedMapperFraction = failedMapperFraction;
+  }
+
+  public long getRelativeTime() {
+    return relativeTime;
+  }
+
+  void setRelativeTime(long relativeTime) {
+    this.relativeTime = relativeTime;
+  }
+
+  public String getQueue() {
+    return queue;
+  }
+
+  void setQueue(String queue) {
+    this.queue = queue;
+  }
+
+  public String getJobName() {
+    return jobName;
+  }
+
+  void setJobName(String jobName) {
+    this.jobName = jobName;
+  }
+
+  public int getClusterMapMB() {
+    return clusterMapMB;
+  }
+
+  void setClusterMapMB(int clusterMapMB) {
+    this.clusterMapMB = clusterMapMB;
+  }
+
+  public int getClusterReduceMB() {
+    return clusterReduceMB;
+  }
+
+  void setClusterReduceMB(int clusterReduceMB) {
+    this.clusterReduceMB = clusterReduceMB;
+  }
+
+  public int getJobMapMB() {
+    return jobMapMB;
+  }
+
+  void setJobMapMB(int jobMapMB) {
+    this.jobMapMB = jobMapMB;
+  }
+
+  public int getJobReduceMB() {
+    return jobReduceMB;
+  }
+
+  void setJobReduceMB(int jobReduceMB) {
+    this.jobReduceMB = jobReduceMB;
+  }
+
+  private void compare1(String c1, String c2, TreePath loc, String eltname)
+      throws DeepInequalityException {
+    if (c1 == null && c2 == null) {
+      return;
+    }
+
+    if (c1 == null || c2 == null || !c1.equals(c2)) {
+      throw new DeepInequalityException(eltname + " miscompared", new TreePath(
+          loc, eltname));
+    }
+  }
+
+  private void compare1(long c1, long c2, TreePath loc, String eltname)
+      throws DeepInequalityException {
+    if (c1 != c2) {
+      throw new DeepInequalityException(eltname + " miscompared", new TreePath(
+          loc, eltname));
+    }
+  }
+
+  private void compare1(Pre21JobHistoryConstants.Values c1,
+      Pre21JobHistoryConstants.Values c2, TreePath loc, String eltname)
+      throws DeepInequalityException {
+    if (c1 != c2) {
+      throw new DeepInequalityException(eltname + " miscompared", new TreePath(
+          loc, eltname));
+    }
+  }
+
+  private void compare1(JobType c1, JobType c2, TreePath loc, String eltname)
+      throws DeepInequalityException {
+    if (c1 != c2) {
+      throw new DeepInequalityException(eltname + " miscompared", new TreePath(
+          loc, eltname));
+    }
+  }
+
+  private void compare1(JobPriority c1, JobPriority c2, TreePath loc,
+      String eltname) throws DeepInequalityException {
+    if (c1 != c2) {
+      throw new DeepInequalityException(eltname + " miscompared", new TreePath(
+          loc, eltname));
+    }
+  }
+
+  private void compare1(int c1, int c2, TreePath loc, String eltname)
+      throws DeepInequalityException {
+    if (c1 != c2) {
+      throw new DeepInequalityException(eltname + " miscompared", new TreePath(
+          loc, eltname));
+    }
+  }
+
+  private void compare1(double c1, double c2, TreePath loc, String eltname)
+      throws DeepInequalityException {
+    if (c1 != c2) {
+      throw new DeepInequalityException(eltname + " miscompared", new TreePath(
+          loc, eltname));
+    }
+  }
+
+  private void compare1(double[] c1, double[] c2, TreePath loc, String eltname)
+      throws DeepInequalityException {
+    if (c1 == null && c2 == null) {
+      return;
+    }
+
+    TreePath recursePath = new TreePath(loc, eltname);
+
+    if (c1 == null || c2 == null || c1.length != c2.length) {
+      throw new DeepInequalityException(eltname + " miscompared", recursePath);
+    }
+
+    for (int i = 0; i < c1.length; ++i) {
+      if (c1[i] != c2[i]) {
+        throw new DeepInequalityException(eltname + " miscompared",
+            new TreePath(loc, eltname, i));
+      }
+    }
+  }
+
+  private void compare1(DeepCompare c1, DeepCompare c2, TreePath loc,
+      String eltname, int index) throws DeepInequalityException {
+    if (c1 == null && c2 == null) {
+      return;
+    }
+
+    TreePath recursePath = new TreePath(loc, eltname, index);
+
+    if (c1 == null || c2 == null) {
+      if (index == -1) {
+        throw new DeepInequalityException(eltname + " miscompared", recursePath);
+      } else {
+        throw new DeepInequalityException(eltname + "[" + index
+            + "] miscompared", recursePath);
+      }
+    }
+
+    c1.deepCompare(c2, recursePath);
+  }
+
+  // I'll treat this as an atomic object type
+  private void compareStrings(List<String> c1, List<String> c2, TreePath loc,
+      String eltname) throws DeepInequalityException {
+    if (c1 == null && c2 == null) {
+      return;
+    }
+
+    TreePath recursePath = new TreePath(loc, eltname);
+
+    if (c1 == null || c2 == null || !c1.equals(c2)) {
+      throw new DeepInequalityException(eltname + " miscompared", recursePath);
+    }
+  }
+
+  private void compareLoggedTasks(List<LoggedTask> c1, List<LoggedTask> c2,
+      TreePath loc, String eltname) throws DeepInequalityException {
+    if (c1 == null && c2 == null) {
+      return;
+    }
+
+    if (c1 == null || c2 == null || c1.size() != c2.size()) {
+      throw new DeepInequalityException(eltname + " miscompared", new TreePath(
+          loc, eltname));
+    }
+
+    for (int i = 0; i < c1.size(); ++i) {
+      c1.get(i).deepCompare(c2.get(i), new TreePath(loc, eltname, i));
+    }
+  }
+
+  private void compareCDFs(List<LoggedDiscreteCDF> c1,
+      List<LoggedDiscreteCDF> c2, TreePath loc, String eltname)
+      throws DeepInequalityException {
+    if (c1 == null && c2 == null) {
+      return;
+    }
+
+    if (c1 == null || c2 == null || c1.size() != c2.size()) {
+      throw new DeepInequalityException(eltname + " miscompared", new TreePath(
+          loc, eltname));
+    }
+
+    for (int i = 0; i < c1.size(); ++i) {
+      c1.get(i).deepCompare(c2.get(i), new TreePath(loc, eltname, i));
+    }
+  }
+
+  public void deepCompare(DeepCompare comparand, TreePath loc)
+      throws DeepInequalityException {
+    if (!(comparand instanceof LoggedJob)) {
+      throw new DeepInequalityException("comparand has wrong type", loc);
+    }
+
+    LoggedJob other = (LoggedJob) comparand;
+
+    compare1(jobID, other.jobID, loc, "jobID");
+    compare1(user, other.user, loc, "user");
+
+    compare1(computonsPerMapInputByte, other.computonsPerMapInputByte, loc,
+        "computonsPerMapInputByte");
+    compare1(computonsPerMapOutputByte, other.computonsPerMapOutputByte, loc,
+        "computonsPerMapOutputByte");
+    compare1(computonsPerReduceInputByte, other.computonsPerReduceInputByte,
+        loc, "computonsPerReduceInputByte");
+    compare1(computonsPerReduceOutputByte, other.computonsPerReduceOutputByte,
+        loc, "computonsPerReduceOutputByte");
+
+    compare1(submitTime, other.submitTime, loc, "submitTime");
+    compare1(launchTime, other.launchTime, loc, "launchTime");
+    compare1(finishTime, other.finishTime, loc, "finishTime");
+
+    compare1(heapMegabytes, other.heapMegabytes, loc, "heapMegabytes");
+
+    compare1(totalMaps, other.totalMaps, loc, "totalMaps");
+    compare1(totalReduces, other.totalReduces, loc, "totalReduces");
+
+    compare1(outcome, other.outcome, loc, "outcome");
+    compare1(jobtype, other.jobtype, loc, "jobtype");
+    compare1(priority, other.priority, loc, "priority");
+
+    compareStrings(directDependantJobs, other.directDependantJobs, loc,
+        "directDependantJobs");
+
+    compareLoggedTasks(mapTasks, other.mapTasks, loc, "mapTasks");
+    compareLoggedTasks(reduceTasks, other.reduceTasks, loc, "reduceTasks");
+    compareLoggedTasks(otherTasks, other.otherTasks, loc, "otherTasks");
+
+    compare1(relativeTime, other.relativeTime, loc, "relativeTime");
+
+    compareCDFs(successfulMapAttemptCDFs, other.successfulMapAttemptCDFs, loc,
+        "successfulMapAttemptCDFs");
+    compareCDFs(failedMapAttemptCDFs, other.failedMapAttemptCDFs, loc,
+        "failedMapAttemptCDFs");
+    compare1(successfulReduceAttemptCDF, other.successfulReduceAttemptCDF, loc,
+        "successfulReduceAttemptCDF", -1);
+    compare1(failedReduceAttemptCDF, other.failedReduceAttemptCDF, loc,
+        "failedReduceAttemptCDF", -1);
+
+    compare1(mapperTriesToSucceed, other.mapperTriesToSucceed, loc,
+        "mapperTriesToSucceed");
+    compare1(failedMapperFraction, other.failedMapperFraction, loc,
+        "failedMapperFraction");
+
+    compare1(queue, other.queue, loc, "queue");
+    compare1(jobName, other.jobName, loc, "jobName");
+
+    compare1(clusterMapMB, other.clusterMapMB, loc, "clusterMapMB");
+    compare1(clusterReduceMB, other.clusterReduceMB, loc, "clusterReduceMB");
+    compare1(jobMapMB, other.jobMapMB, loc, "jobMapMB");
+    compare1(jobReduceMB, other.jobReduceMB, loc, "jobReduceMB");
+  }
+}

+ 98 - 0
src/tools/org/apache/hadoop/tools/rumen/LoggedLocation.java

@@ -0,0 +1,98 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.tools.rumen;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import java.util.TreeSet;
+
+import org.codehaus.jackson.annotate.JsonAnySetter;
+
+/**
+ * A {@link LoggedLocation} is a representation of a point in an hierarchical
+ * network, represented as a series of membership names, broadest first.
+ * 
+ * For example, if your network has <i>hosts</i> grouped into <i>racks</i>, then
+ * in onecluster you might have a node {@code node1} on rack {@code rack1}. This
+ * would be represented with a ArrayList of two layers, with two {@link String}
+ * s being {@code "rack1"} and {@code "node1"}.
+ * 
+ * The details of this class are set up to meet the requirements of the Jackson
+ * JSON parser/generator.
+ * 
+ * All of the public methods are simply accessors for the instance variables we
+ * want to write out in the JSON files.
+ * 
+ */
+public class LoggedLocation implements DeepCompare {
+  /**
+   * The full path from the root of the network to the host.
+   * 
+   * NOTE that this assumes that the network topology is a tree.
+   */
+  List<String> layers = new ArrayList<String>();
+
+  static private Set<String> alreadySeenAnySetterAttributes =
+      new TreeSet<String>();
+
+  public List<String> getLayers() {
+    return layers;
+  }
+
+  void setLayers(List<String> layers) {
+    this.layers = layers;
+  }
+
+  @SuppressWarnings("unused")
+  // for input parameter ignored.
+  @JsonAnySetter
+  public void setUnknownAttribute(String attributeName, Object ignored) {
+    if (!alreadySeenAnySetterAttributes.contains(attributeName)) {
+      alreadySeenAnySetterAttributes.add(attributeName);
+      System.err.println("In LoggedJob, we saw the unknown attribute "
+          + attributeName + ".");
+    }
+  }
+
+  // I'll treat this as an atomic object type
+  private void compareStrings(List<String> c1, List<String> c2, TreePath loc,
+      String eltname) throws DeepInequalityException {
+    if (c1 == null && c2 == null) {
+      return;
+    }
+
+    TreePath recursePath = new TreePath(loc, eltname);
+
+    if (c1 == null || c2 == null || !c1.equals(c2)) {
+      throw new DeepInequalityException(eltname + " miscompared", recursePath);
+    }
+  }
+
+  public void deepCompare(DeepCompare comparand, TreePath loc)
+      throws DeepInequalityException {
+    if (!(comparand instanceof LoggedLocation)) {
+      throw new DeepInequalityException("comparand has wrong type", loc);
+    }
+
+    LoggedLocation other = (LoggedLocation) comparand;
+
+    compareStrings(layers, other.layers, loc, "layers");
+
+  }
+}

+ 168 - 0
src/tools/org/apache/hadoop/tools/rumen/LoggedNetworkTopology.java

@@ -0,0 +1,168 @@
+/**
+ * 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.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.ArrayList;
+import java.util.Comparator;
+
+import org.codehaus.jackson.annotate.JsonAnySetter;
+
+/**
+ * A {@link LoggedNetworkTopology} represents a tree that in turn represents a
+ * hierarchy of hosts. The current version requires the tree to have all leaves
+ * at the same level.
+ * 
+ * All of the public methods are simply accessors for the instance variables we
+ * want to write out in the JSON files.
+ * 
+ */
+public class LoggedNetworkTopology implements DeepCompare {
+  String name;
+  List<LoggedNetworkTopology> children = new ArrayList<LoggedNetworkTopology>();
+
+  static private Set<String> alreadySeenAnySetterAttributes =
+      new TreeSet<String>();
+
+  public LoggedNetworkTopology() {
+    super();
+  }
+
+  @SuppressWarnings("unused")
+  // for input parameter ignored.
+  @JsonAnySetter
+  public void setUnknownAttribute(String attributeName, Object ignored) {
+    if (!alreadySeenAnySetterAttributes.contains(attributeName)) {
+      alreadySeenAnySetterAttributes.add(attributeName);
+      System.err.println("In LoggedJob, we saw the unknown attribute "
+          + attributeName + ".");
+    }
+  }
+
+  /**
+   * We need this because we have to sort the {@code children} field. That field
+   * is set-valued, but if we sort these fields we ensure that comparisons won't
+   * bogusly fail because the hash table happened to enumerate in a different
+   * order.
+   * 
+   */
+  static class TopoSort implements Comparator<LoggedNetworkTopology> {
+    public int compare(LoggedNetworkTopology t1, LoggedNetworkTopology t2) {
+      return t1.name.compareTo(t2.name);
+    }
+  }
+
+  /**
+   * @param hosts
+   *          a HashSet of the {@link ParsedHost}
+   * @param name
+   *          the name of this level's host [for recursive descent]
+   * @param level
+   *          the level number
+   */
+  LoggedNetworkTopology(HashSet<ParsedHost> hosts, String name, int level) {
+
+    this.name = name;
+    this.children = null;
+
+    if (level < ParsedHost.numberOfDistances() - 1) {
+      HashMap<String, HashSet<ParsedHost>> topologies =
+          new HashMap<String, HashSet<ParsedHost>>();
+
+      Iterator<ParsedHost> iter = hosts.iterator();
+
+      while (iter.hasNext()) {
+        ParsedHost host = iter.next();
+
+        String thisComponent = host.nameComponent(level);
+
+        HashSet<ParsedHost> thisSet = topologies.get(thisComponent);
+
+        if (thisSet == null) {
+          thisSet = new HashSet<ParsedHost>();
+          topologies.put(thisComponent, thisSet);
+        }
+
+        thisSet.add(host);
+      }
+
+      children = new ArrayList<LoggedNetworkTopology>();
+
+      for (Map.Entry<String, HashSet<ParsedHost>> ent : topologies.entrySet()) {
+        children.add(new LoggedNetworkTopology(ent.getValue(), ent.getKey(),
+            level + 1));
+      }
+    } else {
+      // nothing to do here
+    }
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  void setName(String name) {
+    this.name = name;
+  }
+
+  public List<LoggedNetworkTopology> getChildren() {
+    return children;
+  }
+
+  void setChildren(List<LoggedNetworkTopology> children) {
+    this.children = children;
+  }
+
+  private void compare1(List<LoggedNetworkTopology> c1,
+      List<LoggedNetworkTopology> c2, TreePath loc, String eltname)
+      throws DeepInequalityException {
+    if (c1 == null && c2 == null) {
+      return;
+    }
+
+    if (c1 == null || c2 == null || c1.size() != c2.size()) {
+      throw new DeepInequalityException(eltname + " miscompared", new TreePath(
+          loc, eltname));
+    }
+
+    Collections.sort(c1, new TopoSort());
+    Collections.sort(c2, new TopoSort());
+
+    for (int i = 0; i < c1.size(); ++i) {
+      c1.get(i).deepCompare(c2.get(i), new TreePath(loc, eltname, i));
+    }
+  }
+
+  public void deepCompare(DeepCompare comparand, TreePath loc)
+      throws DeepInequalityException {
+    if (!(comparand instanceof LoggedNetworkTopology)) {
+      throw new DeepInequalityException("comparand has wrong type", loc);
+    }
+
+    LoggedNetworkTopology other = (LoggedNetworkTopology) comparand;
+
+    compare1(children, other.children, loc, "children");
+  }
+}

+ 101 - 0
src/tools/org/apache/hadoop/tools/rumen/LoggedSingleRelativeRanking.java

@@ -0,0 +1,101 @@
+/**
+ * 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.util.Set;
+import java.util.TreeSet;
+
+import org.codehaus.jackson.annotate.JsonAnySetter;
+
+/**
+ * A {@link LoggedSingleRelativeRanking} represents an X-Y coordinate of a
+ * single point in a discrete CDF.
+ * 
+ * All of the public methods are simply accessors for the instance variables we
+ * want to write out in the JSON files.
+ * 
+ */
+public class LoggedSingleRelativeRanking implements DeepCompare {
+  /**
+   * The Y coordinate, as a fraction {@code ( 0.0D, 1.0D )}. The default value
+   * is there to mark an unfilled-in value.
+   */
+  double relativeRanking = -1.0D;
+  /**
+   * The X coordinate
+   */
+  long datum = -1L;
+
+  static private Set<String> alreadySeenAnySetterAttributes =
+      new TreeSet<String>();
+
+  @SuppressWarnings("unused")
+  // for input parameter ignored.
+  @JsonAnySetter
+  public void setUnknownAttribute(String attributeName, Object ignored) {
+    if (!alreadySeenAnySetterAttributes.contains(attributeName)) {
+      alreadySeenAnySetterAttributes.add(attributeName);
+      System.err.println("In LoggedJob, we saw the unknown attribute "
+          + attributeName + ".");
+    }
+  }
+
+  public double getRelativeRanking() {
+    return relativeRanking;
+  }
+
+  void setRelativeRanking(double relativeRanking) {
+    this.relativeRanking = relativeRanking;
+  }
+
+  public long getDatum() {
+    return datum;
+  }
+
+  void setDatum(long datum) {
+    this.datum = datum;
+  }
+
+  private void compare1(long c1, long c2, TreePath loc, String eltname)
+      throws DeepInequalityException {
+    if (c1 != c2) {
+      throw new DeepInequalityException(eltname + " miscompared", new TreePath(
+          loc, eltname));
+    }
+  }
+
+  private void compare1(double c1, double c2, TreePath loc, String eltname)
+      throws DeepInequalityException {
+    if (c1 != c2) {
+      throw new DeepInequalityException(eltname + " miscompared", new TreePath(
+          loc, eltname));
+    }
+  }
+
+  public void deepCompare(DeepCompare comparand, TreePath loc)
+      throws DeepInequalityException {
+    if (!(comparand instanceof LoggedSingleRelativeRanking)) {
+      throw new DeepInequalityException("comparand has wrong type", loc);
+    }
+
+    LoggedSingleRelativeRanking other = (LoggedSingleRelativeRanking) comparand;
+
+    compare1(relativeRanking, other.relativeRanking, loc, "relativeRanking");
+    compare1(datum, other.datum, loc, "datum");
+  }
+}

+ 266 - 0
src/tools/org/apache/hadoop/tools/rumen/LoggedTask.java

@@ -0,0 +1,266 @@
+/**
+ * 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.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import java.util.TreeSet;
+
+import org.codehaus.jackson.annotate.JsonAnySetter;
+
+/**
+ * A {@link LoggedTask} represents a [hadoop] task that is part of a hadoop job.
+ * It knows about the [pssibly empty] sequence of attempts, its I/O footprint,
+ * and its runtime.
+ * 
+ * All of the public methods are simply accessors for the instance variables we
+ * want to write out in the JSON files.
+ * 
+ */
+public class LoggedTask implements DeepCompare {
+  long inputBytes = -1L;
+  long inputRecords = -1L;
+  long outputBytes = -1L;
+  long outputRecords = -1L;
+  String taskID;
+  long startTime = -1L;
+  long finishTime = -1L;
+  Pre21JobHistoryConstants.Values taskType;
+  Pre21JobHistoryConstants.Values taskStatus;
+  List<LoggedTaskAttempt> attempts = new ArrayList<LoggedTaskAttempt>();
+
+  ArrayList<LoggedLocation> preferredLocations =
+      new ArrayList<LoggedLocation>();
+
+  int numberMaps = -1;
+  int numberReduces = -1;
+
+  static private Set<String> alreadySeenAnySetterAttributes =
+      new TreeSet<String>();
+
+  @SuppressWarnings("unused")
+  // for input parameter ignored.
+  @JsonAnySetter
+  public void setUnknownAttribute(String attributeName, Object ignored) {
+    if (!alreadySeenAnySetterAttributes.contains(attributeName)) {
+      alreadySeenAnySetterAttributes.add(attributeName);
+      System.err.println("In LoggedJob, we saw the unknown attribute "
+          + attributeName + ".");
+    }
+  }
+
+  LoggedTask() {
+    super();
+  }
+
+  public long getInputBytes() {
+    return inputBytes;
+  }
+
+  void setInputBytes(long inputBytes) {
+    this.inputBytes = inputBytes;
+  }
+
+  public long getInputRecords() {
+    return inputRecords;
+  }
+
+  void setInputRecords(long inputRecords) {
+    this.inputRecords = inputRecords;
+  }
+
+  public long getOutputBytes() {
+    return outputBytes;
+  }
+
+  void setOutputBytes(long outputBytes) {
+    this.outputBytes = outputBytes;
+  }
+
+  public long getOutputRecords() {
+    return outputRecords;
+  }
+
+  void setOutputRecords(long outputRecords) {
+    this.outputRecords = outputRecords;
+  }
+
+  public String getTaskID() {
+    return taskID;
+  }
+
+  void setTaskID(String taskID) {
+    this.taskID = taskID;
+  }
+
+  public long getStartTime() {
+    return startTime;
+  }
+
+  void setStartTime(long startTime) {
+    this.startTime = startTime;
+  }
+
+  public long getFinishTime() {
+    return finishTime;
+  }
+
+  void setFinishTime(long finishTime) {
+    this.finishTime = finishTime;
+  }
+
+  public List<LoggedTaskAttempt> getAttempts() {
+    return attempts;
+  }
+
+  void setAttempts(List<LoggedTaskAttempt> attempts) {
+    this.attempts = attempts;
+  }
+
+  public ArrayList<LoggedLocation> getPreferredLocations() {
+    return preferredLocations;
+  }
+
+  void setPreferredLocations(ArrayList<LoggedLocation> preferredLocations) {
+    this.preferredLocations = preferredLocations;
+  }
+
+  public int getNumberMaps() {
+    return numberMaps;
+  }
+
+  void setNumberMaps(int numberMaps) {
+    this.numberMaps = numberMaps;
+  }
+
+  public int getNumberReduces() {
+    return numberReduces;
+  }
+
+  void setNumberReduces(int numberReduces) {
+    this.numberReduces = numberReduces;
+  }
+
+  public Pre21JobHistoryConstants.Values getTaskStatus() {
+    return taskStatus;
+  }
+
+  void setTaskStatus(Pre21JobHistoryConstants.Values taskStatus) {
+    this.taskStatus = taskStatus;
+  }
+
+  public Pre21JobHistoryConstants.Values getTaskType() {
+    return taskType;
+  }
+
+  void setTaskType(Pre21JobHistoryConstants.Values taskType) {
+    this.taskType = taskType;
+  }
+
+  private void compare1(long c1, long c2, TreePath loc, String eltname)
+      throws DeepInequalityException {
+    if (c1 != c2) {
+      throw new DeepInequalityException(eltname + " miscompared", new TreePath(
+          loc, eltname));
+    }
+  }
+
+  private void compare1(String c1, String c2, TreePath loc, String eltname)
+      throws DeepInequalityException {
+    if (c1 == null && c2 == null) {
+      return;
+    }
+    if (c1 == null || c2 == null || !c1.equals(c2)) {
+      throw new DeepInequalityException(eltname + " miscompared", new TreePath(
+          loc, eltname));
+    }
+  }
+
+  private void compare1(Pre21JobHistoryConstants.Values c1,
+      Pre21JobHistoryConstants.Values c2, TreePath loc, String eltname)
+      throws DeepInequalityException {
+    if (c1 == null && c2 == null) {
+      return;
+    }
+    if (c1 == null || c2 == null || !c1.equals(c2)) {
+      throw new DeepInequalityException(eltname + " miscompared", new TreePath(
+          loc, eltname));
+    }
+  }
+
+  private void compareLoggedLocations(ArrayList<LoggedLocation> c1,
+      ArrayList<LoggedLocation> c2, TreePath loc, String eltname)
+      throws DeepInequalityException {
+    if (c1 == null && c2 == null) {
+      return;
+    }
+
+    if (c1 == null || c2 == null || c1.size() != c2.size()) {
+      throw new DeepInequalityException(eltname + " miscompared", new TreePath(
+          loc, eltname));
+    }
+
+    for (int i = 0; i < c1.size(); ++i) {
+      c1.get(i).deepCompare(c2.get(i), new TreePath(loc, eltname, i));
+    }
+  }
+
+  private void compareLoggedTaskAttempts(List<LoggedTaskAttempt> c1,
+      List<LoggedTaskAttempt> c2, TreePath loc, String eltname)
+      throws DeepInequalityException {
+    if (c1 == null && c2 == null) {
+      return;
+    }
+
+    if (c1 == null || c2 == null || c1.size() != c2.size()) {
+      throw new DeepInequalityException(eltname + " miscompared", new TreePath(
+          loc, eltname));
+    }
+
+    for (int i = 0; i < c1.size(); ++i) {
+      c1.get(i).deepCompare(c2.get(i), new TreePath(loc, eltname, i));
+    }
+  }
+
+  public void deepCompare(DeepCompare comparand, TreePath loc)
+      throws DeepInequalityException {
+    if (!(comparand instanceof LoggedTask)) {
+      throw new DeepInequalityException("comparand has wrong type", loc);
+    }
+
+    LoggedTask other = (LoggedTask) comparand;
+
+    compare1(inputBytes, other.inputBytes, loc, "inputBytes");
+    compare1(inputRecords, other.inputRecords, loc, "inputRecords");
+    compare1(outputBytes, other.outputBytes, loc, "outputBytes");
+    compare1(outputRecords, other.outputRecords, loc, "outputRecords");
+
+    compare1(taskID, other.taskID, loc, "taskID");
+
+    compare1(startTime, other.startTime, loc, "startTime");
+    compare1(finishTime, other.finishTime, loc, "finishTime");
+
+    compare1(taskType, other.taskType, loc, "taskType");
+    compare1(taskStatus, other.taskStatus, loc, "taskStatus");
+
+    compareLoggedTaskAttempts(attempts, other.attempts, loc, "attempts");
+    compareLoggedLocations(preferredLocations, other.preferredLocations, loc,
+        "preferredLocations");
+  }
+}

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

@@ -0,0 +1,344 @@
+/**
+ * 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.util.Set;
+import java.util.TreeSet;
+
+import org.codehaus.jackson.annotate.JsonAnySetter;
+
+// HACK ALERT!!!  This "should" have have two subclasses, which might be called
+//                LoggedMapTaskAttempt and LoggedReduceTaskAttempt, but 
+//                the Jackson implementation of JSON doesn't handle a 
+//                superclass-valued field.
+
+/**
+ * A {@link LoggedTaskAttempt} represents an attempt to run an hadoop task in a
+ * hadoop job. Note that a task can have several attempts.
+ * 
+ * All of the public methods are simply accessors for the instance variables we
+ * want to write out in the JSON files.
+ * 
+ */
+public class LoggedTaskAttempt implements DeepCompare {
+
+  String attemptID;
+  Pre21JobHistoryConstants.Values result;
+  long startTime = -1L;
+  long finishTime = -1L;
+  String hostName;
+
+  long hdfsBytesRead = -1L;
+  long hdfsBytesWritten = -1L;
+  long fileBytesRead = -1L;
+  long fileBytesWritten = -1L;
+  long mapInputRecords = -1L;
+  long mapInputBytes = -1L;
+  long mapOutputBytes = -1L;
+  long mapOutputRecords = -1L;
+  long combineInputRecords = -1L;
+  long reduceInputGroups = -1L;
+  long reduceInputRecords = -1L;
+  long reduceShuffleBytes = -1L;
+  long reduceOutputRecords = -1L;
+  long spilledRecords = -1L;
+
+  long shuffleFinished = -1L;
+  long sortFinished = -1L;
+
+  LoggedLocation location;
+
+  LoggedTaskAttempt() {
+    super();
+  }
+
+  static private Set<String> alreadySeenAnySetterAttributes =
+      new TreeSet<String>();
+
+  @SuppressWarnings("unused")
+  // for input parameter ignored.
+  @JsonAnySetter
+  public void setUnknownAttribute(String attributeName, Object ignored) {
+    if (!alreadySeenAnySetterAttributes.contains(attributeName)) {
+      alreadySeenAnySetterAttributes.add(attributeName);
+      System.err.println("In LoggedJob, we saw the unknown attribute "
+          + attributeName + ".");
+    }
+  }
+
+  public long getShuffleFinished() {
+    return shuffleFinished;
+  }
+
+  void setShuffleFinished(long shuffleFinished) {
+    this.shuffleFinished = shuffleFinished;
+  }
+
+  public long getSortFinished() {
+    return sortFinished;
+  }
+
+  void setSortFinished(long sortFinished) {
+    this.sortFinished = sortFinished;
+  }
+
+  public String getAttemptID() {
+    return attemptID;
+  }
+
+  void setAttemptID(String attemptID) {
+    this.attemptID = attemptID;
+  }
+
+  public Pre21JobHistoryConstants.Values getResult() {
+    return result;
+  }
+
+  void setResult(Pre21JobHistoryConstants.Values result) {
+    this.result = result;
+  }
+
+  public long getStartTime() {
+    return startTime;
+  }
+
+  void setStartTime(long startTime) {
+    this.startTime = startTime;
+  }
+
+  public long getFinishTime() {
+    return finishTime;
+  }
+
+  void setFinishTime(long finishTime) {
+    this.finishTime = finishTime;
+  }
+
+  public String getHostName() {
+    return hostName;
+  }
+
+  void setHostName(String hostName) {
+    this.hostName = hostName;
+  }
+
+  public long getHdfsBytesRead() {
+    return hdfsBytesRead;
+  }
+
+  void setHdfsBytesRead(long hdfsBytesRead) {
+    this.hdfsBytesRead = hdfsBytesRead;
+  }
+
+  public long getHdfsBytesWritten() {
+    return hdfsBytesWritten;
+  }
+
+  void setHdfsBytesWritten(long hdfsBytesWritten) {
+    this.hdfsBytesWritten = hdfsBytesWritten;
+  }
+
+  public long getFileBytesRead() {
+    return fileBytesRead;
+  }
+
+  void setFileBytesRead(long fileBytesRead) {
+    this.fileBytesRead = fileBytesRead;
+  }
+
+  public long getFileBytesWritten() {
+    return fileBytesWritten;
+  }
+
+  void setFileBytesWritten(long fileBytesWritten) {
+    this.fileBytesWritten = fileBytesWritten;
+  }
+
+  public long getMapInputRecords() {
+    return mapInputRecords;
+  }
+
+  void setMapInputRecords(long mapInputRecords) {
+    this.mapInputRecords = mapInputRecords;
+  }
+
+  public long getMapOutputBytes() {
+    return mapOutputBytes;
+  }
+
+  void setMapOutputBytes(long mapOutputBytes) {
+    this.mapOutputBytes = mapOutputBytes;
+  }
+
+  public long getMapOutputRecords() {
+    return mapOutputRecords;
+  }
+
+  void setMapOutputRecords(long mapOutputRecords) {
+    this.mapOutputRecords = mapOutputRecords;
+  }
+
+  public long getCombineInputRecords() {
+    return combineInputRecords;
+  }
+
+  void setCombineInputRecords(long combineInputRecords) {
+    this.combineInputRecords = combineInputRecords;
+  }
+
+  public long getReduceInputGroups() {
+    return reduceInputGroups;
+  }
+
+  void setReduceInputGroups(long reduceInputGroups) {
+    this.reduceInputGroups = reduceInputGroups;
+  }
+
+  public long getReduceInputRecords() {
+    return reduceInputRecords;
+  }
+
+  void setReduceInputRecords(long reduceInputRecords) {
+    this.reduceInputRecords = reduceInputRecords;
+  }
+
+  public long getReduceShuffleBytes() {
+    return reduceShuffleBytes;
+  }
+
+  void setReduceShuffleBytes(long reduceShuffleBytes) {
+    this.reduceShuffleBytes = reduceShuffleBytes;
+  }
+
+  public long getReduceOutputRecords() {
+    return reduceOutputRecords;
+  }
+
+  void setReduceOutputRecords(long reduceOutputRecords) {
+    this.reduceOutputRecords = reduceOutputRecords;
+  }
+
+  public long getSpilledRecords() {
+    return spilledRecords;
+  }
+
+  void setSpilledRecords(long spilledRecords) {
+    this.spilledRecords = spilledRecords;
+  }
+
+  public LoggedLocation getLocation() {
+    return location;
+  }
+
+  void setLocation(LoggedLocation location) {
+    this.location = location;
+  }
+
+  public long getMapInputBytes() {
+    return mapInputBytes;
+  }
+
+  void setMapInputBytes(long mapInputBytes) {
+    this.mapInputBytes = mapInputBytes;
+  }
+
+  private void compare1(String c1, String c2, TreePath loc, String eltname)
+      throws DeepInequalityException {
+    if (c1 == null && c2 == null) {
+      return;
+    }
+
+    if (c1 == null || c2 == null || !c1.equals(c2)) {
+      throw new DeepInequalityException(eltname + " miscompared", new TreePath(
+          loc, eltname));
+    }
+  }
+
+  private void compare1(long c1, long c2, TreePath loc, String eltname)
+      throws DeepInequalityException {
+    if (c1 != c2) {
+      throw new DeepInequalityException(eltname + " miscompared", new TreePath(
+          loc, eltname));
+    }
+  }
+
+  private void compare1(Pre21JobHistoryConstants.Values c1,
+      Pre21JobHistoryConstants.Values c2, TreePath loc, String eltname)
+      throws DeepInequalityException {
+    if (c1 != c2) {
+      throw new DeepInequalityException(eltname + " miscompared", new TreePath(
+          loc, eltname));
+    }
+  }
+
+  private void compare1(LoggedLocation c1, LoggedLocation c2, TreePath loc,
+      String eltname) throws DeepInequalityException {
+    if (c1 == null && c2 == null) {
+      return;
+    }
+
+    TreePath recurse = new TreePath(loc, eltname);
+
+    if (c1 == null || c2 == null) {
+      throw new DeepInequalityException(eltname + " miscompared", recurse);
+    }
+
+    c1.deepCompare(c2, recurse);
+  }
+
+  public void deepCompare(DeepCompare comparand, TreePath loc)
+      throws DeepInequalityException {
+    if (!(comparand instanceof LoggedTaskAttempt)) {
+      throw new DeepInequalityException("comparand has wrong type", loc);
+    }
+
+    LoggedTaskAttempt other = (LoggedTaskAttempt) comparand;
+
+    compare1(attemptID, other.attemptID, loc, "attemptID");
+    compare1(result, other.result, loc, "result");
+    compare1(startTime, other.startTime, loc, "startTime");
+    compare1(finishTime, other.finishTime, loc, "finishTime");
+    compare1(hostName, other.hostName, loc, "hostName");
+
+    compare1(hdfsBytesRead, other.hdfsBytesRead, loc, "hdfsBytesRead");
+    compare1(hdfsBytesWritten, other.hdfsBytesWritten, loc, "hdfsBytesWritten");
+    compare1(fileBytesRead, other.fileBytesRead, loc, "fileBytesRead");
+    compare1(fileBytesWritten, other.fileBytesWritten, loc, "fileBytesWritten");
+    compare1(mapInputBytes, other.mapInputBytes, loc, "mapInputBytes");
+    compare1(mapInputRecords, other.mapInputRecords, loc, "mapInputRecords");
+    compare1(mapOutputBytes, other.mapOutputBytes, loc, "mapOutputBytes");
+    compare1(mapOutputRecords, other.mapOutputRecords, loc, "mapOutputRecords");
+    compare1(combineInputRecords, other.combineInputRecords, loc,
+        "combineInputRecords");
+    compare1(reduceInputGroups, other.reduceInputGroups, loc,
+        "reduceInputGroups");
+    compare1(reduceInputRecords, other.reduceInputRecords, loc,
+        "reduceInputRecords");
+    compare1(reduceShuffleBytes, other.reduceShuffleBytes, loc,
+        "reduceShuffleBytes");
+    compare1(reduceOutputRecords, other.reduceOutputRecords, loc,
+        "reduceOutputRecords");
+    compare1(spilledRecords, other.spilledRecords, loc, "spilledRecords");
+
+    compare1(shuffleFinished, other.shuffleFinished, loc, "shuffleFinished");
+    compare1(sortFinished, other.sortFinished, loc, "sortFinished");
+
+    compare1(location, other.location, loc, "location");
+  }
+}

+ 205 - 0
src/tools/org/apache/hadoop/tools/rumen/MachineNode.java

@@ -0,0 +1,205 @@
+/**
+ * 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;
+
+/**
+ * {@link MachineNode} represents the configuration of a cluster node.
+ * {@link MachineNode} should be constructed by {@link MachineNode.Builder}.
+ */
+public final class MachineNode extends Node {
+  long memory = -1; // in KB
+  int mapSlots = 1;
+  int reduceSlots = 1;
+  long memoryPerMapSlot = -1; // in KB
+  long memoryPerReduceSlot = -1; // in KB
+  int numCores = 1;
+  
+  MachineNode(String name, int level) {
+    super(name, level);
+  }
+  
+  @Override
+  public boolean equals(Object obj) {
+    // name/level sufficient
+    return super.equals(obj);
+  }
+
+  @Override
+  public int hashCode() {
+    // match equals
+    return super.hashCode();
+  }
+
+  /**
+   * Get the available physical RAM of the node.
+   * @return The available physical RAM of the node, in KB.
+   */
+  public long getMemory() {
+    return memory;
+  }
+  
+  /**
+   * Get the number of map slots of the node.
+   * @return The number of map slots of the node.
+   */
+  public int getMapSlots() {
+    return mapSlots;
+  }
+  
+  /**
+   * Get the number of reduce slots of the node.
+   * @return The number of reduce slots fo the node.
+   */
+  public int getReduceSlots() {
+    return reduceSlots;
+  }
+  
+  /**
+   * Get the amount of RAM reserved for each map slot.
+   * @return the amount of RAM reserved for each map slot, in KB.
+   */
+  public long getMemoryPerMapSlot() {
+    return memoryPerMapSlot;
+  }
+
+  /**
+   * Get the amount of RAM reserved for each reduce slot.
+   * @return the amount of RAM reserved for each reduce slot, in KB.
+   */
+  public long getMemoryPerReduceSlot() {
+    return memoryPerReduceSlot;
+  }
+  
+  /**
+   * Get the number of cores of the node.
+   * @return the number of cores of the node.
+   */
+  public int getNumCores() {
+    return numCores;
+  }
+
+  /**
+   * Get the rack node that the machine belongs to.
+   * 
+   * @return The rack node that the machine belongs to. Returns null if the
+   *         machine does not belong to any rack.
+   */
+  public RackNode getRackNode() {
+    return (RackNode)getParent();
+  }
+  
+  @Override
+  public synchronized boolean addChild(Node child) {
+    throw new IllegalStateException("Cannot add child to MachineNode");
+  }
+
+  /**
+   * Builder for a NodeInfo object
+   */
+  public static final class Builder {
+    private MachineNode node;
+    
+    /**
+     * Start building a new NodeInfo object.
+     * @param name
+     *          Unique name of the node. Typically the fully qualified domain
+     *          name.
+     */
+    public Builder(String name, int level) {
+      node = new MachineNode(name, level);
+    }
+
+    /**
+     * Set the physical memory of the node.
+     * @param memory Available RAM in KB.
+     */
+    public Builder setMemory(long memory) {
+      node.memory = memory;
+      return this;
+    }
+    
+    /**
+     * Set the number of map slot for the node.
+     * @param mapSlots The number of map slots for the node.
+     */
+    public Builder setMapSlots(int mapSlots) {
+      node.mapSlots = mapSlots;
+      return this;
+    }
+    
+    /**
+     * Set the number of reduce slot for the node.
+     * @param reduceSlots The number of reduce slots for the node.
+     */   
+    public Builder setReduceSlots(int reduceSlots) {
+      node.reduceSlots = reduceSlots;
+      return this;
+    }
+    
+    /**
+     * Set the amount of RAM reserved for each map slot.
+     * @param memoryPerMapSlot The amount of RAM reserved for each map slot, in KB.
+     */
+    public Builder setMemoryPerMapSlot(long memoryPerMapSlot) {
+      node.memoryPerMapSlot = memoryPerMapSlot;
+      return this;
+    }
+    
+    /**
+     * Set the amount of RAM reserved for each reduce slot.
+     * @param memoryPerReduceSlot The amount of RAM reserved for each reduce slot, in KB.
+     */
+    public Builder setMemoryPerReduceSlot(long memoryPerReduceSlot) {
+      node.memoryPerReduceSlot = memoryPerReduceSlot;
+      return this;
+    }
+    
+    /**
+     * Set the number of cores for the node.
+     * @param numCores Number of cores for the node.
+     */
+    public Builder setNumCores(int numCores) {
+      node.numCores = numCores;
+      return this;
+    }
+    
+    /**
+     * Clone the settings from a reference {@link MachineNode} object.
+     * @param ref The reference {@link MachineNode} object.
+     */
+    public Builder cloneFrom(MachineNode ref) {
+      node.memory = ref.memory;
+      node.mapSlots = ref.mapSlots;
+      node.reduceSlots = ref.reduceSlots;
+      node.memoryPerMapSlot = ref.memoryPerMapSlot;
+      node.memoryPerReduceSlot = ref.memoryPerReduceSlot;
+      node.numCores = ref.numCores;
+      return this;
+    }
+    
+    /**
+     * Build the {@link MachineNode} object.
+     * @return The {@link MachineNode} object being built.
+     */
+    public MachineNode build() {
+      MachineNode retVal = node;
+      node = null;
+      return retVal;
+    }
+  }
+}

+ 48 - 0
src/tools/org/apache/hadoop/tools/rumen/MapTaskAttemptInfo.java

@@ -0,0 +1,48 @@
+/**
+ * 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 org.apache.hadoop.mapred.TaskStatus.State;
+
+/**
+ * {@link MapTaskAttemptInfo} represents the information with regard to a
+ * map task attempt.
+ */
+public class MapTaskAttemptInfo extends TaskAttemptInfo {
+  private long runtime;
+
+  public MapTaskAttemptInfo(State state, TaskInfo taskInfo, long runtime) {
+    super(state, taskInfo);
+    this.runtime = runtime;
+  }
+
+  @Override
+  public long getRuntime() {
+    return getMapRuntime();
+  }
+
+  /**
+   * Get the runtime for the <b>map</b> phase of the map-task attempt.
+   * 
+   * @return the runtime for the <b>map</b> phase of the map-task attempt
+   */
+  public long getMapRuntime() {
+    return runtime;
+  }
+
+}

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

@@ -0,0 +1,148 @@
+/**
+ * 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.util.Collections;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
+
+/**
+ * {@link Node} represents a node in the cluster topology. A node can be a
+ * {@MachineNode}, or a {@link RackNode}, etc.
+ */
+public class Node implements Comparable<Node> {
+  private static final SortedSet<Node> EMPTY_SET = 
+    Collections.unmodifiableSortedSet(new TreeSet<Node>());
+  private Node parent;
+  private final String name;
+  private final int level;
+  private SortedSet<Node> children;
+
+  /**
+   * @param name
+   *          A unique name to identify a node in the cluster.
+   * @param level
+   *          The level of the node in the cluster
+   */
+  public Node(String name, int level) {
+    if (name == null) {
+      throw new IllegalArgumentException("Node name cannot be null");
+    }
+
+    if (level < 0) {
+      throw new IllegalArgumentException("Level cannot be negative");
+    }
+
+    this.name = name;
+    this.level = level;
+  }
+
+  /**
+   * Get the name of the node.
+   * 
+   * @return The name of the node.
+   */
+  public String getName() {
+    return name;
+  }
+
+  /**
+   * Get the level of the node.
+   * @return The level of the node.
+   */
+  public int getLevel() {
+    return level;
+  }
+  
+  private void checkChildren() {
+    if (children == null) {
+      children = new TreeSet<Node>();
+    }
+  }
+
+  /**
+   * Add a child node to this node.
+   * @param child The child node to be added. The child node should currently not be belong to another cluster topology.
+   * @return Boolean indicating whether the node is successfully added.
+   */
+  public synchronized boolean addChild(Node child) {
+    if (child.parent != null) {
+      throw new IllegalArgumentException(
+          "The child is already under another node:" + child.parent);
+    }
+    checkChildren();
+    boolean retval = children.add(child);
+    if (retval) child.parent = this;
+    return retval;
+  }
+
+  /**
+   * Does this node have any children?
+   * @return Boolean indicate whether this node has any children.
+   */
+  public synchronized boolean hasChildren() {
+    return children != null && !children.isEmpty();
+  }
+
+  /**
+   * Get the children of this node.
+   * 
+   * @return The children of this node. If no child, an empty set will be
+   *         returned. The returned set is read-only.
+   */
+  public synchronized Set<Node> getChildren() {
+    return (children == null) ? EMPTY_SET : 
+      Collections.unmodifiableSortedSet(children);
+  }
+  
+  /**
+   * Get the parent node.
+   * @return the parent node. If root node, return null.
+   */
+  public Node getParent() {
+    return parent;
+  }
+  
+  @Override
+  public int hashCode() {
+    return name.hashCode();
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj)
+      return true;
+    if (obj == null)
+      return false;
+    if (obj.getClass() != this.getClass())
+      return false;
+    Node other = (Node) obj;
+    return name.equals(other.name);
+  }
+  
+  @Override
+  public String toString() {
+    return "(" + name +", " + level +")";
+  }
+
+  @Override
+  public int compareTo(Node o) {
+    return name.compareTo(o.name);
+  }
+}

+ 38 - 0
src/tools/org/apache/hadoop/tools/rumen/Pair.java

@@ -0,0 +1,38 @@
+/**
+ * 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;
+
+class Pair<CarType, CdrType> {
+  private final CarType car;
+  private final CdrType cdr;
+
+  Pair(CarType car, CdrType cdr) {
+    super();
+
+    this.car = car;
+    this.cdr = cdr;
+  }
+
+  CarType first() {
+    return car;
+  }
+
+  CdrType second() {
+    return cdr;
+  }
+}

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

@@ -0,0 +1,197 @@
+/**
+ * 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.util.regex.Pattern;
+import java.util.regex.Matcher;
+
+import java.io.InputStream;
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+
+import javax.xml.parsers.DocumentBuilderFactory;
+import javax.xml.parsers.DocumentBuilder;
+import javax.xml.parsers.ParserConfigurationException;
+
+import org.w3c.dom.Document;
+import org.w3c.dom.NodeList;
+import org.w3c.dom.Node;
+import org.w3c.dom.Element;
+import org.w3c.dom.Text;
+
+import org.xml.sax.SAXException;
+
+class ParsedConfigFile {
+  private static final Pattern jobIDPattern = Pattern.compile("_(job_[0-9]+_[0-9]+)_");
+  private static final Pattern heapPattern = Pattern.compile("-Xmx([0-9]+)([mMgG])");
+
+  final int heapMegabytes;
+
+  final String queue;
+  final String jobName;
+
+  final int clusterMapMB;
+  final int clusterReduceMB;
+  final int jobMapMB;
+  final int jobReduceMB;
+
+  final String jobID;
+
+  final boolean valid;
+
+  private int maybeGetIntValue(String propName, String attr, String value,
+      int oldValue) {
+    if (propName.equals(attr) && value != null) {
+      try {
+        return Integer.parseInt(value);
+      } catch (NumberFormatException e) {
+        return oldValue;
+      }
+    }
+
+    return oldValue;
+  }
+
+  @SuppressWarnings("hiding")
+  ParsedConfigFile(String filenameLine, String xmlString) {
+    super();
+
+    int heapMegabytes = -1;
+
+    String queue = null;
+    String jobName = null;
+
+    int clusterMapMB = -1;
+    int clusterReduceMB = -1;
+    int jobMapMB = -1;
+    int jobReduceMB = -1;
+
+    String jobID = null;
+
+    boolean valid = true;
+
+    Matcher jobIDMatcher = jobIDPattern.matcher(filenameLine);
+
+    if (jobIDMatcher.find()) {
+      jobID = jobIDMatcher.group(1);
+    }
+
+    try {
+      InputStream is = new ByteArrayInputStream(xmlString.getBytes());
+
+      DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
+
+      DocumentBuilder db = dbf.newDocumentBuilder();
+
+      Document doc = db.parse(is);
+
+      Element root = doc.getDocumentElement();
+
+      if (!"configuration".equals(root.getTagName())) {
+        System.out.print("root is not a configuration node");
+        valid = false;
+      }
+
+      NodeList props = root.getChildNodes();
+
+      for (int i = 0; i < props.getLength(); ++i) {
+        Node propNode = props.item(i);
+        if (!(propNode instanceof Element))
+          continue;
+        Element prop = (Element) propNode;
+        if (!"property".equals(prop.getTagName())) {
+          System.out.print("bad conf file: element not <property>");
+        }
+        NodeList fields = prop.getChildNodes();
+        String attr = null;
+        String value = null;
+        @SuppressWarnings("unused")
+        boolean finalParameter = false;
+        for (int j = 0; j < fields.getLength(); j++) {
+          Node fieldNode = fields.item(j);
+          if (!(fieldNode instanceof Element)) {
+            continue;
+          }
+
+          Element field = (Element) fieldNode;
+          if ("name".equals(field.getTagName()) && field.hasChildNodes()) {
+            attr = ((Text) field.getFirstChild()).getData().trim();
+          }
+          if ("value".equals(field.getTagName()) && field.hasChildNodes()) {
+            value = ((Text) field.getFirstChild()).getData();
+          }
+          if ("final".equals(field.getTagName()) && field.hasChildNodes()) {
+            finalParameter = "true".equals(((Text) field.getFirstChild())
+                .getData());
+          }
+        }
+        if ("mapred.child.java.opts".equals(attr) && value != null) {
+          Matcher matcher = heapPattern.matcher(value);
+          if (matcher.find()) {
+            String heapSize = matcher.group(1);
+
+            heapMegabytes = Integer.parseInt(heapSize);
+
+            if (matcher.group(2).equalsIgnoreCase("G")) {
+              heapMegabytes *= 1024;
+            }
+          }
+        }
+
+        if ("mapred.job.queue.name".equals(attr) && value != null) {
+          queue = value;
+        }
+
+        if ("mapred.job.name".equals(attr) && value != null) {
+          jobName = value;
+        }
+
+        clusterMapMB = maybeGetIntValue("mapred.cluster.map.memory.mb", attr,
+            value, clusterMapMB);
+        clusterReduceMB = maybeGetIntValue("mapred.cluster.reduce.memory.mb",
+            attr, value, clusterReduceMB);
+        jobMapMB = maybeGetIntValue("mapred.job.map.memory.mb", attr, value,
+            jobMapMB);
+        jobReduceMB = maybeGetIntValue("mapred.job.reduce.memory.mb", attr,
+            value, jobReduceMB);
+      }
+
+      valid = true;
+    } catch (ParserConfigurationException e) {
+      valid = false;
+    } catch (SAXException e) {
+      valid = false;
+    } catch (IOException e) {
+      valid = false;
+    }
+
+    this.heapMegabytes = heapMegabytes;
+
+    this.queue = queue;
+    this.jobName = jobName;
+
+    this.clusterMapMB = clusterMapMB;
+    this.clusterReduceMB = clusterReduceMB;
+    this.jobMapMB = jobMapMB;
+    this.jobReduceMB = jobReduceMB;
+
+    this.jobID = jobID;
+
+    this.valid = valid;
+  }
+}

+ 126 - 0
src/tools/org/apache/hadoop/tools/rumen/ParsedHost.java

@@ -0,0 +1,126 @@
+/**
+ * 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.util.ArrayList;
+import java.util.List;
+import java.util.regex.Pattern;
+import java.util.regex.Matcher;
+
+class ParsedHost {
+  private final String rackName;
+  private final String nodeName;
+
+  /**
+   * TODO the following only works for /rack/host format. Change to support
+   * arbitrary level of network names.
+   */
+  private static final Pattern splitPattern = Pattern
+      .compile("/([^/]+)/([^/]+)");
+
+  /**
+   * TODO handle arbitrary level of network names.
+   */
+  static int numberOfDistances() {
+    return 3;
+  }
+
+  String nameComponent(int i) throws IllegalArgumentException {
+    switch (i) {
+    case 0:
+      return rackName;
+
+    case 1:
+      return nodeName;
+
+    default:
+      throw new IllegalArgumentException(
+          "Host location component index out of range.");
+    }
+  }
+
+  @Override
+  public int hashCode() {
+    return rackName.hashCode() * 17 + nodeName.hashCode();
+  }
+
+  public static ParsedHost parse(String name) {
+    // separate out the node name
+    Matcher matcher = splitPattern.matcher(name);
+
+    if (!matcher.matches())
+      return null;
+
+    return new ParsedHost(matcher.group(1), matcher.group(2));
+  }
+
+  public ParsedHost(LoggedLocation loc) {
+    List<String> coordinates = loc.getLayers();
+
+    rackName = coordinates.get(0);
+    nodeName = coordinates.get(1);
+  }
+
+  LoggedLocation makeLoggedLocation() {
+    LoggedLocation result = new LoggedLocation();
+
+    List<String> coordinates = new ArrayList<String>();
+
+    coordinates.add(rackName);
+    coordinates.add(nodeName);
+
+    result.setLayers(coordinates);
+
+    return result;
+  }
+  
+  String getNodeName() {
+    return nodeName;
+  }
+  
+  String getRackName() {
+    return rackName;
+  }
+
+  // expects the broadest name first
+  ParsedHost(String rackName, String nodeName) {
+    this.rackName = rackName;
+    this.nodeName = nodeName;
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (!(other instanceof ParsedHost)) {
+      return false;
+    }
+    ParsedHost host = (ParsedHost) other;
+    return (nodeName.equals(host.nodeName) && rackName.equals(host.rackName));
+  }
+
+  int distance(ParsedHost other) {
+    if (nodeName.equals(other.nodeName)) {
+      return 0;
+    }
+
+    if (rackName.equals(other.rackName)) {
+      return 1;
+    }
+
+    return 2;
+  }
+}

+ 117 - 0
src/tools/org/apache/hadoop/tools/rumen/ParsedLine.java

@@ -0,0 +1,117 @@
+/**
+ * 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.util.Properties;
+import java.util.regex.Pattern;
+
+class ParsedLine {
+  Properties content;
+  LogRecordType type;
+
+  static final Pattern keyValPair = Pattern
+      .compile(" *([a-zA-Z0-9_]+)=\"((?:[^\"\\\\]|\\\\[ .\"\\\\])*)\"");
+
+  @SuppressWarnings("unused") 
+  ParsedLine(String fullLine, int version) {
+    super();
+
+    content = new Properties();
+
+    int firstSpace = fullLine.indexOf(" ");
+
+    if (firstSpace < 0) {
+      firstSpace = fullLine.length();
+    }
+
+    if (firstSpace == 0) {
+      return; // This is a junk line of some sort
+    }
+
+    type = LogRecordType.intern(fullLine.substring(0, firstSpace));
+
+    String propValPairs = fullLine.substring(firstSpace + 1);
+
+    while (propValPairs.length() > 0 && propValPairs.charAt(0) == ' ') {
+      propValPairs = propValPairs.substring(1);
+    }
+
+    int cursor = 0;
+
+    while (cursor < propValPairs.length()) {
+      int equals = propValPairs.indexOf('=', cursor);
+
+      if (equals < 0) {
+        // maybe we do some error processing
+        return;
+      }
+
+      int nextCursor;
+
+      int endValue;
+
+      if (propValPairs.charAt(equals + 1) == '\"') {
+        int closeQuote = propValPairs.indexOf('\"', equals + 2);
+
+        nextCursor = closeQuote + 1;
+
+        endValue = closeQuote;
+
+        if (closeQuote < 0) {
+          endValue = propValPairs.length();
+
+          nextCursor = endValue;
+        }
+      } else {
+        int closeSpace = propValPairs.indexOf(' ', equals + 1);
+
+        if (closeSpace < 0) {
+          closeSpace = propValPairs.length();
+        }
+
+        endValue = closeSpace;
+
+        nextCursor = endValue;
+      }
+
+      content.setProperty(propValPairs.substring(cursor, equals), propValPairs
+          .substring(equals + 2, endValue));
+
+      cursor = nextCursor;
+
+      while (cursor < propValPairs.length()
+          && propValPairs.charAt(cursor) == ' ') {
+        ++cursor;
+      }
+    }
+  }
+
+  protected LogRecordType getType() {
+    return type;
+  }
+
+  protected String get(String key) {
+    return content.getProperty(key);
+  }
+
+  protected long getLong(String key) {
+    String val = get(key);
+
+    return Long.parseLong(val);
+  }
+}

+ 49 - 0
src/tools/org/apache/hadoop/tools/rumen/Pre21JobHistoryConstants.java

@@ -0,0 +1,49 @@
+/**
+ * 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;
+
+/**
+ * 
+ *
+ */
+public class Pre21JobHistoryConstants {
+  
+  /**
+   * Job history files contain key="value" pairs, where keys belong to this enum. 
+   * It acts as a global namespace for all keys. 
+   */
+  static enum Keys {
+    JOBTRACKERID,
+    START_TIME, FINISH_TIME, JOBID, JOBNAME, USER, JOBCONF, SUBMIT_TIME,
+    LAUNCH_TIME, TOTAL_MAPS, TOTAL_REDUCES, FAILED_MAPS, FAILED_REDUCES,
+    FINISHED_MAPS, FINISHED_REDUCES, JOB_STATUS, TASKID, HOSTNAME, TASK_TYPE,
+    ERROR, TASK_ATTEMPT_ID, TASK_STATUS, COPY_PHASE, SORT_PHASE, REDUCE_PHASE,
+    SHUFFLE_FINISHED, SORT_FINISHED, MAP_FINISHED, COUNTERS, SPLITS,
+    JOB_PRIORITY, HTTP_PORT, TRACKER_NAME, STATE_STRING, VERSION
+  }
+
+  /**
+   * This enum contains some of the values commonly used by history log events. 
+   * since values in history can only be strings - Values.name() is used in 
+   * most places in history file. 
+   */
+  public static enum Values {
+    SUCCESS, FAILED, KILLED, MAP, REDUCE, CLEANUP, RUNNING, PREP, SETUP
+  }
+ 
+}

+ 48 - 0
src/tools/org/apache/hadoop/tools/rumen/RackNode.java

@@ -0,0 +1,48 @@
+/**
+ * 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.util.Set;
+
+/**
+ * {@link RackNode} represents a rack node in the cluster topology.
+ */
+public final class RackNode extends Node {
+  public RackNode(String name, int level) {
+    // Hack: ensuring rack name starts with "/".
+    super(name.startsWith("/") ? name : "/" + name, level);
+  }
+  
+  @Override
+  public synchronized boolean addChild(Node child) {
+    if (!(child instanceof MachineNode)) {
+      throw new IllegalArgumentException(
+          "Only MachineNode can be added to RackNode");
+    }
+    return super.addChild(child);
+  }
+  
+  /**
+   * Get the machine nodes that belong to the rack.
+   * @return The machine nodes that belong to the rack.
+   */
+  @SuppressWarnings({ "cast", "unchecked" })
+  public Set<MachineNode> getMachinesInRack() {
+    return (Set<MachineNode>)(Set)getChildren();
+  }
+}

+ 71 - 0
src/tools/org/apache/hadoop/tools/rumen/ReduceTaskAttemptInfo.java

@@ -0,0 +1,71 @@
+/**
+ * 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 org.apache.hadoop.mapred.TaskStatus.State;
+
+/**
+ * {@link ReduceTaskAttemptInfo} represents the information with regard to a
+ * reduce task attempt.
+ */
+public class ReduceTaskAttemptInfo extends TaskAttemptInfo {
+  private long shuffleTime;
+  private long mergeTime;
+  private long reduceTime;
+
+  public ReduceTaskAttemptInfo(State state, TaskInfo taskInfo, long shuffleTime,
+      long mergeTime, long reduceTime) {
+    super(state, taskInfo);
+    this.shuffleTime = shuffleTime;
+    this.mergeTime = mergeTime;
+    this.reduceTime = reduceTime;
+  }
+
+  /**
+   * Get the runtime for the <b>reduce</b> phase of the reduce task-attempt.
+   * 
+   * @return the runtime for the <b>reduce</b> phase of the reduce task-attempt
+   */
+  public long getReduceRuntime() {
+    return reduceTime;
+  }
+
+  /**
+   * Get the runtime for the <b>shuffle</b> phase of the reduce task-attempt.
+   * 
+   * @return the runtime for the <b>shuffle</b> phase of the reduce task-attempt
+   */
+  public long getShuffleRuntime() {
+    return shuffleTime;
+  }
+
+  /**
+   * Get the runtime for the <b>merge</b> phase of the reduce task-attempt
+   * 
+   * @return the runtime for the <b>merge</b> phase of the reduce task-attempt
+   */
+  public long getMergeRuntime() {
+    return mergeTime;
+  }
+
+  @Override
+  public long getRuntime() {
+    return (getShuffleRuntime() + getMergeRuntime() + getReduceRuntime());
+  }
+
+}

+ 64 - 0
src/tools/org/apache/hadoop/tools/rumen/TaskAttemptInfo.java

@@ -0,0 +1,64 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.tools.rumen;
+
+import org.apache.hadoop.mapred.TaskStatus;
+import org.apache.hadoop.mapred.TaskStatus.State;
+
+/**
+ * {@link TaskAttemptInfo} is a collection of statistics about a particular
+ * task-attempt gleaned from job-history of the job.
+ */
+public abstract class TaskAttemptInfo {
+  protected final State state;
+  protected final TaskInfo taskInfo;
+
+  protected TaskAttemptInfo(State state, TaskInfo taskInfo) {
+    if (state == State.SUCCEEDED || state == State.FAILED) {
+      this.state = state;
+    } else {
+      throw new IllegalArgumentException("status cannot be " + state);
+    }
+    this.taskInfo = taskInfo;
+  }
+
+  /**
+   * Get the final {@link TaskStatus.State} of the task-attempt.
+   * 
+   * @return the final <code>State</code> of the task-attempt
+   */
+  public State getRunState() {
+    return state;
+  }
+
+  /**
+   * Get the total runtime for the task-attempt.
+   * 
+   * @return the total runtime for the task-attempt
+   */
+  public abstract long getRuntime();
+
+  /**
+   * Get the {@link TaskInfo} for the given task-attempt.
+   * 
+   * @return the <code>TaskInfo</code> for the given task-attempt
+   */
+  public TaskInfo getTaskInfo() {
+    return taskInfo;
+  }
+}

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

@@ -0,0 +1,73 @@
+/**
+ * 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;
+
+public class TaskInfo {
+  private final long bytesIn;
+  private final int recsIn;
+  private final long bytesOut;
+  private final int recsOut;
+  private final long maxMemory;
+
+  public TaskInfo(long bytesIn, int recsIn, long bytesOut, int recsOut,
+      long maxMemory) {
+    this.bytesIn = bytesIn;
+    this.recsIn = recsIn;
+    this.bytesOut = bytesOut;
+    this.recsOut = recsOut;
+    this.maxMemory = maxMemory;
+  }
+
+  /**
+   * @return Raw bytes read from the FileSystem into the task. Note that this
+   *         may not always match the input bytes to the task.
+   */
+  public long getInputBytes() {
+    return bytesIn;
+  }
+
+  /**
+   * @return Number of records input to this task.
+   */
+  public int getInputRecords() {
+    return recsIn;
+  }
+
+  /**
+   * @return Raw bytes written to the destination FileSystem. Note that this may
+   *         not match output bytes.
+   */
+  public long getOutputBytes() {
+    return bytesOut;
+  }
+
+  /**
+   * @return Number of records output from this task.
+   */
+  public int getOutputRecords() {
+    return recsOut;
+  }
+
+  /**
+   * @return Memory used by the task leq the heap size.
+   */
+  public long getTaskMemory() {
+    return maxMemory;
+  }
+
+}

+ 57 - 0
src/tools/org/apache/hadoop/tools/rumen/TreePath.java

@@ -0,0 +1,57 @@
+/**
+ * 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;
+
+/**
+ * This describes a path from a node to the root. We use it when we compare two
+ * trees during rumen unit tests. If the trees are not identical, this chain
+ * will be converted to a string which describes the path from the root to the
+ * fields that did not compare.
+ * 
+ */
+public class TreePath {
+  final TreePath parent;
+
+  final String fieldName;
+
+  final int index;
+
+  public TreePath(TreePath parent, String fieldName) {
+    super();
+
+    this.parent = parent;
+    this.fieldName = fieldName;
+    this.index = -1;
+  }
+
+  public TreePath(TreePath parent, String fieldName, int index) {
+    super();
+
+    this.parent = parent;
+    this.fieldName = fieldName;
+    this.index = index;
+  }
+
+  @Override
+  public String toString() {
+    String mySegment = fieldName + (index == -1 ? "" : ("[" + index + "]"));
+
+    return ((parent == null) ? "" : parent.toString() + "-->") + mySegment;
+  }
+}

+ 149 - 0
src/tools/org/apache/hadoop/tools/rumen/ZombieCluster.java

@@ -0,0 +1,149 @@
+/**
+ * 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.IOException;
+import java.io.InputStream;
+import java.util.ArrayDeque;
+import java.util.Deque;
+import java.util.IdentityHashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+
+/**
+ * {@link ZombieCluster} rebuilds the cluster topology using the information
+ * obtained from job history logs.
+ */
+public class ZombieCluster extends AbstractClusterStory {
+  private Node root;
+
+  /**
+   * Construct a homogeneous cluster. We assume that the leaves on the topology
+   * are {@link MachineNode}s, and the parents of {@link MachineNode}s are
+   * {@link RackNode}s. We also expect all leaf nodes are on the same level.
+   * 
+   * @param topology
+   *          The network topology.
+   * @param defaultNode
+   *          The default node setting.
+   */
+  ZombieCluster(LoggedNetworkTopology topology, MachineNode defaultNode) {
+    buildCluster(topology, defaultNode);
+  }
+
+  /**
+   * Construct a homogeneous cluster. We assume that the leaves on the topology
+   * are {@link MachineNode}s, and the parents of {@link MachineNode}s are
+   * {@link RackNode}s. We also expect all leaf nodes are on the same level.
+   * 
+   * @param path Path to the JSON-encoded topology file.
+   * @param conf
+   * @param defaultNode
+   *          The default node setting.
+   * @throws IOException 
+   */
+  public ZombieCluster(Path path, MachineNode defaultNode, Configuration conf) throws IOException {
+    this(new ClusterTopologyReader(path, conf).get(), defaultNode);
+  }
+  
+  /**
+   * Construct a homogeneous cluster. We assume that the leaves on the topology
+   * are {@link MachineNode}s, and the parents of {@link MachineNode}s are
+   * {@link RackNode}s. We also expect all leaf nodes are on the same level.
+   * 
+   * @param input The input stream for the JSON-encoded topology file.
+   * @param defaultNode
+   *          The default node setting.
+   * @throws IOException 
+   */
+  public ZombieCluster(InputStream input, MachineNode defaultNode) throws IOException {
+    this(new ClusterTopologyReader(input).get(), defaultNode);
+  }
+
+  @Override
+  public Node getClusterTopology() {
+    return root;
+  }
+
+  private final void buildCluster(LoggedNetworkTopology topology,
+      MachineNode defaultNode) {
+    Map<LoggedNetworkTopology, Integer> levelMapping = 
+      new IdentityHashMap<LoggedNetworkTopology, Integer>();
+    Deque<LoggedNetworkTopology> unvisited = 
+      new ArrayDeque<LoggedNetworkTopology>();
+    unvisited.add(topology);
+    levelMapping.put(topology, 0);
+    
+    // building levelMapping and determine leafLevel
+    int leafLevel = -1; // -1 means leafLevel unknown.
+    for (LoggedNetworkTopology n = unvisited.poll(); n != null; 
+      n = unvisited.poll()) {
+      int level = levelMapping.get(n);
+      List<LoggedNetworkTopology> children = n.getChildren();
+      if (children == null || children.isEmpty()) {
+        if (leafLevel == -1) {
+          leafLevel = level;
+        } else if (leafLevel != level) {
+          throw new IllegalArgumentException(
+              "Leaf nodes are not on the same level");
+        }
+      } else {
+        for (LoggedNetworkTopology child : children) {
+          levelMapping.put(child, level + 1);
+          unvisited.addFirst(child);
+        }
+      }
+    }
+
+    /**
+     * A second-pass dfs traverse of topology tree. path[i] contains the parent
+     * of the node at level i+1.
+     */
+    Node[] path = new Node[leafLevel];
+    unvisited.add(topology);
+    for (LoggedNetworkTopology n = unvisited.poll(); n != null; 
+      n = unvisited.poll()) {
+      int level = levelMapping.get(n);
+      Node current;
+      if (level == leafLevel) { // a machine node
+        MachineNode.Builder builder = new MachineNode.Builder(n.getName(), level);
+        if (defaultNode != null) {
+          builder.cloneFrom(defaultNode);
+        }
+        current = builder.build();
+      } else {
+        current = (level == leafLevel - 1) 
+          ? new RackNode(n.getName(), level) : 
+            new Node(n.getName(), level);
+        path[level] = current;
+        // Add all children to the front of the queue.
+        for (LoggedNetworkTopology child : n.getChildren()) {
+          unvisited.addFirst(child);
+        }
+      }
+      if (level != 0) {
+        path[level - 1].addChild(current);
+      }
+    }
+
+    root = path[0];
+  }
+}

+ 880 - 0
src/tools/org/apache/hadoop/tools/rumen/ZombieJob.java

@@ -0,0 +1,880 @@
+/**
+ * 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.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.HashMap;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.TaskStatus.State;
+import org.apache.hadoop.mapreduce.InputSplit;
+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.mapreduce.lib.input.FileSplit;
+import org.apache.hadoop.tools.rumen.Pre21JobHistoryConstants.Values;
+
+/**
+ * {@link ZombieJob} is a layer above {@link LoggedJob} raw JSON objects.
+ * 
+ * Each {@link ZombieJob} object represents a job in job history. For everything
+ * that exists in job history, contents are returned unchanged faithfully. To
+ * get input splits of a non-exist task, a non-exist task attempt, or an
+ * ill-formed task attempt, proper objects are made up from statistical
+ * sketches.
+ */
+@SuppressWarnings("deprecation")
+public class ZombieJob implements JobStory {
+  static final Log LOG = LogFactory.getLog(ZombieJob.class);
+  private final LoggedJob job;
+  private Map<TaskID, LoggedTask> loggedTaskMap;
+  private Map<TaskAttemptID, LoggedTaskAttempt> loggedTaskAttemptMap;
+  private final Random random;
+  private InputSplit[] splits;
+  private final ClusterStory cluster;
+  private JobConf jobConf;
+
+  private long seed;
+  private boolean hasRandomSeed = false;
+
+  private Map<LoggedDiscreteCDF, CDFRandomGenerator> interpolatorMap =
+      new HashMap<LoggedDiscreteCDF, CDFRandomGenerator>();
+
+  // TODO: Fix ZombieJob to initialize this correctly from observed data
+  double rackLocalOverNodeLocal = 1.5;
+  double rackRemoteOverNodeLocal = 3.0;
+
+  /**
+   * This constructor creates a {@link ZombieJob} with the same semantics as the
+   * {@link LoggedJob} passed in this parameter
+   * 
+   * @param job
+   *          The dead job this ZombieJob instance is based on.
+   * @param cluster
+   *          The cluster topology where the dead job ran on. This argument can
+   *          be null if we do not have knowledge of the cluster topology.
+   * @param seed
+   *          Seed for the random number generator for filling in information
+   *          not available from the ZombieJob.
+   */
+  public ZombieJob(LoggedJob job, ClusterStory cluster, long seed) {
+    if (job == null) {
+      throw new IllegalArgumentException("job is null");
+    }
+    this.job = job;
+    this.cluster = cluster;
+    random = new Random(seed);
+    this.seed = seed;
+    hasRandomSeed = true;
+  }
+
+  /**
+   * This constructor creates a {@link ZombieJob} with the same semantics as the
+   * {@link LoggedJob} passed in this parameter
+   * 
+   * @param job
+   *          The dead job this ZombieJob instance is based on.
+   * @param cluster
+   *          The cluster topology where the dead job ran on. This argument can
+   *          be null if we do not have knowledge of the cluster topology.
+   */
+  public ZombieJob(LoggedJob job, ClusterStory cluster) {
+    this(job, cluster, System.nanoTime());
+  }
+
+  private static State convertState(Values status) {
+    if (status == Values.SUCCESS) {
+      return State.SUCCEEDED;
+    } else if (status == Values.FAILED) {
+      return State.FAILED;
+    } else if (status == Values.KILLED) {
+      return State.KILLED;
+    } else {
+      throw new IllegalArgumentException("unknown status " + status);
+    }
+  }
+
+  @Override
+  public synchronized JobConf getJobConf() {
+    if (jobConf == null) {
+      // TODO : add more to jobConf ?
+      jobConf = new JobConf();
+      jobConf.setJobName(getName());
+      jobConf.setUser(getUser());
+      jobConf.setNumMapTasks(getNumberMaps());
+      jobConf.setNumReduceTasks(getNumberReduces());
+    }
+    return jobConf;
+  }
+  
+  @Override
+  public InputSplit[] getInputSplits() {
+    if (splits == null) {
+      List<InputSplit> splitsList = new ArrayList<InputSplit>();
+      Path emptyPath = new Path("/");
+      int totalHosts = 0; // use to determine avg # of hosts per split.
+      for (LoggedTask mapTask : job.getMapTasks()) {
+        Pre21JobHistoryConstants.Values taskType = mapTask.getTaskType();
+        if (taskType != Pre21JobHistoryConstants.Values.MAP) {
+          LOG.warn("TaskType for a MapTask is not Map. task="
+              + mapTask.getTaskID() + " type="
+              + ((taskType == null) ? "null" : taskType.toString()));
+          continue;
+        }
+        List<LoggedLocation> locations = mapTask.getPreferredLocations();
+        List<String> hostList = new ArrayList<String>();
+        if (locations != null) {
+          for (LoggedLocation location : locations) {
+            List<String> layers = location.getLayers();
+            if (layers.size() == 0) {
+              LOG.warn("Bad location layer format for task "+mapTask.getTaskID());
+              continue;
+            }
+            String host = layers.get(layers.size() - 1);
+            if (host == null) {
+              LOG.warn("Bad location layer format for task "+mapTask.getTaskID() + ": " + layers);
+              continue;
+            }
+            hostList.add(host);
+          }
+        }
+        String[] hosts = hostList.toArray(new String[hostList.size()]);
+        totalHosts += hosts.length;
+        long mapInputBytes = getTaskInfo(mapTask).getInputBytes();
+        if (mapInputBytes < 0) {
+          LOG.warn("InputBytes for task "+mapTask.getTaskID()+" is not defined.");
+          mapInputBytes = 0;
+        }
+       
+        splitsList.add(new FileSplit(emptyPath, 0, mapInputBytes, hosts));
+      }
+
+      // If not all map tasks are in job trace, should make up some splits
+      // for missing map tasks.
+      int totalMaps = job.getTotalMaps();
+      if (totalMaps < splitsList.size()) {
+        LOG.warn("TotalMaps for job " + job.getJobID()
+            + " is less than the total number of map task descriptions ("
+            + totalMaps + "<" + splitsList.size() + ").");
+      }
+
+      int avgHostPerSplit;
+      if (splitsList.size() == 0) {
+        avgHostPerSplit = 3;
+      } else {
+        avgHostPerSplit = totalHosts / splitsList.size();
+        if (avgHostPerSplit == 0) {
+          avgHostPerSplit = 3;
+        }
+      }
+
+      for (int i = splitsList.size(); i < totalMaps; i++) {
+        if (cluster == null) {
+          splitsList.add(new FileSplit(emptyPath, 0, 0, new String[0]));
+        } else {
+          MachineNode[] mNodes = cluster.getRandomMachines(avgHostPerSplit);
+          String[] hosts = new String[mNodes.length];
+          for (int j = 0; j < hosts.length; ++j) {
+            hosts[j] = mNodes[j].getName();
+          }
+          // TODO set size of a split to 0 now.
+          splitsList.add(new FileSplit(emptyPath, 0, 0, hosts));
+        }
+      }
+
+      splits = splitsList.toArray(new InputSplit[splitsList.size()]);
+    }
+    return splits;
+  }
+
+  @Override
+  public String getName() {
+    String jobName = job.getJobName();
+    if (jobName == null) {
+      return "(name unknown)";
+    } else {
+      return jobName;
+    }
+  }
+
+  @Override
+  public JobID getJobID() {
+    return JobID.forName(getLoggedJob().getJobID());
+  }
+
+  private int sanitizeValue(int oldVal, int defaultVal, String name, String id) {
+    if (oldVal == -1) {
+      LOG.warn(name +" not defined for "+id);
+      return defaultVal;
+    }
+    return oldVal;
+  }
+  
+  @Override
+  public int getNumberMaps() {
+    return sanitizeValue(job.getTotalMaps(), 0, "NumberMaps", job.getJobID());
+  }
+
+  @Override
+  public int getNumberReduces() {
+    return sanitizeValue(job.getTotalReduces(), 0, "NumberReduces", job.getJobID());
+  }
+
+  @Override
+  public Values getOutcome() {
+    return job.getOutcome();
+  }
+
+  @Override
+  public long getSubmissionTime() {
+    return job.getSubmitTime() - job.getRelativeTime();
+  }
+
+  /**
+   * Getting the number of map tasks that are actually logged in the trace.
+   * @return The number of map tasks that are actually logged in the trace.
+   */
+  public int getNumLoggedMaps() {
+    return job.getMapTasks().size();
+  }
+
+
+  /**
+   * Getting the number of reduce tasks that are actually logged in the trace.
+   * @return The number of map tasks that are actually logged in the trace.
+   */
+  public int getNumLoggedReduces() {
+    return job.getReduceTasks().size();
+  }
+  
+  /**
+   * Mask the job ID part in a {@link TaskID}.
+   * 
+   * @param taskId
+   *          raw {@link TaskID} read from trace
+   * @return masked {@link TaskID} with empty {@link JobID}.
+   */
+  private TaskID maskTaskID(TaskID taskId) {
+    JobID jobId = new JobID();
+    return new TaskID(jobId, taskId.isMap(), taskId.getId());
+  }
+
+  /**
+   * Mask the job ID part in a {@link TaskAttemptID}.
+   * 
+   * @param attemptId
+   *          raw {@link TaskAttemptID} read from trace
+   * @return masked {@link TaskAttemptID} with empty {@link JobID}.
+   */
+  private TaskAttemptID maskAttemptID(TaskAttemptID attemptId) {
+    JobID jobId = new JobID();
+    TaskID taskId = attemptId.getTaskID();
+    return new TaskAttemptID(jobId.getJtIdentifier(), jobId.getId(),
+        attemptId.isMap(), taskId.getId(), attemptId.getId());
+  }
+
+  private LoggedTask sanitizeLoggedTask(LoggedTask task) {
+    if (task == null) {
+      return null;
+    }
+    if (task.getTaskType() == null) {
+      LOG.warn("Task " + task.getTaskID() + " has nulll TaskType");
+      return null;
+    }
+    if (task.getTaskStatus() == null) {
+      LOG.warn("Task " + task.getTaskID() + " has nulll TaskStatus");
+      return null;
+    }
+    return task;
+  }
+
+  private LoggedTaskAttempt sanitizeLoggedTaskAttempt(LoggedTaskAttempt attempt) {
+    if (attempt == null) {
+      return null;
+    }
+    if (attempt.getResult() == null) {
+      LOG.warn("TaskAttempt " + attempt.getResult() + " has nulll Result");
+      return null;
+    }
+
+    return attempt;
+  }
+
+  /**
+   * Build task mapping and task attempt mapping, to be later used to find
+   * information of a particular {@link TaskID} or {@link TaskAttemptID}.
+   */
+  private synchronized void buildMaps() {
+    if (loggedTaskMap == null) {
+      loggedTaskMap = new HashMap<TaskID, LoggedTask>();
+      loggedTaskAttemptMap = new HashMap<TaskAttemptID, LoggedTaskAttempt>();
+      
+      for (LoggedTask map : job.getMapTasks()) {
+        map = sanitizeLoggedTask(map);
+        if (map != null) {
+          loggedTaskMap.put(maskTaskID(TaskID.forName(map.taskID)), map);
+
+          for (LoggedTaskAttempt mapAttempt : map.getAttempts()) {
+            mapAttempt = sanitizeLoggedTaskAttempt(mapAttempt);
+            if (mapAttempt != null) {
+              TaskAttemptID id = TaskAttemptID.forName(mapAttempt
+                  .getAttemptID());
+              loggedTaskAttemptMap.put(maskAttemptID(id), mapAttempt);
+            }
+          }
+        }
+      }
+      for (LoggedTask reduce : job.getReduceTasks()) {
+        reduce = sanitizeLoggedTask(reduce);
+        if (reduce != null) {
+          loggedTaskMap.put(maskTaskID(TaskID.forName(reduce.taskID)), reduce);
+
+          for (LoggedTaskAttempt reduceAttempt : reduce.getAttempts()) {
+            reduceAttempt = sanitizeLoggedTaskAttempt(reduceAttempt);
+            if (reduceAttempt != null) {
+              TaskAttemptID id = TaskAttemptID.forName(reduceAttempt
+                  .getAttemptID());
+              loggedTaskAttemptMap.put(maskAttemptID(id), reduceAttempt);
+            }
+          }
+        }
+      }
+
+      // TODO: do not care about "other" tasks, "setup" or "clean"
+    }
+  }
+
+  @Override
+  public String getUser() {
+    String retval = job.getUser();
+    return (retval==null)?"(unknown)":retval;
+  }
+
+  /**
+   * Get the underlining {@link LoggedJob} object read directly from the trace.
+   * This is mainly for debugging.
+   * 
+   * @return the underlining {@link LoggedJob} object
+   */
+  public LoggedJob getLoggedJob() {
+    return job;
+  }
+
+  /**
+   * Get a {@link TaskAttemptInfo} with a {@link TaskAttemptID} associated with
+   * taskType, taskNumber, and taskAttemptNumber. This function does not care
+   * about locality, and follows the following decision logic: 1. Make up a
+   * {@link TaskAttemptInfo} if the task attempt is missing in trace, 2. Make up
+   * a {@link TaskAttemptInfo} if the task attempt has a KILLED final status in
+   * trace, 3. Otherwise (final state is SUCCEEDED or FAILED), construct the
+   * {@link TaskAttemptInfo} from the trace.
+   */
+  public TaskAttemptInfo getTaskAttemptInfo(TaskType taskType, int taskNumber,
+      int taskAttemptNumber) {
+    // does not care about locality. assume default locality is NODE_LOCAL.
+    // But if both task and task attempt exist in trace, use logged locality.
+    int locality = 0;
+    LoggedTask loggedTask = getLoggedTask(taskType, taskNumber);
+    if (loggedTask == null) {
+      // TODO insert parameters
+      TaskInfo taskInfo = new TaskInfo(0, 0, 0, 0, 0);
+      return makeUpTaskAttemptInfo(taskType, taskInfo, taskAttemptNumber,
+          taskNumber, locality);
+    }
+
+    LoggedTaskAttempt loggedAttempt = getLoggedTaskAttempt(taskType,
+        taskNumber, taskAttemptNumber);
+    if (loggedAttempt == null) {
+      // Task exists, but attempt is missing.
+      TaskInfo taskInfo = getTaskInfo(loggedTask);
+      return makeUpTaskAttemptInfo(taskType, taskInfo, taskAttemptNumber,
+          taskNumber, locality);
+    } else {
+      // TODO should we handle killed attempts later?
+      if (loggedAttempt.getResult()== Values.KILLED) {
+        TaskInfo taskInfo = getTaskInfo(loggedTask);
+        return makeUpTaskAttemptInfo(taskType, taskInfo, taskAttemptNumber,
+            taskNumber, locality);
+      } else {
+        return getTaskAttemptInfo(loggedTask, loggedAttempt);
+      }
+    }
+  }
+
+  @Override
+  public TaskInfo getTaskInfo(TaskType taskType, int taskNumber) {
+    return getTaskInfo(getLoggedTask(taskType, taskNumber));
+  }
+
+  /**
+   * Get a {@link TaskAttemptInfo} with a {@link TaskAttemptID} associated with
+   * taskType, taskNumber, and taskAttemptNumber. This function considers
+   * locality, and follows the following decision logic: 1. Make up a
+   * {@link TaskAttemptInfo} if the task attempt is missing in trace, 2. Make up
+   * a {@link TaskAttemptInfo} if the task attempt has a KILLED final status in
+   * trace, 3. If final state is FAILED, construct a {@link TaskAttemptInfo}
+   * from the trace, without considering locality. 4. If final state is
+   * SUCCEEDED, construct a {@link TaskAttemptInfo} from the trace, with runtime
+   * scaled according to locality in simulation and locality in trace.
+   */
+  @Override
+  public TaskAttemptInfo getMapTaskAttemptInfoAdjusted(int taskNumber,
+      int taskAttemptNumber, int locality) {
+    TaskType taskType = TaskType.MAP;
+    LoggedTask loggedTask = getLoggedTask(taskType, taskNumber);
+    if (loggedTask == null) {
+      // TODO insert parameters
+      TaskInfo taskInfo = new TaskInfo(0, 0, 0, 0, 0);
+      return makeUpTaskAttemptInfo(taskType, taskInfo, taskAttemptNumber,
+          taskNumber, locality);
+    }
+    LoggedTaskAttempt loggedAttempt = getLoggedTaskAttempt(taskType,
+        taskNumber, taskAttemptNumber);
+    if (loggedAttempt == null) {
+      // Task exists, but attempt is missing.
+      TaskInfo taskInfo = getTaskInfo(loggedTask);
+      return makeUpTaskAttemptInfo(taskType, taskInfo, taskAttemptNumber,
+          taskNumber, locality);
+    } else {
+      // Task and TaskAttempt both exist.
+      if (loggedAttempt.getResult() == Values.KILLED) {
+        TaskInfo taskInfo = getTaskInfo(loggedTask);
+        return makeUpTaskAttemptInfo(taskType, taskInfo, taskAttemptNumber,
+            taskNumber, locality);
+      } else if (loggedAttempt.getResult() == Values.FAILED) {
+        /**
+         * FAILED attempt is not affected by locality however, made-up FAILED
+         * attempts ARE affected by locality, since statistics are present for
+         * attempts of different locality.
+         */
+        return getTaskAttemptInfo(loggedTask, loggedAttempt);
+      } else if (loggedAttempt.getResult() == Values.SUCCESS) {
+        int loggedLocality = getLocality(loggedTask, loggedAttempt);
+        if (locality == loggedLocality) {
+          return getTaskAttemptInfo(loggedTask, loggedAttempt);
+        } else {
+          // attempt succeeded in trace. It is scheduled in simulation with
+          // a different locality.
+          return scaleInfo(loggedTask, loggedAttempt, locality, loggedLocality,
+              rackLocalOverNodeLocal, rackRemoteOverNodeLocal);
+        }
+      } else {
+        throw new IllegalArgumentException(
+            "attempt result is not SUCCEEDED, FAILED or KILLED: "
+                + loggedAttempt.getResult());
+      }
+    }
+  }
+
+  private long sanitizeTaskRuntime(long time, String id) {
+    if (time < 0) {
+      LOG.warn("Negative running time for task "+id+": "+time);
+      return 100L; // set default to 100ms.
+    }
+    return time;
+  }
+  
+  @SuppressWarnings("hiding") 
+  private TaskAttemptInfo scaleInfo(LoggedTask loggedTask,
+      LoggedTaskAttempt loggedAttempt, int locality, int loggedLocality,
+      double rackLocalOverNodeLocal, double rackRemoteOverNodeLocal) {
+    TaskInfo taskInfo = getTaskInfo(loggedTask);
+    double[] factors = new double[] { 1.0, rackLocalOverNodeLocal,
+        rackRemoteOverNodeLocal };
+    double scaleFactor = factors[locality] / factors[loggedLocality];
+    State state = convertState(loggedAttempt.getResult());
+    if (loggedTask.getTaskType() == Values.MAP) {
+      long taskTime = 0;
+      if (loggedAttempt.getStartTime() == 0) {
+        taskTime = makeUpMapRuntime(state, locality);
+      } else {
+        taskTime = loggedAttempt.getFinishTime() - loggedAttempt.getStartTime();
+      }
+      taskTime = sanitizeTaskRuntime(taskTime, loggedAttempt.getAttemptID());
+      taskTime *= scaleFactor;
+      return new MapTaskAttemptInfo(state, taskInfo, taskTime);
+    } else {
+      throw new IllegalArgumentException("taskType can only be MAP: "
+          + loggedTask.getTaskType());
+    }
+  }
+
+  private int getLocality(LoggedTask loggedTask, LoggedTaskAttempt loggedAttempt) {
+    int distance = cluster.getMaximumDistance();
+    String rackHostName = loggedAttempt.getHostName();
+    if (rackHostName == null) {
+      return distance;
+    }
+    MachineNode mn = getMachineNode(rackHostName);
+    if (mn == null) {
+      return distance;
+    }
+    List<LoggedLocation> locations = loggedTask.getPreferredLocations();
+    if (locations != null) {
+      for (LoggedLocation location : locations) {
+        List<String> layers = location.getLayers();
+        if ((layers == null) || (layers.isEmpty())) {
+          continue;
+        }
+        String dataNodeName = layers.get(layers.size()-1);
+        MachineNode dataNode = cluster.getMachineByName(dataNodeName);
+        if (dataNode != null) {
+          distance = Math.min(distance, cluster.distance(mn, dataNode));
+        }
+      }
+    }
+    return distance;
+  }
+
+  private MachineNode getMachineNode(String rackHostName) {
+    ParsedHost parsedHost = ParsedHost.parse(rackHostName);
+    String hostName = (parsedHost == null) ? rackHostName 
+                                           : parsedHost.getNodeName();
+    if (hostName == null) {
+      return null;
+    }
+    return (cluster == null) ? null : cluster.getMachineByName(hostName);
+  }
+
+  private TaskAttemptInfo getTaskAttemptInfo(LoggedTask loggedTask,
+      LoggedTaskAttempt loggedAttempt) {
+    TaskInfo taskInfo = getTaskInfo(loggedTask);
+    State state = convertState(loggedAttempt.getResult());
+    if (loggedTask.getTaskType() == Values.MAP) {
+      long taskTime;
+      if (loggedAttempt.getStartTime() == 0) {
+        int locality = getLocality(loggedTask, loggedAttempt);
+        taskTime = makeUpMapRuntime(state, locality);
+      } else {
+        taskTime = loggedAttempt.getFinishTime() - loggedAttempt.getStartTime();
+      }
+      taskTime = sanitizeTaskRuntime(taskTime, loggedAttempt.getAttemptID());
+      return new MapTaskAttemptInfo(state, taskInfo, taskTime);
+    } else if (loggedTask.getTaskType() == Values.REDUCE) {
+      long startTime = loggedAttempt.getStartTime();
+      long mergeDone = loggedAttempt.getSortFinished();
+      long shuffleDone = loggedAttempt.getShuffleFinished();
+      long finishTime = loggedAttempt.getFinishTime();
+      if (startTime <= 0 || startTime >= finishTime) {
+        // have seen startTime>finishTime.
+        // haven't seen reduce task with startTime=0 ever. But if this happens,
+        // make up a reduceTime with no shuffle/merge.
+        long reduceTime = makeUpReduceRuntime(state);
+        return new ReduceTaskAttemptInfo(state, taskInfo, 0, 0, reduceTime);
+      } else {
+        if (shuffleDone <= 0) {
+          shuffleDone = startTime;
+        }
+        if (mergeDone <= 0) {
+          mergeDone = finishTime;
+        }
+        long shuffleTime = shuffleDone - startTime;
+        long mergeTime = mergeDone - shuffleDone;
+        long reduceTime = finishTime - mergeDone;
+        reduceTime = sanitizeTaskRuntime(reduceTime, loggedAttempt.getAttemptID());
+        
+        return new ReduceTaskAttemptInfo(state, taskInfo, shuffleTime,
+            mergeTime, reduceTime);
+      }
+    } else {
+      throw new IllegalArgumentException("taskType for "
+          + loggedTask.getTaskID() + " is neither MAP nor REDUCE: "
+          + loggedTask.getTaskType());
+    }
+  }
+
+  private TaskInfo getTaskInfo(LoggedTask loggedTask) {
+    List<LoggedTaskAttempt> attempts = loggedTask.getAttempts();
+
+    long inputBytes = -1;
+    long inputRecords = -1;
+    long outputBytes = -1;
+    long outputRecords = -1;
+    long heapMegabytes = -1;
+
+    Values type = loggedTask.getTaskType();
+    if ((type != Values.MAP) && (type != Values.REDUCE)) {
+      throw new IllegalArgumentException(
+          "getTaskInfo only supports MAP or REDUCE tasks: " + type.toString()
+              + " for task = " + loggedTask.getTaskID());
+    }
+
+    for (LoggedTaskAttempt attempt : attempts) {
+      attempt = sanitizeLoggedTaskAttempt(attempt);
+      // ignore bad attempts or unsuccessful attempts.
+      if ((attempt == null) || (attempt.getResult() != Values.SUCCESS)) {
+        continue;
+      }
+
+      if (type == Values.MAP) {
+        inputBytes = attempt.getHdfsBytesRead();
+        inputRecords = attempt.getMapInputRecords();
+        outputBytes =
+            (job.getTotalReduces() > 0) ? attempt.getMapOutputBytes() : attempt
+                .getHdfsBytesWritten();
+        outputRecords = attempt.getMapOutputRecords();
+        heapMegabytes =
+            (job.getJobMapMB() > 0) ? job.getJobMapMB() : job
+                .getHeapMegabytes();
+      } else {
+        inputBytes = attempt.getReduceShuffleBytes();
+        inputRecords = attempt.getReduceInputRecords();
+        outputBytes = attempt.getHdfsBytesWritten();
+        outputRecords = attempt.getReduceOutputRecords();
+        heapMegabytes =
+            (job.getJobReduceMB() > 0) ? job.getJobReduceMB() : job
+                .getHeapMegabytes();
+      }
+      break;
+    }
+
+    TaskInfo taskInfo =
+        new TaskInfo(inputBytes, (int) inputRecords, outputBytes,
+            (int) outputRecords, (int) heapMegabytes);
+    return taskInfo;
+  }
+
+  private TaskAttemptID makeTaskAttemptID(TaskType taskType, int taskNumber,
+      int taskAttemptNumber) {
+    return new TaskAttemptID(new TaskID(JobID.forName(job.getJobID()),
+        TaskType.MAP == taskType, taskNumber), taskAttemptNumber);
+  }
+  
+  private TaskAttemptInfo makeUpTaskAttemptInfo(TaskType taskType, TaskInfo taskInfo,
+      int taskAttemptNumber, int taskNumber, int locality) {
+    if (taskType == TaskType.MAP) {
+      State state = State.SUCCEEDED;
+      long runtime = 0;
+
+      // make up state
+      state = makeUpState(taskAttemptNumber, job.getMapperTriesToSucceed());
+      runtime = makeUpMapRuntime(state, locality);
+      runtime = sanitizeTaskRuntime(runtime, makeTaskAttemptID(taskType,
+          taskNumber, taskAttemptNumber).toString());
+      TaskAttemptInfo tai = new MapTaskAttemptInfo(state, taskInfo, runtime);
+      return tai;
+    } else if (taskType == TaskType.REDUCE) {
+      State state = State.SUCCEEDED;
+      long shuffleTime = 0;
+      long sortTime = 0;
+      long reduceTime = 0;
+
+      // TODO make up state
+      // state = makeUpState(taskAttemptNumber, job.getReducerTriesToSucceed());
+      reduceTime = makeUpReduceRuntime(state);
+      TaskAttemptInfo tai = new ReduceTaskAttemptInfo(state, taskInfo,
+          shuffleTime, sortTime, reduceTime);
+      return tai;
+    }
+
+    throw new IllegalArgumentException("taskType is neither MAP nor REDUCE: "
+        + taskType);
+  }
+
+  private long makeUpReduceRuntime(State state) {
+    long reduceTime = 0;
+    for (int i = 0; i < 5; i++) {
+      reduceTime = doMakeUpReduceRuntime(state);
+      if (reduceTime >= 0) {
+        return reduceTime;
+      }
+    }
+    return 0;
+  }
+
+  private long doMakeUpReduceRuntime(State state) {
+    long reduceTime;
+    try {
+      if (state == State.SUCCEEDED) {
+        reduceTime = makeUpRuntime(job.getSuccessfulReduceAttemptCDF());
+      } else if (state == State.FAILED) {
+        reduceTime = makeUpRuntime(job.getFailedReduceAttemptCDF());
+      } else {
+        throw new IllegalArgumentException(
+            "state is neither SUCCEEDED nor FAILED: " + state);
+      }
+      return reduceTime;
+    } catch (NoValueToMakeUpRuntime e) {
+      return 0;
+    }
+  }
+
+  private long makeUpMapRuntime(State state, int locality) {
+    long runtime;
+    // make up runtime
+    if (state == State.SUCCEEDED || state == State.FAILED) {
+      List<LoggedDiscreteCDF> cdfList =
+          state == State.SUCCEEDED ? job.getSuccessfulMapAttemptCDFs() : job
+              .getFailedMapAttemptCDFs();
+      // XXX MapCDFs is a ArrayList of 4 possible groups: distance=0, 1, 2, and
+      // the last group is "distance cannot be determined". All pig jobs
+      // would have only the 4th group, and pig tasks usually do not have
+      // any locality, so this group should count as "distance=2".
+      // However, setup/cleanup tasks are also counted in the 4th group.
+      // These tasks do not make sense.
+      try {
+        runtime = makeUpRuntime(cdfList.get(locality));
+      } catch (NoValueToMakeUpRuntime e) {
+        runtime = makeUpRuntime(cdfList);
+      }
+    } else {
+      throw new IllegalArgumentException(
+          "state is neither SUCCEEDED nor FAILED: " + state);
+    }
+    return runtime;
+  }
+
+  /**
+   * Perform a weighted random selection on a list of CDFs, and produce a random
+   * variable using the selected CDF.
+   * 
+   * @param mapAttemptCDFs
+   *          A list of CDFs for the distribution of runtime for the 1st, 2nd,
+   *          ... map attempts for the job.
+   */
+  private long makeUpRuntime(List<LoggedDiscreteCDF> mapAttemptCDFs) {
+    int total = 0;
+    for (LoggedDiscreteCDF cdf : mapAttemptCDFs) {
+      total += cdf.getNumberValues();
+    }
+    if (total == 0) {
+      return -1;
+    }
+    int index = random.nextInt(total);
+    for (LoggedDiscreteCDF cdf : mapAttemptCDFs) {
+      if (index >= cdf.getNumberValues()) {
+        index -= cdf.getNumberValues();
+      } else {
+        if (index < 0) {
+          throw new IllegalStateException("application error");
+        }
+        return makeUpRuntime(cdf);
+      }
+    }
+    throw new IllegalStateException("not possible to get here");
+  }
+
+  private long makeUpRuntime(LoggedDiscreteCDF loggedDiscreteCDF) {
+    /*
+     * We need this odd-looking code because if a seed exists we need to ensure
+     * that only one interpolator is generated per LoggedDiscreteCDF, but if no
+     * seed exists then the potentially lengthy process of making an
+     * interpolator can happen outside the lock. makeUpRuntimeCore only locks
+     * around the two hash map accesses.
+     */
+    if (hasRandomSeed) {
+      synchronized (interpolatorMap) {
+        return makeUpRuntimeCore(loggedDiscreteCDF);
+      }
+    }
+
+    return makeUpRuntimeCore(loggedDiscreteCDF);
+  }
+
+  private long makeUpRuntimeCore(LoggedDiscreteCDF loggedDiscreteCDF) {
+    CDFRandomGenerator interpolator;
+
+    synchronized (interpolatorMap) {
+      interpolator = interpolatorMap.get(loggedDiscreteCDF);
+    }
+
+    if (interpolator == null) {
+      if (loggedDiscreteCDF.getNumberValues() == 0) {
+        throw new NoValueToMakeUpRuntime("no value to use to make up runtime");
+      }
+
+      interpolator =
+          hasRandomSeed ? new CDFPiecewiseLinearRandomGenerator(
+              loggedDiscreteCDF, ++seed)
+              : new CDFPiecewiseLinearRandomGenerator(loggedDiscreteCDF);
+
+      /*
+       * It doesn't matter if we compute and store an interpolator twice because
+       * the two instances will be semantically identical and stateless, unless
+       * we're seeded, in which case we're not stateless but this code will be
+       * called synchronizedly.
+       */
+      synchronized (interpolatorMap) {
+        interpolatorMap.put(loggedDiscreteCDF, interpolator);
+      }
+    }
+
+    return interpolator.randomValue();
+  }
+
+  static private class NoValueToMakeUpRuntime extends IllegalArgumentException {
+    static final long serialVersionUID = 1L;
+
+    NoValueToMakeUpRuntime() {
+      super();
+    }
+
+    NoValueToMakeUpRuntime(String detailMessage) {
+      super(detailMessage);
+    }
+
+    NoValueToMakeUpRuntime(String detailMessage, Throwable cause) {
+      super(detailMessage, cause);
+    }
+
+    NoValueToMakeUpRuntime(Throwable cause) {
+      super(cause);
+    }
+  }
+
+  private State makeUpState(int taskAttemptNumber, double[] numAttempts) {
+    if (taskAttemptNumber >= numAttempts.length - 1) {
+      // always succeed
+      return State.SUCCEEDED;
+    } else {
+      double pSucceed = numAttempts[taskAttemptNumber];
+      double pFail = 0;
+      for (int i = taskAttemptNumber + 1; i < numAttempts.length; i++) {
+        pFail += numAttempts[i];
+      }
+      return (random.nextDouble() < pSucceed / (pSucceed + pFail)) ? State.SUCCEEDED
+          : State.FAILED;
+    }
+  }
+
+  private TaskID getMaskedTaskID(TaskType taskType, int taskNumber) {
+    return new TaskID(new JobID(), TaskType.MAP == taskType, taskNumber);
+  }
+
+  private LoggedTask getLoggedTask(TaskType taskType, int taskNumber) {
+    buildMaps();
+    return loggedTaskMap.get(getMaskedTaskID(taskType, taskNumber));
+  }
+
+  private LoggedTaskAttempt getLoggedTaskAttempt(TaskType taskType,
+      int taskNumber, int taskAttemptNumber) {
+    buildMaps();
+    TaskAttemptID id =
+        new TaskAttemptID(getMaskedTaskID(taskType, taskNumber),
+            taskAttemptNumber);
+    return loggedTaskAttemptMap.get(id);
+  }
+
+}

+ 81 - 0
src/tools/org/apache/hadoop/tools/rumen/ZombieJobProducer.java

@@ -0,0 +1,81 @@
+/**
+ * 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.IOException;
+import java.io.InputStream;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+
+/**
+ * Producing {@link JobStory}s from job trace.
+ */
+public class ZombieJobProducer implements JobStoryProducer {
+  private final JobTraceReader reader;
+  private final ZombieCluster cluster;
+
+  private ZombieJobProducer(JobTraceReader reader, ZombieCluster cluster) {
+    this.reader = reader;
+    this.cluster = cluster;
+  }
+
+  /**
+   * Constructor
+   * 
+   * @param path
+   *          Path to the JSON trace file, possibly compressed.
+   * @param cluster
+   *          The topology of the cluster that corresponds to the jobs in the
+   *          trace. The argument can be null if we do not have knowledge of the
+   *          cluster topology.
+   * @param conf
+   * @throws IOException
+   */
+  public ZombieJobProducer(Path path, ZombieCluster cluster, Configuration conf)
+      throws IOException {
+    this(new JobTraceReader(path, conf), cluster);
+  }
+
+  /**
+   * Constructor
+   * 
+   * @param input
+   *          The input stream for the JSON trace.
+   * @param cluster
+   *          The topology of the cluster that corresponds to the jobs in the
+   *          trace. The argument can be null if we do not have knowledge of the
+   *          cluster topology.
+   * @throws IOException
+   */
+  public ZombieJobProducer(InputStream input, ZombieCluster cluster)
+      throws IOException {
+    this(new JobTraceReader(input), cluster);
+  }
+
+  @Override
+  public ZombieJob getNextJob() throws IOException {
+    LoggedJob job = reader.getNext();
+    return (job == null) ? null : new ZombieJob(job, cluster);
+  }
+
+  @Override
+  public void close() throws IOException {
+    reader.close();
+  }
+}