Jelajahi Sumber

HADOOP-2178. Job History on DFS. Contributed by Amareshwari Sri Ramadasu.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/core/trunk@629369 13f79535-47bb-0310-9956-ffa450edef68
Devaraj Das 17 tahun lalu
induk
melakukan
6771646cc0
33 mengubah file dengan 856 tambahan dan 464 penghapusan
  1. 2 0
      CHANGES.txt
  2. 8 0
      build.xml
  3. 28 0
      conf/hadoop-default.xml
  4. 30 2
      docs/cluster_setup.html
  5. 3 4
      docs/cluster_setup.pdf
  6. 82 17
      docs/hadoop-default.html
  7. 45 21
      docs/mapred_tutorial.html
  8. 3 3
      docs/mapred_tutorial.pdf
  9. 4 1
      src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestMultipleCachefiles.java
  10. 4 1
      src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestSymLink.java
  11. 29 0
      src/docs/src/documentation/content/xdocs/cluster_setup.xml
  12. 23 1
      src/docs/src/documentation/content/xdocs/mapred_tutorial.xml
  13. 1 0
      src/docs/src/documentation/content/xdocs/site.xml
  14. 8 68
      src/java/org/apache/hadoop/mapred/DefaultJobHistoryParser.java
  15. 51 0
      src/java/org/apache/hadoop/mapred/JobClient.java
  16. 188 146
      src/java/org/apache/hadoop/mapred/JobHistory.java
  17. 32 0
      src/java/org/apache/hadoop/mapred/JobTracker.java
  18. 35 0
      src/java/org/apache/hadoop/mapred/OutputLogFilter.java
  19. 4 2
      src/test/org/apache/hadoop/mapred/TestClusterMapReduceTestCase.java
  20. 5 3
      src/test/org/apache/hadoop/mapred/TestMiniMRClasspath.java
  21. 5 1
      src/test/org/apache/hadoop/mapred/TestMiniMRWithDFS.java
  22. 3 1
      src/test/org/apache/hadoop/mapred/pipes/TestPipes.java
  23. 13 13
      src/webapps/history/analysejobhistory.jsp
  24. 20 0
      src/webapps/history/index.html
  25. 58 0
      src/webapps/history/jobconf_history.jsp
  26. 25 24
      src/webapps/history/jobdetailshistory.jsp
  27. 104 0
      src/webapps/history/jobhistory.jsp
  28. 5 5
      src/webapps/history/jobtaskshistory.jsp
  29. 35 0
      src/webapps/history/loadhistory.jsp
  30. 2 2
      src/webapps/history/taskdetailshistory.jsp
  31. 0 99
      src/webapps/job/jobhistory.jsp
  32. 1 1
      src/webapps/job/jobtracker.jsp
  33. 0 49
      src/webapps/job/loadhistory.jsp

+ 2 - 0
CHANGES.txt

@@ -14,6 +14,8 @@ Trunk (unreleased changes)
 
     HADOOP-1398.  Add HBase in-memory block cache.  (tomwhite)
 
+    HADOOP-2178.  Job History on DFS. (Amareshwari Sri Ramadasu via ddas)
+
   OPTIMIZATIONS
 
   BUG FIXES

+ 8 - 0
build.xml

@@ -178,6 +178,7 @@
     <mkdir dir="${build.webapps}/job/WEB-INF"/>
     <mkdir dir="${build.webapps}/dfs/WEB-INF"/>
     <mkdir dir="${build.webapps}/datanode/WEB-INF"/>
+    <mkdir dir="${build.webapps}/history/WEB-INF"/>
     <mkdir dir="${build.examples}"/>
     <mkdir dir="${build.anttasks}"/>
     <mkdir dir="${build.dir}/c++"/>
@@ -272,6 +273,13 @@
      webxml="${build.webapps}/dfs/WEB-INF/web.xml">
     </jsp-compile>
 
+    <jsp-compile
+     uriroot="${src.webapps}/history"
+     outputdir="${build.src}"
+     package="org.apache.hadoop.mapred"
+     webxml="${build.webapps}/history/WEB-INF/web.xml">
+    </jsp-compile>
+
     <jsp-compile
      uriroot="${src.webapps}/datanode"
      outputdir="${build.src}"

+ 28 - 0
conf/hadoop-default.xml

@@ -35,6 +35,25 @@
   <description>The max number of log files</description>
 </property>
 
+<property>
+  <name>hadoop.job.history.location</name>
+  <value>file://${hadoop.log.dir}/history</value>
+  <description> If job tracker is static the history files are stored 
+  in this single well known place. By default, it is in the local 
+  file system at ${hadoop.log.dir}/history.
+  </description>
+</property>
+
+<property>
+  <name>hadoop.job.history.user.location</name>
+  <value></value>
+  <description> User can specify a location to store the history files of 
+  a particular job. If nothing is specified, the logs are stored in 
+  output directory. The files are stored in "_logs/history/" in the directory.
+  User can stop logging by giving the value "none". 
+  </description>
+</property>
+
 <property>
   <name>dfs.namenode.logging.level</name>
   <value>info</value>
@@ -506,6 +525,15 @@ creations/deletions), or "all".</description>
   </description>
 </property>
 
+<property>
+  <name>mapred.job.history.http.bindAddress</name>
+  <value>0.0.0.0:0</value>
+  <description>
+    The job history http server bind address and port.
+    If the port is 0 then the server will start on a free port.
+  </description>
+</property>
+
 <property>
   <name>mapred.job.tracker.handler.count</name>
   <value>10</value>

+ 30 - 2
docs/cluster_setup.html

@@ -622,13 +622,41 @@ document.write("Last Published: " + document.lastModified);
           Commons Logging</a> framework for logging. Edit the 
           <span class="codefrag">conf/log4j.properties</span> file to customize the Hadoop 
           daemons' logging configuration (log-formats and so on).</p>
+<a name="N1030D"></a><a name="History+Logging"></a>
+<h5>History Logging</h5>
+<p> The job history files are stored in central location 
+            <span class="codefrag"> hadoop.job.history.location </span> which can be on DFS also,
+            whose default value is <span class="codefrag">${HADOOP_LOG_DIR}/history</span>. 
+            Job history server is started on job tracker. The history 
+            web UI is accessible from job tracker web UI.</p>
+<p> The history files are also logged to user specified directory
+            <span class="codefrag">hadoop.job.history.user.location</span> 
+            which defaults to job output directory. The files are stored in
+            "_logs/history/" in the specified directory. Hence, by default 
+            they will be in "mapred.output.dir/_logs/history/". User can stop
+            logging by giving the value <span class="codefrag">none</span> for 
+            <span class="codefrag">hadoop.job.history.user.location</span> 
+</p>
+<p> User can view logs in specified directory using 
+            the following command <br>
+            
+<span class="codefrag">$ bin/hadoop job -history output-dir</span>
+<br>
+            This will start a stand alone jetty on the client and 
+            load history jsp's. 
+            It will display the port where the server is up at. The server will
+            be up for 30 minutes. User has to use 
+            <span class="codefrag"> http://hostname:port </span> to view the history. User can 
+            also provide http bind address using 
+            <span class="codefrag">mapred.job.history.http.bindAddress</span>
+</p>
 <p>Once all the necessary configuration is complete, distribute the files
       to the <span class="codefrag">HADOOP_CONF_DIR</span> directory on all the machines, 
       typically <span class="codefrag">${HADOOP_HOME}/conf</span>.</p>
 </div>
     
     
-<a name="N10319"></a><a name="Hadoop+Startup"></a>
+<a name="N10343"></a><a name="Hadoop+Startup"></a>
 <h2 class="h3">Hadoop Startup</h2>
 <div class="section">
 <p>To start a Hadoop cluster you will need to start both the HDFS and 
@@ -663,7 +691,7 @@ document.write("Last Published: " + document.lastModified);
 </div>
     
     
-<a name="N1035F"></a><a name="Hadoop+Shutdown"></a>
+<a name="N10389"></a><a name="Hadoop+Shutdown"></a>
 <h2 class="h3">Hadoop Shutdown</h2>
 <div class="section">
 <p>

File diff ditekan karena terlalu besar
+ 3 - 4
docs/cluster_setup.pdf


+ 82 - 17
docs/hadoop-default.html

@@ -17,6 +17,19 @@
 <td><a name="hadoop.logfile.count">hadoop.logfile.count</a></td><td>10</td><td>The max number of log files</td>
 </tr>
 <tr>
+<td><a name="hadoop.job.history.location">hadoop.job.history.location</a></td><td>file://${hadoop.log.dir}/history</td><td> If job tracker is static the history files are stored 
+  in this single well known place. By default, it is in the local 
+  file system at ${hadoop.log.dir}/history.
+  </td>
+</tr>
+<tr>
+<td><a name="hadoop.job.history.user.location">hadoop.job.history.user.location</a></td><td></td><td> User can specify a location to store the history files of 
+  a particular job. If nothing is specified, the logs are stored in 
+  output directory. The files are stored in "_logs/history/" in the directory.
+  User can stop logging by giving the value "none". 
+  </td>
+</tr>
+<tr>
 <td><a name="dfs.namenode.logging.level">dfs.namenode.logging.level</a></td><td>info</td><td>The logging level for dfs namenode. Other values are "dir"(trac
 e namespace mutations), "block"(trace block under/over replications and block
 creations/deletions), or "all".</td>
@@ -62,10 +75,6 @@ creations/deletions), or "all".</td>
   determine the host, port, etc. for a filesystem.</td>
 </tr>
 <tr>
-<td><a name="fs.trash.root">fs.trash.root</a></td><td>${hadoop.tmp.dir}/Trash</td><td>The trash directory, used by FsShell's 'rm' command.
-  </td>
-</tr>
-<tr>
 <td><a name="fs.trash.interval">fs.trash.interval</a></td><td>0</td><td>Number of minutes between trash checkpoints.
   If zero, the trash feature is disabled.
   </td>
@@ -106,25 +115,25 @@ creations/deletions), or "all".</td>
   </td>
 </tr>
 <tr>
-<td><a name="dfs.secondary.http.bindAddress">dfs.secondary.http.bindAddress</a></td><td>0.0.0.0:50090</td><td>
-    The secondary namenode http server bind address and port.
+<td><a name="dfs.secondary.http.address">dfs.secondary.http.address</a></td><td>0.0.0.0:50090</td><td>
+    The secondary namenode http server address and port.
     If the port is 0 then the server will start on a free port.
   </td>
 </tr>
 <tr>
-<td><a name="dfs.datanode.bindAddress">dfs.datanode.bindAddress</a></td><td>0.0.0.0:50010</td><td>
-    The address where the datanode will listen to.
+<td><a name="dfs.datanode.address">dfs.datanode.address</a></td><td>0.0.0.0:50010</td><td>
+    The address where the datanode server will listen to.
     If the port is 0 then the server will start on a free port.
   </td>
 </tr>
 <tr>
-<td><a name="dfs.datanode.http.bindAddress">dfs.datanode.http.bindAddress</a></td><td>0.0.0.0:50075</td><td>
-    The datanode http server bind address and port.
+<td><a name="dfs.datanode.http.address">dfs.datanode.http.address</a></td><td>0.0.0.0:50075</td><td>
+    The datanode http server address and port.
     If the port is 0 then the server will start on a free port.
   </td>
 </tr>
 <tr>
-<td><a name="dfs.http.bindAddress">dfs.http.bindAddress</a></td><td>0.0.0.0:50070</td><td>
+<td><a name="dfs.http.address">dfs.http.address</a></td><td>0.0.0.0:50070</td><td>
     The address and the base port where the dfs namenode web ui will listen on.
     If the port is 0 then the server will start on a free port.
   </td>
@@ -163,6 +172,11 @@ creations/deletions), or "all".</td>
       directories, for redundancy. </td>
 </tr>
 <tr>
+<td><a name="dfs.web.ugi">dfs.web.ugi</a></td><td>webuser,webgroup</td><td>The user account used by the web interface.
+    Syntax: USERNAME,GROUP1,GROUP2, ...
+  </td>
+</tr>
+<tr>
 <td><a name="dfs.permissions">dfs.permissions</a></td><td>true</td><td>
     If "true", enable permission checking in HDFS.
     If "false", permission checking is turned off,
@@ -267,6 +281,12 @@ creations/deletions), or "all".</td>
   </td>
 </tr>
 <tr>
+<td><a name="dfs.namenode.decommission.interval">dfs.namenode.decommission.interval</a></td><td>300</td><td>Namenode periodicity in seconds to check if decommission is complete.</td>
+</tr>
+<tr>
+<td><a name="dfs.replication.interval">dfs.replication.interval</a></td><td>3</td><td>The periodicity in seconds with which the namenode computes repliaction work for datanodes. </td>
+</tr>
+<tr>
 <td><a name="fs.s3.block.size">fs.s3.block.size</a></td><td>67108864</td><td>Block size to use when writing files to S3.</td>
 </tr>
 <tr>
@@ -291,8 +311,14 @@ creations/deletions), or "all".</td>
   </td>
 </tr>
 <tr>
-<td><a name="mapred.job.tracker.http.bindAddress">mapred.job.tracker.http.bindAddress</a></td><td>0.0.0.0:50030</td><td>
-    The job tracker http server bind address and port.
+<td><a name="mapred.job.tracker.http.address">mapred.job.tracker.http.address</a></td><td>0.0.0.0:50030</td><td>
+    The job tracker http server address and port the server will listen on.
+    If the port is 0 then the server will start on a free port.
+  </td>
+</tr>
+<tr>
+<td><a name="mapred.job.history.http.bindAddress">mapred.job.history.http.bindAddress</a></td><td>0.0.0.0:0</td><td>
+    The job history http server bind address and port.
     If the port is 0 then the server will start on a free port.
   </td>
 </tr>
@@ -303,8 +329,10 @@ creations/deletions), or "all".</td>
   </td>
 </tr>
 <tr>
