|
@@ -39,8 +39,11 @@ import org.apache.commons.logging.Log;
|
|
import org.apache.commons.logging.LogFactory;
|
|
import org.apache.commons.logging.LogFactory;
|
|
import org.apache.hadoop.fs.FSDataInputStream;
|
|
import org.apache.hadoop.fs.FSDataInputStream;
|
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
|
|
|
+import org.apache.hadoop.fs.FileStatus;
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
import org.apache.hadoop.fs.Path;
|
|
import org.apache.hadoop.fs.Path;
|
|
|
|
+import org.apache.hadoop.fs.PathFilter;
|
|
|
|
+import org.apache.hadoop.fs.permission.FsPermission;
|
|
import org.apache.hadoop.util.StringUtils;
|
|
import org.apache.hadoop.util.StringUtils;
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -64,6 +67,7 @@ public class JobHistory {
|
|
|
|
|
|
public static final Log LOG = LogFactory.getLog(JobHistory.class);
|
|
public static final Log LOG = LogFactory.getLog(JobHistory.class);
|
|
private static final String DELIMITER = " ";
|
|
private static final String DELIMITER = " ";
|
|
|
|
+ private static final String LINE_DELIMITER = ".";
|
|
private static final String KEY = "(\\w+)";
|
|
private static final String KEY = "(\\w+)";
|
|
private static final String VALUE = "[[^\"]?]+"; // anything but a " in ""
|
|
private static final String VALUE = "[[^\"]?]+"; // anything but a " in ""
|
|
|
|
|
|
@@ -75,6 +79,10 @@ public class JobHistory {
|
|
private static Map<String, ArrayList<PrintWriter>> openJobs =
|
|
private static Map<String, ArrayList<PrintWriter>> openJobs =
|
|
new HashMap<String, ArrayList<PrintWriter>>();
|
|
new HashMap<String, ArrayList<PrintWriter>>();
|
|
private static boolean disableHistory = false;
|
|
private static boolean disableHistory = false;
|
|
|
|
+ private static final String SECONDARY_FILE_SUFFIX = ".recover";
|
|
|
|
+ private static long jobHistoryBlockSize = 0;
|
|
|
|
+ private static int jobHistoryBufferSize = 0;
|
|
|
|
+ private static String jobtrackerHostname;
|
|
/**
|
|
/**
|
|
* Record types are identifiers for each line of log in history files.
|
|
* 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.
|
|
* A record type appears as the first token in a single line of log.
|
|
@@ -93,7 +101,8 @@ public class JobHistory {
|
|
LAUNCH_TIME, TOTAL_MAPS, TOTAL_REDUCES, FAILED_MAPS, FAILED_REDUCES,
|
|
LAUNCH_TIME, TOTAL_MAPS, TOTAL_REDUCES, FAILED_MAPS, FAILED_REDUCES,
|
|
FINISHED_MAPS, FINISHED_REDUCES, JOB_STATUS, TASKID, HOSTNAME, TASK_TYPE,
|
|
FINISHED_MAPS, FINISHED_REDUCES, JOB_STATUS, TASKID, HOSTNAME, TASK_TYPE,
|
|
ERROR, TASK_ATTEMPT_ID, TASK_STATUS, COPY_PHASE, SORT_PHASE, REDUCE_PHASE,
|
|
ERROR, TASK_ATTEMPT_ID, TASK_STATUS, COPY_PHASE, SORT_PHASE, REDUCE_PHASE,
|
|
- SHUFFLE_FINISHED, SORT_FINISHED, COUNTERS, SPLITS
|
|
|
|
|
|
+ SHUFFLE_FINISHED, SORT_FINISHED, COUNTERS, SPLITS, JOB_PRIORITY, HTTP_PORT,
|
|
|
|
+ TRACKER_NAME, STATE_STRING
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -102,7 +111,7 @@ public class JobHistory {
|
|
* most places in history file.
|
|
* most places in history file.
|
|
*/
|
|
*/
|
|
public static enum Values {
|
|
public static enum Values {
|
|
- SUCCESS, FAILED, KILLED, MAP, REDUCE, CLEANUP
|
|
|
|
|
|
+ SUCCESS, FAILED, KILLED, MAP, REDUCE, CLEANUP, RUNNING
|
|
}
|
|
}
|
|
|
|
|
|
// temp buffer for parsed dataa
|
|
// temp buffer for parsed dataa
|
|
@@ -125,6 +134,7 @@ public class JobHistory {
|
|
+ File.separator + "history");
|
|
+ File.separator + "history");
|
|
JOBTRACKER_UNIQUE_STRING = hostname + "_" +
|
|
JOBTRACKER_UNIQUE_STRING = hostname + "_" +
|
|
String.valueOf(jobTrackerStartTime) + "_";
|
|
String.valueOf(jobTrackerStartTime) + "_";
|
|
|
|
+ jobtrackerHostname = hostname;
|
|
Path logDir = new Path(LOG_DIR);
|
|
Path logDir = new Path(LOG_DIR);
|
|
FileSystem fs = logDir.getFileSystem(conf);
|
|
FileSystem fs = logDir.getFileSystem(conf);
|
|
if (!fs.exists(logDir)){
|
|
if (!fs.exists(logDir)){
|
|
@@ -134,6 +144,14 @@ public class JobHistory {
|
|
}
|
|
}
|
|
conf.set("hadoop.job.history.location", LOG_DIR);
|
|
conf.set("hadoop.job.history.location", LOG_DIR);
|
|
disableHistory = false;
|
|
disableHistory = false;
|
|
|
|
+ // set the job history block size
|
|
|
|
+ jobHistoryBlockSize =
|
|
|
|
+ conf.getLong("mapred.jobtracker.job.history.block.size",
|
|
|
|
+ fs.getDefaultBlockSize());
|
|
|
|
+ // set the job history buffer size
|
|
|
|
+ jobHistoryBufferSize =
|
|
|
|
+ Integer.parseInt(conf.get("mapred.jobtracker.job.history.buffer.size",
|
|
|
|
+ "4096"));
|
|
} catch(IOException e) {
|
|
} catch(IOException e) {
|
|
LOG.error("Failed to initialize JobHistory log file", e);
|
|
LOG.error("Failed to initialize JobHistory log file", e);
|
|
disableHistory = true;
|
|
disableHistory = true;
|
|
@@ -159,7 +177,7 @@ public class JobHistory {
|
|
StringBuffer buf = new StringBuffer();
|
|
StringBuffer buf = new StringBuffer();
|
|
while ((line = reader.readLine())!= null){
|
|
while ((line = reader.readLine())!= null){
|
|
buf.append(line);
|
|
buf.append(line);
|
|
- if (!line.trim().endsWith("\"")){
|
|
|
|
|
|
+ if (!line.trim().endsWith("\"" + DELIMITER + LINE_DELIMITER)){
|
|
continue;
|
|
continue;
|
|
}
|
|
}
|
|
parseLine(buf.toString(), l);
|
|
parseLine(buf.toString(), l);
|
|
@@ -206,7 +224,8 @@ public class JobHistory {
|
|
|
|
|
|
static void log(PrintWriter out, RecordTypes recordType, Keys key,
|
|
static void log(PrintWriter out, RecordTypes recordType, Keys key,
|
|
String value){
|
|
String value){
|
|
- out.println(recordType.name() + DELIMITER + key + "=\"" + value + "\"");
|
|
|
|
|
|
+ out.println(recordType.name() + DELIMITER + key + "=\"" + value + "\""
|
|
|
|
+ + DELIMITER + LINE_DELIMITER);
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -228,6 +247,7 @@ public class JobHistory {
|
|
buf.append("\"");
|
|
buf.append("\"");
|
|
buf.append(DELIMITER);
|
|
buf.append(DELIMITER);
|
|
}
|
|
}
|
|
|
|
+ buf.append(LINE_DELIMITER);
|
|
|
|
|
|
for (PrintWriter out : writers) {
|
|
for (PrintWriter out : writers) {
|
|
out.println(buf.toString());
|
|
out.println(buf.toString());
|
|
@@ -419,6 +439,237 @@ public class JobHistory {
|
|
return decodedFileName;
|
|
return decodedFileName;
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ /**
|
|
|
|
+ * Get the job name from the job conf
|
|
|
|
+ */
|
|
|
|
+ static String getJobName(JobConf jobConf) {
|
|
|
|
+ String jobName = jobConf.getJobName();
|
|
|
|
+ if (jobName == null || jobName.length() == 0) {
|
|
|
|
+ jobName = "NA";
|
|
|
|
+ }
|
|
|
|
+ return jobName;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Get the user name from the job conf
|
|
|
|
+ */
|
|
|
|
+ public static String getUserName(JobConf jobConf) {
|
|
|
|
+ String user = jobConf.getUser();
|
|
|
|
+ if (user == null || user.length() == 0) {
|
|
|
|
+ user = "NA";
|
|
|
|
+ }
|
|
|
|
+ return user;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Get the job history file path given the history filename
|
|
|
|
+ */
|
|
|
|
+ public static Path getJobHistoryLogLocation(String logFileName)
|
|
|
|
+ {
|
|
|
|
+ return LOG_DIR == null ? null : new Path(LOG_DIR, logFileName);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Get the user job history file path
|
|
|
|
+ */
|
|
|
|
+ public static Path getJobHistoryLogLocationForUser(String logFileName,
|
|
|
|
+ JobConf jobConf) {
|
|
|
|
+ // find user log directory
|
|
|
|
+ Path userLogFile = null;
|
|
|
|
+ Path outputPath = FileOutputFormat.getOutputPath(jobConf);
|
|
|
|
+ String userLogDir = jobConf.get("hadoop.job.history.user.location",
|
|
|
|
+ outputPath == null
|
|
|
|
+ ? null
|
|
|
|
+ : outputPath.toString());
|
|
|
|
+ if ("none".equals(userLogDir)) {
|
|
|
|
+ userLogDir = null;
|
|
|
|
+ }
|
|
|
|
+ if (userLogDir != null) {
|
|
|
|
+ userLogDir = userLogDir + Path.SEPARATOR + "_logs" + Path.SEPARATOR
|
|
|
|
+ + "history";
|
|
|
|
+ userLogFile = new Path(userLogDir, logFileName);
|
|
|
|
+ }
|
|
|
|
+ return userLogFile;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Generates the job history filename for a new job
|
|
|
|
+ */
|
|
|
|
+ private static String getNewJobHistoryFileName(JobConf jobConf, JobID id) {
|
|
|
|
+ return JOBTRACKER_UNIQUE_STRING
|
|
|
|
+ + id.toString() + "_" + getUserName(jobConf) + "_"
|
|
|
|
+ + trimJobName(getJobName(jobConf));
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Trims the job-name if required
|
|
|
|
+ */
|
|
|
|
+ private static String trimJobName(String jobName) {
|
|
|
|
+ if (jobName.length() > JOB_NAME_TRIM_LENGTH) {
|
|
|
|
+ jobName = jobName.substring(0, JOB_NAME_TRIM_LENGTH);
|
|
|
|
+ }
|
|
|
|
+ return jobName;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Recover the job history filename from the history folder.
|
|
|
|
+ * Uses the following pattern
|
|
|
|
+ * $jt-hostname_[0-9]*_$job-id_$user-$job-name*
|
|
|
|
+ * @param jobConf the job conf
|
|
|
|
+ * @param id job id
|
|
|
|
+ */
|
|
|
|
+ public static synchronized String getJobHistoryFileName(JobConf jobConf,
|
|
|
|
+ JobID id)
|
|
|
|
+ throws IOException {
|
|
|
|
+ String user = getUserName(jobConf);
|
|
|
|
+ String jobName = trimJobName(getJobName(jobConf));
|
|
|
|
+
|
|
|
|
+ FileSystem fs = new Path(LOG_DIR).getFileSystem(jobConf);
|
|
|
|
+ if (LOG_DIR == null) {
|
|
|
|
+ return null;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ // Make the pattern matching the job's history file
|
|
|
|
+ final Pattern historyFilePattern =
|
|
|
|
+ Pattern.compile(jobtrackerHostname + "_" + "[0-9]+" + "_"
|
|
|
|
+ + id.toString() + "_" + user + "_" + jobName + "+");
|
|
|
|
+ // a path filter that matches 4 parts of the filenames namely
|
|
|
|
+ // - jt-hostname
|
|
|
|
+ // - job-id
|
|
|
|
+ // - username
|
|
|
|
+ // - jobname
|
|
|
|
+ PathFilter filter = new PathFilter() {
|
|
|
|
+ public boolean accept(Path path) {
|
|
|
|
+ String fileName = path.getName();
|
|
|
|
+ try {
|
|
|
|
+ fileName = decodeJobHistoryFileName(fileName);
|
|
|
|
+ } catch (IOException ioe) {
|
|
|
|
+ LOG.info("Error while decoding history file " + fileName + "."
|
|
|
|
+ + " Ignoring file.", ioe);
|
|
|
|
+ return false;
|
|
|
|
+ }
|
|
|
|
+ return historyFilePattern.matcher(fileName).find();
|
|
|
|
+ }
|
|
|
|
+ };
|
|
|
|
+
|
|
|
|
+ FileStatus[] statuses = fs.listStatus(new Path(LOG_DIR), filter);
|
|
|
|
+ String filename;
|
|
|
|
+ if (statuses.length == 0) {
|
|
|
|
+ filename =
|
|
|
|
+ encodeJobHistoryFileName(getNewJobHistoryFileName(jobConf, id));
|
|
|
|
+ } else {
|
|
|
|
+ // return filename considering that fact the name can be a
|
|
|
|
+ // secondary filename like filename.recover
|
|
|
|
+ filename = decodeJobHistoryFileName(statuses[0].getPath().getName());
|
|
|
|
+ // Remove the '.recover' suffix if it exists
|
|
|
|
+ if (filename.endsWith(jobName + SECONDARY_FILE_SUFFIX)) {
|
|
|
|
+ int newLength = filename.length() - SECONDARY_FILE_SUFFIX.length();
|
|
|
|
+ filename = filename.substring(0, newLength);
|
|
|
|
+ }
|
|
|
|
+ filename = encodeJobHistoryFileName(filename);
|
|
|
|
+ }
|
|
|
|
+ return filename;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /** Since there was a restart, there should be a master file and
|
|
|
|
+ * a recovery file. Once the recovery is complete, the master should be
|
|
|
|
+ * deleted as an indication that the recovery file should be treated as the
|
|
|
|
+ * master upon completion or next restart.
|
|
|
|
+ * @param fileName the history filename that needs checkpointing
|
|
|
|
+ * @param conf Job conf
|
|
|
|
+ * @throws IOException
|
|
|
|
+ */
|
|
|
|
+ static synchronized void checkpointRecovery(String fileName, JobConf conf)
|
|
|
|
+ throws IOException {
|
|
|
|
+ Path logPath = JobHistory.JobInfo.getJobHistoryLogLocation(fileName);
|
|
|
|
+ if (logPath != null) {
|
|
|
|
+ FileSystem fs = logPath.getFileSystem(conf);
|
|
|
|
+ fs.delete(logPath, false);
|
|
|
|
+ }
|
|
|
|
+ // do the same for the user file too
|
|
|
|
+ logPath = JobHistory.JobInfo.getJobHistoryLogLocationForUser(fileName,
|
|
|
|
+ conf);
|
|
|
|
+ if (logPath != null) {
|
|
|
|
+ FileSystem fs = logPath.getFileSystem(conf);
|
|
|
|
+ fs.delete(logPath, false);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ static String getSecondaryJobHistoryFile(String filename)
|
|
|
|
+ throws IOException {
|
|
|
|
+ return encodeJobHistoryFileName(
|
|
|
|
+ decodeJobHistoryFileName(filename) + SECONDARY_FILE_SUFFIX);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /** Selects one of the two files generated as a part of recovery.
|
|
|
|
+ * The thumb rule is that always select the oldest file.
|
|
|
|
+ * This call makes sure that only one file is left in the end.
|
|
|
|
+ * @param conf job conf
|
|
|
|
+ * @param logFilePath Path of the log file
|
|
|
|
+ * @throws IOException
|
|
|
|
+ */
|
|
|
|
+ public synchronized static Path recoverJobHistoryFile(JobConf conf,
|
|
|
|
+ Path logFilePath)
|
|
|
|
+ throws IOException {
|
|
|
|
+ FileSystem fs = logFilePath.getFileSystem(conf);
|
|
|
|
+ String tmpFilename = getSecondaryJobHistoryFile(logFilePath.getName());
|
|
|
|
+ Path logDir = logFilePath.getParent();
|
|
|
|
+ Path tmpFilePath = new Path(logDir, tmpFilename);
|
|
|
|
+ if (fs.exists(logFilePath)) {
|
|
|
|
+ if (fs.exists(tmpFilePath)) {
|
|
|
|
+ fs.delete(tmpFilePath, false);
|
|
|
|
+ }
|
|
|
|
+ return tmpFilePath;
|
|
|
|
+ } else {
|
|
|
|
+ if (fs.exists(tmpFilePath)) {
|
|
|
|
+ fs.rename(tmpFilePath, logFilePath);
|
|
|
|
+ return tmpFilePath;
|
|
|
|
+ } else {
|
|
|
|
+ return logFilePath;
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /** Finalize the recovery and make one file in the end.
|
|
|
|
+ * This invloves renaming the recover file to the master file.
|
|
|
|
+ * @param id Job id
|
|
|
|
+ * @param conf the job conf
|
|
|
|
+ * @throws IOException
|
|
|
|
+ */
|
|
|
|
+ static synchronized void finalizeRecovery(JobID id, JobConf conf)
|
|
|
|
+ throws IOException {
|
|
|
|
+ String masterLogFileName =
|
|
|
|
+ JobHistory.JobInfo.getJobHistoryFileName(conf, id);
|
|
|
|
+ Path masterLogPath =
|
|
|
|
+ JobHistory.JobInfo.getJobHistoryLogLocation(masterLogFileName);
|
|
|
|
+ String tmpLogFileName = getSecondaryJobHistoryFile(masterLogFileName);
|
|
|
|
+ Path tmpLogPath =
|
|
|
|
+ JobHistory.JobInfo.getJobHistoryLogLocation(tmpLogFileName);
|
|
|
|
+ if (masterLogPath != null) {
|
|
|
|
+ FileSystem fs = masterLogPath.getFileSystem(conf);
|
|
|
|
+
|
|
|
|
+ // rename the tmp file to the master file. Note that this should be
|
|
|
|
+ // done only when the file is closed and handles are released.
|
|
|
|
+ if(fs.exists(tmpLogPath)) {
|
|
|
|
+ fs.rename(tmpLogPath, masterLogPath);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ // do the same for the user file too
|
|
|
|
+ masterLogPath =
|
|
|
|
+ JobHistory.JobInfo.getJobHistoryLogLocationForUser(masterLogFileName,
|
|
|
|
+ conf);
|
|
|
|
+ tmpLogPath =
|
|
|
|
+ JobHistory.JobInfo.getJobHistoryLogLocationForUser(tmpLogFileName,
|
|
|
|
+ conf);
|
|
|
|
+ if (masterLogPath != null) {
|
|
|
|
+ FileSystem fs = masterLogPath.getFileSystem(conf);
|
|
|
|
+ if (fs.exists(tmpLogPath)) {
|
|
|
|
+ fs.rename(tmpLogPath, masterLogPath);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* Log job submitted event to history. Creates a new file in history
|
|
* Log job submitted event to history. Creates a new file in history
|
|
* for the job. if history file creation fails, it disables history
|
|
* for the job. if history file creation fails, it disables history
|
|
@@ -438,46 +689,21 @@ public class JobHistory {
|
|
|
|
|
|
if (!disableHistory){
|
|
if (!disableHistory){
|
|
// Get the username and job name to be used in the actual log filename;
|
|
// Get the username and job name to be used in the actual log filename;
|
|
- // sanity check them too
|
|
|
|
- String jobName = jobConf.getJobName();
|
|
|
|
- if (jobName == null || jobName.length() == 0) {
|
|
|
|
- jobName = "NA";
|
|
|
|
- }
|
|
|
|
|
|
+ // sanity check them too
|
|
|
|
+ String jobName = getJobName(jobConf);
|
|
|
|
|
|
- String user = jobConf.getUser();
|
|
|
|
- if (user == null || user.length() == 0) {
|
|
|
|
- user = "NA";
|
|
|
|
- }
|
|
|
|
|
|
+ String user = getUserName(jobConf);
|
|
|
|
|
|
- // setup the history log file for this job
|
|
|
|
- String trimmedJobName = jobName;
|
|
|
|
- if (jobName.length() > JOB_NAME_TRIM_LENGTH) {
|
|
|
|
- trimmedJobName = jobName.substring(0, JOB_NAME_TRIM_LENGTH);
|
|
|
|
- }
|
|
|
|
|
|
+ // get the history filename
|
|
String logFileName =
|
|
String logFileName =
|
|
- encodeJobHistoryFileName(jobUniqueString + "_" + user + "_" +
|
|
|
|
- trimmedJobName);
|
|
|
|
-
|
|
|
|
- // find user log directory
|
|
|
|
- Path outputPath = FileOutputFormat.getOutputPath(jobConf);
|
|
|
|
- userLogDir = jobConf.get("hadoop.job.history.user.location",
|
|
|
|
- outputPath == null ? null : outputPath.toString());
|
|
|
|
- if ("none".equals(userLogDir)) {
|
|
|
|
- userLogDir = null;
|
|
|
|
- }
|
|
|
|
- if (userLogDir != null) {
|
|
|
|
- userLogDir = userLogDir + Path.SEPARATOR + "_logs" +
|
|
|
|
- Path.SEPARATOR + "history";
|
|
|
|
- }
|
|
|
|
|
|
+ getJobHistoryFileName(jobConf, jobId);
|
|
|
|
|
|
- Path logFile = null;
|
|
|
|
- Path userLogFile = null;
|
|
|
|
- if (LOG_DIR != null ) {
|
|
|
|
- logFile = new Path(LOG_DIR, logFileName);
|
|
|
|
- }
|
|
|
|
- if (userLogDir != null ) {
|
|
|
|
- userLogFile = new Path(userLogDir, logFileName);
|
|
|
|
- }
|
|
|
|
|
|
+ // setup the history log file for this job
|
|
|
|
+ Path logFile = getJobHistoryLogLocation(logFileName);
|
|
|
|
+
|
|
|
|
+ // find user log directory
|
|
|
|
+ Path userLogFile =
|
|
|
|
+ getJobHistoryLogLocationForUser(logFileName, jobConf);
|
|
|
|
|
|
try{
|
|
try{
|
|
ArrayList<PrintWriter> writers = new ArrayList<PrintWriter>();
|
|
ArrayList<PrintWriter> writers = new ArrayList<PrintWriter>();
|
|
@@ -487,14 +713,24 @@ public class JobHistory {
|
|
if (LOG_DIR != null) {
|
|
if (LOG_DIR != null) {
|
|
// create output stream for logging in hadoop.job.history.location
|
|
// create output stream for logging in hadoop.job.history.location
|
|
fs = new Path(LOG_DIR).getFileSystem(jobConf);
|
|
fs = new Path(LOG_DIR).getFileSystem(jobConf);
|
|
- out = fs.create(logFile, true, 4096);
|
|
|
|
|
|
+
|
|
|
|
+ logFile = recoverJobHistoryFile(jobConf, logFile);
|
|
|
|
+
|
|
|
|
+ out = fs.create(logFile, FsPermission.getDefault(), true,
|
|
|
|
+ jobHistoryBufferSize,
|
|
|
|
+ fs.getDefaultReplication(),
|
|
|
|
+ jobHistoryBlockSize, null);
|
|
writer = new PrintWriter(out);
|
|
writer = new PrintWriter(out);
|
|
writers.add(writer);
|
|
writers.add(writer);
|
|
}
|
|
}
|
|
- if (userLogDir != null) {
|
|
|
|
|
|
+ if (userLogFile != null) {
|
|
|
|
+ userLogDir = userLogFile.getParent().toString();
|
|
// create output stream for logging
|
|
// create output stream for logging
|
|
// in hadoop.job.history.user.location
|
|
// in hadoop.job.history.user.location
|
|
- fs = new Path(userLogDir).getFileSystem(jobConf);
|
|
|
|
|
|
+ fs = userLogFile.getFileSystem(jobConf);
|
|
|
|
+
|
|
|
|
+ userLogFile = recoverJobHistoryFile(jobConf, userLogFile);
|
|
|
|
+
|
|
out = fs.create(userLogFile, true, 4096);
|
|
out = fs.create(userLogFile, true, 4096);
|
|
writer = new PrintWriter(out);
|
|
writer = new PrintWriter(out);
|
|
writers.add(writer);
|
|
writers.add(writer);
|
|
@@ -595,8 +831,12 @@ public class JobHistory {
|
|
|
|
|
|
if (null != writer){
|
|
if (null != writer){
|
|
JobHistory.log(writer, RecordTypes.Job,
|
|
JobHistory.log(writer, RecordTypes.Job,
|
|
- new Keys[] {Keys.JOBID, Keys.LAUNCH_TIME, Keys.TOTAL_MAPS, Keys.TOTAL_REDUCES },
|
|
|
|
- new String[] {jobId.toString(), String.valueOf(startTime), String.valueOf(totalMaps), String.valueOf(totalReduces)});
|
|
|
|
|
|
+ new Keys[] {Keys.JOBID, Keys.LAUNCH_TIME, Keys.TOTAL_MAPS,
|
|
|
|
+ Keys.TOTAL_REDUCES, Keys.JOB_STATUS},
|
|
|
|
+ new String[] {jobId.toString(), String.valueOf(startTime),
|
|
|
|
+ String.valueOf(totalMaps),
|
|
|
|
+ String.valueOf(totalReduces),
|
|
|
|
+ Values.RUNNING.name()});
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
@@ -632,7 +872,7 @@ public class JobHistory {
|
|
String.valueOf(finishedReduces),
|
|
String.valueOf(finishedReduces),
|
|
String.valueOf(failedMaps),
|
|
String.valueOf(failedMaps),
|
|
String.valueOf(failedReduces),
|
|
String.valueOf(failedReduces),
|
|
- counters.makeCompactString()});
|
|
|
|
|
|
+ counters.makeEscapedCompactString()});
|
|
for (PrintWriter out : writer) {
|
|
for (PrintWriter out : writer) {
|
|
out.close();
|
|
out.close();
|
|
}
|
|
}
|
|
@@ -666,6 +906,46 @@ public class JobHistory {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
+ /**
|
|
|
|
+ * Log job's priority.
|
|
|
|
+ * @param jobid job id
|
|
|
|
+ * @param priority Jobs priority
|
|
|
|
+ */
|
|
|
|
+ public static void logJobPriority(JobID jobid, JobPriority priority){
|
|
|
|
+ if (!disableHistory){
|
|
|
|
+ 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.JOB_PRIORITY},
|
|
|
|
+ new String[] {jobid.toString(), priority.toString()});
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Log job's submit-time/launch-time
|
|
|
|
+ * @param jobid job id
|
|
|
|
+ * @param submitTime job's submit time
|
|
|
|
+ * @param launchTime job's launch time
|
|
|
|
+ */
|
|
|
|
+ public static void logJobSubmitTime(JobID jobid, long submitTime,
|
|
|
|
+ long launchTime){
|
|
|
|
+ if (!disableHistory){
|
|
|
|
+ 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.SUBMIT_TIME,
|
|
|
|
+ Keys.LAUNCH_TIME},
|
|
|
|
+ new String[] {jobid.toString(),
|
|
|
|
+ String.valueOf(submitTime),
|
|
|
|
+ String.valueOf(launchTime)});
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ }
|
|
}
|
|
}
|
|
/**
|
|
/**
|
|
* Helper class for logging or reading back events related to Task's start, finish or failure.
|
|
* Helper class for logging or reading back events related to Task's start, finish or failure.
|
|
@@ -716,7 +996,7 @@ public class JobHistory {
|
|
Keys.COUNTERS},
|
|
Keys.COUNTERS},
|
|
new String[]{ taskId.toString(), taskType, Values.SUCCESS.name(),
|
|
new String[]{ taskId.toString(), taskType, Values.SUCCESS.name(),
|
|
String.valueOf(finishTime),
|
|
String.valueOf(finishTime),
|
|
- counters.makeCompactString()});
|
|
|
|
|
|
+ counters.makeEscapedCompactString()});
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
@@ -728,15 +1008,31 @@ public class JobHistory {
|
|
* @param error error message for failure.
|
|
* @param error error message for failure.
|
|
*/
|
|
*/
|
|
public static void logFailed(TaskID taskId, String taskType, long time, String error){
|
|
public static void logFailed(TaskID taskId, String taskType, long time, String error){
|
|
|
|
+ logFailed(taskId, taskType, time, error, null);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * @param failedDueToAttempt The attempt that caused the failure, if any
|
|
|
|
+ */
|
|
|
|
+ public static void logFailed(TaskID taskId, String taskType, long time,
|
|
|
|
+ String error,
|
|
|
|
+ TaskAttemptID failedDueToAttempt){
|
|
if (!disableHistory){
|
|
if (!disableHistory){
|
|
ArrayList<PrintWriter> writer = openJobs.get(JOBTRACKER_UNIQUE_STRING
|
|
ArrayList<PrintWriter> writer = openJobs.get(JOBTRACKER_UNIQUE_STRING
|
|
+ taskId.getJobID());
|
|
+ taskId.getJobID());
|
|
|
|
|
|
if (null != writer){
|
|
if (null != writer){
|
|
|
|
+ String failedAttempt = failedDueToAttempt == null
|
|
|
|
+ ? ""
|
|
|
|
+ : failedDueToAttempt.toString();
|
|
JobHistory.log(writer, RecordTypes.Task,
|
|
JobHistory.log(writer, RecordTypes.Task,
|
|
new Keys[]{Keys.TASKID, Keys.TASK_TYPE,
|
|
new Keys[]{Keys.TASKID, Keys.TASK_TYPE,
|
|
- Keys.TASK_STATUS, Keys.FINISH_TIME, Keys.ERROR},
|
|
|
|
- new String[]{ taskId.toString(), taskType, Values.FAILED.name(), String.valueOf(time) , error});
|
|
|
|
|
|
+ Keys.TASK_STATUS, Keys.FINISH_TIME,
|
|
|
|
+ Keys.ERROR, Keys.TASK_ATTEMPT_ID},
|
|
|
|
+ new String[]{ taskId.toString(), taskType,
|
|
|
|
+ Values.FAILED.name(),
|
|
|
|
+ String.valueOf(time) , error,
|
|
|
|
+ failedAttempt});
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
@@ -764,11 +1060,12 @@ public class JobHistory {
|
|
* @param startTime start time of task attempt as reported by task tracker.
|
|
* @param startTime start time of task attempt as reported by task tracker.
|
|
* @param hostName host name of the task attempt.
|
|
* @param hostName host name of the task attempt.
|
|
* @deprecated Use
|
|
* @deprecated Use
|
|
- * {@link #logStarted(TaskAttemptID, long, String, boolean)}
|
|
|
|
|
|
+ * {@link #logStarted(TaskAttemptID, long, String, int,
|
|
|
|
+ * boolean)}
|
|
*/
|
|
*/
|
|
@Deprecated
|
|
@Deprecated
|
|
public static void logStarted(TaskAttemptID taskAttemptId, long startTime, String hostName){
|
|
public static void logStarted(TaskAttemptID taskAttemptId, long startTime, String hostName){
|
|
- logStarted(taskAttemptId, startTime, hostName, false);
|
|
|
|
|
|
+ logStarted(taskAttemptId, startTime, hostName, -1, false);
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -776,11 +1073,13 @@ public class JobHistory {
|
|
*
|
|
*
|
|
* @param taskAttemptId task attempt id
|
|
* @param taskAttemptId task attempt id
|
|
* @param startTime start time of task attempt as reported by task tracker.
|
|
* @param startTime start time of task attempt as reported by task tracker.
|
|
- * @param hostName host name of the task attempt.
|
|
|
|
|
|
+ * @param trackerName name of the tracker executing the task attempt.
|
|
|
|
+ * @param httpPort http port of the task tracker executing the task attempt
|
|
* @param isCleanup Whether the attempt is cleanup or not
|
|
* @param isCleanup Whether the attempt is cleanup or not
|
|
*/
|
|
*/
|
|
public static void logStarted(TaskAttemptID taskAttemptId, long startTime,
|
|
public static void logStarted(TaskAttemptID taskAttemptId, long startTime,
|
|
- String hostName, boolean isCleanup){
|
|
|
|
|
|
+ String trackerName, int httpPort,
|
|
|
|
+ boolean isCleanup){
|
|
if (!disableHistory){
|
|
if (!disableHistory){
|
|
ArrayList<PrintWriter> writer = openJobs.get(JOBTRACKER_UNIQUE_STRING
|
|
ArrayList<PrintWriter> writer = openJobs.get(JOBTRACKER_UNIQUE_STRING
|
|
+ taskAttemptId.getJobID());
|
|
+ taskAttemptId.getJobID());
|
|
@@ -789,12 +1088,13 @@ public class JobHistory {
|
|
JobHistory.log(writer, RecordTypes.MapAttempt,
|
|
JobHistory.log(writer, RecordTypes.MapAttempt,
|
|
new Keys[]{ Keys.TASK_TYPE, Keys.TASKID,
|
|
new Keys[]{ Keys.TASK_TYPE, Keys.TASKID,
|
|
Keys.TASK_ATTEMPT_ID, Keys.START_TIME,
|
|
Keys.TASK_ATTEMPT_ID, Keys.START_TIME,
|
|
- Keys.HOSTNAME},
|
|
|
|
|
|
+ Keys.TRACKER_NAME, Keys.HTTP_PORT},
|
|
new String[]{isCleanup ? Values.CLEANUP.name() :
|
|
new String[]{isCleanup ? Values.CLEANUP.name() :
|
|
Values.MAP.name(),
|
|
Values.MAP.name(),
|
|
taskAttemptId.getTaskID().toString(),
|
|
taskAttemptId.getTaskID().toString(),
|
|
taskAttemptId.toString(),
|
|
taskAttemptId.toString(),
|
|
- String.valueOf(startTime), hostName});
|
|
|
|
|
|
+ String.valueOf(startTime), trackerName,
|
|
|
|
+ String.valueOf(httpPort)});
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
@@ -805,12 +1105,14 @@ public class JobHistory {
|
|
* @param finishTime finish time
|
|
* @param finishTime finish time
|
|
* @param hostName host name
|
|
* @param hostName host name
|
|
* @deprecated Use
|
|
* @deprecated Use
|
|
- * {@link #logFinished(TaskAttemptID, long, String, boolean)}
|
|
|
|
|
|
+ * {@link #logFinished(TaskAttemptID, long, String, boolean, String,
|
|
|
|
+ * Counters)}
|
|
*/
|
|
*/
|
|
@Deprecated
|
|
@Deprecated
|
|
public static void logFinished(TaskAttemptID taskAttemptId, long finishTime,
|
|
public static void logFinished(TaskAttemptID taskAttemptId, long finishTime,
|
|
String hostName){
|
|
String hostName){
|
|
- logFinished(taskAttemptId, finishTime, hostName, false);
|
|
|
|
|
|
+ logFinished(taskAttemptId, finishTime, hostName, false, "",
|
|
|
|
+ new Counters());
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -820,11 +1122,14 @@ public class JobHistory {
|
|
* @param finishTime finish time
|
|
* @param finishTime finish time
|
|
* @param hostName host name
|
|
* @param hostName host name
|
|
* @param isCleanup Whether the attempt is cleanup or not
|
|
* @param isCleanup Whether the attempt is cleanup or not
|
|
|
|
+ * @param stateString state string of the task attempt
|
|
|
|
+ * @param counter counters of the task attempt
|
|
*/
|
|
*/
|
|
public static void logFinished(TaskAttemptID taskAttemptId,
|
|
public static void logFinished(TaskAttemptID taskAttemptId,
|
|
long finishTime,
|
|
long finishTime,
|
|
String hostName,
|
|
String hostName,
|
|
- boolean isCleanup) {
|
|
|
|
|
|
+ boolean isCleanup, String stateString,
|
|
|
|
+ Counters counter) {
|
|
if (!disableHistory){
|
|
if (!disableHistory){
|
|
ArrayList<PrintWriter> writer = openJobs.get(JOBTRACKER_UNIQUE_STRING
|
|
ArrayList<PrintWriter> writer = openJobs.get(JOBTRACKER_UNIQUE_STRING
|
|
+ taskAttemptId.getJobID());
|
|
+ taskAttemptId.getJobID());
|
|
@@ -833,13 +1138,16 @@ public class JobHistory {
|
|
JobHistory.log(writer, RecordTypes.MapAttempt,
|
|
JobHistory.log(writer, RecordTypes.MapAttempt,
|
|
new Keys[]{ Keys.TASK_TYPE, Keys.TASKID,
|
|
new Keys[]{ Keys.TASK_TYPE, Keys.TASKID,
|
|
Keys.TASK_ATTEMPT_ID, Keys.TASK_STATUS,
|
|
Keys.TASK_ATTEMPT_ID, Keys.TASK_STATUS,
|
|
- Keys.FINISH_TIME, Keys.HOSTNAME},
|
|
|
|
|
|
+ Keys.FINISH_TIME, Keys.HOSTNAME,
|
|
|
|
+ Keys.STATE_STRING, Keys.COUNTERS},
|
|
new String[]{isCleanup ? Values.CLEANUP.name() :
|
|
new String[]{isCleanup ? Values.CLEANUP.name() :
|
|
Values.MAP.name(),
|
|
Values.MAP.name(),
|
|
taskAttemptId.getTaskID().toString(),
|
|
taskAttemptId.getTaskID().toString(),
|
|
taskAttemptId.toString(),
|
|
taskAttemptId.toString(),
|
|
Values.SUCCESS.name(),
|
|
Values.SUCCESS.name(),
|
|
- String.valueOf(finishTime), hostName});
|
|
|
|
|
|
+ String.valueOf(finishTime), hostName,
|
|
|
|
+ stateString,
|
|
|
|
+ counter.makeEscapedCompactString()});
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
@@ -951,12 +1259,12 @@ public class JobHistory {
|
|
* @param startTime start time
|
|
* @param startTime start time
|
|
* @param hostName host name
|
|
* @param hostName host name
|
|
* @deprecated Use
|
|
* @deprecated Use
|
|
- * {@link #logStarted(TaskAttemptID, long, String, boolean)}
|
|
|
|
|
|
+ * {@link #logStarted(TaskAttemptID, long, String, int, boolean)}
|
|
*/
|
|
*/
|
|
@Deprecated
|
|
@Deprecated
|
|
public static void logStarted(TaskAttemptID taskAttemptId,
|
|
public static void logStarted(TaskAttemptID taskAttemptId,
|
|
long startTime, String hostName){
|
|
long startTime, String hostName){
|
|
- logStarted(taskAttemptId, startTime, hostName, false);
|
|
|
|
|
|
+ logStarted(taskAttemptId, startTime, hostName, -1, false);
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -964,12 +1272,13 @@ public class JobHistory {
|
|
*
|
|
*
|
|
* @param taskAttemptId task attempt id
|
|
* @param taskAttemptId task attempt id
|
|
* @param startTime start time
|
|
* @param startTime start time
|
|
- * @param hostName host name
|
|
|
|
|
|
+ * @param trackerName tracker name
|
|
|
|
+ * @param httpPort the http port of the tracker executing the task attempt
|
|
* @param isCleanup Whether the attempt is cleanup or not
|
|
* @param isCleanup Whether the attempt is cleanup or not
|
|
*/
|
|
*/
|
|
public static void logStarted(TaskAttemptID taskAttemptId,
|
|
public static void logStarted(TaskAttemptID taskAttemptId,
|
|
- long startTime, String hostName,
|
|
|
|
- boolean isCleanup) {
|
|
|
|
|
|
+ long startTime, String trackerName,
|
|
|
|
+ int httpPort, boolean isCleanup) {
|
|
if (!disableHistory){
|
|
if (!disableHistory){
|
|
ArrayList<PrintWriter> writer = openJobs.get(JOBTRACKER_UNIQUE_STRING
|
|
ArrayList<PrintWriter> writer = openJobs.get(JOBTRACKER_UNIQUE_STRING
|
|
+ taskAttemptId.getJobID());
|
|
+ taskAttemptId.getJobID());
|
|
@@ -978,12 +1287,13 @@ public class JobHistory {
|
|
JobHistory.log(writer, RecordTypes.ReduceAttempt,
|
|
JobHistory.log(writer, RecordTypes.ReduceAttempt,
|
|
new Keys[]{ Keys.TASK_TYPE, Keys.TASKID,
|
|
new Keys[]{ Keys.TASK_TYPE, Keys.TASKID,
|
|
Keys.TASK_ATTEMPT_ID, Keys.START_TIME,
|
|
Keys.TASK_ATTEMPT_ID, Keys.START_TIME,
|
|
- Keys.HOSTNAME},
|
|
|
|
|
|
+ Keys.TRACKER_NAME, Keys.HTTP_PORT},
|
|
new String[]{isCleanup ? Values.CLEANUP.name() :
|
|
new String[]{isCleanup ? Values.CLEANUP.name() :
|
|
Values.REDUCE.name(),
|
|
Values.REDUCE.name(),
|
|
taskAttemptId.getTaskID().toString(),
|
|
taskAttemptId.getTaskID().toString(),
|
|
taskAttemptId.toString(),
|
|
taskAttemptId.toString(),
|
|
- String.valueOf(startTime), hostName});
|
|
|
|
|
|
+ String.valueOf(startTime), trackerName,
|
|
|
|
+ String.valueOf(httpPort)});
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
@@ -996,14 +1306,15 @@ public class JobHistory {
|
|
* @param finishTime finish time of task
|
|
* @param finishTime finish time of task
|
|
* @param hostName host name where task attempt executed
|
|
* @param hostName host name where task attempt executed
|
|
* @deprecated Use
|
|
* @deprecated Use
|
|
- * {@link #logFinished(TaskAttemptID, long, long, long, String, boolean)}
|
|
|
|
|
|
+ * {@link #logFinished(TaskAttemptID, long, long, long, String, boolean,
|
|
|
|
+ * String, Counters)}
|
|
*/
|
|
*/
|
|
@Deprecated
|
|
@Deprecated
|
|
public static void logFinished(TaskAttemptID taskAttemptId, long shuffleFinished,
|
|
public static void logFinished(TaskAttemptID taskAttemptId, long shuffleFinished,
|
|
long sortFinished, long finishTime,
|
|
long sortFinished, long finishTime,
|
|
String hostName){
|
|
String hostName){
|
|
logFinished(taskAttemptId, shuffleFinished, sortFinished,
|
|
logFinished(taskAttemptId, shuffleFinished, sortFinished,
|
|
- finishTime, hostName, false);
|
|
|
|
|
|
+ finishTime, hostName, false, "", new Counters());
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -1015,11 +1326,14 @@ public class JobHistory {
|
|
* @param finishTime finish time of task
|
|
* @param finishTime finish time of task
|
|
* @param hostName host name where task attempt executed
|
|
* @param hostName host name where task attempt executed
|
|
* @param isCleanup Whether the attempt is cleanup or not
|
|
* @param isCleanup Whether the attempt is cleanup or not
|
|
|
|
+ * @param stateString the state string of the attempt
|
|
|
|
+ * @param counter counters of the attempt
|
|
*/
|
|
*/
|
|
public static void logFinished(TaskAttemptID taskAttemptId,
|
|
public static void logFinished(TaskAttemptID taskAttemptId,
|
|
long shuffleFinished,
|
|
long shuffleFinished,
|
|
long sortFinished, long finishTime,
|
|
long sortFinished, long finishTime,
|
|
- String hostName, boolean isCleanup) {
|
|
|
|
|
|
+ String hostName, boolean isCleanup,
|
|
|
|
+ String stateString, Counters counter) {
|
|
if (!disableHistory){
|
|
if (!disableHistory){
|
|
ArrayList<PrintWriter> writer = openJobs.get(JOBTRACKER_UNIQUE_STRING
|
|
ArrayList<PrintWriter> writer = openJobs.get(JOBTRACKER_UNIQUE_STRING
|
|
+ taskAttemptId.getJobID());
|
|
+ taskAttemptId.getJobID());
|
|
@@ -1029,7 +1343,8 @@ public class JobHistory {
|
|
new Keys[]{ Keys.TASK_TYPE, Keys.TASKID,
|
|
new Keys[]{ Keys.TASK_TYPE, Keys.TASKID,
|
|
Keys.TASK_ATTEMPT_ID, Keys.TASK_STATUS,
|
|
Keys.TASK_ATTEMPT_ID, Keys.TASK_STATUS,
|
|
Keys.SHUFFLE_FINISHED, Keys.SORT_FINISHED,
|
|
Keys.SHUFFLE_FINISHED, Keys.SORT_FINISHED,
|
|
- Keys.FINISH_TIME, Keys.HOSTNAME},
|
|
|
|
|
|
+ Keys.FINISH_TIME, Keys.HOSTNAME,
|
|
|
|
+ Keys.STATE_STRING, Keys.COUNTERS},
|
|
new String[]{isCleanup ? Values.CLEANUP.name() :
|
|
new String[]{isCleanup ? Values.CLEANUP.name() :
|
|
Values.REDUCE.name(),
|
|
Values.REDUCE.name(),
|
|
taskAttemptId.getTaskID().toString(),
|
|
taskAttemptId.getTaskID().toString(),
|
|
@@ -1037,7 +1352,9 @@ public class JobHistory {
|
|
Values.SUCCESS.name(),
|
|
Values.SUCCESS.name(),
|
|
String.valueOf(shuffleFinished),
|
|
String.valueOf(shuffleFinished),
|
|
String.valueOf(sortFinished),
|
|
String.valueOf(sortFinished),
|
|
- String.valueOf(finishTime), hostName});
|
|
|
|
|
|
+ String.valueOf(finishTime), hostName,
|
|
|
|
+ stateString,
|
|
|
|
+ counter.makeEscapedCompactString()});
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|