You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by cu...@apache.org on 2006/10/04 23:46:20 UTC
svn commit: r453041 - in /lucene/hadoop/trunk: ./
src/java/org/apache/hadoop/mapred/ src/webapps/job/
Author: cutting
Date: Wed Oct 4 14:46:20 2006
New Revision: 453041
URL: http://svn.apache.org/viewvc?view=rev&rev=453041
Log:
HADOOP-239. Add a persistent job history mechanism. Contributed by Sanjay.
Added:
lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/DefaultJobHistoryParser.java
lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobHistory.java
lucene/hadoop/trunk/src/webapps/job/analysejobhistory.jsp
lucene/hadoop/trunk/src/webapps/job/jobdetailshistory.jsp
lucene/hadoop/trunk/src/webapps/job/jobhistory.jsp
lucene/hadoop/trunk/src/webapps/job/jobtaskshistory.jsp
lucene/hadoop/trunk/src/webapps/job/loadhistory.jsp
lucene/hadoop/trunk/src/webapps/job/taskdetailshistory.jsp
Modified:
lucene/hadoop/trunk/CHANGES.txt
lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobInProgress.java
lucene/hadoop/trunk/src/webapps/job/jobtracker.jsp
Modified: lucene/hadoop/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/CHANGES.txt?view=diff&rev=453041&r1=453040&r2=453041
==============================================================================
--- lucene/hadoop/trunk/CHANGES.txt (original)
+++ lucene/hadoop/trunk/CHANGES.txt Wed Oct 4 14:46:20 2006
@@ -135,6 +135,10 @@
32. HADOOP-343. Fix mapred copying so that a failed tasktracker
doesn't cause other copies to slow. (Sameer Paranjpye via cutting)
+33. HADOOP-239. Add a persistent job history mechanism, so that basic
+ job statistics are not lost after 24 hours and/or when the
+ jobtracker is restarted. (Sanjay Dahiya via cutting)
+
Release 0.6.2 - 2006-09-18
Added: lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/DefaultJobHistoryParser.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/DefaultJobHistoryParser.java?view=auto&rev=453041
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/DefaultJobHistoryParser.java (added)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/DefaultJobHistoryParser.java Wed Oct 4 14:46:20 2006
@@ -0,0 +1,156 @@
+package org.apache.hadoop.mapred;
+
+import java.util.*;
+import java.io.*;
+import org.apache.hadoop.mapred.JobHistory.Keys ;
+
+/**
+ * Default parser for job history files. It creates object model from
+ * job history file.
+ *
+ */
+public class DefaultJobHistoryParser {
+
+ /**
+ * Parses a master index file and returns a Map of
+ * (jobTrakerId - Map (job Id - JobHistory.JobInfo)).
+ * @param historyFile master index history file.
+ * @return a map of values, as described as described earlier.
+ * @throws IOException
+ */
+ public static Map<String, Map<String, JobHistory.JobInfo>> 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.
+ * @throws IOException
+ */
+ public static void parseJobTasks(File jobHistoryFile, JobHistory.JobInfo job)
+ throws IOException {
+ JobHistory.parseHistory(jobHistoryFile,
+ new JobTasksParseListener(job));
+ }
+/**
+ * Listener for Job's history log file, it populates JobHistory.JobInfo
+ * object with data from log file.
+ */
+ static class JobTasksParseListener
+ implements JobHistory.Listener {
+ JobHistory.JobInfo job;
+
+ JobTasksParseListener(JobHistory.JobInfo job) {
+ this.job = job;
+ }
+
+ private JobHistory.Task getTask(String taskId) {
+ JobHistory.Task task = job.getAllTasks().get(taskId);
+ if (null == task) {
+ task = new JobHistory.Task();
+ task.set(Keys.TASKID, taskId);
+ job.getAllTasks().put(taskId, task);
+ }
+ return task;
+ }
+
+ private JobHistory.MapAttempt getMapAttempt(
+ String jobid, String jobTrackerId, String taskId, String taskAttemptId) {
+
+ JobHistory.Task task = getTask(taskId);
+ JobHistory.MapAttempt mapAttempt =
+ (JobHistory.MapAttempt) task.getTaskAttempts().get(taskAttemptId);
+ if (null == mapAttempt) {
+ mapAttempt = new JobHistory.MapAttempt();
+ mapAttempt.set(Keys.TASK_ATTEMPT_ID, taskAttemptId);
+ task.getTaskAttempts().put(taskAttemptId, mapAttempt);
+ }
+ return mapAttempt;
+ }
+
+ private JobHistory.ReduceAttempt getReduceAttempt(
+ String jobid, String jobTrackerId, String taskId, String taskAttemptId) {
+
+ JobHistory.Task task = getTask(taskId);
+ JobHistory.ReduceAttempt reduceAttempt =
+ (JobHistory.ReduceAttempt) task.getTaskAttempts().get(taskAttemptId);
+ if (null == reduceAttempt) {
+ reduceAttempt = new JobHistory.ReduceAttempt();
+ reduceAttempt.set(Keys.TASK_ATTEMPT_ID, taskAttemptId);
+ task.getTaskAttempts().put(taskAttemptId, reduceAttempt);
+ }
+ return reduceAttempt;
+ }
+
+ // JobHistory.Listener implementation
+ public void handle(JobHistory.RecordTypes recType, Map<Keys, String> values)
+ throws IOException {
+ String jobTrackerId = values.get(JobHistory.Keys.JOBTRACKERID);
+ String jobid = values.get(Keys.JOBID);
+
+ if (recType == JobHistory.RecordTypes.Job) {
+ job.handle(values);
+ }if (recType.equals(JobHistory.RecordTypes.Task)) {
+ String taskid = values.get(JobHistory.Keys.TASKID);
+ getTask(taskid).handle(values);
+ } else if (recType.equals(JobHistory.RecordTypes.MapAttempt)) {
+ String taskid = values.get(Keys.TASKID);
+ String mapAttemptId = values.get(Keys.TASK_ATTEMPT_ID);
+
+ getMapAttempt(jobid, jobTrackerId, taskid, mapAttemptId).handle(values);
+ } else if (recType.equals(JobHistory.RecordTypes.ReduceAttempt)) {
+ String taskid = values.get(Keys.TASKID);
+ String reduceAttemptId = values.get(Keys.TASK_ATTEMPT_ID);
+
+ getReduceAttempt(jobid, jobTrackerId, taskid, reduceAttemptId).handle(values);
+ }
+ }
+ }
+
+ /**
+ * Parses and returns a map of values in master index.
+ * @author sanjaydahiya
+ *
+ */
+ static class MasterIndexParseListener
+ implements JobHistory.Listener {
+ Map<String, Map<String, JobHistory.JobInfo>> jobTrackerToJobs = new TreeMap<String, Map<String, JobHistory.JobInfo>>();
+
+ 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 = (String) 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
+ */
+ Map<String, Map<String, JobHistory.JobInfo>> getValues() {
+ return jobTrackerToJobs;
+ }
+ }
+}
Added: lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobHistory.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobHistory.java?view=auto&rev=453041
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobHistory.java (added)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobHistory.java Wed Oct 4 14:46:20 2006
@@ -0,0 +1,748 @@
+package org.apache.hadoop.mapred;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileFilter;
+import java.io.FileOutputStream;
+import java.io.FileReader;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+/**
+ * Provides methods for writing to and reading from job history.
+ * Job History works in an append mode, JobHistory and its inner classes provide methods
+ * to log job events.
+ *
+ * JobHistory is split into multiple files, format of each file is plain text where each line
+ * is of the format [type (key=value)*], where type identifies the type of the record.
+ * Type maps to UID of one of the inner classes of this class.
+ *
+ * Job history is maintained in a master index which contains star/stop times of all jobs with
+ * a few other job level properties. Apart from this each job's history is maintained in a seperate history
+ * file. name of job history files follows the format jobtrackerId_jobid
+ *
+ * For parsing the job history it supports a listener based interface where each line is parsed
+ * and passed to listener. The listener can create an object model of history or look for specific
+ * events and discard rest of the history.
+ *
+ */
+public class JobHistory {
+
+ public static final Log LOG = LogFactory.getLog("org.apache.hadoop.mapred.JobHistory");
+ private static final String DELIMITER = " ";
+ private static final String KEY = "(\\w+)";
+ private static final String VALUE = "[[^\"]?]+" ; // anything but a " in ""
+
+ private static final Pattern pattern = Pattern.compile(KEY + "=" + "\"" + VALUE + "\"");
+
+ 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>();
+ private static boolean disableHistory = false;
+ /**
+ * Record types are identifiers for each line of log in history files.
+ * A record type appears as the first token in a single line of log.
+ */
+ public static enum RecordTypes {Jobtracker, Job, Task, MapAttempt, ReduceAttempt};
+ /**
+ * Job history files contain key="value" pairs, where keys belong to this enum.
+ * It acts as a global namespace for all keys.
+ */
+ public static enum Keys { JOBTRACKERID,
+ START_TIME, FINISH_TIME, JOBID, JOBNAME, USER, JOBCONF,SUBMIT_TIME, LAUNCH_TIME,
+ TOTAL_MAPS, TOTAL_REDUCES, FAILED_MAPS, FAILED_REDUCES, FINISHED_MAPS, FINISHED_REDUCES,
+ JOB_STATUS, TASKID, HOSTNAME, TASK_TYPE, ERROR, TASK_ATTEMPT_ID, TASK_STATUS,
+ COPY_PHASE, SORT_PHASE, REDUCE_PHASE, SHUFFLE_FINISHED, SORT_FINISHED
+ };
+ /**
+ * This enum contains some of the values commonly used by history log events.
+ * since values in history can only be strings - Values.name() is used in
+ * most places in history file.
+ */
+ public static enum Values {
+ SUCCESS, FAILED, KILLED, MAP, REDUCE
+ };
+ // 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(){
+ if( !disableHistory ){
+ try{
+ File logDir = new File(LOG_DIR);
+ if( ! logDir.exists() ){
+ logDir.mkdirs();
+ }
+ 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 ;
+ }
+ }
+ }
+
+
+ /**
+ * 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.
+ * @param path path to history file
+ * @param l Listener for history events
+ * @throws IOException
+ */
+ public static void parseHistory(File path, Listener l) throws IOException{
+ BufferedReader reader = new BufferedReader(new FileReader(path));
+ String line = null ;
+ StringBuffer buf = new StringBuffer();
+ while ((line = reader.readLine())!= null){
+ buf.append(line);
+ if( ! line.trim().endsWith("\"")){
+ continue ;
+ }
+ parseLine(buf.toString(), l );
+ buf = new StringBuffer();
+ }
+ }
+ /**
+ * Parse a single line of history.
+ * @param line
+ * @param l
+ * @throws IOException
+ */
+ private static void parseLine(String line, Listener l)throws IOException{
+ // extract the record type
+ int idx = line.indexOf(' ');
+ String recType = line.substring(0, idx) ;
+ String data = line.substring(idx+1, line.length()) ;
+
+ Matcher matcher = pattern.matcher(data);
+
+ while(matcher.find()){
+ String tuple = matcher.group(0);
+ String []parts = tuple.split("=");
+
+ parseBuffer.put(Keys.valueOf(parts[0]), parts[1].substring(1, parts[1].length() -1));
+ }
+
+ l.handle(RecordTypes.valueOf(recType), parseBuffer);
+
+ parseBuffer.clear();
+ }
+
+
+ /**
+ * Log a raw record type with keys and values. This is method is generally not used directly.
+ * @param recordType type of log event
+ * @param key key
+ * @param value value
+ */
+
+ static void log(PrintWriter out, RecordTypes recordType, Enum key, String value){
+ out.println(recordType.name() + DELIMITER + key + "=\"" + value + "\"");
+ out.flush();
+ }
+
+ /**
+ * Log a number of keys and values with record. the array length of keys and values
+ * should be same.
+ * @param recordType type of log event
+ * @param keys type of log event
+ * @param values type of log event
+ */
+
+ static void log(PrintWriter out, RecordTypes recordType, Enum[] keys, String[] values){
+ StringBuffer buf = new StringBuffer(recordType.name()) ;
+ buf.append(DELIMITER) ;
+ for( int i =0 ; i< keys.length ; i++ ){
+ buf.append(keys[i]);
+ buf.append("=\"");
+ buf.append(values[i]);
+ buf.append("\"");
+ buf.append(DELIMITER);
+ }
+
+ out.println(buf.toString());
+ out.flush();
+ }
+
+ /**
+ * Returns history disable status. by default history is enabled so this
+ * method returns false.
+ * @return true if history logging is disabled, false otherwise.
+ */
+ public static boolean isDisableHistory() {
+ return disableHistory;
+ }
+
+ /**
+ * Enable/disable history logging. Default value is false, so history
+ * is enabled by default.
+ * @param disableHistory true if history should be disabled, false otherwise.
+ */
+ public static void setDisableHistory(boolean disableHistory) {
+ JobHistory.disableHistory = disableHistory;
+ }
+
+ /**
+ * Base class contais utility stuff to manage types key value pairs with enums.
+ */
+ static class KeyValuePair{
+ private Map<Keys, String> values = new HashMap<Keys, String>();
+
+ /**
+ * Get 'String' value for given key. Most of the places use Strings as
+ * values so the default get' method returns 'String'. This method never returns
+ * null to ease on GUIs. if no value is found it returns empty string ""
+ * @param k
+ * @return if null it returns empty string - ""
+ */
+ public String get(Keys k){
+ String s = values.get(k);
+ return s == null ? "" : s ;
+ }
+ /**
+ * Convert value from history to int and return.
+ * if no value is found it returns 0.
+ * @param k key
+ * @return
+ */
+ public int getInt(Keys k){
+ String s = values.get(k);
+ if( null != s ){
+ return Integer.parseInt(s);
+ }
+ return 0;
+ }
+ /**
+ * Convert value from history to int and return.
+ * if no value is found it returns 0.
+ * @param k
+ * @return
+ */
+ public long getLong(Keys k){
+ String s = values.get(k);
+ if( null != s ){
+ return Long.parseLong(s);
+ }
+ return 0;
+ }
+ /**
+ * Set value for the key.
+ * @param k
+ * @param s
+ */
+ public void set(Keys k, String s){
+ values.put(k, s);
+ }
+ /**
+ * Adds all values in the Map argument to its own values.
+ * @param m
+ */
+ public void set(Map<Keys, String> m){
+ values.putAll(m);
+ }
+ /**
+ * Reads values back from the history, input is same Map as passed to Listener by parseHistory().
+ * @param values
+ */
+ public void handle(Map<Keys, String> values){
+ set(values);
+ }
+ /**
+ * Returns Map containing all key-values.
+ * @return
+ */
+ public Map<Keys, String> getValues(){
+ return values;
+ }
+ }
+
+ /**
+ * Helper class for logging or reading back events related to job start, finish or failure.
+ */
+ public static class JobInfo extends KeyValuePair{
+
+ private Map<String, Task> allTasks = new TreeMap<String, Task>();
+
+ /** Create new JobInfo */
+ public JobInfo(String jobId){
+ set(Keys.JOBID, jobId);
+ }
+
+ /**
+ * Returns all map and reduce tasks <taskid-Task>.
+ * @return
+ */
+ public Map<String, Task> getAllTasks() { return allTasks; }
+
+ /**
+ * Log job submitted event to history. Creates a new file in history
+ * for the job. if history file creation fails, it disables history
+ * for all other events.
+ * @param jobId job id assigned by job tracker.
+ * @param jobName job name as given by user in job conf
+ * @param user user name
+ * @param submitTime time when job tracker received the job
+ * @param jobConf path to job conf xml file in HDFS.
+ */
+ public static void logSubmitted(String jobId, String jobName, String user,
+ long submitTime, String jobConf){
+
+ if( ! disableHistory ){
+ synchronized(MASTER_INDEX_LOG_FILE){
+ JobHistory.log(masterIndex, RecordTypes.Job,
+ new Enum[]{Keys.JOBID, Keys.JOBNAME, Keys.USER, Keys.SUBMIT_TIME, Keys.JOBCONF },
+ new String[]{jobId, jobName, user, String.valueOf(submitTime),jobConf });
+ }
+ // setup the history log file for this job
+ String logFileName = JOBTRACKER_START_TIME + "_" + jobId ;
+ File logFile = new File(LOG_DIR + File.separator + logFileName);
+
+ try{
+ PrintWriter writer = new PrintWriter(logFile);
+ openJobs.put(logFileName, writer);
+ // add to writer as well
+ JobHistory.log(writer, RecordTypes.Job,
+ new Enum[]{Keys.JOBID, Keys.JOBNAME, Keys.USER, Keys.SUBMIT_TIME, Keys.JOBCONF },
+ new String[]{jobId, jobName, user, String.valueOf(submitTime) ,jobConf});
+
+ }catch(IOException e){
+ LOG.error("Failed creating job history log file, disabling history", e);
+ disableHistory = true ;
+ }
+ }
+ }
+ /**
+ * Logs launch time of job.
+ * @param jobId job id, assigned by jobtracker.
+ * @param startTime start time of job.
+ * @param totalMaps total maps assigned by jobtracker.
+ * @param totalReduces total reduces.
+ */
+ 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 Enum[] {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 = (PrintWriter)openJobs.get(logFileName);
+
+ if( null != writer ){
+ JobHistory.log(writer, RecordTypes.Job,
+ new Enum[] {Keys.JOBID, Keys.LAUNCH_TIME,Keys.TOTAL_MAPS, Keys.TOTAL_REDUCES },
+ new String[] {jobId, String.valueOf(startTime), String.valueOf(totalMaps), String.valueOf(totalReduces)} ) ;
+ }
+ }
+ }
+ /**
+ * Log job finished. closes the job file in history.
+ * @param jobId job id, assigned by jobtracker.
+ * @param finishTime finish time of job in ms.
+ * @param finishedMaps no of maps successfully finished.
+ * @param finishedReduces no of reduces finished sucessfully.
+ * @param failedMaps no of failed map tasks.
+ * @param failedReduces no of failed reduce tasks.
+ */
+ public static void logFinished(String jobId, long finishTime, int finishedMaps, int finishedReduces,
+ int failedMaps, int failedReduces){
+ if( ! disableHistory ){
+ synchronized(MASTER_INDEX_LOG_FILE){
+ JobHistory.log(masterIndex, RecordTypes.Job,
+ new Enum[] {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);
+ if( null != writer){
+ JobHistory.log(writer, RecordTypes.Job,
+ new Enum[] {Keys.JOBID, Keys.FINISH_TIME, Keys.JOB_STATUS, Keys.FINISHED_MAPS, Keys.FINISHED_REDUCES,
+ Keys.FAILED_MAPS, Keys.FAILED_REDUCES},
+ new String[] {jobId, "" + finishTime, Values.SUCCESS.name(),
+ String.valueOf(finishedMaps), String.valueOf(finishedReduces),
+ String.valueOf(failedMaps), String.valueOf(failedReduces)} ) ;
+ writer.close();
+ openJobs.remove(logFileName);
+ }
+ Thread historyCleaner = new Thread( new HistoryCleaner() );
+ historyCleaner.start();
+ }
+ }
+ /**
+ * Logs job failed event. Closes the job history log file.
+ * @param jobid job id
+ * @param timestamp time when job failure was detected in ms.
+ * @param finishedMaps no finished map tasks.
+ * @param finishedReduces no of finished reduce tasks.
+ */
+ 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 Enum[] {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 = (PrintWriter)openJobs.get(logFileName);
+ if( null != writer){
+ JobHistory.log(writer, RecordTypes.Job,
+ new Enum[] {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);
+ }
+ }
+ }
+ }
+ /**
+ * Helper class for logging or reading back events related to Task's start, finish or failure.
+ * All events logged by this class are logged in a separate file per job in
+ * job tracker history. These events map to TIPs in jobtracker.
+ */
+ public static class Task extends KeyValuePair{
+ private Map <String, TaskAttempt> taskAttempts = new TreeMap<String, TaskAttempt>();
+
+ /**
+ * Log start time of task (TIP).
+ * @param jobId job id
+ * @param taskId task id
+ * @param taskType MAP or REDUCE
+ * @param startTime startTime of tip.
+ */
+ public static void logStarted(String jobId, String taskId, String taskType,
+ long startTime){
+ if( ! disableHistory ){
+ PrintWriter writer = (PrintWriter)openJobs.get(JOBTRACKER_START_TIME + "_" + jobId);
+ if( null != writer ){
+ JobHistory.log(writer, RecordTypes.Task, new Enum[]{Keys.TASKID, Keys.TASK_TYPE , Keys.START_TIME},
+ new String[]{taskId, taskType, String.valueOf(startTime)}) ;
+ }
+ }
+ }
+ /**
+ * Log finish time of task.
+ * @param jobId job id
+ * @param taskId task id
+ * @param taskType MAP or REDUCE
+ * @param finishTime finish timeof task in ms
+ */
+ public static void logFinished(String jobId, String taskId, String taskType,
+ long finishTime){
+ if( ! disableHistory ){
+ PrintWriter writer = (PrintWriter)openJobs.get(JOBTRACKER_START_TIME + "_" + jobId);
+ if( null != writer ){
+ JobHistory.log(writer, RecordTypes.Task, new Enum[]{Keys.TASKID, Keys.TASK_TYPE,
+ Keys.TASK_STATUS, Keys.FINISH_TIME},
+ new String[]{ taskId,taskType, Values.SUCCESS.name(), String.valueOf(finishTime)}) ;
+ }
+ }
+ }
+ /**
+ * Log job failed event.
+ * @param jobId jobid
+ * @param taskId task id
+ * @param taskType MAP or REDUCE.
+ * @param time timestamp when job failed detected.
+ * @param error error message for failure.
+ */
+ public static void logFailed(String jobId, String taskId, String taskType, long time, String error){
+ if( ! disableHistory ){
+ PrintWriter writer = (PrintWriter)openJobs.get(JOBTRACKER_START_TIME + "_" + jobId);
+ if( null != writer ){
+ JobHistory.log(writer, RecordTypes.Task, new Enum[]{Keys.TASKID, Keys.TASK_TYPE,
+ Keys.TASK_STATUS, Keys.FINISH_TIME, Keys.ERROR},
+ new String[]{ taskId, taskType, Values.FAILED.name(), String.valueOf(time) , error}) ;
+ }
+ }
+ }
+ /**
+ * Returns all task attempts for this task. <task attempt id - TaskAttempt>
+ * @return
+ */
+ public Map<String, TaskAttempt> getTaskAttempts(){
+ return this.taskAttempts;
+ }
+ }
+ /**
+ * Base class for Map and Reduce TaskAttempts.
+ */
+ public static class TaskAttempt extends Task{}
+
+ /**
+ * Helper class for logging or reading back events related to start, finish or failure of
+ * a Map Attempt on a node.
+ */
+ public static class MapAttempt extends TaskAttempt{
+ /**
+ * Log start time of this map task attempt.
+ * @param jobId job id
+ * @param taskId task id
+ * @param taskAttemptId task attempt id
+ * @param startTime start time of task attempt as reported by task tracker.
+ * @param hostName host name of the task attempt.
+ */
+ public static void logStarted(String jobId, String taskId,String taskAttemptId, long startTime, String hostName){
+ if( ! disableHistory ){
+ PrintWriter writer = (PrintWriter)openJobs.get(JOBTRACKER_START_TIME + "_" + jobId);
+ if( null != writer ){
+ JobHistory.log( writer, RecordTypes.MapAttempt,
+ new Enum[]{ Keys.TASK_TYPE, Keys.TASKID,
+ Keys.TASK_ATTEMPT_ID, Keys.START_TIME, Keys.HOSTNAME},
+ new String[]{Values.MAP.name(), taskId,
+ taskAttemptId, String.valueOf(startTime), hostName} ) ;
+ }
+ }
+ }
+ /**
+ * Log finish time of map task attempt.
+ * @param jobId job id
+ * @param taskId task id
+ * @param taskAttemptId task attempt id
+ * @param finishTime finish time
+ * @param hostName host name
+ */
+ public static void logFinished(String jobId, String taskId, String taskAttemptId, long finishTime, String hostName){
+ if( ! disableHistory ){
+ PrintWriter writer = (PrintWriter)openJobs.get(JOBTRACKER_START_TIME + "_" + jobId);
+ if( null != writer ){
+ JobHistory.log(writer, RecordTypes.MapAttempt,
+ new Enum[]{ Keys.TASK_TYPE, Keys.TASKID, Keys.TASK_ATTEMPT_ID, Keys.TASK_STATUS,
+ Keys.FINISH_TIME, Keys.HOSTNAME},
+ new String[]{Values.MAP.name(), taskId, taskAttemptId, Values.SUCCESS.name(),
+ String.valueOf(finishTime), hostName} ) ;
+ }
+ }
+ }
+ /**
+ * Log task attempt failed event.
+ * @param jobId jobid
+ * @param taskId taskid
+ * @param taskAttemptId task attempt id
+ * @param timestamp timestamp
+ * @param hostName hostname of this task attempt.
+ * @param error error message if any for this task attempt.
+ */
+ public static void logFailed(String jobId, String taskId, String taskAttemptId,
+ long timestamp, String hostName, String error){
+ if( ! disableHistory ){
+ PrintWriter writer = (PrintWriter)openJobs.get(JOBTRACKER_START_TIME + "_" + jobId);
+ if( null != writer ){
+ JobHistory.log( writer, RecordTypes.MapAttempt,
+ new Enum[]{Keys.TASK_TYPE, Keys.TASKID, Keys.TASK_ATTEMPT_ID, Keys.TASK_STATUS,
+ Keys.FINISH_TIME, Keys.HOSTNAME, Keys.ERROR},
+ new String[]{ Values.MAP.name(), taskId, taskAttemptId, Values.FAILED.name(),
+ String.valueOf(timestamp), hostName, error} ) ;
+ }
+ }
+ }
+ }
+ /**
+ * Helper class for logging or reading back events related to start, finish or failure of
+ * a Map Attempt on a node.
+ */
+ public static class ReduceAttempt extends TaskAttempt{
+ /**
+ * Log start time of Reduce task attempt.
+ * @param jobId job id
+ * @param taskId task id (tip)
+ * @param taskAttemptId task attempt id
+ * @param startTime start time
+ * @param hostName host name
+ */
+ public static void logStarted(String jobId, String taskId, String taskAttemptId,
+ long startTime, String hostName){
+ if( ! disableHistory ){
+ PrintWriter writer = (PrintWriter)openJobs.get(JOBTRACKER_START_TIME + "_" + jobId);
+ if( null != writer ){
+ JobHistory.log( writer, RecordTypes.ReduceAttempt,
+ new Enum[]{ Keys.TASK_TYPE, Keys.TASKID,
+ Keys.TASK_ATTEMPT_ID, Keys.START_TIME, Keys.HOSTNAME},
+ new String[]{Values.REDUCE.name(), taskId,
+ taskAttemptId, String.valueOf(startTime), hostName} ) ;
+ }
+ }
+ }
+ /**
+ * Log finished event of this task.
+ * @param jobId job id
+ * @param taskId task id
+ * @param taskAttemptId task attempt id
+ * @param shuffleFinished shuffle finish time
+ * @param sortFinished sort finish time
+ * @param finishTime finish time of task
+ * @param hostName host name where task attempt executed
+ */
+ public static void logFinished(String jobId, String taskId, String taskAttemptId,
+ long shuffleFinished, long sortFinished, long finishTime, String hostName){
+ if( ! disableHistory ){
+ PrintWriter writer = (PrintWriter)openJobs.get(JOBTRACKER_START_TIME + "_" + jobId);
+ if( null != writer ){
+ JobHistory.log( writer, RecordTypes.ReduceAttempt,
+ new Enum[]{ Keys.TASK_TYPE, Keys.TASKID, Keys.TASK_ATTEMPT_ID, Keys.TASK_STATUS,
+ Keys.SHUFFLE_FINISHED, Keys.SORT_FINISHED, Keys.FINISH_TIME, Keys.HOSTNAME},
+ new String[]{Values.REDUCE.name(), taskId, taskAttemptId, Values.SUCCESS.name(),
+ String.valueOf(shuffleFinished), String.valueOf(sortFinished),
+ String.valueOf(finishTime), hostName} ) ;
+ }
+ }
+ }
+ /**
+ * Log failed reduce task attempt.
+ * @param jobId job id
+ * @param taskId task id
+ * @param taskAttemptId task attempt id
+ * @param timestamp time stamp when task failed
+ * @param hostName host name of the task attempt.
+ * @param error error message of the task.
+ */
+ public static void logFailed(String jobId, String taskId,String taskAttemptId, long timestamp,
+ String hostName, String error){
+ if( ! disableHistory ){
+ PrintWriter writer = (PrintWriter)openJobs.get(JOBTRACKER_START_TIME + "_" + jobId);
+ if( null != writer ){
+ JobHistory.log( writer, RecordTypes.ReduceAttempt,
+ new Enum[]{ Keys.TASK_TYPE, Keys.TASKID, Keys.TASK_ATTEMPT_ID,Keys.TASK_STATUS,
+ Keys.FINISH_TIME, Keys.HOSTNAME, Keys.ERROR },
+ new String[]{ Values.REDUCE.name(), taskId, taskAttemptId, Values.FAILED.name(),
+ String.valueOf(timestamp), hostName, error } ) ;
+ }
+ }
+ }
+ }
+ /**
+ * Callback interface for reading back log events from JobHistory. This interface
+ * should be implemented and passed to JobHistory.parseHistory()
+ *
+ */
+ public static interface Listener{
+ /**
+ * Callback method for history parser.
+ * @param recType type of record, which is the first entry in the line.
+ * @param values a map of key-value pairs as thry appear in history.
+ * @throws IOException
+ */
+ public void handle(RecordTypes recType, Map<Keys, String> values) throws IOException;
+ }
+
+ /**
+ * Delete history files older than one month. Update master index and remove all
+ * jobs older than one month. Also if a job tracker has no jobs in last one month
+ * remove reference to the job tracker.
+ * @author sanjaydahiya
+ *
+ */
+ public static class HistoryCleaner implements Runnable{
+ static final long ONE_DAY_IN_MS = 24 * 60 * 60 * 1000L;
+ static final long THIRTY_DAYS_IN_MS = 30 * ONE_DAY_IN_MS;
+ private long now ;
+ private static boolean isRunning = false;
+ private static long lastRan ;
+
+ /**
+ * Cleans up history data.
+ */
+ public void run(){
+ if( isRunning ){
+ return ;
+ }
+ now = System.currentTimeMillis() ;
+ // clean history only once a day at max
+ if( lastRan ==0 || (now - lastRan) < ONE_DAY_IN_MS ){
+ return ;
+ }
+ 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.
+ for (String jobTrackerId : jobTrackersToJobs.keySet()){
+ Map<String, JobHistory.JobInfo> jobs = jobTrackersToJobs.get(jobTrackerId);
+ for(Iterator iter = jobs.keySet().iterator(); iter.hasNext() ; iter.next()){
+ JobHistory.JobInfo job = jobs.get(iter.next());
+ if( now - job.getLong(Keys.SUBMIT_TIME) > THIRTY_DAYS_IN_MS ) {
+ iter.remove();
+ }
+ if( jobs.size() == 0 ){
+ iter.remove();
+ }
+ }
+ }
+ masterIndex.close();
+ masterIndex = new PrintWriter(logFile);
+ // delete old history and write back to a new file
+ for (String jobTrackerId : jobTrackersToJobs.keySet()){
+ Map<String, JobHistory.JobInfo> jobs = jobTrackersToJobs.get(jobTrackerId);
+
+ log(masterIndex, RecordTypes.Jobtracker, Keys.START_TIME, jobTrackerId);
+
+ for(String jobId : jobs.keySet() ){
+ JobHistory.JobInfo job = jobs.get(jobId);
+ 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
+ if( now - file.lastModified() > THIRTY_DAYS_IN_MS ){
+ return true ;
+ }
+ return false;
+ }
+ });
+ for( File f : oldFiles){
+ f.delete();
+ LOG.info("Deleting old history file : " + f.getName());
+ }
+ isRunning = false ;
+ }
+ }
+}
\ No newline at end of file
Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobInProgress.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobInProgress.java?view=diff&rev=453041&r1=453040&r2=453041
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobInProgress.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobInProgress.java Wed Oct 4 14:46:20 2006
@@ -20,7 +20,8 @@
import org.apache.hadoop.fs.*;
import org.apache.hadoop.conf.*;
import org.apache.hadoop.mapred.JobTracker.JobTrackerMetrics;
-
+import org.apache.hadoop.mapred.JobHistory.Keys ;
+import org.apache.hadoop.mapred.JobHistory.Values ;
import java.io.*;
import java.net.*;
import java.util.*;
@@ -47,6 +48,8 @@
int runningReduceTasks = 0;
int finishedMapTasks = 0;
int finishedReduceTasks = 0;
+ int failedMapTasks = 0 ;
+ int failedReduceTasks = 0 ;
JobTracker jobtracker = null;
HashMap hostToMaps = new HashMap();
@@ -93,6 +96,9 @@
this.numMapTasks = conf.getNumMapTasks();
this.numReduceTasks = conf.getNumReduceTasks();
+ JobHistory.JobInfo.logSubmitted(jobid, conf.getJobName(), conf.getUser(),
+ System.currentTimeMillis(), jobFile);
+
}
/**
@@ -183,6 +189,8 @@
this.status = new JobStatus(status.getJobId(), 0.0f, 0.0f, JobStatus.RUNNING);
tasksInited = true;
+
+ JobHistory.JobInfo.logStarted(profile.getJobId(), System.currentTimeMillis(), numMapTasks, numReduceTasks);
}
/////////////////////////////////////////////////////
@@ -331,6 +339,9 @@
Task result = maps[target].getTaskToRun(tts.getTrackerName());
if (!wasRunning) {
runningMapTasks += 1;
+ JobHistory.Task.logStarted(profile.getJobId(),
+ maps[target].getTIPId(), Values.MAP.name(),
+ System.currentTimeMillis());
}
return result;
}
@@ -357,6 +368,9 @@
Task result = reduces[target].getTaskToRun(tts.getTrackerName());
if (!wasRunning) {
runningReduceTasks += 1;
+ JobHistory.Task.logStarted(profile.getJobId(),
+ reduces[target].getTIPId(), Values.REDUCE.name(),
+ System.currentTimeMillis());
}
return result;
}
@@ -457,6 +471,30 @@
} else {
LOG.info("Task '" + taskid + "' has completed " + tip.getTIPId() +
" successfully.");
+
+ String taskTrackerName = status.getTaskTracker();
+ TaskTrackerStatus taskTracker = this.jobtracker.getTaskTracker(taskTrackerName);
+
+ if(status.getIsMap()){
+ JobHistory.MapAttempt.logStarted(profile.getJobId(),
+ tip.getTIPId(), status.getTaskId(), status.getStartTime(),
+ taskTracker.getHost());
+ JobHistory.MapAttempt.logFinished(profile.getJobId(),
+ tip.getTIPId(), status.getTaskId(), status.getFinishTime(),
+ taskTracker.getHost());
+ JobHistory.Task.logFinished(profile.getJobId(), tip.getTIPId(),
+ Values.MAP.name(), status.getFinishTime());
+ }else{
+ JobHistory.ReduceAttempt.logStarted(profile.getJobId(),
+ tip.getTIPId(), status.getTaskId(), status.getStartTime(),
+ taskTracker.getHost());
+ JobHistory.ReduceAttempt.logFinished(profile.getJobId(),
+ tip.getTIPId(), status.getTaskId(), status.getShuffleFinishTime(),
+ status.getSortFinishTime(), status.getFinishTime(),
+ taskTracker.getHost());
+ JobHistory.Task.logFinished(profile.getJobId(), tip.getTIPId(),
+ Values.REDUCE.name(), status.getFinishTime());
+ }
}
tip.completed(taskid);
@@ -503,6 +541,8 @@
garbageCollect();
LOG.info("Job " + this.status.getJobId() +
" has completed successfully.");
+ JobHistory.JobInfo.logFinished(this.status.getJobId(), finishTime,
+ this.finishedMapTasks, this.finishedReduceTasks, failedMapTasks, failedReduceTasks);
metrics.completeJob();
}
}
@@ -525,7 +565,9 @@
for (int i = 0; i < reduces.length; i++) {
reduces[i].kill();
}
-
+ JobHistory.JobInfo.logFinished(this.status.getJobId(), finishTime,
+ this.finishedMapTasks, this.finishedReduceTasks, failedMapTasks,
+ failedReduceTasks);
garbageCollect();
}
}
@@ -565,12 +607,33 @@
}
}
+ // update job history
+ String taskTrackerName = status.getTaskTracker();
+ TaskTrackerStatus taskTracker = this.jobtracker.getTaskTracker(taskTrackerName);
+ if(status.getIsMap()){
+ JobHistory.MapAttempt.logStarted(profile.getJobId(),
+ tip.getTIPId(), status.getTaskId(), status.getStartTime(),
+ taskTracker.getHost());
+ JobHistory.MapAttempt.logFailed(profile.getJobId(),
+ tip.getTIPId(), status.getTaskId(), System.currentTimeMillis(),
+ taskTracker.getHost(), status.getDiagnosticInfo());
+ }else{
+ JobHistory.ReduceAttempt.logStarted(profile.getJobId(),
+ tip.getTIPId(), status.getTaskId(), status.getStartTime(),
+ taskTracker.getHost());
+ JobHistory.ReduceAttempt.logFailed(profile.getJobId(),
+ tip.getTIPId(), status.getTaskId(), System.currentTimeMillis(),
+ taskTracker.getHost(), status.getDiagnosticInfo());
+ }
+
// After this, try to assign tasks with the one after this, so that
// the failed task goes to the end of the list.
if (tip.isMapTask()) {
firstMapToTry = (tip.getIdWithinJob() + 1) % maps.length;
+ failedMapTasks++;
} else {
firstReduceToTry = (tip.getIdWithinJob() + 1) % reduces.length;
+ failedReduceTasks++;
}
//
@@ -578,10 +641,15 @@
//
if (tip.isFailed()) {
LOG.info("Aborting job " + profile.getJobId());
+ JobHistory.JobInfo.logFailed(profile.getJobId(),
+ System.currentTimeMillis(), this.finishedMapTasks, this.finishedReduceTasks);
kill();
}
jobtracker.removeTaskEntry(taskid);
+ JobHistory.Task.logFailed(profile.getJobId(), tip.getTIPId(),
+ tip.isMapTask() ? Values.MAP.name():Values.REDUCE.name(),
+ System.currentTimeMillis(), status.getDiagnosticInfo());
}
/**
@@ -604,6 +672,9 @@
reason,
trackerName, phase);
updateTaskStatus(tip, status, metrics);
+ JobHistory.Task.logFailed(profile.getJobId(), tip.getTIPId(),
+ tip.isMapTask() ? Values.MAP.name() : Values.REDUCE.name(),
+ System.currentTimeMillis(), reason);
}
Added: lucene/hadoop/trunk/src/webapps/job/analysejobhistory.jsp
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/webapps/job/analysejobhistory.jsp?view=auto&rev=453041
==============================================================================
--- lucene/hadoop/trunk/src/webapps/job/analysejobhistory.jsp (added)
+++ lucene/hadoop/trunk/src/webapps/job/analysejobhistory.jsp Wed Oct 4 14:46:20 2006
@@ -0,0 +1,114 @@
+<%@ page
+ contentType="text/html; charset=UTF-8"
+ import="javax.servlet.http.*"
+ import="java.io.*"
+ import="java.util.*"
+ import="org.apache.hadoop.mapred.*"
+ import="org.apache.hadoop.util.*"
+ import="java.text.SimpleDateFormat"
+ import="org.apache.hadoop.mapred.JobHistory.*"
+%>
+<jsp:include page="loadhistory.jsp">
+ <jsp:param name="jobid" value="<%=request.getParameter("jobid") %>"/>
+ <jsp:param name="jobTrackerId" value="<%=request.getParameter("jobTrackerId") %>"/>
+</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 numTasks = request.getParameter("numTasks");
+ int showTasks = 10 ;
+ if( numTasks != null ) {
+ showTasks = Integer.parseInt(numTasks);
+ }
+
+ JobInfo job = (JobInfo)request.getSession().getAttribute("job");
+
+%>
+<h2>Hadoop Job <a href="jobdetailshistory.jsp?jobid=<%=jobid%>&&jobTrackerId=<%=jobTrackerId %>"><%=jobid %> </a></h2>
+
+<b>User : </b> <%=job.get(Keys.USER) %><br/>
+<b>JobName : </b> <%=job.get(Keys.JOBNAME) %><br/>
+<b>JobConf : </b> <%=job.get(Keys.JOBCONF) %><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/>
+<hr/>
+<center>
+<%
+ Map<String, JobHistory.Task> tasks = job.getAllTasks();
+ int finishedMaps = job.getInt(Keys.FINISHED_MAPS) ;
+ int finishedReduces = job.getInt(Keys.FINISHED_REDUCES) ;
+ if( finishedMaps == 0 || finishedReduces == 0 ){
+ out.print("<h3>No Analysis available as job did not finish</h3>");
+ return ;
+ }
+
+ JobHistory.Task [] mapTasks = new JobHistory.Task[finishedMaps];
+ JobHistory.Task [] reduceTasks = new JobHistory.Task[finishedReduces];
+ int mapIndex = 0 , reduceIndex=0;
+
+ for( JobHistory.Task task : tasks.values() ) {
+ if( Values.MAP.name().equals(task.get(Keys.TASK_TYPE)) ){
+ mapTasks[mapIndex++] = task ;
+ }else{
+ reduceTasks[reduceIndex++] = task;
+ }
+ }
+
+ Comparator<JobHistory.Task> c = new Comparator<JobHistory.Task>(){
+ public int compare(JobHistory.Task t1, JobHistory.Task t2){
+ Long l1 = new Long(t1.getLong(Keys.FINISH_TIME) - t1.getLong(Keys.START_TIME));
+ Long l2 = new Long(t2.getLong(Keys.FINISH_TIME) - t2.getLong(Keys.START_TIME)) ;
+ return l2.compareTo(l1);
+ }
+ };
+ Arrays.sort(mapTasks, c);
+ Arrays.sort(reduceTasks, c);
+
+ JobHistory.Task minMap = mapTasks[mapTasks.length-1] ;
+ JobHistory.Task minReduce = reduceTasks[reduceTasks.length-1] ;
+
+%>
+
+<h3>Time taken by best performing Map task
+<a href="taskdetailshistory.jsp?jobid=<%=jobid%>&jobTrackerId=<%=jobTrackerId%>&taskid=<%=minMap.get(Keys.TASKID)%>">
+<%=minMap.get(Keys.TASKID) %></a> : <%=StringUtils.formatTimeDiff(minMap.getLong(Keys.FINISH_TIME), minMap.getLong(Keys.START_TIME) ) %></h3>
+<h3>Worse performing map tasks</h3>
+<table border="2" cellpadding="5" cellspacing="2">
+<tr><td>Task Id</td><td>Time taken</td></tr>
+<%
+ 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)%>">
+ <%=mapTasks[i].get(Keys.TASKID) %></a></td>
+ <td><%=StringUtils.formatTimeDiff(mapTasks[i].getLong(Keys.FINISH_TIME), mapTasks[i].getLong(Keys.START_TIME)) %></td>
+ </tr>
+<%
+ }
+%>
+</table>
+<h3>Time taken by best performing Reduce task :
+<a href="taskdetailshistory.jsp?jobid=<%=jobid%>&jobTrackerId=<%=jobTrackerId%>&taskid=<%=minReduce.get(Keys.TASKID)%>">
+<%=minReduce.get(Keys.TASKID) %></a> : <%=StringUtils.formatTimeDiff(minReduce.getLong(Keys.FINISH_TIME), minReduce.getLong(Keys.START_TIME) ) %></h3>
+
+<h3>Worse performing reduce tasks</h3>
+<table border="2" cellpadding="5" cellspacing="2">
+<tr><td>Task Id</td><td>Time taken</td></tr>
+<%
+ 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)%>">
+ <%=reduceTasks[i].get(Keys.TASKID) %></a></td>
+ <td><%=StringUtils.formatTimeDiff(reduceTasks[i].getLong(Keys.FINISH_TIME), reduceTasks[i].getLong(Keys.START_TIME)) %></td>
+ </tr>
+<%
+ }
+%>
+</table>
+ </center>
+ </body></html>
\ No newline at end of file
Added: lucene/hadoop/trunk/src/webapps/job/jobdetailshistory.jsp
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/webapps/job/jobdetailshistory.jsp?view=auto&rev=453041
==============================================================================
--- lucene/hadoop/trunk/src/webapps/job/jobdetailshistory.jsp (added)
+++ lucene/hadoop/trunk/src/webapps/job/jobdetailshistory.jsp Wed Oct 4 14:46:20 2006
@@ -0,0 +1,106 @@
+<%@ page
+ contentType="text/html; charset=UTF-8"
+ import="javax.servlet.http.*"
+ import="java.io.*"
+ import="java.util.*"
+ import="org.apache.hadoop.mapred.*"
+ import="org.apache.hadoop.util.*"
+ import="java.text.SimpleDateFormat"
+ import="org.apache.hadoop.mapred.JobHistory.*"
+%>
+<jsp:include page="loadhistory.jsp">
+ <jsp:param name="jobid" value="<%=request.getParameter("jobid") %>"/>
+ <jsp:param name="jobTrackerId" value="<%=request.getParameter("jobTrackerId") %>"/>
+</jsp:include>
+<%! static SimpleDateFormat dateFormat = new SimpleDateFormat("d-MMM-yyyy HH:mm:ss") ; %>
+<%
+ String jobid = request.getParameter("jobid");
+ String jobTrackerId = request.getParameter("jobTrackerId");
+
+ JobInfo job = (JobInfo)request.getSession().getAttribute("job");
+%>
+<html><body>
+<h2>Hadoop Job <%=jobid %> </h2>
+
+<b>User : </b> <%=job.get(Keys.USER) %><br/>
+<b>JobName : </b> <%=job.get(Keys.JOBNAME) %><br/>
+<b>JobConf : </b> <%=job.get(Keys.JOBCONF) %><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><a href="analysejobhistory.jsp?jobid=<%=jobid %>&jobTrackerId=<%=jobTrackerId %>">Analyse This Job</a></b>
+<hr/>
+<center>
+<%
+ Map<String, JobHistory.Task> tasks = job.getAllTasks();
+ int totalMaps = 0 ;
+ int totalReduces = 0;
+ int failedMaps = 0;
+ int failedReduces = 0 ;
+
+ long mapStarted = 0 ;
+ long mapFinished = 0 ;
+ long reduceStarted = 0 ;
+ long reduceFinished = 0;
+
+ for( JobHistory.Task task : tasks.values() ) {
+
+ long startTime = task.getLong(Keys.START_TIME) ;
+ long finishTime = task.getLong(Keys.FINISH_TIME) ;
+
+ if( Values.MAP.name().equals(task.get(Keys.TASK_TYPE)) ){
+ totalMaps++;
+ if( mapStarted==0 || mapStarted > startTime ){
+ mapStarted = startTime;
+ }
+ if( mapFinished < finishTime ){
+ mapFinished = finishTime ;
+ }
+ if(Values.FAILED.name().equals(task.get(Keys.TASK_STATUS) )) {
+ failedMaps++;
+ }
+ }else{
+ totalReduces++;
+ if( reduceStarted==0||reduceStarted > startTime ){
+ reduceStarted = startTime ;
+ }
+ if( reduceFinished < finishTime ){
+ reduceFinished = finishTime;
+ }
+ if( Values.FAILED.name().equals(task.get(Keys.TASK_STATUS) )) {
+ failedReduces++;
+ }
+ }
+ }
+%>
+<table border="2" cellpadding="5" cellspacing="2">
+<tr>
+<td>Kind</td><td>Total Tasks</td><td>Finished tasks</td><td>Failed tasks</td><td>Start Time</td><td>Finish Time</td>
+</tr>
+<tr>
+<td>Map</td>
+ <td><a href="jobtaskshistory.jsp?jobid=<%=jobid %>&jobTrackerId=<%=jobTrackerId %>&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 %>">
+ <%=job.getInt(Keys.FINISHED_MAPS) %></a></td>
+ <td><a href="jobtaskshistory.jsp?jobid=<%=jobid %>&jobTrackerId=<%=jobTrackerId %>&taskType=<%=Values.MAP.name() %>&status=<%=Values.FAILED %>">
+ <%=failedMaps %></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">
+ <%=totalReduces%></a></td>
+ <td><a href="jobtaskshistory.jsp?jobid=<%=jobid %>&jobTrackerId=<%=jobTrackerId %>&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 %>">
+ <%=failedReduces%></a></td>
+ <td><%=StringUtils.getFormattedTimeWithDiff(dateFormat, reduceStarted, 0) %></td>
+ <td><%=StringUtils.getFormattedTimeWithDiff(dateFormat, reduceFinished, reduceStarted) %></td>
+</tr>
+ </table>
+ </center>
+
+</body></html>
\ No newline at end of file
Added: lucene/hadoop/trunk/src/webapps/job/jobhistory.jsp
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/webapps/job/jobhistory.jsp?view=auto&rev=453041
==============================================================================
--- lucene/hadoop/trunk/src/webapps/job/jobhistory.jsp (added)
+++ lucene/hadoop/trunk/src/webapps/job/jobhistory.jsp Wed Oct 4 14:46:20 2006
@@ -0,0 +1,95 @@
+<%@ 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.*"
+%>
+<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>
+<%
+ Map<String, Map<String, JobInfo>> jobTrackerToJobs =
+ (Map<String, Map<String, JobInfo>>)request.getSession().getAttribute("jobHistory");
+
+ if( null == jobTrackerToJobs ){
+ out.println("NULL !!!");
+ return ;
+ }
+
+ for(String trackerStartTime : jobTrackerToJobs.keySet() ){
+ Map<String, JobInfo> jobs = (Map<String, JobInfo>)jobTrackerToJobs.get(trackerStartTime) ;
+%>
+<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) == null ) {
+ 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>
\ No newline at end of file
Added: lucene/hadoop/trunk/src/webapps/job/jobtaskshistory.jsp
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/webapps/job/jobtaskshistory.jsp?view=auto&rev=453041
==============================================================================
--- lucene/hadoop/trunk/src/webapps/job/jobtaskshistory.jsp (added)
+++ lucene/hadoop/trunk/src/webapps/job/jobtaskshistory.jsp Wed Oct 4 14:46:20 2006
@@ -0,0 +1,59 @@
+<%@ page
+ contentType="text/html; charset=UTF-8"
+ import="javax.servlet.http.*"
+ import="java.io.*"
+ import="java.util.*"
+ import="org.apache.hadoop.mapred.*"
+ import="org.apache.hadoop.util.*"
+ import="java.text.SimpleDateFormat"
+ import="org.apache.hadoop.mapred.JobHistory.*"
+%>
+<jsp:include page="loadhistory.jsp">
+ <jsp:param name="jobid" value="<%=request.getParameter("jobid") %>"/>
+ <jsp:param name="jobTrackerId" value="<%=request.getParameter("jobTrackerId") %>"/>
+</jsp:include>
+<%! private static SimpleDateFormat dateFormat = new SimpleDateFormat("d/MM HH:mm:ss") ; %>
+
+<%
+ String jobid = request.getParameter("jobid");
+ String jobTrackerId = request.getParameter("jobTrackerId");
+ String taskStatus = request.getParameter("status");
+ String taskType = request.getParameter("taskType");
+
+ JobHistory.JobInfo job = (JobHistory.JobInfo)request.getSession().getAttribute("job");
+
+ Map<String, JobHistory.Task> tasks = job.getAllTasks();
+%>
+<html>
+<body>
+<h2><%=taskStatus%> <%=taskType %> task list for <a href="jobdetailshistory.jsp?jobid=<%=jobid%>&&jobTrackerId=<%=jobTrackerId %>"><%=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>
+<%
+ for( JobHistory.Task task : tasks.values() ) {
+ if( taskType.equals(task.get(Keys.TASK_TYPE) ) ){
+ if( taskStatus.equals(task.get(Keys.TASK_STATUS)) || taskStatus.equals("all")){
+ printTask(jobid, jobTrackerId, task, out);
+ }
+ }
+ }
+%>
+</table>
+<%!
+ 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)+"\">" +
+ task.get(Keys.TASKID) + "</a></td>");
+ out.print("<td>" + StringUtils.getFormattedTimeWithDiff(dateFormat,
+ task.getLong(Keys.START_TIME), 0 ) + "</td>");
+ out.print("<td>" + StringUtils.getFormattedTimeWithDiff(dateFormat,
+ task.getLong(Keys.FINISH_TIME), task.getLong(Keys.START_TIME) ) + "</td>");
+ out.print("<td>" + task.get(Keys.ERROR) + "</td>");
+ out.print("</tr>");
+ }
+%>
+</center>
+</body>
+</html>
Modified: lucene/hadoop/trunk/src/webapps/job/jobtracker.jsp
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/webapps/job/jobtracker.jsp?view=diff&rev=453041&r1=453040&r2=453041
==============================================================================
--- lucene/hadoop/trunk/src/webapps/job/jobtracker.jsp (original)
+++ lucene/hadoop/trunk/src/webapps/job/jobtracker.jsp Wed Oct 4 14:46:20 2006
@@ -114,7 +114,8 @@
<hr>
<h2>Local logs</h2>
-<a href="/logs/">Log</a> directory
+<a href="/logs/">Log</a> directory, <a href="jobhistory.jsp?historyFile=JobHistory.log&reload=true">
+Job Tracker History</a>
<hr>
<a href="http://lucene.apache.org/hadoop">Hadoop</a>, 2006.<br>
Added: lucene/hadoop/trunk/src/webapps/job/loadhistory.jsp
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/webapps/job/loadhistory.jsp?view=auto&rev=453041
==============================================================================
--- lucene/hadoop/trunk/src/webapps/job/loadhistory.jsp (added)
+++ lucene/hadoop/trunk/src/webapps/job/loadhistory.jsp Wed Oct 4 14:46:20 2006
@@ -0,0 +1,49 @@
+<%@ 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);
+ }
+ }
+%>
Added: lucene/hadoop/trunk/src/webapps/job/taskdetailshistory.jsp
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/webapps/job/taskdetailshistory.jsp?view=auto&rev=453041
==============================================================================
--- lucene/hadoop/trunk/src/webapps/job/taskdetailshistory.jsp (added)
+++ lucene/hadoop/trunk/src/webapps/job/taskdetailshistory.jsp Wed Oct 4 14:46:20 2006
@@ -0,0 +1,69 @@
+<%@ page
+ contentType="text/html; charset=UTF-8"
+ import="javax.servlet.http.*"
+ import="java.io.*"
+ import="java.util.*"
+ import="org.apache.hadoop.mapred.*"
+ import="org.apache.hadoop.util.*"
+ import="java.text.SimpleDateFormat"
+ import="org.apache.hadoop.mapred.JobHistory.*"
+%>
+<jsp:include page="loadhistory.jsp">
+ <jsp:param name="jobid" value="<%=request.getParameter("jobid") %>"/>
+ <jsp:param name="jobTrackerId" value="<%=request.getParameter("jobTrackerId") %>"/>
+</jsp:include>
+<%! private static SimpleDateFormat dateFormat = new SimpleDateFormat("d/MM HH:mm:ss") ; %>
+
+<%
+ String jobid = request.getParameter("jobid");
+ String jobTrackerId = request.getParameter("jobTrackerId");
+ String taskid = request.getParameter("taskid");
+
+ JobHistory.JobInfo job = (JobHistory.JobInfo)request.getSession().getAttribute("job");
+ JobHistory.Task task = job.getAllTasks().get(taskid);
+%>
+<html>
+<body>
+<h2><%=taskid %> attempts for <a href="jobdetailshistory.jsp?jobid=<%=jobid%>&&jobTrackerId=<%=jobTrackerId %>"> <%=jobid %> </a></h2>
+<center>
+<table border="2" cellpadding="5" cellspacing="2">
+<tr><td>Task Id</td><td>Start Time</td>
+<%
+ if( Values.REDUCE.name().equals(task.get(Keys.TASK_TYPE) ) ){
+%>
+ <td>Shuffle Finished</td><td>Sort Finished</td>
+<%
+ }
+%>
+<td>Finish Time</td><td>Host</td><td>Error</td></tr>
+<%
+ for( JobHistory.TaskAttempt attempt : task.getTaskAttempts().values() ) {
+ printTaskAttempt(attempt, task.get(Keys.TASK_TYPE), out);
+ }
+%>
+</table>
+<%!
+ private void printTaskAttempt(JobHistory.TaskAttempt taskAttempt, String type, JspWriter out) throws IOException{
+ out.print("<tr>");
+ out.print("<td>" + taskAttempt.get(Keys.TASK_ATTEMPT_ID) + "</td>");
+ out.print("<td>" + StringUtils.getFormattedTimeWithDiff(dateFormat, taskAttempt.getLong(Keys.START_TIME), 0 ) + "</td>") ;
+ if(Values.REDUCE.name().equals(type) ){
+ JobHistory.ReduceAttempt reduceAttempt = (JobHistory.ReduceAttempt)taskAttempt ;
+ out.print("<td>" +
+ StringUtils.getFormattedTimeWithDiff(dateFormat,
+ reduceAttempt.getLong(Keys.SHUFFLE_FINISHED),
+ reduceAttempt.getLong(Keys.START_TIME)) + "</td>") ;
+ out.print("<td>" + StringUtils.getFormattedTimeWithDiff(dateFormat,
+ reduceAttempt.getLong(Keys.SORT_FINISHED),
+ reduceAttempt.getLong(Keys.SHUFFLE_FINISHED)) + "</td>") ;
+ }
+ out.print("<td>"+ StringUtils.getFormattedTimeWithDiff(dateFormat, taskAttempt.getLong(Keys.FINISH_TIME),
+ taskAttempt.getLong(Keys.START_TIME) ) + "</td>") ;
+ out.print("<td>" + taskAttempt.get(Keys.HOSTNAME) + "</td>");
+ out.print("<td>" + taskAttempt.get(Keys.ERROR) + "</td>");
+ out.print("</tr>");
+ }
+%>
+</center>
+</body>
+</html>