-<td><a name="mapred.task.tracker.report.bindAddress">mapred.task.tracker.report.bindAddress</a></td><td>127.0.0.1:0</td><td>The interface that task processes use to communicate
-  with their parent tasktracker process.</td>
+<td><a name="mapred.task.tracker.report.address">mapred.task.tracker.report.address</a></td><td>127.0.0.1:0</td><td>The interface and port that task tracker server listens on. 
+  Since it is only connected to by the tasks, it uses the local interface.
+  EXPERT ONLY. Should only be changed if your host does not have the loopback 
+  interface.</td>
 </tr>
 <tr>
 <td><a name="mapred.local.dir">mapred.local.dir</a></td><td>${hadoop.tmp.dir}/mapred/local</td><td>The local directory where MapReduce stores intermediate
@@ -410,6 +438,15 @@ creations/deletions), or "all".</td>
   </td>
 </tr>
 <tr>
+<td><a name="mapred.child.tmp">mapred.child.tmp</a></td><td>./tmp</td><td> To set the value of tmp directory for map and reduce tasks.
+  If the value is an absolute path, it is directly assigned. Otherwise, it is
+  prepended with task's working directory. The java tasks are executed with
+  option -Djava.io.tmpdir='the absolute path of the tmp dir'. Pipes and
+  streaming are set with environment variable,
+   TMPDIR='the absolute path of the tmp dir'
+  </td>
+</tr>
+<tr>
 <td><a name="mapred.inmem.merge.threshold">mapred.inmem.merge.threshold</a></td><td>1000</td><td>The threshold, in terms of the number of files 
   for the in-memory merge process. When we accumulate threshold number of files
   we initiate the in-memory merge and spill to disk. A value of 0 or less than
@@ -452,8 +489,8 @@ creations/deletions), or "all".</td>
   </td>
 </tr>
 <tr>
-<td><a name="mapred.task.tracker.http.bindAddress">mapred.task.tracker.http.bindAddress</a></td><td>0.0.0.0:50060</td><td>
-    The task tracker http server bind address and port.
+<td><a name="mapred.task.tracker.http.address">mapred.task.tracker.http.address</a></td><td>0.0.0.0:50060</td><td>
+    The task tracker http server address and port.
     If the port is 0 then the server will start on a free port.
   </td>
 </tr>
@@ -564,6 +601,22 @@ creations/deletions), or "all".</td>
     </td>
 </tr>
 <tr>
+<td><a name="mapred.task.profile">mapred.task.profile</a></td><td>false</td><td>To set whether the system should collect profiler
+     information for some of the tasks in this job? The information is stored
+     in the the user log directory. The value is "true" if task profiling
+     is enabled.</td>
+</tr>
+<tr>
+<td><a name="mapred.task.profile.maps">mapred.task.profile.maps</a></td><td>0-2</td><td> To set the ranges of map tasks to profile.
+    mapred.task.profile has to be set to true for the value to be accounted.
+    </td>
+</tr>
+<tr>
+<td><a name="mapred.task.profile.reduces">mapred.task.profile.reduces</a></td><td>0-2</td><td> To set the ranges of reduce tasks to profile.
+    mapred.task.profile has to be set to true for the value to be accounted.
+    </td>
+</tr>
+<tr>
 <td><a name="ipc.client.timeout">ipc.client.timeout</a></td><td>60000</td><td>Defines the timeout for IPC calls in milliseconds.</td>
 </tr>
 <tr>
@@ -596,6 +649,18 @@ creations/deletions), or "all".</td>
   </td>
 </tr>
 <tr>
+<td><a name="ipc.server.tcpnodelay">ipc.server.tcpnodelay</a></td><td>false</td><td>Turn on/off Nagle's algorithm for the TCP socket connection on 
+  the server. Setting to true disables the algorithm and may decrease latency
+  with a cost of more/smaller packets. 
+  </td>
+</tr>
+<tr>
+<td><a name="ipc.client.tcpnodelay">ipc.client.tcpnodelay</a></td><td>false</td><td>Turn on/off Nagle's algorithm for the TCP socket connection on 
+  the client. Setting to true disables the algorithm and may decrease latency
+  with a cost of more/smaller packets. 
+  </td>
+</tr>
+<tr>
 <td><a name="job.end.retry.attempts">job.end.retry.attempts</a></td><td>0</td><td>Indicates how many times hadoop should attempt to contact the
                notification URL </td>
 </tr>

+ 45 - 21
docs/mapred_tutorial.html

@@ -283,7 +283,7 @@ document.write("Last Published: " + document.lastModified);
 <a href="#Example%3A+WordCount+v2.0">Example: WordCount v2.0</a>
 <ul class="minitoc">
 <li>
-<a href="#Source+Code-N10BBE">Source Code</a>
+<a href="#Source+Code-N10BDE">Source Code</a>
 </li>
 <li>
 <a href="#Sample+Runs">Sample Runs</a>
@@ -1570,10 +1570,34 @@ document.write("Last Published: " + document.lastModified);
           </li>
         
 </ol>
+<p> Job history files are also logged to user specified directory
+        <span class="codefrag">hadoop.job.history.user.location</span> 
+        which defaults to job output directory. The files are stored in
+        "_logs/history/" in the specified directory. Hence, by default they will
+        be in mapred.output.dir/_logs/history. User can stop
+        logging by giving the value <span class="codefrag">none</span> for 
+        <span class="codefrag">hadoop.job.history.user.location</span>
+</p>
+<p> User can view logs in specified directory using 
+        the following command <br>
+        
+<span class="codefrag">$ bin/hadoop job -history output-dir</span>
+<br>
+        This will start a stand alone jetty on the client and 
+        load history jsp's. 
+        It will display the port where the server is up at. The server will
+        be up for 30 minutes. User has to use 
+        <span class="codefrag"> http://hostname:port </span> to view the history. User can 
+        also provide http bind address using 
+        <span class="codefrag">mapred.job.history.http.bindAddress</span>
+</p>
+<p> User can use 
+        <a href="api/org/apache/hadoop/mapred/OutputLogFilter.html">OutputLogFilter</a>
+        to filter log files from the output directory listing. </p>
 <p>Normally the user creates the application, describes various facets 
         of the job via <span class="codefrag">JobConf</span>, and then uses the 
         <span class="codefrag">JobClient</span> to submit the job and monitor its progress.</p>
-<a name="N108F7"></a><a name="Job+Control"></a>
+<a name="N10917"></a><a name="Job+Control"></a>
 <h4>Job Control</h4>
 <p>Users may need to chain map-reduce jobs to accomplish complex
           tasks which cannot be done via a single map-reduce job. This is fairly
@@ -1609,7 +1633,7 @@ document.write("Last Published: " + document.lastModified);
             </li>
           
 </ul>
-<a name="N10921"></a><a name="Job+Input"></a>
+<a name="N10941"></a><a name="Job+Input"></a>
 <h3 class="h4">Job Input</h3>
 <p>
 <a href="api/org/apache/hadoop/mapred/InputFormat.html">
@@ -1657,7 +1681,7 @@ document.write("Last Published: " + document.lastModified);
         appropriate <span class="codefrag">CompressionCodec</span>. However, it must be noted that
         compressed files with the above extensions cannot be <em>split</em> and 
         each compressed file is processed in its entirety by a single mapper.</p>
-<a name="N1098B"></a><a name="InputSplit"></a>
+<a name="N109AB"></a><a name="InputSplit"></a>
 <h4>InputSplit</h4>
 <p>
 <a href="api/org/apache/hadoop/mapred/InputSplit.html">
@@ -1671,7 +1695,7 @@ document.write("Last Published: " + document.lastModified);
           FileSplit</a> is the default <span class="codefrag">InputSplit</span>. It sets 
           <span class="codefrag">map.input.file</span> to the path of the input file for the
           logical split.</p>
-<a name="N109B0"></a><a name="RecordReader"></a>
+<a name="N109D0"></a><a name="RecordReader"></a>
 <h4>RecordReader</h4>
 <p>
 <a href="api/org/apache/hadoop/mapred/RecordReader.html">
@@ -1683,7 +1707,7 @@ document.write("Last Published: " + document.lastModified);
           for processing. <span class="codefrag">RecordReader</span> thus assumes the 
           responsibility of processing record boundaries and presents the tasks 
           with keys and values.</p>
-<a name="N109D3"></a><a name="Job+Output"></a>
+<a name="N109F3"></a><a name="Job+Output"></a>
 <h3 class="h4">Job Output</h3>
 <p>
 <a href="api/org/apache/hadoop/mapred/OutputFormat.html">
@@ -1708,7 +1732,7 @@ document.write("Last Published: " + document.lastModified);
 <p>
 <span class="codefrag">TextOutputFormat</span> is the default 
         <span class="codefrag">OutputFormat</span>.</p>
-<a name="N109FC"></a><a name="Task+Side-Effect+Files"></a>
+<a name="N10A1C"></a><a name="Task+Side-Effect+Files"></a>
 <h4>Task Side-Effect Files</h4>
 <p>In some applications, component tasks need to create and/or write to
           side-files, which differ from the actual job-output files.</p>
@@ -1734,7 +1758,7 @@ document.write("Last Published: " + document.lastModified);
           JobConf.getOutputPath()</a>, and the framework will promote them 
           similarly for succesful task-attempts, thus eliminating the need to 
           pick unique paths per task-attempt.</p>
-<a name="N10A31"></a><a name="RecordWriter"></a>
+<a name="N10A51"></a><a name="RecordWriter"></a>
 <h4>RecordWriter</h4>
 <p>
 <a href="api/org/apache/hadoop/mapred/RecordWriter.html">
@@ -1742,9 +1766,9 @@ document.write("Last Published: " + document.lastModified);
           pairs to an output file.</p>
 <p>RecordWriter implementations write the job outputs to the 
           <span class="codefrag">FileSystem</span>.</p>
-<a name="N10A48"></a><a name="Other+Useful+Features"></a>
+<a name="N10A68"></a><a name="Other+Useful+Features"></a>
 <h3 class="h4">Other Useful Features</h3>
-<a name="N10A4E"></a><a name="Counters"></a>
+<a name="N10A6E"></a><a name="Counters"></a>
 <h4>Counters</h4>
 <p>
 <span class="codefrag">Counters</span> represent global counters, defined either by 
@@ -1758,7 +1782,7 @@ document.write("Last Published: " + document.lastModified);
           Reporter.incrCounter(Enum, long)</a> in the <span class="codefrag">map</span> and/or 
           <span class="codefrag">reduce</span> methods. These counters are then globally 
           aggregated by the framework.</p>
-<a name="N10A79"></a><a name="DistributedCache"></a>
+<a name="N10A99"></a><a name="DistributedCache"></a>
 <h4>DistributedCache</h4>
 <p>
 <a href="api/org/apache/hadoop/filecache/DistributedCache.html">
@@ -1791,7 +1815,7 @@ document.write("Last Published: " + document.lastModified);
           <a href="api/org/apache/hadoop/filecache/DistributedCache.html#createSymlink(org.apache.hadoop.conf.Configuration)">
           DistributedCache.createSymlink(Path, Configuration)</a> api. Files 
           have <em>execution permissions</em> set.</p>
-<a name="N10AB7"></a><a name="Tool"></a>
+<a name="N10AD7"></a><a name="Tool"></a>
 <h4>Tool</h4>
 <p>The <a href="api/org/apache/hadoop/util/Tool.html">Tool</a> 
           interface supports the handling of generic Hadoop command-line options.
@@ -1831,7 +1855,7 @@ document.write("Last Published: " + document.lastModified);
             </span>
           
 </p>
-<a name="N10AE9"></a><a name="IsolationRunner"></a>
+<a name="N10B09"></a><a name="IsolationRunner"></a>
 <h4>IsolationRunner</h4>
 <p>
 <a href="api/org/apache/hadoop/mapred/IsolationRunner.html">
@@ -1855,13 +1879,13 @@ document.write("Last Published: " + document.lastModified);
 <p>
 <span class="codefrag">IsolationRunner</span> will run the failed task in a single 
           jvm, which can be in the debugger, over precisely the same input.</p>
-<a name="N10B1C"></a><a name="JobControl"></a>
+<a name="N10B3C"></a><a name="JobControl"></a>
 <h4>JobControl</h4>
 <p>
 <a href="api/org/apache/hadoop/mapred/jobcontrol/package-summary.html">
           JobControl</a> is a utility which encapsulates a set of Map-Reduce jobs
           and their dependencies.</p>
-<a name="N10B29"></a><a name="Data+Compression"></a>
+<a name="N10B49"></a><a name="Data+Compression"></a>
 <h4>Data Compression</h4>
 <p>Hadoop Map-Reduce provides facilities for the application-writer to
           specify compression for both intermediate map-outputs and the
@@ -1875,7 +1899,7 @@ document.write("Last Published: " + document.lastModified);
           codecs for reasons of both performance (zlib) and non-availability of
           Java libraries (lzo). More details on their usage and availability are
           available <a href="native_libraries.html">here</a>.</p>
-<a name="N10B49"></a><a name="Intermediate+Outputs"></a>
+<a name="N10B69"></a><a name="Intermediate+Outputs"></a>
 <h5>Intermediate Outputs</h5>
 <p>Applications can control compression of intermediate map-outputs
             via the 
@@ -1896,7 +1920,7 @@ document.write("Last Published: " + document.lastModified);
             <a href="api/org/apache/hadoop/mapred/JobConf.html#setMapOutputCompressionType(org.apache.hadoop.io.SequenceFile.CompressionType)">
             JobConf.setMapOutputCompressionType(SequenceFile.CompressionType)</a> 
             api.</p>
-<a name="N10B75"></a><a name="Job+Outputs"></a>
+<a name="N10B95"></a><a name="Job+Outputs"></a>
 <h5>Job Outputs</h5>
 <p>Applications can control compression of job-outputs via the
             <a href="api/org/apache/hadoop/mapred/OutputFormatBase.html#setCompressOutput(org.apache.hadoop.mapred.JobConf,%20boolean)">
@@ -1916,7 +1940,7 @@ document.write("Last Published: " + document.lastModified);
 </div>
 
     
-<a name="N10BA4"></a><a name="Example%3A+WordCount+v2.0"></a>
+<a name="N10BC4"></a><a name="Example%3A+WordCount+v2.0"></a>
 <h2 class="h3">Example: WordCount v2.0</h2>
 <div class="section">
 <p>Here is a more complete <span class="codefrag">WordCount</span> which uses many of the
