diff --git a/hraven-core/src/main/java/com/twitter/hraven/Constants.java b/hraven-core/src/main/java/com/twitter/hraven/Constants.java index b8f1cdd..0e9ae0a 100644 --- a/hraven-core/src/main/java/com/twitter/hraven/Constants.java +++ b/hraven-core/src/main/java/com/twitter/hraven/Constants.java @@ -263,8 +263,10 @@ public class Constants { * job_201306192120_0003_1371677828795_hadoop_word+count * and conf file name is * job_201306192120_0003_1371677828795_hadoop_conf.xml + * in hadoop 2.0, job history file names are named as + * job_1374258111572_0003-1374260622449-userName1-TeraGen-1374260635219-2-0-SUCCEEDED-default.jhist */ - public static final String JOB_FILENAME_PATTERN_REGEX = ".*(job_[0-9]*_[0-9]*)_([0-9]*[aA-zZ]_)*(.*)$"; + public static final String JOB_FILENAME_PATTERN_REGEX = ".*(job_[0-9]*_[0-9]*)(-|_)([0-9]*[aA-zZ]*)*(.*)$"; // JobHistory file name parsing related public static final String JOB_CONF_FILE_END = "(.*)(conf.xml)$"; diff --git a/hraven-core/src/main/java/com/twitter/hraven/JobHistoryKeys.java b/hraven-core/src/main/java/com/twitter/hraven/JobHistoryKeys.java index 1e3f920..ea8844c 100644 --- a/hraven-core/src/main/java/com/twitter/hraven/JobHistoryKeys.java +++ b/hraven-core/src/main/java/com/twitter/hraven/JobHistoryKeys.java @@ -3,8 +3,11 @@ import java.util.HashMap; import java.util.Map; +import org.apache.commons.lang.StringUtils; import org.apache.hadoop.hbase.util.Bytes; +import com.google.common.collect.Maps; + /** * Contains the extract of the keys enum from * {@link org.apache.hadoop.mapreduce.JobHistoryCopy} class @@ -24,19 +27,108 @@ */ public enum JobHistoryKeys { - 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, COUNTERS, - SPLITS, JOB_PRIORITY, HTTP_PORT, - TRACKER_NAME, STATE_STRING, VERSION, - MAP_COUNTERS, REDUCE_COUNTERS, - VIEW_JOB, MODIFY_JOB, JOB_QUEUE; + JOBTRACKERID(String.class, null), + START_TIME(Long.class, "startTime"), + FINISH_TIME(Long.class, "finishTime"), + JOBID(String.class, "jobid"), + JOBNAME(String.class, "jobName"), + USER(String.class, "userName"), + JOBCONF(String.class, "jobConfPath"), + SUBMIT_TIME(Long.class, "submitTime"), + LAUNCH_TIME(Long.class, "launchTime"), + TOTAL_MAPS(Long.class, "totalMaps"), + TOTAL_REDUCES(Long.class, "totalReduces"), + FAILED_MAPS(Long.class, "failedMaps"), + FAILED_REDUCES(Long.class, "failedReduces"), + FINISHED_MAPS(Long.class, "finishedMaps"), + FINISHED_REDUCES(Long.class, "finishedReduces"), + JOB_STATUS(String.class, "jobStatus"), + TASKID(String.class, "taskid"), + HOSTNAME(String.class, "hostname"), + TASK_TYPE(String.class, "taskType"), + ERROR(String.class, "error"), + TASK_ATTEMPT_ID(String.class, "attemptId"), + TASK_STATUS(String.class, "taskStatus"), + COPY_PHASE(String.class, null), + SORT_PHASE(String.class, null), + REDUCE_PHASE(String.class, null), + SHUFFLE_FINISHED(Long.class, "shuffleFinishTime"), + SORT_FINISHED(Long.class, "sortFinishTime"), + COUNTERS(String.class, null), + SPLITS(String.class, "splitLocations"), + JOB_PRIORITY(String.class, null), + HTTP_PORT(Integer.class, "httpPort"), + TRACKER_NAME(String.class, "trackerName"), + STATE_STRING(String.class, "state"), + VERSION(String.class, null), + MAP_COUNTERS(String.class, null), + REDUCE_COUNTERS(String.class, null), + VIEW_JOB(String.class, null), + MODIFY_JOB(String.class, null), + JOB_QUEUE(String.class, "jobQueueName"), + // hadoop 2.0 related keys {@link JobHistoryParser} + applicationAttemptId(String.class, null), + containerId(String.class, null), + successfulAttemptId(String.class, null), + failedDueToAttempt(String.class, null), + workflowId(String.class, null), + workflowName(String.class, null), + workflowNodeName(String.class, null), + workflowAdjacencies(String.class, null), + locality(String.class, null), + avataar(String.class, null), + nodeManagerHost(String.class, null), + nodeManagerPort(Integer.class, null), + nodeManagerHttpPort(Integer.class, null), + acls(String.class, null), + uberized(String.class, null), + shufflePort(Integer.class, null), + mapFinishTime(Long.class, null), + port(Integer.class, null), + rackname(String.class, null), + clockSplits(String.class, null), + cpuUsages(String.class, null), + physMemKbytes(String.class, null), + vMemKbytes(String.class, null), + status(String.class, null), + TOTAL_COUNTERS(String.class, null), + TASK_COUNTERS(String.class, null), + TASK_ATTEMPT_COUNTERS(String.class, null); + + private final Class className; + private final String mapping; + + private JobHistoryKeys(Class className, String mapping) { + this.className = className; + this.mapping = mapping; + } + + public Class getClassName() { + return className; + } + + /** + * Data types represented by each of the defined job history field names + */ + public static Map> KEY_TYPES = Maps.newHashMap(); + static { + for (JobHistoryKeys t : JobHistoryKeys.values()) { + KEY_TYPES.put(t, t.getClassName()); + } + } + + /** + * Mapping the keys in 2.0 job history files to 1.0 key names + */ + public static Map HADOOP2_TO_HADOOP1_MAPPING = Maps.newHashMap(); + static { + for (JobHistoryKeys t : JobHistoryKeys.values()) { + if (StringUtils.isNotEmpty(t.mapping)) { + HADOOP2_TO_HADOOP1_MAPPING.put(t.mapping,t.toString()); + } + } + } + /** * Job history key names as bytes @@ -48,52 +140,4 @@ public enum JobHistoryKeys { KEYS_TO_BYTES.put(k, Bytes.toBytes(k.toString().toLowerCase())); } } - - /** - * Data types represented by each of the defined job history field names - */ - @SuppressWarnings("rawtypes") - public static Map KEY_TYPES = new HashMap(); - static { - KEY_TYPES.put(JOBTRACKERID, String.class); - KEY_TYPES.put(START_TIME, Long.class); - KEY_TYPES.put(FINISH_TIME, Long.class); - KEY_TYPES.put(JOBID, String.class); - KEY_TYPES.put(JOBNAME, String.class); - KEY_TYPES.put(USER, String.class); - KEY_TYPES.put(JOBCONF, String.class); - KEY_TYPES.put(SUBMIT_TIME, Long.class); - KEY_TYPES.put(LAUNCH_TIME, Long.class); - KEY_TYPES.put(TOTAL_MAPS, Long.class); - KEY_TYPES.put(TOTAL_REDUCES, Long.class); - KEY_TYPES.put(FAILED_MAPS, Long.class); - KEY_TYPES.put(FAILED_REDUCES, Long.class); - KEY_TYPES.put(FINISHED_MAPS, Long.class); - KEY_TYPES.put(FINISHED_REDUCES, Long.class); - KEY_TYPES.put(JOB_STATUS, String.class); - KEY_TYPES.put(TASKID, String.class); - KEY_TYPES.put(HOSTNAME, String.class); - KEY_TYPES.put(TASK_TYPE, String.class); - KEY_TYPES.put(ERROR, String.class); - KEY_TYPES.put(TASK_ATTEMPT_ID, String.class); - KEY_TYPES.put(TASK_STATUS, String.class); - KEY_TYPES.put(COPY_PHASE, String.class); - KEY_TYPES.put(SORT_PHASE, String.class); - KEY_TYPES.put(REDUCE_PHASE, String.class); - KEY_TYPES.put(SHUFFLE_FINISHED, Long.class); - KEY_TYPES.put(SORT_FINISHED, Long.class); - KEY_TYPES.put(COUNTERS, String.class); - KEY_TYPES.put(SPLITS, String.class); - KEY_TYPES.put(JOB_PRIORITY, String.class); - KEY_TYPES.put(HTTP_PORT, Integer.class); - KEY_TYPES.put(TRACKER_NAME, String.class); - KEY_TYPES.put(STATE_STRING, String.class); - KEY_TYPES.put(VERSION, String.class); - KEY_TYPES.put(MAP_COUNTERS, String.class); - KEY_TYPES.put(REDUCE_COUNTERS, String.class); - KEY_TYPES.put(VIEW_JOB, String.class); - KEY_TYPES.put(MODIFY_JOB, String.class); - KEY_TYPES.put(JOB_QUEUE, String.class); - } - } \ No newline at end of file diff --git a/hraven-core/src/main/java/com/twitter/hraven/datasource/JobHistoryRawService.java b/hraven-core/src/main/java/com/twitter/hraven/datasource/JobHistoryRawService.java index 5303bf2..d1f3d01 100644 --- a/hraven-core/src/main/java/com/twitter/hraven/datasource/JobHistoryRawService.java +++ b/hraven-core/src/main/java/com/twitter/hraven/datasource/JobHistoryRawService.java @@ -193,7 +193,7 @@ public Scan getHistoryRawTableScan(String cluster, String minJobId, } scan.setStartRow(startRow); - LOG.info("Starting raw table scan at " + Bytes.toStringBinary(startRow)); + LOG.info("Starting raw table scan at " + Bytes.toStringBinary(startRow) + " " + idConv.fromBytes(startRow)); FilterList filters = new FilterList(FilterList.Operator.MUST_PASS_ALL); @@ -210,7 +210,7 @@ public Scan getHistoryRawTableScan(String cluster, String minJobId, InclusiveStopFilter inclusiveStopFilter = new InclusiveStopFilter(lastRow); filters.addFilter(inclusiveStopFilter); LOG.info("Stopping raw table scan (stop filter) at " - + Bytes.toStringBinary(lastRow)); + + Bytes.toStringBinary(lastRow) + " " + idConv.fromBytes(lastRow)); // Add one to the jobSequence of the maximum JobId. JobId maximumJobId = new JobId(maxJobId); @@ -323,6 +323,24 @@ public String getRawJobHistory(QualifiedJobId jobId) throws IOException { return historyData; } + /** + * Returns the raw job history file as a byte array stored for the given cluster and job ID. + * @param jobId the cluster and job ID to look up + * @return the stored job history file contents or {@code null} if no corresponding record was found + * @throws IOException + */ + public byte[] getRawJobHistoryBytes(QualifiedJobId jobId) throws IOException { + byte[] historyData = null; + byte[] rowKey = idConv.toBytes(jobId); + Get get = new Get(rowKey); + get.addColumn(Constants.RAW_FAM_BYTES, Constants.JOBHISTORY_COL_BYTES); + Result result = rawTable.get(get); + if (result != null && !result.isEmpty()) { + historyData = result.getValue(Constants.RAW_FAM_BYTES, Constants.JOBHISTORY_COL_BYTES); + } + return historyData; + } + /** * @param result * from the {@link Scan} from diff --git a/hraven-core/src/test/java/com/twitter/hraven/TestJobHistoryKeys.java b/hraven-core/src/test/java/com/twitter/hraven/TestJobHistoryKeys.java index 51c7328..b899493 100644 --- a/hraven-core/src/test/java/com/twitter/hraven/TestJobHistoryKeys.java +++ b/hraven-core/src/test/java/com/twitter/hraven/TestJobHistoryKeys.java @@ -8,19 +8,29 @@ public class TestJobHistoryKeys { private enum test_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, + 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, COUNTERS, - SPLITS, JOB_PRIORITY, HTTP_PORT, - TRACKER_NAME, STATE_STRING, VERSION, - MAP_COUNTERS, REDUCE_COUNTERS, - VIEW_JOB, MODIFY_JOB, JOB_QUEUE; + SPLITS, JOB_PRIORITY, HTTP_PORT, + TRACKER_NAME, STATE_STRING, VERSION, + MAP_COUNTERS, REDUCE_COUNTERS, + VIEW_JOB, MODIFY_JOB, JOB_QUEUE, + // hadoop 2.0 related keys {@link JobHistoryParser} + applicationAttemptId, containerId, nodeManagerHost, + successfulAttemptId, failedDueToAttempt, + workflowId, workflowName, workflowNodeName, + workflowAdjacencies, locality, avataar, + nodeManagerPort, nodeManagerHttpPort, + acls, uberized, shufflePort, mapFinishTime, + port, rackname, clockSplits, cpuUsages, + physMemKbytes, vMemKbytes, status, TOTAL_COUNTERS, + TASK_COUNTERS, TASK_ATTEMPT_COUNTERS; } @Test diff --git a/hraven-etl/src/main/java/com/twitter/hraven/etl/JobFile.java b/hraven-etl/src/main/java/com/twitter/hraven/etl/JobFile.java index 98f1224..99eb72f 100644 --- a/hraven-etl/src/main/java/com/twitter/hraven/etl/JobFile.java +++ b/hraven-etl/src/main/java/com/twitter/hraven/etl/JobFile.java @@ -85,10 +85,10 @@ private void parseFilename() { Matcher confMatcher = CONF_PATTERN.matcher(filename); if (confMatcher.matches()) { isJobConfFile = true; - LOG.debug("Job Conf file " + filename); + LOG.debug("Job Conf file " + filename + " with job id: " + jobid); } else { isJobHistoryFile = true; - LOG.debug("Job History file " + filename); + LOG.debug("Job History file " + filename + " with job id: " + jobid); } } else { diff --git a/hraven-etl/src/main/java/com/twitter/hraven/etl/JobFileModifiedRangePathFilter.java b/hraven-etl/src/main/java/com/twitter/hraven/etl/JobFileModifiedRangePathFilter.java index 7fc39b7..c283a89 100644 --- a/hraven-etl/src/main/java/com/twitter/hraven/etl/JobFileModifiedRangePathFilter.java +++ b/hraven-etl/src/main/java/com/twitter/hraven/etl/JobFileModifiedRangePathFilter.java @@ -17,6 +17,8 @@ import java.io.IOException; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; @@ -45,6 +47,7 @@ public class JobFileModifiedRangePathFilter extends JobFilePathFilter { * The configuration of this processing job (not the files we are processing). */ private final Configuration myConf; + private static Log LOG = LogFactory.getLog(JobFileModifiedRangePathFilter.class); /** * Constructs a filter that accepts only JobFiles with lastModification time @@ -100,7 +103,6 @@ public boolean accept(Path path) { FileSystem fs = path.getFileSystem(myConf); FileStatus fileStatus = fs.getFileStatus(path); long fileModificationTimeMillis = fileStatus.getModificationTime(); - return accept(fileModificationTimeMillis); } catch (IOException e) { throw new ImportException("Cannot determine file modification time of " @@ -108,6 +110,7 @@ public boolean accept(Path path) { } } else { // Reject anything that does not match a job conf filename. + LOG.info(" Not a valid job conf / job history file "+ path.getName()); return false; } } diff --git a/hraven-etl/src/main/java/com/twitter/hraven/etl/JobHistoryFileParser.java b/hraven-etl/src/main/java/com/twitter/hraven/etl/JobHistoryFileParser.java index eb85687..3f0041e 100644 --- a/hraven-etl/src/main/java/com/twitter/hraven/etl/JobHistoryFileParser.java +++ b/hraven-etl/src/main/java/com/twitter/hraven/etl/JobHistoryFileParser.java @@ -35,7 +35,7 @@ public interface JobHistoryFileParser { * * @throws ProcessingException */ - public void parse(InputStream historyFile, JobKey jobKey); + public void parse(byte[] historyFile, JobKey jobKey); /** * Return the generated list of job puts assembled when history file is diff --git a/hraven-etl/src/main/java/com/twitter/hraven/etl/JobHistoryFileParserFactory.java b/hraven-etl/src/main/java/com/twitter/hraven/etl/JobHistoryFileParserFactory.java index 406a68c..61f548f 100644 --- a/hraven-etl/src/main/java/com/twitter/hraven/etl/JobHistoryFileParserFactory.java +++ b/hraven-etl/src/main/java/com/twitter/hraven/etl/JobHistoryFileParserFactory.java @@ -15,68 +15,104 @@ */ package com.twitter.hraven.etl; -import java.io.InputStream; +import org.apache.commons.lang.StringUtils; /** * Deal with {@link JobHistoryFileParser} implementations. + * Creates an appropriate Job History File Parser Object based + * on the type of job history file */ public class JobHistoryFileParserFactory { - /** - * determines the verison of hadoop that the history file belongs to - * - * @return currently returns 1 for hadoop 1 (pre MAPREDUCE-1016) for newer - * job history files, this method will look at the history file and - * return values appropriately - * - * (newer job history files have "AVRO-JSON" as the signature at the - * start of the file, REFERENCE: - * https://issues.apache.org/jira/browse/MAPREDUCE-1016? \ - * focusedCommentId=12763160& \ - * page=com.atlassian.jira.plugin.system - * .issuetabpanels:comment-tabpanel#comment-12763160 - * - */ - public static int getVersion(InputStream historyFile) { - return 1; - } + /** + * NOTE that this version string is a replica of + * {@link org.apache.hadoop.mapreduce.jobhistory.EventWriter} Since that class is not public, the + * VERSION variable there becomes package-level visible and hence we need a replica + */ + public static final String HADOOP2_VERSION_STRING = "Avro-Json"; + public static final String HADOOP1_VERSION_STRING = "Meta VERSION=\"1\" ."; + private static final int HADOOP2_VERSION_LENGTH = 9; + private static final int HADOOP1_VERSION_LENGTH = 18; + private static final int HISTORY_FILE_VERSION1 = 1; + private static final int HISTORY_FILE_VERSION2 = 2; - /** - * creates an instance of {@link JobHistoryParseHadoop1} - * - * to be later enhanced to return either {@link JobHistoryParseHadoop1} or - * an object that can parse post MAPREDUCE-1016 job history files - * - * @param historyFile - * : input stream to the history file contents - * - * @return an object of {@link JobHistoryParseHadoop1} that can parse Hadoop - * 1.0 (pre MAPREDUCE-1016) generated job history files Or return - * null if either input is null - * - */ - public static JobHistoryFileParser createJobHistoryFileParser( - InputStream historyFile) throws IllegalArgumentException { + /** + * determines the verison of hadoop that the history file belongs to + * + * @return + * returns 1 for hadoop 1 (pre MAPREDUCE-1016) + * returns 2 for newer job history files + * (newer job history files have "AVRO-JSON" as the signature at the start of the file, + * REFERENCE: https://issues.apache.org/jira/browse/MAPREDUCE-1016? \ + * focusedCommentId=12763160& \ page=com.atlassian.jira.plugin.system + * .issuetabpanels:comment-tabpanel#comment-12763160 + * + * @throws IllegalArgumentException if neither match + */ + public static int getVersion(byte[] historyFileContents) { + if(historyFileContents.length > HADOOP2_VERSION_LENGTH) { + // the first 10 bytes in a hadoop2.0 history file contain Avro-Json + String version2Part = new String(historyFileContents, 0, HADOOP2_VERSION_LENGTH); + if (StringUtils.equalsIgnoreCase(version2Part, HADOOP2_VERSION_STRING)) { + return HISTORY_FILE_VERSION2; + } else { + if(historyFileContents.length > HADOOP1_VERSION_LENGTH) { + // the first 18 bytes in a hadoop1.0 history file contain Meta VERSION="1" . + String version1Part = new String(historyFileContents, 0, HADOOP1_VERSION_LENGTH); + if (StringUtils.equalsIgnoreCase(version1Part, HADOOP1_VERSION_STRING)) { + return HISTORY_FILE_VERSION1; + } + } + } + } + // throw an exception if we did not find any matching version + throw new IllegalArgumentException(" Unknown format of job history file: " + historyFileContents); + } - if (historyFile == null) { - throw new IllegalArgumentException( - "Job history input stream should not be null"); - } + /** + * creates an instance of {@link JobHistoryParseHadoop1} + * or + * {@link JobHistoryParseHadoop2} that can parse post MAPREDUCE-1016 job history files + * + * @param historyFile: history file contents + * + * @return an object that can parse job history files + * Or return null if either input is null + */ + public static JobHistoryFileParser createJobHistoryFileParser( + byte[] historyFileContents) throws IllegalArgumentException { - int version = getVersion(historyFile); + if (historyFileContents == null) { + throw new IllegalArgumentException( + "Job history contents should not be null"); + } - switch (version) { - case 1: - return new JobHistoryFileParserHadoop1(); + int version = getVersion(historyFileContents); - /* - * right now, the default won't be in any code path but as we add - * support for post MAPREDUCE-1016 and Hadoop 2.0 this would be - * relevant - */ - default: - throw new IllegalArgumentException( - " Unknown format of job history file"); - } - } + switch (version) { + case 1: + return new JobHistoryFileParserHadoop1(); + + case 2: + return new JobHistoryFileParserHadoop2(); + + default: + throw new IllegalArgumentException( + " Unknown format of job history file "); + } + } + + /** + * @return HISTORY_FILE_VERSION1 + */ + public static int getHistoryFileVersion1() { + return HISTORY_FILE_VERSION1; + } + + /** + * @return HISTORY_FILE_VERSION2 + */ + public static int getHistoryFileVersion2() { + return HISTORY_FILE_VERSION2; + } } diff --git a/hraven-etl/src/main/java/com/twitter/hraven/etl/JobHistoryFileParserHadoop1.java b/hraven-etl/src/main/java/com/twitter/hraven/etl/JobHistoryFileParserHadoop1.java index edd4651..ecae30c 100644 --- a/hraven-etl/src/main/java/com/twitter/hraven/etl/JobHistoryFileParserHadoop1.java +++ b/hraven-etl/src/main/java/com/twitter/hraven/etl/JobHistoryFileParserHadoop1.java @@ -15,8 +15,8 @@ */ package com.twitter.hraven.etl; +import java.io.ByteArrayInputStream; import java.io.IOException; -import java.io.InputStream; import java.util.List; import org.apache.commons.logging.Log; @@ -44,12 +44,12 @@ public class JobHistoryFileParserHadoop1 implements JobHistoryFileParser { * */ @Override - public void parse(InputStream historyFile, JobKey jobKey) + public void parse(byte[] historyFile, JobKey jobKey) throws ProcessingException { try { jobHistoryListener = new JobHistoryListener(jobKey); - JobHistoryCopy.parseHistoryFromIS(historyFile, jobHistoryListener); + JobHistoryCopy.parseHistoryFromIS(new ByteArrayInputStream(historyFile), jobHistoryListener); } catch (IOException ioe) { LOG.error(" Exception during parsing hadoop 1.0 file ", ioe); throw new ProcessingException( diff --git a/hraven-etl/src/main/java/com/twitter/hraven/etl/JobHistoryFileParserHadoop2.java b/hraven-etl/src/main/java/com/twitter/hraven/etl/JobHistoryFileParserHadoop2.java new file mode 100644 index 0000000..557972c --- /dev/null +++ b/hraven-etl/src/main/java/com/twitter/hraven/etl/JobHistoryFileParserHadoop2.java @@ -0,0 +1,660 @@ +/* + * Copyright 2013 Twitter, Inc. Licensed under the Apache License, Version 2.0 (the "License"); you + * may not use this file except in compliance with the License. You may obtain a copy of the License + * at http://www.apache.org/licenses/LICENSE-2.0 Unless required by applicable law or agreed to in + * writing, software distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific + * language governing permissions and limitations under the License. + */ +package com.twitter.hraven.etl; + +import java.io.EOFException; +import java.io.IOException; +import java.util.Arrays; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericDatumReader; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.io.DatumReader; +import org.apache.avro.io.Decoder; +import org.apache.avro.io.DecoderFactory; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.mapred.JobHistoryCopy.RecordTypes; +import org.codehaus.jettison.json.JSONArray; +import org.codehaus.jettison.json.JSONException; +import org.codehaus.jettison.json.JSONObject; + +import com.google.common.collect.Maps; +import com.twitter.hraven.Constants; +import com.twitter.hraven.JobHistoryKeys; +import com.twitter.hraven.JobKey; +import com.twitter.hraven.TaskKey; +import com.twitter.hraven.util.ByteArrayWrapper; +import com.twitter.hraven.datasource.JobKeyConverter; +import com.twitter.hraven.datasource.ProcessingException; +import com.twitter.hraven.datasource.TaskKeyConverter; + +/** + * Deal with JobHistory file parsing for job history files which are generated after MAPREDUCE-1016 + * (hadoop 1.x (0.21 and later) and hadoop 2.x) + */ +public class JobHistoryFileParserHadoop2 implements JobHistoryFileParser { + + private JobKey jobKey; + /** Job ID, minus the leading "job_" */ + private String jobNumber = ""; + private byte[] jobKeyBytes; + private List jobPuts = new LinkedList(); + private List taskPuts = new LinkedList(); + private JobKeyConverter jobKeyConv = new JobKeyConverter(); + private TaskKeyConverter taskKeyConv = new TaskKeyConverter(); + + private static final String AM_ATTEMPT_PREFIX = "AM_"; + private static final String TASK_PREFIX = "task_"; + private static final String TASK_ATTEMPT_PREFIX = "attempt_"; + + private static final Log LOG = LogFactory.getLog(JobHistoryFileParserHadoop2.class); + + private Schema schema; + private Decoder decoder; + private DatumReader reader; + + private static final String TYPE = "type"; + private static final String EVENT = "event"; + private static final String NAME = "name"; + private static final String FIELDS = "fields"; + private static final String COUNTS = "counts"; + private static final String GROUPS = "groups"; + private static final String VALUE = "value"; + private static final String TASKID = "taskid"; + private static final String APPLICATION_ATTEMPTID = "applicationAttemptId"; + private static final String ATTEMPTID = "attemptId"; + + private static final String TYPE_INT = "int"; + private static final String TYPE_BOOLEAN = "boolean"; + private static final String TYPE_LONG = "long"; + private static final String TYPE_STRING = "String"; + /** only acls in the job history file seem to be of this type: map of strings */ + private static final String TYPE_MAP_STRINGS = "{\"type\":\"map\",\"values\":\"string\"}"; + /** + * vMemKbytes, clockSplit, physMemKbytes, cpuUsages are arrays of ints See MAPREDUCE-5432 + */ + private static final String TYPE_ARRAY_INTS = "{\"type\":\"array\",\"items\":\"int\"}"; + /** this is part of {@link org.apache.hadoop.mapreduce.jobhistory.TaskFailedEvent.java} */ + private static final String NULL_STRING = "[\"null\",\"string\"]"; + + public static enum Hadoop2RecordType { + /** + * populating this map since the symbols and key to get the types of fields the symbol denotes + * the record in the file (like JOB_SUBMITTED) and it's value in the map (like JobSubmitted) + * helps us get the types of fields that that record contains (this type information is present + * in the schema) + */ + JobFinished("JOB_FINISHED"), + JobInfoChange("JOB_INFO_CHANGED"), + JobInited("JOB_INITED"), + AMStarted("AM_STARTED"), + JobPriorityChange("JOB_PRIORITY_CHANGED"), + JobStatusChanged("JOB_STATUS_CHANGED"), + JobSubmitted("JOB_SUBMITTED"), + JobUnsuccessfulCompletion("JOB_KILLED","JOB_FAILED"), + MapAttemptFinished("MAP_ATTEMPT_FINISHED"), + ReduceAttemptFinished("REDUCE_ATTEMPT_FINISHED"), + TaskAttemptFinished("CLEANUP_ATTEMPT_FINISHED"), + TaskAttemptStarted("CLEANUP_ATTEMPT_STARTED", + "SETUP_ATTEMPT_STARTED", + "REDUCE_ATTEMPT_STARTED", + "MAP_ATTEMPT_STARTED"), + TaskAttemptUnsuccessfulCompletion("CLEANUP_ATTEMPT_KILLED", + "CLEANUP_ATTEMPT_FAILED", + "SETUP_ATTEMPT_KILLED", + "SETUP_ATTEMPT_FAILED", + "REDUCE_ATTEMPT_KILLED", + "REDUCE_ATTEMPT_FAILED", + "MAP_ATTEMPT_KILLED", + "MAP_ATTEMPT_FAILED"), + TaskFailed("TASK_FAILED"), + TaskFinished("TASK_FINISHED"), + TaskStarted("TASK_STARTED"), + TaskUpdated("TASK_UPDATED"); + + private final String[] recordNames; + + private Hadoop2RecordType(String... recordNames) { + if (recordNames != null) { + this.recordNames = recordNames; + } else { + this.recordNames = new String[0]; + } + } + + public String[] getRecordNames() { + return recordNames; + } + } + + public static enum CounterTypes { + counters, mapCounters, reduceCounters, totalCounters + } + private static Map EVENT_RECORD_NAMES = Maps.newHashMap(); + private static final Set COUNTER_NAMES = new HashSet(); + private Map> fieldTypes = + new HashMap>(); + + /** + * populates the COUNTER_NAMES hash set and EVENT_RECORD_NAMES hash map + */ + static { + /** + * populate the hash set for counter names + */ + for (CounterTypes ct : CounterTypes.values()) { + COUNTER_NAMES.add(ct.toString()); + } + + /** + * populate the hash map of EVENT_RECORD_NAMES + */ + for (Hadoop2RecordType t : Hadoop2RecordType.values()) { + for (String name : t.getRecordNames()) { + EVENT_RECORD_NAMES.put(name, t); + } + } + + } + + JobHistoryFileParserHadoop2() { + } + + /** + * {@inheritDoc} + */ + @Override + public void parse(byte[] historyFileContents, JobKey jobKey) throws ProcessingException { + + this.jobKey = jobKey; + this.jobKeyBytes = jobKeyConv.toBytes(jobKey); + setJobId(jobKey.getJobId().getJobIdString()); + + try { + FSDataInputStream in = + new FSDataInputStream(new ByteArrayWrapper(historyFileContents)); + + /** first line is the version, ignore it */ + String versionIgnore = in.readLine(); + + /** second line in file is the schema */ + this.schema = schema.parse(in.readLine()); + + /** now figure out the schema */ + understandSchema(schema.toString()); + + /** now read the rest of the file */ + this.reader = new GenericDatumReader(schema); + this.decoder = DecoderFactory.get().jsonDecoder(schema, in); + + GenericRecord record = null; + Hadoop2RecordType recType = null; + try { + while ((record = reader.read(null, decoder)) != null) { + if (record.get(TYPE) != null) { + recType = EVENT_RECORD_NAMES.get(record.get(TYPE).toString()); + } else { + throw new ProcessingException("expected one of " + + Arrays.asList(Hadoop2RecordType.values()) + + " \n but not found, cannot process this record! " + jobKey); + } + if (recType == null) { + throw new ProcessingException("new record type has surfaced: " + + record.get(TYPE).toString() + " cannot process this record! " + jobKey); + } + // GenericRecord's get returns an Object + Object eDetails = record.get(EVENT); + + // confirm that we got an "event" object + if (eDetails != null) { + JSONObject eventDetails = new JSONObject(eDetails.toString()); + processRecords(recType, eventDetails); + } else { + throw new ProcessingException("expected event details but not found " + + record.get(TYPE).toString() + " cannot process this record! " + jobKey); + } + } + } catch (EOFException eof) { + // not an error, simply end of file + LOG.info("Done parsing file, reached eof for " + jobKey); + } + } catch (IOException ioe) { + throw new ProcessingException(" Unable to parse history file in function parse, " + + "cannot process this record!" + jobKey + " error: ", ioe); + } catch (JSONException jse) { + throw new ProcessingException(" Unable to parse history file in function parse, " + + "cannot process this record! " + jobKey + " error: ", jse); + } catch (IllegalArgumentException iae) { + throw new ProcessingException(" Unable to parse history file in function parse, " + + "cannot process this record! " + jobKey + " error: ", iae); + } + + LOG.info("For " + this.jobKey + " #jobPuts " + jobPuts.size() + " #taskPuts: " + + taskPuts.size()); + } + + /** + * understand the schema so that we can parse the rest of the file + * @throws JSONException + */ + private void understandSchema(String schema) throws JSONException { + + JSONObject j1 = new JSONObject(schema); + JSONArray fields = j1.getJSONArray(FIELDS); + + String fieldName; + String fieldTypeValue; + Object recName; + + for (int k = 0; k < fields.length(); k++) { + if (fields.get(k) == null) { + continue; + } + JSONObject allEvents = new JSONObject(fields.get(k).toString()); + Object name = allEvents.get(NAME); + if (name != null) { + if (name.toString().equalsIgnoreCase(EVENT)) { + JSONArray allTypeDetails = allEvents.getJSONArray(TYPE); + for (int i = 0; i < allTypeDetails.length(); i++) { + JSONObject actual = (JSONObject) allTypeDetails.get(i); + JSONArray types = actual.getJSONArray(FIELDS); + Map typeDetails = new HashMap(); + for (int j = 0; j < types.length(); j++) { + if (types.getJSONObject(j) == null ) { + continue; + } + fieldName = types.getJSONObject(j).getString(NAME); + fieldTypeValue = types.getJSONObject(j).getString(TYPE); + if ((fieldName != null) && (fieldTypeValue != null)) { + typeDetails.put(fieldName, fieldTypeValue); + } + } + + recName = actual.get(NAME); + if (recName != null) { + /* the next statement may throw an IllegalArgumentException if + * it finds a new string that's not part of the Hadoop2RecordType enum + * that way we know what types of events we are parsing + */ + fieldTypes.put(Hadoop2RecordType.valueOf(recName.toString()), typeDetails); + } + } + } + } + } + } + + /** + * process the counter details example line in .jhist file for counters: { "name":"MAP_COUNTERS", + * "groups":[ { "name":"org.apache.hadoop.mapreduce.FileSystemCounter", + * "displayName":"File System Counters", "counts":[ { "name":"HDFS_BYTES_READ", + * "displayName":"HDFS: Number of bytes read", "value":480 }, { "name":"HDFS_BYTES_WRITTEN", + * "displayName":"HDFS: Number of bytes written", "value":0 } ] }, { + * "name":"org.apache.hadoop.mapreduce.TaskCounter", "displayName":"Map-Reduce Framework", + * "counts":[ { "name":"MAP_INPUT_RECORDS", "displayName":"Map input records", "value":10 }, { + * "name":"MAP_OUTPUT_RECORDS", "displayName":"Map output records", "value":10 } ] } ] } + */ + private void processCounters(Put p, JSONObject eventDetails, String key) { + + try { + JSONObject jsonCounters = eventDetails.getJSONObject(key); + String counterMetaGroupName = jsonCounters.getString(NAME); + JSONArray groups = jsonCounters.getJSONArray(GROUPS); + for (int i = 0; i < groups.length(); i++) { + JSONObject aCounter = groups.getJSONObject(i); + JSONArray counts = aCounter.getJSONArray(COUNTS); + for (int j = 0; j < counts.length(); j++) { + JSONObject countDetails = counts.getJSONObject(j); + populatePut(p, Constants.INFO_FAM_BYTES, counterMetaGroupName, aCounter.get(NAME) + .toString(), countDetails.get(NAME).toString(), countDetails.getLong(VALUE)); + } + } + } catch (JSONException e) { + throw new ProcessingException(" Caught json exception while processing counters ", e); + } + + } + + /** + * process the event details as per their data type from schema definition + * @throws JSONException + */ + private void + processAllTypes(Put p, Hadoop2RecordType recType, JSONObject eventDetails, String key) + throws JSONException { + + if (COUNTER_NAMES.contains(key)) { + processCounters(p, eventDetails, key); + } else { + String type = fieldTypes.get(recType).get(key); + if (type.equalsIgnoreCase(TYPE_STRING)) { + String value = eventDetails.getString(key); + populatePut(p, Constants.INFO_FAM_BYTES, key, value); + } else if (type.equalsIgnoreCase(TYPE_LONG)) { + long value = eventDetails.getLong(key); + populatePut(p, Constants.INFO_FAM_BYTES, key, value); + } else if (type.equalsIgnoreCase(TYPE_INT)) { + int value = eventDetails.getInt(key); + populatePut(p, Constants.INFO_FAM_BYTES, key, value); + } else if (type.equalsIgnoreCase(TYPE_BOOLEAN)) { + boolean value = eventDetails.getBoolean(key); + populatePut(p, Constants.INFO_FAM_BYTES, key, Boolean.toString(value)); + } else if (type.equalsIgnoreCase(TYPE_ARRAY_INTS)) { + String value = eventDetails.getString(key); + populatePut(p, Constants.INFO_FAM_BYTES, key, value); + } else if (type.equalsIgnoreCase(NULL_STRING)) { + // usually seen in FAILED tasks + String value = eventDetails.getString(key); + populatePut(p, Constants.INFO_FAM_BYTES, key, value); + } else if (type.equalsIgnoreCase(TYPE_MAP_STRINGS)) { + JSONObject ms = new JSONObject(eventDetails.get(key).toString()); + populatePut(p, Constants.INFO_FAM_BYTES, key, ms.toString()); + } else { + throw new ProcessingException("Encountered a new type " + type + + " unable to complete processing " + this.jobKey); + } + } + } + + /** + * iterate over the event details and prepare puts + * @throws JSONException + */ + private void iterateAndPreparePuts(JSONObject eventDetails, Put p, Hadoop2RecordType recType) + throws JSONException { + Iterator keys = eventDetails.keys(); + while (keys.hasNext()) { + String key = (String) keys.next(); + processAllTypes(p, recType, eventDetails, key); + } + } + + /** + * process individual records + * @throws JSONException + */ + private void processRecords(Hadoop2RecordType recType, JSONObject eventDetails) + throws JSONException { + + switch (recType) { + case JobFinished: + case JobInfoChange: + case JobInited: + case JobPriorityChange: + case JobStatusChanged: + case JobSubmitted: + case JobUnsuccessfulCompletion: + Put pJob = new Put(this.jobKeyBytes); + iterateAndPreparePuts(eventDetails, pJob, recType); + this.jobPuts.add(pJob); + break; + + case AMStarted: + byte[] amAttemptIdKeyBytes = + getAMKey(AM_ATTEMPT_PREFIX, eventDetails.getString(APPLICATION_ATTEMPTID)); + // generate a new put per AM Attempt + Put pAM = new Put(amAttemptIdKeyBytes); + pAM.add(Constants.INFO_FAM_BYTES, Constants.RECORD_TYPE_COL_BYTES, + Bytes.toBytes(RecordTypes.Task.toString())); + iterateAndPreparePuts(eventDetails, pAM, recType); + taskPuts.add(pAM); + break; + + case MapAttemptFinished: + byte[] taskMAttemptIdKeyBytes = + getTaskKey(TASK_ATTEMPT_PREFIX, this.jobNumber, eventDetails.getString(ATTEMPTID)); + Put pMTaskAttempt = new Put(taskMAttemptIdKeyBytes); + pMTaskAttempt.add(Constants.INFO_FAM_BYTES, Constants.RECORD_TYPE_COL_BYTES, + Bytes.toBytes(RecordTypes.MapAttempt.toString())); + iterateAndPreparePuts(eventDetails, pMTaskAttempt, recType); + this.taskPuts.add(pMTaskAttempt); + break; + + case ReduceAttemptFinished: + byte[] taskRAttemptIdKeyBytes = + getTaskKey(TASK_ATTEMPT_PREFIX, this.jobNumber, eventDetails.getString(ATTEMPTID)); + Put pRTaskAttempt = new Put(taskRAttemptIdKeyBytes); + pRTaskAttempt.add(Constants.INFO_FAM_BYTES, Constants.RECORD_TYPE_COL_BYTES, + Bytes.toBytes(RecordTypes.ReduceAttempt.toString())); + iterateAndPreparePuts(eventDetails, pRTaskAttempt, recType); + this.taskPuts.add(pRTaskAttempt); + break; + + case TaskAttemptFinished: + case TaskAttemptStarted: + case TaskAttemptUnsuccessfulCompletion: + byte[] taskAttemptIdKeyBytes = + getTaskKey(TASK_ATTEMPT_PREFIX, this.jobNumber, eventDetails.getString(ATTEMPTID)); + Put pTaskAttempt = new Put(taskAttemptIdKeyBytes); + pTaskAttempt.add(Constants.INFO_FAM_BYTES, Constants.RECORD_TYPE_COL_BYTES, + Bytes.toBytes(RecordTypes.Task.toString())); + iterateAndPreparePuts(eventDetails, pTaskAttempt, recType); + taskPuts.add(pTaskAttempt); + break; + + case TaskFailed: + case TaskStarted: + case TaskUpdated: + case TaskFinished: + byte[] taskIdKeyBytes = + getTaskKey(TASK_PREFIX, this.jobNumber, eventDetails.getString(TASKID)); + Put pTask = new Put(taskIdKeyBytes); + pTask.add(Constants.INFO_FAM_BYTES, Constants.RECORD_TYPE_COL_BYTES, + Bytes.toBytes(RecordTypes.Task.toString())); + iterateAndPreparePuts(eventDetails, pTask, recType); + taskPuts.add(pTask); + break; + default: + LOG.error("Check if recType was modified and has new members?"); + throw new ProcessingException("Check if recType was modified and has new members? " + recType); + } + } + + /** + * Sets the job ID and strips out the job number (job ID minus the "job_" prefix). + * @param id + */ + private void setJobId(String id) { + if (id != null && id.startsWith("job_") && id.length() > 4) { + this.jobNumber = id.substring(4); + } + } + + /** + * maintains compatibility between hadoop 1.0 keys and hadoop 2.0 keys. It also confirms that this + * key exists in JobHistoryKeys enum + * @throws IllegalArgumentException NullPointerException + */ + private String getKey(String key) throws IllegalArgumentException { + String checkKey = + JobHistoryKeys.HADOOP2_TO_HADOOP1_MAPPING.containsKey(key) ? JobHistoryKeys.HADOOP2_TO_HADOOP1_MAPPING + .get(key) : key; + return (JobHistoryKeys.valueOf(checkKey).toString()); + } + + /** + * populates a put for long values + * @param {@link Put} p + * @param {@link Constants} family + * @param String key + * @param long value + */ + private void populatePut(Put p, byte[] family, String key, long value) { + + byte[] valueBytes = null; + valueBytes = (value != 0L) ? Bytes.toBytes(value) : Constants.ZERO_LONG_BYTES; + byte[] qualifier = Bytes.toBytes(getKey(key).toLowerCase()); + p.add(family, qualifier, valueBytes); + } + + /** + * gets the int values as ints or longs some keys in 2.0 are now int, they were longs in 1.0 this + * will maintain compatiblity between 1.0 and 2.0 by casting those ints to long + * + * keeping this function package level visible (unit testing) + * @throws IllegalArgumentException if new key is encountered + */ + byte[] getValue(String key, int value) { + byte[] valueBytes = null; + Class clazz = JobHistoryKeys.KEY_TYPES.get(JobHistoryKeys.valueOf(key)); + if (clazz == null) { + throw new IllegalArgumentException(" unknown key " + key + " encountered while parsing " + + this.jobKey); + } + if (Long.class.equals(clazz)) { + valueBytes = (value != 0L) ? Bytes.toBytes(new Long(value)) : Constants.ZERO_LONG_BYTES; + } else { + valueBytes = (value != 0) ? Bytes.toBytes(value) : Constants.ZERO_INT_BYTES; + } + return valueBytes; + } + + /** + * populates a put for int values + * @param {@link Put} p + * @param {@link Constants} family + * @param String key + * @param int value + */ + private void populatePut(Put p, byte[] family, String key, int value) { + + String jobHistoryKey = getKey(key); + byte[] valueBytes = getValue(jobHistoryKey, value); + byte[] qualifier = Bytes.toBytes(jobHistoryKey.toLowerCase()); + p.add(family, qualifier, valueBytes); + } + + /** + * populates a put for string values + * @param {@link Put} p + * @param {@link Constants} family + * @param {@link String} key + * @param String value + */ + private void populatePut(Put p, byte[] family, String key, String value) { + byte[] valueBytes = null; + valueBytes = Bytes.toBytes(value); + byte[] qualifier = Bytes.toBytes(getKey(key).toLowerCase()); + p.add(family, qualifier, valueBytes); + } + + /** + * populates a put for {@link Counters} + * @param {@link Put} p + * @param {@link Constants} family + * @param String key + * @param String groupName + * @param String counterName + * @param long counterValue + */ + private void populatePut(Put p, byte[] family, String key, String groupName, String counterName, + Long counterValue) { + byte[] counterPrefix = null; + + try { + switch (JobHistoryKeys.valueOf(JobHistoryKeys.class, key)) { + case COUNTERS: + case TOTAL_COUNTERS: + case TASK_COUNTERS: + case TASK_ATTEMPT_COUNTERS: + counterPrefix = Bytes.add(Constants.COUNTER_COLUMN_PREFIX_BYTES, Constants.SEP_BYTES); + break; + case MAP_COUNTERS: + counterPrefix = Bytes.add(Constants.MAP_COUNTER_COLUMN_PREFIX_BYTES, Constants.SEP_BYTES); + break; + case REDUCE_COUNTERS: + counterPrefix = + Bytes.add(Constants.REDUCE_COUNTER_COLUMN_PREFIX_BYTES, Constants.SEP_BYTES); + break; + default: + throw new IllegalArgumentException("Unknown counter type " + key.toString()); + } + } catch (IllegalArgumentException iae) { + throw new ProcessingException("Unknown counter type " + key, iae); + } catch (NullPointerException npe) { + throw new ProcessingException("Null counter type " + key, npe); + } + + byte[] groupPrefix = Bytes.add(counterPrefix, Bytes.toBytes(groupName), Constants.SEP_BYTES); + byte[] qualifier = Bytes.add(groupPrefix, Bytes.toBytes(counterName)); + p.add(family, qualifier, Bytes.toBytes(counterValue)); + } + + /** + * Returns the Task ID or Task Attempt ID, stripped of the leading job ID, appended to the job row + * key. + */ + public byte[] getTaskKey(String prefix, String jobNumber, String fullId) { + String taskComponent = fullId; + if (fullId == null) { + taskComponent = ""; + } else { + String expectedPrefix = prefix + jobNumber + "_"; + if ((fullId.startsWith(expectedPrefix)) && (fullId.length() > expectedPrefix.length())) { + taskComponent = fullId.substring(expectedPrefix.length()); + } + } + return taskKeyConv.toBytes(new TaskKey(this.jobKey, taskComponent)); + } + + /** + * Returns the AM Attempt id stripped of the leading job ID, appended to the job row key. + */ + public byte[] getAMKey(String prefix, String fullId) { + + String taskComponent = prefix + fullId; + return taskKeyConv.toBytes(new TaskKey(this.jobKey, taskComponent)); + } + + /** + * {@inheritDoc} + */ + @Override + public List getJobPuts() { + return jobPuts; + } + + /** + * {@inheritDoc} + */ + @Override + public List getTaskPuts() { + return taskPuts; + } + + /** + * utitlity function for printing all puts + */ + public void printAllPuts(List p) { + for (Put p1 : p) { + Map> d = p1.getFamilyMap(); + for (byte[] k : d.keySet()) { + System.out.println(" k " + Bytes.toString(k)); + } + for (List lkv : d.values()) { + for (KeyValue kv : lkv) { + System.out.println("\n row: " + taskKeyConv.fromBytes(kv.getRow()) + + "\n " + Bytes.toString(kv.getQualifier()) + ": " + Bytes.toString(kv.getValue())); + } + } + } + } +} diff --git a/hraven-etl/src/main/java/com/twitter/hraven/mapreduce/HadoopCompat.java b/hraven-etl/src/main/java/com/twitter/hraven/mapreduce/HadoopCompat.java new file mode 100644 index 0000000..b2df900 --- /dev/null +++ b/hraven-etl/src/main/java/com/twitter/hraven/mapreduce/HadoopCompat.java @@ -0,0 +1,143 @@ +/** +Copyright 2013 Twitter, Inc. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + +http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package com.twitter.hraven.mapreduce; + +import java.lang.reflect.Constructor; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import org.apache.hadoop.mapreduce.Counter; +import org.apache.hadoop.mapreduce.TaskInputOutputContext; + +/* + * This is based on ContextFactory.java from hadoop-2.0.x sources. + */ + +/** + * Utility methods to allow applications to deal with inconsistencies between MapReduce Context + * Objects API between Hadoop 1.x and 2.x. + */ +public class HadoopCompat { + + private static final boolean useV21; + private static final Constructor GENERIC_COUNTER_CONSTRUCTOR; + private static final Method GET_COUNTER_METHOD; + private static final Method INCREMENT_COUNTER_METHOD; + + static { + boolean v21 = true; + final String PACKAGE = "org.apache.hadoop.mapreduce"; + try { + Class.forName(PACKAGE + ".task.JobContextImpl"); + } catch (ClassNotFoundException cnfe) { + v21 = false; + } + useV21 = v21; + Class genericCounterCls; + try { + if (v21) { + genericCounterCls = Class.forName(PACKAGE + ".counters.GenericCounter"); + } else { + genericCounterCls = Class.forName("org.apache.hadoop.mapred.Counters$Counter"); + } + } catch (ClassNotFoundException e) { + throw new IllegalArgumentException("Can't find class", e); + } + try { + GENERIC_COUNTER_CONSTRUCTOR = + genericCounterCls.getDeclaredConstructor(String.class, String.class, Long.TYPE); + GENERIC_COUNTER_CONSTRUCTOR.setAccessible(true); + + if (useV21) { + Method get_counter; + try { + get_counter = + Class.forName(PACKAGE + ".TaskAttemptContext").getMethod("getCounter", String.class, + String.class); + } catch (Exception e) { + get_counter = + Class.forName(PACKAGE + ".TaskInputOutputContext").getMethod("getCounter", + String.class, String.class); + } + GET_COUNTER_METHOD = get_counter; + } else { + GET_COUNTER_METHOD = + Class.forName(PACKAGE + ".TaskInputOutputContext").getMethod("getCounter", + String.class, String.class); + } + INCREMENT_COUNTER_METHOD = + Class.forName(PACKAGE + ".Counter").getMethod("increment", Long.TYPE); + } catch (SecurityException e) { + throw new IllegalArgumentException("Can't run constructor ", e); + } catch (NoSuchMethodException e) { + throw new IllegalArgumentException("Can't find constructor ", e); + } catch (ClassNotFoundException e) { + throw new IllegalArgumentException("Can't find class", e); + } + } + + /** + * True if runtime Hadoop version is 2.x, false otherwise. + */ + public static boolean isVersion2x() { + return useV21; + } + + /** + * @return with Hadoop 2 : new GenericCounter(args),
+ * with Hadoop 1 : new Counter(args) + */ + public static Counter newGenericCounter(String name, String displayName, long value) { + try { + return (Counter) GENERIC_COUNTER_CONSTRUCTOR.newInstance(name, displayName, value); + } catch (InstantiationException e) { + throw new IllegalArgumentException("Can't instantiate Counter", e); + } catch (IllegalAccessException e) { + throw new IllegalArgumentException("Can't instantiate Counter", e); + } catch (InvocationTargetException e) { + throw new IllegalArgumentException("Can't instantiate Counter", e); + } + } + + /** + * Invokes a method and rethrows any exception as runtime excetpions. + */ + private static Object invoke(Method method, Object obj, Object... args) { + try { + return method.invoke(obj, args); + } catch (IllegalAccessException e) { + throw new IllegalArgumentException("Can't invoke method " + method.getName(), e); + } catch (InvocationTargetException e) { + throw new IllegalArgumentException("Can't invoke method " + method.getName(), e); + } + } + + + /** + * Invoke getCounter() on TaskInputOutputContext. Works with both Hadoop 1 and 2. + */ + public static Counter getCounter(TaskInputOutputContext context, String groupName, + String counterName) { + return (Counter) invoke(GET_COUNTER_METHOD, context, groupName, counterName); + } + + /** + * Increment the counter. Works with both Hadoop 1 and 2 + */ + public static void incrementCounter(Counter counter, long increment) { + invoke(INCREMENT_COUNTER_METHOD, counter, increment); + } +} \ No newline at end of file diff --git a/hraven-etl/src/main/java/com/twitter/hraven/mapreduce/JobFileTableMapper.java b/hraven-etl/src/main/java/com/twitter/hraven/mapreduce/JobFileTableMapper.java index 23fe1c4..eea1d31 100644 --- a/hraven-etl/src/main/java/com/twitter/hraven/mapreduce/JobFileTableMapper.java +++ b/hraven-etl/src/main/java/com/twitter/hraven/mapreduce/JobFileTableMapper.java @@ -16,12 +16,11 @@ package com.twitter.hraven.mapreduce; import java.io.IOException; -import java.io.InputStream; import java.util.List; - import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; @@ -29,7 +28,6 @@ import org.apache.hadoop.io.Writable; import org.apache.hadoop.io.WritableComparable; import org.apache.hadoop.mapreduce.Mapper; - import com.twitter.hraven.Constants; import com.twitter.hraven.JobDesc; import com.twitter.hraven.JobDescFactory; @@ -171,13 +169,20 @@ protected void map( jobDesc.getAppId(), jobDesc.getVersion(), submitTimeMillis); context.progress(); - InputStream jobHistoryInputStream = rawService - .getJobHistoryInputStreamFromResult(value); + KeyValue keyValue = value.getColumnLatest(Constants.RAW_FAM_BYTES, + Constants.JOBHISTORY_COL_BYTES); + byte[] historyFileContents = null; + if (keyValue == null) { + throw new MissingColumnInResultException(Constants.RAW_FAM_BYTES, + Constants.JOBHISTORY_COL_BYTES); + } else { + historyFileContents = keyValue.getValue(); + } JobHistoryFileParser historyFileParser = JobHistoryFileParserFactory - .createJobHistoryFileParser(jobHistoryInputStream); + .createJobHistoryFileParser(historyFileContents); - historyFileParser.parse(jobHistoryInputStream, jobKey); + historyFileParser.parse(historyFileContents, jobKey); puts = historyFileParser.getJobPuts(); if (puts == null) { @@ -230,10 +235,10 @@ protected void map( if (success) { // Update counter to indicate failure. - context.getCounter(ProcessingCounter.RAW_ROW_SUCCESS_COUNT).increment(1); + HadoopCompat.incrementCounter(context.getCounter(ProcessingCounter.RAW_ROW_SUCCESS_COUNT), 1); } else { // Update counter to indicate failure. - context.getCounter(ProcessingCounter.RAW_ROW_ERROR_COUNT).increment(1); + HadoopCompat.incrementCounter(context.getCounter(ProcessingCounter.RAW_ROW_ERROR_COUNT),1); } // Indicate that we processed the RAW successfully so that we can skip it diff --git a/hraven-etl/src/test/java/com/twitter/hraven/TestJobFile.java b/hraven-etl/src/test/java/com/twitter/hraven/TestJobFile.java index 9feeaa7..ba17bcf 100644 --- a/hraven-etl/src/test/java/com/twitter/hraven/TestJobFile.java +++ b/hraven-etl/src/test/java/com/twitter/hraven/TestJobFile.java @@ -25,17 +25,26 @@ /** * Test the {@link JobFile} class. - * */ public class TestJobFile { - final static String VALID_JOB_CONF_FILENAME = "hostname1.example.com_1333569494142_job_201204041958_150125_conf.xml"; - final static String VALID_JOB_HISTORY_FILENAME = "hostname1.example.com_1333569494142_job_201204041958_1599_hadoop_App1%3Asomething%3Axyz%2F04%2F03-00%3A00%3A"; - final static String VALID_JOB_CONF_FILENAME2 = "hostname2.example.com_1334279672946_job_201204130114_0020_conf.xml"; - final static String VALID_JOB_HISTORY_FILENAME2 = "hostname2.example.com_1334279672946_job_201204130114_0020_user1_JobConfParser"; - final static String VALID_JOB_HISTORY_FILENAME3 = "job_201306192220_0001_1371680576348_hadoop_word+count"; - final static String VALID_JOB_CONF_FILENAME3 = "job_201306192220_0001_1371680576348_hadoop_conf.xml"; - final static String VALID_JOB_HISTORY_FILENAME4 = "job_201306192220_0001_1371680576348_hadoop_job_12345_12345"; + final static String VALID_JOB_CONF_FILENAME = + "hostname1.example.com_1333569494142_job_201204041958_150125_conf.xml"; + final static String VALID_JOB_HISTORY_FILENAME = + "hostname1.example.com_1333569494142_job_201204041958_1599_hadoop_App1%3Asomething%3Axyz%2F04%2F03-00%3A00%3A"; + final static String VALID_JOB_CONF_FILENAME2 = + "hostname2.example.com_1334279672946_job_201204130114_0020_conf.xml"; + final static String VALID_JOB_HISTORY_FILENAME2 = + "hostname2.example.com_1334279672946_job_201204130114_0020_user1_JobConfParser"; + final static String VALID_JOB_HISTORY_FILENAME3 = + "job_201306192220_0001_1371680576348_hadoop_word+count"; + final static String VALID_JOB_CONF_FILENAME3 = + "job_201306192220_0001_1371680576348_hadoop_conf.xml"; + final static String VALID_JOB_HISTORY_FILENAME4 = + "job_201306192220_0001_1371680576348_hadoop_job_12345_12345"; + /** 2.0 job history file name */ + final static String VALID_JOB_HISTORY_FILENAME5 = + "job_1374258111572_0003-1374260622449-userName1-TeraGen-1374260635219-2-0-SUCCEEDED-default.jhist"; final static String INVALID_JOB_FILENAME = "jabbedabbedoo.txt"; @@ -44,11 +53,10 @@ public class TestJobFile { */ @Test public void testJobConfFile() { - + JobFile jobFile = new JobFile(VALID_JOB_CONF_FILENAME); assertTrue("This should be a valid jobfile", jobFile.isJobConfFile()); - assertFalse("this should not be a job history file", - jobFile.isJobHistoryFile()); + assertFalse("this should not be a job history file", jobFile.isJobHistoryFile()); assertEquals("job_201204041958_150125", jobFile.getJobid()); jobFile = new JobFile(VALID_JOB_HISTORY_FILENAME); @@ -58,8 +66,7 @@ public void testJobConfFile() { jobFile = new JobFile(VALID_JOB_CONF_FILENAME2); assertTrue("This should be a valid jobfile", jobFile.isJobConfFile()); - assertFalse("this should not be a job history file", - jobFile.isJobHistoryFile()); + assertFalse("this should not be a job history file", jobFile.isJobHistoryFile()); assertEquals("job_201204130114_0020", jobFile.getJobid()); jobFile = new JobFile(VALID_JOB_HISTORY_FILENAME2); @@ -75,23 +82,31 @@ public void testJobConfFile() { jobFile = new JobFile(VALID_JOB_CONF_FILENAME3); assertTrue("This should be a valid jobfile", jobFile.isJobConfFile()); - assertFalse("this should not be a job history file", - jobFile.isJobHistoryFile()); + assertFalse("this should not be a job history file", jobFile.isJobHistoryFile()); assertEquals("job_201306192220_0001", jobFile.getJobid()); jobFile = new JobFile(INVALID_JOB_FILENAME); assertFalse("This should not be a valid jobfile", jobFile.isJobConfFile()); - assertFalse("this should not be a job history file", - jobFile.isJobHistoryFile()); + assertFalse("this should not be a job history file", jobFile.isJobHistoryFile()); jobFile = new JobFile(VALID_JOB_HISTORY_FILENAME4); assertFalse("This should not be a valid job conf file", jobFile.isJobConfFile()); assertTrue("this should be a job history file", jobFile.isJobHistoryFile()); - /* confirm that the job id was parsed correctly. Note that the - * history filename contains a "job at the end of the file name + /* + * confirm that the job id was parsed correctly. Note that the history filename contains a + * "job at the end of the file name */ assertEquals("job_201306192220_0001", jobFile.getJobid()); + jobFile = new JobFile(VALID_JOB_HISTORY_FILENAME5); + assertFalse("This should not be a valid job conf file", jobFile.isJobConfFile()); + assertTrue("this should be a job history file", jobFile.isJobHistoryFile()); + /* + * confirm that the job id was parsed correctly. Note that the history filename is a 2.0 job + * history file name + */ + assertEquals("job_1374258111572_0003", jobFile.getJobid()); + } } diff --git a/hraven-etl/src/test/java/com/twitter/hraven/etl/TestJobHistoryFileParserFactory.java b/hraven-etl/src/test/java/com/twitter/hraven/etl/TestJobHistoryFileParserFactory.java index e70757d..09f9d5a 100644 --- a/hraven-etl/src/test/java/com/twitter/hraven/etl/TestJobHistoryFileParserFactory.java +++ b/hraven-etl/src/test/java/com/twitter/hraven/etl/TestJobHistoryFileParserFactory.java @@ -15,10 +15,10 @@ */ package com.twitter.hraven.etl; +import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; -import java.io.ByteArrayInputStream; import org.junit.Test; /** @@ -32,10 +32,8 @@ public void testCreateJobHistoryFileParserCorrectCreation() { String jHist = "Meta VERSION=\"1\" .\n" + "Job JOBID=\"job_201301010000_12345\""; - byte[] fileContents = jHist.getBytes(); JobHistoryFileParser historyFileParser = JobHistoryFileParserFactory - .createJobHistoryFileParser(new ByteArrayInputStream( - fileContents)); + .createJobHistoryFileParser(jHist.getBytes()); assertNotNull(historyFileParser); @@ -47,11 +45,48 @@ public void testCreateJobHistoryFileParserCorrectCreation() { } /** + * check the versions in history files across hadoop 1 and hadoop 2 + */ + @Test + public void testGetVersion() { + String jHist1 = "Meta VERSION=\"1\" .\n" + "Job JOBID=\"job_201301010000_12345\""; + int version1 = JobHistoryFileParserFactory.getVersion(jHist1.getBytes()); + // confirm that we get back hadoop 1.0 version + assertEquals(JobHistoryFileParserFactory.getHistoryFileVersion1(), version1); + + String jHist2 = "Avro-Json\n" + + "{\"type\":\"record\",\"name\":\"Event\", " + + "\"namespace\":\"org.apache.hadoop.mapreduce.jobhistory\",\"fields\":[]\""; + int version2 = JobHistoryFileParserFactory.getVersion(jHist2.getBytes()); + // confirm that we get back hadoop 2.0 version + assertEquals(JobHistoryFileParserFactory.getHistoryFileVersion2(), version2); + } + + /** + * confirm that exception is thrown on incorrect input + */ + @Test(expected = IllegalArgumentException.class) + public void testGetVersionIncorrect2() { + String jHist2 = + "Avro-HELLO-Json\n" + "{\"type\":\"record\",\"name\":\"Event\", " + + "\"namespace\":\"org.apache.hadoop.mapreduce.jobhistory\",\"fields\":[]\""; + JobHistoryFileParserFactory.getVersion(jHist2.getBytes()); + } + + /** + * confirm that exception is thrown on incorrect input + */ + @Test(expected = IllegalArgumentException.class) + public void testGetVersionIncorrect1() { + String jHist1 = "Meta HELLO VERSION=\"1\" .\n" + "Job JOBID=\"job_201301010000_12345\""; + JobHistoryFileParserFactory.getVersion(jHist1.getBytes()); + } + + /** * confirm that exception is thrown on null input */ @Test(expected = IllegalArgumentException.class) public void testCreateJobHistoryFileParserNullCreation() { - JobHistoryFileParser historyFileParser = JobHistoryFileParserFactory .createJobHistoryFileParser(null); assertNull(historyFileParser); diff --git a/hraven-etl/src/test/java/com/twitter/hraven/etl/TestJobHistoryFileParserHadoop2.java b/hraven-etl/src/test/java/com/twitter/hraven/etl/TestJobHistoryFileParserHadoop2.java new file mode 100644 index 0000000..97f228c --- /dev/null +++ b/hraven-etl/src/test/java/com/twitter/hraven/etl/TestJobHistoryFileParserHadoop2.java @@ -0,0 +1,134 @@ +/* + * Copyright 2013 Twitter, Inc. Licensed under the Apache License, Version 2.0 (the "License"); you + * may not use this file except in compliance with the License. You may obtain a copy of the License + * at http://www.apache.org/licenses/LICENSE-2.0 Unless required by applicable law or agreed to in + * writing, software distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific + * language governing permissions and limitations under the License. + */ +package com.twitter.hraven.etl; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.util.Bytes; +import org.junit.Test; +import com.google.common.io.Files; +import com.twitter.hraven.JobHistoryKeys; +import com.twitter.hraven.JobKey; +import com.twitter.hraven.datasource.JobKeyConverter; +import com.twitter.hraven.datasource.TaskKeyConverter; +import java.io.File; +import java.io.IOException; +import java.util.Arrays; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +/** + * Test {@link JobHistoryFileParserHadoop2} + */ +public class TestJobHistoryFileParserHadoop2 { + + @Test + public void testCreateJobHistoryFileParserCorrectCreation() throws IOException { + + final String JOB_HISTORY_FILE_NAME = + "src/test/resources/job_1329348432655_0001-1329348443227-user-Sleep+job-1329348468601-10-1-SUCCEEDED-default.jhist"; + + File jobHistoryfile = new File(JOB_HISTORY_FILE_NAME); + byte[] contents = Files.toByteArray(jobHistoryfile); + JobHistoryFileParser historyFileParser = + JobHistoryFileParserFactory.createJobHistoryFileParser(contents); + assertNotNull(historyFileParser); + + // confirm that we get back an object that can parse hadoop 2.0 files + assertTrue(historyFileParser instanceof JobHistoryFileParserHadoop2); + + JobKey jobKey = new JobKey("cluster1", "user", "Sleep", 1, "job_1329348432655_0001"); + historyFileParser.parse(contents, jobKey); + + List jobPuts = historyFileParser.getJobPuts(); + assertEquals(4, jobPuts.size()); + + JobKeyConverter jobKeyConv = new JobKeyConverter(); + assertEquals("cluster1!user!Sleep!1!job_1329348432655_0001", + jobKeyConv.fromBytes(jobPuts.get(0).getRow()).toString()); + + List taskPuts = historyFileParser.getTaskPuts(); + assertEquals(taskPuts.size(), 45); + + TaskKeyConverter taskKeyConv = new TaskKeyConverter(); + + Set putRowKeys = + new HashSet(Arrays.asList( + "cluster1!user!Sleep!1!job_1329348432655_0001!AM_appattempt_1329348432655_0001_000001", + "cluster1!user!Sleep!1!job_1329348432655_0001!r_000000", + "cluster1!user!Sleep!1!job_1329348432655_0001!r_000000_0", + "cluster1!user!Sleep!1!job_1329348432655_0001!m_000000", + "cluster1!user!Sleep!1!job_1329348432655_0001!m_000000_0", + "cluster1!user!Sleep!1!job_1329348432655_0001!m_000009", + "cluster1!user!Sleep!1!job_1329348432655_0001!m_000009_0", + "cluster1!user!Sleep!1!job_1329348432655_0001!m_000008", + "cluster1!user!Sleep!1!job_1329348432655_0001!m_000008_0", + "cluster1!user!Sleep!1!job_1329348432655_0001!m_000007", + "cluster1!user!Sleep!1!job_1329348432655_0001!m_000007_0", + "cluster1!user!Sleep!1!job_1329348432655_0001!m_000006", + "cluster1!user!Sleep!1!job_1329348432655_0001!m_000006_0", + "cluster1!user!Sleep!1!job_1329348432655_0001!m_000005", + "cluster1!user!Sleep!1!job_1329348432655_0001!m_000005_0", + "cluster1!user!Sleep!1!job_1329348432655_0001!m_000004", + "cluster1!user!Sleep!1!job_1329348432655_0001!m_000004_0", + "cluster1!user!Sleep!1!job_1329348432655_0001!m_000003", + "cluster1!user!Sleep!1!job_1329348432655_0001!m_000003_0", + "cluster1!user!Sleep!1!job_1329348432655_0001!m_000002", + "cluster1!user!Sleep!1!job_1329348432655_0001!m_000002_0", + "cluster1!user!Sleep!1!job_1329348432655_0001!m_000001", + "cluster1!user!Sleep!1!job_1329348432655_0001!m_000001_0")); + + String tKey; + for (Put p : taskPuts) { + tKey = taskKeyConv.fromBytes(p.getRow()).toString(); + assertTrue(putRowKeys.contains(tKey)); + } + } + + /** + * To ensure we write these keys as Longs, not as ints + */ + @Test + public void testLongExpGetValuesIntBytes() { + + String[] keysToBeChecked = {"totalMaps", "totalReduces", "finishedMaps", + "finishedReduces", "failedMaps", "failedReduces"}; + byte[] byteValue = null; + int intValue10 = 10; + long longValue10 = 10L; + + JobHistoryFileParserHadoop2 jh = new JobHistoryFileParserHadoop2(); + + for(String key: keysToBeChecked) { + byteValue = jh.getValue(JobHistoryKeys.HADOOP2_TO_HADOOP1_MAPPING.get(key), intValue10); + assertEquals(Bytes.toLong(byteValue), longValue10); + } + } + + /** + * To ensure we write these keys as ints + */ + @Test + public void testIntExpGetValuesIntBytes() { + + String[] keysToBeChecked = {"httpPort"}; + byte[] byteValue = null; + int intValue10 = 10; + + JobHistoryFileParserHadoop2 jh = new JobHistoryFileParserHadoop2(); + + for(String key: keysToBeChecked) { + byteValue = jh.getValue(JobHistoryKeys.HADOOP2_TO_HADOOP1_MAPPING.get(key), intValue10); + assertEquals(Bytes.toInt(byteValue), intValue10); + } + } +} diff --git a/hraven-etl/src/test/resources/job_1329348432655_0001-1329348443227-user-Sleep+job-1329348468601-10-1-SUCCEEDED-default.jhist b/hraven-etl/src/test/resources/job_1329348432655_0001-1329348443227-user-Sleep+job-1329348468601-10-1-SUCCEEDED-default.jhist new file mode 100644 index 0000000..4849718 --- /dev/null +++ b/hraven-etl/src/test/resources/job_1329348432655_0001-1329348443227-user-Sleep+job-1329348468601-10-1-SUCCEEDED-default.jhist @@ -0,0 +1,51 @@ +Avro-Json +{"type":"record","name":"Event","namespace":"org.apache.hadoop.mapreduce.jobhistory","fields":[{"name":"type","type":{"type":"enum","name":"EventType","symbols":["JOB_SUBMITTED","JOB_INITED","JOB_FINISHED","JOB_PRIORITY_CHANGED","JOB_STATUS_CHANGED","JOB_FAILED","JOB_KILLED","JOB_INFO_CHANGED","TASK_STARTED","TASK_FINISHED","TASK_FAILED","TASK_UPDATED","NORMALIZED_RESOURCE","MAP_ATTEMPT_STARTED","MAP_ATTEMPT_FINISHED","MAP_ATTEMPT_FAILED","MAP_ATTEMPT_KILLED","REDUCE_ATTEMPT_STARTED","REDUCE_ATTEMPT_FINISHED","REDUCE_ATTEMPT_FAILED","REDUCE_ATTEMPT_KILLED","SETUP_ATTEMPT_STARTED","SETUP_ATTEMPT_FINISHED","SETUP_ATTEMPT_FAILED","SETUP_ATTEMPT_KILLED","CLEANUP_ATTEMPT_STARTED","CLEANUP_ATTEMPT_FINISHED","CLEANUP_ATTEMPT_FAILED","CLEANUP_ATTEMPT_KILLED","AM_STARTED"]}},{"name":"event","type":[{"type":"record","name":"JobFinished","fields":[{"name":"jobid","type":"string"},{"name":"finishTime","type":"long"},{"name":"finishedMaps","type":"int"},{"name":"finishedReduces","type":"int"},{"name":"failedMaps","type":"int"},{"name":"failedReduces","type":"int"},{"name":"totalCounters","type":{"type":"record","name":"JhCounters","fields":[{"name":"name","type":"string"},{"name":"groups","type":{"type":"array","items":{"type":"record","name":"JhCounterGroup","fields":[{"name":"name","type":"string"},{"name":"displayName","type":"string"},{"name":"counts","type":{"type":"array","items":{"type":"record","name":"JhCounter","fields":[{"name":"name","type":"string"},{"name":"displayName","type":"string"},{"name":"value","type":"long"}]}}}]}}}]}},{"name":"mapCounters","type":"JhCounters"},{"name":"reduceCounters","type":"JhCounters"}]},{"type":"record","name":"JobInfoChange","fields":[{"name":"jobid","type":"string"},{"name":"submitTime","type":"long"},{"name":"launchTime","type":"long"}]},{"type":"record","name":"JobInited","fields":[{"name":"jobid","type":"string"},{"name":"launchTime","type":"long"},{"name":"totalMaps","type":"int"},{"name":"totalReduces","type":"int"},{"name":"jobStatus","type":"string"},{"name":"uberized","type":"boolean"}]},{"type":"record","name":"AMStarted","fields":[{"name":"applicationAttemptId","type":"string"},{"name":"startTime","type":"long"},{"name":"containerId","type":"string"},{"name":"nodeManagerHost","type":"string"},{"name":"nodeManagerPort","type":"int"},{"name":"nodeManagerHttpPort","type":"int"}]},{"type":"record","name":"JobPriorityChange","fields":[{"name":"jobid","type":"string"},{"name":"priority","type":"string"}]},{"type":"record","name":"JobStatusChanged","fields":[{"name":"jobid","type":"string"},{"name":"jobStatus","type":"string"}]},{"type":"record","name":"JobSubmitted","fields":[{"name":"jobid","type":"string"},{"name":"jobName","type":"string"},{"name":"userName","type":"string"},{"name":"submitTime","type":"long"},{"name":"jobConfPath","type":"string"},{"name":"acls","type":{"type":"map","values":"string"}},{"name":"jobQueueName","type":"string"}]},{"type":"record","name":"JobUnsuccessfulCompletion","fields":[{"name":"jobid","type":"string"},{"name":"finishTime","type":"long"},{"name":"finishedMaps","type":"int"},{"name":"finishedReduces","type":"int"},{"name":"jobStatus","type":"string"}]},{"type":"record","name":"MapAttemptFinished","fields":[{"name":"taskid","type":"string"},{"name":"attemptId","type":"string"},{"name":"taskType","type":"string"},{"name":"taskStatus","type":"string"},{"name":"mapFinishTime","type":"long"},{"name":"finishTime","type":"long"},{"name":"hostname","type":"string"},{"name":"port","type":"int"},{"name":"rackname","type":"string"},{"name":"state","type":"string"},{"name":"counters","type":"JhCounters"},{"name":"clockSplits","type":{"type":"array","items":"int"}},{"name":"cpuUsages","type":{"type":"array","items":"int"}},{"name":"vMemKbytes","type":{"type":"array","items":"int"}},{"name":"physMemKbytes","type":{"type":"array","items":"int"}}]},{"type":"record","name":"ReduceAttemptFinished","fields":[{"name":"taskid","type":"string"},{"name":"attemptId","type":"string"},{"name":"taskType","type":"string"},{"name":"taskStatus","type":"string"},{"name":"shuffleFinishTime","type":"long"},{"name":"sortFinishTime","type":"long"},{"name":"finishTime","type":"long"},{"name":"hostname","type":"string"},{"name":"port","type":"int"},{"name":"rackname","type":"string"},{"name":"state","type":"string"},{"name":"counters","type":"JhCounters"},{"name":"clockSplits","type":{"type":"array","items":"int"}},{"name":"cpuUsages","type":{"type":"array","items":"int"}},{"name":"vMemKbytes","type":{"type":"array","items":"int"}},{"name":"physMemKbytes","type":{"type":"array","items":"int"}}]},{"type":"record","name":"TaskAttemptFinished","fields":[{"name":"taskid","type":"string"},{"name":"attemptId","type":"string"},{"name":"taskType","type":"string"},{"name":"taskStatus","type":"string"},{"name":"finishTime","type":"long"},{"name":"rackname","type":"string"},{"name":"hostname","type":"string"},{"name":"state","type":"string"},{"name":"counters","type":"JhCounters"}]},{"type":"record","name":"TaskAttemptStarted","fields":[{"name":"taskid","type":"string"},{"name":"taskType","type":"string"},{"name":"attemptId","type":"string"},{"name":"startTime","type":"long"},{"name":"trackerName","type":"string"},{"name":"httpPort","type":"int"},{"name":"shufflePort","type":"int"},{"name":"containerId","type":"string"}]},{"type":"record","name":"TaskAttemptUnsuccessfulCompletion","fields":[{"name":"taskid","type":"string"},{"name":"taskType","type":"string"},{"name":"attemptId","type":"string"},{"name":"finishTime","type":"long"},{"name":"hostname","type":"string"},{"name":"port","type":"int"},{"name":"rackname","type":"string"},{"name":"status","type":"string"},{"name":"error","type":"string"},{"name":"clockSplits","type":{"type":"array","items":"int"}},{"name":"cpuUsages","type":{"type":"array","items":"int"}},{"name":"vMemKbytes","type":{"type":"array","items":"int"}},{"name":"physMemKbytes","type":{"type":"array","items":"int"}}]},{"type":"record","name":"TaskFailed","fields":[{"name":"taskid","type":"string"},{"name":"taskType","type":"string"},{"name":"finishTime","type":"long"},{"name":"error","type":"string"},{"name":"failedDueToAttempt","type":["null","string"]},{"name":"status","type":"string"}]},{"type":"record","name":"TaskFinished","fields":[{"name":"taskid","type":"string"},{"name":"taskType","type":"string"},{"name":"finishTime","type":"long"},{"name":"status","type":"string"},{"name":"counters","type":"JhCounters"}]},{"type":"record","name":"TaskStarted","fields":[{"name":"taskid","type":"string"},{"name":"taskType","type":"string"},{"name":"startTime","type":"long"},{"name":"splitLocations","type":"string"}]},{"type":"record","name":"TaskUpdated","fields":[{"name":"taskid","type":"string"},{"name":"finishTime","type":"long"}]}]}]} +{"type":"AM_STARTED","event":{"org.apache.hadoop.mapreduce.jobhistory.AMStarted":{"applicationAttemptId":"appattempt_1329348432655_0001_000001","startTime":1329348445605,"containerId":"container_1329348432655_0001_01_000001","nodeManagerHost":"localhost","nodeManagerPort":45454,"nodeManagerHttpPort":9999}}} + {"type":"JOB_SUBMITTED","event":{"org.apache.hadoop.mapreduce.jobhistory.JobSubmitted":{"jobid":"job_1329348432655_0001","jobName":"Sleep job","userName":"user","submitTime":1329348443227,"jobConfPath":"hdfs://localhost:8021/tmp/hadoop-yarn/staging/user/.staging/job_1329348432655_0001/job.xml","acls":{},"jobQueueName":"default"}}} + {"type":"JOB_INITED","event":{"org.apache.hadoop.mapreduce.jobhistory.JobInited":{"jobid":"job_1329348432655_0001","launchTime":1329348448308,"totalMaps":10,"totalReduces":1,"jobStatus":"INITED","uberized":false}}} + {"type":"JOB_INFO_CHANGED","event":{"org.apache.hadoop.mapreduce.jobhistory.JobInfoChange":{"jobid":"job_1329348432655_0001","submitTime":1329348443227,"launchTime":1329348448308}}} + {"type":"TASK_STARTED","event":{"org.apache.hadoop.mapreduce.jobhistory.TaskStarted":{"taskid":"task_1329348432655_0001_m_000000","taskType":"MAP","startTime":1329348448373,"splitLocations":""}}} + {"type":"TASK_STARTED","event":{"org.apache.hadoop.mapreduce.jobhistory.TaskStarted":{"taskid":"task_1329348432655_0001_m_000001","taskType":"MAP","startTime":1329348448387,"splitLocations":""}}} + {"type":"TASK_STARTED","event":{"org.apache.hadoop.mapreduce.jobhistory.TaskStarted":{"taskid":"task_1329348432655_0001_m_000002","taskType":"MAP","startTime":1329348448387,"splitLocations":""}}} + {"type":"TASK_STARTED","event":{"org.apache.hadoop.mapreduce.jobhistory.TaskStarted":{"taskid":"task_1329348432655_0001_m_000003","taskType":"MAP","startTime":1329348448387,"splitLocations":""}}} + {"type":"TASK_STARTED","event":{"org.apache.hadoop.mapreduce.jobhistory.TaskStarted":{"taskid":"task_1329348432655_0001_m_000004","taskType":"MAP","startTime":1329348448387,"splitLocations":""}}} + {"type":"TASK_STARTED","event":{"org.apache.hadoop.mapreduce.jobhistory.TaskStarted":{"taskid":"task_1329348432655_0001_m_000005","taskType":"MAP","startTime":1329348448388,"splitLocations":""}}} + {"type":"TASK_STARTED","event":{"org.apache.hadoop.mapreduce.jobhistory.TaskStarted":{"taskid":"task_1329348432655_0001_m_000006","taskType":"MAP","startTime":1329348448388,"splitLocations":""}}} + {"type":"TASK_STARTED","event":{"org.apache.hadoop.mapreduce.jobhistory.TaskStarted":{"taskid":"task_1329348432655_0001_m_000007","taskType":"MAP","startTime":1329348448388,"splitLocations":""}}} + {"type":"TASK_STARTED","event":{"org.apache.hadoop.mapreduce.jobhistory.TaskStarted":{"taskid":"task_1329348432655_0001_m_000008","taskType":"MAP","startTime":1329348448388,"splitLocations":""}}} + {"type":"TASK_STARTED","event":{"org.apache.hadoop.mapreduce.jobhistory.TaskStarted":{"taskid":"task_1329348432655_0001_m_000009","taskType":"MAP","startTime":1329348448388,"splitLocations":""}}} + {"type":"TASK_STARTED","event":{"org.apache.hadoop.mapreduce.jobhistory.TaskStarted":{"taskid":"task_1329348432655_0001_r_000000","taskType":"REDUCE","startTime":1329348448388,"splitLocations":""}}} + {"type":"MAP_ATTEMPT_STARTED","event":{"org.apache.hadoop.mapreduce.jobhistory.TaskAttemptStarted":{"taskid":"task_1329348432655_0001_m_000000","taskType":"MAP","attemptId":"attempt_1329348432655_0001_m_000000_0","startTime":1329348450485,"trackerName":"localhost","httpPort":9999,"shufflePort":8080,"containerId":"container_1329348432655_0001_01_000002"}}} + {"type":"MAP_ATTEMPT_STARTED","event":{"org.apache.hadoop.mapreduce.jobhistory.TaskAttemptStarted":{"taskid":"task_1329348432655_0001_m_000002","taskType":"MAP","attemptId":"attempt_1329348432655_0001_m_000002_0","startTime":1329348450537,"trackerName":"localhost","httpPort":9999,"shufflePort":8080,"containerId":"container_1329348432655_0001_01_000004"}}} + {"type":"MAP_ATTEMPT_STARTED","event":{"org.apache.hadoop.mapreduce.jobhistory.TaskAttemptStarted":{"taskid":"task_1329348432655_0001_m_000004","taskType":"MAP","attemptId":"attempt_1329348432655_0001_m_000004_0","startTime":1329348450538,"trackerName":"localhost","httpPort":9999,"shufflePort":8080,"containerId":"container_1329348432655_0001_01_000006"}}} + {"type":"MAP_ATTEMPT_STARTED","event":{"org.apache.hadoop.mapreduce.jobhistory.TaskAttemptStarted":{"taskid":"task_1329348432655_0001_m_000001","taskType":"MAP","attemptId":"attempt_1329348432655_0001_m_000001_0","startTime":1329348450576,"trackerName":"localhost","httpPort":9999,"shufflePort":8080,"containerId":"container_1329348432655_0001_01_000003"}}} + {"type":"MAP_ATTEMPT_STARTED","event":{"org.apache.hadoop.mapreduce.jobhistory.TaskAttemptStarted":{"taskid":"task_1329348432655_0001_m_000003","taskType":"MAP","attemptId":"attempt_1329348432655_0001_m_000003_0","startTime":1329348450579,"trackerName":"localhost","httpPort":9999,"shufflePort":8080,"containerId":"container_1329348432655_0001_01_000005"}}} + {"type":"MAP_ATTEMPT_STARTED","event":{"org.apache.hadoop.mapreduce.jobhistory.TaskAttemptStarted":{"taskid":"task_1329348432655_0001_m_000005","taskType":"MAP","attemptId":"attempt_1329348432655_0001_m_000005_0","startTime":1329348450580,"trackerName":"localhost","httpPort":9999,"shufflePort":8080,"containerId":"container_1329348432655_0001_01_000007"}}} + {"type":"MAP_ATTEMPT_STARTED","event":{"org.apache.hadoop.mapreduce.jobhistory.TaskAttemptStarted":{"taskid":"task_1329348432655_0001_m_000006","taskType":"MAP","attemptId":"attempt_1329348432655_0001_m_000006_0","startTime":1329348450581,"trackerName":"localhost","httpPort":9999,"shufflePort":8080,"containerId":"container_1329348432655_0001_01_000008"}}} + {"type":"MAP_ATTEMPT_FINISHED","event":{"org.apache.hadoop.mapreduce.jobhistory.MapAttemptFinished":{"taskid":"task_1329348432655_0001_m_000005","attemptId":"attempt_1329348432655_0001_m_000005_0","taskType":"MAP","taskStatus":"SUCCEEDED","mapFinishTime":1329348458880,"finishTime":1329348461951,"hostname":"localhost","port":45454,"rackname":"/default-rack","state":"Sleeping... (1) ms left","counters":{"name":"COUNTERS","groups":[{"name":"org.apache.hadoop.mapreduce.FileSystemCounter","displayName":"File System Counters","counts":[{"name":"FILE_BYTES_READ","displayName":"FILE: Number of bytes read","value":120},{"name":"FILE_BYTES_WRITTEN","displayName":"FILE: Number of bytes written","value":48051},{"name":"FILE_READ_OPS","displayName":"FILE: Number of read operations","value":0},{"name":"FILE_LARGE_READ_OPS","displayName":"FILE: Number of large read operations","value":0},{"name":"FILE_WRITE_OPS","displayName":"FILE: Number of write operations","value":0},{"name":"HDFS_BYTES_READ","displayName":"HDFS: Number of bytes read","value":48},{"name":"HDFS_BYTES_WRITTEN","displayName":"HDFS: Number of bytes written","value":0},{"name":"HDFS_READ_OPS","displayName":"HDFS: Number of read operations","value":1},{"name":"HDFS_LARGE_READ_OPS","displayName":"HDFS: Number of large read operations","value":0},{"name":"HDFS_WRITE_OPS","displayName":"HDFS: Number of write operations","value":0}]},{"name":"org.apache.hadoop.mapreduce.TaskCounter","displayName":"Map-Reduce Framework","counts":[{"name":"MAP_INPUT_RECORDS","displayName":"Map input records","value":1},{"name":"MAP_OUTPUT_RECORDS","displayName":"Map output records","value":1},{"name":"MAP_OUTPUT_BYTES","displayName":"Map output bytes","value":4},{"name":"MAP_OUTPUT_MATERIALIZED_BYTES","displayName":"Map output materialized bytes","value":12},{"name":"SPLIT_RAW_BYTES","displayName":"Input split bytes","value":48},{"name":"COMBINE_INPUT_RECORDS","displayName":"Combine input records","value":0},{"name":"SPILLED_RECORDS","displayName":"Spilled Records","value":1},{"name":"FAILED_SHUFFLE","displayName":"Failed Shuffles","value":0},{"name":"MERGED_MAP_OUTPUTS","displayName":"Merged Map outputs","value":0},{"name":"GC_TIME_MILLIS","displayName":"GC time elapsed (ms)","value":518},{"name":"CPU_MILLISECONDS","displayName":"CPU time spent (ms)","value":340},{"name":"PHYSICAL_MEMORY_BYTES","displayName":"Physical memory (bytes) snapshot","value":183832576},{"name":"VIRTUAL_MEMORY_BYTES","displayName":"Virtual memory (bytes) snapshot","value":701161472},{"name":"COMMITTED_HEAP_BYTES","displayName":"Total committed heap usage (bytes)","value":181272576}]},{"name":"org.apache.hadoop.mapreduce.lib.input.FileInputFormatCounter","displayName":"File Input Format Counters ","counts":[{"name":"BYTES_READ","displayName":"Bytes Read","value":0}]}]},"clockSplits":[8691,129,128,129,128,129,128,129,128,129,128,129],"cpuUsages":[28,28,29,28,28,29,28,28,29,28,28,29],"vMemKbytes":[28530,85590,142651,199711,256772,313833,370894,427954,485015,542076,599136,656197],"physMemKbytes":[7479,22440,37400,52360,67321,82281,97242,112201,127162,142123,157082,172043]}}} + {"type":"MAP_ATTEMPT_FINISHED","event":{"org.apache.hadoop.mapreduce.jobhistory.MapAttemptFinished":{"taskid":"task_1329348432655_0001_m_000002","attemptId":"attempt_1329348432655_0001_m_000002_0","taskType":"MAP","taskStatus":"SUCCEEDED","mapFinishTime":1329348459298,"finishTime":1329348461952,"hostname":"localhost","port":45454,"rackname":"/default-rack","state":"Sleeping... (1) ms left","counters":{"name":"COUNTERS","groups":[{"name":"org.apache.hadoop.mapreduce.FileSystemCounter","displayName":"File System Counters","counts":[{"name":"FILE_BYTES_READ","displayName":"FILE: Number of bytes read","value":120},{"name":"FILE_BYTES_WRITTEN","displayName":"FILE: Number of bytes written","value":48051},{"name":"FILE_READ_OPS","displayName":"FILE: Number of read operations","value":0},{"name":"FILE_LARGE_READ_OPS","displayName":"FILE: Number of large read operations","value":0},{"name":"FILE_WRITE_OPS","displayName":"FILE: Number of write operations","value":0},{"name":"HDFS_BYTES_READ","displayName":"HDFS: Number of bytes read","value":48},{"name":"HDFS_BYTES_WRITTEN","displayName":"HDFS: Number of bytes written","value":0},{"name":"HDFS_READ_OPS","displayName":"HDFS: Number of read operations","value":1},{"name":"HDFS_LARGE_READ_OPS","displayName":"HDFS: Number of large read operations","value":0},{"name":"HDFS_WRITE_OPS","displayName":"HDFS: Number of write operations","value":0}]},{"name":"org.apache.hadoop.mapreduce.TaskCounter","displayName":"Map-Reduce Framework","counts":[{"name":"MAP_INPUT_RECORDS","displayName":"Map input records","value":1},{"name":"MAP_OUTPUT_RECORDS","displayName":"Map output records","value":1},{"name":"MAP_OUTPUT_BYTES","displayName":"Map output bytes","value":4},{"name":"MAP_OUTPUT_MATERIALIZED_BYTES","displayName":"Map output materialized bytes","value":12},{"name":"SPLIT_RAW_BYTES","displayName":"Input split bytes","value":48},{"name":"COMBINE_INPUT_RECORDS","displayName":"Combine input records","value":0},{"name":"SPILLED_RECORDS","displayName":"Spilled Records","value":1},{"name":"FAILED_SHUFFLE","displayName":"Failed Shuffles","value":0},{"name":"MERGED_MAP_OUTPUTS","displayName":"Merged Map outputs","value":0},{"name":"GC_TIME_MILLIS","displayName":"GC time elapsed (ms)","value":3},{"name":"CPU_MILLISECONDS","displayName":"CPU time spent (ms)","value":330},{"name":"PHYSICAL_MEMORY_BYTES","displayName":"Physical memory (bytes) snapshot","value":188891136},{"name":"VIRTUAL_MEMORY_BYTES","displayName":"Virtual memory (bytes) snapshot","value":706727936},{"name":"COMMITTED_HEAP_BYTES","displayName":"Total committed heap usage (bytes)","value":181272576}]},{"name":"org.apache.hadoop.mapreduce.lib.input.FileInputFormatCounter","displayName":"File Input Format Counters ","counts":[{"name":"BYTES_READ","displayName":"Bytes Read","value":0}]}]},"clockSplits":[9743,109,109,109,109,110,109,109,109,109,109,110],"cpuUsages":[27,28,27,28,27,28,27,28,27,28,27,28],"vMemKbytes":[28756,86270,143784,201297,258811,316324,373838,431351,488866,546379,603892,661407],"physMemKbytes":[7686,23058,38430,53801,69174,84545,99918,115289,130662,146034,161405,176778]}}} + {"type":"MAP_ATTEMPT_FINISHED","event":{"org.apache.hadoop.mapreduce.jobhistory.MapAttemptFinished":{"taskid":"task_1329348432655_0001_m_000006","attemptId":"attempt_1329348432655_0001_m_000006_0","taskType":"MAP","taskStatus":"SUCCEEDED","mapFinishTime":1329348459712,"finishTime":1329348461952,"hostname":"localhost","port":45454,"rackname":"/default-rack","state":"Sleeping... (1) ms left","counters":{"name":"COUNTERS","groups":[{"name":"org.apache.hadoop.mapreduce.FileSystemCounter","displayName":"File System Counters","counts":[{"name":"FILE_BYTES_READ","displayName":"FILE: Number of bytes read","value":120},{"name":"FILE_BYTES_WRITTEN","displayName":"FILE: Number of bytes written","value":48051},{"name":"FILE_READ_OPS","displayName":"FILE: Number of read operations","value":0},{"name":"FILE_LARGE_READ_OPS","displayName":"FILE: Number of large read operations","value":0},{"name":"FILE_WRITE_OPS","displayName":"FILE: Number of write operations","value":0},{"name":"HDFS_BYTES_READ","displayName":"HDFS: Number of bytes read","value":48},{"name":"HDFS_BYTES_WRITTEN","displayName":"HDFS: Number of bytes written","value":0},{"name":"HDFS_READ_OPS","displayName":"HDFS: Number of read operations","value":1},{"name":"HDFS_LARGE_READ_OPS","displayName":"HDFS: Number of large read operations","value":0},{"name":"HDFS_WRITE_OPS","displayName":"HDFS: Number of write operations","value":0}]},{"name":"org.apache.hadoop.mapreduce.TaskCounter","displayName":"Map-Reduce Framework","counts":[{"name":"MAP_INPUT_RECORDS","displayName":"Map input records","value":1},{"name":"MAP_OUTPUT_RECORDS","displayName":"Map output records","value":1},{"name":"MAP_OUTPUT_BYTES","displayName":"Map output bytes","value":4},{"name":"MAP_OUTPUT_MATERIALIZED_BYTES","displayName":"Map output materialized bytes","value":12},{"name":"SPLIT_RAW_BYTES","displayName":"Input split bytes","value":48},{"name":"COMBINE_INPUT_RECORDS","displayName":"Combine input records","value":0},{"name":"SPILLED_RECORDS","displayName":"Spilled Records","value":1},{"name":"FAILED_SHUFFLE","displayName":"Failed Shuffles","value":0},{"name":"MERGED_MAP_OUTPUTS","displayName":"Merged Map outputs","value":0},{"name":"GC_TIME_MILLIS","displayName":"GC time elapsed (ms)","value":105},{"name":"CPU_MILLISECONDS","displayName":"CPU time spent (ms)","value":340},{"name":"PHYSICAL_MEMORY_BYTES","displayName":"Physical memory (bytes) snapshot","value":175243264},{"name":"VIRTUAL_MEMORY_BYTES","displayName":"Virtual memory (bytes) snapshot","value":699457536},{"name":"COMMITTED_HEAP_BYTES","displayName":"Total committed heap usage (bytes)","value":165478400}]},{"name":"org.apache.hadoop.mapreduce.lib.input.FileInputFormatCounter","displayName":"File Input Format Counters ","counts":[{"name":"BYTES_READ","displayName":"Bytes Read","value":0}]}]},"clockSplits":[10240,61,62,61,61,62,61,61,62,61,61,62],"cpuUsages":[28,28,29,28,28,29,28,28,29,28,28,29],"vMemKbytes":[28461,85383,142305,199226,256149,313070,369993,426914,483837,540759,597680,654603],"physMemKbytes":[7130,21391,35653,49914,64175,78436,92698,106959,121221,135482,149743,164005]}}} + {"type":"TASK_FINISHED","event":{"org.apache.hadoop.mapreduce.jobhistory.TaskFinished":{"taskid":"task_1329348432655_0001_m_000005","taskType":"MAP","finishTime":1329348461951,"status":"SUCCEEDED","counters":{"name":"COUNTERS","groups":[{"name":"org.apache.hadoop.mapreduce.FileSystemCounter","displayName":"File System Counters","counts":[{"name":"FILE_BYTES_READ","displayName":"FILE: Number of bytes read","value":120},{"name":"FILE_BYTES_WRITTEN","displayName":"FILE: Number of bytes written","value":48051},{"name":"FILE_READ_OPS","displayName":"FILE: Number of read operations","value":0},{"name":"FILE_LARGE_READ_OPS","displayName":"FILE: Number of large read operations","value":0},{"name":"FILE_WRITE_OPS","displayName":"FILE: Number of write operations","value":0},{"name":"HDFS_BYTES_READ","displayName":"HDFS: Number of bytes read","value":48},{"name":"HDFS_BYTES_WRITTEN","displayName":"HDFS: Number of bytes written","value":0},{"name":"HDFS_READ_OPS","displayName":"HDFS: Number of read operations","value":1},{"name":"HDFS_LARGE_READ_OPS","displayName":"HDFS: Number of large read operations","value":0},{"name":"HDFS_WRITE_OPS","displayName":"HDFS: Number of write operations","value":0}]},{"name":"org.apache.hadoop.mapreduce.TaskCounter","displayName":"Map-Reduce Framework","counts":[{"name":"MAP_INPUT_RECORDS","displayName":"Map input records","value":1},{"name":"MAP_OUTPUT_RECORDS","displayName":"Map output records","value":1},{"name":"MAP_OUTPUT_BYTES","displayName":"Map output bytes","value":4},{"name":"MAP_OUTPUT_MATERIALIZED_BYTES","displayName":"Map output materialized bytes","value":12},{"name":"SPLIT_RAW_BYTES","displayName":"Input split bytes","value":48},{"name":"COMBINE_INPUT_RECORDS","displayName":"Combine input records","value":0},{"name":"SPILLED_RECORDS","displayName":"Spilled Records","value":1},{"name":"FAILED_SHUFFLE","displayName":"Failed Shuffles","value":0},{"name":"MERGED_MAP_OUTPUTS","displayName":"Merged Map outputs","value":0},{"name":"GC_TIME_MILLIS","displayName":"GC time elapsed (ms)","value":518},{"name":"CPU_MILLISECONDS","displayName":"CPU time spent (ms)","value":340},{"name":"PHYSICAL_MEMORY_BYTES","displayName":"Physical memory (bytes) snapshot","value":183832576},{"name":"VIRTUAL_MEMORY_BYTES","displayName":"Virtual memory (bytes) snapshot","value":701161472},{"name":"COMMITTED_HEAP_BYTES","displayName":"Total committed heap usage (bytes)","value":181272576}]},{"name":"org.apache.hadoop.mapreduce.lib.input.FileInputFormatCounter","displayName":"File Input Format Counters ","counts":[{"name":"BYTES_READ","displayName":"Bytes Read","value":0}]}]}}}} + {"type":"TASK_FINISHED","event":{"org.apache.hadoop.mapreduce.jobhistory.TaskFinished":{"taskid":"task_1329348432655_0001_m_000002","taskType":"MAP","finishTime":1329348461952,"status":"SUCCEEDED","counters":{"name":"COUNTERS","groups":[{"name":"org.apache.hadoop.mapreduce.FileSystemCounter","displayName":"File System Counters","counts":[{"name":"FILE_BYTES_READ","displayName":"FILE: Number of bytes read","value":120},{"name":"FILE_BYTES_WRITTEN","displayName":"FILE: Number of bytes written","value":48051},{"name":"FILE_READ_OPS","displayName":"FILE: Number of read operations","value":0},{"name":"FILE_LARGE_READ_OPS","displayName":"FILE: Number of large read operations","value":0},{"name":"FILE_WRITE_OPS","displayName":"FILE: Number of write operations","value":0},{"name":"HDFS_BYTES_READ","displayName":"HDFS: Number of bytes read","value":48},{"name":"HDFS_BYTES_WRITTEN","displayName":"HDFS: Number of bytes written","value":0},{"name":"HDFS_READ_OPS","displayName":"HDFS: Number of read operations","value":1},{"name":"HDFS_LARGE_READ_OPS","displayName":"HDFS: Number of large read operations","value":0},{"name":"HDFS_WRITE_OPS","displayName":"HDFS: Number of write operations","value":0}]},{"name":"org.apache.hadoop.mapreduce.TaskCounter","displayName":"Map-Reduce Framework","counts":[{"name":"MAP_INPUT_RECORDS","displayName":"Map input records","value":1},{"name":"MAP_OUTPUT_RECORDS","displayName":"Map output records","value":1},{"name":"MAP_OUTPUT_BYTES","displayName":"Map output bytes","value":4},{"name":"MAP_OUTPUT_MATERIALIZED_BYTES","displayName":"Map output materialized bytes","value":12},{"name":"SPLIT_RAW_BYTES","displayName":"Input split bytes","value":48},{"name":"COMBINE_INPUT_RECORDS","displayName":"Combine input records","value":0},{"name":"SPILLED_RECORDS","displayName":"Spilled Records","value":1},{"name":"FAILED_SHUFFLE","displayName":"Failed Shuffles","value":0},{"name":"MERGED_MAP_OUTPUTS","displayName":"Merged Map outputs","value":0},{"name":"GC_TIME_MILLIS","displayName":"GC time elapsed (ms)","value":3},{"name":"CPU_MILLISECONDS","displayName":"CPU time spent (ms)","value":330},{"name":"PHYSICAL_MEMORY_BYTES","displayName":"Physical memory (bytes) snapshot","value":188891136},{"name":"VIRTUAL_MEMORY_BYTES","displayName":"Virtual memory (bytes) snapshot","value":706727936},{"name":"COMMITTED_HEAP_BYTES","displayName":"Total committed heap usage (bytes)","value":181272576}]},{"name":"org.apache.hadoop.mapreduce.lib.input.FileInputFormatCounter","displayName":"File Input Format Counters ","counts":[{"name":"BYTES_READ","displayName":"Bytes Read","value":0}]}]}}}} + {"type":"TASK_FINISHED","event":{"org.apache.hadoop.mapreduce.jobhistory.TaskFinished":{"taskid":"task_1329348432655_0001_m_000006","taskType":"MAP","finishTime":1329348461952,"status":"SUCCEEDED","counters":{"name":"COUNTERS","groups":[{"name":"org.apache.hadoop.mapreduce.FileSystemCounter","displayName":"File System Counters","counts":[{"name":"FILE_BYTES_READ","displayName":"FILE: Number of bytes read","value":120},{"name":"FILE_BYTES_WRITTEN","displayName":"FILE: Number of bytes written","value":48051},{"name":"FILE_READ_OPS","displayName":"FILE: Number of read operations","value":0},{"name":"FILE_LARGE_READ_OPS","displayName":"FILE: Number of large read operations","value":0},{"name":"FILE_WRITE_OPS","displayName":"FILE: Number of write operations","value":0},{"name":"HDFS_BYTES_READ","displayName":"HDFS: Number of bytes read","value":48},{"name":"HDFS_BYTES_WRITTEN","displayName":"HDFS: Number of bytes written","value":0},{"name":"HDFS_READ_OPS","displayName":"HDFS: Number of read operations","value":1},{"name":"HDFS_LARGE_READ_OPS","displayName":"HDFS: Number of large read operations","value":0},{"name":"HDFS_WRITE_OPS","displayName":"HDFS: Number of write operations","value":0}]},{"name":"org.apache.hadoop.mapreduce.TaskCounter","displayName":"Map-Reduce Framework","counts":[{"name":"MAP_INPUT_RECORDS","displayName":"Map input records","value":1},{"name":"MAP_OUTPUT_RECORDS","displayName":"Map output records","value":1},{"name":"MAP_OUTPUT_BYTES","displayName":"Map output bytes","value":4},{"name":"MAP_OUTPUT_MATERIALIZED_BYTES","displayName":"Map output materialized bytes","value":12},{"name":"SPLIT_RAW_BYTES","displayName":"Input split bytes","value":48},{"name":"COMBINE_INPUT_RECORDS","displayName":"Combine input records","value":0},{"name":"SPILLED_RECORDS","displayName":"Spilled Records","value":1},{"name":"FAILED_SHUFFLE","displayName":"Failed Shuffles","value":0},{"name":"MERGED_MAP_OUTPUTS","displayName":"Merged Map outputs","value":0},{"name":"GC_TIME_MILLIS","displayName":"GC time elapsed (ms)","value":105},{"name":"CPU_MILLISECONDS","displayName":"CPU time spent (ms)","value":340},{"name":"PHYSICAL_MEMORY_BYTES","displayName":"Physical memory (bytes) snapshot","value":175243264},{"name":"VIRTUAL_MEMORY_BYTES","displayName":"Virtual memory (bytes) snapshot","value":699457536},{"name":"COMMITTED_HEAP_BYTES","displayName":"Total committed heap usage (bytes)","value":165478400}]},{"name":"org.apache.hadoop.mapreduce.lib.input.FileInputFormatCounter","displayName":"File Input Format Counters ","counts":[{"name":"BYTES_READ","displayName":"Bytes Read","value":0}]}]}}}} + {"type":"MAP_ATTEMPT_STARTED","event":{"org.apache.hadoop.mapreduce.jobhistory.TaskAttemptStarted":{"taskid":"task_1329348432655_0001_m_000007","taskType":"MAP","attemptId":"attempt_1329348432655_0001_m_000007_0","startTime":1329348462091,"trackerName":"localhost","httpPort":9999,"shufflePort":8080,"containerId":"container_1329348432655_0001_01_000009"}}} + {"type":"MAP_ATTEMPT_FINISHED","event":{"org.apache.hadoop.mapreduce.jobhistory.MapAttemptFinished":{"taskid":"task_1329348432655_0001_m_000004","attemptId":"attempt_1329348432655_0001_m_000004_0","taskType":"MAP","taskStatus":"SUCCEEDED","mapFinishTime":1329348459434,"finishTime":1329348462091,"hostname":"localhost","port":45454,"rackname":"/default-rack","state":"Sleeping... (1) ms left","counters":{"name":"COUNTERS","groups":[{"name":"org.apache.hadoop.mapreduce.FileSystemCounter","displayName":"File System Counters","counts":[{"name":"FILE_BYTES_READ","displayName":"FILE: Number of bytes read","value":120},{"name":"FILE_BYTES_WRITTEN","displayName":"FILE: Number of bytes written","value":48051},{"name":"FILE_READ_OPS","displayName":"FILE: Number of read operations","value":0},{"name":"FILE_LARGE_READ_OPS","displayName":"FILE: Number of large read operations","value":0},{"name":"FILE_WRITE_OPS","displayName":"FILE: Number of write operations","value":0},{"name":"HDFS_BYTES_READ","displayName":"HDFS: Number of bytes read","value":48},{"name":"HDFS_BYTES_WRITTEN","displayName":"HDFS: Number of bytes written","value":0},{"name":"HDFS_READ_OPS","displayName":"HDFS: Number of read operations","value":1},{"name":"HDFS_LARGE_READ_OPS","displayName":"HDFS: Number of large read operations","value":0},{"name":"HDFS_WRITE_OPS","displayName":"HDFS: Number of write operations","value":0}]},{"name":"org.apache.hadoop.mapreduce.TaskCounter","displayName":"Map-Reduce Framework","counts":[{"name":"MAP_INPUT_RECORDS","displayName":"Map input records","value":1},{"name":"MAP_OUTPUT_RECORDS","displayName":"Map output records","value":1},{"name":"MAP_OUTPUT_BYTES","displayName":"Map output bytes","value":4},{"name":"MAP_OUTPUT_MATERIALIZED_BYTES","displayName":"Map output materialized bytes","value":12},{"name":"SPLIT_RAW_BYTES","displayName":"Input split bytes","value":48},{"name":"COMBINE_INPUT_RECORDS","displayName":"Combine input records","value":0},{"name":"SPILLED_RECORDS","displayName":"Spilled Records","value":1},{"name":"FAILED_SHUFFLE","displayName":"Failed Shuffles","value":0},{"name":"MERGED_MAP_OUTPUTS","displayName":"Merged Map outputs","value":0},{"name":"GC_TIME_MILLIS","displayName":"GC time elapsed (ms)","value":323},{"name":"CPU_MILLISECONDS","displayName":"CPU time spent (ms)","value":330},{"name":"PHYSICAL_MEMORY_BYTES","displayName":"Physical memory (bytes) snapshot","value":189214720},{"name":"VIRTUAL_MEMORY_BYTES","displayName":"Virtual memory (bytes) snapshot","value":707006464},{"name":"COMMITTED_HEAP_BYTES","displayName":"Total committed heap usage (bytes)","value":181272576}]},{"name":"org.apache.hadoop.mapreduce.lib.input.FileInputFormatCounter","displayName":"File Input Format Counters ","counts":[{"name":"BYTES_READ","displayName":"Bytes Read","value":0}]}]},"clockSplits":[10303,81,82,81,82,81,81,82,81,82,81,82],"cpuUsages":[27,28,27,28,27,28,27,28,27,28,27,28],"vMemKbytes":[28767,86304,143840,201376,258913,316449,373986,431521,489058,546595,604130,661667],"physMemKbytes":[7698,23097,38495,53893,69292,84690,100089,115486,130885,146284,161681,177080]}}} + {"type":"TASK_FINISHED","event":{"org.apache.hadoop.mapreduce.jobhistory.TaskFinished":{"taskid":"task_1329348432655_0001_m_000004","taskType":"MAP","finishTime":1329348462091,"status":"SUCCEEDED","counters":{"name":"COUNTERS","groups":[{"name":"org.apache.hadoop.mapreduce.FileSystemCounter","displayName":"File System Counters","counts":[{"name":"FILE_BYTES_READ","displayName":"FILE: Number of bytes read","value":120},{"name":"FILE_BYTES_WRITTEN","displayName":"FILE: Number of bytes written","value":48051},{"name":"FILE_READ_OPS","displayName":"FILE: Number of read operations","value":0},{"name":"FILE_LARGE_READ_OPS","displayName":"FILE: Number of large read operations","value":0},{"name":"FILE_WRITE_OPS","displayName":"FILE: Number of write operations","value":0},{"name":"HDFS_BYTES_READ","displayName":"HDFS: Number of bytes read","value":48},{"name":"HDFS_BYTES_WRITTEN","displayName":"HDFS: Number of bytes written","value":0},{"name":"HDFS_READ_OPS","displayName":"HDFS: Number of read operations","value":1},{"name":"HDFS_LARGE_READ_OPS","displayName":"HDFS: Number of large read operations","value":0},{"name":"HDFS_WRITE_OPS","displayName":"HDFS: Number of write operations","value":0}]},{"name":"org.apache.hadoop.mapreduce.TaskCounter","displayName":"Map-Reduce Framework","counts":[{"name":"MAP_INPUT_RECORDS","displayName":"Map input records","value":1},{"name":"MAP_OUTPUT_RECORDS","displayName":"Map output records","value":1},{"name":"MAP_OUTPUT_BYTES","displayName":"Map output bytes","value":4},{"name":"MAP_OUTPUT_MATERIALIZED_BYTES","displayName":"Map output materialized bytes","value":12},{"name":"SPLIT_RAW_BYTES","displayName":"Input split bytes","value":48},{"name":"COMBINE_INPUT_RECORDS","displayName":"Combine input records","value":0},{"name":"SPILLED_RECORDS","displayName":"Spilled Records","value":1},{"name":"FAILED_SHUFFLE","displayName":"Failed Shuffles","value":0},{"name":"MERGED_MAP_OUTPUTS","displayName":"Merged Map outputs","value":0},{"name":"GC_TIME_MILLIS","displayName":"GC time elapsed (ms)","value":323},{"name":"CPU_MILLISECONDS","displayName":"CPU time spent (ms)","value":330},{"name":"PHYSICAL_MEMORY_BYTES","displayName":"Physical memory (bytes) snapshot","value":189214720},{"name":"VIRTUAL_MEMORY_BYTES","displayName":"Virtual memory (bytes) snapshot","value":707006464},{"name":"COMMITTED_HEAP_BYTES","displayName":"Total committed heap usage (bytes)","value":181272576}]},{"name":"org.apache.hadoop.mapreduce.lib.input.FileInputFormatCounter","displayName":"File Input Format Counters ","counts":[{"name":"BYTES_READ","displayName":"Bytes Read","value":0}]}]}}}} + {"type":"MAP_ATTEMPT_FINISHED","event":{"org.apache.hadoop.mapreduce.jobhistory.MapAttemptFinished":{"taskid":"task_1329348432655_0001_m_000001","attemptId":"attempt_1329348432655_0001_m_000001_0","taskType":"MAP","taskStatus":"SUCCEEDED","mapFinishTime":1329348461344,"finishTime":1329348462170,"hostname":"localhost","port":45454,"rackname":"/default-rack","state":"Sleeping... (1) ms left","counters":{"name":"COUNTERS","groups":[{"name":"org.apache.hadoop.mapreduce.FileSystemCounter","displayName":"File System Counters","counts":[{"name":"FILE_BYTES_READ","displayName":"FILE: Number of bytes read","value":120},{"name":"FILE_BYTES_WRITTEN","displayName":"FILE: Number of bytes written","value":48051},{"name":"FILE_READ_OPS","displayName":"FILE: Number of read operations","value":0},{"name":"FILE_LARGE_READ_OPS","displayName":"FILE: Number of large read operations","value":0},{"name":"FILE_WRITE_OPS","displayName":"FILE: Number of write operations","value":0},{"name":"HDFS_BYTES_READ","displayName":"HDFS: Number of bytes read","value":48},{"name":"HDFS_BYTES_WRITTEN","displayName":"HDFS: Number of bytes written","value":0},{"name":"HDFS_READ_OPS","displayName":"HDFS: Number of read operations","value":1},{"name":"HDFS_LARGE_READ_OPS","displayName":"HDFS: Number of large read operations","value":0},{"name":"HDFS_WRITE_OPS","displayName":"HDFS: Number of write operations","value":0}]},{"name":"org.apache.hadoop.mapreduce.TaskCounter","displayName":"Map-Reduce Framework","counts":[{"name":"MAP_INPUT_RECORDS","displayName":"Map input records","value":1},{"name":"MAP_OUTPUT_RECORDS","displayName":"Map output records","value":1},{"name":"MAP_OUTPUT_BYTES","displayName":"Map output bytes","value":4},{"name":"MAP_OUTPUT_MATERIALIZED_BYTES","displayName":"Map output materialized bytes","value":12},{"name":"SPLIT_RAW_BYTES","displayName":"Input split bytes","value":48},{"name":"COMBINE_INPUT_RECORDS","displayName":"Combine input records","value":0},{"name":"SPILLED_RECORDS","displayName":"Spilled Records","value":1},{"name":"FAILED_SHUFFLE","displayName":"Failed Shuffles","value":0},{"name":"MERGED_MAP_OUTPUTS","displayName":"Merged Map outputs","value":0},{"name":"GC_TIME_MILLIS","displayName":"GC time elapsed (ms)","value":257},{"name":"CPU_MILLISECONDS","displayName":"CPU time spent (ms)","value":380},{"name":"PHYSICAL_MEMORY_BYTES","displayName":"Physical memory (bytes) snapshot","value":184819712},{"name":"VIRTUAL_MEMORY_BYTES","displayName":"Virtual memory (bytes) snapshot","value":708714496},{"name":"COMMITTED_HEAP_BYTES","displayName":"Total committed heap usage (bytes)","value":165478400}]},{"name":"org.apache.hadoop.mapreduce.lib.input.FileInputFormatCounter","displayName":"File Input Format Counters ","counts":[{"name":"BYTES_READ","displayName":"Bytes Read","value":0}]}]},"clockSplits":[10971,50,51,50,50,51,50,50,51,50,50,51],"cpuUsages":[31,32,32,31,32,32,31,32,32,31,32,32],"vMemKbytes":[28837,86512,144188,201863,259538,317213,374889,432564,490240,547915,605590,663266],"physMemKbytes":[7520,22560,37601,52641,67682,82723,97764,112804,127845,142886,157926,172967]}}} + {"type":"TASK_FINISHED","event":{"org.apache.hadoop.mapreduce.jobhistory.TaskFinished":{"taskid":"task_1329348432655_0001_m_000001","taskType":"MAP","finishTime":1329348462170,"status":"SUCCEEDED","counters":{"name":"COUNTERS","groups":[{"name":"org.apache.hadoop.mapreduce.FileSystemCounter","displayName":"File System Counters","counts":[{"name":"FILE_BYTES_READ","displayName":"FILE: Number of bytes read","value":120},{"name":"FILE_BYTES_WRITTEN","displayName":"FILE: Number of bytes written","value":48051},{"name":"FILE_READ_OPS","displayName":"FILE: Number of read operations","value":0},{"name":"FILE_LARGE_READ_OPS","displayName":"FILE: Number of large read operations","value":0},{"name":"FILE_WRITE_OPS","displayName":"FILE: Number of write operations","value":0},{"name":"HDFS_BYTES_READ","displayName":"HDFS: Number of bytes read","value":48},{"name":"HDFS_BYTES_WRITTEN","displayName":"HDFS: Number of bytes written","value":0},{"name":"HDFS_READ_OPS","displayName":"HDFS: Number of read operations","value":1},{"name":"HDFS_LARGE_READ_OPS","displayName":"HDFS: Number of large read operations","value":0},{"name":"HDFS_WRITE_OPS","displayName":"HDFS: Number of write operations","value":0}]},{"name":"org.apache.hadoop.mapreduce.TaskCounter","displayName":"Map-Reduce Framework","counts":[{"name":"MAP_INPUT_RECORDS","displayName":"Map input records","value":1},{"name":"MAP_OUTPUT_RECORDS","displayName":"Map output records","value":1},{"name":"MAP_OUTPUT_BYTES","displayName":"Map output bytes","value":4},{"name":"MAP_OUTPUT_MATERIALIZED_BYTES","displayName":"Map output materialized bytes","value":12},{"name":"SPLIT_RAW_BYTES","displayName":"Input split bytes","value":48},{"name":"COMBINE_INPUT_RECORDS","displayName":"Combine input records","value":0},{"name":"SPILLED_RECORDS","displayName":"Spilled Records","value":1},{"name":"FAILED_SHUFFLE","displayName":"Failed Shuffles","value":0},{"name":"MERGED_MAP_OUTPUTS","displayName":"Merged Map outputs","value":0},{"name":"GC_TIME_MILLIS","displayName":"GC time elapsed (ms)","value":257},{"name":"CPU_MILLISECONDS","displayName":"CPU time spent (ms)","value":380},{"name":"PHYSICAL_MEMORY_BYTES","displayName":"Physical memory (bytes) snapshot","value":184819712},{"name":"VIRTUAL_MEMORY_BYTES","displayName":"Virtual memory (bytes) snapshot","value":708714496},{"name":"COMMITTED_HEAP_BYTES","displayName":"Total committed heap usage (bytes)","value":165478400}]},{"name":"org.apache.hadoop.mapreduce.lib.input.FileInputFormatCounter","displayName":"File Input Format Counters ","counts":[{"name":"BYTES_READ","displayName":"Bytes Read","value":0}]}]}}}} + {"type":"MAP_ATTEMPT_FINISHED","event":{"org.apache.hadoop.mapreduce.jobhistory.MapAttemptFinished":{"taskid":"task_1329348432655_0001_m_000003","attemptId":"attempt_1329348432655_0001_m_000003_0","taskType":"MAP","taskStatus":"SUCCEEDED","mapFinishTime":1329348461821,"finishTime":1329348462178,"hostname":"localhost","port":45454,"rackname":"/default-rack","state":"Sleeping... (1) ms left","counters":{"name":"COUNTERS","groups":[{"name":"org.apache.hadoop.mapreduce.FileSystemCounter","displayName":"File System Counters","counts":[{"name":"FILE_BYTES_READ","displayName":"FILE: Number of bytes read","value":120},{"name":"FILE_BYTES_WRITTEN","displayName":"FILE: Number of bytes written","value":48051},{"name":"FILE_READ_OPS","displayName":"FILE: Number of read operations","value":0},{"name":"FILE_LARGE_READ_OPS","displayName":"FILE: Number of large read operations","value":0},{"name":"FILE_WRITE_OPS","displayName":"FILE: Number of write operations","value":0},{"name":"HDFS_BYTES_READ","displayName":"HDFS: Number of bytes read","value":48},{"name":"HDFS_BYTES_WRITTEN","displayName":"HDFS: Number of bytes written","value":0},{"name":"HDFS_READ_OPS","displayName":"HDFS: Number of read operations","value":1},{"name":"HDFS_LARGE_READ_OPS","displayName":"HDFS: Number of large read operations","value":0},{"name":"HDFS_WRITE_OPS","displayName":"HDFS: Number of write operations","value":0}]},{"name":"org.apache.hadoop.mapreduce.TaskCounter","displayName":"Map-Reduce Framework","counts":[{"name":"MAP_INPUT_RECORDS","displayName":"Map input records","value":1},{"name":"MAP_OUTPUT_RECORDS","displayName":"Map output records","value":1},{"name":"MAP_OUTPUT_BYTES","displayName":"Map output bytes","value":4},{"name":"MAP_OUTPUT_MATERIALIZED_BYTES","displayName":"Map output materialized bytes","value":12},{"name":"SPLIT_RAW_BYTES","displayName":"Input split bytes","value":48},{"name":"COMBINE_INPUT_RECORDS","displayName":"Combine input records","value":0},{"name":"SPILLED_RECORDS","displayName":"Spilled Records","value":1},{"name":"FAILED_SHUFFLE","displayName":"Failed Shuffles","value":0},{"name":"MERGED_MAP_OUTPUTS","displayName":"Merged Map outputs","value":0},{"name":"GC_TIME_MILLIS","displayName":"GC time elapsed (ms)","value":796},{"name":"CPU_MILLISECONDS","displayName":"CPU time spent (ms)","value":380},{"name":"PHYSICAL_MEMORY_BYTES","displayName":"Physical memory (bytes) snapshot","value":188272640},{"name":"VIRTUAL_MEMORY_BYTES","displayName":"Virtual memory (bytes) snapshot","value":705773568},{"name":"COMMITTED_HEAP_BYTES","displayName":"Total committed heap usage (bytes)","value":181272576}]},{"name":"org.apache.hadoop.mapreduce.lib.input.FileInputFormatCounter","displayName":"File Input Format Counters ","counts":[{"name":"BYTES_READ","displayName":"Bytes Read","value":0}]}]},"clockSplits":[11518,6,6,7,6,6,6,7,6,6,6,7],"cpuUsages":[31,32,32,31,32,32,31,32,32,31,32,32],"vMemKbytes":[28718,86154,143590,201025,258462,315897,373334,430769,488206,545642,603077,660514],"physMemKbytes":[7660,22982,38304,53625,68947,84268,99590,114911,130234,145555,160876,176199]}}} + {"type":"TASK_FINISHED","event":{"org.apache.hadoop.mapreduce.jobhistory.TaskFinished":{"taskid":"task_1329348432655_0001_m_000003","taskType":"MAP","finishTime":1329348462178,"status":"SUCCEEDED","counters":{"name":"COUNTERS","groups":[{"name":"org.apache.hadoop.mapreduce.FileSystemCounter","displayName":"File System Counters","counts":[{"name":"FILE_BYTES_READ","displayName":"FILE: Number of bytes read","value":120},{"name":"FILE_BYTES_WRITTEN","displayName":"FILE: Number of bytes written","value":48051},{"name":"FILE_READ_OPS","displayName":"FILE: Number of read operations","value":0},{"name":"FILE_LARGE_READ_OPS","displayName":"FILE: Number of large read operations","value":0},{"name":"FILE_WRITE_OPS","displayName":"FILE: Number of write operations","value":0},{"name":"HDFS_BYTES_READ","displayName":"HDFS: Number of bytes read","value":48},{"name":"HDFS_BYTES_WRITTEN","displayName":"HDFS: Number of bytes written","value":0},{"name":"HDFS_READ_OPS","displayName":"HDFS: Number of read operations","value":1},{"name":"HDFS_LARGE_READ_OPS","displayName":"HDFS: Number of large read operations","value":0},{"name":"HDFS_WRITE_OPS","displayName":"HDFS: Number of write operations","value":0}]},{"name":"org.apache.hadoop.mapreduce.TaskCounter","displayName":"Map-Reduce Framework","counts":[{"name":"MAP_INPUT_RECORDS","displayName":"Map input records","value":1},{"name":"MAP_OUTPUT_RECORDS","displayName":"Map output records","value":1},{"name":"MAP_OUTPUT_BYTES","displayName":"Map output bytes","value":4},{"name":"MAP_OUTPUT_MATERIALIZED_BYTES","displayName":"Map output materialized bytes","value":12},{"name":"SPLIT_RAW_BYTES","displayName":"Input split bytes","value":48},{"name":"COMBINE_INPUT_RECORDS","displayName":"Combine input records","value":0},{"name":"SPILLED_RECORDS","displayName":"Spilled Records","value":1},{"name":"FAILED_SHUFFLE","displayName":"Failed Shuffles","value":0},{"name":"MERGED_MAP_OUTPUTS","displayName":"Merged Map outputs","value":0},{"name":"GC_TIME_MILLIS","displayName":"GC time elapsed (ms)","value":796},{"name":"CPU_MILLISECONDS","displayName":"CPU time spent (ms)","value":380},{"name":"PHYSICAL_MEMORY_BYTES","displayName":"Physical memory (bytes) snapshot","value":188272640},{"name":"VIRTUAL_MEMORY_BYTES","displayName":"Virtual memory (bytes) snapshot","value":705773568},{"name":"COMMITTED_HEAP_BYTES","displayName":"Total committed heap usage (bytes)","value":181272576}]},{"name":"org.apache.hadoop.mapreduce.lib.input.FileInputFormatCounter","displayName":"File Input Format Counters ","counts":[{"name":"BYTES_READ","displayName":"Bytes Read","value":0}]}]}}}} + {"type":"MAP_ATTEMPT_FINISHED","event":{"org.apache.hadoop.mapreduce.jobhistory.MapAttemptFinished":{"taskid":"task_1329348432655_0001_m_000000","attemptId":"attempt_1329348432655_0001_m_000000_0","taskType":"MAP","taskStatus":"SUCCEEDED","mapFinishTime":1329348462400,"finishTime":1329348462562,"hostname":"localhost","port":45454,"rackname":"/default-rack","state":"Sleeping... (1) ms left","counters":{"name":"COUNTERS","groups":[{"name":"org.apache.hadoop.mapreduce.FileSystemCounter","displayName":"File System Counters","counts":[{"name":"FILE_BYTES_READ","displayName":"FILE: Number of bytes read","value":120},{"name":"FILE_BYTES_WRITTEN","displayName":"FILE: Number of bytes written","value":48051},{"name":"FILE_READ_OPS","displayName":"FILE: Number of read operations","value":0},{"name":"FILE_LARGE_READ_OPS","displayName":"FILE: Number of large read operations","value":0},{"name":"FILE_WRITE_OPS","displayName":"FILE: Number of write operations","value":0},{"name":"HDFS_BYTES_READ","displayName":"HDFS: Number of bytes read","value":48},{"name":"HDFS_BYTES_WRITTEN","displayName":"HDFS: Number of bytes written","value":0},{"name":"HDFS_READ_OPS","displayName":"HDFS: Number of read operations","value":1},{"name":"HDFS_LARGE_READ_OPS","displayName":"HDFS: Number of large read operations","value":0},{"name":"HDFS_WRITE_OPS","displayName":"HDFS: Number of write operations","value":0}]},{"name":"org.apache.hadoop.mapreduce.TaskCounter","displayName":"Map-Reduce Framework","counts":[{"name":"MAP_INPUT_RECORDS","displayName":"Map input records","value":1},{"name":"MAP_OUTPUT_RECORDS","displayName":"Map output records","value":1},{"name":"MAP_OUTPUT_BYTES","displayName":"Map output bytes","value":4},{"name":"MAP_OUTPUT_MATERIALIZED_BYTES","displayName":"Map output materialized bytes","value":12},{"name":"SPLIT_RAW_BYTES","displayName":"Input split bytes","value":48},{"name":"COMBINE_INPUT_RECORDS","displayName":"Combine input records","value":0},{"name":"SPILLED_RECORDS","displayName":"Spilled Records","value":1},{"name":"FAILED_SHUFFLE","displayName":"Failed Shuffles","value":0},{"name":"MERGED_MAP_OUTPUTS","displayName":"Merged Map outputs","value":0},{"name":"GC_TIME_MILLIS","displayName":"GC time elapsed (ms)","value":11},{"name":"CPU_MILLISECONDS","displayName":"CPU time spent (ms)","value":320},{"name":"PHYSICAL_MEMORY_BYTES","displayName":"Physical memory (bytes) snapshot","value":181645312},{"name":"VIRTUAL_MEMORY_BYTES","displayName":"Virtual memory (bytes) snapshot","value":706129920},{"name":"COMMITTED_HEAP_BYTES","displayName":"Total committed heap usage (bytes)","value":165478400}]},{"name":"org.apache.hadoop.mapreduce.lib.input.FileInputFormatCounter","displayName":"File Input Format Counters ","counts":[{"name":"BYTES_READ","displayName":"Bytes Read","value":0}]}]},"clockSplits":[11930,13,13,13,13,13,13,13,13,13,13,14],"cpuUsages":[26,27,27,26,27,27,26,27,27,26,27,27],"vMemKbytes":[28732,86197,143662,201127,258592,316057,373522,430987,488452,545917,603382,660847],"physMemKbytes":[7391,22173,36955,51737,66520,81302,96085,110866,125649,140432,155213,169996]}}} + {"type":"TASK_FINISHED","event":{"org.apache.hadoop.mapreduce.jobhistory.TaskFinished":{"taskid":"task_1329348432655_0001_m_000000","taskType":"MAP","finishTime":1329348462562,"status":"SUCCEEDED","counters":{"name":"COUNTERS","groups":[{"name":"org.apache.hadoop.mapreduce.FileSystemCounter","displayName":"File System Counters","counts":[{"name":"FILE_BYTES_READ","displayName":"FILE: Number of bytes read","value":120},{"name":"FILE_BYTES_WRITTEN","displayName":"FILE: Number of bytes written","value":48051},{"name":"FILE_READ_OPS","displayName":"FILE: Number of read operations","value":0},{"name":"FILE_LARGE_READ_OPS","displayName":"FILE: Number of large read operations","value":0},{"name":"FILE_WRITE_OPS","displayName":"FILE: Number of write operations","value":0},{"name":"HDFS_BYTES_READ","displayName":"HDFS: Number of bytes read","value":48},{"name":"HDFS_BYTES_WRITTEN","displayName":"HDFS: Number of bytes written","value":0},{"name":"HDFS_READ_OPS","displayName":"HDFS: Number of read operations","value":1},{"name":"HDFS_LARGE_READ_OPS","displayName":"HDFS: Number of large read operations","value":0},{"name":"HDFS_WRITE_OPS","displayName":"HDFS: Number of write operations","value":0}]},{"name":"org.apache.hadoop.mapreduce.TaskCounter","displayName":"Map-Reduce Framework","counts":[{"name":"MAP_INPUT_RECORDS","displayName":"Map input records","value":1},{"name":"MAP_OUTPUT_RECORDS","displayName":"Map output records","value":1},{"name":"MAP_OUTPUT_BYTES","displayName":"Map output bytes","value":4},{"name":"MAP_OUTPUT_MATERIALIZED_BYTES","displayName":"Map output materialized bytes","value":12},{"name":"SPLIT_RAW_BYTES","displayName":"Input split bytes","value":48},{"name":"COMBINE_INPUT_RECORDS","displayName":"Combine input records","value":0},{"name":"SPILLED_RECORDS","displayName":"Spilled Records","value":1},{"name":"FAILED_SHUFFLE","displayName":"Failed Shuffles","value":0},{"name":"MERGED_MAP_OUTPUTS","displayName":"Merged Map outputs","value":0},{"name":"GC_TIME_MILLIS","displayName":"GC time elapsed (ms)","value":11},{"name":"CPU_MILLISECONDS","displayName":"CPU time spent (ms)","value":320},{"name":"PHYSICAL_MEMORY_BYTES","displayName":"Physical memory (bytes) snapshot","value":181645312},{"name":"VIRTUAL_MEMORY_BYTES","displayName":"Virtual memory (bytes) snapshot","value":706129920},{"name":"COMMITTED_HEAP_BYTES","displayName":"Total committed heap usage (bytes)","value":165478400}]},{"name":"org.apache.hadoop.mapreduce.lib.input.FileInputFormatCounter","displayName":"File Input Format Counters ","counts":[{"name":"BYTES_READ","displayName":"Bytes Read","value":0}]}]}}}} + {"type":"MAP_ATTEMPT_STARTED","event":{"org.apache.hadoop.mapreduce.jobhistory.TaskAttemptStarted":{"taskid":"task_1329348432655_0001_m_000008","taskType":"MAP","attemptId":"attempt_1329348432655_0001_m_000008_0","startTime":1329348462765,"trackerName":"localhost","httpPort":9999,"shufflePort":8080,"containerId":"container_1329348432655_0001_01_000010"}}} + {"type":"MAP_ATTEMPT_STARTED","event":{"org.apache.hadoop.mapreduce.jobhistory.TaskAttemptStarted":{"taskid":"task_1329348432655_0001_m_000009","taskType":"MAP","attemptId":"attempt_1329348432655_0001_m_000009_0","startTime":1329348462792,"trackerName":"localhost","httpPort":9999,"shufflePort":8080,"containerId":"container_1329348432655_0001_01_000011"}}} + {"type":"REDUCE_ATTEMPT_STARTED","event":{"org.apache.hadoop.mapreduce.jobhistory.TaskAttemptStarted":{"taskid":"task_1329348432655_0001_r_000000","taskType":"REDUCE","attemptId":"attempt_1329348432655_0001_r_000000_0","startTime":1329348464995,"trackerName":"localhost","httpPort":9999,"shufflePort":8080,"containerId":"container_1329348432655_0001_01_000014"}}} + {"type":"MAP_ATTEMPT_FINISHED","event":{"org.apache.hadoop.mapreduce.jobhistory.MapAttemptFinished":{"taskid":"task_1329348432655_0001_m_000007","attemptId":"attempt_1329348432655_0001_m_000007_0","taskType":"MAP","taskStatus":"SUCCEEDED","mapFinishTime":1329348465534,"finishTime":1329348465965,"hostname":"localhost","port":45454,"rackname":"/default-rack","state":"Sleeping... (1) ms left","counters":{"name":"COUNTERS","groups":[{"name":"org.apache.hadoop.mapreduce.FileSystemCounter","displayName":"File System Counters","counts":[{"name":"FILE_BYTES_READ","displayName":"FILE: Number of bytes read","value":120},{"name":"FILE_BYTES_WRITTEN","displayName":"FILE: Number of bytes written","value":48051},{"name":"FILE_READ_OPS","displayName":"FILE: Number of read operations","value":0},{"name":"FILE_LARGE_READ_OPS","displayName":"FILE: Number of large read operations","value":0},{"name":"FILE_WRITE_OPS","displayName":"FILE: Number of write operations","value":0},{"name":"HDFS_BYTES_READ","displayName":"HDFS: Number of bytes read","value":48},{"name":"HDFS_BYTES_WRITTEN","displayName":"HDFS: Number of bytes written","value":0},{"name":"HDFS_READ_OPS","displayName":"HDFS: Number of read operations","value":1},{"name":"HDFS_LARGE_READ_OPS","displayName":"HDFS: Number of large read operations","value":0},{"name":"HDFS_WRITE_OPS","displayName":"HDFS: Number of write operations","value":0}]},{"name":"org.apache.hadoop.mapreduce.TaskCounter","displayName":"Map-Reduce Framework","counts":[{"name":"MAP_INPUT_RECORDS","displayName":"Map input records","value":1},{"name":"MAP_OUTPUT_RECORDS","displayName":"Map output records","value":1},{"name":"MAP_OUTPUT_BYTES","displayName":"Map output bytes","value":4},{"name":"MAP_OUTPUT_MATERIALIZED_BYTES","displayName":"Map output materialized bytes","value":12},{"name":"SPLIT_RAW_BYTES","displayName":"Input split bytes","value":48},{"name":"COMBINE_INPUT_RECORDS","displayName":"Combine input records","value":0},{"name":"SPILLED_RECORDS","displayName":"Spilled Records","value":1},{"name":"FAILED_SHUFFLE","displayName":"Failed Shuffles","value":0},{"name":"MERGED_MAP_OUTPUTS","displayName":"Merged Map outputs","value":0},{"name":"GC_TIME_MILLIS","displayName":"GC time elapsed (ms)","value":194},{"name":"CPU_MILLISECONDS","displayName":"CPU time spent (ms)","value":320},{"name":"PHYSICAL_MEMORY_BYTES","displayName":"Physical memory (bytes) snapshot","value":185327616},{"name":"VIRTUAL_MEMORY_BYTES","displayName":"Virtual memory (bytes) snapshot","value":713089024},{"name":"COMMITTED_HEAP_BYTES","displayName":"Total committed heap usage (bytes)","value":165478400}]},{"name":"org.apache.hadoop.mapreduce.lib.input.FileInputFormatCounter","displayName":"File Input Format Counters ","counts":[{"name":"BYTES_READ","displayName":"Bytes Read","value":0}]}]},"clockSplits":[3464,18,19,18,18,19,18,18,19,18,18,19],"cpuUsages":[26,27,27,26,27,27,26,27,27,26,27,27],"vMemKbytes":[29015,87046,145078,203109,261140,319171,377203,435234,493266,551297,609328,667360],"physMemKbytes":[7541,22623,37705,52786,67869,82950,98033,113114,128197,143279,158360,173443]}}} + {"type":"TASK_FINISHED","event":{"org.apache.hadoop.mapreduce.jobhistory.TaskFinished":{"taskid":"task_1329348432655_0001_m_000007","taskType":"MAP","finishTime":1329348465965,"status":"SUCCEEDED","counters":{"name":"COUNTERS","groups":[{"name":"org.apache.hadoop.mapreduce.FileSystemCounter","displayName":"File System Counters","counts":[{"name":"FILE_BYTES_READ","displayName":"FILE: Number of bytes read","value":120},{"name":"FILE_BYTES_WRITTEN","displayName":"FILE: Number of bytes written","value":48051},{"name":"FILE_READ_OPS","displayName":"FILE: Number of read operations","value":0},{"name":"FILE_LARGE_READ_OPS","displayName":"FILE: Number of large read operations","value":0},{"name":"FILE_WRITE_OPS","displayName":"FILE: Number of write operations","value":0},{"name":"HDFS_BYTES_READ","displayName":"HDFS: Number of bytes read","value":48},{"name":"HDFS_BYTES_WRITTEN","displayName":"HDFS: Number of bytes written","value":0},{"name":"HDFS_READ_OPS","displayName":"HDFS: Number of read operations","value":1},{"name":"HDFS_LARGE_READ_OPS","displayName":"HDFS: Number of large read operations","value":0},{"name":"HDFS_WRITE_OPS","displayName":"HDFS: Number of write operations","value":0}]},{"name":"org.apache.hadoop.mapreduce.TaskCounter","displayName":"Map-Reduce Framework","counts":[{"name":"MAP_INPUT_RECORDS","displayName":"Map input records","value":1},{"name":"MAP_OUTPUT_RECORDS","displayName":"Map output records","value":1},{"name":"MAP_OUTPUT_BYTES","displayName":"Map output bytes","value":4},{"name":"MAP_OUTPUT_MATERIALIZED_BYTES","displayName":"Map output materialized bytes","value":12},{"name":"SPLIT_RAW_BYTES","displayName":"Input split bytes","value":48},{"name":"COMBINE_INPUT_RECORDS","displayName":"Combine input records","value":0},{"name":"SPILLED_RECORDS","displayName":"Spilled Records","value":1},{"name":"FAILED_SHUFFLE","displayName":"Failed Shuffles","value":0},{"name":"MERGED_MAP_OUTPUTS","displayName":"Merged Map outputs","value":0},{"name":"GC_TIME_MILLIS","displayName":"GC time elapsed (ms)","value":194},{"name":"CPU_MILLISECONDS","displayName":"CPU time spent (ms)","value":320},{"name":"PHYSICAL_MEMORY_BYTES","displayName":"Physical memory (bytes) snapshot","value":185327616},{"name":"VIRTUAL_MEMORY_BYTES","displayName":"Virtual memory (bytes) snapshot","value":713089024},{"name":"COMMITTED_HEAP_BYTES","displayName":"Total committed heap usage (bytes)","value":165478400}]},{"name":"org.apache.hadoop.mapreduce.lib.input.FileInputFormatCounter","displayName":"File Input Format Counters ","counts":[{"name":"BYTES_READ","displayName":"Bytes Read","value":0}]}]}}}} + {"type":"MAP_ATTEMPT_FINISHED","event":{"org.apache.hadoop.mapreduce.jobhistory.MapAttemptFinished":{"taskid":"task_1329348432655_0001_m_000009","attemptId":"attempt_1329348432655_0001_m_000009_0","taskType":"MAP","taskStatus":"SUCCEEDED","mapFinishTime":1329348465986,"finishTime":1329348466363,"hostname":"localhost","port":45454,"rackname":"/default-rack","state":"Sleeping... (1) ms left","counters":{"name":"COUNTERS","groups":[{"name":"org.apache.hadoop.mapreduce.FileSystemCounter","displayName":"File System Counters","counts":[{"name":"FILE_BYTES_READ","displayName":"FILE: Number of bytes read","value":120},{"name":"FILE_BYTES_WRITTEN","displayName":"FILE: Number of bytes written","value":48051},{"name":"FILE_READ_OPS","displayName":"FILE: Number of read operations","value":0},{"name":"FILE_LARGE_READ_OPS","displayName":"FILE: Number of large read operations","value":0},{"name":"FILE_WRITE_OPS","displayName":"FILE: Number of write operations","value":0},{"name":"HDFS_BYTES_READ","displayName":"HDFS: Number of bytes read","value":48},{"name":"HDFS_BYTES_WRITTEN","displayName":"HDFS: Number of bytes written","value":0},{"name":"HDFS_READ_OPS","displayName":"HDFS: Number of read operations","value":1},{"name":"HDFS_LARGE_READ_OPS","displayName":"HDFS: Number of large read operations","value":0},{"name":"HDFS_WRITE_OPS","displayName":"HDFS: Number of write operations","value":0}]},{"name":"org.apache.hadoop.mapreduce.TaskCounter","displayName":"Map-Reduce Framework","counts":[{"name":"MAP_INPUT_RECORDS","displayName":"Map input records","value":1},{"name":"MAP_OUTPUT_RECORDS","displayName":"Map output records","value":1},{"name":"MAP_OUTPUT_BYTES","displayName":"Map output bytes","value":4},{"name":"MAP_OUTPUT_MATERIALIZED_BYTES","displayName":"Map output materialized bytes","value":12},{"name":"SPLIT_RAW_BYTES","displayName":"Input split bytes","value":48},{"name":"COMBINE_INPUT_RECORDS","displayName":"Combine input records","value":0},{"name":"SPILLED_RECORDS","displayName":"Spilled Records","value":1},{"name":"FAILED_SHUFFLE","displayName":"Failed Shuffles","value":0},{"name":"MERGED_MAP_OUTPUTS","displayName":"Merged Map outputs","value":0},{"name":"GC_TIME_MILLIS","displayName":"GC time elapsed (ms)","value":23},{"name":"CPU_MILLISECONDS","displayName":"CPU time spent (ms)","value":330},{"name":"PHYSICAL_MEMORY_BYTES","displayName":"Physical memory (bytes) snapshot","value":182169600},{"name":"VIRTUAL_MEMORY_BYTES","displayName":"Virtual memory (bytes) snapshot","value":705945600},{"name":"COMMITTED_HEAP_BYTES","displayName":"Total committed heap usage (bytes)","value":165478400}]},{"name":"org.apache.hadoop.mapreduce.lib.input.FileInputFormatCounter","displayName":"File Input Format Counters ","counts":[{"name":"BYTES_READ","displayName":"Bytes Read","value":0}]}]},"clockSplits":[3223,21,21,21,21,21,20,21,21,21,21,21],"cpuUsages":[27,28,27,28,27,28,27,28,27,28,27,28],"vMemKbytes":[28725,86175,143625,201074,258525,315974,373425,430874,488325,545775,603224,660675],"physMemKbytes":[7412,22237,37062,51887,66712,81537,96362,111187,126012,140837,155662,170487]}}} + {"type":"TASK_FINISHED","event":{"org.apache.hadoop.mapreduce.jobhistory.TaskFinished":{"taskid":"task_1329348432655_0001_m_000009","taskType":"MAP","finishTime":1329348466363,"status":"SUCCEEDED","counters":{"name":"COUNTERS","groups":[{"name":"org.apache.hadoop.mapreduce.FileSystemCounter","displayName":"File System Counters","counts":[{"name":"FILE_BYTES_READ","displayName":"FILE: Number of bytes read","value":120},{"name":"FILE_BYTES_WRITTEN","displayName":"FILE: Number of bytes written","value":48051},{"name":"FILE_READ_OPS","displayName":"FILE: Number of read operations","value":0},{"name":"FILE_LARGE_READ_OPS","displayName":"FILE: Number of large read operations","value":0},{"name":"FILE_WRITE_OPS","displayName":"FILE: Number of write operations","value":0},{"name":"HDFS_BYTES_READ","displayName":"HDFS: Number of bytes read","value":48},{"name":"HDFS_BYTES_WRITTEN","displayName":"HDFS: Number of bytes written","value":0},{"name":"HDFS_READ_OPS","displayName":"HDFS: Number of read operations","value":1},{"name":"HDFS_LARGE_READ_OPS","displayName":"HDFS: Number of large read operations","value":0},{"name":"HDFS_WRITE_OPS","displayName":"HDFS: Number of write operations","value":0}]},{"name":"org.apache.hadoop.mapreduce.TaskCounter","displayName":"Map-Reduce Framework","counts":[{"name":"MAP_INPUT_RECORDS","displayName":"Map input records","value":1},{"name":"MAP_OUTPUT_RECORDS","displayName":"Map output records","value":1},{"name":"MAP_OUTPUT_BYTES","displayName":"Map output bytes","value":4},{"name":"MAP_OUTPUT_MATERIALIZED_BYTES","displayName":"Map output materialized bytes","value":12},{"name":"SPLIT_RAW_BYTES","displayName":"Input split bytes","value":48},{"name":"COMBINE_INPUT_RECORDS","displayName":"Combine input records","value":0},{"name":"SPILLED_RECORDS","displayName":"Spilled Records","value":1},{"name":"FAILED_SHUFFLE","displayName":"Failed Shuffles","value":0},{"name":"MERGED_MAP_OUTPUTS","displayName":"Merged Map outputs","value":0},{"name":"GC_TIME_MILLIS","displayName":"GC time elapsed (ms)","value":23},{"name":"CPU_MILLISECONDS","displayName":"CPU time spent (ms)","value":330},{"name":"PHYSICAL_MEMORY_BYTES","displayName":"Physical memory (bytes) snapshot","value":182169600},{"name":"VIRTUAL_MEMORY_BYTES","displayName":"Virtual memory (bytes) snapshot","value":705945600},{"name":"COMMITTED_HEAP_BYTES","displayName":"Total committed heap usage (bytes)","value":165478400}]},{"name":"org.apache.hadoop.mapreduce.lib.input.FileInputFormatCounter","displayName":"File Input Format Counters ","counts":[{"name":"BYTES_READ","displayName":"Bytes Read","value":0}]}]}}}} + {"type":"MAP_ATTEMPT_FINISHED","event":{"org.apache.hadoop.mapreduce.jobhistory.MapAttemptFinished":{"taskid":"task_1329348432655_0001_m_000008","attemptId":"attempt_1329348432655_0001_m_000008_0","taskType":"MAP","taskStatus":"SUCCEEDED","mapFinishTime":1329348467231,"finishTime":1329348467421,"hostname":"localhost","port":45454,"rackname":"/default-rack","state":"Sleeping... (1) ms left","counters":{"name":"COUNTERS","groups":[{"name":"org.apache.hadoop.mapreduce.FileSystemCounter","displayName":"File System Counters","counts":[{"name":"FILE_BYTES_READ","displayName":"FILE: Number of bytes read","value":120},{"name":"FILE_BYTES_WRITTEN","displayName":"FILE: Number of bytes written","value":48051},{"name":"FILE_READ_OPS","displayName":"FILE: Number of read operations","value":0},{"name":"FILE_LARGE_READ_OPS","displayName":"FILE: Number of large read operations","value":0},{"name":"FILE_WRITE_OPS","displayName":"FILE: Number of write operations","value":0},{"name":"HDFS_BYTES_READ","displayName":"HDFS: Number of bytes read","value":48},{"name":"HDFS_BYTES_WRITTEN","displayName":"HDFS: Number of bytes written","value":0},{"name":"HDFS_READ_OPS","displayName":"HDFS: Number of read operations","value":1},{"name":"HDFS_LARGE_READ_OPS","displayName":"HDFS: Number of large read operations","value":0},{"name":"HDFS_WRITE_OPS","displayName":"HDFS: Number of write operations","value":0}]},{"name":"org.apache.hadoop.mapreduce.TaskCounter","displayName":"Map-Reduce Framework","counts":[{"name":"MAP_INPUT_RECORDS","displayName":"Map input records","value":1},{"name":"MAP_OUTPUT_RECORDS","displayName":"Map output records","value":1},{"name":"MAP_OUTPUT_BYTES","displayName":"Map output bytes","value":4},{"name":"MAP_OUTPUT_MATERIALIZED_BYTES","displayName":"Map output materialized bytes","value":12},{"name":"SPLIT_RAW_BYTES","displayName":"Input split bytes","value":48},{"name":"COMBINE_INPUT_RECORDS","displayName":"Combine input records","value":0},{"name":"SPILLED_RECORDS","displayName":"Spilled Records","value":1},{"name":"FAILED_SHUFFLE","displayName":"Failed Shuffles","value":0},{"name":"MERGED_MAP_OUTPUTS","displayName":"Merged Map outputs","value":0},{"name":"GC_TIME_MILLIS","displayName":"GC time elapsed (ms)","value":12},{"name":"CPU_MILLISECONDS","displayName":"CPU time spent (ms)","value":320},{"name":"PHYSICAL_MEMORY_BYTES","displayName":"Physical memory (bytes) snapshot","value":181297152},{"name":"VIRTUAL_MEMORY_BYTES","displayName":"Virtual memory (bytes) snapshot","value":705019904},{"name":"COMMITTED_HEAP_BYTES","displayName":"Total committed heap usage (bytes)","value":165478400}]},{"name":"org.apache.hadoop.mapreduce.lib.input.FileInputFormatCounter","displayName":"File Input Format Counters ","counts":[{"name":"BYTES_READ","displayName":"Bytes Read","value":0}]}]},"clockSplits":[4483,15,16,15,16,15,15,16,15,16,15,16],"cpuUsages":[26,27,27,26,27,27,26,27,27,26,27,27],"vMemKbytes":[28686,86061,143436,200810,258185,315560,372935,430309,487684,545059,602433,659808],"physMemKbytes":[7377,22131,36885,51638,66393,81146,95901,110654,125409,140163,154916,169671]}}} + {"type":"TASK_FINISHED","event":{"org.apache.hadoop.mapreduce.jobhistory.TaskFinished":{"taskid":"task_1329348432655_0001_m_000008","taskType":"MAP","finishTime":1329348467421,"status":"SUCCEEDED","counters":{"name":"COUNTERS","groups":[{"name":"org.apache.hadoop.mapreduce.FileSystemCounter","displayName":"File System Counters","counts":[{"name":"FILE_BYTES_READ","displayName":"FILE: Number of bytes read","value":120},{"name":"FILE_BYTES_WRITTEN","displayName":"FILE: Number of bytes written","value":48051},{"name":"FILE_READ_OPS","displayName":"FILE: Number of read operations","value":0},{"name":"FILE_LARGE_READ_OPS","displayName":"FILE: Number of large read operations","value":0},{"name":"FILE_WRITE_OPS","displayName":"FILE: Number of write operations","value":0},{"name":"HDFS_BYTES_READ","displayName":"HDFS: Number of bytes read","value":48},{"name":"HDFS_BYTES_WRITTEN","displayName":"HDFS: Number of bytes written","value":0},{"name":"HDFS_READ_OPS","displayName":"HDFS: Number of read operations","value":1},{"name":"HDFS_LARGE_READ_OPS","displayName":"HDFS: Number of large read operations","value":0},{"name":"HDFS_WRITE_OPS","displayName":"HDFS: Number of write operations","value":0}]},{"name":"org.apache.hadoop.mapreduce.TaskCounter","displayName":"Map-Reduce Framework","counts":[{"name":"MAP_INPUT_RECORDS","displayName":"Map input records","value":1},{"name":"MAP_OUTPUT_RECORDS","displayName":"Map output records","value":1},{"name":"MAP_OUTPUT_BYTES","displayName":"Map output bytes","value":4},{"name":"MAP_OUTPUT_MATERIALIZED_BYTES","displayName":"Map output materialized bytes","value":12},{"name":"SPLIT_RAW_BYTES","displayName":"Input split bytes","value":48},{"name":"COMBINE_INPUT_RECORDS","displayName":"Combine input records","value":0},{"name":"SPILLED_RECORDS","displayName":"Spilled Records","value":1},{"name":"FAILED_SHUFFLE","displayName":"Failed Shuffles","value":0},{"name":"MERGED_MAP_OUTPUTS","displayName":"Merged Map outputs","value":0},{"name":"GC_TIME_MILLIS","displayName":"GC time elapsed (ms)","value":12},{"name":"CPU_MILLISECONDS","displayName":"CPU time spent (ms)","value":320},{"name":"PHYSICAL_MEMORY_BYTES","displayName":"Physical memory (bytes) snapshot","value":181297152},{"name":"VIRTUAL_MEMORY_BYTES","displayName":"Virtual memory (bytes) snapshot","value":705019904},{"name":"COMMITTED_HEAP_BYTES","displayName":"Total committed heap usage (bytes)","value":165478400}]},{"name":"org.apache.hadoop.mapreduce.lib.input.FileInputFormatCounter","displayName":"File Input Format Counters ","counts":[{"name":"BYTES_READ","displayName":"Bytes Read","value":0}]}]}}}} + {"type":"REDUCE_ATTEMPT_FINISHED","event":{"org.apache.hadoop.mapreduce.jobhistory.ReduceAttemptFinished":{"taskid":"task_1329348432655_0001_r_000000","attemptId":"attempt_1329348432655_0001_r_000000_0","taskType":"REDUCE","taskStatus":"SUCCEEDED","shuffleFinishTime":1329348468462,"sortFinishTime":1329348468517,"finishTime":1329348468600,"hostname":"localhost","port":45454,"rackname":"/default-rack","state":"Sleeping... (1) ms left > reduce","counters":{"name":"COUNTERS","groups":[{"name":"org.apache.hadoop.mapreduce.FileSystemCounter","displayName":"File System Counters","counts":[{"name":"FILE_BYTES_READ","displayName":"FILE: Number of bytes read","value":186},{"name":"FILE_BYTES_WRITTEN","displayName":"FILE: Number of bytes written","value":48074},{"name":"FILE_READ_OPS","displayName":"FILE: Number of read operations","value":0},{"name":"FILE_LARGE_READ_OPS","displayName":"FILE: Number of large read operations","value":0},{"name":"FILE_WRITE_OPS","displayName":"FILE: Number of write operations","value":0},{"name":"HDFS_BYTES_READ","displayName":"HDFS: Number of bytes read","value":0},{"name":"HDFS_BYTES_WRITTEN","displayName":"HDFS: Number of bytes written","value":0},{"name":"HDFS_READ_OPS","displayName":"HDFS: Number of read operations","value":0},{"name":"HDFS_LARGE_READ_OPS","displayName":"HDFS: Number of large read operations","value":0},{"name":"HDFS_WRITE_OPS","displayName":"HDFS: Number of write operations","value":0}]},{"name":"org.apache.hadoop.mapreduce.TaskCounter","displayName":"Map-Reduce Framework","counts":[{"name":"COMBINE_INPUT_RECORDS","displayName":"Combine input records","value":0},{"name":"COMBINE_OUTPUT_RECORDS","displayName":"Combine output records","value":0},{"name":"REDUCE_INPUT_GROUPS","displayName":"Reduce input groups","value":1},{"name":"REDUCE_SHUFFLE_BYTES","displayName":"Reduce shuffle bytes","value":120},{"name":"REDUCE_INPUT_RECORDS","displayName":"Reduce input records","value":10},{"name":"REDUCE_OUTPUT_RECORDS","displayName":"Reduce output records","value":0},{"name":"SPILLED_RECORDS","displayName":"Spilled Records","value":10},{"name":"SHUFFLED_MAPS","displayName":"Shuffled Maps ","value":10},{"name":"FAILED_SHUFFLE","displayName":"Failed Shuffles","value":0},{"name":"MERGED_MAP_OUTPUTS","displayName":"Merged Map outputs","value":10},{"name":"GC_TIME_MILLIS","displayName":"GC time elapsed (ms)","value":14},{"name":"CPU_MILLISECONDS","displayName":"CPU time spent (ms)","value":1070},{"name":"PHYSICAL_MEMORY_BYTES","displayName":"Physical memory (bytes) snapshot","value":82780160},{"name":"VIRTUAL_MEMORY_BYTES","displayName":"Virtual memory (bytes) snapshot","value":714436608},{"name":"COMMITTED_HEAP_BYTES","displayName":"Total committed heap usage (bytes)","value":60555264}]},{"name":"Shuffle Errors","displayName":"Shuffle Errors","counts":[{"name":"BAD_ID","displayName":"BAD_ID","value":0},{"name":"CONNECTION","displayName":"CONNECTION","value":0},{"name":"IO_ERROR","displayName":"IO_ERROR","value":0},{"name":"WRONG_LENGTH","displayName":"WRONG_LENGTH","value":0},{"name":"WRONG_MAP","displayName":"WRONG_MAP","value":0},{"name":"WRONG_REDUCE","displayName":"WRONG_REDUCE","value":0}]},{"name":"org.apache.hadoop.mapreduce.lib.output.FileOutputFormatCounter","displayName":"File Output Format Counters ","counts":[{"name":"BYTES_WRITTEN","displayName":"Bytes Written","value":0}]}]},"clockSplits":[3530,6,7,6,7,6,6,7,6,7,6,7],"cpuUsages":[89,89,89,89,89,90,89,89,89,89,89,90],"vMemKbytes":[29070,87211,145352,203493,261634,319775,377916,436057,494198,552339,610480,668621],"physMemKbytes":[3367,10104,16841,23577,30314,37051,43788,50524,57261,63998,70734,77471]}}} + {"type":"TASK_FINISHED","event":{"org.apache.hadoop.mapreduce.jobhistory.TaskFinished":{"taskid":"task_1329348432655_0001_r_000000","taskType":"REDUCE","finishTime":1329348468600,"status":"SUCCEEDED","counters":{"name":"COUNTERS","groups":[{"name":"org.apache.hadoop.mapreduce.FileSystemCounter","displayName":"File System Counters","counts":[{"name":"FILE_BYTES_READ","displayName":"FILE: Number of bytes read","value":186},{"name":"FILE_BYTES_WRITTEN","displayName":"FILE: Number of bytes written","value":48074},{"name":"FILE_READ_OPS","displayName":"FILE: Number of read operations","value":0},{"name":"FILE_LARGE_READ_OPS","displayName":"FILE: Number of large read operations","value":0},{"name":"FILE_WRITE_OPS","displayName":"FILE: Number of write operations","value":0},{"name":"HDFS_BYTES_READ","displayName":"HDFS: Number of bytes read","value":0},{"name":"HDFS_BYTES_WRITTEN","displayName":"HDFS: Number of bytes written","value":0},{"name":"HDFS_READ_OPS","displayName":"HDFS: Number of read operations","value":0},{"name":"HDFS_LARGE_READ_OPS","displayName":"HDFS: Number of large read operations","value":0},{"name":"HDFS_WRITE_OPS","displayName":"HDFS: Number of write operations","value":0}]},{"name":"org.apache.hadoop.mapreduce.TaskCounter","displayName":"Map-Reduce Framework","counts":[{"name":"COMBINE_INPUT_RECORDS","displayName":"Combine input records","value":0},{"name":"COMBINE_OUTPUT_RECORDS","displayName":"Combine output records","value":0},{"name":"REDUCE_INPUT_GROUPS","displayName":"Reduce input groups","value":1},{"name":"REDUCE_SHUFFLE_BYTES","displayName":"Reduce shuffle bytes","value":120},{"name":"REDUCE_INPUT_RECORDS","displayName":"Reduce input records","value":10},{"name":"REDUCE_OUTPUT_RECORDS","displayName":"Reduce output records","value":0},{"name":"SPILLED_RECORDS","displayName":"Spilled Records","value":10},{"name":"SHUFFLED_MAPS","displayName":"Shuffled Maps ","value":10},{"name":"FAILED_SHUFFLE","displayName":"Failed Shuffles","value":0},{"name":"MERGED_MAP_OUTPUTS","displayName":"Merged Map outputs","value":10},{"name":"GC_TIME_MILLIS","displayName":"GC time elapsed (ms)","value":14},{"name":"CPU_MILLISECONDS","displayName":"CPU time spent (ms)","value":1070},{"name":"PHYSICAL_MEMORY_BYTES","displayName":"Physical memory (bytes) snapshot","value":82780160},{"name":"VIRTUAL_MEMORY_BYTES","displayName":"Virtual memory (bytes) snapshot","value":714436608},{"name":"COMMITTED_HEAP_BYTES","displayName":"Total committed heap usage (bytes)","value":60555264}]},{"name":"Shuffle Errors","displayName":"Shuffle Errors","counts":[{"name":"BAD_ID","displayName":"BAD_ID","value":0},{"name":"CONNECTION","displayName":"CONNECTION","value":0},{"name":"IO_ERROR","displayName":"IO_ERROR","value":0},{"name":"WRONG_LENGTH","displayName":"WRONG_LENGTH","value":0},{"name":"WRONG_MAP","displayName":"WRONG_MAP","value":0},{"name":"WRONG_REDUCE","displayName":"WRONG_REDUCE","value":0}]},{"name":"org.apache.hadoop.mapreduce.lib.output.FileOutputFormatCounter","displayName":"File Output Format Counters ","counts":[{"name":"BYTES_WRITTEN","displayName":"Bytes Written","value":0}]}]}}}} + {"type":"JOB_FINISHED","event":{"org.apache.hadoop.mapreduce.jobhistory.JobFinished":{"jobid":"job_1329348432655_0001","finishTime":1329348468601,"finishedMaps":10,"finishedReduces":1,"failedMaps":0,"failedReduces":0,"totalCounters":{"name":"TOTAL_COUNTERS","groups":[{"name":"Shuffle Errors","displayName":"Shuffle Errors","counts":[{"name":"BAD_ID","displayName":"BAD_ID","value":0},{"name":"CONNECTION","displayName":"CONNECTION","value":0},{"name":"IO_ERROR","displayName":"IO_ERROR","value":0},{"name":"WRONG_LENGTH","displayName":"WRONG_LENGTH","value":0},{"name":"WRONG_MAP","displayName":"WRONG_MAP","value":0},{"name":"WRONG_REDUCE","displayName":"WRONG_REDUCE","value":0}]},{"name":"org.apache.hadoop.mapreduce.FileSystemCounter","displayName":"File System Counters","counts":[{"name":"FILE_BYTES_READ","displayName":"FILE: Number of bytes read","value":1386},{"name":"FILE_BYTES_WRITTEN","displayName":"FILE: Number of bytes written","value":528584},{"name":"FILE_READ_OPS","displayName":"FILE: Number of read operations","value":0},{"name":"FILE_LARGE_READ_OPS","displayName":"FILE: Number of large read operations","value":0},{"name":"FILE_WRITE_OPS","displayName":"FILE: Number of write operations","value":0},{"name":"HDFS_BYTES_READ","displayName":"HDFS: Number of bytes read","value":480},{"name":"HDFS_BYTES_WRITTEN","displayName":"HDFS: Number of bytes written","value":0},{"name":"HDFS_READ_OPS","displayName":"HDFS: Number of read operations","value":10},{"name":"HDFS_LARGE_READ_OPS","displayName":"HDFS: Number of large read operations","value":0},{"name":"HDFS_WRITE_OPS","displayName":"HDFS: Number of write operations","value":0}]},{"name":"org.apache.hadoop.mapreduce.JobCounter","displayName":"Job Counters ","counts":[{"name":"TOTAL_LAUNCHED_MAPS","displayName":"Launched map tasks","value":10},{"name":"TOTAL_LAUNCHED_REDUCES","displayName":"Launched reduce tasks","value":1},{"name":"OTHER_LOCAL_MAPS","displayName":"Other local map tasks","value":10},{"name":"SLOTS_MILLIS_MAPS","displayName":"Total time spent by all maps in occupied slots (ms)","value":0},{"name":"SLOTS_MILLIS_REDUCES","displayName":"Total time spent by all reduces in occupied slots (ms)","value":0}]},{"name":"org.apache.hadoop.mapreduce.TaskCounter","displayName":"Map-Reduce Framework","counts":[{"name":"MAP_INPUT_RECORDS","displayName":"Map input records","value":10},{"name":"MAP_OUTPUT_RECORDS","displayName":"Map output records","value":10},{"name":"MAP_OUTPUT_BYTES","displayName":"Map output bytes","value":40},{"name":"MAP_OUTPUT_MATERIALIZED_BYTES","displayName":"Map output materialized bytes","value":120},{"name":"SPLIT_RAW_BYTES","displayName":"Input split bytes","value":480},{"name":"COMBINE_INPUT_RECORDS","displayName":"Combine input records","value":0},{"name":"COMBINE_OUTPUT_RECORDS","displayName":"Combine output records","value":0},{"name":"REDUCE_INPUT_GROUPS","displayName":"Reduce input groups","value":1},{"name":"REDUCE_SHUFFLE_BYTES","displayName":"Reduce shuffle bytes","value":120},{"name":"REDUCE_INPUT_RECORDS","displayName":"Reduce input records","value":10},{"name":"REDUCE_OUTPUT_RECORDS","displayName":"Reduce output records","value":0},{"name":"SPILLED_RECORDS","displayName":"Spilled Records","value":20},{"name":"SHUFFLED_MAPS","displayName":"Shuffled Maps ","value":10},{"name":"FAILED_SHUFFLE","displayName":"Failed Shuffles","value":0},{"name":"MERGED_MAP_OUTPUTS","displayName":"Merged Map outputs","value":10},{"name":"GC_TIME_MILLIS","displayName":"GC time elapsed (ms)","value":2256},{"name":"CPU_MILLISECONDS","displayName":"CPU time spent (ms)","value":4460},{"name":"PHYSICAL_MEMORY_BYTES","displayName":"Physical memory (bytes) snapshot","value":1923493888},{"name":"VIRTUAL_MEMORY_BYTES","displayName":"Virtual memory (bytes) snapshot","value":7773462528},{"name":"COMMITTED_HEAP_BYTES","displayName":"Total committed heap usage (bytes)","value":1778515968}]},{"name":"org.apache.hadoop.mapreduce.lib.input.FileInputFormatCounter","displayName":"File Input Format Counters ","counts":[{"name":"BYTES_READ","displayName":"Bytes Read","value":0}]},{"name":"org.apache.hadoop.mapreduce.lib.output.FileOutputFormatCounter","displayName":"File Output Format Counters ","counts":[{"name":"BYTES_WRITTEN","displayName":"Bytes Written","value":0}]}]},"mapCounters":{"name":"MAP_COUNTERS","groups":[{"name":"org.apache.hadoop.mapreduce.FileSystemCounter","displayName":"File System Counters","counts":[{"name":"FILE_BYTES_READ","displayName":"FILE: Number of bytes read","value":1200},{"name":"FILE_BYTES_WRITTEN","displayName":"FILE: Number of bytes written","value":480510},{"name":"FILE_READ_OPS","displayName":"FILE: Number of read operations","value":0},{"name":"FILE_LARGE_READ_OPS","displayName":"FILE: Number of large read operations","value":0},{"name":"FILE_WRITE_OPS","displayName":"FILE: Number of write operations","value":0},{"name":"HDFS_BYTES_READ","displayName":"HDFS: Number of bytes read","value":480},{"name":"HDFS_BYTES_WRITTEN","displayName":"HDFS: Number of bytes written","value":0},{"name":"HDFS_READ_OPS","displayName":"HDFS: Number of read operations","value":10},{"name":"HDFS_LARGE_READ_OPS","displayName":"HDFS: Number of large read operations","value":0},{"name":"HDFS_WRITE_OPS","displayName":"HDFS: Number of write operations","value":0}]},{"name":"org.apache.hadoop.mapreduce.TaskCounter","displayName":"Map-Reduce Framework","counts":[{"name":"MAP_INPUT_RECORDS","displayName":"Map input records","value":10},{"name":"MAP_OUTPUT_RECORDS","displayName":"Map output records","value":10},{"name":"MAP_OUTPUT_BYTES","displayName":"Map output bytes","value":40},{"name":"MAP_OUTPUT_MATERIALIZED_BYTES","displayName":"Map output materialized bytes","value":120},{"name":"SPLIT_RAW_BYTES","displayName":"Input split bytes","value":480},{"name":"COMBINE_INPUT_RECORDS","displayName":"Combine input records","value":0},{"name":"SPILLED_RECORDS","displayName":"Spilled Records","value":10},{"name":"FAILED_SHUFFLE","displayName":"Failed Shuffles","value":0},{"name":"MERGED_MAP_OUTPUTS","displayName":"Merged Map outputs","value":0},{"name":"GC_TIME_MILLIS","displayName":"GC time elapsed (ms)","value":2242},{"name":"CPU_MILLISECONDS","displayName":"CPU time spent (ms)","value":3390},{"name":"PHYSICAL_MEMORY_BYTES","displayName":"Physical memory (bytes) snapshot","value":1840713728},{"name":"VIRTUAL_MEMORY_BYTES","displayName":"Virtual memory (bytes) snapshot","value":7059025920},{"name":"COMMITTED_HEAP_BYTES","displayName":"Total committed heap usage (bytes)","value":1717960704}]},{"name":"org.apache.hadoop.mapreduce.lib.input.FileInputFormatCounter","displayName":"File Input Format Counters ","counts":[{"name":"BYTES_READ","displayName":"Bytes Read","value":0}]}]},"reduceCounters":{"name":"REDUCE_COUNTERS","groups":[{"name":"Shuffle Errors","displayName":"Shuffle Errors","counts":[{"name":"BAD_ID","displayName":"BAD_ID","value":0},{"name":"CONNECTION","displayName":"CONNECTION","value":0},{"name":"IO_ERROR","displayName":"IO_ERROR","value":0},{"name":"WRONG_LENGTH","displayName":"WRONG_LENGTH","value":0},{"name":"WRONG_MAP","displayName":"WRONG_MAP","value":0},{"name":"WRONG_REDUCE","displayName":"WRONG_REDUCE","value":0}]},{"name":"org.apache.hadoop.mapreduce.FileSystemCounter","displayName":"File System Counters","counts":[{"name":"FILE_BYTES_READ","displayName":"FILE: Number of bytes read","value":186},{"name":"FILE_BYTES_WRITTEN","displayName":"FILE: Number of bytes written","value":48074},{"name":"FILE_READ_OPS","displayName":"FILE: Number of read operations","value":0},{"name":"FILE_LARGE_READ_OPS","displayName":"FILE: Number of large read operations","value":0},{"name":"FILE_WRITE_OPS","displayName":"FILE: Number of write operations","value":0},{"name":"HDFS_BYTES_READ","displayName":"HDFS: Number of bytes read","value":0},{"name":"HDFS_BYTES_WRITTEN","displayName":"HDFS: Number of bytes written","value":0},{"name":"HDFS_READ_OPS","displayName":"HDFS: Number of read operations","value":0},{"name":"HDFS_LARGE_READ_OPS","displayName":"HDFS: Number of large read operations","value":0},{"name":"HDFS_WRITE_OPS","displayName":"HDFS: Number of write operations","value":0}]},{"name":"org.apache.hadoop.mapreduce.TaskCounter","displayName":"Map-Reduce Framework","counts":[{"name":"COMBINE_INPUT_RECORDS","displayName":"Combine input records","value":0},{"name":"COMBINE_OUTPUT_RECORDS","displayName":"Combine output records","value":0},{"name":"REDUCE_INPUT_GROUPS","displayName":"Reduce input groups","value":1},{"name":"REDUCE_SHUFFLE_BYTES","displayName":"Reduce shuffle bytes","value":120},{"name":"REDUCE_INPUT_RECORDS","displayName":"Reduce input records","value":10},{"name":"REDUCE_OUTPUT_RECORDS","displayName":"Reduce output records","value":0},{"name":"SPILLED_RECORDS","displayName":"Spilled Records","value":10},{"name":"SHUFFLED_MAPS","displayName":"Shuffled Maps ","value":10},{"name":"FAILED_SHUFFLE","displayName":"Failed Shuffles","value":0},{"name":"MERGED_MAP_OUTPUTS","displayName":"Merged Map outputs","value":10},{"name":"GC_TIME_MILLIS","displayName":"GC time elapsed (ms)","value":14},{"name":"CPU_MILLISECONDS","displayName":"CPU time spent (ms)","value":1070},{"name":"PHYSICAL_MEMORY_BYTES","displayName":"Physical memory (bytes) snapshot","value":82780160},{"name":"VIRTUAL_MEMORY_BYTES","displayName":"Virtual memory (bytes) snapshot","value":714436608},{"name":"COMMITTED_HEAP_BYTES","displayName":"Total committed heap usage (bytes)","value":60555264}]},{"name":"org.apache.hadoop.mapreduce.lib.output.FileOutputFormatCounter","displayName":"File Output Format Counters ","counts":[{"name":"BYTES_WRITTEN","displayName":"Bytes Written","value":0}]}]}}}}