@@ -1926,7 +1950,7 @@ document.write("Last Published: " + document.lastModified);
       <a href="quickstart.html#SingleNodeSetup">pseudo-distributed</a> or
       <a href="quickstart.html#Fully-Distributed+Operation">fully-distributed</a> 
       Hadoop installation.</p>
-<a name="N10BBE"></a><a name="Source+Code-N10BBE"></a>
+<a name="N10BDE"></a><a name="Source+Code-N10BDE"></a>
 <h3 class="h4">Source Code</h3>
 <table class="ForrestTable" cellspacing="1" cellpadding="4">
           
@@ -3136,7 +3160,7 @@ document.write("Last Published: " + document.lastModified);
 </tr>
         
 </table>
-<a name="N11320"></a><a name="Sample+Runs"></a>
+<a name="N11340"></a><a name="Sample+Runs"></a>
 <h3 class="h4">Sample Runs</h3>
 <p>Sample text-files as input:</p>
 <p>
@@ -3304,7 +3328,7 @@ document.write("Last Published: " + document.lastModified);
 <br>
         
 </p>
-<a name="N113F4"></a><a name="Highlights"></a>
+<a name="N11414"></a><a name="Highlights"></a>
 <h3 class="h4">Highlights</h3>
 <p>The second version of <span class="codefrag">WordCount</span> improves upon the 
         previous one by using some features offered by the Map-Reduce framework:

File diff ditekan karena terlalu besar
+ 3 - 3
docs/mapred_tutorial.pdf


+ 4 - 1
src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestMultipleCachefiles.java

@@ -23,6 +23,7 @@ import java.io.*;
 import java.util.*;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.*;
 import org.apache.hadoop.dfs.MiniDFSCluster;
@@ -100,7 +101,9 @@ public class TestMultipleCachefiles extends TestCase
         job.go();
         String line = null;
         String line2 = null;
-        Path[] fileList = fileSys.listPaths(new Path(OUTPUT_DIR));
+        Path[] fileList = FileUtil.stat2Paths(fileSys.listStatus(
+                                     new Path(OUTPUT_DIR),
+                                     new OutputLogFilter()));
         for (int i = 0; i < fileList.length; i++){
           System.out.println(fileList[i].toString());
           BufferedReader bread =

+ 4 - 1
src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestSymLink.java

@@ -23,6 +23,7 @@ import java.io.*;
 import java.util.*;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.*;
 import org.apache.hadoop.dfs.MiniDFSCluster;
@@ -91,7 +92,9 @@ public class TestSymLink extends TestCase
         job = new StreamJob(argv, mayExit);      
         job.go();
         String line = null;
-        Path[] fileList = fileSys.listPaths(new Path(OUTPUT_DIR));
+        Path[] fileList = FileUtil.stat2Paths(fileSys.listStatus(
+                                                new Path(OUTPUT_DIR),
+                                                new OutputLogFilter()));
         for (int i = 0; i < fileList.length; i++){
           System.out.println(fileList[i].toString());
           BufferedReader bread =

+ 29 - 0
src/docs/src/documentation/content/xdocs/cluster_setup.xml

@@ -365,6 +365,35 @@
           Commons Logging</a> framework for logging. Edit the 
           <code>conf/log4j.properties</code> file to customize the Hadoop 
           daemons' logging configuration (log-formats and so on).</p>
+          
+          <section>
+            <title>History Logging</title>
+            
+            <p> The job history files are stored in central location 
+            <code> hadoop.job.history.location </code> which can be on DFS also,
+            whose default value is <code>${HADOOP_LOG_DIR}/history</code>. 
+            Job history server is started on job tracker. The history 
+            web UI is accessible from job tracker web UI.</p>
+            
+            <p> The history files are also logged to user specified directory
+            <code>hadoop.job.history.user.location</code> 
+            which defaults to job output directory. The files are stored in
+            "_logs/history/" in the specified directory. Hence, by default 
+            they will be in "mapred.output.dir/_logs/history/". User can stop
+            logging by giving the value <code>none</code> for 
+            <code>hadoop.job.history.user.location</code> </p>
+            
+            <p> User can view logs in specified directory using 
+            the following command <br/>
+            <code>$ bin/hadoop job -history output-dir</code><br/>
+            This will start a stand alone jetty on the client and 
+            load history jsp's. 
+            It will display the port where the server is up at. The server will
+            be up for 30 minutes. User has to use 
+            <code> http://hostname:port </code> to view the history. User can 
+            also provide http bind address using 
+            <code>mapred.job.history.http.bindAddress</code></p>
+          </section>
         </section>
       </section>
       

+ 23 - 1
src/docs/src/documentation/content/xdocs/mapred_tutorial.xml

@@ -1110,7 +1110,29 @@
             monitoring it's status.
           </li>
         </ol>
-  
+        <p> Job history files are also logged to user specified directory
+        <code>hadoop.job.history.user.location</code> 
+        which defaults to job output directory. The files are stored in
+        "_logs/history/" in the specified directory. Hence, by default they will
+        be in mapred.output.dir/_logs/history. User can stop
+        logging by giving the value <code>none</code> for 
+        <code>hadoop.job.history.user.location</code></p>
+
+        <p> User can view logs in specified directory using 
+        the following command <br/>
+        <code>$ bin/hadoop job -history output-dir</code><br/>
+        This will start a stand alone jetty on the client and 
+        load history jsp's. 
+        It will display the port where the server is up at. The server will
+        be up for 30 minutes. User has to use 
+        <code> http://hostname:port </code> to view the history. User can 
+        also provide http bind address using 
+        <code>mapred.job.history.http.bindAddress</code></p>
+            
+        <p> User can use 
+        <a href="ext:api/org/apache/hadoop/mapred/outputlogfilter">OutputLogFilter</a>
+        to filter log files from the output directory listing. </p>
+        
         <p>Normally the user creates the application, describes various facets 
         of the job via <code>JobConf</code>, and then uses the 
         <code>JobClient</code> to submit the job and monitor its progress.</p>

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

@@ -169,6 +169,7 @@ See http://forrest.apache.org/docs/linking.html for more info.
                 <setcompressoutput href="#setCompressOutput(org.apache.hadoop.mapred.JobConf,%20boolean)" />
                 <setoutputcompressorclass href="#setOutputCompressorClass(org.apache.hadoop.mapred.JobConf,%20java.lang.Class)" />
               </outputformatbase>
+              <outputlogfilter href="OutputLogFilter.html" />
               <sequencefileoutputformat href="SequenceFileOutputFormat.html">
                 <setoutputcompressiontype href="#setOutputCompressionType(org.apache.hadoop.mapred.JobConf,%20org.apache.hadoop.io.SequenceFile.CompressionType)" />
               </sequencefileoutputformat>

+ 8 - 68
src/java/org/apache/hadoop/mapred/DefaultJobHistoryParser.java

@@ -20,6 +20,8 @@ package org.apache.hadoop.mapred;
 
 import java.util.*;
 import java.io.*;
+
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.mapred.JobHistory.Keys; 
 import org.apache.hadoop.mapred.JobHistory.Values;
 
@@ -35,40 +37,20 @@ public class DefaultJobHistoryParser {
   // to cast to this type without generating compiler warnings, which is only
   // possible if it is a non-generic class.
 
-  /**
-   * Contents of a job history file. Maps: 
-   * <xmp>jobTrackerId -> <jobId, JobHistory.JobInfo>*</xmp>
-   */
-  public static class MasterIndex 
-    extends TreeMap<String, Map<String, JobHistory.JobInfo>> {
-    
-  }
-
-  /**
-   * Parses a master index file and returns a {@link MasterIndex}.
-   * @param historyFile master index history file. 
-   * @return a {@link MasterIndex}.  
-   * @throws IOException
-   */
-  public static MasterIndex parseMasterIndex(File historyFile)
-    throws IOException {
-    MasterIndexParseListener parser = new MasterIndexParseListener();
-    JobHistory.parseHistory(historyFile, parser);
-
-    return parser.getValues();
-  }
-
   /**
    * Populates a JobInfo object from the job's history log file. 
    * @param jobHistoryFile history file for this job. 
    * @param job a precreated JobInfo object, should be non-null. 
+   * @param fs FileSystem where historyFile is present. 
    * @throws IOException
    */
-  public static void parseJobTasks(File jobHistoryFile, JobHistory.JobInfo job)
+  public static void parseJobTasks(String jobHistoryFile, 
+                       JobHistory.JobInfo job, FileSystem fs)
     throws IOException {
-    JobHistory.parseHistory(jobHistoryFile, 
-                            new JobTasksParseListener(job));
+    JobHistory.parseHistoryFromFS(jobHistoryFile, 
+                            new JobTasksParseListener(job), fs);
   }
+  
   /**
    * Listener for Job's history log file, it populates JobHistory.JobInfo 
    * object with data from log file. 
@@ -144,48 +126,6 @@ public class DefaultJobHistoryParser {
     }
   }
 
-  /**
-   * Parses and returns a map of values in master index. 
-   * 
-   */
-  static class MasterIndexParseListener
-    implements JobHistory.Listener {
-    MasterIndex jobTrackerToJobs = new MasterIndex();
-
-    Map<String, JobHistory.JobInfo> activeJobs = null;
-    String currentTracker; 
-    
-    // Implement JobHistory.Listener
-
-    public void handle(JobHistory.RecordTypes recType, Map<Keys, String> values)
-      throws IOException {
- 
-      if (recType.equals(JobHistory.RecordTypes.Jobtracker)) {
-        activeJobs = new TreeMap<String, JobHistory.JobInfo>();
-        currentTracker = values.get(Keys.START_TIME);
-        jobTrackerToJobs.put(currentTracker, activeJobs);
-      } else if (recType.equals(JobHistory.RecordTypes.Job)) {
-        String jobId = values.get(Keys.JOBID);
-        JobHistory.JobInfo job = activeJobs.get(jobId);
-        if (null == job) {
-          job = new JobHistory.JobInfo(jobId);
-          job.set(Keys.JOBTRACKERID, currentTracker);
-          activeJobs.put(jobId, job);
-        }
-        job.handle(values);
-      }
-    }
-
-    /**
-     * Return map of parsed values. 
-     * @return
-     */ 
-    MasterIndex getValues() {
-      return jobTrackerToJobs;
-    }
-  }
-  
-  
   // call this only for jobs that succeeded for better results. 
   static class FailedOnNodesFilter implements JobHistory.Listener {
     private Map<String, Set<String>> badNodesToNumFailedTasks =

+ 51 - 0
src/java/org/apache/hadoop/mapred/JobClient.java

@@ -1009,6 +1009,7 @@ public class JobClient extends Configured implements MRConstants, Tool  {
     System.out.printf("\t-status\t<job-id>\n");
     System.out.printf("\t-kill\t<job-id>\n");
     System.out.printf("\t-events\t<job-id> <from-event-#> <#-of-events>\n");
+    System.out.printf("\t-history\t<jobOutputDir>\n");
     System.out.printf("\t-list\n");
     System.out.printf("\t-list\tall\n");
     System.out.printf("\t-kill-task <task-id>\n");
@@ -1022,11 +1023,13 @@ public class JobClient extends Configured implements MRConstants, Tool  {
     String submitJobFile = null;
     String jobid = null;
     String taskid = null;
+    String outputDir = null;
     int fromEvent = 0;
     int nEvents = 0;
     boolean getStatus = false;
     boolean killJob = false;
     boolean listEvents = false;
+    boolean viewHistory = false;
     boolean listJobs = false;
     boolean listAllJobs = false;
     boolean killTask = false;
@@ -1056,6 +1059,11 @@ public class JobClient extends Configured implements MRConstants, Tool  {
       fromEvent = Integer.parseInt(argv[2]);
       nEvents = Integer.parseInt(argv[3]);
       listEvents = true;
+    } else if ("-history".equals(argv[0])) {
+      if (argv.length != 2)
+        displayUsage();
+        outputDir = argv[1];
+        viewHistory = true;
     } else if ("-list".equals(argv[0])) {
       if (argv.length != 1 && !(argv.length == 2 && "all".equals(argv[1])))
         displayUsage();
@@ -1112,6 +1120,10 @@ public class JobClient extends Configured implements MRConstants, Tool  {
           System.out.println("Killed job " + jobid);
           exitCode = 0;
         }
+      } else if (viewHistory) {
+    	// start http server
+        viewHistory(outputDir);
+        exitCode = 0;
       } else if (listEvents) {
         listEvents(jobid, fromEvent, nEvents);
         exitCode = 0;
@@ -1144,6 +1156,45 @@ public class JobClient extends Configured implements MRConstants, Tool  {
     return exitCode;
   }
 
+  private void viewHistory(String outputDir) 
+    throws IOException {
+
+    Path output = new Path(outputDir);
+    FileSystem fs = output.getFileSystem(getConf());
+
+    // start http server used to provide an HTML view on Job history
+    StatusHttpServer infoServer;
+    String infoAddr = new JobConf(getConf()).get(
+             "mapred.job.history.http.bindAddress", "0.0.0.0:0");
+    InetSocketAddress infoSocAddr = NetUtils.createSocketAddr(infoAddr);
+    String infoBindAddress = infoSocAddr.getHostName();
+    int tmpInfoPort = infoSocAddr.getPort();
+    infoServer = new StatusHttpServer("history", infoBindAddress, tmpInfoPort,
+                                       tmpInfoPort == 0);
+    infoServer.setAttribute("fileSys", fs);
+    infoServer.setAttribute("historyLogDir", outputDir + "/_logs/history");
+    infoServer.start();
+    int infoPort = infoServer.getPort();
+    getConf().set("mapred.job.history.http.bindAddress", 
+        infoBindAddress + ":" + infoPort);
+    LOG.info("JobHistory webserver up at: " + infoPort);
+
+    // let the server be up for 30 minutes.
+    try {
+      Thread.sleep(30 * 60 * 1000);
+    } catch (InterruptedException ie) {}
+      
+    // stop infoServer
+    if (infoServer != null) {
+      LOG.info("Stopping infoServer");
+      try {
+        infoServer.stop();
+      } catch (InterruptedException ex) {
+        ex.printStackTrace();
+      }
+    } 
+  }
+  
   /**
    * List the events for the given job
    * @param jobId the job id for the job's events to list

+ 188 - 146
src/java/org/apache/hadoop/mapred/JobHistory.java

@@ -24,7 +24,9 @@ import java.io.FileFilter;
 import java.io.FileOutputStream;
 import java.io.FileReader;
 import java.io.IOException;
+import java.io.InputStreamReader;
 import java.io.PrintWriter;
+import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.Map;
@@ -34,7 +36,12 @@ import java.util.regex.Pattern;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.util.StringUtils;
+
 /**
  * Provides methods for writing to and reading from job history. 
  * Job History works in an append mode, JobHistory and its inner classes provide methods 
@@ -60,13 +67,14 @@ public class JobHistory {
   private static final String VALUE = "[[^\"]?]+"; // anything but a " in ""
   
   private static final Pattern pattern = Pattern.compile(KEY + "=" + "\"" + VALUE + "\"");
+  private static final int MAX_FILENAME_SIZE = 255;
   
-  public static final String JOBTRACKER_START_TIME = String.valueOf(System.currentTimeMillis()); 
-  private static final String LOG_DIR = System.getProperty("hadoop.log.dir") + File.separator + "history"; 
-  public static final String MASTER_INDEX_LOG_FILE = "JobHistory.log"; 
-  
-  private static PrintWriter masterIndex = null;
-  private static Map<String, PrintWriter> openJobs = new HashMap<String, PrintWriter>(); 
+  public static final String JOBTRACKER_START_TIME =
+                               String.valueOf(System.currentTimeMillis());
+  private static String JOBTRACKER_UNIQUE_STRING = null;
+  private static String LOG_DIR = null;
+  private static Map<String, ArrayList<PrintWriter>> openJobs = 
+                     new HashMap<String, ArrayList<PrintWriter>>();
   private static boolean disableHistory = false; 
   /**
    * Record types are identifiers for each line of log in history files. 
@@ -101,27 +109,23 @@ public class JobHistory {
   // temp buffer for parsed dataa
   private static Map<Keys,String> parseBuffer = new HashMap<Keys, String>(); 
 
-  // init log files
-  static { init(); } 
-  
   /**
    * Initialize JobHistory files. 
    *
    */
-  private static void init(){
+  public static void init(JobConf conf, String hostname){
     if (!disableHistory){
       try{
-        File logDir = new File(LOG_DIR); 
-        if (!logDir.exists()){
-          if (!logDir.mkdirs()){
+        LOG_DIR = conf.get("hadoop.job.history.location");
+        JOBTRACKER_UNIQUE_STRING = hostname + "_" + 
+                                   JOBTRACKER_START_TIME + "_";
+        Path logDir = new Path(LOG_DIR);
+        FileSystem fs = logDir.getFileSystem(conf);
+        if (!fs.exists(logDir)){
+          if (!fs.mkdirs(logDir)){
             throw new IOException("Mkdirs failed to create " + logDir.toString());
           }
         }
-        masterIndex = 
-          new PrintWriter(
-                          new FileOutputStream(new File(LOG_DIR + File.separator + MASTER_INDEX_LOG_FILE), true));
-        // add jobtracker id = tracker start time
-        log(masterIndex, RecordTypes.Jobtracker, Keys.START_TIME, JOBTRACKER_START_TIME);  
       }catch(IOException e){
         LOG.error("Failed to initialize JobHistory log file", e); 
         disableHistory = true; 
@@ -129,17 +133,19 @@ public class JobHistory {
     }
   }
 
-
   /**
-   * Parses history file and invokes Listener.handle() for each line of history. It can 
-   * be used for looking through history files for specific items without having to keep 
-   * whlole history in memory. 
+   * Parses history file and invokes Listener.handle() for 
+   * each line of history. It can be used for looking through history
+   * files for specific items without having to keep whole history in memory. 
    * @param path path to history file
    * @param l Listener for history events 
+   * @param fs FileSystem where history file is present
    * @throws IOException
    */
-  public static void parseHistory(File path, Listener l) throws IOException{
-    BufferedReader reader = new BufferedReader(new FileReader(path));
+  public static void parseHistoryFromFS(String path, Listener l, FileSystem fs)
+  throws IOException{
+    FSDataInputStream in = fs.open(new Path(path));
+    BufferedReader reader = new BufferedReader(new InputStreamReader (in));
     try {
       String line = null; 
       StringBuffer buf = new StringBuffer(); 
@@ -155,6 +161,7 @@ public class JobHistory {
       try { reader.close(); } catch (IOException ex) {}
     }
   }
+
   /**
    * Parse a single line of history. 
    * @param line
@@ -203,8 +210,8 @@ public class JobHistory {
    * @param values type of log event
    */
 
-  static void log(PrintWriter out, RecordTypes recordType, Keys[] keys, 
-                  String[] values){
+  static void log(ArrayList<PrintWriter> writers, RecordTypes recordType, 
+                  Keys[] keys, String[] values) {
     StringBuffer buf = new StringBuffer(recordType.name()); 
     buf.append(DELIMITER); 
     for(int i =0; i< keys.length; i++){
@@ -215,8 +222,10 @@ public class JobHistory {
       buf.append(DELIMITER); 
     }
     
-    out.println(buf.toString());
-    out.flush(); 
+    for (PrintWriter out : writers) {
+      out.println(buf.toString());
+      out.flush();
+    }
   }
   
   /**
@@ -331,7 +340,8 @@ public class JobHistory {
      * @return the path of the job file on the local file system 
      */
     public static String getLocalJobFilePath(String jobId){
-      return LOG_DIR + File.separator + jobId + "_conf.xml";
+      return System.getProperty("hadoop.log.dir") + File.separator +
+               jobId + "_conf.xml";
     }
     
     /**
@@ -347,23 +357,63 @@ public class JobHistory {
                                     String jobConfPath, long submitTime) {
       String jobName = jobConf.getJobName();
       String user = jobConf.getUser(); 
+      FileSystem fs = null;
+      String userLogDir = null;
+      String jobUniqueString = JOBTRACKER_UNIQUE_STRING + jobId;
+
       if (!disableHistory){
-        synchronized(MASTER_INDEX_LOG_FILE){
-          JobHistory.log(masterIndex, RecordTypes.Job, 
-                         new Keys[]{Keys.JOBID, Keys.JOBNAME, Keys.USER, Keys.SUBMIT_TIME, Keys.JOBCONF }, 
-                         new String[]{jobId, jobName, user, 
-                                      String.valueOf(submitTime), jobConfPath}
-                        );
-        }
         // setup the history log file for this job
-        String logFileName =  JOBTRACKER_START_TIME + "_" + jobId; 
-        File logFile = new File(LOG_DIR + File.separator + logFileName);
-        
+        String logFileName = jobUniqueString +  
+                             "_" + user+ "_" + jobName;
+        if (logFileName.length() > MAX_FILENAME_SIZE) {
+          logFileName = logFileName.substring(0, MAX_FILENAME_SIZE-1);
+        }
+
+        // find user log directory 
+        Path outputPath = jobConf.getOutputPath();
+        userLogDir = jobConf.get("hadoop.job.history.user.location",
+        		outputPath == null ? null : outputPath.toString());
+        if ("none".equals(userLogDir)) {
+          userLogDir = null;
+        }
+        if (userLogDir != null) {
+          userLogDir = userLogDir + "/_logs/history";
+        }
+
+        String logFile = null;
+        String userLogFile = null;
+        if (LOG_DIR != null ) {
+          logFile = LOG_DIR + File.separator + logFileName;
+        }
+        if (userLogDir != null ) {
+          userLogFile = userLogDir + File.separator + logFileName;
+        }
+
         try{
-          PrintWriter writer = new PrintWriter(logFile);
-          openJobs.put(logFileName, writer);
-          // add to writer as well 
-          JobHistory.log(writer, RecordTypes.Job, 
+          ArrayList<PrintWriter> writers = new ArrayList<PrintWriter>();
+          FSDataOutputStream out = null;
+          PrintWriter writer = null;
+
+          if (LOG_DIR != null) {
+            // create output stream for logging in hadoop.job.history.location
+            fs = new Path(LOG_DIR).getFileSystem(jobConf);
+            out = fs.create(new Path(logFile), true, 4096);
+            writer = new PrintWriter(out);
+            writers.add(writer);
+          }
+          if (userLogDir != null) {
+            // create output stream for logging 
+            // in hadoop.job.history.user.location
+            fs = new Path(userLogDir).getFileSystem(jobConf);
+            out = fs.create(new Path(userLogFile), true, 4096);
+            writer = new PrintWriter(out);
+            writers.add(writer);
+          }
+
+          openJobs.put(jobUniqueString, writers);
+
+          //add to writer as well 
+          JobHistory.log(writers, RecordTypes.Job, 
                          new Keys[]{Keys.JOBID, Keys.JOBNAME, Keys.USER, Keys.SUBMIT_TIME, Keys.JOBCONF }, 
                          new String[]{jobId, jobName, user, 
                                       String.valueOf(submitTime) , jobConfPath}
@@ -374,7 +424,7 @@ public class JobHistory {
           disableHistory = true; 
         }
       }
-      /* Storing the job conf on the local file system */
+      // Always store job conf on local file system 
       String localJobFilePath =  JobInfo.getLocalJobFilePath(jobId); 
       File localJobFile = new File(localJobFilePath);
       FileOutputStream jobOut = null;
@@ -393,10 +443,53 @@ public class JobHistory {
             jobOut.close();
           } catch (IOException ie) {
             LOG.info("Failed to close the job configuration file " 
-                     + StringUtils.stringifyException(ie));
+                       + StringUtils.stringifyException(ie));
           }
         }
       }
+
+      /* Storing the job conf on the log dir */
+      Path jobFilePath = null;
+      if (LOG_DIR != null) {
+        jobFilePath = new Path(LOG_DIR + File.separator + 
+                               jobUniqueString + "_conf.xml");
+      }
+      Path userJobFilePath = null;
+      if (userLogDir != null) {
+        userJobFilePath = new Path(userLogDir + File.separator +
+                                   jobUniqueString + "_conf.xml");
+      }
+      FSDataOutputStream jobFileOut = null;
+      try {
+        if (LOG_DIR != null) {
+          fs = new Path(LOG_DIR).getFileSystem(jobConf);
+          if (!fs.exists(jobFilePath)) {
+            jobFileOut = fs.create(jobFilePath);
+            jobConf.write(jobFileOut);
+            jobFileOut.close();
+          }
+        } 
+        if (userLogDir != null) {
+          fs = new Path(userLogDir).getFileSystem(jobConf);
+          jobFileOut = fs.create(userJobFilePath);
+          jobConf.write(jobFileOut);
+        }
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Job conf for " + jobId + " stored at " 
+                    + jobFilePath + "and" + userJobFilePath );
+        }
+      } catch (IOException ioe) {
+        LOG.error("Failed to store job conf on the local filesystem ", ioe);
+      } finally {
+        if (jobFileOut != null) {
+          try {
+            jobFileOut.close();
+          } catch (IOException ie) {
+            LOG.info("Failed to close the job configuration file " 
+                     + StringUtils.stringifyException(ie));
+          }
+        }
+      } 
     }
     /**
      * Logs launch time of job. 
@@ -407,16 +500,9 @@ public class JobHistory {
      */
     public static void logStarted(String jobId, long startTime, int totalMaps, int totalReduces){
       if (!disableHistory){
-        synchronized(MASTER_INDEX_LOG_FILE){
-          JobHistory.log(masterIndex, RecordTypes.Job, 
-                         new Keys[] {Keys.JOBID, Keys.LAUNCH_TIME, Keys.TOTAL_MAPS, Keys.TOTAL_REDUCES },
-                         new String[] {jobId,  String.valueOf(startTime), 
-                                       String.valueOf(totalMaps), String.valueOf(totalReduces) }); 
-        }
-        
-        String logFileName =  JOBTRACKER_START_TIME + "_" + jobId; 
-        PrintWriter writer = openJobs.get(logFileName); 
-        
+        String logFileKey =  JOBTRACKER_UNIQUE_STRING + jobId; 
+        ArrayList<PrintWriter> writer = openJobs.get(logFileKey); 
+
         if (null != writer){
           JobHistory.log(writer, RecordTypes.Job, 
                          new Keys[] {Keys.JOBID, Keys.LAUNCH_TIME, Keys.TOTAL_MAPS, Keys.TOTAL_REDUCES },
@@ -439,20 +525,10 @@ public class JobHistory {
                                    int failedMaps, int failedReduces,
                                    Counters counters){
       if (!disableHistory){
-        synchronized(MASTER_INDEX_LOG_FILE){
-          JobHistory.log(masterIndex, RecordTypes.Job,          
-                         new Keys[] {Keys.JOBID, Keys.FINISH_TIME, 
-                                     Keys.JOB_STATUS, Keys.FINISHED_MAPS, 
-                                     Keys.FINISHED_REDUCES},
-                         new String[] {jobId,  "" + finishTime, 
-                                       Values.SUCCESS.name(), 
-                                       String.valueOf(finishedMaps), 
-                                       String.valueOf(finishedReduces)});
-        }
-        
         // close job file for this job
-        String logFileName = JOBTRACKER_START_TIME + "_" + jobId; 
-        PrintWriter writer = openJobs.get(logFileName); 
+        String logFileKey =  JOBTRACKER_UNIQUE_STRING + jobId; 
+        ArrayList<PrintWriter> writer = openJobs.get(logFileKey); 
+
         if (null != writer){
           JobHistory.log(writer, RecordTypes.Job,          
                          new Keys[] {Keys.JOBID, Keys.FINISH_TIME, 
@@ -467,8 +543,10 @@ public class JobHistory {
                                        String.valueOf(failedMaps), 
                                        String.valueOf(failedReduces),
                                        counters.makeCompactString()});
-          writer.close();
-          openJobs.remove(logFileName); 
+          for (PrintWriter out : writer) {
+            out.close();
+          }
+          openJobs.remove(logFileKey); 
         }
         Thread historyCleaner  = new Thread(new HistoryCleaner());
         historyCleaner.start(); 
@@ -483,21 +561,18 @@ public class JobHistory {
      */
     public static void logFailed(String jobid, long timestamp, int finishedMaps, int finishedReduces){
       if (!disableHistory){
-        synchronized(MASTER_INDEX_LOG_FILE){
-          JobHistory.log(masterIndex, RecordTypes.Job,
-                         new Keys[] {Keys.JOBID, Keys.FINISH_TIME, Keys.JOB_STATUS, Keys.FINISHED_MAPS, Keys.FINISHED_REDUCES },
-                         new String[] {jobid,  String.valueOf(timestamp), Values.FAILED.name(), String.valueOf(finishedMaps), 
-                                       String.valueOf(finishedReduces)}); 
-        }
-        String logFileName =  JOBTRACKER_START_TIME + "_" + jobid; 
-        PrintWriter writer = openJobs.get(logFileName); 
+        String logFileKey =  JOBTRACKER_UNIQUE_STRING + jobid; 
+        ArrayList<PrintWriter> writer = openJobs.get(logFileKey); 
+
         if (null != writer){
           JobHistory.log(writer, RecordTypes.Job,
                          new Keys[] {Keys.JOBID, Keys.FINISH_TIME, Keys.JOB_STATUS, Keys.FINISHED_MAPS, Keys.FINISHED_REDUCES },
                          new String[] {jobid,  String.valueOf(timestamp), Values.FAILED.name(), String.valueOf(finishedMaps), 
                                        String.valueOf(finishedReduces)}); 
-          writer.close();
-          openJobs.remove(logFileName); 
+          for (PrintWriter out : writer) {
+            out.close();
+          }
+          openJobs.remove(logFileKey); 
         }
       }
     }
@@ -520,7 +595,9 @@ public class JobHistory {
     public static void logStarted(String jobId, String taskId, String taskType, 
                                   long startTime){
       if (!disableHistory){
-        PrintWriter writer = openJobs.get(JOBTRACKER_START_TIME + "_" + jobId); 
+        ArrayList<PrintWriter> writer = openJobs.get(JOBTRACKER_UNIQUE_STRING 
+                                                     + jobId); 
+
         if (null != writer){
           JobHistory.log(writer, RecordTypes.Task, 
                          new Keys[]{Keys.TASKID, Keys.TASK_TYPE , Keys.START_TIME}, 
@@ -538,7 +615,9 @@ public class JobHistory {
     public static void logFinished(String jobId, String taskId, String taskType, 
                                    long finishTime, Counters counters){
       if (!disableHistory){
-        PrintWriter writer = openJobs.get(JOBTRACKER_START_TIME + "_" + jobId); 
+        ArrayList<PrintWriter> writer = openJobs.get(JOBTRACKER_UNIQUE_STRING 
+                                                     + jobId); 
+
         if (null != writer){
           JobHistory.log(writer, RecordTypes.Task, 
                          new Keys[]{Keys.TASKID, Keys.TASK_TYPE, 
@@ -560,7 +639,9 @@ public class JobHistory {
      */
     public static void logFailed(String jobId, String taskId, String taskType, long time, String error){
       if (!disableHistory){
-        PrintWriter writer = openJobs.get(JOBTRACKER_START_TIME + "_" + jobId); 
+        ArrayList<PrintWriter> writer = openJobs.get(JOBTRACKER_UNIQUE_STRING 
+                                                     + jobId); 
+
         if (null != writer){
           JobHistory.log(writer, RecordTypes.Task, 
                          new Keys[]{Keys.TASKID, Keys.TASK_TYPE, 
@@ -597,7 +678,9 @@ public class JobHistory {
      */
     public static void logStarted(String jobId, String taskId, String taskAttemptId, long startTime, String hostName){
       if (!disableHistory){
-        PrintWriter writer = openJobs.get(JOBTRACKER_START_TIME + "_" + jobId);
+        ArrayList<PrintWriter> writer = openJobs.get(JOBTRACKER_UNIQUE_STRING 
+                                                     + jobId); 
+
         if (null != writer){
           JobHistory.log(writer, RecordTypes.MapAttempt, 
                          new Keys[]{ Keys.TASK_TYPE, Keys.TASKID, 
@@ -620,7 +703,9 @@ public class JobHistory {
                                    String taskAttemptId, long finishTime, 
                                    String hostName){
       if (!disableHistory){
-        PrintWriter writer = openJobs.get(JOBTRACKER_START_TIME + "_" + jobId);
+        ArrayList<PrintWriter> writer = openJobs.get(JOBTRACKER_UNIQUE_STRING 
+                                                     + jobId); 
+
         if (null != writer){
           JobHistory.log(writer, RecordTypes.MapAttempt, 
                          new Keys[]{ Keys.TASK_TYPE, Keys.TASKID, 
@@ -644,7 +729,9 @@ public class JobHistory {
     public static void logFailed(String jobId, String taskId, String taskAttemptId, 
                                  long timestamp, String hostName, String error){
       if (!disableHistory){
-        PrintWriter writer = openJobs.get(JOBTRACKER_START_TIME + "_" + jobId);
+        ArrayList<PrintWriter> writer = openJobs.get(JOBTRACKER_UNIQUE_STRING 
+                                                     + jobId); 
+
         if (null != writer){
           JobHistory.log(writer, RecordTypes.MapAttempt, 
                          new Keys[]{Keys.TASK_TYPE, Keys.TASKID, Keys.TASK_ATTEMPT_ID, Keys.TASK_STATUS, 
@@ -666,7 +753,9 @@ public class JobHistory {
     public static void logKilled(String jobId, String taskId, String taskAttemptId, 
                                  long timestamp, String hostName, String error){
       if (!disableHistory){
-        PrintWriter writer = openJobs.get(JOBTRACKER_START_TIME + "_" + jobId);
+        ArrayList<PrintWriter> writer = openJobs.get(JOBTRACKER_UNIQUE_STRING 
+                                                     + jobId); 
+
         if (null != writer){
           JobHistory.log(writer, RecordTypes.MapAttempt, 
                          new Keys[]{Keys.TASK_TYPE, Keys.TASKID, Keys.TASK_ATTEMPT_ID, Keys.TASK_STATUS, 
@@ -693,7 +782,9 @@ public class JobHistory {
     public static void logStarted(String jobId, String taskId, String taskAttemptId, 
                                   long startTime, String hostName){
       if (!disableHistory){
-        PrintWriter writer = openJobs.get(JOBTRACKER_START_TIME + "_" + jobId);
+        ArrayList<PrintWriter> writer = openJobs.get(JOBTRACKER_UNIQUE_STRING 
+                                                     + jobId); 
+
         if (null != writer){
           JobHistory.log(writer, RecordTypes.ReduceAttempt, 
                          new Keys[]{  Keys.TASK_TYPE, Keys.TASKID, 
@@ -718,7 +809,9 @@ public class JobHistory {
                                    long sortFinished, long finishTime, 
                                    String hostName){
       if (!disableHistory){
-        PrintWriter writer = openJobs.get(JOBTRACKER_START_TIME + "_" + jobId);
+        ArrayList<PrintWriter> writer = openJobs.get(JOBTRACKER_UNIQUE_STRING 
+                                                     + jobId); 
+
         if (null != writer){
           JobHistory.log(writer, RecordTypes.ReduceAttempt, 
                          new Keys[]{ Keys.TASK_TYPE, Keys.TASKID, 
@@ -743,7 +836,9 @@ public class JobHistory {
     public static void logFailed(String jobId, String taskId, String taskAttemptId, long timestamp, 
                                  String hostName, String error){
       if (!disableHistory){
-        PrintWriter writer = openJobs.get(JOBTRACKER_START_TIME + "_" + jobId);
+        ArrayList<PrintWriter> writer = openJobs.get(JOBTRACKER_UNIQUE_STRING 
+                                                     + jobId); 
+
         if (null != writer){
           JobHistory.log(writer, RecordTypes.ReduceAttempt, 
                          new Keys[]{  Keys.TASK_TYPE, Keys.TASKID, Keys.TASK_ATTEMPT_ID, Keys.TASK_STATUS, 
@@ -765,7 +860,9 @@ public class JobHistory {
     public static void logKilled(String jobId, String taskId, String taskAttemptId, long timestamp, 
                                  String hostName, String error){
       if (!disableHistory){
-        PrintWriter writer = openJobs.get(JOBTRACKER_START_TIME + "_" + jobId);
+        ArrayList<PrintWriter> writer = openJobs.get(JOBTRACKER_UNIQUE_STRING 
+                                                     + jobId); 
+
         if (null != writer){
           JobHistory.log(writer, RecordTypes.ReduceAttempt, 
                          new Keys[]{  Keys.TASK_TYPE, Keys.TASKID, 
@@ -822,61 +919,6 @@ public class JobHistory {
       }
       lastRan = now;  
       isRunning = true; 
-      // update master Index first
-      try{
-        File logFile = new File(
-                                LOG_DIR + File.separator + MASTER_INDEX_LOG_FILE); 
-        
-        synchronized(MASTER_INDEX_LOG_FILE){
-          Map<String, Map<String, JobHistory.JobInfo>> jobTrackersToJobs = 
-            DefaultJobHistoryParser.parseMasterIndex(logFile);
-          
-          // find job that started more than one month back and remove them
-          // for jobtracker instances which dont have a job in past one month 
-          // remove the jobtracker start timestamp as well.
-          Iterator<Map<String, JobHistory.JobInfo>> jobTrackerItr =
-            jobTrackersToJobs.values().iterator();
-          while (jobTrackerItr.hasNext()) {
-            Map<String, JobHistory.JobInfo> jobs = jobTrackerItr.next();
-            Iterator<Map.Entry<String, JobHistory.JobInfo>> jobItr = 
-                   jobs.entrySet().iterator();
-            while (jobItr.hasNext()) {
-              Map.Entry<String, JobHistory.JobInfo> item = jobItr.next();
-              if (now - item.getValue().getLong(Keys.SUBMIT_TIME) > 
-                  THIRTY_DAYS_IN_MS) {
-                jobItr.remove(); 
-              }
-            }
-            if (jobs.size() == 0){
-              jobTrackerItr.remove(); 
-            }
-          }
-          masterIndex.close(); 
-          masterIndex = new PrintWriter(logFile);
-          // delete old history and write back to a new file
-          for (Map.Entry<String, Map<String, JobHistory.JobInfo>> entry :
-                  jobTrackersToJobs.entrySet()) {
-            String jobTrackerId = entry.getKey();
-            Map<String, JobHistory.JobInfo> jobs = entry.getValue();
-
-            
-            log(masterIndex, RecordTypes.Jobtracker, Keys.START_TIME, jobTrackerId);
-
-            for(JobHistory.JobInfo job : jobs.values()){
-              Map<Keys, String> values = job.getValues();
-              
-              log(masterIndex, RecordTypes.Job, 
-                  values.keySet().toArray(new Keys[0]), 
-                  values.values().toArray(new String[0])); 
-
-            }
-            masterIndex.flush();
-          }
-        }
-      }catch(IOException e){
-        LOG.error("Failed loading history log for cleanup", e);
-      }
-      
       File[] oldFiles = new File(LOG_DIR).listFiles(new FileFilter(){
           public boolean accept(File file){
             // delete if older than 30 days

+ 32 - 0
src/java/org/apache/hadoop/mapred/JobTracker.java

@@ -593,6 +593,7 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
 
   // Used to provide an HTML view on Job, Task, and TaskTracker structures
   StatusHttpServer infoServer;
+  StatusHttpServer historyServer;
   int infoPort;
 
   Server interTrackerServer;
@@ -701,6 +702,26 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
       Thread.sleep(SYSTEM_DIR_CLEANUP_RETRY_PERIOD);
     }
 
+    // start history viewing server.
+    JobHistory.init(conf, this.localMachine); 
+    String histAddr = conf.get("mapred.job.history.http.bindAddress",
+                                  "0.0.0.0:0");
+    InetSocketAddress historySocAddr = NetUtils.createSocketAddr(histAddr);
+    String historyBindAddress = historySocAddr.getHostName();
+    int tmpHistoryPort = historySocAddr.getPort();
+    historyServer = new StatusHttpServer("history", historyBindAddress, 
+                       tmpHistoryPort, tmpHistoryPort == 0);
+    String historyLogDir = conf.get("hadoop.job.history.location");
+    historyServer.setAttribute("historyLogDir", historyLogDir);
+    FileSystem fileSys = new Path(historyLogDir).getFileSystem(conf);
+    historyServer.setAttribute("fileSys", fileSys);
+    historyServer.start();
+    this.conf.set("mapred.job.history.http.bindAddress", 
+                (this.localMachine + ":" + historyServer.getPort()));
+    LOG.info("JobHistory webserver on JobTracker up at: " +
+              historyServer.getPort());
+
+
     // Same with 'localDir' except it's always on the local disk.
     jobConf.deleteLocalFiles(SUBDIR);
     synchronized (this) {
@@ -719,6 +740,9 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
     return NetUtils.createSocketAddr(jobTrackerStr);
   }
 
+  public String getHistoryAddress() {
+    return conf.get("mapred.job.history.http.bindAddress");
+  }
 
   /**
    * Run forever
@@ -754,6 +778,14 @@ public class JobTracker implements MRConstants, InterTrackerProtocol, JobSubmiss
         ex.printStackTrace();
       }
     }
+    if (this.historyServer != null) {
+      LOG.info("Stopping historyServer");
+      try {
+        this.historyServer.stop();
+      } catch (InterruptedException ex) {
+        ex.printStackTrace();
+      }
+    }
     if (this.interTrackerServer != null) {
       LOG.info("Stopping interTrackerServer");
       this.interTrackerServer.stop();

+ 35 - 0
src/java/org/apache/hadoop/mapred/OutputLogFilter.java

@@ -0,0 +1,35 @@
+/**
+ * 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;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
+
+/**
+ * This class filters log files from directory given
+ * It doesnt accept paths having _logs.
+ * This can be used to list paths of output directory as follows:
+ *   Path[] fileList = FileUtil.stat2Paths(fs.listStatus(outDir,
+ *                                   new OutputLogFilter()));
+ */
+public class OutputLogFilter implements PathFilter {
+  public boolean accept(Path path) {
+    return !(path.toString().contains("_logs"));
+  }
+}

+ 4 - 2
src/test/org/apache/hadoop/mapred/TestClusterMapReduceTestCase.java

@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.mapred;
 
+import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
@@ -61,8 +62,9 @@ public class TestClusterMapReduceTestCase extends ClusterMapReduceTestCase {
 
     JobClient.runJob(conf);
 
-    Path[] outputFiles = getFileSystem().listPaths(getOutputDir());
-
+    Path[] outputFiles = FileUtil.stat2Paths(
+                           getFileSystem().listStatus(getOutputDir(),
+                           new OutputLogFilter()));
     if (outputFiles.length > 0) {
       InputStream is = getFileSystem().open(outputFiles[0]);
       BufferedReader reader = new BufferedReader(new InputStreamReader(is));

+ 5 - 3
src/test/org/apache/hadoop/mapred/TestMiniMRClasspath.java

@@ -24,6 +24,7 @@ import junit.framework.TestCase;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.dfs.MiniDFSCluster;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.Text;
@@ -76,7 +77,8 @@ public class TestMiniMRClasspath extends TestCase {
     StringBuffer result = new StringBuffer();
     {
       Path[] parents = fs.listPaths(outDir.getParent());
-      Path[] fileList = fs.listPaths(outDir);
+      Path[] fileList = FileUtil.stat2Paths(fs.listStatus(outDir,
+              new OutputLogFilter()));
       for(int i=0; i < fileList.length; ++i) {
         BufferedReader file = 
           new BufferedReader(new InputStreamReader(fs.open(fileList[i])));
@@ -130,8 +132,8 @@ public class TestMiniMRClasspath extends TestCase {
     conf.setJar("build/test/testjar/testjob.jar");
     JobClient.runJob(conf);
     StringBuffer result = new StringBuffer();
-
-    Path[] fileList = fs.listPaths(outDir);
+    Path[] fileList = FileUtil.stat2Paths(fs.listStatus(outDir,
+                                 new OutputLogFilter()));
     for (int i = 0; i < fileList.length; ++i) {
       BufferedReader file = new BufferedReader(new InputStreamReader(
                                                                      fs.open(fileList[i])));

+ 5 - 1
src/test/org/apache/hadoop/mapred/TestMiniMRWithDFS.java

@@ -27,6 +27,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.dfs.MiniDFSCluster;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.Text;
@@ -91,8 +92,11 @@ public class TestMiniMRWithDFS extends TestCase {
     FileSystem fs = outDir.getFileSystem(conf);
     StringBuffer result = new StringBuffer();
     {
-      Path[] fileList = fs.listPaths(outDir);
+      
+      Path[] fileList = FileUtil.stat2Paths(fs.listStatus(outDir,
+                                   new OutputLogFilter()));
       for(int i=0; i < fileList.length; ++i) {
+        LOG.info("File list[" + i + "]" + ": "+ fileList[i]);
         BufferedReader file = 
           new BufferedReader(new InputStreamReader(fs.open(fileList[i])));
         String line = file.readLine();

+ 3 - 1
src/test/org/apache/hadoop/mapred/pipes/TestPipes.java

@@ -32,6 +32,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.MiniMRCluster;
+import org.apache.hadoop.mapred.OutputLogFilter;
 import org.apache.hadoop.mapred.RunningJob;
 import org.apache.hadoop.mapred.TestMiniMRWithDFS;
 import org.apache.hadoop.util.StringUtils;
@@ -144,7 +145,8 @@ public class TestPipes extends TestCase {
     RunningJob result = Submitter.submitJob(job);
     assertTrue("pipes job failed", result.isSuccessful());
     List<String> results = new ArrayList<String>();
-    for (Path p:fs.listPaths(outputPath)) {
+    for (Path p:FileUtil.stat2Paths(fs.listStatus(outputPath,
+    		                        new OutputLogFilter()))) {
       results.add(TestMiniMRWithDFS.readOutput(p, job));
     }
     assertEquals("number of reduces is wrong", 

+ 13 - 13
src/webapps/job/analysejobhistory.jsp → src/webapps/history/analysejobhistory.jsp

@@ -10,13 +10,13 @@
 %>
 <jsp:include page="loadhistory.jsp">
 	<jsp:param name="jobid" value="<%=request.getParameter("jobid") %>"/>
-	<jsp:param name="jobTrackerId" value="<%=request.getParameter("jobTrackerId") %>"/>
+	<jsp:param name="logFile" value="<%=request.getParameter("logFile") %>"/>
 </jsp:include>
 <%!	private static SimpleDateFormat dateFormat = new SimpleDateFormat("d/MM HH:mm:ss") ; %>
 <html><body>
 <%
 	String jobid = request.getParameter("jobid");
-	String jobTrackerId = request.getParameter("jobTrackerId");
+	String logFile = request.getParameter("logFile");
 	String numTasks = request.getParameter("numTasks");
 	int showTasks = 10 ; 
 	if( numTasks != null ) {
@@ -26,7 +26,7 @@
 	JobInfo job = (JobInfo)request.getSession().getAttribute("job");
 
 %>
-<h2>Hadoop Job <a href="jobdetailshistory.jsp?jobid=<%=jobid%>&&jobTrackerId=<%=jobTrackerId %>"><%=jobid %> </a></h2>
+<h2>Hadoop Job <a href="jobdetailshistory.jsp?jobid=<%=jobid%>&&logFile=<%=logFile %>"><%=jobid %> </a></h2>
 
 <b>User : </b> <%=job.get(Keys.USER) %><br/> 
 <b>JobName : </b> <%=job.get(Keys.JOBNAME) %><br/> 
@@ -105,7 +105,7 @@
 %>
 
 <h3>Time taken by best performing Map task 
-<a href="taskdetailshistory.jsp?jobid=<%=jobid%>&jobTrackerId=<%=jobTrackerId%>&taskid=<%=minMap.get(Keys.TASKID)%>">
+<a href="taskdetailshistory.jsp?jobid=<%=jobid%>&logFile=<%=logFile%>&taskid=<%=minMap.get(Keys.TASKID)%>">
 <%=minMap.get(Keys.TASKID) %></a> : <%=StringUtils.formatTimeDiff(minMap.getLong(Keys.FINISH_TIME), minMap.getLong(Keys.START_TIME) ) %></h3>
 <h3>Average time taken by Map tasks: 
 <%=StringUtils.formatTimeDiff(avgMapTime, 0) %></h3>
@@ -116,7 +116,7 @@
 	for( int i=0;i<showTasks && i<mapTasks.length; i++){
 %>
 		<tr>
-			<td><a href="taskdetailshistory.jsp?jobid=<%=jobid%>&jobTrackerId=<%=jobTrackerId%>&taskid=<%=mapTasks[i].get(Keys.TASKID)%>">
+			<td><a href="taskdetailshistory.jsp?jobid=<%=jobid%>&logFile=<%=logFile%>&taskid=<%=mapTasks[i].get(Keys.TASKID)%>">
   		    <%=mapTasks[i].get(Keys.TASKID) %></a></td>
 			<td><%=StringUtils.formatTimeDiff(mapTasks[i].getLong(Keys.FINISH_TIME), mapTasks[i].getLong(Keys.START_TIME)) %></td>
 		</tr>
@@ -137,7 +137,7 @@
     JobHistory.Task lastMap = mapTasks[0] ;
 %>
 <h3>The last Map task 
-<a href="taskdetailshistory.jsp?jobid=<%=jobid%>&jobTrackerId=<%=jobTrackerId%>
+<a href="taskdetailshistory.jsp?jobid=<%=jobid%>&logFile=<%=logFile%>
 &taskid=<%=lastMap.get(Keys.TASKID)%>"><%=lastMap.get(Keys.TASKID) %></a> 
 finished at (relative to the Job launch time): 
 <%=StringUtils.getFormattedTimeWithDiff(dateFormat, 
@@ -145,7 +145,7 @@ finished at (relative to the Job launch time):
                               job.getLong(Keys.LAUNCH_TIME) ) %></h3>
 <hr/>
 <h3>Time taken by best performing shuffle
-<a href="taskdetailshistory.jsp?jobid=<%=jobid%>&jobTrackerId=<%=jobTrackerId%>
+<a href="taskdetailshistory.jsp?jobid=<%=jobid%>&logFile=<%=logFile%>
 &taskid=<%=minShuffle.get(Keys.TASKID)%>"><%=minShuffle.get(Keys.TASKID)%></a> : 
 <%=StringUtils.formatTimeDiff(minShuffle.getLong(Keys.SHUFFLE_FINISHED), 
                               minShuffle.getLong(Keys.START_TIME) ) %></h3>
@@ -158,8 +158,8 @@ finished at (relative to the Job launch time):
 	for( int i=0;i<showTasks && i<reduceTasks.length; i++){
 %>
       <tr>
-	    <td><a href="taskdetailshistory.jsp?jobid=<%=jobid%>&jobTrackerId=
-	    <%=jobTrackerId%>&taskid=<%=reduceTasks[i].get(Keys.TASKID)%>">
+	    <td><a href="taskdetailshistory.jsp?jobid=<%=jobid%>&logFile=
+	    <%=logFile%>&taskid=<%=reduceTasks[i].get(Keys.TASKID)%>">
 			<%=reduceTasks[i].get(Keys.TASKID) %></a></td>
 	    <td><%=
 	      StringUtils.formatTimeDiff(
@@ -185,7 +185,7 @@ finished at (relative to the Job launch time):
 %>
 
 <h3>The last Shuffle  
-<a href="taskdetailshistory.jsp?jobid=<%=jobid%>&jobTrackerId=<%=jobTrackerId%>
+<a href="taskdetailshistory.jsp?jobid=<%=jobid%>&logFile=<%=logFile%>
 &taskid=<%=lastShuffle.get(Keys.TASKID)%>"><%=lastShuffle.get(Keys.TASKID)%>
 </a> finished at (relative to the Job launch time): 
 <%=StringUtils.getFormattedTimeWithDiff(dateFormat,
@@ -207,7 +207,7 @@ finished at (relative to the Job launch time):
 %>
 <hr/>
 <h3>Time taken by best performing Reduce task : 
-<a href="taskdetailshistory.jsp?jobid=<%=jobid%>&jobTrackerId=<%=jobTrackerId%>&taskid=<%=minReduce.get(Keys.TASKID)%>">
+<a href="taskdetailshistory.jsp?jobid=<%=jobid%>&logFile=<%=logFile%>&taskid=<%=minReduce.get(Keys.TASKID)%>">
 <%=minReduce.get(Keys.TASKID) %></a> : 
 <%=StringUtils.formatTimeDiff(minReduce.getLong(Keys.FINISH_TIME),
     minReduce.getLong(Keys.SHUFFLE_FINISHED) ) %></h3>
@@ -221,7 +221,7 @@ finished at (relative to the Job launch time):
 	for( int i=0;i<showTasks && i<reduceTasks.length; i++){
 %>
 		<tr>
-			<td><a href="taskdetailshistory.jsp?jobid=<%=jobid%>&jobTrackerId=<%=jobTrackerId%>&taskid=<%=reduceTasks[i].get(Keys.TASKID)%>">
+			<td><a href="taskdetailshistory.jsp?jobid=<%=jobid%>&logFile=<%=logFile%>&taskid=<%=reduceTasks[i].get(Keys.TASKID)%>">
 			<%=reduceTasks[i].get(Keys.TASKID) %></a></td>
 			<td><%=StringUtils.formatTimeDiff(
 			    reduceTasks[i].getLong(Keys.FINISH_TIME), 
@@ -237,7 +237,7 @@ finished at (relative to the Job launch time):
 %>
 
 <h3>The last Reduce task 
-<a href="taskdetailshistory.jsp?jobid=<%=jobid%>&jobTrackerId=<%=jobTrackerId%>
+<a href="taskdetailshistory.jsp?jobid=<%=jobid%>&logFile=<%=logFile%>
 &taskid=<%=lastReduce.get(Keys.TASKID)%>"><%=lastReduce.get(Keys.TASKID)%>
 </a> finished at (relative to the Job launch time): 
 <%=StringUtils.getFormattedTimeWithDiff(dateFormat,

+ 20 - 0
src/webapps/history/index.html

@@ -0,0 +1,20 @@
+<meta HTTP-EQUIV="REFRESH" content="0;url=jobhistory.jsp"/>
+<html>
+
+<head>
+<title>Hadoop Administration - History</title>
+</head>
+
+<body>
+
+<h1>Hadoop Administration - History</h1>
+
+<ul>
+
+<li><a href="jobhistory.jsp">Job History</a></li>
+
+</ul>
+
+</body>
+
+</html>

+ 58 - 0
src/webapps/history/jobconf_history.jsp

@@ -0,0 +1,58 @@
+<%@ page
+  contentType="text/html; charset=UTF-8"
+  import="javax.servlet.*"
+  import="javax.servlet.http.*"
+  import="java.io.*"
+  import="java.net.URL"
+  import="org.apache.hadoop.mapred.*"
+  import="org.apache.hadoop.fs.*"
+  import="org.apache.hadoop.util.*"
+%>
+
+
+<%
+  JobTracker tracker = (JobTracker) application.getAttribute("job.tracker");
+  String jobId = request.getParameter("jobid");
+  if (jobId == null) {
+    out.println("<h2>Missing 'jobid' for fetching job configuration!</h2>");
+ 	return;
+  }
+%>
+  
+<html>
+
+<title>Job Configuration: JobId - <%= jobId %></title>
+
+<body>
+<h2>Job Configuration: JobId - <%= jobId %></h2><br>
+
+<%
+  Path logDir = new Path(request.getParameter("jobLogDir"));
+  Path jobFilePath = new Path(logDir, 
+                       request.getParameter("jobUniqueString") + "_conf.xml");
+  FileSystem fs = (FileSystem)request.getSession().getAttribute("fs");
+  FSDataInputStream jobFile = null; 
+  try {
+    jobFile = fs.open(jobFilePath);
+    JobConf jobConf = new JobConf(jobFilePath);
+    XMLUtils.transform(
+        jobConf.getConfResourceAsInputStream("webapps/static/jobconf.xsl"),
+        jobFile, out);
+  } catch (Exception e) {
+    out.println("Failed to retreive job configuration for job '" + jobId + "!");
+    out.println(e);
+  } finally {
+    if (jobFile != null) {
+      try { 
+        jobFile.close(); 
+      } catch (IOException e) {}
+    }
+  } 
+%>
+
+<br>
+<hr>
+<a href="http://lucene.apache.org/hadoop">Hadoop</a>, 2007.<br>
+
+</body>
+</html>

+ 25 - 24
src/webapps/job/jobdetailshistory.jsp → src/webapps/history/jobdetailshistory.jsp

@@ -3,6 +3,7 @@
   import="javax.servlet.http.*"
   import="java.io.*"
   import="java.util.*"
+  import="org.apache.hadoop.fs.*"
   import="org.apache.hadoop.mapred.*"
   import="org.apache.hadoop.util.*"
   import="java.text.SimpleDateFormat"
@@ -10,26 +11,31 @@
 %>
 <jsp:include page="loadhistory.jsp">
 	<jsp:param name="jobid" value="<%=request.getParameter("jobid") %>"/>
-	<jsp:param name="jobTrackerId" value="<%=request.getParameter("jobTrackerId") %>"/>
+	<jsp:param name="logFile" value="<%=request.getParameter("logFile") %>"/>
 </jsp:include>
 <%! static SimpleDateFormat dateFormat = new SimpleDateFormat("d-MMM-yyyy HH:mm:ss") ; %>
 <%
 	String jobid = request.getParameter("jobid");
-	String jobTrackerId = request.getParameter("jobTrackerId");
+	String logFile = request.getParameter("logFile");
+	
+	Path jobFile = new Path(logFile);
+	String[] jobDetails = jobFile.getName().split("_");
+    String jobUniqueString = jobDetails[0] + "_" +jobDetails[1] + "_" + jobid ;
 	
 	JobInfo job = (JobInfo)request.getSession().getAttribute("job");
+	FileSystem fs = (FileSystem)request.getSession().getAttribute("fs");
 %>
 <html><body>
-<h2>Hadoop Job <%=jobid %> </h2>
+<h2>Hadoop Job <%=jobid %> on <a href="jobhistory.jsp">History Viewer</a></h2>
 
 <b>User: </b> <%=job.get(Keys.USER) %><br/> 
 <b>JobName: </b> <%=job.get(Keys.JOBNAME) %><br/> 
-<b>JobConf: </b> <a href="jobconf.jsp?jobid=<%=jobid %>"> 
+<b>JobConf: </b> <a href="jobconf_history.jsp?jobid=<%=jobid%>&jobLogDir=<%=new Path(logFile).getParent().toString()%>&jobUniqueString=<%=jobUniqueString%>"> 
                  <%=job.get(Keys.JOBCONF) %></a><br/> 
 <b>Submitted At: </b> <%=StringUtils.getFormattedTimeWithDiff(dateFormat, job.getLong(Keys.SUBMIT_TIME), 0 )  %><br/> 
 <b>Launched At: </b> <%=StringUtils.getFormattedTimeWithDiff(dateFormat, job.getLong(Keys.LAUNCH_TIME), job.getLong(Keys.SUBMIT_TIME)) %><br/>
 <b>Finished At: </b>  <%=StringUtils.getFormattedTimeWithDiff(dateFormat, job.getLong(Keys.FINISH_TIME), job.getLong(Keys.LAUNCH_TIME)) %><br/>
-<b>Status: </b> <%= ((job.get(Keys.JOB_STATUS) == null)?"Incomplete" :job.get(Keys.JOB_STATUS)) %><br/> 
+<b>Status: </b> <%= ((job.get(Keys.JOB_STATUS) == "")?"Incomplete" :job.get(Keys.JOB_STATUS)) %><br/> 
 <%
 	Map<String, JobHistory.Task> tasks = job.getAllTasks();
 	int totalMaps = 0 ; 
@@ -51,7 +57,7 @@
 	  long startTime = task.getLong(Keys.START_TIME) ; 
 	  long finishTime = task.getLong(Keys.FINISH_TIME) ; 
 	  
-          allHosts.put(task.get(Keys.HOSTNAME), null);
+          allHosts.put(task.get(Keys.HOSTNAME), "");
 
 	  if( Values.MAP.name().equals(task.get(Keys.TASK_TYPE)) ){
 	    if( mapStarted==0 || mapStarted > startTime ){
@@ -91,8 +97,7 @@
 	  }
 	}
 %>
-<b>Number of nodes used: </b> <%=allHosts.size() %><br/>
-<b><a href="analysejobhistory.jsp?jobid=<%=jobid %>&jobTrackerId=<%=jobTrackerId %>">Analyse This Job</a></b> 
+<b><a href="analysejobhistory.jsp?jobid=<%=jobid %>&logFile=<%=logFile %>">Analyse This Job</a></b> 
 <hr/>
 <center>
 <table border="2" cellpadding="5" cellspacing="2">
@@ -101,26 +106,26 @@
 </tr>
 <tr>
 <td>Map</td>
-	<td><a href="jobtaskshistory.jsp?jobid=<%=jobid %>&jobTrackerId=<%=jobTrackerId %>&taskType=<%=Values.MAP.name() %>&status=all">
+	<td><a href="jobtaskshistory.jsp?jobid=<%=jobid %>&logFile=<%=logFile %>&taskType=<%=Values.MAP.name() %>&status=all">
 	  <%=totalMaps %></a></td>
-	<td><a href="jobtaskshistory.jsp?jobid=<%=jobid %>&jobTrackerId=<%=jobTrackerId %>&taskType=<%=Values.MAP.name() %>&status=<%=Values.SUCCESS %>">
+	<td><a href="jobtaskshistory.jsp?jobid=<%=jobid %>&logFile=<%=logFile %>&taskType=<%=Values.MAP.name() %>&status=<%=Values.SUCCESS %>">
 	  <%=job.getInt(Keys.FINISHED_MAPS) %></a></td>
-	<td><a href="jobtaskshistory.jsp?jobid=<%=jobid %>&jobTrackerId=<%=jobTrackerId %>&taskType=<%=Values.MAP.name() %>&status=<%=Values.FAILED %>">
+	<td><a href="jobtaskshistory.jsp?jobid=<%=jobid %>&logFile=<%=logFile %>&taskType=<%=Values.MAP.name() %>&status=<%=Values.FAILED %>">
 	  <%=failedMaps %></a></td>
-	<td><a href="jobtaskshistory.jsp?jobid=<%=jobid %>&jobTrackerId=<%=jobTrackerId %>&taskType=<%=Values.MAP.name() %>&status=<%=Values.KILLED %>">
+	<td><a href="jobtaskshistory.jsp?jobid=<%=jobid %>&logFile=<%=logFile %>&taskType=<%=Values.MAP.name() %>&status=<%=Values.KILLED %>">
 	  <%=killedMaps %></a></td>
 	<td><%=StringUtils.getFormattedTimeWithDiff(dateFormat, mapStarted, 0) %></td>
 	<td><%=StringUtils.getFormattedTimeWithDiff(dateFormat, mapFinished, mapStarted) %></td>
 </tr>
 <tr>
 <td>Reduce</td>
-	<td><a href="jobtaskshistory.jsp?jobid=<%=jobid %>&jobTrackerId=<%=jobTrackerId %>&taskType=<%=Values.REDUCE.name() %>&status=all">
+	<td><a href="jobtaskshistory.jsp?jobid=<%=jobid %>&logFile=<%=logFile %>&taskType=<%=Values.REDUCE.name() %>&status=all">
 	  <%=totalReduces%></a></td>
-	<td><a href="jobtaskshistory.jsp?jobid=<%=jobid %>&jobTrackerId=<%=jobTrackerId %>&taskType=<%=Values.REDUCE.name() %>&status=<%=Values.SUCCESS %>">
+	<td><a href="jobtaskshistory.jsp?jobid=<%=jobid %>&logFile=<%=logFile %>&taskType=<%=Values.REDUCE.name() %>&status=<%=Values.SUCCESS %>">
 	  <%=job.getInt(Keys.FINISHED_REDUCES)%></a></td>
-	<td><a href="jobtaskshistory.jsp?jobid=<%=jobid %>&jobTrackerId=<%=jobTrackerId %>&taskType=<%=Values.REDUCE.name() %>&status=<%=Values.FAILED %>">
+	<td><a href="jobtaskshistory.jsp?jobid=<%=jobid %>&logFile=<%=logFile %>&taskType=<%=Values.REDUCE.name() %>&status=<%=Values.FAILED %>">
 	  <%=failedReduces%></a></td>
-	<td><a href="jobtaskshistory.jsp?jobid=<%=jobid %>&jobTrackerId=<%=jobTrackerId %>&taskType=<%=Values.REDUCE.name() %>&status=<%=Values.KILLED %>">
+	<td><a href="jobtaskshistory.jsp?jobid=<%=jobid %>&logFile=<%=logFile %>&taskType=<%=Values.REDUCE.name() %>&status=<%=Values.KILLED %>">
 	  <%=killedReduces%></a></td>  
 	<td><%=StringUtils.getFormattedTimeWithDiff(dateFormat, reduceStarted, 0) %></td>
 	<td><%=StringUtils.getFormattedTimeWithDiff(dateFormat, reduceFinished, reduceStarted) %></td>
@@ -130,9 +135,7 @@
 <br/>
  <%
 	DefaultJobHistoryParser.FailedOnNodesFilter filter = new DefaultJobHistoryParser.FailedOnNodesFilter();
-	String dir = System.getProperty("hadoop.log.dir") + File.separator + "history" ; 
- 
-	JobHistory.parseHistory(new File(dir, jobTrackerId+"_" + jobid), filter); 
+	JobHistory.parseHistoryFromFS(logFile, filter, fs); 
 	Map<String, Set<String>> badNodes = filter.getValues(); 
 	if( badNodes.size() > 0 ) {
  %>
@@ -150,7 +153,7 @@
 <%
 		for( String t : failedTasks ) {
 %>
-		 <a href="taskdetailshistory.jsp?jobid=<%=jobid%>&jobTrackerId=<%=jobTrackerId %>&taskid=<%=t %>"><%=t %></a>,&nbsp;
+		 <a href="taskdetailshistory.jsp?jobid=<%=jobid%>&logFile=<%=logFile %>&taskid=<%=t %>"><%=t %></a>,&nbsp;
 <%		  
 		}
 %>	
@@ -164,9 +167,7 @@
 <br/>
  <%
 	DefaultJobHistoryParser.KilledOnNodesFilter killedFilter = new DefaultJobHistoryParser.KilledOnNodesFilter();
-	dir = System.getProperty("hadoop.log.dir") + File.separator + "history" ; 
- 
-	JobHistory.parseHistory(new File(dir, jobTrackerId+"_" + jobid), filter); 
+	JobHistory.parseHistoryFromFS(logFile, filter, fs); 
 	badNodes = killedFilter.getValues(); 
 	if( badNodes.size() > 0 ) {
  %>
@@ -184,7 +185,7 @@
 <%
 		for( String t : killedTasks ) {
 %>
-		 <a href="taskdetailshistory.jsp?jobid=<%=jobid%>&jobTrackerId=<%=jobTrackerId %>&taskid=<%=t %>"><%=t %></a>,&nbsp;
+		 <a href="taskdetailshistory.jsp?jobid=<%=jobid%>&logFile=<%=logFile %>&taskid=<%=t %>"><%=t %></a>,&nbsp;
 <%		  
 		}
 %>	

+ 104 - 0
src/webapps/history/jobhistory.jsp

@@ -0,0 +1,104 @@
+<%@ page
+  contentType="text/html; charset=UTF-8"
+  import="java.io.*"
+  import="java.util.*"
+  import="org.apache.hadoop.mapred.*"
+  import="org.apache.hadoop.util.*"
+  import="org.apache.hadoop.fs.*"
+  import="javax.servlet.jsp.*"
+  import="java.text.SimpleDateFormat"
+  import="org.apache.hadoop.mapred.*"
+  import="org.apache.hadoop.mapred.JobHistory.*"
+%>
+<%!	
+	private static SimpleDateFormat dateFormat = new SimpleDateFormat("d/MM HH:mm:ss") ;
+%>
+<html>
+<head>
+<title>Hadoop Map/Reduce Administration</title>
+<link rel="stylesheet" type="text/css" href="/static/hadoop.css">
+</head>
+<body>
+<h1>Hadoop Map/Reduce History Viewer</h1>
+<hr>
+<h2>Available History </h2>
+<%
+    PathFilter jobLogFileFilter = new PathFilter() {
+      public boolean accept(Path path) {
+        return !(path.getName().endsWith(".xml"));
+      }
+    };
+    
+	FileSystem fs = (FileSystem) application.getAttribute("fileSys");
+	String historyLogDir = (String) application.getAttribute("historyLogDir");
+	Path[] jobFiles = fs.listPaths(new Path(historyLogDir), jobLogFileFilter);
+
+    // sort the files on creation time.
+    Arrays.sort(jobFiles, new Comparator<Path>() {
+      public int compare(Path p1, Path p2) {
+        String[] split1 = p1.getName().split("_");
+        String[] split2 = p2.getName().split("_");
+        
+        // compare job tracker start time
+        int res = new Date(Long.parseLong(split1[1])).compareTo(
+                             new Date(Long.parseLong(split2[1])));
+        if (res == 0) {
+          res = new Date(Long.parseLong(split1[3])).compareTo(
+                           new Date(Long.parseLong(split2[3])));
+        }
+        if (res == 0) {
+          Long l1 = Long.parseLong(split1[4]);
+          res = l1.compareTo(Long.parseLong(split2[4]));
+        }
+        
+        return res;
+      }
+    });
+
+    if (null == jobFiles ){
+      out.println("NULL !!!"); 
+      return ; 
+    }
+       
+    out.print("<table align=center border=2 cellpadding=\"5\" cellspacing=\"2\">");
+    out.print("<tr><td align=\"center\" colspan=\"9\"><b>Available Jobs </b></td></tr>\n");
+    out.print("<tr>");
+    out.print("<td>Job tracker Host Name</td>" +
+              "<td>Job tracker Start time</td>" +
+              "<td>Job Id</td><td>Name</td><td>User</td>") ; 
+    out.print("</tr>"); 
+    for (Path jobFile: jobFiles) {
+      String[] jobDetails = jobFile.getName().split("_");
+      String trackerHostName = jobDetails[0];
+      String trackerStartTime = jobDetails[1];
+      String jobId = jobDetails[2] + "_" +jobDetails[3] + "_" + jobDetails[4] ;
+      String user = jobDetails[5];
+      String jobName = jobDetails[6];
+      
+%>
+<center>
+<%	
+
+	  printJob(trackerHostName, trackerStartTime, jobId,
+               jobName, user, jobFile.toString(), out) ; 
+%>
+</center> 
+<%
+	} // end while trackers 
+%>
+<%!
+	private void printJob(String trackerHostName, String trackerid,
+                          String jobId, String jobName,
+                          String user, String logFile, JspWriter out)
+    throws IOException{
+	    out.print("<tr>"); 
+	    out.print("<td>" + trackerHostName + "</td>"); 
+	    out.print("<td>" + new Date(Long.parseLong(trackerid)) + "</td>"); 
+	    out.print("<td>" + "<a href=\"jobdetailshistory.jsp?jobid="+ jobId + 
+	        "&logFile=" + logFile +"\">" + jobId + "</a></td>"); 
+	    out.print("<td>" + jobName + "</td>"); 
+	    out.print("<td>" + user + "</td>"); 
+	    out.print("</tr>");
+	}
+ %> 
+</body></html>

+ 5 - 5
src/webapps/job/jobtaskshistory.jsp → src/webapps/history/jobtaskshistory.jsp

@@ -10,13 +10,13 @@
 %>
 <jsp:include page="loadhistory.jsp">
 	<jsp:param name="jobid" value="<%=request.getParameter("jobid") %>"/>
-	<jsp:param name="jobTrackerId" value="<%=request.getParameter("jobTrackerId") %>"/>
+	<jsp:param name="logFile" value="<%=request.getParameter("logFile") %>"/>
 </jsp:include>
 <%!	private static SimpleDateFormat dateFormat = new SimpleDateFormat("d/MM HH:mm:ss") ; %>
 
 <%	
 	String jobid = request.getParameter("jobid");
-	String jobTrackerId = request.getParameter("jobTrackerId");
+	String logFile = request.getParameter("logFile");
 	String taskStatus = request.getParameter("status"); 
 	String taskType = request.getParameter("taskType"); 
 	
@@ -26,7 +26,7 @@
 %>
 <html>
 <body>
-<h2><%=taskStatus%> <%=taskType %> task list for <a href="jobdetailshistory.jsp?jobid=<%=jobid%>&&jobTrackerId=<%=jobTrackerId %>"><%=jobid %> </a></h2>
+<h2><%=taskStatus%> <%=taskType %> task list for <a href="jobdetailshistory.jsp?jobid=<%=jobid%>&&logFile=<%=logFile %>"><%=jobid %> </a></h2>
 <center>
 <table border="2" cellpadding="5" cellspacing="2">
 <tr><td>Task Id</td><td>Start Time</td><td>Finish Time<br/></td><td>Error</td></tr>
@@ -36,7 +36,7 @@
             Map <String, TaskAttempt> taskAttempts = task.getTaskAttempts();
             for (JobHistory.TaskAttempt taskAttempt : taskAttempts.values()) {
 	      if( taskStatus.equals(taskAttempt.get(Keys.TASK_STATUS)) || taskStatus.equals("all")){
-	         printTask(jobid, jobTrackerId, task, out); 
+	         printTask(jobid, logFile, task, out); 
 	      }
             }
 	  }
@@ -47,7 +47,7 @@
 	private void printTask(String jobid, String trackerId, JobHistory.Task task, JspWriter out) throws IOException{
   		out.print("<tr>"); 
   		out.print("<td>" + "<a href=\"taskdetailshistory.jsp?jobid=" + jobid + 
-  		    "&jobTrackerId="+ trackerId +"&taskid="+task.get(Keys.TASKID)+"\">" + 
+  		    "&logFile="+ trackerId +"&taskid="+task.get(Keys.TASKID)+"\">" + 
   		    task.get(Keys.TASKID) + "</a></td>");
   		out.print("<td>" + StringUtils.getFormattedTimeWithDiff(dateFormat, 
   		    task.getLong(Keys.START_TIME), 0 ) + "</td>");

+ 35 - 0
src/webapps/history/loadhistory.jsp

@@ -0,0 +1,35 @@
+<%@ page
+  contentType="text/html; charset=UTF-8"
+  import="java.io.*"
+  import="java.util.*"
+  import="org.apache.hadoop.mapred.*"
+  import="org.apache.hadoop.fs.*"
+  import="org.apache.hadoop.util.*"
+  import="javax.servlet.jsp.*"
+  import="java.text.SimpleDateFormat"  
+  import="org.apache.hadoop.mapred.JobHistory.*"
+%>
+<%
+    PathFilter jobLogFileFilter = new PathFilter() {
+      public boolean accept(Path path) {
+        return !(path.getName().endsWith(".xml"));
+      }
+    };
+    
+	FileSystem fs = (FileSystem) application.getAttribute("fileSys");
+	String jobId =  (String)request.getParameter("jobid");
+	JobHistory.JobInfo job = (JobHistory.JobInfo)request.getSession().getAttribute("job");
+	if (null != job && (!jobId.equals(job.get(Keys.JOBID)))){
+      // remove jobInfo from session, keep only one job in session at a time
+      request.getSession().removeAttribute("job"); 
+      job = null ; 
+    }
+	
+	if (null == job) {
+      String jobLogFile = (String)request.getParameter("logFile");
+      job = new JobHistory.JobInfo(jobId); 
+      DefaultJobHistoryParser.parseJobTasks(jobLogFile, job, fs) ; 
+      request.getSession().setAttribute("job", job);
+      request.getSession().setAttribute("fs", fs);
+	}
+%>

+ 2 - 2
src/webapps/job/taskdetailshistory.jsp → src/webapps/history/taskdetailshistory.jsp

@@ -16,7 +16,7 @@
 
 <%	
 	String jobid = request.getParameter("jobid");
-	String jobTrackerId = request.getParameter("jobTrackerId");
+	String logFile = request.getParameter("logFile");
 	String taskid = request.getParameter("taskid"); 
 
 	JobHistory.JobInfo job = (JobHistory.JobInfo)request.getSession().getAttribute("job");
@@ -24,7 +24,7 @@
 %>
 <html>
 <body>
-<h2><%=taskid %> attempts for <a href="jobdetailshistory.jsp?jobid=<%=jobid%>&&jobTrackerId=<%=jobTrackerId %>"> <%=jobid %> </a></h2>
+<h2><%=taskid %> attempts for <a href="jobdetailshistory.jsp?jobid=<%=jobid%>&&logFile=<%=logFile %>"> <%=jobid %> </a></h2>
 <center>
 <table border="2" cellpadding="5" cellspacing="2">
 <tr><td>Task Id</td><td>Start Time</td>

+ 0 - 99
src/webapps/job/jobhistory.jsp

@@ -1,99 +0,0 @@
-<%@ page
-  contentType="text/html; charset=UTF-8"
-  import="java.io.*"
-  import="java.util.*"
-  import="org.apache.hadoop.mapred.*"
-  import="org.apache.hadoop.util.*"
-  import="javax.servlet.jsp.*"
-  import="java.text.SimpleDateFormat"
-  import="org.apache.hadoop.mapred.*"
-  import="org.apache.hadoop.mapred.JobHistory.*"
-%>
-<jsp:include page="loadhistory.jsp">
-	<jsp:param name="historyFile" value="<%=request.getParameter("historyFile") %>"/>
-	<jsp:param name="reload" value="<%=request.getParameter("reload") %>"/>
-</jsp:include>
-<%!	
-	private static SimpleDateFormat dateFormat = new SimpleDateFormat("d/MM HH:mm:ss") ;
-%>
-
-<html><body>
-<%
-		DefaultJobHistoryParser.MasterIndex jobTrackerToJobs = 
-		  (DefaultJobHistoryParser.MasterIndex)request.getSession().
-		  getAttribute("jobHistory"); 
-		
-		if( null == jobTrackerToJobs ){
-		  out.println("NULL !!!"); 
-		  return ; 
-		}
-
-    for (Map.Entry<String, Map<String, JobInfo>> entry : 
-            jobTrackerToJobs.entrySet()) {
-      String trackerStartTime = entry.getKey();
-      Map<String, JobInfo> jobs = entry.getValue();
-%>
-<h2>JobTracker started at: <%=new Date(Long.parseLong(trackerStartTime)) %></h2>
-<hr/>
-<h3>Completed Jobs </h3>
-<center>
-<%	
-		printJobs(trackerStartTime, jobs, Values.SUCCESS.name(), out) ; 
-%>
-</center> 
-<h3>Failed Jobs </h3>
-<center>
-<%	
-		printJobs(trackerStartTime, jobs, Values.FAILED.name() , out) ; 
-%>
-</center>
-<h3>Incomplete Jobs </h3>
-<center>
-<%	
-		printJobs(trackerStartTime, jobs, null , out) ; 
-%>
-</center>
-<hr/><br/>
-<%
-		} // end while trackers 
-%>
- 
-<%!
-	private void printJobs(String trackerid, Map<String, JobInfo> jobs, String status, JspWriter out) throws IOException{
-	  if( jobs.size() == 0 ) {
-	    out.print("<h3>No Jobs available</h3>"); 
-	    return ; 
-	  }
-      out.print("<table border=2 cellpadding=\"5\" cellspacing=\"2\">");
-      out.print("<tr>");
-   	  out.print("<td align=\"center\">Job Id</td><td>Name</td><td>User</td><td>Submit Time</td>" + 
-   	      "<td>Finish Time</td><td>Total Maps</td><td>Total Reduces</td>" +
-   		  "<td>Finished Maps</td><td>Finished Reduces</td>") ; 
-   	  out.print("</tr>"); 
-   	      
-	  for( JobInfo job : jobs.values() ) {
-		if( null != status && status.equals(job.get(Keys.JOB_STATUS)) ) {
-		  printJob(trackerid, job, out); 
-		}else if( status == null && job.get(Keys.JOB_STATUS).length() == 0 ) {
-		  printJob(trackerid, job, out); 
-		}
-	  }
-	  out.print("</table>");
-	}
-
-	private void printJob(String trackerid, JobInfo job, JspWriter out)throws IOException{
-	    out.print("<tr>"); 
-	    out.print("<td>" + "<a href=\"jobdetailshistory.jsp?jobid="+ job.get(Keys.JOBID) + 
-	        "&jobTrackerId=" +trackerid  + "\">" + job.get(Keys.JOBID) + "</a></td>"); 
-	    out.print("<td>" + job.get(Keys.JOBNAME) + "</td>"); 
-	    out.print("<td>" + job.get(Keys.USER) + "</td>"); 
-	    out.print("<td>" + StringUtils.getFormattedTimeWithDiff(dateFormat, job.getLong(Keys.SUBMIT_TIME),0) + "</td>"); 
-	    out.print("<td>" + StringUtils.getFormattedTimeWithDiff(dateFormat, job.getLong(Keys.FINISH_TIME) , job.getLong(Keys.SUBMIT_TIME) ) + "</td>");
-	    out.print("<td>" + job.get(Keys.TOTAL_MAPS) + "</td>"); 
-	    out.print("<td>" + job.get(Keys.TOTAL_REDUCES) + "</td>"); 
-	    out.print("<td>" + job.get(Keys.FINISHED_MAPS) + "</td>"); 
-	    out.print("<td>" + job.get(Keys.FINISHED_REDUCES) + "</td>"); 
-	    out.print("</tr>");
-	}
- %>  
-</body></html>

+ 1 - 1
src/webapps/job/jobtracker.jsp

@@ -134,7 +134,7 @@
 <hr>
 
 <h2>Local logs</h2>
-<a href="logs/">Log</a> directory, <a href="jobhistory.jsp?historyFile=JobHistory.log&reload=true">
+<a href="logs/">Log</a> directory, <a href="http://<%=tracker.getHistoryAddress()%>">
 Job Tracker History</a>
 
 <%

+ 0 - 49
src/webapps/job/loadhistory.jsp

@@ -1,49 +0,0 @@
-<%@ page
-  contentType="text/html; charset=UTF-8"
-  import="java.io.*"
-  import="java.util.*"
-  import="org.apache.hadoop.mapred.*"
-  import="org.apache.hadoop.util.*"
-  import="javax.servlet.jsp.*"
-  import="java.text.SimpleDateFormat"  
-  import="org.apache.hadoop.mapred.JobHistory.*"
-%>
-<%
-	// Reload master index or a job file in session
-	String reload = request.getParameter("reload"); 
-	String jobid = request.getParameter("jobid"); 
-	String jobTrackerId = request.getParameter("jobTrackerId"); 
-	
-	String jobLogDir = System.getProperty("hadoop.log.dir") + File.separator + "history" ; 
-	
-	String masterIndex = request.getParameter("historyFile"); ;
-	
-	if( null != masterIndex ) {
-		String filePath = jobLogDir + File.separator + masterIndex ;
-		File historyFile = new File(filePath); 
-		if( null == request.getSession().getAttribute("jobHistory") || "true".equals(reload) ){
-		  request.getSession().setAttribute("jobHistory", 
-				DefaultJobHistoryParser.parseMasterIndex(historyFile)); 
-		}
-	}
-
-	if( jobid != null && jobTrackerId != null ) {
-	  
-		JobHistory.JobInfo job = (JobHistory.JobInfo)request.getSession().getAttribute("job");
-		if( null != job && (! jobid.equals(job.get(Keys.JOBID)) || 
-		    ! jobTrackerId.equals(job.get(Keys.JOBTRACKERID)))){
-		  // remove jobInfo from session, keep only one job in session at a time
-		  request.getSession().removeAttribute("job"); 
-		  job = null ; 
-		}
-		
-		if( null == job ) {
-  		  String jobLogFile = jobTrackerId + "_" + jobid; 
-		  job = new JobHistory.JobInfo(jobid); 
-		  job.set(Keys.JOBTRACKERID, jobTrackerId);
-	      DefaultJobHistoryParser.parseJobTasks(
-		  	    new File(jobLogDir + File.separator + jobLogFile), job) ; 
-		  request.getSession().setAttribute("job", job); 
-		}
-	}
-%>

Beberapa file tidak ditampilkan karena terlalu banyak file yang berubah dalam diff ini