You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tez.apache.org by je...@apache.org on 2015/12/07 23:42:54 UTC
[1/5] tez git commit: TEZ-2973. Backport Analyzers to branch-0.7
Repository: tez
Updated Branches:
refs/heads/branch-0.7 868ca531c -> 8c8db7c51
http://git-wip-us.apache.org/repos/asf/tez/blob/8c8db7c5/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/utils/SVGUtils.java
----------------------------------------------------------------------
diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/utils/SVGUtils.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/utils/SVGUtils.java
new file mode 100644
index 0000000..b91cdc4
--- /dev/null
+++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/utils/SVGUtils.java
@@ -0,0 +1,334 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.tez.analyzer.utils;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.io.output.FileWriterWithEncoding;
+import org.apache.tez.analyzer.plugins.CriticalPathAnalyzer.CriticalPathStep;
+import org.apache.tez.analyzer.plugins.CriticalPathAnalyzer.CriticalPathStep.EntityType;
+import org.apache.tez.history.parser.datamodel.DagInfo;
+import org.apache.tez.history.parser.datamodel.TaskAttemptInfo;
+
+import com.google.common.base.Joiner;
+
+import java.io.BufferedWriter;
+import java.io.IOException;
+import java.text.DecimalFormat;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+public class SVGUtils {
+
+ private static int MAX_DAG_RUNTIME = 0;
+ private static final int SCREEN_WIDTH = 1800;
+
+ public SVGUtils() {
+ }
+
+ private int Y_MAX;
+ private int X_MAX;
+ private static final DecimalFormat secondFormat = new DecimalFormat("#.##");
+ private static final int X_BASE = 100;
+ private static final int Y_BASE = 100;
+ private static final int TICK = 1;
+ private static final int STEP_GAP = 50;
+ private static final int TEXT_SIZE = 20;
+ private static final String RUNTIME_COLOR = "LightGreen";
+ private static final String ALLOCATION_OVERHEAD_COLOR = "GoldenRod";
+ private static final String LAUNCH_OVERHEAD_COLOR = "DarkSalmon";
+ private static final String BORDER_COLOR = "Sienna";
+ private static final String VERTEX_INIT_COMMIT_COLOR = "LightSalmon";
+ private static final String CRITICAL_COLOR = "IndianRed";
+ private static final float RECT_OPACITY = 1.0f;
+ private static final String TITLE_BR = " ";
+
+ public static String getTimeStr(final long millis) {
+ long minutes = TimeUnit.MILLISECONDS.toMinutes(millis)
+ - TimeUnit.HOURS.toMinutes(TimeUnit.MILLISECONDS.toHours(millis));
+ long hours = TimeUnit.MILLISECONDS.toHours(millis);
+ StringBuilder b = new StringBuilder();
+ b.append(hours == 0 ? "" : String.valueOf(hours) + "h");
+ b.append(minutes == 0 ? "" : String.valueOf(minutes) + "m");
+ long seconds = millis - TimeUnit.MINUTES.toMillis(
+ TimeUnit.MILLISECONDS.toMinutes(millis));
+ b.append(secondFormat.format(seconds/1000.0) + "s");
+
+ return b.toString();
+ }
+
+ List<String> svgLines = new LinkedList<String>();
+
+ private final int addOffsetX(int x) {
+ int xOff = x + X_BASE;
+ X_MAX = Math.max(X_MAX, xOff);
+ return xOff;
+ }
+
+ private final int addOffsetY(int y) {
+ int yOff = y + Y_BASE;
+ Y_MAX = Math.max(Y_MAX, yOff);
+ return yOff;
+ }
+
+ private int scaleDown(int len) {
+ return Math.round((len * 1.0f / MAX_DAG_RUNTIME) * SCREEN_WIDTH);
+ }
+
+ private void addRectStr(int x, int width, int y, int height,
+ String fillColor, String borderColor, float opacity, String title) {
+ String rectStyle = "stroke: " + borderColor + "; fill: " + fillColor + "; opacity: " + opacity;
+ String rectStr = "<rect x=\"" + addOffsetX(scaleDown(x)) + "\""
+ + " y=\"" + addOffsetY(y) + "\""
+ + " width=\"" + scaleDown(width) + "\""
+ + " height=\"" + height + "\""
+ + " style=\"" + rectStyle + "\""
+ + " >"
+ + " <title>" + title +"</title>"
+ + " </rect>";
+ svgLines.add(rectStr);
+ }
+
+ private void addTextStr(int x, int y, String text, String anchor, int size, String title, boolean italic) {
+ String textStyle = "text-anchor: " + anchor + "; font-style: " + (italic?"italic":"normal") +
+ "; font-size: " + size + "px;";
+ String textStr = "<text x=\"" + addOffsetX(scaleDown(x)) + "\" "
+ + "y=\"" + addOffsetY(y) + "\" "
+ + "style=\"" + textStyle + "\" transform=\"\">"
+ + text
+ + " <title>" + title +"</title>"
+ + "</text>";
+ svgLines.add(textStr);
+ }
+
+ private void addLineStr(int x1, int y1, int x2, int y2, String color, String title, int width) {
+ String style = "stroke: " + color + "; stroke-width:" + width;
+ String str = "<line x1=\"" + addOffsetX(scaleDown(x1)) + "\""
+ + " y1=\"" + addOffsetY(y1) + "\""
+ + " x2=\"" + addOffsetX(scaleDown(x2)) + "\""
+ + " y2=\"" + addOffsetY(y2) + "\""
+ + " style=\"" + style + "\""
+ + " >"
+ + " <title>" + title +"</title>"
+ + " </line>";
+ svgLines.add(str);
+ }
+
+ public void drawStep(CriticalPathStep step, long dagStartTime, int yOffset) {
+ if (step.getType() != EntityType.ATTEMPT) {
+ // draw initial vertex or final commit overhead
+ StringBuilder title = new StringBuilder();
+ String text = null;
+ if (step.getType() == EntityType.VERTEX_INIT) {
+ String vertex = step.getAttempt().getTaskInfo().getVertexInfo().getVertexName();
+ text = vertex + " : Init";
+ title.append(text).append(TITLE_BR);
+ } else {
+ text = "Output Commit";
+ title.append(text).append(TITLE_BR);
+ }
+ title.append("Critical Path Dependency: " + step.getReason()).append(TITLE_BR);
+ title.append(
+ "Critical Time: " + getTimeStr(step.getStopCriticalTime() - step.getStartCriticalTime()))
+ .append("");
+ title.append(Joiner.on(TITLE_BR).join(step.getNotes()));
+ String titleStr = title.toString();
+ int stopTimeInterval = (int) (step.getStopCriticalTime() - dagStartTime);
+ int startTimeInterval = (int) (step.getStartCriticalTime() - dagStartTime);
+ addRectStr(startTimeInterval,
+ (stopTimeInterval - startTimeInterval), yOffset * STEP_GAP, STEP_GAP,
+ VERTEX_INIT_COMMIT_COLOR, BORDER_COLOR, RECT_OPACITY, titleStr);
+ addTextStr((stopTimeInterval + startTimeInterval) / 2,
+ (yOffset * STEP_GAP + STEP_GAP / 2),
+ text, "middle",
+ TEXT_SIZE, titleStr, false);
+ } else {
+ TaskAttemptInfo attempt = step.getAttempt();
+ int startCriticalTimeInterval = (int) (step.getStartCriticalTime() - dagStartTime);
+ int stopCriticalTimeInterval = (int) (step.getStopCriticalTime() - dagStartTime);
+ int creationTimeInterval = (int) (attempt.getCreationTime() - dagStartTime);
+ int allocationTimeInterval = attempt.getAllocationTime() > 0 ?
+ (int) (attempt.getAllocationTime() - dagStartTime) : 0;
+ int launchTimeInterval = attempt.getStartTime() > 0 ?
+ (int) (attempt.getStartTime() - dagStartTime) : 0;
+ int finishTimeInterval = (int) (attempt.getFinishTime() - dagStartTime);
+ System.out.println(attempt.getTaskAttemptId() + " " + creationTimeInterval + " "
+ + allocationTimeInterval + " " + launchTimeInterval + " " + finishTimeInterval);
+
+ StringBuilder title = new StringBuilder();
+ title.append("Attempt: " + attempt.getTaskAttemptId()).append(TITLE_BR);
+ title.append("Critical Path Dependency: " + step.getReason()).append(TITLE_BR);
+ title.append("Completion Status: " + attempt.getDetailedStatus()).append(TITLE_BR);
+ title.append(
+ "Critical Time Contribution: " +
+ getTimeStr(step.getStopCriticalTime() - step.getStartCriticalTime())).append(TITLE_BR);
+ title.append("Critical start at: " + getTimeStr(startCriticalTimeInterval)).append(TITLE_BR);
+ title.append("Critical stop at: " + getTimeStr(stopCriticalTimeInterval)).append(TITLE_BR);
+ title.append("Created at: " + getTimeStr(creationTimeInterval)).append(TITLE_BR);
+ if (allocationTimeInterval > 0) {
+ title.append("Allocated at: " + getTimeStr(allocationTimeInterval)).append(TITLE_BR);
+ }
+ if (launchTimeInterval > 0) {
+ title.append("Launched at: " + getTimeStr(launchTimeInterval)).append(TITLE_BR);
+ }
+ title.append("Finished at: " + getTimeStr(finishTimeInterval)).append(TITLE_BR);
+ title.append(Joiner.on(TITLE_BR).join(step.getNotes()));
+ String titleStr = title.toString();
+
+ // handle cases when attempt fails before allocation or launch
+ if (allocationTimeInterval > 0) {
+ addRectStr(creationTimeInterval, allocationTimeInterval - creationTimeInterval,
+ yOffset * STEP_GAP, STEP_GAP, ALLOCATION_OVERHEAD_COLOR, BORDER_COLOR, RECT_OPACITY,
+ titleStr);
+ if (launchTimeInterval > 0) {
+ addRectStr(allocationTimeInterval, launchTimeInterval - allocationTimeInterval,
+ yOffset * STEP_GAP, STEP_GAP, LAUNCH_OVERHEAD_COLOR, BORDER_COLOR, RECT_OPACITY,
+ titleStr);
+ addRectStr(launchTimeInterval, finishTimeInterval - launchTimeInterval, yOffset * STEP_GAP,
+ STEP_GAP, RUNTIME_COLOR, BORDER_COLOR, RECT_OPACITY, titleStr);
+ } else {
+ // no launch - so allocate to finish drawn - ended while launching
+ addRectStr(allocationTimeInterval, finishTimeInterval - allocationTimeInterval, yOffset * STEP_GAP,
+ STEP_GAP, LAUNCH_OVERHEAD_COLOR, BORDER_COLOR, RECT_OPACITY, titleStr);
+ }
+ } else {
+ // no allocation - so create to finish drawn - ended while allocating
+ addRectStr(creationTimeInterval, finishTimeInterval - creationTimeInterval, yOffset * STEP_GAP,
+ STEP_GAP, ALLOCATION_OVERHEAD_COLOR, BORDER_COLOR, RECT_OPACITY, titleStr);
+ }
+
+ addTextStr((finishTimeInterval + creationTimeInterval) / 2,
+ (yOffset * STEP_GAP + STEP_GAP / 2), attempt.getShortName(), "middle", TEXT_SIZE,
+ titleStr, !attempt.isSucceeded());
+ }
+ }
+
+ private void drawCritical(DagInfo dagInfo, List<CriticalPathStep> criticalPath) {
+ long dagStartTime = dagInfo.getStartTime();
+ int dagStartTimeInterval = 0; // this is 0 since we are offseting from the dag start time
+ int dagFinishTimeInterval = (int) (dagInfo.getFinishTime() - dagStartTime);
+ if (dagInfo.getFinishTime() <= 0) {
+ // AM crashed. no dag finish time written
+ dagFinishTimeInterval =(int) (criticalPath.get(criticalPath.size()-1).getStopCriticalTime()
+ - dagStartTime);
+ }
+ MAX_DAG_RUNTIME = dagFinishTimeInterval;
+
+ // draw grid
+ addLineStr(dagStartTimeInterval, 0, dagFinishTimeInterval, 0, BORDER_COLOR, "", TICK);
+ int yGrid = (criticalPath.size() + 2)*STEP_GAP;
+ for (int i=0; i<11; ++i) {
+ int x = Math.round(((dagFinishTimeInterval - dagStartTimeInterval)/10.0f)*i);
+ addLineStr(x, 0, x, yGrid, BORDER_COLOR, "", TICK);
+ addTextStr(x, 0, getTimeStr(x), "left", TEXT_SIZE, "", false);
+ }
+ addLineStr(dagStartTimeInterval, yGrid, dagFinishTimeInterval, yGrid, BORDER_COLOR, "", TICK);
+ addTextStr((dagFinishTimeInterval + dagStartTimeInterval) / 2, yGrid + STEP_GAP,
+ "Critical Path for " + dagInfo.getName() + " (" + dagInfo.getDagId() + ")", "middle",
+ TEXT_SIZE, "", false);
+
+ // draw steps
+ for (int i=1; i<=criticalPath.size(); ++i) {
+ CriticalPathStep step = criticalPath.get(i-1);
+ drawStep(step, dagStartTime, i);
+ }
+
+ // draw critical path on top
+ for (int i=1; i<=criticalPath.size(); ++i) {
+ CriticalPathStep step = criticalPath.get(i-1);
+ boolean isLast = i == criticalPath.size();
+
+ // draw critical path for step
+ int startCriticalTimeInterval = (int) (step.getStartCriticalTime() - dagStartTime);
+ int stopCriticalTimeInterval = (int) (step.getStopCriticalTime() - dagStartTime);
+ addLineStr(startCriticalTimeInterval, (i + 1) * STEP_GAP, stopCriticalTimeInterval,
+ (i + 1) * STEP_GAP, CRITICAL_COLOR, "Critical Time " + step.getAttempt().getShortName(), TICK*5);
+
+ if (isLast) {
+ // last step. add commit overhead
+ int stepStopCriticalTimeInterval = (int) (step.getStopCriticalTime() - dagStartTime);
+ addLineStr(stepStopCriticalTimeInterval, (i + 1) * STEP_GAP, dagFinishTimeInterval,
+ (i + 1) * STEP_GAP, CRITICAL_COLOR,
+ "Critical Time " + step.getAttempt().getTaskInfo().getVertexInfo().getVertexName(), TICK*5);
+ } else {
+ // connect to next step in critical path
+ addLineStr(stopCriticalTimeInterval, (i + 1) * STEP_GAP, stopCriticalTimeInterval,
+ (i + 2) * STEP_GAP, CRITICAL_COLOR, "Critical Time " + step.getAttempt().getShortName(), TICK*5);
+ }
+ }
+
+ // draw legend
+ int legendX = 0;
+ int legendY = (criticalPath.size() + 2) * STEP_GAP;
+ int legendWidth = dagFinishTimeInterval/5;
+
+ addRectStr(legendX, legendWidth, legendY, STEP_GAP/2, VERTEX_INIT_COMMIT_COLOR, BORDER_COLOR, RECT_OPACITY, "");
+ addTextStr(legendX, legendY + STEP_GAP/3, "Vertex Init/Commit Overhead", "left", TEXT_SIZE, "", false);
+ legendY += STEP_GAP/2;
+ addRectStr(legendX, legendWidth, legendY, STEP_GAP/2, ALLOCATION_OVERHEAD_COLOR, BORDER_COLOR, RECT_OPACITY, "");
+ addTextStr(legendX, legendY + STEP_GAP/3, "Task Allocation Overhead", "left", TEXT_SIZE, "", false);
+ legendY += STEP_GAP/2;
+ addRectStr(legendX, legendWidth, legendY, STEP_GAP/2, LAUNCH_OVERHEAD_COLOR, BORDER_COLOR, RECT_OPACITY, "");
+ addTextStr(legendX, legendY + STEP_GAP/3, "Task Launch Overhead", "left", TEXT_SIZE, "", false);
+ legendY += STEP_GAP/2;
+ addRectStr(legendX, legendWidth, legendY, STEP_GAP/2, RUNTIME_COLOR, BORDER_COLOR, RECT_OPACITY, "");
+ addTextStr(legendX, legendY + STEP_GAP/3, "Task Execution Time", "left", TEXT_SIZE, "", false);
+
+ Y_MAX += Y_BASE*2;
+ X_MAX += X_BASE*2;
+ }
+
+ public void saveCriticalPathAsSVG(DagInfo dagInfo,
+ String fileName, List<CriticalPathStep> criticalPath) {
+ drawCritical(dagInfo, criticalPath);
+ saveFileStr(fileName);
+ }
+
+ private void saveFileStr(String fileName) {
+ String header = "<?xml version=\"1.0\" standalone=\"no\"?> "
+ + "<!DOCTYPE svg PUBLIC \"-//W3C//DTD SVG 1.1//EN\" "
+ + "\"http://www.w3.org/Graphics/SVG/1.1/DTD/svg11.dtd\">"
+ + "<svg xmlns=\"http://www.w3.org/2000/svg\" version=\"1.1\" "
+ + "xmlns:xlink=\"http://www.w3.org/1999/xlink\" "
+ + "height=\"" + Y_MAX + "\" "
+ + "width=\"" + X_MAX + "\"> "
+ + "<script type=\"text/ecmascript\" "
+ + "xlink:href=\"http://code.jquery.com/jquery-2.1.1.min.js\" />";
+ String footer = "</svg>";
+ String newline = System.getProperty("line.separator");
+ BufferedWriter writer = null;
+ try {
+ writer = new BufferedWriter(new FileWriterWithEncoding(fileName, "UTF-8"));
+ writer.write(header);
+ writer.write(newline);
+ for (String str : svgLines) {
+ writer.write(str);
+ writer.write(newline);
+ }
+ writer.write(footer);
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ } finally {
+ if (writer != null) {
+ IOUtils.closeQuietly(writer);
+ }
+ }
+
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/8c8db7c5/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/utils/Utils.java
----------------------------------------------------------------------
diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/utils/Utils.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/utils/Utils.java
new file mode 100644
index 0000000..8bcf265
--- /dev/null
+++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/utils/Utils.java
@@ -0,0 +1,100 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.analyzer.utils;
+
+import com.sun.istack.Nullable;
+import org.apache.tez.dag.utils.Graph;
+import org.apache.tez.history.parser.datamodel.AdditionalInputOutputDetails;
+import org.apache.tez.history.parser.datamodel.DagInfo;
+import org.apache.tez.history.parser.datamodel.EdgeInfo;
+import org.apache.tez.history.parser.datamodel.VertexInfo;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+public class Utils {
+
+ private static Pattern sanitizeLabelPattern = Pattern.compile("[:\\-\\W]+");
+
+ public static String getShortClassName(String className) {
+ int pos = className.lastIndexOf(".");
+ if (pos != -1 && pos < className.length() - 1) {
+ return className.substring(pos + 1);
+ }
+ return className;
+ }
+
+ public static String sanitizeLabelForViz(String label) {
+ Matcher m = sanitizeLabelPattern.matcher(label);
+ return m.replaceAll("_");
+ }
+
+ public static void generateDAGVizFile(DagInfo dagInfo, String fileName,
+ @Nullable List<String> criticalVertices) throws IOException {
+ Graph graph = new Graph(sanitizeLabelForViz(dagInfo.getName()));
+
+ for (VertexInfo v : dagInfo.getVertices()) {
+ String nodeLabel = sanitizeLabelForViz(v.getVertexName())
+ + "[" + getShortClassName(v.getProcessorClassName()
+ + ", tasks=" + v.getTasks().size() + ", time=" + v.getTimeTaken() +" ms]");
+ Graph.Node n = graph.newNode(sanitizeLabelForViz(v.getVertexName()), nodeLabel);
+
+ boolean criticalVertex = (criticalVertices != null) ? criticalVertices.contains(v
+ .getVertexName()) : false;
+ if (criticalVertex) {
+ n.setColor("red");
+ }
+
+
+ for (AdditionalInputOutputDetails input : v.getAdditionalInputInfoList()) {
+ Graph.Node inputNode = graph.getNode(sanitizeLabelForViz(v.getVertexName())
+ + "_" + sanitizeLabelForViz(input.getName()));
+ inputNode.setLabel(sanitizeLabelForViz(v.getVertexName())
+ + "[" + sanitizeLabelForViz(input.getName()) + "]");
+ inputNode.setShape("box");
+ inputNode.addEdge(n, "Input name=" + input.getName()
+ + " [inputClass=" + getShortClassName(input.getClazz())
+ + ", initializer=" + getShortClassName(input.getInitializer()) + "]");
+ }
+ for (AdditionalInputOutputDetails output : v.getAdditionalOutputInfoList()) {
+ Graph.Node outputNode = graph.getNode(sanitizeLabelForViz(v.getVertexName())
+ + "_" + sanitizeLabelForViz(output.getName()));
+ outputNode.setLabel(sanitizeLabelForViz(v.getVertexName())
+ + "[" + sanitizeLabelForViz(output.getName()) + "]");
+ outputNode.setShape("box");
+ n.addEdge(outputNode, "Output name=" + output.getName()
+ + " [outputClass=" + getShortClassName(output.getClazz())
+ + ", committer=" + getShortClassName(output.getInitializer()) + "]");
+ }
+
+ }
+
+ for (EdgeInfo e : dagInfo.getEdges()) {
+ Graph.Node n = graph.getNode(sanitizeLabelForViz(e.getInputVertexName()));
+ n.addEdge(graph.getNode(sanitizeLabelForViz(e.getOutputVertexName())),
+ "[input=" + getShortClassName(e.getEdgeSourceClass())
+ + ", output=" + getShortClassName(e.getEdgeDestinationClass())
+ + ", dataMovement=" + e.getDataMovementType().trim() + "]");
+ }
+
+ graph.save(fileName);
+ }
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/8c8db7c5/tez-tools/analyzers/job-analyzer/src/test/java/org/apache/tez/analyzer/TestAnalyzer.java
----------------------------------------------------------------------
diff --git a/tez-tools/analyzers/job-analyzer/src/test/java/org/apache/tez/analyzer/TestAnalyzer.java b/tez-tools/analyzers/job-analyzer/src/test/java/org/apache/tez/analyzer/TestAnalyzer.java
new file mode 100644
index 0000000..f680f59
--- /dev/null
+++ b/tez-tools/analyzers/job-analyzer/src/test/java/org/apache/tez/analyzer/TestAnalyzer.java
@@ -0,0 +1,823 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.analyzer;
+
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.util.Collections;
+import java.util.List;
+import java.util.Random;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.server.namenode.EditLogFileOutputStream;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.tez.analyzer.plugins.CriticalPathAnalyzer;
+import org.apache.tez.analyzer.plugins.CriticalPathAnalyzer.CriticalPathDependency;
+import org.apache.tez.analyzer.plugins.CriticalPathAnalyzer.CriticalPathStep;
+import org.apache.tez.analyzer.plugins.CriticalPathAnalyzer.CriticalPathStep.EntityType;
+import org.apache.tez.client.TezClient;
+import org.apache.tez.dag.api.DAG;
+import org.apache.tez.dag.api.TezConfiguration;
+import org.apache.tez.dag.api.TezConstants;
+import org.apache.tez.dag.api.client.DAGClient;
+import org.apache.tez.dag.api.client.DAGStatus;
+import org.apache.tez.dag.history.logging.ats.ATSHistoryLoggingService;
+import org.apache.tez.dag.history.logging.impl.SimpleHistoryLoggingService;
+import org.apache.tez.dag.records.TaskAttemptTerminationCause;
+import org.apache.tez.dag.records.TezDAGID;
+import org.apache.tez.history.ATSImportTool;
+import org.apache.tez.history.parser.ATSFileParser;
+import org.apache.tez.history.parser.SimpleHistoryParser;
+import org.apache.tez.history.parser.datamodel.DagInfo;
+import org.apache.tez.test.SimpleTestDAG;
+import org.apache.tez.test.SimpleTestDAG3Vertices;
+import org.apache.tez.test.TestInput;
+import org.apache.tez.test.TestProcessor;
+import org.apache.tez.test.dag.SimpleReverseVTestDAG;
+import org.apache.tez.test.dag.SimpleVTestDAG;
+import org.apache.tez.tests.MiniTezClusterWithTimeline;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.Lists;
+
+public class TestAnalyzer {
+ private static final Logger LOG = LoggerFactory.getLogger(TestAnalyzer.class);
+
+ private static String TEST_ROOT_DIR =
+ "target" + Path.SEPARATOR + TestAnalyzer.class.getName() + "-tmpDir";
+ private static String DOWNLOAD_DIR = TEST_ROOT_DIR + Path.SEPARATOR + "download";
+ private final static String SIMPLE_HISTORY_DIR = "/tmp/simplehistory/";
+ private final static String HISTORY_TXT = "history.txt";
+
+ private static MiniDFSCluster dfsCluster;
+ private static MiniTezClusterWithTimeline miniTezCluster;
+
+ private static Configuration conf = new Configuration();
+ private static FileSystem fs;
+
+ private static TezClient tezSession = null;
+
+ private boolean usingATS = true;
+ private boolean downloadedSimpleHistoryFile = false;
+
+ @BeforeClass
+ public static void setupClass() throws Exception {
+ conf = new Configuration();
+ conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_EDITS_NOEDITLOGCHANNELFLUSH, false);
+ EditLogFileOutputStream.setShouldSkipFsyncForTesting(true);
+ conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, TEST_ROOT_DIR);
+ dfsCluster =
+ new MiniDFSCluster.Builder(conf).numDataNodes(1).format(true).build();
+ fs = dfsCluster.getFileSystem();
+ conf.set("fs.defaultFS", fs.getUri().toString());
+
+ setupTezCluster();
+ }
+
+ @AfterClass
+ public static void tearDownClass() throws Exception {
+ LOG.info("Stopping mini clusters");
+ if (miniTezCluster != null) {
+ miniTezCluster.stop();
+ miniTezCluster = null;
+ }
+ if (dfsCluster != null) {
+ dfsCluster.shutdown();
+ dfsCluster = null;
+ }
+ }
+
+ private CriticalPathAnalyzer setupCPAnalyzer() {
+ Configuration analyzerConf = new Configuration(false);
+ analyzerConf.setBoolean(CriticalPathAnalyzer.DRAW_SVG, false);
+ CriticalPathAnalyzer cp = new CriticalPathAnalyzer();
+ cp.setConf(analyzerConf);
+ return cp;
+ }
+
+ private static void setupTezCluster() throws Exception {
+ // make the test run faster by speeding heartbeat frequency
+ conf.setInt(YarnConfiguration.RM_NM_HEARTBEAT_INTERVAL_MS, 100);
+ conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
+ conf.setBoolean(TezConfiguration.TEZ_AM_ALLOW_DISABLED_TIMELINE_DOMAINS, true);
+ conf.set(TezConfiguration.TEZ_HISTORY_LOGGING_SERVICE_CLASS, ATSHistoryLoggingService
+ .class.getName());
+
+ miniTezCluster =
+ new MiniTezClusterWithTimeline(TestAnalyzer.class.getName(), 1, 1, 1, true);
+
+ miniTezCluster.init(conf);
+ miniTezCluster.start();
+ }
+
+ private TezConfiguration createCommonTezLog() throws Exception {
+ TezConfiguration tezConf = new TezConfiguration(miniTezCluster.getConfig());
+
+ tezConf.setInt(TezConfiguration.TEZ_AM_RM_HEARTBEAT_INTERVAL_MS_MAX, 100);
+ Path remoteStagingDir = dfsCluster.getFileSystem().makeQualified(new Path(TEST_ROOT_DIR, String
+ .valueOf(new Random().nextInt(100000))));
+
+ tezConf.set(TezConfiguration.TEZ_AM_STAGING_DIR,
+ remoteStagingDir.toString());
+ tezConf.setBoolean(TezConfiguration.TEZ_AM_NODE_BLACKLISTING_ENABLED, false);
+
+ return tezConf;
+ }
+
+ private void createTezSessionATS() throws Exception {
+ TezConfiguration tezConf = createCommonTezLog();
+ tezConf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
+ tezConf.set(YarnConfiguration.TIMELINE_SERVICE_WEBAPP_ADDRESS, "0.0.0.0:8188");
+ tezConf.setBoolean(TezConfiguration.TEZ_AM_ALLOW_DISABLED_TIMELINE_DOMAINS, true);
+ tezConf.set(TezConfiguration.TEZ_HISTORY_LOGGING_SERVICE_CLASS,
+ ATSHistoryLoggingService.class.getName());
+
+ Path remoteStagingDir = dfsCluster.getFileSystem().makeQualified(new Path(TEST_ROOT_DIR, String
+ .valueOf(new Random().nextInt(100000))));
+
+ tezConf.set(TezConfiguration.TEZ_AM_STAGING_DIR,
+ remoteStagingDir.toString());
+ tezConf.setBoolean(TezConfiguration.TEZ_AM_NODE_BLACKLISTING_ENABLED, false);
+
+ tezSession = TezClient.create("TestAnalyzer", tezConf, true);
+ tezSession.start();
+ }
+
+ private void createTezSessionSimpleHistory() throws Exception {
+ TezConfiguration tezConf = createCommonTezLog();
+ tezConf.set(TezConfiguration.TEZ_HISTORY_LOGGING_SERVICE_CLASS,
+ SimpleHistoryLoggingService.class.getName());
+
+ tezConf.set(TezConfiguration.TEZ_SIMPLE_HISTORY_LOGGING_DIR, SIMPLE_HISTORY_DIR);
+
+ Path remoteStagingDir = dfsCluster.getFileSystem().makeQualified(new Path(TEST_ROOT_DIR, String
+ .valueOf(new Random().nextInt(100000))));
+
+ tezConf.set(TezConfiguration.TEZ_AM_STAGING_DIR,
+ remoteStagingDir.toString());
+ tezConf.setBoolean(TezConfiguration.TEZ_AM_NODE_BLACKLISTING_ENABLED, false);
+
+ tezSession = TezClient.create("TestFaultTolerance", tezConf, true);
+ tezSession.start();
+ }
+
+ private StepCheck createStep(String attempt, CriticalPathDependency reason) {
+ return createStep(attempt, reason, null, null);
+ }
+
+ private StepCheck createStep(String attempt, CriticalPathDependency reason,
+ TaskAttemptTerminationCause errCause, List<String> notes) {
+ return new StepCheck(attempt, reason, errCause, notes);
+ }
+
+ private class StepCheck {
+ String attempt; // attempt is the TaskAttemptInfo short name with regex
+ CriticalPathDependency reason;
+ TaskAttemptTerminationCause errCause;
+ List<String> notesStr;
+
+ StepCheck(String attempt, CriticalPathDependency reason,
+ TaskAttemptTerminationCause cause, List<String> notes) {
+ this.attempt = attempt;
+ this.reason = reason;
+ this.errCause = cause;
+ this.notesStr = notes;
+ }
+ String getAttemptDetail() {
+ return attempt;
+ }
+ CriticalPathDependency getReason() {
+ return reason;
+ }
+ TaskAttemptTerminationCause getErrCause() {
+ return errCause;
+ }
+ List<String> getNotesStr() {
+ return notesStr;
+ }
+ }
+
+ private void runDAG(DAG dag, DAGStatus.State finalState) throws Exception {
+ tezSession.waitTillReady();
+ LOG.info("ABC Running DAG name: " + dag.getName());
+ DAGClient dagClient = tezSession.submitDAG(dag);
+ DAGStatus dagStatus = dagClient.getDAGStatus(null);
+ while (!dagStatus.isCompleted()) {
+ LOG.info("Waiting for dag to complete. Sleeping for 500ms."
+ + " DAG name: " + dag.getName()
+ + " DAG appContext: " + dagClient.getExecutionContext()
+ + " Current state: " + dagStatus.getState());
+ Thread.sleep(100);
+ dagStatus = dagClient.getDAGStatus(null);
+ }
+
+ Assert.assertEquals(finalState, dagStatus.getState());
+ }
+
+ private void verify(ApplicationId appId, int dagNum, List<StepCheck[]> steps) throws Exception {
+ String dagId = TezDAGID.getInstance(appId, dagNum).toString();
+ DagInfo dagInfo = getDagInfo(dagId);
+
+ verifyCriticalPath(dagInfo, steps);
+ }
+
+ private DagInfo getDagInfo(String dagId) throws Exception {
+ // sleep for a bit to let ATS events be sent from AM
+ DagInfo dagInfo = null;
+ if (usingATS) {
+ //Export the data from ATS
+ String[] args = { "--dagId=" + dagId, "--downloadDir=" + DOWNLOAD_DIR };
+
+ int result = ATSImportTool.process(args);
+ assertTrue(result == 0);
+
+ //Parse ATS data and verify results
+ //Parse downloaded contents
+ File downloadedFile = new File(DOWNLOAD_DIR
+ + Path.SEPARATOR + dagId + ".zip");
+ ATSFileParser parser = new ATSFileParser(downloadedFile);
+ dagInfo = parser.getDAGData(dagId);
+ assertTrue(dagInfo.getDagId().equals(dagId));
+ } else {
+ if (!downloadedSimpleHistoryFile) {
+ downloadedSimpleHistoryFile = true;
+ TezDAGID tezDAGID = TezDAGID.fromString(dagId);
+ ApplicationAttemptId applicationAttemptId = ApplicationAttemptId.newInstance(tezDAGID
+ .getApplicationId(), 1);
+ Path historyPath = new Path(miniTezCluster.getConfig().get("fs.defaultFS")
+ + SIMPLE_HISTORY_DIR + HISTORY_TXT + "."
+ + applicationAttemptId);
+ FileSystem fs = historyPath.getFileSystem(miniTezCluster.getConfig());
+
+ Path localPath = new Path(DOWNLOAD_DIR, HISTORY_TXT);
+ fs.copyToLocalFile(historyPath, localPath);
+ }
+ //Now parse via SimpleHistory
+ File localFile = new File(DOWNLOAD_DIR, HISTORY_TXT);
+ SimpleHistoryParser parser = new SimpleHistoryParser(localFile);
+ dagInfo = parser.getDAGData(dagId);
+ assertTrue(dagInfo.getDagId().equals(dagId));
+ }
+ return dagInfo;
+ }
+
+ private void verifyCriticalPath(DagInfo dagInfo, List<StepCheck[]> stepsOptions) throws Exception {
+ CriticalPathAnalyzer cp = setupCPAnalyzer();
+ cp.analyze(dagInfo);
+
+ List<CriticalPathStep> criticalPath = cp.getCriticalPath();
+
+ for (CriticalPathStep step : criticalPath) {
+ LOG.info("ABC Step: " + step.getType());
+ if (step.getType() == EntityType.ATTEMPT) {
+ LOG.info("ABC Attempt: " + step.getAttempt().getShortName()
+ + " " + step.getAttempt().getDetailedStatus());
+ }
+ LOG.info("ABC Reason: " + step.getReason());
+ String notes = Joiner.on(";").join(step.getNotes());
+ LOG.info("ABC Notes: " + notes);
+ }
+
+ boolean foundMatchingLength = false;
+ for (StepCheck[] steps : stepsOptions) {
+ if (steps.length + 2 == criticalPath.size()) {
+ foundMatchingLength = true;
+ Assert.assertEquals(CriticalPathStep.EntityType.VERTEX_INIT, criticalPath.get(0).getType());
+ Assert.assertEquals(criticalPath.get(1).getAttempt().getShortName(),
+ criticalPath.get(0).getAttempt().getShortName());
+
+ for (int i=1; i<criticalPath.size() - 1; ++i) {
+ StepCheck check = steps[i-1];
+ CriticalPathStep step = criticalPath.get(i);
+ Assert.assertEquals(CriticalPathStep.EntityType.ATTEMPT, step.getType());
+ Assert.assertTrue(check.getAttemptDetail(),
+ step.getAttempt().getShortName().matches(check.getAttemptDetail()));
+ Assert.assertEquals(steps[i-1].getReason(), step.getReason());
+ if (check.getErrCause() != null) {
+ Assert.assertEquals(check.getErrCause(),
+ TaskAttemptTerminationCause.valueOf(step.getAttempt().getTerminationCause()));
+ }
+ if (check.getNotesStr() != null) {
+ String notes = Joiner.on("#").join(step.getNotes());
+ for (String note : check.getNotesStr()) {
+ Assert.assertTrue(note, notes.contains(notes));
+ }
+ }
+ }
+
+ Assert.assertEquals(CriticalPathStep.EntityType.DAG_COMMIT,
+ criticalPath.get(criticalPath.size() - 1).getType());
+ break;
+ }
+ }
+
+ Assert.assertTrue(foundMatchingLength);
+
+ }
+
+ @Test (timeout=300000)
+ public void testWithATS() throws Exception {
+ usingATS = true;
+ createTezSessionATS();
+ runTests();
+ }
+
+ @Test (timeout=300000)
+ public void testWithSimpleHistory() throws Exception {
+ usingATS = false;
+ createTezSessionSimpleHistory();
+ runTests();
+ }
+
+ private void runTests() throws Exception {
+ ApplicationId appId = tezSession.getAppMasterApplicationId();
+ List<List<StepCheck[]>> stepsOptions = Lists.newArrayList();
+ // run all test dags
+ stepsOptions.add(testAttemptOfDownstreamVertexConnectedWithTwoUpstreamVerticesFailure());
+ stepsOptions.add(testInputFailureCausesRerunOfTwoVerticesWithoutExit());
+ stepsOptions.add(testMultiVersionInputFailureWithoutExit());
+ stepsOptions.add(testCascadingInputFailureWithoutExitSuccess());
+ stepsOptions.add(testTaskMultipleFailures());
+ stepsOptions.add(testBasicInputFailureWithoutExit());
+ stepsOptions.add(testBasicTaskFailure());
+ stepsOptions.add(testBasicSuccessScatterGather());
+ stepsOptions.add(testMultiVersionInputFailureWithExit());
+ stepsOptions.add(testBasicInputFailureWithExit());
+ stepsOptions.add(testInputFailureRerunCanSendOutputToTwoDownstreamVertices());
+ stepsOptions.add(testCascadingInputFailureWithExitSuccess());
+ stepsOptions.add(testInternalPreemption());
+
+ // close session to flush
+ if (tezSession != null) {
+ tezSession.stop();
+ }
+ Thread.sleep((TezConstants.TEZ_DAG_SLEEP_TIME_BEFORE_EXIT*3)/2);
+
+ // verify all dags
+ for (int i=0; i<stepsOptions.size(); ++i) {
+ verify(appId, i+1, stepsOptions.get(i));
+ }
+ }
+
+ private List<StepCheck[]> testBasicSuccessScatterGather() throws Exception {
+ Configuration testConf = new Configuration(false);
+ testConf.setInt(SimpleTestDAG.TEZ_SIMPLE_DAG_NUM_TASKS, 1);
+ StepCheck[] check = {
+ createStep("v1 : 000000_0", CriticalPathDependency.INIT_DEPENDENCY),
+ createStep("v2 : 000000_0", CriticalPathDependency.DATA_DEPENDENCY)
+ };
+ DAG dag = SimpleTestDAG.createDAG("testBasicSuccessScatterGather", testConf);
+ runDAG(dag, DAGStatus.State.SUCCEEDED);
+ return Collections.singletonList(check);
+ }
+
+ private List<StepCheck[]> testBasicTaskFailure() throws Exception {
+ Configuration testConf = new Configuration(false);
+ testConf.setInt(SimpleTestDAG.TEZ_SIMPLE_DAG_NUM_TASKS, 1);
+ testConf.setBoolean(TestProcessor.getVertexConfName(
+ TestProcessor.TEZ_FAILING_PROCESSOR_DO_FAIL, "v1"), true);
+ testConf.set(TestProcessor.getVertexConfName(
+ TestProcessor.TEZ_FAILING_PROCESSOR_FAILING_TASK_INDEX, "v1"), "0");
+ testConf.setInt(TestProcessor.getVertexConfName(
+ TestProcessor.TEZ_FAILING_PROCESSOR_FAILING_UPTO_TASK_ATTEMPT, "v1"), 0);
+
+ StepCheck[] check = {
+ createStep("v1 : 000000_0", CriticalPathDependency.INIT_DEPENDENCY),
+ createStep("v1 : 000000_1", CriticalPathDependency.RETRY_DEPENDENCY),
+ createStep("v2 : 000000_0", CriticalPathDependency.DATA_DEPENDENCY),
+ };
+ DAG dag = SimpleTestDAG.createDAG("testBasicTaskFailure", testConf);
+ runDAG(dag, DAGStatus.State.SUCCEEDED);
+ return Collections.singletonList(check);
+ }
+
+ private List<StepCheck[]> testTaskMultipleFailures() throws Exception {
+ Configuration testConf = new Configuration(false);
+ testConf.setInt(SimpleTestDAG.TEZ_SIMPLE_DAG_NUM_TASKS, 1);
+ testConf.setBoolean(TestProcessor.getVertexConfName(
+ TestProcessor.TEZ_FAILING_PROCESSOR_DO_FAIL, "v1"), true);
+ testConf.set(TestProcessor.getVertexConfName(
+ TestProcessor.TEZ_FAILING_PROCESSOR_FAILING_TASK_INDEX, "v1"), "0");
+ testConf.setInt(TestProcessor.getVertexConfName(
+ TestProcessor.TEZ_FAILING_PROCESSOR_FAILING_UPTO_TASK_ATTEMPT, "v1"), 1);
+
+ StepCheck[] check = {
+ createStep("v1 : 000000_0", CriticalPathDependency.INIT_DEPENDENCY),
+ createStep("v1 : 000000_1", CriticalPathDependency.RETRY_DEPENDENCY),
+ createStep("v1 : 000000_2", CriticalPathDependency.RETRY_DEPENDENCY),
+ createStep("v2 : 000000_0", CriticalPathDependency.DATA_DEPENDENCY),
+ };
+
+ DAG dag = SimpleTestDAG.createDAG("testTaskMultipleFailures", testConf);
+ runDAG(dag, DAGStatus.State.SUCCEEDED);
+ return Collections.singletonList(check);
+ }
+
+ private List<StepCheck[]> testBasicInputFailureWithExit() throws Exception {
+ Configuration testConf = new Configuration(false);
+ testConf.setInt(SimpleTestDAG.TEZ_SIMPLE_DAG_NUM_TASKS, 1);
+ testConf.setBoolean(TestInput.getVertexConfName(
+ TestInput.TEZ_FAILING_INPUT_DO_FAIL, "v2"), true);
+ testConf.setBoolean(TestInput.getVertexConfName(
+ TestInput.TEZ_FAILING_INPUT_DO_FAIL_AND_EXIT, "v2"), true);
+ testConf.set(TestInput.getVertexConfName(
+ TestInput.TEZ_FAILING_INPUT_FAILING_TASK_INDEX, "v2"), "0");
+ testConf.set(TestInput.getVertexConfName(
+ TestInput.TEZ_FAILING_INPUT_FAILING_TASK_ATTEMPT, "v2"), "0");
+ testConf.set(TestInput.getVertexConfName(
+ TestInput.TEZ_FAILING_INPUT_FAILING_INPUT_INDEX, "v2"), "0");
+
+ StepCheck[] check = {
+ createStep("v1 : 000000_0", CriticalPathDependency.INIT_DEPENDENCY),
+ createStep("v2 : 000000_0", CriticalPathDependency.DATA_DEPENDENCY),
+ createStep("v1 : 000000_1", CriticalPathDependency.OUTPUT_RECREATE_DEPENDENCY),
+ createStep("v2 : 000000_1", CriticalPathDependency.DATA_DEPENDENCY),
+ };
+
+ DAG dag = SimpleTestDAG.createDAG("testBasicInputFailureWithExit", testConf);
+ runDAG(dag, DAGStatus.State.SUCCEEDED);
+ return Collections.singletonList(check);
+ }
+
+ private List<StepCheck[]> testBasicInputFailureWithoutExit() throws Exception {
+ Configuration testConf = new Configuration(false);
+ testConf.setInt(SimpleTestDAG.TEZ_SIMPLE_DAG_NUM_TASKS, 1);
+ testConf.setBoolean(TestInput.getVertexConfName(
+ TestInput.TEZ_FAILING_INPUT_DO_FAIL, "v2"), true);
+ testConf.set(TestInput.getVertexConfName(
+ TestInput.TEZ_FAILING_INPUT_FAILING_TASK_INDEX, "v2"), "0");
+ testConf.set(TestInput.getVertexConfName(
+ TestInput.TEZ_FAILING_INPUT_FAILING_TASK_ATTEMPT, "v2"), "0");
+ testConf.set(TestInput.getVertexConfName(
+ TestInput.TEZ_FAILING_INPUT_FAILING_INPUT_INDEX, "v2"), "0");
+
+ StepCheck[] check = {
+ createStep("v1 : 000000_0", CriticalPathDependency.INIT_DEPENDENCY),
+ createStep("v2 : 000000_0", CriticalPathDependency.DATA_DEPENDENCY),
+ createStep("v1 : 000000_1", CriticalPathDependency.OUTPUT_RECREATE_DEPENDENCY),
+ createStep("v2 : 000000_0", CriticalPathDependency.DATA_DEPENDENCY),
+ };
+
+ DAG dag = SimpleTestDAG.createDAG("testBasicInputFailureWithoutExit", testConf);
+ runDAG(dag, DAGStatus.State.SUCCEEDED);
+ return Collections.singletonList(check);
+ }
+
+ private List<StepCheck[]> testMultiVersionInputFailureWithExit() throws Exception {
+ Configuration testConf = new Configuration(false);
+ testConf.setInt(SimpleTestDAG.TEZ_SIMPLE_DAG_NUM_TASKS, 1);
+ testConf.setBoolean(TestInput.getVertexConfName(
+ TestInput.TEZ_FAILING_INPUT_DO_FAIL, "v2"), true);
+ testConf.setBoolean(TestInput.getVertexConfName(
+ TestInput.TEZ_FAILING_INPUT_DO_FAIL_AND_EXIT, "v2"), true);
+ testConf.set(TestInput.getVertexConfName(
+ TestInput.TEZ_FAILING_INPUT_FAILING_TASK_INDEX, "v2"), "0");
+ testConf.set(TestInput.getVertexConfName(
+ TestInput.TEZ_FAILING_INPUT_FAILING_TASK_ATTEMPT, "v2"), "0,1");
+ testConf.set(TestInput.getVertexConfName(
+ TestInput.TEZ_FAILING_INPUT_FAILING_INPUT_INDEX, "v2"), "0");
+ testConf.setInt(TestInput.getVertexConfName(
+ TestInput.TEZ_FAILING_INPUT_FAILING_UPTO_INPUT_ATTEMPT, "v2"), 1);
+
+ StepCheck[] check = {
+ createStep("v1 : 000000_0", CriticalPathDependency.INIT_DEPENDENCY),
+ createStep("v2 : 000000_0", CriticalPathDependency.DATA_DEPENDENCY),
+ createStep("v1 : 000000_1", CriticalPathDependency.OUTPUT_RECREATE_DEPENDENCY),
+ createStep("v2 : 000000_1", CriticalPathDependency.DATA_DEPENDENCY),
+ createStep("v1 : 000000_2", CriticalPathDependency.OUTPUT_RECREATE_DEPENDENCY),
+ createStep("v2 : 000000_2", CriticalPathDependency.DATA_DEPENDENCY),
+ };
+
+ DAG dag = SimpleTestDAG.createDAG("testMultiVersionInputFailureWithExit", testConf);
+ runDAG(dag, DAGStatus.State.SUCCEEDED);
+ return Collections.singletonList(check);
+ }
+
+ private List<StepCheck[]> testMultiVersionInputFailureWithoutExit() throws Exception {
+ Configuration testConf = new Configuration(false);
+ testConf.setInt(SimpleTestDAG.TEZ_SIMPLE_DAG_NUM_TASKS, 1);
+ testConf.setBoolean(TestInput.getVertexConfName(
+ TestInput.TEZ_FAILING_INPUT_DO_FAIL, "v2"), true);
+ testConf.set(TestInput.getVertexConfName(
+ TestInput.TEZ_FAILING_INPUT_FAILING_TASK_INDEX, "v2"), "0");
+ testConf.set(TestInput.getVertexConfName(
+ TestInput.TEZ_FAILING_INPUT_FAILING_TASK_ATTEMPT, "v2"), "0");
+ testConf.set(TestInput.getVertexConfName(
+ TestInput.TEZ_FAILING_INPUT_FAILING_INPUT_INDEX, "v2"), "0");
+ testConf.setInt(TestInput.getVertexConfName(
+ TestInput.TEZ_FAILING_INPUT_FAILING_UPTO_INPUT_ATTEMPT, "v2"), 1);
+
+ StepCheck[] check = {
+ createStep("v1 : 000000_0", CriticalPathDependency.INIT_DEPENDENCY),
+ createStep("v2 : 000000_0", CriticalPathDependency.DATA_DEPENDENCY),
+ createStep("v1 : 000000_1", CriticalPathDependency.OUTPUT_RECREATE_DEPENDENCY),
+ createStep("v2 : 000000_0", CriticalPathDependency.DATA_DEPENDENCY),
+ createStep("v1 : 000000_2", CriticalPathDependency.OUTPUT_RECREATE_DEPENDENCY),
+ createStep("v2 : 000000_0", CriticalPathDependency.DATA_DEPENDENCY),
+ };
+
+ DAG dag = SimpleTestDAG.createDAG("testMultiVersionInputFailureWithoutExit", testConf);
+ runDAG(dag, DAGStatus.State.SUCCEEDED);
+ return Collections.singletonList(check);
+ }
+
+ /**
+ * Sets configuration for cascading input failure tests that
+ * use SimpleTestDAG3Vertices.
+ * @param testConf configuration
+ * @param failAndExit whether input failure should trigger attempt exit
+ */
+ private void setCascadingInputFailureConfig(Configuration testConf,
+ boolean failAndExit,
+ int numTasks) {
+ // v2 attempt0 succeeds.
+ // v2 all tasks attempt1 input0 fail up to version 0.
+ testConf.setInt(SimpleTestDAG3Vertices.TEZ_SIMPLE_DAG_NUM_TASKS, numTasks);
+ testConf.setBoolean(TestInput.getVertexConfName(
+ TestInput.TEZ_FAILING_INPUT_DO_FAIL, "v2"), true);
+ testConf.setBoolean(TestInput.getVertexConfName(
+ TestInput.TEZ_FAILING_INPUT_DO_FAIL_AND_EXIT, "v2"), failAndExit);
+ testConf.set(TestInput.getVertexConfName(
+ TestInput.TEZ_FAILING_INPUT_FAILING_TASK_INDEX, "v2"), "-1");
+ testConf.set(TestInput.getVertexConfName(
+ TestInput.TEZ_FAILING_INPUT_FAILING_TASK_ATTEMPT, "v2"), "1");
+ testConf.set(TestInput.getVertexConfName(
+ TestInput.TEZ_FAILING_INPUT_FAILING_INPUT_INDEX, "v2"), "0");
+ testConf.setInt(TestInput.getVertexConfName(
+ TestInput.TEZ_FAILING_INPUT_FAILING_UPTO_INPUT_ATTEMPT, "v2"),
+ 0);
+
+ //v3 task0 attempt0 all inputs fails up to version 0.
+ testConf.setBoolean(TestInput.getVertexConfName(
+ TestInput.TEZ_FAILING_INPUT_DO_FAIL, "v3"), true);
+ testConf.setBoolean(TestInput.getVertexConfName(
+ TestInput.TEZ_FAILING_INPUT_DO_FAIL_AND_EXIT, "v3"), failAndExit);
+ testConf.set(TestInput.getVertexConfName(
+ TestInput.TEZ_FAILING_INPUT_FAILING_TASK_INDEX, "v3"), "0");
+ testConf.set(TestInput.getVertexConfName(
+ TestInput.TEZ_FAILING_INPUT_FAILING_TASK_ATTEMPT, "v3"), "0");
+ testConf.set(TestInput.getVertexConfName(
+ TestInput.TEZ_FAILING_INPUT_FAILING_INPUT_INDEX, "v3"), "-1");
+ testConf.setInt(TestInput.getVertexConfName(
+ TestInput.TEZ_FAILING_INPUT_FAILING_UPTO_INPUT_ATTEMPT, "v3"),
+ 0);
+ }
+
+ /**
+ * Test cascading input failure without exit. Expecting success.
+ * v1 -- v2 -- v3
+ * v3 all-tasks attempt0 input0 fails. Wait. Triggering v2 rerun.
+ * v2 task0 attempt1 input0 fails. Wait. Triggering v1 rerun.
+ * v1 attempt1 rerun and succeeds. v2 accepts v1 attempt1 output. v2 attempt1 succeeds.
+ * v3 attempt0 accepts v2 attempt1 output.
+ *
+ * AM vertex succeeded order is v1, v2, v1, v2, v3.
+ * @throws Exception
+ */
+ private List<StepCheck[]> testCascadingInputFailureWithoutExitSuccess() throws Exception {
+ Configuration testConf = new Configuration(false);
+ setCascadingInputFailureConfig(testConf, false, 1);
+
+ StepCheck[] check = {
+ createStep("v1 : 000000_0", CriticalPathDependency.INIT_DEPENDENCY),
+ createStep("v2 : 000000_0", CriticalPathDependency.DATA_DEPENDENCY),
+ createStep("v3 : 000000_0", CriticalPathDependency.DATA_DEPENDENCY),
+ createStep("v2 : 000000_1", CriticalPathDependency.OUTPUT_RECREATE_DEPENDENCY),
+ createStep("v1 : 000000_1", CriticalPathDependency.OUTPUT_RECREATE_DEPENDENCY),
+ createStep("v2 : 000000_1", CriticalPathDependency.DATA_DEPENDENCY),
+ createStep("v3 : 000000_0", CriticalPathDependency.DATA_DEPENDENCY),
+ };
+
+ DAG dag = SimpleTestDAG3Vertices.createDAG(
+ "testCascadingInputFailureWithoutExitSuccess", testConf);
+ runDAG(dag, DAGStatus.State.SUCCEEDED);
+ return Collections.singletonList(check);
+ }
+
+ /**
+ * Test cascading input failure with exit. Expecting success.
+ * v1 -- v2 -- v3
+ * v3 all-tasks attempt0 input0 fails. v3 attempt0 exits. Triggering v2 rerun.
+ * v2 task0 attempt1 input0 fails. v2 attempt1 exits. Triggering v1 rerun.
+ * v1 attempt1 rerun and succeeds. v2 accepts v1 attempt1 output. v2 attempt2 succeeds.
+ * v3 attempt1 accepts v2 attempt2 output.
+ *
+ * AM vertex succeeded order is v1, v2, v3, v1, v2, v3.
+ * @throws Exception
+ */
+ private List<StepCheck[]> testCascadingInputFailureWithExitSuccess() throws Exception {
+ Configuration testConf = new Configuration(false);
+ setCascadingInputFailureConfig(testConf, true, 1);
+
+ StepCheck[] check = {
+ createStep("v1 : 000000_0", CriticalPathDependency.INIT_DEPENDENCY),
+ createStep("v2 : 000000_0", CriticalPathDependency.DATA_DEPENDENCY),
+ createStep("v3 : 000000_0", CriticalPathDependency.DATA_DEPENDENCY),
+ createStep("v2 : 000000_1", CriticalPathDependency.OUTPUT_RECREATE_DEPENDENCY),
+ createStep("v1 : 000000_1", CriticalPathDependency.OUTPUT_RECREATE_DEPENDENCY),
+ createStep("v2 : 000000_2", CriticalPathDependency.DATA_DEPENDENCY),
+ createStep("v3 : 000000_1", CriticalPathDependency.DATA_DEPENDENCY),
+ };
+
+ DAG dag = SimpleTestDAG3Vertices.createDAG(
+ "testCascadingInputFailureWithExitSuccess", testConf);
+ runDAG(dag, DAGStatus.State.SUCCEEDED);
+ return Collections.singletonList(check);
+ }
+
+ /**
+ * 1 NM is running and can run 4 containers based on YARN mini cluster defaults and
+ * Tez defaults for AM/task memory
+ * v3 task0 reports read errors against both tasks of v2. This re-starts both of them.
+ * Now all 4 slots are occupied 1 AM + 3 tasks
+ * Now retries of v2 report read error against 1 task of v1. That re-starts.
+ * Retry of v1 task has no space - so it preempts the least priority task (current tez logic)
+ * v3 is preempted and re-run. Shows up on critical path as preempted failure.
+ * Also v1 retry attempts note show that it caused preemption of v3
+ * @throws Exception
+ */
+ private List<StepCheck[]> testInternalPreemption() throws Exception {
+ Configuration testConf = new Configuration(false);
+ setCascadingInputFailureConfig(testConf, false, 2);
+
+ StepCheck[] check = {
+ createStep("v1 : 00000[01]_0", CriticalPathDependency.INIT_DEPENDENCY),
+ createStep("v2 : 00000[01]_0", CriticalPathDependency.DATA_DEPENDENCY),
+ createStep("v3 : 000000_0", CriticalPathDependency.DATA_DEPENDENCY,
+ TaskAttemptTerminationCause.INTERNAL_PREEMPTION, null),
+ createStep("v2 : 00000[01]_1", CriticalPathDependency.OUTPUT_RECREATE_DEPENDENCY),
+ createStep("v1 : 000000_1", CriticalPathDependency.OUTPUT_RECREATE_DEPENDENCY,
+ null, Collections.singletonList("preemption of v3")),
+ createStep("v2 : 00000[01]_1", CriticalPathDependency.DATA_DEPENDENCY),
+ createStep("v3 : 000000_1", CriticalPathDependency.DATA_DEPENDENCY)
+ };
+
+ DAG dag = SimpleTestDAG3Vertices.createDAG(
+ "testInternalPreemption", testConf);
+ runDAG(dag, DAGStatus.State.SUCCEEDED);
+ return Collections.singletonList(check);
+ }
+
+ /**
+ * Input failure of v3 causes rerun of both both v1 and v2 vertices.
+ * v1 v2
+ * \ /
+ * v3
+ *
+ * @throws Exception
+ */
+ private List<StepCheck[]> testInputFailureCausesRerunOfTwoVerticesWithoutExit() throws Exception {
+ Configuration testConf = new Configuration(false);
+ testConf.setInt(SimpleVTestDAG.TEZ_SIMPLE_V_DAG_NUM_TASKS, 1);
+ testConf.setBoolean(TestInput.getVertexConfName(
+ TestInput.TEZ_FAILING_INPUT_DO_FAIL, "v3"), true);
+ testConf.setBoolean(TestInput.getVertexConfName(
+ TestInput.TEZ_FAILING_INPUT_DO_FAIL_AND_EXIT, "v3"), false);
+ testConf.set(TestInput.getVertexConfName(
+ TestInput.TEZ_FAILING_INPUT_FAILING_TASK_INDEX, "v3"), "0");
+ testConf.set(TestInput.getVertexConfName(
+ TestInput.TEZ_FAILING_INPUT_FAILING_TASK_ATTEMPT, "v3"), "0");
+ testConf.set(TestInput.getVertexConfName(
+ TestInput.TEZ_FAILING_INPUT_FAILING_INPUT_INDEX, "v3"), "-1");
+ testConf.set(TestInput.getVertexConfName(
+ TestInput.TEZ_FAILING_INPUT_FAILING_UPTO_INPUT_ATTEMPT, "v3"), "1");
+
+ StepCheck[] check = {
+ // use regex for either vertices being possible on the path
+ createStep("v[12] : 000000_0", CriticalPathDependency.INIT_DEPENDENCY),
+ createStep("v3 : 000000_0", CriticalPathDependency.DATA_DEPENDENCY),
+ createStep("v[12] : 000000_[01]", CriticalPathDependency.OUTPUT_RECREATE_DEPENDENCY),
+ createStep("v3 : 000000_0", CriticalPathDependency.DATA_DEPENDENCY),
+ createStep("v[12] : 000000_[012]", CriticalPathDependency.OUTPUT_RECREATE_DEPENDENCY),
+ createStep("v3 : 000000_0", CriticalPathDependency.DATA_DEPENDENCY),
+ createStep("v[12] : 000000_[12]", CriticalPathDependency.OUTPUT_RECREATE_DEPENDENCY),
+ createStep("v3 : 000000_0", CriticalPathDependency.DATA_DEPENDENCY),
+ createStep("v[12] : 000000_2", CriticalPathDependency.OUTPUT_RECREATE_DEPENDENCY),
+ createStep("v3 : 000000_0", CriticalPathDependency.DATA_DEPENDENCY),
+ };
+
+ DAG dag = SimpleVTestDAG.createDAG(
+ "testInputFailureCausesRerunOfTwoVerticesWithoutExit", testConf);
+ runDAG(dag, DAGStatus.State.SUCCEEDED);
+ return Collections.singletonList(check);
+ }
+
+ /**
+ * Downstream(v3) attempt failure of a vertex connected with
+ * 2 upstream vertices..
+ * v1 v2
+ * \ /
+ * v3
+ *
+ * @throws Exception
+ */
+ private List<StepCheck[]> testAttemptOfDownstreamVertexConnectedWithTwoUpstreamVerticesFailure()
+ throws Exception {
+ Configuration testConf = new Configuration(false);
+ testConf.setInt(SimpleVTestDAG.TEZ_SIMPLE_V_DAG_NUM_TASKS, 1);
+ testConf.setBoolean(TestProcessor.getVertexConfName(
+ TestProcessor.TEZ_FAILING_PROCESSOR_DO_FAIL, "v3"), true);
+ testConf.set(TestProcessor.getVertexConfName(
+ TestProcessor.TEZ_FAILING_PROCESSOR_FAILING_TASK_INDEX, "v3"), "0");
+ testConf.setInt(TestProcessor.getVertexConfName(
+ TestProcessor.TEZ_FAILING_PROCESSOR_FAILING_UPTO_TASK_ATTEMPT, "v3"), 1);
+
+ StepCheck[] check = {
+ // use regex for either vertices being possible on the path
+ createStep("v[12] : 000000_0", CriticalPathDependency.INIT_DEPENDENCY),
+ createStep("v3 : 000000_0", CriticalPathDependency.DATA_DEPENDENCY),
+ createStep("v3 : 000000_1", CriticalPathDependency.RETRY_DEPENDENCY),
+ createStep("v3 : 000000_2", CriticalPathDependency.RETRY_DEPENDENCY),
+ };
+
+ DAG dag = SimpleVTestDAG.createDAG(
+ "testAttemptOfDownstreamVertexConnectedWithTwoUpstreamVerticesFailure", testConf);
+ runDAG(dag, DAGStatus.State.SUCCEEDED);
+ return Collections.singletonList(check);
+ }
+
+ /**
+ * Input failure of v2,v3 trigger v1 rerun.
+ * Both v2 and v3 report error on v1 and dont exit. So one of them triggers next
+ * version of v1 and also consume the output of the next version. While the other
+ * consumes the output of the next version of v1.
+ * Reruns can send output to 2 downstream vertices.
+ * v1
+ * / \
+ * v2 v3
+ *
+ * Also covers multiple consumer vertices report failure against same producer task.
+ * @throws Exception
+ */
+ private List<StepCheck[]> testInputFailureRerunCanSendOutputToTwoDownstreamVertices() throws Exception {
+ Configuration testConf = new Configuration(false);
+ testConf.setInt(SimpleReverseVTestDAG.TEZ_SIMPLE_REVERSE_V_DAG_NUM_TASKS, 1);
+ testConf.setBoolean(TestInput.getVertexConfName(
+ TestInput.TEZ_FAILING_INPUT_DO_FAIL, "v2"), true);
+ testConf.setBoolean(TestInput.getVertexConfName(
+ TestInput.TEZ_FAILING_INPUT_DO_FAIL_AND_EXIT, "v2"), false);
+ testConf.set(TestInput.getVertexConfName(
+ TestInput.TEZ_FAILING_INPUT_FAILING_TASK_INDEX, "v2"), "-1");
+ testConf.set(TestInput.getVertexConfName(
+ TestInput.TEZ_FAILING_INPUT_FAILING_TASK_ATTEMPT, "v2"), "0");
+ testConf.set(TestInput.getVertexConfName(
+ TestInput.TEZ_FAILING_INPUT_FAILING_INPUT_INDEX, "v2"), "-1");
+ testConf.set(TestInput.getVertexConfName(
+ TestInput.TEZ_FAILING_INPUT_FAILING_UPTO_INPUT_ATTEMPT, "v2"), "0");
+
+ testConf.setBoolean(TestInput.getVertexConfName(
+ TestInput.TEZ_FAILING_INPUT_DO_FAIL, "v3"), true);
+ testConf.setBoolean(TestInput.getVertexConfName(
+ TestInput.TEZ_FAILING_INPUT_DO_FAIL_AND_EXIT, "v3"), false);
+ testConf.set(TestInput.getVertexConfName(
+ TestInput.TEZ_FAILING_INPUT_FAILING_TASK_INDEX, "v3"), "-1");
+ testConf.set(TestInput.getVertexConfName(
+ TestInput.TEZ_FAILING_INPUT_FAILING_TASK_ATTEMPT, "v3"), "0");
+ testConf.set(TestInput.getVertexConfName(
+ TestInput.TEZ_FAILING_INPUT_FAILING_INPUT_INDEX, "v3"), "-1");
+ testConf.set(TestInput.getVertexConfName(
+ TestInput.TEZ_FAILING_INPUT_FAILING_UPTO_INPUT_ATTEMPT, "v3"), "0");
+
+ StepCheck[] check = {
+ // use regex for either vertices being possible on the path
+ createStep("v1 : 000000_0", CriticalPathDependency.INIT_DEPENDENCY),
+ createStep("v[23] : 000000_0", CriticalPathDependency.DATA_DEPENDENCY),
+ createStep("v1 : 000000_1", CriticalPathDependency.OUTPUT_RECREATE_DEPENDENCY),
+ createStep("v[23] : 000000_0", CriticalPathDependency.DATA_DEPENDENCY),
+ };
+ DAG dag = SimpleReverseVTestDAG.createDAG(
+ "testInputFailureRerunCanSendOutputToTwoDownstreamVertices", testConf);
+ runDAG(dag, DAGStatus.State.SUCCEEDED);
+ return Collections.singletonList(check);
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/8c8db7c5/tez-tools/analyzers/pom.xml
----------------------------------------------------------------------
diff --git a/tez-tools/analyzers/pom.xml b/tez-tools/analyzers/pom.xml
new file mode 100644
index 0000000..afc70a6
--- /dev/null
+++ b/tez-tools/analyzers/pom.xml
@@ -0,0 +1,51 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ 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. See accompanying LICENSE file.
+-->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+ xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+ <modelVersion>4.0.0</modelVersion>
+ <parent>
+ <groupId>org.apache.tez</groupId>
+ <artifactId>tez-tools</artifactId>
+ <version>0.7.1-SNAPSHOT</version>
+ </parent>
+ <artifactId>tez-perf-analyzer</artifactId>
+ <packaging>pom</packaging>
+
+ <profiles>
+ <profile>
+ <id>hadoop24</id>
+ <activation>
+ <activeByDefault>false</activeByDefault>
+ </activation>
+ <modules>
+ <module>job-analyzer</module>
+ </modules>
+ </profile>
+ <profile>
+ <id>hadoop26</id>
+ <activation>
+ <property>
+ <name>!skipATS</name>
+ </property>
+ </activation>
+ <modules>
+ <module>job-analyzer</module>
+ </modules>
+ </profile>
+ </profiles>
+
+</project>
http://git-wip-us.apache.org/repos/asf/tez/blob/8c8db7c5/tez-tools/pom.xml
----------------------------------------------------------------------
diff --git a/tez-tools/pom.xml b/tez-tools/pom.xml
index f487fcb..1d6f929 100644
--- a/tez-tools/pom.xml
+++ b/tez-tools/pom.xml
@@ -26,6 +26,10 @@
<artifactId>tez-tools</artifactId>
<packaging>pom</packaging>
+ <modules>
+ <module>analyzers</module>
+ </modules>
+
<build>
<plugins>
<plugin>
[5/5] tez git commit: TEZ-2973. Backport Analyzers to branch-0.7
Posted by je...@apache.org.
TEZ-2973. Backport Analyzers to branch-0.7
Project: http://git-wip-us.apache.org/repos/asf/tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/8c8db7c5
Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/8c8db7c5
Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/8c8db7c5
Branch: refs/heads/branch-0.7
Commit: 8c8db7c516d0cd1ca256cddbe76bbe306e23f9c2
Parents: 868ca53
Author: Jonathan Eagles <je...@yahoo-inc.com>
Authored: Mon Dec 7 16:41:00 2015 -0600
Committer: Jonathan Eagles <je...@yahoo-inc.com>
Committed: Mon Dec 7 16:41:00 2015 -0600
----------------------------------------------------------------------
CHANGES.txt | 1 +
pom.xml | 5 +
.../tez/common/counters/FileSystemCounter.java | 4 +
.../java/org/apache/tez/dag/utils/Graph.java | 15 +-
tez-plugins/pom.xml | 2 +
.../tez-history-parser/findbugs-exclude.xml | 16 +
tez-plugins/tez-history-parser/pom.xml | 190 +++++
.../org/apache/tez/history/ATSImportTool.java | 463 +++++++++++
.../org/apache/tez/history/parser/ATSData.java | 48 ++
.../tez/history/parser/ATSFileParser.java | 227 +++++
.../tez/history/parser/SimpleHistoryParser.java | 242 ++++++
.../datamodel/AdditionalInputOutputDetails.java | 71 ++
.../tez/history/parser/datamodel/BaseInfo.java | 142 ++++
.../history/parser/datamodel/BaseParser.java | 114 +++
.../tez/history/parser/datamodel/Constants.java | 64 ++
.../tez/history/parser/datamodel/Container.java | 70 ++
.../tez/history/parser/datamodel/DagInfo.java | 586 +++++++++++++
.../tez/history/parser/datamodel/EdgeInfo.java | 112 +++
.../tez/history/parser/datamodel/Event.java | 63 ++
.../parser/datamodel/TaskAttemptInfo.java | 379 +++++++++
.../tez/history/parser/datamodel/TaskInfo.java | 354 ++++++++
.../history/parser/datamodel/VersionInfo.java | 45 +
.../history/parser/datamodel/VertexInfo.java | 636 ++++++++++++++
.../apache/tez/history/parser/utils/Utils.java | 139 ++++
.../apache/tez/history/TestHistoryParser.java | 813 ++++++++++++++++++
.../analyzers/job-analyzer/findbugs-exclude.xml | 28 +
tez-tools/analyzers/job-analyzer/pom.xml | 168 ++++
.../java/org/apache/tez/analyzer/Analyzer.java | 64 ++
.../java/org/apache/tez/analyzer/CSVResult.java | 115 +++
.../java/org/apache/tez/analyzer/Result.java | 39 +
.../tez/analyzer/plugins/AnalyzerDriver.java | 59 ++
.../plugins/ContainerReuseAnalyzer.java | 97 +++
.../analyzer/plugins/CriticalPathAnalyzer.java | 646 +++++++++++++++
.../tez/analyzer/plugins/LocalityAnalyzer.java | 204 +++++
.../analyzer/plugins/ShuffleTimeAnalyzer.java | 223 +++++
.../tez/analyzer/plugins/SkewAnalyzer.java | 323 ++++++++
.../tez/analyzer/plugins/SlowNodeAnalyzer.java | 197 +++++
.../analyzer/plugins/SlowTaskIdentifier.java | 126 +++
.../analyzer/plugins/SlowestVertexAnalyzer.java | 219 +++++
.../tez/analyzer/plugins/SpillAnalyzerImpl.java | 145 ++++
.../plugins/TaskConcurrencyAnalyzer.java | 148 ++++
.../tez/analyzer/plugins/TezAnalyzerBase.java | 213 +++++
.../VertexLevelCriticalPathAnalyzer.java | 152 ++++
.../org/apache/tez/analyzer/utils/SVGUtils.java | 334 ++++++++
.../org/apache/tez/analyzer/utils/Utils.java | 100 +++
.../org/apache/tez/analyzer/TestAnalyzer.java | 823 +++++++++++++++++++
tez-tools/analyzers/pom.xml | 51 ++
tez-tools/pom.xml | 4 +
48 files changed, 9275 insertions(+), 4 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/tez/blob/8c8db7c5/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 7fb7957..8c23aa6 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -8,6 +8,7 @@ INCOMPATIBLE CHANGES
TEZ-2949. Allow duplicate dag names within session for Tez.
ALL CHANGES
+ TEZ-2973. Backport Analyzers to branch-0.7
TEZ-2975. Bump up apache commons dependency.
TEZ-2970. Re-localization in TezChild does not use correct UGI.
TEZ-2968. Counter limits exception causes AM to crash.
http://git-wip-us.apache.org/repos/asf/tez/blob/8c8db7c5/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 3b7aaf4..169353f 100644
--- a/pom.xml
+++ b/pom.xml
@@ -176,6 +176,11 @@
<version>${pig.version}</version>
</dependency>
<dependency>
+ <groupId>io.dropwizard.metrics</groupId>
+ <artifactId>metrics-core</artifactId>
+ <version>3.1.0</version>
+ </dependency>
+ <dependency>
<groupId>org.slf4j</groupId>
<artifactId>slf4j-api</artifactId>
<version>1.7.5</version>
http://git-wip-us.apache.org/repos/asf/tez/blob/8c8db7c5/tez-api/src/main/java/org/apache/tez/common/counters/FileSystemCounter.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/common/counters/FileSystemCounter.java b/tez-api/src/main/java/org/apache/tez/common/counters/FileSystemCounter.java
index 57d1053..73e3581 100644
--- a/tez-api/src/main/java/org/apache/tez/common/counters/FileSystemCounter.java
+++ b/tez-api/src/main/java/org/apache/tez/common/counters/FileSystemCounter.java
@@ -27,4 +27,8 @@ public enum FileSystemCounter {
READ_OPS,
LARGE_READ_OPS,
WRITE_OPS,
+ HDFS_BYTES_READ,
+ HDFS_BYTES_WRITTEN,
+ FILE_BYTES_READ,
+ FILE_BYTES_WRITTEN
}
http://git-wip-us.apache.org/repos/asf/tez/blob/8c8db7c5/tez-dag/src/main/java/org/apache/tez/dag/utils/Graph.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/utils/Graph.java b/tez-dag/src/main/java/org/apache/tez/dag/utils/Graph.java
index cc9033d..1d8e395 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/utils/Graph.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/utils/Graph.java
@@ -62,6 +62,7 @@ public class Graph {
List<Edge> outs;
String label;
String shape;
+ String color;
public Node(String id) {
this(id, null);
@@ -104,6 +105,10 @@ public class Graph {
public void setShape(String shape) {
this.shape = shape;
}
+
+ public void setColor(String color) {
+ this.color = color;
+ }
}
private String name;
@@ -196,17 +201,19 @@ public class Graph {
for (Node n : nodes) {
if (n.shape != null && !n.shape.isEmpty()) {
sb.append(String.format(
- "%s%s [ label = %s, shape = %s ];",
+ "%s%s [ label = %s, shape = %s , color= %s];",
indent,
wrapSafeString(n.getUniqueId()),
wrapSafeString(n.getLabel()),
- wrapSafeString(n.shape)));
+ wrapSafeString(n.shape),
+ wrapSafeString(n.color == null ? "black" : n.color)));
} else {
sb.append(String.format(
- "%s%s [ label = %s ];",
+ "%s%s [ label = %s , color= %s ];",
indent,
wrapSafeString(n.getUniqueId()),
- wrapSafeString(n.getLabel())));
+ wrapSafeString(n.getLabel()),
+ wrapSafeString(n.color == null ? "black" : n.color)));
}
sb.append(System.getProperty("line.separator"));
List<Edge> combinedOuts = combineEdges(n.outs);
http://git-wip-us.apache.org/repos/asf/tez/blob/8c8db7c5/tez-plugins/pom.xml
----------------------------------------------------------------------
diff --git a/tez-plugins/pom.xml b/tez-plugins/pom.xml
index 89b9d1e..fe7c61a 100644
--- a/tez-plugins/pom.xml
+++ b/tez-plugins/pom.xml
@@ -34,6 +34,7 @@
</activation>
<modules>
<module>tez-yarn-timeline-history</module>
+ <module>tez-history-parser</module>
</modules>
</profile>
<profile>
@@ -46,6 +47,7 @@
<modules>
<module>tez-yarn-timeline-history</module>
<module>tez-yarn-timeline-history-with-acls</module>
+ <module>tez-history-parser</module>
</modules>
</profile>
</profiles>
http://git-wip-us.apache.org/repos/asf/tez/blob/8c8db7c5/tez-plugins/tez-history-parser/findbugs-exclude.xml
----------------------------------------------------------------------
diff --git a/tez-plugins/tez-history-parser/findbugs-exclude.xml b/tez-plugins/tez-history-parser/findbugs-exclude.xml
new file mode 100644
index 0000000..5b11308
--- /dev/null
+++ b/tez-plugins/tez-history-parser/findbugs-exclude.xml
@@ -0,0 +1,16 @@
+<!--
+ 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. See accompanying LICENSE file.
+-->
+<FindBugsFilter>
+
+</FindBugsFilter>
http://git-wip-us.apache.org/repos/asf/tez/blob/8c8db7c5/tez-plugins/tez-history-parser/pom.xml
----------------------------------------------------------------------
diff --git a/tez-plugins/tez-history-parser/pom.xml b/tez-plugins/tez-history-parser/pom.xml
new file mode 100644
index 0000000..7dce608
--- /dev/null
+++ b/tez-plugins/tez-history-parser/pom.xml
@@ -0,0 +1,190 @@
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements. See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License. You may obtain a copy of the License at
+
+ http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+ xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+ <modelVersion>4.0.0</modelVersion>
+ <parent>
+ <groupId>org.apache.tez</groupId>
+ <artifactId>tez-plugins</artifactId>
+ <version>0.7.1-SNAPSHOT</version>
+ </parent>
+ <artifactId>tez-history-parser</artifactId>
+
+ <dependencies>
+ <dependency>
+ <groupId>org.apache.tez</groupId>
+ <artifactId>tez-api</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.tez</groupId>
+ <artifactId>tez-common</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.tez</groupId>
+ <artifactId>tez-dag</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.tez</groupId>
+ <artifactId>tez-tests</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.tez</groupId>
+ <artifactId>tez-tests</artifactId>
+ <type>test-jar</type>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.tez</groupId>
+ <artifactId>tez-yarn-timeline-history</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.tez</groupId>
+ <artifactId>tez-yarn-timeline-history</artifactId>
+ <scope>test</scope>
+ <type>test-jar</type>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-common</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-yarn-api</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-yarn-common</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-yarn-client</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-yarn-server-tests</artifactId>
+ <scope>test</scope>
+ <type>test-jar</type>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-hdfs</artifactId>
+ <scope>test</scope>
+ <type>test-jar</type>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-common</artifactId>
+ <scope>test</scope>
+ <type>test-jar</type>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-mapreduce-client-core</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-mapreduce-client-common</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-mapreduce-client-jobclient</artifactId>
+ <scope>test</scope>
+ <type>test-jar</type>
+ </dependency>
+ <dependency>
+ <groupId>org.codehaus.jettison</groupId>
+ <artifactId>jettison</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>com.google.guava</groupId>
+ <artifactId>guava</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.slf4j</groupId>
+ <artifactId>slf4j-api</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.mockito</groupId>
+ <artifactId>mockito-all</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>junit</groupId>
+ <artifactId>junit</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>com.sun.jersey</groupId>
+ <artifactId>jersey-json</artifactId>
+ </dependency>
+ </dependencies>
+
+ <build>
+ <plugins>
+ <plugin>
+ <groupId>org.apache.rat</groupId>
+ <artifactId>apache-rat-plugin</artifactId>
+ </plugin>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-deploy-plugin</artifactId>
+ <configuration>
+ <skip>true</skip>
+ </configuration>
+ </plugin>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-jar-plugin</artifactId>
+ <configuration>
+ <archive>
+ <manifest>
+ <mainClass>org.apache.tez.history.ATSImportTool</mainClass>
+ </manifest>
+ </archive>
+ </configuration>
+ </plugin>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-assembly-plugin</artifactId>
+ <configuration>
+ <descriptorRefs>
+ <descriptorRef>jar-with-dependencies</descriptorRef>
+ </descriptorRefs>
+ <archive>
+ <manifest>
+ <addClasspath>true</addClasspath>
+ <mainClass>org.apache.tez.history.ATSImportTool</mainClass>
+ </manifest>
+ </archive>
+ </configuration>
+ <executions>
+ <execution>
+ <id>assemble-all</id>
+ <phase>package</phase>
+ <goals>
+ <goal>single</goal>
+ </goals>
+ </execution>
+ </executions>
+ </plugin>
+ </plugins>
+ </build>
+</project>
http://git-wip-us.apache.org/repos/asf/tez/blob/8c8db7c5/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/ATSImportTool.java
----------------------------------------------------------------------
diff --git a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/ATSImportTool.java b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/ATSImportTool.java
new file mode 100644
index 0000000..3efeb5a
--- /dev/null
+++ b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/ATSImportTool.java
@@ -0,0 +1,463 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.history;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
+import com.sun.jersey.api.client.Client;
+import com.sun.jersey.api.client.ClientHandlerException;
+import com.sun.jersey.api.client.ClientResponse;
+import com.sun.jersey.api.client.UniformInterfaceException;
+import com.sun.jersey.api.client.WebResource;
+import com.sun.jersey.api.client.config.ClientConfig;
+import com.sun.jersey.api.client.config.DefaultClientConfig;
+import com.sun.jersey.client.urlconnection.HttpURLConnectionFactory;
+import com.sun.jersey.client.urlconnection.URLConnectionClientHandler;
+import com.sun.jersey.json.impl.provider.entity.JSONRootElementProvider;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.GnuParser;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.MissingOptionException;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.OptionBuilder;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.ParseException;
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.io.LineIterator;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.http.HttpConfig;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.tez.dag.api.TezException;
+import org.apache.tez.dag.records.TezDAGID;
+import org.apache.tez.history.parser.datamodel.Constants;
+import org.apache.tez.history.parser.utils.Utils;
+import org.codehaus.jettison.json.JSONArray;
+import org.codehaus.jettison.json.JSONException;
+import org.codehaus.jettison.json.JSONObject;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.ws.rs.core.MediaType;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.net.HttpURLConnection;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.net.URL;
+import java.net.URLEncoder;
+import java.util.zip.ZipEntry;
+import java.util.zip.ZipOutputStream;
+
+import static org.apache.hadoop.classification.InterfaceStability.Evolving;
+
+/**
+ * <pre>
+ * Simple tool which imports ATS data pertaining to a DAG (Dag, Vertex, Task, Attempt)
+ * and creates a zip file out of it.
+ *
+ * usage:
+ *
+ * java -cp tez-history-parser-x.y.z-jar-with-dependencies.jar org.apache.tez.history.ATSImportTool
+ *
+ * OR
+ *
+ * HADOOP_CLASSPATH=$TEZ_HOME/*:$TEZ_HOME/lib/*:$HADOOP_CLASSPATH hadoop jar tez-history-parser-x.y.z.jar org.apache.tez.history.ATSImportTool
+ *
+ *
+ * --yarnTimelineAddress <yarnTimelineAddress> Optional. Yarn Timeline Address(e.g http://clusterATSNode:8188)
+ * --batchSize <batchSize> Optional. batch size for downloading data
+ * --dagId <dagId> DagId that needs to be downloaded
+ * --downloadDir <downloadDir> download directory where data needs to be downloaded
+ * --help print help
+ *
+ * </pre>
+ */
+@Evolving
+public class ATSImportTool extends Configured implements Tool {
+
+ private static final Logger LOG = LoggerFactory.getLogger(ATSImportTool.class);
+
+ private static final String BATCH_SIZE = "batchSize";
+ private static final int BATCH_SIZE_DEFAULT = 100;
+
+ private static final String YARN_TIMELINE_SERVICE_ADDRESS = "yarnTimelineAddress";
+ private static final String DAG_ID = "dagId";
+ private static final String BASE_DOWNLOAD_DIR = "downloadDir";
+
+ private static final String HTTPS_SCHEME = "https://";
+ private static final String HTTP_SCHEME = "http://";
+
+ private static final String VERTEX_QUERY_STRING = "%s/%s?limit=%s&primaryFilter=%s:%s";
+ private static final String TASK_QUERY_STRING = "%s/%s?limit=%s&primaryFilter=%s:%s";
+ private static final String TASK_ATTEMPT_QUERY_STRING = "%s/%s?limit=%s&primaryFilter=%s:%s";
+ private static final String UTF8 = "UTF-8";
+
+ private static final String LINE_SEPARATOR = System.getProperty("line.separator");
+
+ private final int batchSize;
+ private final String baseUri;
+ private final String dagId;
+
+ private final File zipFile;
+ private final Client httpClient;
+ private final TezDAGID tezDAGID;
+
+ public ATSImportTool(String baseUri, String dagId, File downloadDir, int batchSize) {
+ Preconditions.checkArgument(!Strings.isNullOrEmpty(dagId), "dagId can not be null or empty");
+ Preconditions.checkArgument(downloadDir != null, "downloadDir can not be null");
+ tezDAGID = TezDAGID.fromString(dagId);
+
+ this.baseUri = baseUri;
+ this.batchSize = batchSize;
+ this.dagId = dagId;
+
+ this.httpClient = getHttpClient();
+
+ this.zipFile = new File(downloadDir, this.dagId + ".zip");
+
+ boolean result = downloadDir.mkdirs();
+ LOG.trace("Result of creating dir {}={}", downloadDir, result);
+ if (!downloadDir.exists()) {
+ throw new IllegalArgumentException("dir=" + downloadDir + " does not exist");
+ }
+
+ LOG.info("Using baseURL={}, dagId={}, batchSize={}, downloadDir={}", baseUri, dagId,
+ batchSize, downloadDir);
+ }
+
+ /**
+ * Download data from ATS for specific DAG
+ *
+ * @throws Exception
+ */
+ private void download() throws Exception {
+ FileOutputStream fos = null;
+ try {
+ fos = new FileOutputStream(zipFile, false);
+ ZipOutputStream zos = new ZipOutputStream(fos);
+ downloadData(zos);
+ IOUtils.closeQuietly(zos);
+ } catch (Exception e) {
+ LOG.error("Exception in download", e);
+ throw e;
+ } finally {
+ if (httpClient != null) {
+ httpClient.destroy();
+ }
+ IOUtils.closeQuietly(fos);
+ }
+ }
+
+ /**
+ * Download DAG data (DAG, Vertex, Task, TaskAttempts) from ATS and write to zip file
+ *
+ * @param zos
+ * @throws TezException
+ * @throws JSONException
+ * @throws IOException
+ */
+ private void downloadData(ZipOutputStream zos) throws TezException, JSONException, IOException {
+ JSONObject finalJson = new JSONObject();
+
+ //Download application details (TEZ_VERSION etc)
+ String tezAppId = "tez_" + tezDAGID.getApplicationId().toString();
+ String tezAppUrl = String.format("%s/%s/%s", baseUri, Constants.TEZ_APPLICATION, tezAppId);
+ JSONObject tezAppJson = getJsonRootEntity(tezAppUrl);
+ finalJson.put(Constants.APPLICATION, tezAppJson);
+
+ //Download dag
+ String dagUrl = String.format("%s/%s/%s", baseUri, Constants.TEZ_DAG_ID, dagId);
+ JSONObject dagRoot = getJsonRootEntity(dagUrl);
+ finalJson.put(Constants.DAG, dagRoot);
+
+ //Create a zip entry with dagId as its name.
+ ZipEntry zipEntry = new ZipEntry(dagId);
+ zos.putNextEntry(zipEntry);
+ //Write in formatted way
+ IOUtils.write(finalJson.toString(4), zos, UTF8);
+
+ //Download vertex
+ String vertexURL =
+ String.format(VERTEX_QUERY_STRING, baseUri,
+ Constants.TEZ_VERTEX_ID, batchSize, Constants.TEZ_DAG_ID, dagId);
+ downloadJSONArrayFromATS(vertexURL, zos, Constants.VERTICES);
+
+ //Download task
+ String taskURL = String.format(TASK_QUERY_STRING, baseUri,
+ Constants.TEZ_TASK_ID, batchSize, Constants.TEZ_DAG_ID, dagId);
+ downloadJSONArrayFromATS(taskURL, zos, Constants.TASKS);
+
+ //Download task attempts
+ String taskAttemptURL = String.format(TASK_ATTEMPT_QUERY_STRING, baseUri,
+ Constants.TEZ_TASK_ATTEMPT_ID, batchSize, Constants.TEZ_DAG_ID, dagId);
+ downloadJSONArrayFromATS(taskAttemptURL, zos, Constants.TASK_ATTEMPTS);
+ }
+
+ /**
+ * Download data from ATS in batches
+ *
+ * @param url
+ * @param zos
+ * @param tag
+ * @throws IOException
+ * @throws TezException
+ * @throws JSONException
+ */
+ private void downloadJSONArrayFromATS(String url, ZipOutputStream zos, String tag)
+ throws IOException, TezException, JSONException {
+
+ Preconditions.checkArgument(zos != null, "ZipOutputStream can not be null");
+
+ String baseUrl = url;
+ JSONArray entities;
+
+ long downloadedCount = 0;
+ while ((entities = getJsonRootEntity(url).optJSONArray(Constants.ENTITIES)) != null
+ && entities.length() > 0) {
+
+ int limit = (entities.length() >= batchSize) ? (entities.length() - 1) : entities.length();
+ LOG.debug("Limit={}, downloaded entities len={}", limit, entities.length());
+
+ //write downloaded part to zipfile. This is done to avoid any memory pressure when
+ // downloading and writing 1000s of tasks.
+ ZipEntry zipEntry = new ZipEntry("part-" + System.currentTimeMillis() + ".json");
+ zos.putNextEntry(zipEntry);
+ JSONObject finalJson = new JSONObject();
+ finalJson.put(tag, entities);
+ IOUtils.write(finalJson.toString(4), zos, "UTF-8");
+ downloadedCount += entities.length();
+
+ if (entities.length() < batchSize) {
+ break;
+ }
+
+ //Set the last item in entities as the fromId
+ url = baseUrl + "&fromId="
+ + entities.getJSONObject(entities.length() - 1).getString(Constants.ENTITY);
+
+ String firstItem = entities.getJSONObject(0).getString(Constants.ENTITY);
+ String lastItem = entities.getJSONObject(entities.length() - 1).getString(Constants.ENTITY);
+ LOG.info("Downloaded={}, First item={}, LastItem={}, new url={}", downloadedCount,
+ firstItem, lastItem, url);
+ }
+ }
+
+ private void logErrorMessage(ClientResponse response) throws IOException {
+ LOG.error("Response status={}", response.getClientResponseStatus().toString());
+ LineIterator it = null;
+ try {
+ it = IOUtils.lineIterator(response.getEntityInputStream(), UTF8);
+ while (it.hasNext()) {
+ String line = it.nextLine();
+ LOG.error(line);
+ }
+ } finally {
+ if (it != null) {
+ it.close();
+ }
+ }
+ }
+
+ //For secure cluster, this should work as long as valid ticket is available in the node.
+ private JSONObject getJsonRootEntity(String url) throws TezException, IOException {
+ try {
+ WebResource wr = getHttpClient().resource(url);
+ ClientResponse response = wr.accept(MediaType.APPLICATION_JSON_TYPE)
+ .type(MediaType.APPLICATION_JSON_TYPE)
+ .get(ClientResponse.class);
+
+ if (response.getClientResponseStatus() != ClientResponse.Status.OK) {
+ // In the case of secure cluster, if there is any auth exception it sends the data back as
+ // a html page and JSON parsing could throw exceptions. Instead, get the stream contents
+ // completely and log it in case of error.
+ logErrorMessage(response);
+ throw new TezException("Failed to get response from YARN Timeline: url: " + url);
+ }
+ return response.getEntity(JSONObject.class);
+ } catch (ClientHandlerException e) {
+ throw new TezException("Error processing response from YARN Timeline. URL=" + url, e);
+ } catch (UniformInterfaceException e) {
+ throw new TezException("Error accessing content from YARN Timeline - unexpected response. "
+ + "URL=" + url, e);
+ } catch (IllegalArgumentException e) {
+ throw new TezException("Error accessing content from YARN Timeline - invalid url. URL=" + url,
+ e);
+ }
+ }
+
+ private Client getHttpClient() {
+ if (httpClient == null) {
+ ClientConfig config = new DefaultClientConfig(JSONRootElementProvider.App.class);
+ HttpURLConnectionFactory urlFactory = new PseudoAuthenticatedURLConnectionFactory();
+ return new Client(new URLConnectionClientHandler(urlFactory), config);
+ }
+ return httpClient;
+ }
+
+ static class PseudoAuthenticatedURLConnectionFactory implements HttpURLConnectionFactory {
+ @Override
+ public HttpURLConnection getHttpURLConnection(URL url) throws IOException {
+ String tokenString = (url.getQuery() == null ? "?" : "&") + "user.name=" +
+ URLEncoder.encode(UserGroupInformation.getCurrentUser().getShortUserName(), "UTF8");
+ return (HttpURLConnection) (new URL(url.toString() + tokenString)).openConnection();
+ }
+ }
+
+ @Override
+ public int run(String[] args) throws Exception {
+ try {
+ download();
+ return 0;
+ } catch (Exception e) {
+ e.printStackTrace();
+ LOG.error("Error occurred when downloading data ", e);
+ return -1;
+ }
+ }
+
+ private static Options buildOptions() {
+ Option dagIdOption = OptionBuilder.withArgName(DAG_ID).withLongOpt(DAG_ID)
+ .withDescription("DagId that needs to be downloaded").hasArg().isRequired(true).create();
+
+ Option downloadDirOption = OptionBuilder.withArgName(BASE_DOWNLOAD_DIR).withLongOpt
+ (BASE_DOWNLOAD_DIR)
+ .withDescription("Download directory where data needs to be downloaded").hasArg()
+ .isRequired(true).create();
+
+ Option atsAddressOption = OptionBuilder.withArgName(YARN_TIMELINE_SERVICE_ADDRESS).withLongOpt(
+ YARN_TIMELINE_SERVICE_ADDRESS)
+ .withDescription("Optional. ATS address (e.g http://clusterATSNode:8188)").hasArg()
+ .isRequired(false)
+ .create();
+
+ Option batchSizeOption = OptionBuilder.withArgName(BATCH_SIZE).withLongOpt(BATCH_SIZE)
+ .withDescription("Optional. batch size for downloading data").hasArg()
+ .isRequired(false)
+ .create();
+
+ Option help = OptionBuilder.withArgName("help").withLongOpt("help")
+ .withDescription("print help").isRequired(false).create();
+
+ Options opts = new Options();
+ opts.addOption(dagIdOption);
+ opts.addOption(downloadDirOption);
+ opts.addOption(atsAddressOption);
+ opts.addOption(batchSizeOption);
+ opts.addOption(help);
+ return opts;
+ }
+
+ static void printHelp(Options options) {
+ HelpFormatter formatter = new HelpFormatter();
+ formatter.setWidth(240);
+ String help = LINE_SEPARATOR
+ + "java -cp tez-history-parser-x.y.z-jar-with-dependencies.jar org.apache.tez.history.ATSImportTool"
+ + LINE_SEPARATOR
+ + "OR"
+ + LINE_SEPARATOR
+ + "HADOOP_CLASSPATH=$TEZ_HOME/*:$TEZ_HOME/lib/*:$HADOOP_CLASSPATH hadoop jar "
+ + "tez-history-parser-x.y.z.jar " + ATSImportTool.class.getName()
+ + LINE_SEPARATOR;
+ formatter.printHelp(240, help, "Options", options, "", true);
+ }
+
+ static boolean hasHttpsPolicy(Configuration conf) {
+ YarnConfiguration yarnConf = new YarnConfiguration(conf);
+ return (HttpConfig.Policy.HTTPS_ONLY == HttpConfig.Policy.fromString(yarnConf
+ .get(YarnConfiguration.YARN_HTTP_POLICY_KEY, YarnConfiguration.YARN_HTTP_POLICY_DEFAULT)));
+ }
+
+ static String getBaseTimelineURL(String yarnTimelineAddress, Configuration conf)
+ throws TezException {
+ boolean isHttps = hasHttpsPolicy(conf);
+
+ if (yarnTimelineAddress == null) {
+ if (isHttps) {
+ yarnTimelineAddress = conf.get(Constants.TIMELINE_SERVICE_WEBAPP_HTTPS_ADDRESS_CONF_NAME);
+ } else {
+ yarnTimelineAddress = conf.get(Constants.TIMELINE_SERVICE_WEBAPP_HTTP_ADDRESS_CONF_NAME);
+ }
+ Preconditions.checkArgument(!Strings.isNullOrEmpty(yarnTimelineAddress), "Yarn timeline address can"
+ + " not be empty. Please check configurations.");
+ } else {
+ yarnTimelineAddress = yarnTimelineAddress.trim();
+ Preconditions.checkArgument(!Strings.isNullOrEmpty(yarnTimelineAddress), "Yarn timeline address can"
+ + " not be empty. Please provide valid url with --" +
+ YARN_TIMELINE_SERVICE_ADDRESS + " option");
+ }
+
+ yarnTimelineAddress = yarnTimelineAddress.toLowerCase();
+ if (!yarnTimelineAddress.startsWith(HTTP_SCHEME)
+ && !yarnTimelineAddress.startsWith(HTTPS_SCHEME)) {
+ yarnTimelineAddress = ((isHttps) ? HTTPS_SCHEME : HTTP_SCHEME) + yarnTimelineAddress;
+ }
+
+ try {
+ yarnTimelineAddress = new URI(yarnTimelineAddress).normalize().toString().trim();
+ yarnTimelineAddress = (yarnTimelineAddress.endsWith("/")) ?
+ yarnTimelineAddress.substring(0, yarnTimelineAddress.length() - 1) :
+ yarnTimelineAddress;
+ } catch (URISyntaxException e) {
+ throw new TezException("Please provide a valid URL. url=" + yarnTimelineAddress, e);
+ }
+
+ return Joiner.on("").join(yarnTimelineAddress, Constants.RESOURCE_URI_BASE);
+ }
+
+ @VisibleForTesting
+ public static int process(String[] args) throws Exception {
+ Options options = buildOptions();
+ try {
+ Configuration conf = new Configuration();
+ CommandLine cmdLine = new GnuParser().parse(options, args);
+ String dagId = cmdLine.getOptionValue(DAG_ID);
+
+ File downloadDir = new File(cmdLine.getOptionValue(BASE_DOWNLOAD_DIR));
+
+ String yarnTimelineAddress = cmdLine.getOptionValue(YARN_TIMELINE_SERVICE_ADDRESS);
+ String baseTimelineURL = getBaseTimelineURL(yarnTimelineAddress, conf);
+
+ int batchSize = (cmdLine.hasOption(BATCH_SIZE)) ?
+ (Integer.parseInt(cmdLine.getOptionValue(BATCH_SIZE))) : BATCH_SIZE_DEFAULT;
+
+ return ToolRunner.run(conf, new ATSImportTool(baseTimelineURL, dagId,
+ downloadDir, batchSize), args);
+ } catch (ParseException e) {
+ LOG.error("Error in parsing options ", e);
+ printHelp(options);
+ throw e;
+ } catch (Exception e) {
+ LOG.error("Error in processing ", e);
+ throw e;
+ }
+ }
+
+ public static void main(String[] args) throws Exception {
+ Utils.setupRootLogger();
+ int res = process(args);
+ System.exit(res);
+ }
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/8c8db7c5/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/ATSData.java
----------------------------------------------------------------------
diff --git a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/ATSData.java b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/ATSData.java
new file mode 100644
index 0000000..f504007
--- /dev/null
+++ b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/ATSData.java
@@ -0,0 +1,48 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.history.parser;
+
+import org.apache.tez.dag.api.TezException;
+import org.apache.tez.history.parser.datamodel.DagInfo;
+import org.codehaus.jettison.json.JSONException;
+
+import static org.apache.hadoop.classification.InterfaceStability.Evolving;
+
+/**
+ * Main interface to pull data from ATS.
+ * <p/>
+ * It is possible that to have ATS data store in any DB (e.g LevelDB or HBase). Depending on
+ * store, there can be multiple implementations to pull data from these stores and create the
+ * DagInfo object for analysis.
+ * <p/>
+ */
+@Evolving
+public interface ATSData {
+
+ /**
+ * Get the DAG representation for processing
+ *
+ * @param dagId
+ * @return DagInfo
+ * @throws JSONException
+ * @throws TezException
+ */
+ public DagInfo getDAGData(String dagId) throws TezException;
+
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/8c8db7c5/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/ATSFileParser.java
----------------------------------------------------------------------
diff --git a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/ATSFileParser.java b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/ATSFileParser.java
new file mode 100644
index 0000000..aae20eb
--- /dev/null
+++ b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/ATSFileParser.java
@@ -0,0 +1,227 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.history.parser;
+
+import com.google.common.base.Preconditions;
+import org.apache.commons.io.IOUtils;
+import org.apache.tez.dag.api.TezException;
+import org.apache.tez.history.parser.datamodel.BaseParser;
+import org.apache.tez.history.parser.datamodel.Constants;
+import org.apache.tez.history.parser.datamodel.DagInfo;
+import org.apache.tez.history.parser.datamodel.TaskAttemptInfo;
+import org.apache.tez.history.parser.datamodel.TaskInfo;
+import org.apache.tez.history.parser.datamodel.VersionInfo;
+import org.apache.tez.history.parser.datamodel.VertexInfo;
+import org.codehaus.jettison.json.JSONArray;
+import org.codehaus.jettison.json.JSONException;
+import org.codehaus.jettison.json.JSONObject;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Enumeration;
+import java.util.zip.ZipEntry;
+import java.util.zip.ZipFile;
+
+import static org.apache.hadoop.classification.InterfaceAudience.Public;
+import static org.apache.hadoop.classification.InterfaceStability.Evolving;
+
+/**
+ * Simple class to parse ATS zip file of a DAG and generate the relevant in-memory structure
+ * (DagInfo) necessary for processing later.
+ */
+
+@Public
+@Evolving
+public class ATSFileParser extends BaseParser implements ATSData {
+
+ private static final Logger LOG = LoggerFactory.getLogger(ATSFileParser.class);
+
+ private final File atsZipFile;
+
+ public ATSFileParser(File atsZipFile) throws TezException {
+ super();
+ Preconditions.checkArgument(atsZipFile.exists(), "Zipfile " + atsZipFile + " does not exist");
+ this.atsZipFile = atsZipFile;
+ }
+
+ @Override
+ public DagInfo getDAGData(String dagId) throws TezException {
+ try {
+ parseATSZipFile(atsZipFile);
+
+ linkParsedContents();
+
+ return dagInfo;
+ } catch (IOException e) {
+ LOG.error("Error in reading DAG ", e);
+ throw new TezException(e);
+ } catch (JSONException e) {
+ LOG.error("Error in parsing DAG ", e);
+ throw new TezException(e);
+ } catch (InterruptedException e) {
+ throw new TezException(e);
+ }
+ }
+
+ /**
+ * Parse vertices json
+ *
+ * @param verticesJson
+ * @throws JSONException
+ */
+ private void processVertices(JSONArray verticesJson) throws JSONException {
+ //Process vertex information
+ Preconditions.checkState(verticesJson != null, "Vertex json can not be null");
+ if (verticesJson != null) {
+ LOG.info("Started parsing vertex");
+ for (int i = 0; i < verticesJson.length(); i++) {
+ VertexInfo vertexInfo = VertexInfo.create(verticesJson.getJSONObject(i));
+ vertexList.add(vertexInfo);
+ }
+ LOG.info("Finished parsing vertex");
+ }
+ }
+
+ /**
+ * Parse Tasks json
+ *
+ * @param tasksJson
+ * @throws JSONException
+ */
+ private void processTasks(JSONArray tasksJson) throws JSONException {
+ //Process Task information
+ Preconditions.checkState(tasksJson != null, "Task json can not be null");
+ if (tasksJson != null) {
+ LOG.debug("Started parsing task");
+ for (int i = 0; i < tasksJson.length(); i++) {
+ TaskInfo taskInfo = TaskInfo.create(tasksJson.getJSONObject(i));
+ taskList.add(taskInfo);
+ }
+ LOG.debug("Finished parsing task");
+ }
+ }
+
+ /**
+ * Parse TaskAttempt json
+ *
+ * @param taskAttemptsJson
+ * @throws JSONException
+ */
+ private void processAttempts(JSONArray taskAttemptsJson) throws JSONException {
+ //Process TaskAttempt information
+ Preconditions.checkState(taskAttemptsJson != null, "Attempts json can not be null");
+ if (taskAttemptsJson != null) {
+ LOG.debug("Started parsing task attempts");
+ for (int i = 0; i < taskAttemptsJson.length(); i++) {
+ TaskAttemptInfo attemptInfo = TaskAttemptInfo.create(taskAttemptsJson.getJSONObject(i));
+ attemptList.add(attemptInfo);
+ }
+ LOG.debug("Finished parsing task attempts");
+ }
+ }
+
+ /**
+ * Parse TezApplication json
+ *
+ * @param tezApplicationJson
+ * @throws JSONException
+ */
+ private void processApplication(JSONObject tezApplicationJson) throws JSONException {
+ if (tezApplicationJson != null) {
+ LOG.debug("Started parsing tez application");
+ JSONObject otherInfoNode = tezApplicationJson.optJSONObject(Constants.OTHER_INFO);
+ if (otherInfoNode != null) {
+ JSONObject tezVersion = otherInfoNode.optJSONObject(Constants.TEZ_VERSION);
+ if (tezVersion != null) {
+ String version = tezVersion.optString(Constants.VERSION);
+ String buildTime = tezVersion.optString(Constants.BUILD_TIME);
+ String revision = tezVersion.optString(Constants.REVISION);
+ this.versionInfo = new VersionInfo(version, buildTime, revision);
+ }
+ //TODO: might need to parse config info? (e.g, hive settings etc. could consume memory)
+ }
+ LOG.debug("Finished parsing tez application");
+ }
+ }
+
+ private JSONObject readJson(InputStream in) throws IOException, JSONException {
+ //Read entire content to memory
+ final ByteArrayOutputStream bout = new ByteArrayOutputStream();
+ IOUtils.copy(in, bout);
+ return new JSONObject(new String(bout.toByteArray(), "UTF-8"));
+ }
+
+ /**
+ * Read zip file contents. Every file can contain "dag", "vertices", "tasks", "task_attempts"
+ *
+ * @param atsFile
+ * @throws IOException
+ * @throws JSONException
+ */
+ private void parseATSZipFile(File atsFile)
+ throws IOException, JSONException, TezException, InterruptedException {
+ final ZipFile atsZipFile = new ZipFile(atsFile);
+ try {
+ Enumeration<? extends ZipEntry> zipEntries = atsZipFile.entries();
+ while (zipEntries.hasMoreElements()) {
+ ZipEntry zipEntry = zipEntries.nextElement();
+ LOG.info("Processing " + zipEntry.getName());
+ InputStream inputStream = atsZipFile.getInputStream(zipEntry);
+ JSONObject jsonObject = readJson(inputStream);
+
+ //This json can contain dag, vertices, tasks, task_attempts
+ JSONObject dagJson = jsonObject.optJSONObject(Constants.DAG);
+ if (dagJson != null) {
+ //TODO: support for multiple dags per ATS file later.
+ dagInfo = DagInfo.create(dagJson);
+ }
+
+ //Process vertex
+ JSONArray vertexJson = jsonObject.optJSONArray(Constants.VERTICES);
+ if (vertexJson != null) {
+ processVertices(vertexJson);
+ }
+
+ //Process task
+ JSONArray taskJson = jsonObject.optJSONArray(Constants.TASKS);
+ if (taskJson != null) {
+ processTasks(taskJson);
+ }
+
+ //Process task attempts
+ JSONArray attemptsJson = jsonObject.optJSONArray(Constants.TASK_ATTEMPTS);
+ if (attemptsJson != null) {
+ processAttempts(attemptsJson);
+ }
+
+ //Process application (mainly versionInfo)
+ JSONObject tezAppJson = jsonObject.optJSONObject(Constants.APPLICATION);
+ if (tezAppJson != null) {
+ processApplication(tezAppJson);
+ }
+ }
+ } finally {
+ atsZipFile.close();
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/8c8db7c5/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/SimpleHistoryParser.java
----------------------------------------------------------------------
diff --git a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/SimpleHistoryParser.java b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/SimpleHistoryParser.java
new file mode 100644
index 0000000..4d3e96f
--- /dev/null
+++ b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/SimpleHistoryParser.java
@@ -0,0 +1,242 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.tez.history.parser;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
+import com.google.common.collect.Maps;
+import org.apache.tez.dag.api.TezException;
+import org.apache.tez.dag.history.logging.impl.SimpleHistoryLoggingService;
+import org.apache.tez.dag.records.TezDAGID;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+import org.apache.tez.dag.records.TezTaskID;
+import org.apache.tez.dag.records.TezVertexID;
+import org.apache.tez.history.parser.datamodel.BaseParser;
+import org.apache.tez.history.parser.datamodel.Constants;
+import org.apache.tez.history.parser.datamodel.DagInfo;
+import org.apache.tez.history.parser.datamodel.TaskAttemptInfo;
+import org.apache.tez.history.parser.datamodel.TaskInfo;
+import org.apache.tez.history.parser.datamodel.VertexInfo;
+import org.codehaus.jettison.json.JSONArray;
+import org.codehaus.jettison.json.JSONException;
+import org.codehaus.jettison.json.JSONObject;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Scanner;
+
+/**
+ * Parser utility to parse data generated by SimpleHistoryLogging to in-memory datamodel provided
+ * in org.apache.tez.history.parser.datamodel
+ * <p/>
+ * <p/>
+ * Most of the information should be available. Minor info like VersionInfo may not be available,
+ * as it is not captured in SimpleHistoryLogging.
+ */
+public class SimpleHistoryParser extends BaseParser {
+ private static final Logger LOG = LoggerFactory.getLogger(SimpleHistoryParser.class);
+ private static final String UTF8 = "UTF-8";
+ private final File historyFile;
+
+
+ public SimpleHistoryParser(File historyFile) {
+ super();
+ Preconditions.checkArgument(historyFile.exists(), historyFile + " does not exist");
+ this.historyFile = historyFile;
+ }
+
+ /**
+ * Get in-memory representation of DagInfo
+ *
+ * @return DagInfo
+ * @throws TezException
+ */
+ public DagInfo getDAGData(String dagId) throws TezException {
+ try {
+ Preconditions.checkArgument(!Strings.isNullOrEmpty(dagId), "Please provide valid dagId");
+ dagId = dagId.trim();
+ parseContents(historyFile, dagId);
+ linkParsedContents();
+ return dagInfo;
+ } catch (IOException e) {
+ LOG.error("Error in reading DAG ", e);
+ throw new TezException(e);
+ } catch (JSONException e) {
+ LOG.error("Error in parsing DAG ", e);
+ throw new TezException(e);
+ }
+ }
+
+ private void populateOtherInfo(JSONObject source, JSONObject destination) throws JSONException {
+ if (source == null || destination == null) {
+ return;
+ }
+ for (Iterator it = source.keys(); it.hasNext(); ) {
+ String key = (String) it.next();
+ Object val = source.get(key);
+ destination.put(key, val);
+ }
+ }
+
+ private void populateOtherInfo(JSONObject source, String entityName,
+ Map<String, JSONObject> destMap) throws JSONException {
+ JSONObject destinationJson = destMap.get(entityName);
+ JSONObject destOtherInfo = destinationJson.getJSONObject(Constants.OTHER_INFO);
+ populateOtherInfo(source, destOtherInfo);
+ }
+
+ private void parseContents(File historyFile, String dagId)
+ throws JSONException, FileNotFoundException, TezException {
+ Scanner scanner = new Scanner(historyFile, UTF8);
+ scanner.useDelimiter(SimpleHistoryLoggingService.RECORD_SEPARATOR);
+ JSONObject dagJson = null;
+ Map<String, JSONObject> vertexJsonMap = Maps.newHashMap();
+ Map<String, JSONObject> taskJsonMap = Maps.newHashMap();
+ Map<String, JSONObject> attemptJsonMap = Maps.newHashMap();
+ TezDAGID tezDAGID = TezDAGID.fromString(dagId);
+ while (scanner.hasNext()) {
+ String line = scanner.next();
+ JSONObject jsonObject = new JSONObject(line);
+ String entity = jsonObject.getString(Constants.ENTITY);
+ String entityType = jsonObject.getString(Constants.ENTITY_TYPE);
+ if (entityType.equals(Constants.TEZ_DAG_ID)) {
+ if (!dagId.equals(entity)) {
+ LOG.warn(dagId + " is not matching with " + entity);
+ continue;
+ }
+ // Club all DAG related information together (DAG_INIT, DAG_FINISH etc). Each of them
+ // would have a set of entities in otherinfo (e.g vertex mapping, dagPlan, start/finish
+ // time etc).
+ if (dagJson == null) {
+ dagJson = jsonObject;
+ }
+ JSONObject otherInfo = jsonObject.optJSONObject(Constants.OTHER_INFO);
+ JSONObject dagOtherInfo = dagJson.getJSONObject(Constants.OTHER_INFO);
+ populateOtherInfo(otherInfo, dagOtherInfo);
+ }
+ else if (entityType.equals(Constants.TEZ_VERTEX_ID)) {
+ String vertexName = entity;
+ TezVertexID tezVertexID = TezVertexID.fromString(vertexName);
+ if (!tezDAGID.equals(tezVertexID.getDAGId())) {
+ LOG.warn(vertexName + " does not belong to " + tezDAGID);
+ continue;
+ }
+ if (!vertexJsonMap.containsKey(vertexName)) {
+ vertexJsonMap.put(vertexName, jsonObject);
+ }
+ JSONObject otherInfo = jsonObject.optJSONObject(Constants.OTHER_INFO);
+ populateOtherInfo(otherInfo, vertexName, vertexJsonMap);
+ }
+ else if (entityType.equals(Constants.TEZ_TASK_ID)) {
+ String taskName = entity;
+ TezTaskID tezTaskID = TezTaskID.fromString(taskName);
+ if (!tezDAGID.equals(tezTaskID.getVertexID().getDAGId())) {
+ LOG.warn(taskName + " does not belong to " + tezDAGID);
+ continue;
+ }
+ if (!taskJsonMap.containsKey(taskName)) {
+ taskJsonMap.put(taskName, jsonObject);
+ }
+ JSONObject otherInfo = jsonObject.optJSONObject(Constants.OTHER_INFO);
+ populateOtherInfo(otherInfo, taskName, taskJsonMap);
+ }
+ else if (entityType.equals(Constants.TEZ_TASK_ATTEMPT_ID)) {
+ String taskAttemptName = entity;
+ TezTaskAttemptID tezAttemptId = TezTaskAttemptID.fromString(taskAttemptName);
+ if (!tezDAGID.equals(tezAttemptId.getTaskID().getVertexID().getDAGId())) {
+ LOG.warn(taskAttemptName + " does not belong to " + tezDAGID);
+ continue;
+ }
+ if (!attemptJsonMap.containsKey(taskAttemptName)) {
+ attemptJsonMap.put(taskAttemptName, jsonObject);
+ }
+ JSONObject otherInfo = jsonObject.optJSONObject(Constants.OTHER_INFO);
+ populateOtherInfo(otherInfo, taskAttemptName, attemptJsonMap);
+ }
+ }
+ scanner.close();
+ if (dagJson != null) {
+ this.dagInfo = DagInfo.create(dagJson);
+ } else {
+ LOG.error("Dag is not yet parsed. Looks like partial file.");
+ throw new TezException(
+ "Please provide a valid/complete history log file containing " + dagId);
+ }
+ for (JSONObject jsonObject : vertexJsonMap.values()) {
+ VertexInfo vertexInfo = VertexInfo.create(jsonObject);
+ this.vertexList.add(vertexInfo);
+ LOG.debug("Parsed vertex {}", vertexInfo.getVertexName());
+ }
+ for (JSONObject jsonObject : taskJsonMap.values()) {
+ TaskInfo taskInfo = TaskInfo.create(jsonObject);
+ this.taskList.add(taskInfo);
+ LOG.debug("Parsed task {}", taskInfo.getTaskId());
+ }
+ for (JSONObject jsonObject : attemptJsonMap.values()) {
+ /**
+ * For converting SimpleHistoryLogging to in-memory representation
+ *
+ * We need to get "relatedEntities":[{"entity":"cn055-10.l42scl.hortonworks.com:58690",
+ * "entitytype":"nodeId"},{"entity":"container_1438652049951_0008_01_000152",
+ * "entitytype":"containerId"} and populate it in otherInfo object so that in-memory
+ * representation can parse it correctly
+ */
+ JSONArray relatedEntities = jsonObject.optJSONArray(Constants.RELATED_ENTITIES);
+ if (relatedEntities == null) {
+ //This can happen when CONTAINER_EXITED abruptly. (e.g Container failed, exitCode=1)
+ LOG.debug("entity {} did not have related entities",
+ jsonObject.optJSONObject(Constants.ENTITY));
+ } else {
+ JSONObject subJsonObject = relatedEntities.optJSONObject(0);
+ if (subJsonObject != null) {
+ String nodeId = subJsonObject.optString(Constants.ENTITY_TYPE);
+ if (!Strings.isNullOrEmpty(nodeId) && nodeId.equalsIgnoreCase(Constants.NODE_ID)) {
+ //populate it in otherInfo
+ JSONObject otherInfo = jsonObject.optJSONObject(Constants.OTHER_INFO);
+ String nodeIdVal = subJsonObject.optString(Constants.ENTITY);
+ if (otherInfo != null && nodeIdVal != null) {
+ otherInfo.put(Constants.NODE_ID, nodeIdVal);
+ }
+ }
+ }
+
+ subJsonObject = relatedEntities.optJSONObject(1);
+ if (subJsonObject != null) {
+ String containerId = subJsonObject.optString(Constants.ENTITY_TYPE);
+ if (!Strings.isNullOrEmpty(containerId) && containerId
+ .equalsIgnoreCase(Constants.CONTAINER_ID)) {
+ //populate it in otherInfo
+ JSONObject otherInfo = jsonObject.optJSONObject(Constants.OTHER_INFO);
+ String containerIdVal = subJsonObject.optString(Constants.ENTITY);
+ if (otherInfo != null && containerIdVal != null) {
+ otherInfo.put(Constants.CONTAINER_ID, containerIdVal);
+ }
+ }
+ }
+ }
+ TaskAttemptInfo attemptInfo = TaskAttemptInfo.create(jsonObject);
+ this.attemptList.add(attemptInfo);
+ LOG.debug("Parsed task attempt {}", attemptInfo.getTaskAttemptId());
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/8c8db7c5/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/AdditionalInputOutputDetails.java
----------------------------------------------------------------------
diff --git a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/AdditionalInputOutputDetails.java b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/AdditionalInputOutputDetails.java
new file mode 100644
index 0000000..b853d5c
--- /dev/null
+++ b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/AdditionalInputOutputDetails.java
@@ -0,0 +1,71 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.history.parser.datamodel;
+
+import static org.apache.hadoop.classification.InterfaceAudience.Public;
+import static org.apache.hadoop.classification.InterfaceStability.Evolving;
+
+/**
+ * Additional input/ouput information present in DAG.
+ */
+
+@Public
+@Evolving
+public class AdditionalInputOutputDetails {
+ private final String name;
+ private final String clazz;
+ private final String initializer;
+ private final String userPayloadText;
+
+ public AdditionalInputOutputDetails(String name, String clazz, String initializer,
+ String userPayloadText) {
+ this.name = name;
+ this.clazz = clazz;
+ this.initializer = initializer;
+ this.userPayloadText = userPayloadText;
+ }
+
+ public final String getName() {
+ return name;
+ }
+
+ public final String getClazz() {
+ return clazz;
+ }
+
+ public final String getInitializer() {
+ return initializer;
+ }
+
+ public final String getUserPayloadText() {
+ return userPayloadText;
+ }
+
+ @Override
+ public String toString() {
+ StringBuilder sb = new StringBuilder();
+ sb.append("[");
+ sb.append("name=").append(name).append(", ");
+ sb.append("clazz=").append(clazz).append(", ");
+ sb.append("initializer=").append(initializer).append(", ");
+ sb.append("userPayloadText=").append(userPayloadText);
+ sb.append("]");
+ return sb.toString();
+ }
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/8c8db7c5/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/BaseInfo.java
----------------------------------------------------------------------
diff --git a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/BaseInfo.java b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/BaseInfo.java
new file mode 100644
index 0000000..3f9666a
--- /dev/null
+++ b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/BaseInfo.java
@@ -0,0 +1,142 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.history.parser.datamodel;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import org.apache.tez.common.counters.CounterGroup;
+import org.apache.tez.common.counters.TezCounter;
+import org.apache.tez.common.counters.TezCounters;
+import org.apache.tez.history.parser.utils.Utils;
+import org.codehaus.jettison.json.JSONException;
+import org.codehaus.jettison.json.JSONObject;
+
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.hadoop.classification.InterfaceAudience.Public;
+import static org.apache.hadoop.classification.InterfaceStability.Evolving;
+
+@Public
+@Evolving
+public abstract class BaseInfo {
+
+ protected TezCounters tezCounters;
+ protected List<Event> eventList;
+
+ BaseInfo(JSONObject jsonObject) throws JSONException {
+ final JSONObject otherInfoNode = jsonObject.getJSONObject(Constants.OTHER_INFO);
+ //parse tez counters
+ tezCounters = Utils.parseTezCountersFromJSON(
+ otherInfoNode.optJSONObject(Constants.COUNTERS));
+
+ //parse events
+ eventList = Lists.newArrayList();
+ Utils.parseEvents(jsonObject.optJSONArray(Constants.EVENTS), eventList);
+ }
+
+ public TezCounters getTezCounters() {
+ return tezCounters;
+ }
+
+ /**
+ * Get start time w.r.t DAG
+ *
+ * @return long
+ */
+ public abstract long getStartTimeInterval();
+
+ /**
+ * Get finish time w.r.t DAG
+ *
+ * @return long
+ */
+ public abstract long getFinishTimeInterval();
+
+ /**
+ * Get absolute start time
+ *
+ * @return long
+ */
+ public abstract long getStartTime();
+
+ /**
+ * Get absolute finish time
+ *
+ * @return long
+ */
+ public abstract long getFinishTime();
+
+ public abstract String getDiagnostics();
+
+ public List<Event> getEvents() {
+ return eventList;
+ }
+
+ /**
+ * Get counter for a specific counter group name.
+ * If counterGroupName is not mentioned, it would end up returning counter found in all
+ * groups
+ *
+ * @param counterGroupName
+ * @param counter
+ * @return Map<String, TezCounter> tez counter at every counter group level
+ */
+ public Map<String, TezCounter> getCounter(String counterGroupName, String counter) {
+ //TODO: FS, TaskCounters are directly getting added as TezCounters always pass those. Need a
+ // way to get rid of these.
+ Map<String, TezCounter> result = Maps.newHashMap();
+ Iterator<String> iterator = tezCounters.getGroupNames().iterator();
+ boolean found = false;
+ while (iterator.hasNext()) {
+ CounterGroup counterGroup = tezCounters.getGroup(iterator.next());
+ if (counterGroupName != null) {
+ String groupName = counterGroup.getName();
+ if (groupName.equals(counterGroupName)) {
+ found = true;
+ }
+ }
+
+ //Explicitly mention that no need to create the counter if not present
+ TezCounter tezCounter = counterGroup.getUnderlyingGroup().findCounter(counter, false);
+ if (tezCounter != null) {
+ result.put(counterGroup.getName(), tezCounter);
+ }
+
+ if (found) {
+ //Retrieved counter specific to a counter group. Safe to exit.
+ break;
+ }
+
+ }
+ return result;
+ }
+
+ /**
+ * Find a counter in all counter groups
+ *
+ * @param counter
+ * @return Map of countergroup to TezCounter mapping
+ */
+ public Map<String, TezCounter> getCounter(String counter) {
+ return getCounter(null, counter);
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/8c8db7c5/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/BaseParser.java
----------------------------------------------------------------------
diff --git a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/BaseParser.java b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/BaseParser.java
new file mode 100644
index 0000000..62ba474
--- /dev/null
+++ b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/BaseParser.java
@@ -0,0 +1,114 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.history.parser.datamodel;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Predicate;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+import org.apache.tez.dag.records.TezTaskID;
+
+import java.util.List;
+
+public abstract class BaseParser {
+
+ protected DagInfo dagInfo;
+ protected VersionInfo versionInfo;
+ protected final List<VertexInfo> vertexList;
+ protected final List<TaskInfo> taskList;
+ protected final List<TaskAttemptInfo> attemptList;
+
+
+ public BaseParser() {
+ vertexList = Lists.newLinkedList();
+ taskList = Lists.newLinkedList();
+ attemptList = Lists.newLinkedList();
+ }
+
+ /**
+ * link the parsed contents, so that it becomes easier to iterate from DAG-->Task and Task--DAG.
+ * e.g Link vertex to dag, task to vertex, attempt to task etc
+ */
+ protected void linkParsedContents() {
+ //Link vertex to DAG
+ for (VertexInfo vertexInfo : vertexList) {
+ vertexInfo.setDagInfo(dagInfo);
+ }
+
+ //Link task to vertex
+ for (TaskInfo taskInfo : taskList) {
+ //Link vertex to task
+ String vertexId = TezTaskID.fromString(taskInfo.getTaskId()).getVertexID().toString();
+ VertexInfo vertexInfo = dagInfo.getVertexFromId(vertexId);
+ Preconditions.checkState(vertexInfo != null, "VertexInfo for " + vertexId + " can't be "
+ + "null");
+ taskInfo.setVertexInfo(vertexInfo);
+ }
+
+ //Link task attempt to task
+ for (TaskAttemptInfo attemptInfo : attemptList) {
+ //Link task to task attempt
+ TezTaskAttemptID taskAttemptId = TezTaskAttemptID.fromString(attemptInfo
+ .getTaskAttemptId());
+ VertexInfo vertexInfo = dagInfo.getVertexFromId(taskAttemptId.getTaskID()
+ .getVertexID().toString());
+ Preconditions.checkState(vertexInfo != null, "Vertex " + taskAttemptId.getTaskID()
+ .getVertexID().toString() + " is not present in DAG");
+ TaskInfo taskInfo = vertexInfo.getTask(taskAttemptId.getTaskID().toString());
+ attemptInfo.setTaskInfo(taskInfo);
+ }
+
+ //Set container details
+ for (VertexInfo vertexInfo : dagInfo.getVertices()) {
+ for (TaskAttemptInfo taskAttemptInfo : vertexInfo.getTaskAttempts()) {
+ dagInfo.addContainerMapping(taskAttemptInfo.getContainer(), taskAttemptInfo);
+ }
+ }
+
+
+ //Set reference time for all events
+ for (VertexInfo vertexInfo : dagInfo.getVertices()) {
+ setReferenceTime(vertexInfo.getEvents(), dagInfo.getStartTimeInterval());
+ for (TaskInfo taskInfo : vertexInfo.getTasks()) {
+ setReferenceTime(taskInfo.getEvents(), dagInfo.getStartTimeInterval());
+ for (TaskAttemptInfo taskAttemptInfo : taskInfo.getTaskAttempts()) {
+ setReferenceTime(taskAttemptInfo.getEvents(), dagInfo.getStartTimeInterval());
+ }
+ }
+ }
+
+ dagInfo.setVersionInfo(versionInfo);
+ }
+
+ /**
+ * Set reference time to all events
+ *
+ * @param eventList
+ * @param referenceTime
+ */
+ private void setReferenceTime(List<Event> eventList, final long referenceTime) {
+ Iterables.all(eventList, new Predicate<Event>() {
+ @Override public boolean apply(Event input) {
+ input.setReferenceTime(referenceTime);
+ return false;
+ }
+ });
+ }
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/8c8db7c5/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/Constants.java
----------------------------------------------------------------------
diff --git a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/Constants.java b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/Constants.java
new file mode 100644
index 0000000..dce79e2
--- /dev/null
+++ b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/Constants.java
@@ -0,0 +1,64 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.history.parser.datamodel;
+
+import org.apache.tez.common.ATSConstants;
+
+import static org.apache.hadoop.classification.InterfaceAudience.Private;
+
+@Private
+public class Constants extends ATSConstants {
+
+ public static final String EVENT_TIME_STAMP = "timestamp";
+ public static final String TEZ_APPLICATION = "TEZ_APPLICATION";
+ public static final String TEZ_TASK_ID = "TEZ_TASK_ID";
+ public static final String TEZ_TASK_ATTEMPT_ID = "TEZ_TASK_ATTEMPT_ID";
+
+ public static final String EDGE_ID = "edgeId";
+ public static final String INPUT_VERTEX_NAME = "inputVertexName";
+ public static final String OUTPUT_VERTEX_NAME = "outputVertexName";
+ public static final String DATA_MOVEMENT_TYPE = "dataMovementType";
+ public static final String EDGE_SOURCE_CLASS = "edgeSourceClass";
+ public static final String EDGE_DESTINATION_CLASS = "edgeDestinationClass";
+ public static final String INPUT_PAYLOAD_TEXT = "inputUserPayloadAsText";
+ public static final String OUTPUT_PAYLOAD_TEXT = "outputUserPayloadAsText";
+
+ public static final String EDGES = "edges";
+ public static final String OUT_EDGE_IDS = "outEdgeIds";
+ public static final String IN_EDGE_IDS = "inEdgeIds";
+ public static final String ADDITIONAL_INPUTS = "additionalInputs";
+ public static final String ADDITIONAL_OUTPUTS = "additionalOutputs";
+
+ public static final String NAME = "name";
+ public static final String CLASS = "class";
+ public static final String INITIALIZER = "initializer";
+ public static final String USER_PAYLOAD_TEXT = "userPayloadAsText";
+
+ public static final String DAG_CONTEXT = "dagContext";
+
+ //constants for ATS data export
+ public static final String DAG = "dag";
+ public static final String VERTICES = "vertices";
+ public static final String TASKS = "tasks";
+ public static final String TASK_ATTEMPTS = "task_attempts";
+ public static final String APPLICATION = "application";
+
+
+
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/8c8db7c5/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/Container.java
----------------------------------------------------------------------
diff --git a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/Container.java b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/Container.java
new file mode 100644
index 0000000..4e01d1b
--- /dev/null
+++ b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/Container.java
@@ -0,0 +1,70 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.history.parser.datamodel;
+
+import com.google.common.base.Objects;
+
+import static org.apache.hadoop.classification.InterfaceAudience.Public;
+import static org.apache.hadoop.classification.InterfaceStability.Evolving;
+
+@Public
+@Evolving
+public class Container {
+
+ private final String id;
+ private final String host;
+
+ public Container(String id, String host) {
+ this.id = id;
+ this.host = host;
+ }
+
+ public final String getId() {
+ return id;
+ }
+
+ public final String getHost() {
+ return host;
+ }
+
+ @Override public String toString() {
+ StringBuilder sb = new StringBuilder();
+ sb.append("[");
+ sb.append("id=").append(id).append(", ");
+ sb.append("host=").append(host);
+ sb.append("]");
+ return sb.toString();
+ }
+
+ @Override public int hashCode() {
+ return Objects.hashCode(id, host);
+ }
+
+ @Override public boolean equals(Object obj) {
+ if (this == obj) {
+ return true;
+ }
+ if (obj == null || getClass() != obj.getClass()) {
+ return false;
+ }
+ final Container other = (Container) obj;
+ return Objects.equal(this.id, other.id)
+ && Objects.equal(this.host, other.host);
+ }
+}
[2/5] tez git commit: TEZ-2973. Backport Analyzers to branch-0.7
Posted by je...@apache.org.
http://git-wip-us.apache.org/repos/asf/tez/blob/8c8db7c5/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/LocalityAnalyzer.java
----------------------------------------------------------------------
diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/LocalityAnalyzer.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/LocalityAnalyzer.java
new file mode 100644
index 0000000..ec72df1
--- /dev/null
+++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/LocalityAnalyzer.java
@@ -0,0 +1,204 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.analyzer.plugins;
+
+import com.google.common.collect.Lists;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.tez.analyzer.Analyzer;
+import org.apache.tez.analyzer.CSVResult;
+import org.apache.tez.common.counters.DAGCounter;
+import org.apache.tez.common.counters.FileSystemCounter;
+import org.apache.tez.common.counters.TezCounter;
+import org.apache.tez.dag.api.TezConfiguration;
+import org.apache.tez.dag.api.TezException;
+import org.apache.tez.history.parser.datamodel.DagInfo;
+import org.apache.tez.history.parser.datamodel.TaskAttemptInfo;
+import org.apache.tez.history.parser.datamodel.VertexInfo;
+
+import java.util.List;
+import java.util.Map;
+
+
+/**
+ * Get locality information for tasks for vertices and get their task execution times.
+ * This would be helpeful to co-relate if the vertex runtime is anyways related to the data
+ * locality.
+ */
+public class LocalityAnalyzer extends TezAnalyzerBase implements Analyzer {
+
+ private final String[] headers = { "vertexName", "numTasks", "dataLocalRatio", "rackLocalRatio",
+ "otherRatio", "avgDataLocalTaskRuntime", "avgRackLocalTaskRuntime",
+ "avgOtherLocalTaskRuntime", "noOfInputs", "avgHDFSBytesRead_DataLocal",
+ "avgHDFSBytesRead_RackLocal", "avgHDFSBytesRead_Others", "recommendation" };
+
+ private static final String DATA_LOCAL_RATIO = "tez.locality-analyzer.data.local.ratio";
+ private static final float DATA_LOCAL_RATIO_DEFAULT = 0.5f;
+
+ private final Configuration config;
+
+ private final CSVResult csvResult;
+
+ public LocalityAnalyzer(Configuration config) {
+ this.config = config;
+ csvResult = new CSVResult(headers);
+ }
+
+ @Override
+ public void analyze(DagInfo dagInfo) throws TezException {
+ for (VertexInfo vertexInfo : dagInfo.getVertices()) {
+ String vertexName = vertexInfo.getVertexName();
+
+ Map<String, TezCounter> dataLocalTask = vertexInfo.getCounter(DAGCounter.class.getName(),
+ DAGCounter.DATA_LOCAL_TASKS.toString());
+ Map<String, TezCounter> rackLocalTask = vertexInfo.getCounter(DAGCounter.class.getName(),
+ DAGCounter.RACK_LOCAL_TASKS.toString());
+
+ long dataLocalTasks = 0;
+ long rackLocalTasks = 0;
+
+ if (!dataLocalTask.isEmpty()) {
+ dataLocalTasks = dataLocalTask.get(DAGCounter.class.getName()).getValue();
+ }
+
+ if (!rackLocalTask.isEmpty()) {
+ rackLocalTasks = rackLocalTask.get(DAGCounter.class.getName()).getValue();
+ }
+
+ long totalVertexTasks = vertexInfo.getNumTasks();
+
+ if (dataLocalTasks > 0 || rackLocalTasks > 0) {
+ //compute locality details.
+ float dataLocalRatio = dataLocalTasks * 1.0f / totalVertexTasks;
+ float rackLocalRatio = rackLocalTasks * 1.0f / totalVertexTasks;
+ float othersRatio = (totalVertexTasks - (dataLocalTasks + rackLocalTasks)) * 1.0f /
+ totalVertexTasks;
+
+ List<String> record = Lists.newLinkedList();
+ record.add(vertexName);
+ record.add(totalVertexTasks + "");
+ record.add(dataLocalRatio + "");
+ record.add(rackLocalRatio + "");
+ record.add(othersRatio + "");
+
+ TaskAttemptDetails dataLocalResult = computeAverages(vertexInfo,
+ DAGCounter.DATA_LOCAL_TASKS);
+ TaskAttemptDetails rackLocalResult = computeAverages(vertexInfo,
+ DAGCounter.RACK_LOCAL_TASKS);
+ TaskAttemptDetails otherTaskResult = computeAverages(vertexInfo,
+ DAGCounter.OTHER_LOCAL_TASKS);
+
+ record.add(dataLocalResult.avgRuntime + "");
+ record.add(rackLocalResult.avgRuntime + "");
+ record.add(otherTaskResult.avgRuntime + "");
+
+ //Get the number of inputs to this vertex
+ record.add(vertexInfo.getInputEdges().size()
+ + vertexInfo.getAdditionalInputInfoList().size() + "");
+
+ //Get the avg HDFS bytes read in this vertex for different type of locality
+ record.add(dataLocalResult.avgHDFSBytesRead + "");
+ record.add(rackLocalResult.avgHDFSBytesRead + "");
+ record.add(otherTaskResult.avgHDFSBytesRead + "");
+
+ String recommendation = "";
+ if (dataLocalRatio < config.getFloat(DATA_LOCAL_RATIO, DATA_LOCAL_RATIO_DEFAULT)) {
+ recommendation = "Data locality is poor for this vertex. Try tuning "
+ + TezConfiguration.TEZ_AM_CONTAINER_REUSE_LOCALITY_DELAY_ALLOCATION_MILLIS + ", "
+ + TezConfiguration.TEZ_AM_CONTAINER_REUSE_RACK_FALLBACK_ENABLED + ", "
+ + TezConfiguration.TEZ_AM_CONTAINER_REUSE_NON_LOCAL_FALLBACK_ENABLED;
+ }
+
+ record.add(recommendation);
+ csvResult.addRecord(record.toArray(new String[record.size()]));
+ }
+ }
+ }
+
+ /**
+ * Compute counter averages for specific vertex
+ *
+ * @param vertexInfo
+ * @param counter
+ * @return task attempt details
+ */
+ private TaskAttemptDetails computeAverages(VertexInfo vertexInfo, DAGCounter counter) {
+ long totalTime = 0;
+ long totalTasks = 0;
+ long totalHDFSBytesRead = 0;
+
+ TaskAttemptDetails result = new TaskAttemptDetails();
+
+ for(TaskAttemptInfo attemptInfo : vertexInfo.getTaskAttempts()) {
+ Map<String, TezCounter> localityCounter = attemptInfo.getCounter(DAGCounter.class.getName(),
+ counter.toString());
+
+ if (!localityCounter.isEmpty() &&
+ localityCounter.get(DAGCounter.class.getName()).getValue() > 0) {
+ totalTime += attemptInfo.getTimeTaken();
+ totalTasks++;
+
+ //get HDFSBytes read counter
+ Map<String, TezCounter> hdfsBytesReadCounter = attemptInfo.getCounter(FileSystemCounter
+ .class.getName(), FileSystemCounter.HDFS_BYTES_READ.name());
+ for(Map.Entry<String, TezCounter> entry : hdfsBytesReadCounter.entrySet()) {
+ totalHDFSBytesRead += entry.getValue().getValue();
+ }
+ }
+ }
+ if (totalTasks > 0) {
+ result.avgRuntime = (totalTime * 1.0f / totalTasks);
+ result.avgHDFSBytesRead = (totalHDFSBytesRead * 1.0f / totalTasks);
+ }
+ return result;
+ }
+
+ @Override public CSVResult getResult() throws TezException {
+ return csvResult;
+ }
+
+ @Override public String getName() {
+ return "Locality Analyzer";
+ }
+
+ @Override public String getDescription() {
+ return "Analyze for locality information (data local, rack local, off-rack)";
+ }
+
+ @Override public Configuration getConfiguration() {
+ return config;
+ }
+
+ /**
+ * Placeholder for task attempt details
+ */
+ static class TaskAttemptDetails {
+ float avgHDFSBytesRead;
+ float avgRuntime;
+ }
+
+ public static void main(String[] args) throws Exception {
+ Configuration config = new Configuration();
+ LocalityAnalyzer analyzer = new LocalityAnalyzer(config);
+ int res = ToolRunner.run(config, analyzer, args);
+ analyzer.printResults();
+ System.exit(res);
+ }
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/8c8db7c5/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/ShuffleTimeAnalyzer.java
----------------------------------------------------------------------
diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/ShuffleTimeAnalyzer.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/ShuffleTimeAnalyzer.java
new file mode 100644
index 0000000..57e91c6
--- /dev/null
+++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/ShuffleTimeAnalyzer.java
@@ -0,0 +1,223 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.analyzer.plugins;
+
+import com.google.common.base.Strings;
+import com.google.common.collect.Lists;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.tez.analyzer.Analyzer;
+import org.apache.tez.analyzer.CSVResult;
+import org.apache.tez.common.counters.TaskCounter;
+import org.apache.tez.common.counters.TezCounter;
+import org.apache.tez.dag.api.TezException;
+import org.apache.tez.history.parser.datamodel.DagInfo;
+import org.apache.tez.history.parser.datamodel.TaskAttemptInfo;
+import org.apache.tez.history.parser.datamodel.VertexInfo;
+
+import java.util.List;
+import java.util.Map;
+
+
+/**
+ * Analyze the time taken by merge phase, shuffle phase, time taken to do realistic work etc in
+ * tasks.
+ *
+ * Just dump REDUCE_INPUT_GROUPS, REDUCE_INPUT_RECORDS, its ratio and SHUFFLE_BYTES for tasks
+ * grouped by vertices. Provide time taken as well. Just render it as a table for now.
+ *
+ */
+public class ShuffleTimeAnalyzer extends TezAnalyzerBase implements Analyzer {
+
+ /**
+ * ratio of (total time taken by task - shuffle time) / (total time taken by task)
+ */
+ private static final String REAL_WORK_DONE_RATIO = "tez.shuffle-time-analyzer.real-work.done.ratio";
+ private static final float REAL_WORK_DONE_RATIO_DEFAULT = 0.5f;
+
+ /**
+ * Number of min records that needs to get in as reduce input records.
+ */
+ private static final String MIN_SHUFFLE_RECORDS = "tez.shuffle-time-analyzer.shuffle.min.records";
+ private static final long MIN_SHUFFLE_RECORDS_DEFAULT = 10000;
+
+ private static final String[] headers = { "vertexName", "taskAttemptId", "Node", "counterGroup",
+ "Comments", "REDUCE_INPUT_GROUPS", "REDUCE_INPUT_RECORDS", "ratio", "SHUFFLE_BYTES",
+ "TotalTime", "Time_taken_to_receive_all_events", "MERGE_PHASE_TIME", "SHUFFLE_PHASE_TIME",
+ "TimeTaken_For_Real_Task", "FIRST_EVENT_RECEIVED", "LAST_EVENT_RECEIVED",
+ "SHUFFLE_BYTES_DISK_DIRECT" };
+
+ private final CSVResult csvResult = new CSVResult(headers);
+
+ private final Configuration config;
+
+ private final float realWorkDoneRatio;
+ private final long minShuffleRecords;
+
+
+ public ShuffleTimeAnalyzer(Configuration config) {
+ this.config = config;
+
+ realWorkDoneRatio = config.getFloat
+ (REAL_WORK_DONE_RATIO, REAL_WORK_DONE_RATIO_DEFAULT);
+ minShuffleRecords = config.getLong(MIN_SHUFFLE_RECORDS, MIN_SHUFFLE_RECORDS_DEFAULT);
+ }
+
+ @Override
+ public void analyze(DagInfo dagInfo) throws TezException {
+
+ for (VertexInfo vertexInfo : dagInfo.getVertices()) {
+ for (TaskAttemptInfo attemptInfo : vertexInfo.getTaskAttempts()) {
+ //counter_group (basically source) --> counter
+ Map<String, TezCounter> reduceInputGroups = attemptInfo.getCounter(TaskCounter
+ .REDUCE_INPUT_GROUPS.toString());
+ Map<String, TezCounter> reduceInputRecords = attemptInfo.getCounter(TaskCounter
+ .REDUCE_INPUT_RECORDS.toString());
+
+ if (reduceInputGroups == null) {
+ continue;
+ }
+
+ for (Map.Entry<String, TezCounter> entry : reduceInputGroups.entrySet()) {
+ String counterGroupName = entry.getKey();
+ long reduceInputGroupsVal = entry.getValue().getValue();
+ long reduceInputRecordsVal = (reduceInputRecords.get(counterGroupName) != null) ?
+ reduceInputRecords.get(counterGroupName).getValue() : 0;
+
+ if (reduceInputRecordsVal <= 0) {
+ continue;
+ }
+ float ratio = (reduceInputGroupsVal * 1.0f / reduceInputRecordsVal);
+
+ if (ratio > 0 && reduceInputRecordsVal > minShuffleRecords) {
+ List<String> result = Lists.newLinkedList();
+ result.add(vertexInfo.getVertexName());
+ result.add(attemptInfo.getTaskAttemptId());
+ result.add(attemptInfo.getNodeId());
+ result.add(counterGroupName);
+
+ //Real work done in the task
+ String comments = "";
+ String mergePhaseTime = getCounterValue(TaskCounter.MERGE_PHASE_TIME,
+ counterGroupName, attemptInfo);
+ String timeTakenForRealWork = "";
+ if (!Strings.isNullOrEmpty(mergePhaseTime)) {
+ long realWorkDone = attemptInfo.getTimeTaken() - Long.parseLong(mergePhaseTime);
+
+ if ((realWorkDone * 1.0f / attemptInfo.getTimeTaken()) < realWorkDoneRatio) {
+ comments = "Time taken in shuffle is more than the actual work being done in task. "
+ + " Check if source/destination machine is a slow node. Check if merge phase "
+ + "time is more to understand disk bottlenecks in this node. Check for skew";
+ }
+
+ timeTakenForRealWork = Long.toString(realWorkDone);
+ }
+ result.add(comments);
+
+ result.add(reduceInputGroupsVal + "");
+ result.add(reduceInputRecordsVal + "");
+ result.add("" + (1.0f * reduceInputGroupsVal / reduceInputRecordsVal));
+ result.add(getCounterValue(TaskCounter.SHUFFLE_BYTES, counterGroupName, attemptInfo));
+
+ result.add(Long.toString(attemptInfo.getTimeTaken()));
+
+ //Total time taken for receiving all events from source tasks
+ result.add(getOverheadFromSourceTasks(counterGroupName, attemptInfo));
+ result.add(getCounterValue(TaskCounter.MERGE_PHASE_TIME, counterGroupName, attemptInfo));
+ result.add(getCounterValue(TaskCounter.SHUFFLE_PHASE_TIME, counterGroupName, attemptInfo));
+
+ result.add(timeTakenForRealWork);
+
+ result.add(getCounterValue(TaskCounter.FIRST_EVENT_RECEIVED, counterGroupName, attemptInfo));
+ result.add(getCounterValue(TaskCounter.LAST_EVENT_RECEIVED, counterGroupName, attemptInfo));
+ result.add(getCounterValue(TaskCounter.SHUFFLE_BYTES_DISK_DIRECT, counterGroupName, attemptInfo));
+
+ csvResult.addRecord(result.toArray(new String[result.size()]));
+ }
+ }
+ }
+ }
+
+ }
+
+ /**
+ * Time taken to receive all events from source tasks
+ *
+ * @param counterGroupName
+ * @param attemptInfo
+ * @return String
+ */
+ private String getOverheadFromSourceTasks(String counterGroupName, TaskAttemptInfo attemptInfo) {
+ String firstEventReceived = getCounterValue(TaskCounter.FIRST_EVENT_RECEIVED,
+ counterGroupName, attemptInfo);
+ String lastEventReceived = getCounterValue(TaskCounter.LAST_EVENT_RECEIVED,
+ counterGroupName, attemptInfo);
+
+ if (!Strings.isNullOrEmpty(firstEventReceived) && !Strings.isNullOrEmpty(lastEventReceived)) {
+ return Long.toString(Long.parseLong(lastEventReceived) - Long.parseLong(firstEventReceived));
+ } else {
+ return "";
+ }
+ }
+
+ private String getCounterValue(TaskCounter counter, String counterGroupName,
+ TaskAttemptInfo attemptInfo) {
+ Map<String, TezCounter> tezCounterMap = attemptInfo.getCounter(counter.toString());
+ if (tezCounterMap != null) {
+ for (Map.Entry<String, TezCounter> entry : tezCounterMap.entrySet()) {
+ String groupName = entry.getKey();
+ long val = entry.getValue().getValue();
+ if (groupName.equals(counterGroupName)) {
+ return Long.toString(val);
+ }
+ }
+ }
+ return "";
+ }
+
+ @Override
+ public CSVResult getResult() throws TezException {
+ return csvResult;
+ }
+
+ @Override
+ public String getName() {
+ return "Shuffle time analyzer";
+ }
+
+ @Override
+ public String getDescription() {
+ return "Analyze the time taken for shuffle, merge "
+ + "and the real work done in the task";
+ }
+
+ @Override
+ public Configuration getConfiguration() {
+ return config;
+ }
+
+ public static void main(String[] args) throws Exception {
+ Configuration config = new Configuration();
+ ShuffleTimeAnalyzer analyzer = new ShuffleTimeAnalyzer(config);
+ int res = ToolRunner.run(config, analyzer, args);
+ analyzer.printResults();
+ System.exit(res);
+ }
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/8c8db7c5/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/SkewAnalyzer.java
----------------------------------------------------------------------
diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/SkewAnalyzer.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/SkewAnalyzer.java
new file mode 100644
index 0000000..067d871
--- /dev/null
+++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/SkewAnalyzer.java
@@ -0,0 +1,323 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.analyzer.plugins;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.tez.analyzer.Analyzer;
+import org.apache.tez.analyzer.CSVResult;
+import org.apache.tez.common.counters.TaskCounter;
+import org.apache.tez.common.counters.TezCounter;
+import org.apache.tez.dag.api.TezException;
+import org.apache.tez.history.parser.datamodel.DagInfo;
+import org.apache.tez.history.parser.datamodel.TaskAttemptInfo;
+import org.apache.tez.history.parser.datamodel.VertexInfo;
+
+import java.util.List;
+import java.util.Map;
+
+
+/**
+ * <p/>
+ * Identify the skew (RECORD_INPUT_GROUPS / REDUCE_INPUT_RECORDS) ratio for all task attempts
+ * and report if they are below a certain threshold.
+ * <p/>
+ * <p/>
+ * - Case 1: Ratio of (reduce_input_groups / reduce_input_records) < 0.2 && SHUFFLE_BYTES > 1 GB
+ * per task attempt from a source. This means couple of keys having too many records. Either
+ * partitioning is wrong, or we need to increase memory limit for this vertex.
+ * <p/>
+ * - Case 2: Ratio of (reduce_input_groups / reduce_input_records) > 0.6 & Number of reduce input
+ * records in task attempt is closer to say 60% of overall number of records
+ * in vertex level & numTasks in vertex is greater than 1. This might have any number of reducer
+ * groups. This means that, partitioning is wrong (can also consider reducing number of tasks
+ * for that vertex). In some cases, too many reducers are launched and this can help find those.
+ * <p/>
+ * - Case 3: Ratio of (reduce_input_groups / reduce_input_records) is between 0.2 & 0.6 per task
+ * attempt & numTasks is greater than 1 & SHUFFLE_BYTES > 1 GB per task attempt from a
+ * source. This means, may be consider increasing parallelism based on the task attempt runtime.
+ * <p/>
+ */
+public class SkewAnalyzer extends TezAnalyzerBase implements Analyzer {
+
+ /**
+ * Amount of bytes that was sent as shuffle bytes from source. If it is below this threshold,
+ * it would not be considered for analysis.
+ */
+ private static final String SHUFFLE_BYTES_PER_ATTEMPT_PER_SOURCE = "tez.skew-analyzer.shuffle"
+ + ".bytes.per.source";
+ private static final long SHUFFLE_BYTES_PER_ATTEMPT_PER_SOURCE_DEFAULT = 900 * 1024 * 1024l;
+
+ //Min reducer input group : reducer keys ratio for computation
+ private static final String ATTEMPT_SHUFFLE_KEY_GROUP_MIN_RATIO = "tez.skew-analyzer.shuffle.key"
+ + ".group.min.ratio";
+ private static final float ATTEMPT_SHUFFLE_KEY_GROUP_MIN_RATIO_DEFAULT = 0.2f;
+
+ //Max reducer input group : reducer keys ratio for computation
+ private static final String ATTEMPT_SHUFFLE_KEY_GROUP_MAX_RATIO = "tez.skew-analyzer.shuffle.key"
+ + ".group.max.ratio";
+ private static final float ATTEMPT_SHUFFLE_KEY_GROUP_MAX_RATIO_DEFAULT = 0.4f;
+
+
+
+ private static final String[] headers = { "vertexName", "taskAttemptId", "counterGroup", "node",
+ "REDUCE_INPUT_GROUPS", "REDUCE_INPUT_RECORDS", "ratio", "SHUFFLE_BYTES", "timeTaken",
+ "observation" };
+
+ private final CSVResult csvResult = new CSVResult(headers);
+
+ private final Configuration config;
+
+ private final float minRatio;
+ private final float maxRatio;
+ private final long maxShuffleBytesPerSource;
+
+ public SkewAnalyzer(Configuration config) {
+ this.config = config;
+ maxRatio = config.getFloat(ATTEMPT_SHUFFLE_KEY_GROUP_MAX_RATIO,
+ ATTEMPT_SHUFFLE_KEY_GROUP_MAX_RATIO_DEFAULT);
+ minRatio = config.getFloat(ATTEMPT_SHUFFLE_KEY_GROUP_MIN_RATIO,
+ ATTEMPT_SHUFFLE_KEY_GROUP_MIN_RATIO_DEFAULT);
+ maxShuffleBytesPerSource = config.getLong(SHUFFLE_BYTES_PER_ATTEMPT_PER_SOURCE,
+ SHUFFLE_BYTES_PER_ATTEMPT_PER_SOURCE_DEFAULT);
+ }
+
+ @Override
+ public void analyze(DagInfo dagInfo) throws TezException {
+ Preconditions.checkArgument(dagInfo != null, "DAG can't be null");
+ analyzeReducers(dagInfo);
+ }
+
+ private void analyzeReducers(DagInfo dagInfo) {
+ for (VertexInfo vertexInfo : dagInfo.getVertices()) {
+ for (TaskAttemptInfo attemptInfo : vertexInfo.getTaskAttempts()) {
+ analyzeGroupSkewPerSource(attemptInfo);
+ analyzeRecordSkewPerSource(attemptInfo);
+ analyzeForParallelism(attemptInfo);
+ }
+ }
+ }
+
+ /**
+ * Analyze scenario where couple keys are having too many records per source
+ *
+ * @param attemptInfo
+ */
+ private void analyzeGroupSkewPerSource(TaskAttemptInfo attemptInfo) {
+
+ //counter_group (basically source) --> counter
+ Map<String, TezCounter> reduceInputGroups = attemptInfo.getCounter(TaskCounter
+ .REDUCE_INPUT_GROUPS.toString());
+ Map<String, TezCounter> reduceInputRecords = attemptInfo.getCounter(TaskCounter
+ .REDUCE_INPUT_RECORDS.toString());
+ Map<String, TezCounter> shuffleBytes = attemptInfo.getCounter(TaskCounter.SHUFFLE_BYTES.toString());
+
+
+ //tez counter for every source
+ for (Map.Entry<String, TezCounter> entry : reduceInputGroups.entrySet()) {
+ if (entry.getKey().equals(TaskCounter.class.getName())) {
+ //TODO: Tez counters always ends up adding fgroups and groups, due to which we end up
+ // getting TaskCounter details as well.
+ continue;
+ }
+
+ String counterGroup = entry.getKey();
+ long inputGroupsCount = entry.getValue().getValue();
+ long inputRecordsCount = (reduceInputRecords.get(counterGroup) != null) ? reduceInputRecords
+ .get(counterGroup).getValue() : 0;
+ long shuffleBytesPerSource = (shuffleBytes.get(counterGroup) != null) ? shuffleBytes.get
+ (counterGroup).getValue() : 0;
+
+ float ratio = (inputGroupsCount * 1.0f / inputRecordsCount);
+
+ //Case 1: Couple of keys having too many records per source.
+ if (shuffleBytesPerSource > maxShuffleBytesPerSource) {
+ if (ratio < minRatio) {
+ List<String> result = Lists.newLinkedList();
+ result.add(attemptInfo.getTaskInfo().getVertexInfo().getVertexName());
+ result.add(attemptInfo.getTaskAttemptId());
+ result.add(counterGroup);
+ result.add(attemptInfo.getNodeId());
+ result.add(inputGroupsCount + "");
+ result.add(inputRecordsCount + "");
+ result.add(ratio + "");
+ result.add(shuffleBytesPerSource + "");
+ result.add(attemptInfo.getTimeTaken() + "");
+ result.add("Please check partitioning. Otherwise consider increasing memLimit");
+
+ csvResult.addRecord(result.toArray(new String[result.size()]));
+ }
+ }
+ }
+ }
+
+ /**
+ * Analyze scenario where one task is getting > 60% of the vertex level records
+ *
+ * @param attemptInfo
+ */
+ private void analyzeRecordSkewPerSource(TaskAttemptInfo attemptInfo) {
+
+ Map<String, TezCounter> vertexLevelReduceInputRecords =
+ attemptInfo.getTaskInfo().getVertexInfo()
+ .getCounter(TaskCounter.REDUCE_INPUT_RECORDS.toString());
+
+ int vertexNumTasks = attemptInfo.getTaskInfo().getVertexInfo().getNumTasks();
+
+ //counter_group (basically source) --> counter
+ Map<String, TezCounter> reduceInputGroups = attemptInfo.getCounter(TaskCounter
+ .REDUCE_INPUT_GROUPS.toString());
+ Map<String, TezCounter> reduceInputRecords = attemptInfo.getCounter(TaskCounter
+ .REDUCE_INPUT_RECORDS.toString());
+ Map<String, TezCounter> shuffleBytes = attemptInfo.getCounter(TaskCounter.SHUFFLE_BYTES.toString());
+
+
+ //tez counter for every source
+ for (Map.Entry<String, TezCounter> entry : reduceInputGroups.entrySet()) {
+ if (entry.getKey().equals(TaskCounter.class.getName())) {
+ //TODO: Tez counters always ends up adding fgroups and groups, due to which we end up
+ // getting TaskCounter details as well.
+ continue;
+ }
+
+ String counterGroup = entry.getKey();
+ long inputGroupsCount = entry.getValue().getValue();
+ long inputRecordsCount = (reduceInputRecords.get(counterGroup) != null) ? reduceInputRecords
+ .get(counterGroup).getValue() : 0;
+ long shuffleBytesPerSource = (shuffleBytes.get(counterGroup) != null) ?shuffleBytes.get
+ (counterGroup).getValue() : 0;
+ long vertexLevelInputRecordsCount = (vertexLevelReduceInputRecords.get(counterGroup) !=
+ null) ?
+ vertexLevelReduceInputRecords.get(counterGroup).getValue() : 0;
+
+ float ratio = (inputRecordsCount * 1.0f / vertexLevelInputRecordsCount);
+
+ if (vertexNumTasks > 1) {
+ if (ratio > maxRatio) {
+ //input records > 60% of vertex level record count
+ if (inputRecordsCount > (vertexLevelInputRecordsCount * 0.60)) {
+ List<String> result = Lists.newLinkedList();
+ result.add(attemptInfo.getTaskInfo().getVertexInfo().getVertexName());
+ result.add(attemptInfo.getTaskAttemptId());
+ result.add(counterGroup);
+ result.add(attemptInfo.getNodeId());
+ result.add(inputGroupsCount + "");
+ result.add(inputRecordsCount + "");
+ result.add(ratio + "");
+ result.add(shuffleBytesPerSource + "");
+ result.add(attemptInfo.getTimeTaken() + "");
+ result.add("Some task attempts are getting > 60% of reduce input records. "
+ + "Consider adjusting parallelism & check partition logic");
+
+ csvResult.addRecord(result.toArray(new String[result.size()]));
+
+ }
+ }
+ }
+ }
+ }
+
+ /**
+ * Analyze scenario where a vertex would need to increase parallelism
+ *
+ * @param attemptInfo
+ */
+ private void analyzeForParallelism(TaskAttemptInfo attemptInfo) {
+
+ //counter_group (basically source) --> counter
+ Map<String, TezCounter> reduceInputGroups = attemptInfo.getCounter(TaskCounter
+ .REDUCE_INPUT_GROUPS.toString());
+ Map<String, TezCounter> reduceInputRecords = attemptInfo.getCounter(TaskCounter
+ .REDUCE_INPUT_RECORDS.toString());
+ Map<String, TezCounter> shuffleBytes = attemptInfo.getCounter(TaskCounter.SHUFFLE_BYTES.toString());
+
+ //tez counter for every source
+ for (Map.Entry<String, TezCounter> entry : reduceInputGroups.entrySet()) {
+ if (entry.getKey().equals(TaskCounter.class.getName())) {
+ //TODO: Tez counters always ends up adding fgroups and groups, due to which we end up
+ // getting TaskCounter details as well.
+ continue;
+ }
+
+ String counterGroup = entry.getKey();
+ long inputGroupsCount = entry.getValue().getValue();
+ long inputRecordsCount = (reduceInputRecords.get(counterGroup) != null) ? reduceInputRecords
+ .get(counterGroup).getValue() : 0;
+ long shuffleBytesPerSource = (shuffleBytes.get(counterGroup) != null) ? shuffleBytes.get
+ (counterGroup).getValue() : 0;
+
+ float ratio = (inputGroupsCount * 1.0f / inputRecordsCount);
+
+ //Case 3: Shuffle_Bytes > 1 GB. Ratio between 0.2 & < 0.6. Consider increasing
+ // parallelism based on task runtime.
+ if (shuffleBytesPerSource > SHUFFLE_BYTES_PER_ATTEMPT_PER_SOURCE_DEFAULT) {
+ if (ratio > minRatio && ratio < maxRatio) {
+ //couple of keys have too many records. Classic case of partition issue.
+ List<String> result = Lists.newLinkedList();
+ result.add(attemptInfo.getTaskInfo().getVertexInfo().getVertexName());
+ result.add(attemptInfo.getTaskAttemptId());
+ result.add(counterGroup);
+ result.add(attemptInfo.getNodeId());
+ result.add(inputGroupsCount + "");
+ result.add(inputRecordsCount + "");
+ result.add(ratio + "");
+ result.add(shuffleBytesPerSource + "");
+ result.add(attemptInfo.getTimeTaken() + "");
+ result.add("Consider increasing parallelism.");
+
+ csvResult.addRecord(result.toArray(new String[result.size()]));
+ }
+ }
+ }
+
+
+ }
+
+
+ @Override
+ public CSVResult getResult() throws TezException {
+ return csvResult;
+ }
+
+ @Override
+ public String getName() {
+ return "Skew Analyzer";
+ }
+
+ @Override
+ public String getDescription() {
+ return "Analyzer reducer skews by mining reducer task counters";
+ }
+
+ @Override
+ public Configuration getConfiguration() {
+ return null;
+ }
+
+ public static void main(String[] args) throws Exception {
+ Configuration config = new Configuration();
+ SkewAnalyzer analyzer = new SkewAnalyzer(config);
+ int res = ToolRunner.run(config, analyzer, args);
+ analyzer.printResults();
+ System.exit(res);
+ }
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/8c8db7c5/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/SlowNodeAnalyzer.java
----------------------------------------------------------------------
diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/SlowNodeAnalyzer.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/SlowNodeAnalyzer.java
new file mode 100644
index 0000000..a810a8a
--- /dev/null
+++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/SlowNodeAnalyzer.java
@@ -0,0 +1,197 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.analyzer.plugins;
+
+import com.google.common.base.Predicate;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Multimap;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.tez.analyzer.Analyzer;
+import org.apache.tez.analyzer.CSVResult;
+import org.apache.tez.common.counters.FileSystemCounter;
+import org.apache.tez.common.counters.TaskCounter;
+import org.apache.tez.common.counters.TezCounter;
+import org.apache.tez.common.counters.TezCounters;
+import org.apache.tez.dag.api.TezException;
+import org.apache.tez.dag.api.oldrecords.TaskAttemptState;
+import org.apache.tez.history.parser.datamodel.DagInfo;
+import org.apache.tez.history.parser.datamodel.TaskAttemptInfo;
+
+import java.util.Collection;
+import java.util.List;
+
+
+/**
+ * This will provide the set of nodes participated in the DAG in descending order of task execution
+ * time.
+ * <p/>
+ * Combine it with other counters to understand slow nodes better.
+ */
+public class SlowNodeAnalyzer extends TezAnalyzerBase implements Analyzer {
+
+ private static final Log LOG = LogFactory.getLog(SlowNodeAnalyzer.class);
+
+ private static final String[] headers = { "nodeName", "noOfTasksExecuted", "noOfKilledTasks",
+ "noOfFailedTasks", "avgSucceededTaskExecutionTime", "avgKilledTaskExecutionTime",
+ "avgFailedTaskExecutionTime", "avgHDFSBytesRead", "avgHDFSBytesWritten",
+ "avgFileBytesRead", "avgFileBytesWritten", "avgGCTimeMillis", "avgCPUTimeMillis" };
+
+ private final CSVResult csvResult = new CSVResult(headers);
+
+ private final Configuration config;
+
+ public SlowNodeAnalyzer(Configuration config) {
+ this.config = config;
+ }
+
+ @Override
+ public void analyze(DagInfo dagInfo) throws TezException {
+ Multimap<String, TaskAttemptInfo> nodeDetails = dagInfo.getNodeDetails();
+ for (String nodeName : nodeDetails.keySet()) {
+ List<String> record = Lists.newLinkedList();
+
+ Collection<TaskAttemptInfo> taskAttemptInfos = nodeDetails.get(nodeName);
+
+ record.add(nodeName);
+ record.add(taskAttemptInfos.size() + "");
+ record.add(getNumberOfTasks(taskAttemptInfos, TaskAttemptState.KILLED) + "");
+ record.add(getNumberOfTasks(taskAttemptInfos, TaskAttemptState.FAILED) + "");
+
+ Iterable<TaskAttemptInfo> succeedTasks = getFilteredTaskAttempts(taskAttemptInfos,
+ TaskAttemptState.SUCCEEDED);
+ record.add(getAvgTaskExecutionTime(succeedTasks) + "");
+
+ Iterable<TaskAttemptInfo> killedTasks = getFilteredTaskAttempts(taskAttemptInfos,
+ TaskAttemptState.KILLED);
+ record.add(getAvgTaskExecutionTime(killedTasks) + "");
+
+ Iterable<TaskAttemptInfo> failedTasks = getFilteredTaskAttempts(taskAttemptInfos,
+ TaskAttemptState.FAILED);
+ record.add(getAvgTaskExecutionTime(failedTasks) + "");
+
+ record.add(getAvgCounter(taskAttemptInfos, FileSystemCounter.class
+ .getName(), FileSystemCounter.HDFS_BYTES_READ.name()) + "");
+ record.add(getAvgCounter(taskAttemptInfos, FileSystemCounter.class
+ .getName(), FileSystemCounter.HDFS_BYTES_WRITTEN.name()) + "");
+ record.add(getAvgCounter(taskAttemptInfos, FileSystemCounter.class
+ .getName(), FileSystemCounter.FILE_BYTES_READ.name()) + "");
+ record.add(getAvgCounter(taskAttemptInfos, FileSystemCounter.class
+ .getName(), FileSystemCounter.FILE_BYTES_WRITTEN.name()) + "");
+ record.add(getAvgCounter(taskAttemptInfos, TaskCounter.class
+ .getName(), TaskCounter.GC_TIME_MILLIS.name()) + "");
+ record.add(getAvgCounter(taskAttemptInfos, TaskCounter.class
+ .getName(), TaskCounter.CPU_MILLISECONDS.name()) + "");
+
+ csvResult.addRecord(record.toArray(new String[record.size()]));
+ }
+ }
+
+ private Iterable<TaskAttemptInfo> getFilteredTaskAttempts(Collection<TaskAttemptInfo>
+ taskAttemptInfos, final TaskAttemptState status) {
+ return Iterables.filter(taskAttemptInfos, new
+ Predicate<TaskAttemptInfo>() {
+ @Override public boolean apply(TaskAttemptInfo input) {
+ return input.getStatus().equalsIgnoreCase(status.toString());
+ }
+ });
+ }
+
+ private float getAvgTaskExecutionTime(Iterable<TaskAttemptInfo> taskAttemptInfos) {
+ long totalTime = 0;
+ int size = 0;
+ for (TaskAttemptInfo attemptInfo : taskAttemptInfos) {
+ totalTime += attemptInfo.getTimeTaken();
+ size++;
+ }
+ return (size > 0) ? (totalTime * 1.0f / size) : 0;
+ }
+
+ private int getNumberOfTasks(Collection<TaskAttemptInfo> taskAttemptInfos, TaskAttemptState
+ status) {
+ int tasks = 0;
+ for (TaskAttemptInfo attemptInfo : taskAttemptInfos) {
+ if (attemptInfo.getStatus().equalsIgnoreCase(status.toString())) {
+ tasks++;
+ }
+ }
+ return tasks;
+ }
+
+ private float getAvgCounter(Collection<TaskAttemptInfo> taskAttemptInfos, String
+ counterGroupName, String counterName) {
+ long total = 0;
+ int taskCount = 0;
+ for (TaskAttemptInfo attemptInfo : taskAttemptInfos) {
+ TezCounters tezCounters = attemptInfo.getTezCounters();
+ TezCounter counter = tezCounters.findCounter(counterGroupName, counterName);
+ if (counter != null) {
+ total += counter.getValue();
+ taskCount++;
+ } else {
+ LOG.info("Could not find counterGroupName=" + counterGroupName + ", counter=" +
+ counterName + " in " + attemptInfo);
+ }
+ }
+ return (taskCount > 0) ? (total * 1.0f / taskCount) : 0;
+ }
+
+ @Override
+ public CSVResult getResult() throws TezException {
+ return csvResult;
+ }
+
+ @Override
+ public String getName() {
+ return "Slow Node Analyzer";
+ }
+
+ @Override
+ public String getDescription() {
+ StringBuilder sb = new StringBuilder();
+ sb.append("Analyze node details for the DAG.").append("\n");
+ sb.append("This could be used to find out the set of nodes where the tasks are taking more "
+ + "time on average.").append("\n");
+ sb.append("This could be used to find out the set of nodes where the tasks are taking more "
+ + "time on average and to understand whether too many tasks got scheduled on a node.")
+ .append("\n");
+ sb.append("One needs to combine the task execution time with other metrics like bytes "
+ + "read/written etc to get better idea of bad nodes. In order to understand the slow "
+ + "nodes due to network, it might be worthwhile to consider the shuffle performance "
+ + "analyzer tool in tez-tools").append("\n");
+ return sb.toString();
+ }
+
+ @Override
+ public Configuration getConfiguration() {
+ return config;
+ }
+
+ public static void main(String[] args) throws Exception {
+ Configuration config = new Configuration();
+ SlowNodeAnalyzer analyzer = new SlowNodeAnalyzer(config);
+ int res = ToolRunner.run(config, analyzer, args);
+ analyzer.printResults();
+ System.exit(res);
+ }
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/8c8db7c5/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/SlowTaskIdentifier.java
----------------------------------------------------------------------
diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/SlowTaskIdentifier.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/SlowTaskIdentifier.java
new file mode 100644
index 0000000..d2474ad
--- /dev/null
+++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/SlowTaskIdentifier.java
@@ -0,0 +1,126 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.analyzer.plugins;
+
+import com.google.common.collect.Lists;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.tez.analyzer.Analyzer;
+import org.apache.tez.analyzer.CSVResult;
+import org.apache.tez.dag.api.TezException;
+import org.apache.tez.history.parser.datamodel.DagInfo;
+import org.apache.tez.history.parser.datamodel.TaskAttemptInfo;
+import org.apache.tez.history.parser.datamodel.VertexInfo;
+
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+
+
+/**
+ * Analyze slow tasks in the DAG. Top 100 tasks are listed by default.
+ *
+ * <p/>
+ * //TODO: We do not get counters for killed task attempts yet.
+ */
+public class SlowTaskIdentifier extends TezAnalyzerBase implements Analyzer {
+
+ private static final String[] headers = { "vertexName", "taskAttemptId",
+ "Node", "taskDuration", "Status", "diagnostics",
+ "NoOfInputs" };
+
+ private final CSVResult csvResult;
+
+ private static final String NO_OF_TASKS = "tez.slow-task-analyzer.task.count";
+ private static final int NO_OF_TASKS_DEFAULT = 100;
+
+ private final Configuration config;
+
+ public SlowTaskIdentifier(Configuration config) {
+ this.config = config;
+ this.csvResult = new CSVResult(headers);
+ }
+
+ @Override
+ public void analyze(DagInfo dagInfo) throws TezException {
+ List<TaskAttemptInfo> taskAttempts = Lists.newArrayList();
+ for(VertexInfo vertexInfo : dagInfo.getVertices()) {
+ taskAttempts.addAll(vertexInfo.getTaskAttempts());
+ }
+
+ //sort them by runtime in descending order
+ Collections.sort(taskAttempts, new Comparator<TaskAttemptInfo>() {
+ @Override public int compare(TaskAttemptInfo o1, TaskAttemptInfo o2) {
+ return (o1.getTimeTaken() > o2.getTimeTaken()) ? -1 :
+ ((o1.getTimeTaken() == o2.getTimeTaken()) ?
+ 0 : 1);
+ }
+ });
+
+ int limit = Math.min(taskAttempts.size(),
+ Math.max(0, config.getInt(NO_OF_TASKS, NO_OF_TASKS_DEFAULT)));
+
+ if (limit == 0) {
+ return;
+ }
+
+ for (int i = 0; i < limit - 1; i++) {
+ List<String> record = Lists.newLinkedList();
+ record.add(taskAttempts.get(i).getTaskInfo().getVertexInfo().getVertexName());
+ record.add(taskAttempts.get(i).getTaskAttemptId());
+ record.add(taskAttempts.get(i).getContainer().getHost());
+ record.add(taskAttempts.get(i).getTimeTaken() + "");
+ record.add(taskAttempts.get(i).getStatus());
+ record.add(taskAttempts.get(i).getDiagnostics());
+ record.add(taskAttempts.get(i).getTaskInfo().getVertexInfo().getInputEdges().size() + "");
+
+ csvResult.addRecord(record.toArray(new String[record.size()]));
+ }
+
+ }
+
+ @Override
+ public CSVResult getResult() throws TezException {
+ return csvResult;
+ }
+
+ @Override
+ public String getName() {
+ return "Slow Task Identifier";
+ }
+
+ @Override
+ public String getDescription() {
+ return "Identifies slow tasks in the DAG";
+ }
+
+ @Override
+ public Configuration getConfiguration() {
+ return config;
+ }
+
+ public static void main(String[] args) throws Exception {
+ Configuration config = new Configuration();
+ SlowTaskIdentifier analyzer = new SlowTaskIdentifier(config);
+ int res = ToolRunner.run(config, analyzer, args);
+ analyzer.printResults();
+ System.exit(res);
+ }
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/8c8db7c5/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/SlowestVertexAnalyzer.java
----------------------------------------------------------------------
diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/SlowestVertexAnalyzer.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/SlowestVertexAnalyzer.java
new file mode 100644
index 0000000..33f2421
--- /dev/null
+++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/SlowestVertexAnalyzer.java
@@ -0,0 +1,219 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.analyzer.plugins;
+
+import com.codahale.metrics.Histogram;
+import com.codahale.metrics.MetricRegistry;
+import com.google.common.collect.Lists;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.tez.analyzer.Analyzer;
+import org.apache.tez.analyzer.CSVResult;
+import org.apache.tez.common.counters.TaskCounter;
+import org.apache.tez.common.counters.TezCounter;
+import org.apache.tez.dag.api.TezException;
+import org.apache.tez.history.parser.datamodel.DagInfo;
+import org.apache.tez.history.parser.datamodel.TaskAttemptInfo;
+import org.apache.tez.history.parser.datamodel.TaskInfo;
+import org.apache.tez.history.parser.datamodel.VertexInfo;
+
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Identify the slowest vertex in the DAG.
+ */
+public class SlowestVertexAnalyzer extends TezAnalyzerBase implements Analyzer {
+
+ private static final String[] headers = { "vertexName", "taskAttempts", "totalTime",
+ "shuffleTime", "shuffleTime_Max", "LastEventReceived", "LastEventReceivedFrom",
+ "75thPercentile", "95thPercentile", "98thPercentile", "Median",
+ "observation", "comments" };
+
+ private final CSVResult csvResult = new CSVResult(headers);
+
+ private final Configuration config;
+ private final MetricRegistry metrics = new MetricRegistry();
+ private Histogram taskAttemptRuntimeHistorgram;
+
+ private final static String MAX_VERTEX_RUNTIME = "tez.slowest-vertex-analyzer.max.vertex.runtime";
+ private final static long MAX_VERTEX_RUNTIME_DEFAULT = 100000;
+
+ private final long vertexRuntimeThreshold;
+
+ public SlowestVertexAnalyzer(Configuration config) {
+ this.config = config;
+ this.vertexRuntimeThreshold = Math.max(1, config.getLong(MAX_VERTEX_RUNTIME,
+ MAX_VERTEX_RUNTIME_DEFAULT));
+
+ }
+
+ private long getTaskRuntime(VertexInfo vertexInfo) {
+ TaskInfo firstTaskToStart = vertexInfo.getFirstTaskToStart();
+ TaskInfo lastTaskToFinish = vertexInfo.getLastTaskToFinish();
+
+ DagInfo dagInfo = vertexInfo.getDagInfo();
+ long totalTime = ((lastTaskToFinish == null) ?
+ dagInfo.getFinishTime() : lastTaskToFinish.getFinishTime()) -
+ ((firstTaskToStart == null) ? dagInfo.getStartTime() : firstTaskToStart.getStartTime());
+ return totalTime;
+ }
+
+ @Override
+ public void analyze(DagInfo dagInfo) throws TezException {
+
+ for (VertexInfo vertexInfo : dagInfo.getVertices()) {
+ String vertexName = vertexInfo.getVertexName();
+ if (vertexInfo.getFirstTaskToStart() == null || vertexInfo.getLastTaskToFinish() == null) {
+ continue;
+ }
+
+ long totalTime = getTaskRuntime(vertexInfo);
+
+ long slowestLastEventTime = Long.MIN_VALUE;
+ String maxSourceName = "";
+ taskAttemptRuntimeHistorgram = metrics.histogram(vertexName);
+
+
+ for (TaskAttemptInfo attemptInfo : vertexInfo.getTaskAttempts()) {
+
+ taskAttemptRuntimeHistorgram.update(attemptInfo.getTimeTaken());
+
+ //Get the last event received from the incoming vertices
+ Map<String, TezCounter> lastEventReceivedMap = attemptInfo.getCounter(
+ TaskCounter.LAST_EVENT_RECEIVED.toString());
+
+ for (Map.Entry<String, TezCounter> entry : lastEventReceivedMap.entrySet()) {
+ if (entry.getKey().equals(TaskCounter.class.getName())) {
+ //TODO: Tez counters always ends up adding fgroups and groups, due to which we end up
+ // getting TaskCounter details as well.
+ continue;
+ }
+ //Find the slowest last event received
+ if (entry.getValue().getValue() > slowestLastEventTime) {
+ slowestLastEventTime = entry.getValue().getValue();
+ maxSourceName = entry.getKey();
+ }
+ }
+ }
+
+ long shuffleMax = Long.MIN_VALUE;
+ String shuffleMaxSource = "";
+ for (TaskAttemptInfo attemptInfo : vertexInfo.getTaskAttempts()) {
+ //Get the last event received from the incoming vertices
+ Map<String, TezCounter> lastEventReceivedMap = attemptInfo.getCounter(
+ TaskCounter.SHUFFLE_PHASE_TIME.toString());
+
+ for (Map.Entry<String, TezCounter> entry : lastEventReceivedMap.entrySet()) {
+ if (entry.getKey().equals(TaskCounter.class.getName())) {
+ //ignore. TODO: hack for taskcounter issue
+ continue;
+ }
+ //Find the slowest last event received
+ if (entry.getValue().getValue() > shuffleMax) {
+ shuffleMax = entry.getValue().getValue();
+ shuffleMaxSource = entry.getKey();
+ }
+ }
+ }
+
+ String comments = "";
+
+ List<String> record = Lists.newLinkedList();
+ record.add(vertexName);
+ record.add(vertexInfo.getTaskAttempts().size() + "");
+ record.add(totalTime + "");
+ record.add(Math.max(0, shuffleMax) + "");
+ record.add(shuffleMaxSource);
+ record.add(Math.max(0, slowestLastEventTime) + "");
+ record.add(maxSourceName);
+ //Finding out real_work done at vertex level might be meaningless (as it is quite posisble
+ // that it went to starvation).
+
+ StringBuilder sb = new StringBuilder();
+ double percentile75 = taskAttemptRuntimeHistorgram.getSnapshot().get75thPercentile();
+ double percentile95 = taskAttemptRuntimeHistorgram.getSnapshot().get95thPercentile();
+ double percentile98 = taskAttemptRuntimeHistorgram.getSnapshot().get98thPercentile();
+ double percentile99 = taskAttemptRuntimeHistorgram.getSnapshot().get99thPercentile();
+ double medianAttemptRuntime = taskAttemptRuntimeHistorgram.getSnapshot().getMedian();
+
+ record.add("75th=" + percentile75);
+ record.add("95th=" + percentile95);
+ record.add("98th=" + percentile98);
+ record.add("median=" + medianAttemptRuntime);
+
+ if (percentile75 / percentile99 < 0.5) {
+ //looks like some straggler task is there.
+ sb.append("Looks like some straggler task is there");
+ }
+
+ record.add(sb.toString());
+
+ if (totalTime > 0 && vertexInfo.getTaskAttempts().size() > 0) {
+ if ((shuffleMax * 1.0f / totalTime) > 0.5) {
+ if ((slowestLastEventTime * 1.0f / totalTime) > 0.5) {
+ comments = "This vertex is slow due to its dependency on parent. Got a lot delayed last"
+ + " event received";
+ } else {
+ comments =
+ "Spending too much time on shuffle. Check shuffle bytes from previous vertex";
+ }
+ } else {
+ if (totalTime > vertexRuntimeThreshold) { //greater than X seconds.
+ comments = "Concentrate on this vertex (totalTime > " + vertexRuntimeThreshold
+ + " seconds)";
+ }
+ }
+ }
+
+ record.add(comments);
+ csvResult.addRecord(record.toArray(new String[record.size()]));
+ }
+ }
+
+
+ @Override
+ public CSVResult getResult() throws TezException {
+ return csvResult;
+ }
+
+ @Override
+ public String getName() {
+ return "SlowVertexAnalyzer";
+ }
+
+ @Override
+ public String getDescription() {
+ return "Identify the slowest vertex in the DAG, which needs to be looked into first";
+ }
+
+ @Override
+ public Configuration getConfiguration() {
+ return config;
+ }
+
+ public static void main(String[] args) throws Exception {
+ Configuration config = new Configuration();
+ SlowestVertexAnalyzer analyzer = new SlowestVertexAnalyzer(config);
+ int res = ToolRunner.run(config, analyzer, args);
+ analyzer.printResults();
+ System.exit(res);
+ }
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/8c8db7c5/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/SpillAnalyzerImpl.java
----------------------------------------------------------------------
diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/SpillAnalyzerImpl.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/SpillAnalyzerImpl.java
new file mode 100644
index 0000000..d69ca23
--- /dev/null
+++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/SpillAnalyzerImpl.java
@@ -0,0 +1,145 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.analyzer.plugins;
+
+import com.google.common.collect.Lists;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.tez.analyzer.Analyzer;
+import org.apache.tez.analyzer.CSVResult;
+import org.apache.tez.common.counters.TaskCounter;
+import org.apache.tez.common.counters.TezCounter;
+import org.apache.tez.dag.api.TezException;
+import org.apache.tez.history.parser.datamodel.DagInfo;
+import org.apache.tez.history.parser.datamodel.TaskAttemptInfo;
+import org.apache.tez.history.parser.datamodel.VertexInfo;
+import org.apache.tez.runtime.library.api.TezRuntimeConfiguration;
+
+import java.util.List;
+import java.util.Map;
+
+
+/**
+ * Find out tasks which have more than 1 spill (ADDITIONAL_SPILL_COUNT).
+ * <p/>
+ * Accompany this with OUTPUT_BYTES (> 1 GB data written)
+ */
+public class SpillAnalyzerImpl extends TezAnalyzerBase implements Analyzer {
+
+ private static final String[] headers = { "vertexName", "taskAttemptId",
+ "Node", "counterGroupName",
+ "spillCount", "taskDuration",
+ "OUTPUT_BYTES", "OUTPUT_RECORDS",
+ "SPILLED_RECORDS", "Recommendation" };
+
+ private final CSVResult csvResult;
+
+ /**
+ * Minimum output bytes that should be chunrned out by a task
+ */
+ private static final String OUTPUT_BYTES_THRESHOLD = "tez.spill-analyzer.min.output.bytes"
+ + ".threshold";
+ private static long OUTPUT_BYTES_THRESHOLD_DEFAULT = 1 * 1024 * 1024 * 1024l;
+
+ private final long minOutputBytesPerTask;
+
+ private final Configuration config;
+
+ public SpillAnalyzerImpl(Configuration config) {
+ this.config = config;
+ minOutputBytesPerTask = Math.max(0, config.getLong(OUTPUT_BYTES_THRESHOLD,
+ OUTPUT_BYTES_THRESHOLD_DEFAULT));
+ this.csvResult = new CSVResult(headers);
+ }
+
+ @Override
+ public void analyze(DagInfo dagInfo) throws TezException {
+ for (VertexInfo vertexInfo : dagInfo.getVertices()) {
+ String vertexName = vertexInfo.getVertexName();
+
+ for (TaskAttemptInfo attemptInfo : vertexInfo.getTaskAttempts()) {
+ //Get ADDITIONAL_SPILL_COUNT, OUTPUT_BYTES for every source
+ Map<String, TezCounter> spillCountMap =
+ attemptInfo.getCounter(TaskCounter.ADDITIONAL_SPILL_COUNT.name());
+ Map<String, TezCounter> spilledRecordsMap =
+ attemptInfo.getCounter(TaskCounter.SPILLED_RECORDS.name());
+ Map<String, TezCounter> outputRecordsMap =
+ attemptInfo.getCounter(TaskCounter.OUTPUT_RECORDS.name());
+
+ Map<String, TezCounter> outputBytesMap =
+ attemptInfo.getCounter(TaskCounter.OUTPUT_BYTES.name());
+
+ for (Map.Entry<String, TezCounter> entry : spillCountMap.entrySet()) {
+ String source = entry.getKey();
+ long spillCount = entry.getValue().getValue();
+ long outBytes = outputBytesMap.get(source).getValue();
+
+ long outputRecords = outputRecordsMap.get(source).getValue();
+ long spilledRecords = spilledRecordsMap.get(source).getValue();
+
+ if (spillCount > 1 && outBytes > minOutputBytesPerTask) {
+ List<String> recorList = Lists.newLinkedList();
+ recorList.add(vertexName);
+ recorList.add(attemptInfo.getTaskAttemptId());
+ recorList.add(attemptInfo.getNodeId());
+ recorList.add(source);
+ recorList.add(spillCount + "");
+ recorList.add(attemptInfo.getTimeTaken() + "");
+ recorList.add(outBytes + "");
+ recorList.add(outputRecords + "");
+ recorList.add(spilledRecords + "");
+ recorList.add("Consider increasing " + TezRuntimeConfiguration.TEZ_RUNTIME_IO_SORT_MB
+ + ". Try increasing container size.");
+
+ csvResult.addRecord(recorList.toArray(new String[recorList.size()]));
+ }
+ }
+ }
+ }
+ }
+
+ @Override
+ public CSVResult getResult() throws TezException {
+ return csvResult;
+ }
+
+ @Override
+ public String getName() {
+ return "SpillAnalyzer";
+ }
+
+ @Override
+ public String getDescription() {
+ return "Analyze spill details in the task";
+ }
+
+ @Override
+ public Configuration getConfiguration() {
+ return config;
+ }
+
+ public static void main(String[] args) throws Exception {
+ Configuration config = new Configuration();
+ SpillAnalyzerImpl analyzer = new SpillAnalyzerImpl(config);
+ int res = ToolRunner.run(config, analyzer, args);
+ analyzer.printResults();
+ System.exit(res);
+ }
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/8c8db7c5/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/TaskConcurrencyAnalyzer.java
----------------------------------------------------------------------
diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/TaskConcurrencyAnalyzer.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/TaskConcurrencyAnalyzer.java
new file mode 100644
index 0000000..070294f
--- /dev/null
+++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/TaskConcurrencyAnalyzer.java
@@ -0,0 +1,148 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.analyzer.plugins;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.TreeMultiset;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.tez.analyzer.Analyzer;
+import org.apache.tez.analyzer.CSVResult;
+import org.apache.tez.dag.api.TezException;
+import org.apache.tez.history.parser.datamodel.DagInfo;
+import org.apache.tez.history.parser.datamodel.TaskAttemptInfo;
+import org.apache.tez.history.parser.datamodel.VertexInfo;
+
+import java.util.Comparator;
+import java.util.List;
+
+/**
+ * Analyze concurrent tasks running in every vertex at regular intervals.
+ */
+public class TaskConcurrencyAnalyzer extends TezAnalyzerBase implements Analyzer {
+
+ private static final String[] headers = { "time", "vertexName", "concurrentTasksRunning" };
+
+ private final CSVResult csvResult;
+ private final Configuration config;
+
+ public TaskConcurrencyAnalyzer(Configuration conf) {
+ this.csvResult = new CSVResult(headers);
+ this.config = conf;
+ }
+
+ private enum EventType {START, FINISH}
+
+ static class TimeInfo {
+ EventType eventType;
+ long timestamp;
+ int concurrentTasks;
+
+ public TimeInfo(EventType eventType, long timestamp) {
+ this.eventType = eventType;
+ this.timestamp = timestamp;
+ }
+ }
+
+ @Override
+ public void analyze(DagInfo dagInfo) throws TezException {
+
+ //For each vertex find the concurrent tasks running at any point
+ for (VertexInfo vertexInfo : dagInfo.getVertices()) {
+ List<TaskAttemptInfo> taskAttempts =
+ Lists.newLinkedList(vertexInfo.getTaskAttempts(true, null));
+
+ String vertexName = vertexInfo.getVertexName();
+
+ /**
+ * - Get sorted multi-set of timestamps (S1, S2,...E1, E2..). Possible to have multiple
+ * tasks starting/ending at same time.
+ * - Walk through the set
+ * - Increment concurrent tasks when start event is encountered
+ * - Decrement concurrent tasks when start event is encountered
+ */
+ TreeMultiset<TimeInfo> timeInfoSet = TreeMultiset.create(new Comparator<TimeInfo>() {
+ @Override public int compare(TimeInfo o1, TimeInfo o2) {
+ return (o1.timestamp < o2.timestamp) ? -1 :
+ ((o1.timestamp == o2.timestamp) ? 0 : 1);
+ }
+ });
+
+ for (TaskAttemptInfo attemptInfo : taskAttempts) {
+ TimeInfo startTimeInfo = new TimeInfo(EventType.START, attemptInfo.getStartTime());
+ TimeInfo stopTimeInfo = new TimeInfo(EventType.FINISH, attemptInfo.getFinishTime());
+
+ timeInfoSet.add(startTimeInfo);
+ timeInfoSet.add(stopTimeInfo);
+ }
+
+ //Compute concurrent tasks in the list now.
+ int concurrentTasks = 0;
+ for(TimeInfo timeInfo : timeInfoSet.elementSet()) {
+ switch (timeInfo.eventType) {
+ case START:
+ concurrentTasks += timeInfoSet.count(timeInfo);
+ break;
+ case FINISH:
+ concurrentTasks -= timeInfoSet.count(timeInfo);
+ break;
+ default:
+ break;
+ }
+ timeInfo.concurrentTasks = concurrentTasks;
+ addToResult(vertexName, timeInfo.timestamp, timeInfo.concurrentTasks);
+ }
+ }
+ }
+
+ private void addToResult(String vertexName, long currentTime, int concurrentTasks) {
+ String[] record = { currentTime + "", vertexName, concurrentTasks + "" };
+ csvResult.addRecord(record);
+ }
+
+ @Override
+ public CSVResult getResult() throws TezException {
+ return csvResult;
+ }
+
+ @Override
+ public String getName() {
+ return "TaskConcurrencyAnalyzer";
+ }
+
+ @Override
+ public String getDescription() {
+ return "Analyze how many tasks were running in every vertex at given point in time. This "
+ + "would be helpful in understanding whether any starvation was there or not.";
+ }
+
+ @Override
+ public Configuration getConfiguration() {
+ return config;
+ }
+
+ public static void main(String[] args) throws Exception {
+ Configuration config = new Configuration();
+ TaskConcurrencyAnalyzer analyzer = new TaskConcurrencyAnalyzer(config);
+ int res = ToolRunner.run(config, analyzer, args);
+ analyzer.printResults();
+ System.exit(res);
+ }
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/8c8db7c5/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/TezAnalyzerBase.java
----------------------------------------------------------------------
diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/TezAnalyzerBase.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/TezAnalyzerBase.java
new file mode 100644
index 0000000..73e731a
--- /dev/null
+++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/TezAnalyzerBase.java
@@ -0,0 +1,213 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.analyzer.plugins;
+
+import java.io.File;
+import java.util.Iterator;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.GnuParser;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.OptionBuilder;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.ParseException;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.util.Tool;
+import org.apache.tez.analyzer.Analyzer;
+import org.apache.tez.analyzer.CSVResult;
+import org.apache.tez.analyzer.Result;
+import org.apache.tez.dag.api.TezException;
+import org.apache.tez.history.ATSImportTool;
+import org.apache.tez.history.parser.ATSFileParser;
+import org.apache.tez.history.parser.SimpleHistoryParser;
+import org.apache.tez.history.parser.datamodel.DagInfo;
+
+import com.google.common.base.Preconditions;
+
+public abstract class TezAnalyzerBase extends Configured implements Tool, Analyzer {
+
+
+ private static final String EVENT_FILE_NAME = "eventFileName";
+ private static final String OUTPUT_DIR = "outputDir";
+ private static final String SAVE_RESULTS = "saveResults";
+ private static final String DAG_ID = "dagId";
+ private static final String FROM_SIMPLE_HISTORY = "fromSimpleHistory";
+ private static final String HELP = "help";
+
+ private static final int SEPARATOR_WIDTH = 80;
+ private static final int MIN_COL_WIDTH = 12;
+
+ private String outputDir;
+ private boolean saveResults = false;
+
+ @SuppressWarnings("static-access")
+ private static Options buildOptions() {
+ Option dagIdOption = OptionBuilder.withArgName(DAG_ID).withLongOpt(DAG_ID)
+ .withDescription("DagId that needs to be analyzed").hasArg().isRequired(true).create();
+
+ Option outputDirOption = OptionBuilder.withArgName(OUTPUT_DIR).withLongOpt(OUTPUT_DIR)
+ .withDescription("Directory to write outputs to.").hasArg().isRequired(false).create();
+
+ Option saveResults = OptionBuilder.withArgName(SAVE_RESULTS).withLongOpt(SAVE_RESULTS)
+ .withDescription("Saves results to output directory (optional)")
+ .hasArg(false).isRequired(false).create();
+
+ Option eventFileNameOption = OptionBuilder.withArgName(EVENT_FILE_NAME).withLongOpt
+ (EVENT_FILE_NAME)
+ .withDescription("File with event data for the DAG").hasArg()
+ .isRequired(false).create();
+
+ Option fromSimpleHistoryOption = OptionBuilder.withArgName(FROM_SIMPLE_HISTORY).withLongOpt
+ (FROM_SIMPLE_HISTORY)
+ .withDescription("Event data from Simple History logging. Must also specify event file")
+ .isRequired(false).create();
+
+ Option help = OptionBuilder.withArgName(HELP).withLongOpt
+ (HELP)
+ .withDescription("print help")
+ .isRequired(false).create();
+
+ Options opts = new Options();
+ opts.addOption(dagIdOption);
+ opts.addOption(outputDirOption);
+ opts.addOption(saveResults);
+ opts.addOption(eventFileNameOption);
+ opts.addOption(fromSimpleHistoryOption);
+ opts.addOption(help);
+ return opts;
+ }
+
+ protected String getOutputDir() {
+ return outputDir;
+ }
+
+ private void printUsage() {
+ System.err.println("Analyzer base options are");
+ Options options = buildOptions();
+ for (Object obj : options.getOptions()) {
+ Option option = (Option) obj;
+ System.err.println(option.getArgName() + " : " + option.getDescription());
+ }
+ }
+
+ @Override
+ public int run(String[] args) throws Exception {
+ //Parse downloaded contents
+ CommandLine cmdLine = null;
+ try {
+ cmdLine = new GnuParser().parse(buildOptions(), args);
+ } catch (ParseException e) {
+ System.err.println("Invalid options on command line");
+ printUsage();
+ return -1;
+ }
+ saveResults = cmdLine.hasOption(SAVE_RESULTS);
+
+ if(cmdLine.hasOption(HELP)) {
+ printUsage();
+ return 0;
+ }
+
+ outputDir = cmdLine.getOptionValue(OUTPUT_DIR);
+ if (outputDir == null) {
+ outputDir = System.getProperty("user.dir");
+ }
+
+ File file = null;
+ if (cmdLine.hasOption(EVENT_FILE_NAME)) {
+ file = new File(cmdLine.getOptionValue(EVENT_FILE_NAME));
+ }
+
+ String dagId = cmdLine.getOptionValue(DAG_ID);
+
+ DagInfo dagInfo = null;
+
+ if (file == null) {
+ if (cmdLine.hasOption(FROM_SIMPLE_HISTORY)) {
+ System.err.println("Event file name must be specified when using simple history");
+ printUsage();
+ return -2;
+ }
+ // using ATS - try to download directly
+ String[] importArgs = { "--dagId=" + dagId, "--downloadDir=" + outputDir };
+
+ int result = ATSImportTool.process(importArgs);
+ if (result != 0) {
+ System.err.println("Error downloading data from ATS");
+ return -3;
+ }
+
+ //Parse ATS data and verify results
+ //Parse downloaded contents
+ file = new File(outputDir
+ + Path.SEPARATOR + dagId
+ + Path.SEPARATOR + dagId + ".zip");
+ }
+
+ Preconditions.checkState(file != null);
+ if (!cmdLine.hasOption(FROM_SIMPLE_HISTORY)) {
+ ATSFileParser parser = new ATSFileParser(file);
+ dagInfo = parser.getDAGData(dagId);
+ } else {
+ SimpleHistoryParser parser = new SimpleHistoryParser(file);
+ dagInfo = parser.getDAGData(dagId);
+ }
+ Preconditions.checkState(dagInfo.getDagId().equals(dagId));
+ analyze(dagInfo);
+ Result result = getResult();
+ if (saveResults && (result instanceof CSVResult)) {
+ String fileName = outputDir + File.separator
+ + this.getClass().getName() + "_" + dagInfo.getDagId() + ".csv";
+ ((CSVResult) result).dumpToFile(fileName);
+ System.out.println("Saved results in " + fileName);
+ }
+ return 0;
+ }
+
+ public void printResults() throws TezException {
+ Result result = getResult();
+ if (result instanceof CSVResult) {
+ String[] headers = ((CSVResult) result).getHeaders();
+
+ StringBuilder formatBuilder = new StringBuilder();
+ int size = Math.max(MIN_COL_WIDTH, SEPARATOR_WIDTH / headers.length);
+ for (int i = 0; i < headers.length; i++) {
+ formatBuilder.append("%-").append(size).append("s ");
+ }
+ String format = formatBuilder.toString();
+
+ StringBuilder separator = new StringBuilder();
+ for (int i = 0; i < SEPARATOR_WIDTH; i++) {
+ separator.append("-");
+ }
+
+ System.out.println(separator);
+ System.out.println(String.format(format.toString(), (String[]) headers));
+ System.out.println(separator);
+
+ Iterator<String[]> recordsIterator = ((CSVResult) result).getRecordsIterator();
+ while (recordsIterator.hasNext()) {
+ String line = String.format(format, (String[]) recordsIterator.next());
+ System.out.println(line);
+ }
+ System.out.println(separator);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/8c8db7c5/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/VertexLevelCriticalPathAnalyzer.java
----------------------------------------------------------------------
diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/VertexLevelCriticalPathAnalyzer.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/VertexLevelCriticalPathAnalyzer.java
new file mode 100644
index 0000000..06b8983
--- /dev/null
+++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/VertexLevelCriticalPathAnalyzer.java
@@ -0,0 +1,152 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.analyzer.plugins;
+
+import com.google.common.base.Functions;
+import com.google.common.base.Splitter;
+import com.google.common.base.Strings;
+import com.google.common.collect.ImmutableSortedMap;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Ordering;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.tez.analyzer.Analyzer;
+import org.apache.tez.analyzer.CSVResult;
+import org.apache.tez.analyzer.utils.Utils;
+import org.apache.tez.dag.api.TezException;
+import org.apache.tez.history.parser.datamodel.DagInfo;
+import org.apache.tez.history.parser.datamodel.VertexInfo;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Identify a set of vertices which fall in the critical path in a DAG.
+ */
+public class VertexLevelCriticalPathAnalyzer extends TezAnalyzerBase implements Analyzer {
+ private final Configuration config;
+
+ private static final String[] headers = { "CriticalPath", "Score" };
+
+ private final CSVResult csvResult;
+
+ private static final String DOT_FILE_DIR = "tez.critical-path.analyzer.dot.output.loc";
+ private static final String DOT_FILE_DIR_DEFAULT = "."; //current directory
+
+ private final String dotFileLocation;
+
+ private static final String CONNECTOR = "-->";
+
+ public VertexLevelCriticalPathAnalyzer(Configuration config) {
+ this.config = config;
+ this.csvResult = new CSVResult(headers);
+ this.dotFileLocation = config.get(DOT_FILE_DIR, DOT_FILE_DIR_DEFAULT);
+ }
+
+ @Override public void analyze(DagInfo dagInfo) throws TezException {
+ Map<String, Long> result = Maps.newLinkedHashMap();
+ getCriticalPath("", dagInfo.getVertices().get(dagInfo.getVertices().size() - 1), 0, result);
+
+ Map<String, Long> sortedByValues = sortByValues(result);
+ for (Map.Entry<String, Long> entry : sortedByValues.entrySet()) {
+ List<String> record = Lists.newLinkedList();
+ record.add(entry.getKey());
+ record.add(entry.getValue() + "");
+ csvResult.addRecord(record.toArray(new String[record.size()]));
+ }
+
+ String dotFile = dotFileLocation + File.separator + dagInfo.getDagId() + ".dot";
+ try {
+ List<String> criticalVertices = null;
+ if (!sortedByValues.isEmpty()) {
+ String criticalPath = sortedByValues.keySet().iterator().next();
+ criticalVertices = getVertexNames(criticalPath);
+ } else {
+ criticalVertices = Lists.newLinkedList();
+ }
+ Utils.generateDAGVizFile(dagInfo, dotFile, criticalVertices);
+ } catch (IOException e) {
+ throw new TezException(e);
+ }
+ }
+
+ @Override
+ public CSVResult getResult() throws TezException {
+ return csvResult;
+ }
+
+ @Override
+ public String getName() {
+ return "CriticalPathAnalyzer";
+ }
+
+ @Override
+ public String getDescription() {
+ return "Analyze vertex level critical path of the DAG";
+ }
+
+ @Override
+ public Configuration getConfiguration() {
+ return config;
+ }
+
+ private static Map<String, Long> sortByValues(Map<String, Long> result) {
+ //Sort result by time in reverse order
+ final Ordering<String> reversValueOrdering =
+ Ordering.natural().reverse().nullsLast().onResultOf(Functions.forMap(result, null));
+ Map<String, Long> orderedMap = ImmutableSortedMap.copyOf(result, reversValueOrdering);
+ return orderedMap;
+ }
+
+ private static void getCriticalPath(String predecessor, VertexInfo dest, long time,
+ Map<String, Long> result) {
+ String destVertexName = (dest != null) ? (dest.getVertexName()) : "";
+
+ if (dest != null) {
+ time += dest.getTimeTaken();
+ predecessor += destVertexName + CONNECTOR;
+
+ for (VertexInfo incomingVertex : dest.getInputVertices()) {
+ getCriticalPath(predecessor, incomingVertex, time, result);
+ }
+
+ result.put(predecessor, time);
+ }
+ }
+
+ private static List<String> getVertexNames(String criticalPath) {
+ if (Strings.isNullOrEmpty(criticalPath)) {
+ return Lists.newLinkedList();
+ }
+ return Lists.newLinkedList(Splitter.on(CONNECTOR).trimResults().omitEmptyStrings().split
+ (criticalPath));
+ }
+
+ public static void main(String[] args) throws Exception {
+ Configuration config = new Configuration();
+ VertexLevelCriticalPathAnalyzer analyzer = new VertexLevelCriticalPathAnalyzer(config);
+ int res = ToolRunner.run(config, analyzer, args);
+ analyzer.printResults();
+ System.exit(res);
+ }
+}
[3/5] tez git commit: TEZ-2973. Backport Analyzers to branch-0.7
Posted by je...@apache.org.
http://git-wip-us.apache.org/repos/asf/tez/blob/8c8db7c5/tez-plugins/tez-history-parser/src/test/java/org/apache/tez/history/TestHistoryParser.java
----------------------------------------------------------------------
diff --git a/tez-plugins/tez-history-parser/src/test/java/org/apache/tez/history/TestHistoryParser.java b/tez-plugins/tez-history-parser/src/test/java/org/apache/tez/history/TestHistoryParser.java
new file mode 100644
index 0000000..2b23294
--- /dev/null
+++ b/tez-plugins/tez-history-parser/src/test/java/org/apache/tez/history/TestHistoryParser.java
@@ -0,0 +1,813 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.history;
+
+import com.google.common.collect.Sets;
+import com.sun.tools.internal.ws.processor.ProcessorException;
+import org.apache.commons.cli.ParseException;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang.RandomStringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.server.namenode.EditLogFileOutputStream;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
+import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.tez.client.CallerContext;
+import org.apache.tez.client.TezClient;
+import org.apache.tez.common.counters.DAGCounter;
+import org.apache.tez.common.counters.TaskCounter;
+import org.apache.tez.common.counters.TezCounter;
+import org.apache.tez.dag.api.DAG;
+import org.apache.tez.dag.api.DataSinkDescriptor;
+import org.apache.tez.dag.api.DataSourceDescriptor;
+import org.apache.tez.dag.api.Edge;
+import org.apache.tez.dag.api.EdgeProperty;
+import org.apache.tez.dag.api.ProcessorDescriptor;
+import org.apache.tez.dag.api.TezConfiguration;
+import org.apache.tez.dag.api.TezException;
+import org.apache.tez.dag.api.Vertex;
+import org.apache.tez.dag.api.client.DAGClient;
+import org.apache.tez.dag.api.client.StatusGetOpts;
+import org.apache.tez.dag.api.event.VertexState;
+import org.apache.tez.dag.api.oldrecords.TaskAttemptState;
+import org.apache.tez.dag.api.oldrecords.TaskState;
+import org.apache.tez.dag.app.dag.DAGState;
+import org.apache.tez.dag.history.logging.ats.ATSHistoryLoggingService;
+import org.apache.tez.dag.history.logging.impl.SimpleHistoryLoggingService;
+import org.apache.tez.dag.records.TezDAGID;
+import org.apache.tez.examples.WordCount;
+import org.apache.tez.history.parser.ATSFileParser;
+import org.apache.tez.history.parser.SimpleHistoryParser;
+import org.apache.tez.history.parser.datamodel.BaseInfo;
+import org.apache.tez.history.parser.datamodel.DagInfo;
+import org.apache.tez.history.parser.datamodel.EdgeInfo;
+import org.apache.tez.history.parser.datamodel.TaskAttemptInfo;
+import org.apache.tez.history.parser.datamodel.TaskAttemptInfo.DataDependencyEvent;
+import org.apache.tez.history.parser.datamodel.TaskInfo;
+import org.apache.tez.history.parser.datamodel.VersionInfo;
+import org.apache.tez.history.parser.datamodel.VertexInfo;
+import org.apache.tez.mapreduce.input.MRInput;
+import org.apache.tez.mapreduce.output.MROutput;
+import org.apache.tez.mapreduce.processor.SimpleMRProcessor;
+import org.apache.tez.runtime.api.ProcessorContext;
+import org.apache.tez.runtime.library.api.TezRuntimeConfiguration;
+import org.apache.tez.runtime.library.conf.OrderedPartitionedKVEdgeConfig;
+import org.apache.tez.runtime.library.input.OrderedGroupedKVInput;
+import org.apache.tez.runtime.library.output.OrderedPartitionedKVOutput;
+import org.apache.tez.runtime.library.partitioner.HashPartitioner;
+import org.apache.tez.tests.MiniTezClusterWithTimeline;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.Assert.*;
+
+public class TestHistoryParser {
+
+ private static MiniDFSCluster miniDFSCluster;
+ private static MiniTezClusterWithTimeline miniTezCluster;
+
+ //location within miniHDFS cluster's hdfs
+ private static Path inputLoc = new Path("/tmp/sample.txt");
+
+ private final static String INPUT = "Input";
+ private final static String OUTPUT = "Output";
+ private final static String TOKENIZER = "Tokenizer";
+ private final static String SUMMATION = "Summation";
+ private final static String SIMPLE_HISTORY_DIR = "/tmp/simplehistory/";
+ private final static String HISTORY_TXT = "history.txt";
+
+ private static Configuration conf = new Configuration();
+ private static FileSystem fs;
+ private static String TEST_ROOT_DIR =
+ "target" + Path.SEPARATOR + TestHistoryParser.class.getName() + "-tmpDir";
+ private static String TEZ_BASE_DIR =
+ "target" + Path.SEPARATOR + TestHistoryParser.class.getName() + "-tez";
+ private static String DOWNLOAD_DIR = TEST_ROOT_DIR + Path.SEPARATOR + "download";
+
+ @BeforeClass
+ public static void setupCluster() throws Exception {
+ conf = new Configuration();
+ conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_EDITS_NOEDITLOGCHANNELFLUSH, false);
+ EditLogFileOutputStream.setShouldSkipFsyncForTesting(true);
+ conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, TEST_ROOT_DIR);
+ miniDFSCluster =
+ new MiniDFSCluster.Builder(conf).numDataNodes(1).format(true).build();
+ fs = miniDFSCluster.getFileSystem();
+ conf.set("fs.defaultFS", fs.getUri().toString());
+
+ setupTezCluster();
+ }
+
+ @AfterClass
+ public static void shutdownCluster() {
+ try {
+ if (miniDFSCluster != null) {
+ miniDFSCluster.shutdown();
+ }
+ if (miniTezCluster != null) {
+ miniTezCluster.stop();
+ }
+ } finally {
+ try {
+ FileUtils.deleteDirectory(new File(TEST_ROOT_DIR));
+ FileUtils.deleteDirectory(new File(TEZ_BASE_DIR));
+ } catch (IOException e) {
+ //safe to ignore
+ }
+ }
+ }
+
+ // @Before
+ public static void setupTezCluster() throws Exception {
+ conf.setInt(TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_CONNECT_TIMEOUT, 3 * 1000);
+ conf.setInt(TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_READ_TIMEOUT, 3 * 1000);
+ conf.setInt(TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_FETCH_FAILURES_LIMIT, 2);
+
+ //Enable per edge counters
+ conf.setBoolean(TezConfiguration.TEZ_TASK_GENERATE_COUNTERS_PER_IO, true);
+ conf.setBoolean(TezConfiguration.TEZ_AM_ALLOW_DISABLED_TIMELINE_DOMAINS, true);
+ conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
+ conf.set(TezConfiguration.TEZ_HISTORY_LOGGING_SERVICE_CLASS, ATSHistoryLoggingService
+ .class.getName());
+
+ conf.set(TezConfiguration.TEZ_SIMPLE_HISTORY_LOGGING_DIR, SIMPLE_HISTORY_DIR);
+
+ miniTezCluster =
+ new MiniTezClusterWithTimeline(TEZ_BASE_DIR, 1, 1, 1, true);
+
+ miniTezCluster.init(conf);
+ miniTezCluster.start();
+
+ createSampleFile(inputLoc);
+
+ TezConfiguration tezConf = new TezConfiguration(miniTezCluster.getConfig());
+ tezConf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
+ tezConf.set(YarnConfiguration.TIMELINE_SERVICE_WEBAPP_ADDRESS, "0.0.0.0:8188");
+ tezConf.setBoolean(TezConfiguration.TEZ_AM_ALLOW_DISABLED_TIMELINE_DOMAINS, true);
+ tezConf.set(TezConfiguration.TEZ_HISTORY_LOGGING_SERVICE_CLASS,
+ ATSHistoryLoggingService.class.getName());
+
+ }
+
+
+ /**
+ * Run a word count example in mini cluster and check if it is possible to download
+ * data from ATS and parse it. Also, run with SimpleHistoryLogging option and verify
+ * if it matches with ATS data.
+ *
+ * @throws Exception
+ */
+ @Test
+ public void testParserWithSuccessfulJob() throws Exception {
+ //Run basic word count example.
+ String dagId = runWordCount(WordCount.TokenProcessor.class.getName(),
+ WordCount.SumProcessor.class.getName(), "WordCount", true);
+
+ //Export the data from ATS
+ String[] args = { "--dagId=" + dagId, "--downloadDir=" + DOWNLOAD_DIR };
+
+ int result = ATSImportTool.process(args);
+ assertTrue(result == 0);
+
+ //Parse ATS data and verify results
+ DagInfo dagInfoFromATS = getDagInfo(dagId);
+ verifyDagInfo(dagInfoFromATS, true);
+ verifyJobSpecificInfo(dagInfoFromATS);
+
+ //Now run with SimpleHistoryLogging
+ dagId = runWordCount(WordCount.TokenProcessor.class.getName(),
+ WordCount.SumProcessor.class.getName(), "WordCount", false);
+ Thread.sleep(10000); //For all flushes to happen and to avoid half-cooked download.
+
+ DagInfo shDagInfo = getDagInfoFromSimpleHistory(dagId);
+ verifyDagInfo(shDagInfo, false);
+ verifyJobSpecificInfo(shDagInfo);
+
+ //Compare dagInfo by parsing ATS data with DagInfo obtained by parsing SimpleHistoryLog
+ isDAGEqual(dagInfoFromATS, shDagInfo);
+ }
+
+ private DagInfo getDagInfoFromSimpleHistory(String dagId) throws TezException, IOException {
+ TezDAGID tezDAGID = TezDAGID.fromString(dagId);
+ ApplicationAttemptId applicationAttemptId = ApplicationAttemptId.newInstance(tezDAGID
+ .getApplicationId(), 1);
+ Path historyPath = new Path(conf.get("fs.defaultFS")
+ + SIMPLE_HISTORY_DIR + HISTORY_TXT + "."
+ + applicationAttemptId);
+ FileSystem fs = historyPath.getFileSystem(conf);
+
+ Path localPath = new Path(DOWNLOAD_DIR, HISTORY_TXT);
+ fs.copyToLocalFile(historyPath, localPath);
+ File localFile = new File(DOWNLOAD_DIR, HISTORY_TXT);
+
+ //Now parse via SimpleHistory
+ SimpleHistoryParser parser = new SimpleHistoryParser(localFile);
+ DagInfo dagInfo = parser.getDAGData(dagId);
+ assertTrue(dagInfo.getDagId().equals(dagId));
+ return dagInfo;
+ }
+
+ private void verifyJobSpecificInfo(DagInfo dagInfo) {
+ //Job specific
+ assertTrue(dagInfo.getNumVertices() == 2);
+ assertTrue(dagInfo.getName().equals("WordCount"));
+ assertTrue(dagInfo.getVertex(TOKENIZER).getProcessorClassName().equals(
+ WordCount.TokenProcessor.class.getName()));
+ assertTrue(dagInfo.getVertex(SUMMATION).getProcessorClassName()
+ .equals(WordCount.SumProcessor.class.getName()));
+ assertTrue(dagInfo.getEdges().size() == 1);
+ EdgeInfo edgeInfo = dagInfo.getEdges().iterator().next();
+ assertTrue(edgeInfo.getDataMovementType().
+ equals(EdgeProperty.DataMovementType.SCATTER_GATHER.toString()));
+ assertTrue(edgeInfo.getSourceVertex().getVertexName().equals(TOKENIZER));
+ assertTrue(edgeInfo.getDestinationVertex().getVertexName().equals(SUMMATION));
+ assertTrue(edgeInfo.getInputVertexName().equals(TOKENIZER));
+ assertTrue(edgeInfo.getOutputVertexName().equals(SUMMATION));
+ assertTrue(edgeInfo.getEdgeSourceClass().equals(OrderedPartitionedKVOutput.class.getName()));
+ assertTrue(edgeInfo.getEdgeDestinationClass().equals(OrderedGroupedKVInput.class.getName()));
+ assertTrue(dagInfo.getVertices().size() == 2);
+ String lastSourceTA = null;
+ String lastDataEventSourceTA = null;
+ for (VertexInfo vertexInfo : dagInfo.getVertices()) {
+ assertTrue(vertexInfo.getKilledTasksCount() == 0);
+ assertTrue(vertexInfo.getInitRequestedTime() > 0);
+ assertTrue(vertexInfo.getInitTime() > 0);
+ assertTrue(vertexInfo.getStartRequestedTime() > 0);
+ assertTrue(vertexInfo.getStartTime() > 0);
+ assertTrue(vertexInfo.getFinishTime() > 0);
+ long finishTime = 0;
+ for (TaskInfo taskInfo : vertexInfo.getTasks()) {
+ assertTrue(taskInfo.getNumberOfTaskAttempts() == 1);
+ assertTrue(taskInfo.getMaxTaskAttemptDuration() >= 0);
+ assertTrue(taskInfo.getMinTaskAttemptDuration() >= 0);
+ assertTrue(taskInfo.getAvgTaskAttemptDuration() >= 0);
+ assertTrue(taskInfo.getLastTaskAttemptToFinish() != null);
+ assertTrue(taskInfo.getContainersMapping().size() > 0);
+ assertTrue(taskInfo.getSuccessfulTaskAttempts().size() > 0);
+ assertTrue(taskInfo.getFailedTaskAttempts().size() == 0);
+ assertTrue(taskInfo.getKilledTaskAttempts().size() == 0);
+ List<TaskAttemptInfo> attempts = taskInfo.getTaskAttempts();
+ if (vertexInfo.getVertexName().equals(TOKENIZER)) {
+ // get the last task to finish and track its successful attempt
+ if (finishTime < taskInfo.getFinishTime()) {
+ finishTime = taskInfo.getFinishTime();
+ lastSourceTA = taskInfo.getSuccessfulAttemptId();
+ }
+ } else {
+ for (TaskAttemptInfo attempt : attempts) {
+ DataDependencyEvent item = attempt.getLastDataEvents().get(0);
+ assertTrue(item.getTimestamp() > 0);
+
+ if (lastDataEventSourceTA == null) {
+ lastDataEventSourceTA = item.getTaskAttemptId();
+ } else {
+ // all attempts should have the same last data event source TA
+ assertTrue(lastDataEventSourceTA.equals(item.getTaskAttemptId()));
+ }
+ }
+ }
+ for (TaskAttemptInfo attemptInfo : taskInfo.getTaskAttempts()) {
+ assertTrue(attemptInfo.getCreationTime() > 0);
+ assertTrue(attemptInfo.getAllocationTime() > 0);
+ assertTrue(attemptInfo.getStartTime() > 0);
+ }
+ }
+ assertTrue(vertexInfo.getLastTaskToFinish() != null);
+ if (vertexInfo.getVertexName().equals(TOKENIZER)) {
+ assertTrue(vertexInfo.getInputEdges().size() == 0);
+ assertTrue(vertexInfo.getOutputEdges().size() == 1);
+ assertTrue(vertexInfo.getOutputVertices().size() == 1);
+ assertTrue(vertexInfo.getInputVertices().size() == 0);
+ } else {
+ assertTrue(vertexInfo.getInputEdges().size() == 1);
+ assertTrue(vertexInfo.getOutputEdges().size() == 0);
+ assertTrue(vertexInfo.getOutputVertices().size() == 0);
+ assertTrue(vertexInfo.getInputVertices().size() == 1);
+ }
+ }
+ assertTrue(lastSourceTA.equals(lastDataEventSourceTA));
+ }
+
+ /**
+ * Run a word count example in mini cluster.
+ * Provide invalid URL for ATS.
+ *
+ * @throws Exception
+ */
+ @Test
+ public void testParserWithSuccessfulJob_InvalidATS() throws Exception {
+ //Run basic word count example.
+ String dagId = runWordCount(WordCount.TokenProcessor.class.getName(),
+ WordCount.SumProcessor.class.getName(), "WordCount-With-WrongATS-URL", true);
+
+ //Export the data from ATS
+ String atsAddress = "--atsAddress=http://atsHost:8188";
+ String[] args = { "--dagId=" + dagId,
+ "--downloadDir=" + DOWNLOAD_DIR,
+ atsAddress
+ };
+
+ try {
+ int result = ATSImportTool.process(args);
+ fail("Should have failed with processException");
+ } catch(ParseException e) {
+ //expects exception
+ }
+ }
+
+ /**
+ * Run a failed job and parse the data from ATS
+ */
+ @Test
+ public void testParserWithFailedJob() throws Exception {
+ //Run a job which would fail
+ String dagId = runWordCount(WordCount.TokenProcessor.class.getName(), FailProcessor.class
+ .getName(), "WordCount-With-Exception", true);
+
+ //Export the data from ATS
+ String[] args = { "--dagId=" + dagId, "--downloadDir=" + DOWNLOAD_DIR };
+
+ int result = ATSImportTool.process(args);
+ assertTrue(result == 0);
+
+ //Parse ATS data
+ DagInfo dagInfo = getDagInfo(dagId);
+
+ //Verify DAGInfo. Verifies vertex, task, taskAttempts in recursive manner
+ verifyDagInfo(dagInfo, true);
+
+ //Dag specific
+ VertexInfo summationVertex = dagInfo.getVertex(SUMMATION);
+ assertTrue(summationVertex.getFailedTasks().size() == 1); //1 task, 4 attempts failed
+ assertTrue(summationVertex.getFailedTasks().get(0).getFailedTaskAttempts().size() == 4);
+ assertTrue(summationVertex.getStatus().equals(VertexState.FAILED.toString()));
+
+ assertTrue(dagInfo.getFailedVertices().size() == 1);
+ assertTrue(dagInfo.getFailedVertices().get(0).getVertexName().equals(SUMMATION));
+ assertTrue(dagInfo.getSuccessfullVertices().size() == 1);
+ assertTrue(dagInfo.getSuccessfullVertices().get(0).getVertexName().equals(TOKENIZER));
+
+ assertTrue(dagInfo.getStatus().equals(DAGState.FAILED.toString()));
+
+ verifyCounter(dagInfo.getCounter(DAGCounter.NUM_FAILED_TASKS.toString()), null, 4);
+ verifyCounter(dagInfo.getCounter(DAGCounter.NUM_SUCCEEDED_TASKS.toString()), null, 1);
+ verifyCounter(dagInfo.getCounter(DAGCounter.TOTAL_LAUNCHED_TASKS.toString()), null, 5);
+
+ verifyCounter(dagInfo.getCounter(TaskCounter.INPUT_RECORDS_PROCESSED.toString()),
+ "TaskCounter_Tokenizer_INPUT_Input", 10);
+ verifyCounter(dagInfo.getCounter(TaskCounter.ADDITIONAL_SPILLS_BYTES_READ.toString()),
+ "TaskCounter_Tokenizer_OUTPUT_Summation", 0);
+ verifyCounter(dagInfo.getCounter(TaskCounter.OUTPUT_RECORDS.toString()),
+ "TaskCounter_Tokenizer_OUTPUT_Summation",
+ 20); //Every line has 2 words. 10 lines x 2 words = 20
+ verifyCounter(dagInfo.getCounter(TaskCounter.SPILLED_RECORDS.toString()),
+ "TaskCounter_Tokenizer_OUTPUT_Summation", 20); //Same as above
+
+ for (TaskInfo taskInfo : summationVertex.getTasks()) {
+ TaskAttemptInfo lastAttempt = null;
+ for (TaskAttemptInfo attemptInfo : taskInfo.getTaskAttempts()) {
+ if (lastAttempt != null) {
+ // failed attempt should be causal TA of next attempt
+ assertTrue(lastAttempt.getTaskAttemptId().equals(attemptInfo.getCreationCausalTA()));
+ assertTrue(lastAttempt.getTerminationCause() != null);
+ }
+ lastAttempt = attemptInfo;
+ }
+ }
+
+ //TODO: Need to check for SUMMATION vertex counters. Since all attempts are failed, counters are not getting populated.
+ //TaskCounter.REDUCE_INPUT_RECORDS
+
+ //Verify if the processor exception is given in diagnostics
+ assertTrue(dagInfo.getDiagnostics().contains("Failing this processor for some reason"));
+
+ }
+
+ /**
+ * Adding explicit equals here instead of in DAG/Vertex/Edge where hashCode also needs to
+ * change. Also, some custom comparisons are done here for unit testing.
+ */
+ private void isDAGEqual(DagInfo dagInfo1, DagInfo dagInfo2) {
+ assertNotNull(dagInfo1);
+ assertNotNull(dagInfo2);
+ assertEquals(dagInfo1.getStatus(), dagInfo2.getStatus());
+ isEdgeEqual(dagInfo1.getEdges(), dagInfo2.getEdges());
+ isVertexEqual(dagInfo1.getVertices(), dagInfo2.getVertices());
+ }
+
+ private void isVertexEqual(VertexInfo vertexInfo1, VertexInfo vertexInfo2) {
+ assertTrue(vertexInfo1 != null);
+ assertTrue(vertexInfo2 != null);
+ assertTrue(vertexInfo1.getVertexName().equals(vertexInfo2.getVertexName()));
+ assertTrue(vertexInfo1.getProcessorClassName().equals(vertexInfo2.getProcessorClassName()));
+ assertTrue(vertexInfo1.getNumTasks() == vertexInfo2.getNumTasks());
+ assertTrue(vertexInfo1.getCompletedTasksCount() == vertexInfo2.getCompletedTasksCount());
+ assertTrue(vertexInfo1.getStatus().equals(vertexInfo2.getStatus()));
+
+ isEdgeEqual(vertexInfo1.getInputEdges(), vertexInfo2.getInputEdges());
+ isEdgeEqual(vertexInfo1.getOutputEdges(), vertexInfo2.getOutputEdges());
+
+ assertTrue(vertexInfo1.getInputVertices().size() == vertexInfo2.getInputVertices().size());
+ assertTrue(vertexInfo1.getOutputVertices().size() == vertexInfo2.getOutputVertices().size());
+
+ assertTrue(vertexInfo1.getNumTasks() == vertexInfo2.getNumTasks());
+ isTaskEqual(vertexInfo1.getTasks(), vertexInfo2.getTasks());
+ }
+
+ private void isVertexEqual(List<VertexInfo> vertexList1, List<VertexInfo> vertexList2) {
+ assertTrue("Vertices sizes should be the same", vertexList1.size() == vertexList2.size());
+ Iterator<VertexInfo> it1 = vertexList1.iterator();
+ Iterator<VertexInfo> it2 = vertexList2.iterator();
+ while (it1.hasNext()) {
+ assertTrue(it2.hasNext());
+ VertexInfo info1 = it1.next();
+ VertexInfo info2 = it2.next();
+ isVertexEqual(info1, info2);
+ }
+ }
+
+ private void isEdgeEqual(EdgeInfo edgeInfo1, EdgeInfo edgeInfo2) {
+ assertTrue(edgeInfo1 != null);
+ assertTrue(edgeInfo2 != null);
+ String info1 = edgeInfo1.toString();
+ String info2 = edgeInfo1.toString();
+ assertTrue(info1.equals(info2));
+ }
+
+ private void isEdgeEqual(Collection<EdgeInfo> info1, Collection<EdgeInfo> info2) {
+ assertTrue("sizes should be the same", info1.size() == info1.size());
+ Iterator<EdgeInfo> it1 = info1.iterator();
+ Iterator<EdgeInfo> it2 = info2.iterator();
+ while (it1.hasNext()) {
+ assertTrue(it2.hasNext());
+ isEdgeEqual(it1.next(), it2.next());
+ }
+ }
+
+ private void isTaskEqual(Collection<TaskInfo> info1, Collection<TaskInfo> info2) {
+ assertTrue("sizes should be the same", info1.size() == info1.size());
+ Iterator<TaskInfo> it1 = info1.iterator();
+ Iterator<TaskInfo> it2 = info2.iterator();
+ while (it1.hasNext()) {
+ assertTrue(it2.hasNext());
+ isTaskEqual(it1.next(), it2.next());
+ }
+ }
+
+ private void isTaskEqual(TaskInfo taskInfo1, TaskInfo taskInfo2) {
+ assertTrue(taskInfo1 != null);
+ assertTrue(taskInfo2 != null);
+ assertTrue(taskInfo1.getVertexInfo() != null);
+ assertTrue(taskInfo2.getVertexInfo() != null);
+ assertTrue(taskInfo1.getStatus().equals(taskInfo2.getStatus()));
+ assertTrue(
+ taskInfo1.getVertexInfo().getVertexName()
+ .equals(taskInfo2.getVertexInfo().getVertexName()));
+ isTaskAttemptEqual(taskInfo1.getTaskAttempts(), taskInfo2.getTaskAttempts());
+
+ //Verify counters
+ isCountersSame(taskInfo1, taskInfo2);
+ }
+
+ private void isCountersSame(BaseInfo info1, BaseInfo info2) {
+ isCounterSame(info1.getCounter(TaskCounter.ADDITIONAL_SPILL_COUNT.name()),
+ info2.getCounter(TaskCounter.ADDITIONAL_SPILL_COUNT.name()));
+
+ isCounterSame(info1.getCounter(TaskCounter.SPILLED_RECORDS.name()),
+ info2.getCounter(TaskCounter.SPILLED_RECORDS.name()));
+
+ isCounterSame(info1.getCounter(TaskCounter.OUTPUT_RECORDS.name()),
+ info2.getCounter(TaskCounter.OUTPUT_RECORDS.name()));
+
+ isCounterSame(info1.getCounter(TaskCounter.OUTPUT_BYTES.name()),
+ info2.getCounter(TaskCounter.OUTPUT_BYTES.name()));
+
+ isCounterSame(info1.getCounter(TaskCounter.OUTPUT_RECORDS.name()),
+ info2.getCounter(TaskCounter.OUTPUT_RECORDS.name()));
+
+ isCounterSame(info1.getCounter(TaskCounter.REDUCE_INPUT_GROUPS.name()),
+ info2.getCounter(TaskCounter.REDUCE_INPUT_GROUPS.name()));
+
+ isCounterSame(info1.getCounter(TaskCounter.REDUCE_INPUT_RECORDS.name()),
+ info2.getCounter(TaskCounter.REDUCE_INPUT_RECORDS.name()));
+ }
+
+ private void isCounterSame(Map<String, TezCounter> counter1, Map<String, TezCounter> counter2) {
+ for (Map.Entry<String, TezCounter> entry : counter1.entrySet()) {
+ String source = entry.getKey();
+ long val = entry.getValue().getValue();
+
+ //check if other counter has the same value
+ assertTrue(counter2.containsKey(entry.getKey()));
+ assertTrue(counter2.get(entry.getKey()).getValue() == val);
+ }
+ }
+
+ private void isTaskAttemptEqual(Collection<TaskAttemptInfo> info1,
+ Collection<TaskAttemptInfo> info2) {
+ assertTrue("sizes should be the same", info1.size() == info1.size());
+ Iterator<TaskAttemptInfo> it1 = info1.iterator();
+ Iterator<TaskAttemptInfo> it2 = info2.iterator();
+ while (it1.hasNext()) {
+ assertTrue(it2.hasNext());
+ isTaskAttemptEqual(it1.next(), it2.next());
+ }
+ }
+
+ private void isTaskAttemptEqual(TaskAttemptInfo info1, TaskAttemptInfo info2) {
+ assertTrue(info1 != null);
+ assertTrue(info2 != null);
+ assertTrue(info1.getTaskInfo() != null);
+ assertTrue(info2.getTaskInfo() != null);
+ assertTrue(info1.getStatus().equals(info2.getStatus()));
+ assertTrue(info1.getTaskInfo().getVertexInfo().getVertexName().equals(info2.getTaskInfo()
+ .getVertexInfo().getVertexName()));
+
+ //Verify counters
+ isCountersSame(info1, info2);
+ }
+
+
+ /**
+ * Create sample file for wordcount program
+ *
+ * @param inputLoc
+ * @throws IOException
+ */
+ private static void createSampleFile(Path inputLoc) throws IOException {
+ fs.deleteOnExit(inputLoc);
+ FSDataOutputStream out = fs.create(inputLoc);
+ BufferedWriter writer = new BufferedWriter(new OutputStreamWriter(out));
+ for (int i = 0; i < 10; i++) {
+ writer.write("Sample " + RandomStringUtils.randomAlphanumeric(5));
+ writer.newLine();
+ }
+ writer.close();
+ }
+
+ private DagInfo getDagInfo(String dagId) throws TezException {
+ //Parse downloaded contents
+ File downloadedFile = new File(DOWNLOAD_DIR
+ + Path.SEPARATOR + dagId + ".zip");
+ ATSFileParser parser = new ATSFileParser(downloadedFile);
+ DagInfo dagInfo = parser.getDAGData(dagId);
+ assertTrue(dagInfo.getDagId().equals(dagId));
+ return dagInfo;
+ }
+
+ private void verifyCounter(Map<String, TezCounter> counterMap,
+ String counterGroupName, long expectedVal) {
+ //Iterate through group-->tezCounter
+ for (Map.Entry<String, TezCounter> entry : counterMap.entrySet()) {
+ if (counterGroupName != null) {
+ if (entry.getKey().equals(counterGroupName)) {
+ assertTrue(entry.getValue().getValue() == expectedVal);
+ }
+ } else {
+ assertTrue(entry.getValue().getValue() == expectedVal);
+ }
+ }
+ }
+
+ TezClient getTezClient(boolean withTimeline) throws Exception {
+ TezConfiguration tezConf = new TezConfiguration(miniTezCluster.getConfig());
+ if (withTimeline) {
+ tezConf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, withTimeline);
+ tezConf.set(YarnConfiguration.TIMELINE_SERVICE_WEBAPP_ADDRESS, "0.0.0.0:8188");
+ tezConf.set(TezConfiguration.TEZ_HISTORY_LOGGING_SERVICE_CLASS,
+ ATSHistoryLoggingService.class.getName());
+ } else {
+ tezConf.set(TezConfiguration.TEZ_HISTORY_LOGGING_SERVICE_CLASS,
+ SimpleHistoryLoggingService.class.getName());
+ }
+ tezConf.setBoolean(TezConfiguration.TEZ_AM_ALLOW_DISABLED_TIMELINE_DOMAINS, true);
+
+ TezClient tezClient = TezClient.create("WordCount", tezConf, false);
+ tezClient.start();
+ tezClient.waitTillReady();
+ return tezClient;
+ }
+
+ private String runWordCount(String tokenizerProcessor, String summationProcessor,
+ String dagName, boolean withTimeline)
+ throws Exception {
+ //HDFS path
+ Path outputLoc = new Path("/tmp/outPath_" + System.currentTimeMillis());
+
+ DataSourceDescriptor dataSource = MRInput.createConfigBuilder(conf,
+ TextInputFormat.class, inputLoc.toString()).build();
+
+ DataSinkDescriptor dataSink =
+ MROutput.createConfigBuilder(conf, TextOutputFormat.class, outputLoc.toString()).build();
+
+ Vertex tokenizerVertex = Vertex.create(TOKENIZER, ProcessorDescriptor.create(
+ tokenizerProcessor)).addDataSource(INPUT, dataSource);
+
+ OrderedPartitionedKVEdgeConfig edgeConf = OrderedPartitionedKVEdgeConfig
+ .newBuilder(Text.class.getName(), IntWritable.class.getName(),
+ HashPartitioner.class.getName()).build();
+
+ Vertex summationVertex = Vertex.create(SUMMATION,
+ ProcessorDescriptor.create(summationProcessor), 1).addDataSink(OUTPUT, dataSink);
+
+ // Create DAG and add the vertices. Connect the producer and consumer vertices via the edge
+ DAG dag = DAG.create(dagName);
+ dag.addVertex(tokenizerVertex).addVertex(summationVertex).addEdge(
+ Edge.create(tokenizerVertex, summationVertex, edgeConf.createDefaultEdgeProperty()));
+
+ TezClient tezClient = getTezClient(withTimeline);
+
+ // Update Caller Context
+ CallerContext callerContext = CallerContext.create("TezExamples", "Tez WordCount Example Job");
+ ApplicationId appId = tezClient.getAppMasterApplicationId();
+ if (appId == null) {
+ appId = ApplicationId.newInstance(1001l, 1);
+ }
+ callerContext.setCallerIdAndType(appId.toString(), "TezApplication");
+ dag.setCallerContext(callerContext);
+
+ DAGClient client = tezClient.submitDAG(dag);
+ client.waitForCompletionWithStatusUpdates(Sets.newHashSet(StatusGetOpts.GET_COUNTERS));
+ TezDAGID tezDAGID = TezDAGID.getInstance(tezClient.getAppMasterApplicationId(), 1);
+
+ if (tezClient != null) {
+ tezClient.stop();
+ }
+ return tezDAGID.toString();
+ }
+
+ /**
+ * Processor which would just throw exception.
+ */
+ public static class FailProcessor extends SimpleMRProcessor {
+ public FailProcessor(ProcessorContext context) {
+ super(context);
+ }
+
+ @Override
+ public void run() throws Exception {
+ throw new Exception("Failing this processor for some reason");
+ }
+ }
+
+ private void verifyDagInfo(DagInfo dagInfo, boolean ats) {
+ if (ats) {
+ VersionInfo versionInfo = dagInfo.getVersionInfo();
+ assertTrue(versionInfo != null); //should be present post 0.5.4
+ assertTrue(versionInfo.getVersion() != null);
+ assertTrue(versionInfo.getRevision() != null);
+ assertTrue(versionInfo.getBuildTime() != null);
+ }
+
+ assertTrue(dagInfo.getStartTime() > 0);
+ assertTrue(dagInfo.getFinishTimeInterval() > 0);
+ assertTrue(dagInfo.getStartTimeInterval() == 0);
+ assertTrue(dagInfo.getStartTime() > 0);
+ if (dagInfo.getStatus().equalsIgnoreCase(DAGState.SUCCEEDED.toString())) {
+ assertTrue(dagInfo.getFinishTime() >= dagInfo.getStartTime());
+ }
+ assertTrue(dagInfo.getFinishTimeInterval() > dagInfo.getStartTimeInterval());
+
+ assertTrue(dagInfo.getStartTime() > dagInfo.getSubmitTime());
+ assertTrue(dagInfo.getTimeTaken() > 0);
+
+ assertNotNull(dagInfo.getCallerContext());
+ assertEquals("TezExamples", dagInfo.getCallerContext().getContext());
+ assertEquals("Tez WordCount Example Job", dagInfo.getCallerContext().getBlob());
+ assertNotNull(dagInfo.getCallerContext().getCallerId());
+ assertEquals("TezApplication", dagInfo.getCallerContext().getCallerType());
+
+ //Verify all vertices
+ for (VertexInfo vertexInfo : dagInfo.getVertices()) {
+ verifyVertex(vertexInfo, vertexInfo.getFailedTasksCount() > 0);
+ }
+
+ VertexInfo fastestVertex = dagInfo.getFastestVertex();
+ assertTrue(fastestVertex != null);
+
+ if (dagInfo.getStatus().equals(DAGState.SUCCEEDED)) {
+ assertTrue(dagInfo.getSlowestVertex() != null);
+ }
+ }
+
+ private void verifyVertex(VertexInfo vertexInfo, boolean hasFailedTasks) {
+ assertTrue(vertexInfo != null);
+ if (hasFailedTasks) {
+ assertTrue(vertexInfo.getFailedTasksCount() > 0);
+ }
+ assertTrue(vertexInfo.getStartTimeInterval() > 0);
+ assertTrue(vertexInfo.getStartTime() > 0);
+ assertTrue(vertexInfo.getFinishTimeInterval() > 0);
+ assertTrue(vertexInfo.getStartTimeInterval() < vertexInfo.getFinishTimeInterval());
+ assertTrue(vertexInfo.getVertexName() != null);
+ if (!hasFailedTasks) {
+ assertTrue(vertexInfo.getFinishTime() > 0);
+ assertTrue(vertexInfo.getFailedTasks().size() == 0);
+ assertTrue(vertexInfo.getSucceededTasksCount() == vertexInfo.getSuccessfulTasks().size());
+ assertTrue(vertexInfo.getFailedTasksCount() == 0);
+ assertTrue(vertexInfo.getAvgTaskDuration() > 0);
+ assertTrue(vertexInfo.getMaxTaskDuration() > 0);
+ assertTrue(vertexInfo.getMinTaskDuration() > 0);
+ assertTrue(vertexInfo.getTimeTaken() > 0);
+ assertTrue(vertexInfo.getStatus().equalsIgnoreCase(VertexState.SUCCEEDED.toString()));
+ assertTrue(vertexInfo.getCompletedTasksCount() > 0);
+ assertTrue(vertexInfo.getFirstTaskToStart() != null);
+ assertTrue(vertexInfo.getSucceededTasksCount() > 0);
+ assertTrue(vertexInfo.getTasks().size() > 0);
+ }
+
+ for (TaskInfo taskInfo : vertexInfo.getTasks()) {
+ if (taskInfo.getStatus().equals(TaskState.SUCCEEDED.toString())) {
+ verifyTask(taskInfo, false);
+ }
+ }
+
+ for (TaskInfo taskInfo : vertexInfo.getFailedTasks()) {
+ verifyTask(taskInfo, true);
+ }
+
+ assertTrue(vertexInfo.getProcessorClassName() != null);
+ assertTrue(vertexInfo.getStatus() != null);
+ assertTrue(vertexInfo.getDagInfo() != null);
+ assertTrue(vertexInfo.getInitTimeInterval() > 0);
+ assertTrue(vertexInfo.getNumTasks() > 0);
+ }
+
+ private void verifyTask(TaskInfo taskInfo, boolean hasFailedAttempts) {
+ assertTrue(taskInfo != null);
+ assertTrue(taskInfo.getStatus() != null);
+ assertTrue(taskInfo.getStartTimeInterval() > 0);
+
+ //Not testing for killed attempts. So if there are no failures, it should succeed
+ if (!hasFailedAttempts) {
+ assertTrue(taskInfo.getStatus().equals(TaskState.SUCCEEDED.toString()));
+ assertTrue(taskInfo.getFinishTimeInterval() > 0 && taskInfo.getFinishTime() > taskInfo
+ .getFinishTimeInterval());
+ assertTrue(
+ taskInfo.getStartTimeInterval() > 0 && taskInfo.getStartTime() > taskInfo.getStartTimeInterval());
+ assertTrue(taskInfo.getSuccessfulAttemptId() != null);
+ assertTrue(taskInfo.getSuccessfulTaskAttempt() != null);
+ }
+ assertTrue(taskInfo.getTaskId() != null);
+
+ for (TaskAttemptInfo attemptInfo : taskInfo.getTaskAttempts()) {
+ verifyTaskAttemptInfo(attemptInfo);
+ }
+ }
+
+ private void verifyTaskAttemptInfo(TaskAttemptInfo attemptInfo) {
+ if (attemptInfo.getStatus() != null && attemptInfo.getStatus()
+ .equals(TaskAttemptState.SUCCEEDED)) {
+ assertTrue(attemptInfo.getStartTimeInterval() > 0);
+ assertTrue(attemptInfo.getFinishTimeInterval() > 0);
+ assertTrue(attemptInfo.getCreationTime() > 0);
+ assertTrue(attemptInfo.getAllocationTime() > 0);
+ assertTrue(attemptInfo.getStartTime() > 0);
+ assertTrue(attemptInfo.getFinishTime() > 0);
+ assertTrue(attemptInfo.getFinishTime() > attemptInfo.getStartTime());
+ assertTrue(attemptInfo.getFinishTime() > attemptInfo.getFinishTimeInterval());
+ assertTrue(attemptInfo.getStartTime() > attemptInfo.getStartTimeInterval());
+ assertTrue(attemptInfo.getNodeId() != null);
+ assertTrue(attemptInfo.getTimeTaken() != -1);
+ assertTrue(attemptInfo.getEvents() != null);
+ assertTrue(attemptInfo.getTezCounters() != null);
+ assertTrue(attemptInfo.getContainer() != null);
+ }
+ assertTrue(attemptInfo.getTaskInfo() != null);
+ }
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/8c8db7c5/tez-tools/analyzers/job-analyzer/findbugs-exclude.xml
----------------------------------------------------------------------
diff --git a/tez-tools/analyzers/job-analyzer/findbugs-exclude.xml b/tez-tools/analyzers/job-analyzer/findbugs-exclude.xml
new file mode 100644
index 0000000..5bebb05
--- /dev/null
+++ b/tez-tools/analyzers/job-analyzer/findbugs-exclude.xml
@@ -0,0 +1,28 @@
+<!--
+ 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. See accompanying LICENSE file.
+-->
+<FindBugsFilter>
+
+
+ <Match>
+ <Class name="org.apache.tez.analyzer.CSVResult"/>
+ <Bug pattern="EI_EXPOSE_REP2"/>
+ </Match>
+
+ <Match>
+ <Class name="org.apache.tez.analyzer.CSVResult"/>
+ <Bug pattern="EI_EXPOSE_REP"/>
+ </Match>
+
+
+</FindBugsFilter>
http://git-wip-us.apache.org/repos/asf/tez/blob/8c8db7c5/tez-tools/analyzers/job-analyzer/pom.xml
----------------------------------------------------------------------
diff --git a/tez-tools/analyzers/job-analyzer/pom.xml b/tez-tools/analyzers/job-analyzer/pom.xml
new file mode 100644
index 0000000..627c444
--- /dev/null
+++ b/tez-tools/analyzers/job-analyzer/pom.xml
@@ -0,0 +1,168 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ 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. See accompanying LICENSE file.
+-->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+ xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+ <modelVersion>4.0.0</modelVersion>
+ <parent>
+ <groupId>org.apache.tez</groupId>
+ <artifactId>tez-perf-analyzer</artifactId>
+ <version>0.7.1-SNAPSHOT</version>
+ </parent>
+ <artifactId>tez-job-analyzer</artifactId>
+
+ <dependencies>
+ <dependency>
+ <groupId>io.dropwizard.metrics</groupId>
+ <artifactId>metrics-core</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.tez</groupId>
+ <artifactId>tez-history-parser</artifactId>
+ <version>${project.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.tez</groupId>
+ <artifactId>tez-api</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.tez</groupId>
+ <artifactId>tez-common</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.tez</groupId>
+ <artifactId>tez-dag</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.tez</groupId>
+ <artifactId>tez-tests</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.tez</groupId>
+ <artifactId>tez-tests</artifactId>
+ <type>test-jar</type>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.tez</groupId>
+ <artifactId>tez-yarn-timeline-history</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.tez</groupId>
+ <artifactId>tez-yarn-timeline-history</artifactId>
+ <scope>test</scope>
+ <type>test-jar</type>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-common</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-yarn-api</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-yarn-common</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-yarn-client</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-yarn-server-tests</artifactId>
+ <scope>test</scope>
+ <type>test-jar</type>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-hdfs</artifactId>
+ <scope>test</scope>
+ <type>test-jar</type>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-common</artifactId>
+ <scope>test</scope>
+ <type>test-jar</type>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-mapreduce-client-core</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-mapreduce-client-common</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.hadoop</groupId>
+ <artifactId>hadoop-mapreduce-client-jobclient</artifactId>
+ <scope>test</scope>
+ <type>test-jar</type>
+ </dependency>
+ <dependency>
+ <groupId>org.codehaus.jettison</groupId>
+ <artifactId>jettison</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>com.google.guava</groupId>
+ <artifactId>guava</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.slf4j</groupId>
+ <artifactId>slf4j-api</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.mockito</groupId>
+ <artifactId>mockito-all</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>junit</groupId>
+ <artifactId>junit</artifactId>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>com.sun.jersey</groupId>
+ <artifactId>jersey-json</artifactId>
+ </dependency>
+ </dependencies>
+
+ <build>
+ <plugins>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-jar-plugin</artifactId>
+ <configuration>
+ <archive>
+ <manifest>
+ <mainClass>org.apache.tez.analyzer.plugins.AnalyzerDriver</mainClass>
+ </manifest>
+ </archive>
+ </configuration>
+ </plugin>
+ <plugin>
+ <groupId>org.apache.rat</groupId>
+ <artifactId>apache-rat-plugin</artifactId>
+ </plugin>
+ </plugins>
+ </build>
+
+</project>
http://git-wip-us.apache.org/repos/asf/tez/blob/8c8db7c5/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/Analyzer.java
----------------------------------------------------------------------
diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/Analyzer.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/Analyzer.java
new file mode 100644
index 0000000..6021c58
--- /dev/null
+++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/Analyzer.java
@@ -0,0 +1,64 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.analyzer;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.tez.dag.api.TezException;
+import org.apache.tez.history.parser.datamodel.DagInfo;
+
+
+public interface Analyzer {
+
+ /**
+ * Analyze Dag
+ *
+ * @param dagInfo
+ * @throws TezException
+ */
+ public void analyze(DagInfo dagInfo) throws TezException;
+
+ /**
+ * Get the result of analysis
+ *
+ * @return analysis result
+ * @throws TezException
+ */
+ public Result getResult() throws TezException;
+
+ /**
+ * Get name of the analyzer
+ *
+ * @return name of analyze
+ */
+ public String getName();
+
+ /**
+ * Get description of the analyzer
+ *
+ * @return description of analyzer
+ */
+ public String getDescription();
+
+ /**
+ * Get config properties related to this analyzer
+ *
+ * @return config related to analyzer
+ */
+ public Configuration getConfiguration();
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/8c8db7c5/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/CSVResult.java
----------------------------------------------------------------------
diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/CSVResult.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/CSVResult.java
new file mode 100644
index 0000000..5246c68
--- /dev/null
+++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/CSVResult.java
@@ -0,0 +1,115 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.analyzer;
+
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
+import com.google.common.collect.Iterators;
+import com.google.common.collect.Lists;
+import org.apache.tez.dag.api.TezException;
+
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.nio.charset.Charset;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.List;
+
+/**
+ * Simple placeholder for storing CSV results.
+ * Contains headers and records in string format.
+ */
+public class CSVResult implements Result {
+
+ private final String[] headers;
+ private final List<String[]> recordsList;
+ private String comments;
+
+ public CSVResult(String[] header) {
+ this.headers = header;
+ recordsList = Lists.newLinkedList();
+ }
+
+ public String[] getHeaders() {
+ return headers;
+ }
+
+ public void addRecord(String[] record) {
+ Preconditions.checkArgument(record != null, "Record can't be null");
+ Preconditions.checkArgument(record.length == headers.length, "Record length" + record.length +
+ " does not match headers length " + headers.length);
+ recordsList.add(record);
+ }
+
+ public Iterator<String[]> getRecordsIterator() {
+ return Iterators.unmodifiableIterator(recordsList.iterator());
+ }
+
+
+ public void setComments(String comments) {
+ this.comments = comments;
+ }
+
+ @Override public String toJson() throws TezException {
+ return "";
+ }
+
+ @Override public String getComments() {
+ return comments;
+ }
+
+ @Override public String toString() {
+ return "CSVResult{" +
+ "headers=" + Arrays.toString(headers) +
+ ", recordsList=" + recordsList +
+ '}';
+ }
+
+ //For testing
+ public void dumpToFile(String fileName) throws IOException {
+ OutputStreamWriter writer = new OutputStreamWriter(
+ new FileOutputStream(new File(fileName)),
+ Charset.forName("UTF-8").newEncoder());
+ BufferedWriter bw = new BufferedWriter(writer);
+ bw.write(Joiner.on(",").join(headers));
+ bw.newLine();
+ for (String[] record : recordsList) {
+
+ if (record.length != headers.length) {
+ continue; //LOG error msg?
+ }
+
+ StringBuilder sb = new StringBuilder();
+ for(int i=0;i<record.length;i++) {
+ sb.append(!Strings.isNullOrEmpty(record[i]) ? record[i] : " ");
+ if (i < record.length - 1) {
+ sb.append(",");
+ }
+ }
+ bw.write(sb.toString());
+ bw.newLine();
+ }
+ bw.flush();
+ bw.close();
+ }
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/8c8db7c5/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/Result.java
----------------------------------------------------------------------
diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/Result.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/Result.java
new file mode 100644
index 0000000..d1881eb
--- /dev/null
+++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/Result.java
@@ -0,0 +1,39 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.analyzer;
+
+import org.apache.tez.dag.api.TezException;
+
+public interface Result {
+
+ /**
+ * Convert result to json format
+ *
+ * @return json
+ * @throws TezException
+ */
+ public String toJson() throws TezException;
+
+ /**
+ * Recommendation / comments about the analysis if any.
+ *
+ * @return comments
+ */
+ public String getComments();
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/8c8db7c5/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/AnalyzerDriver.java
----------------------------------------------------------------------
diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/AnalyzerDriver.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/AnalyzerDriver.java
new file mode 100644
index 0000000..57b21cb
--- /dev/null
+++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/AnalyzerDriver.java
@@ -0,0 +1,59 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.analyzer.plugins;
+
+import org.apache.hadoop.util.ProgramDriver;
+
+public class AnalyzerDriver {
+
+ public static void main(String argv[]){
+ int exitCode = -1;
+ ProgramDriver pgd = new ProgramDriver();
+ try {
+ pgd.addClass("CriticalPath", CriticalPathAnalyzer.class,
+ "Find the critical path of a DAG");
+ pgd.addClass("ContainerReuseAnalyzer", ContainerReuseAnalyzer.class,
+ "Print container reuse details in a DAG");
+ pgd.addClass("LocalityAnalyzer", LocalityAnalyzer.class,
+ "Print locality details in a DAG");
+ pgd.addClass("ShuffleTimeAnalyzer", ShuffleTimeAnalyzer.class,
+ "Analyze the shuffle time details in a DAG");
+ pgd.addClass("SkewAnalyzer", SkewAnalyzer.class,
+ "Analyze the skew details in a DAG");
+ pgd.addClass("SlowestVertexAnalyzer", SlowestVertexAnalyzer.class,
+ "Print slowest vertex details in a DAG");
+ pgd.addClass("SlowNodeAnalyzer", SlowNodeAnalyzer.class,
+ "Print node details in a DAG");
+ pgd.addClass("SlowTaskIdentifier", SlowTaskIdentifier.class,
+ "Print slow task details in a DAG");
+ pgd.addClass("SpillAnalyzer", SpillAnalyzerImpl.class,
+ "Print spill details in a DAG");
+ pgd.addClass("TaskConcurrencyAnalyzer", TaskConcurrencyAnalyzer.class,
+ "Print the task concurrency details in a DAG");
+ pgd.addClass("VertexLevelCriticalPathAnalyzer", VertexLevelCriticalPathAnalyzer.class,
+ "Find critical path at vertex level in a DAG");
+ exitCode = pgd.run(argv);
+ } catch(Throwable e){
+ e.printStackTrace();
+ }
+
+ System.exit(exitCode);
+ }
+
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/tez/blob/8c8db7c5/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/ContainerReuseAnalyzer.java
----------------------------------------------------------------------
diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/ContainerReuseAnalyzer.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/ContainerReuseAnalyzer.java
new file mode 100644
index 0000000..5b862f8
--- /dev/null
+++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/ContainerReuseAnalyzer.java
@@ -0,0 +1,97 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.analyzer.plugins;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Multimap;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.tez.analyzer.Analyzer;
+import org.apache.tez.analyzer.CSVResult;
+import org.apache.tez.dag.api.TezException;
+import org.apache.tez.history.parser.datamodel.Container;
+import org.apache.tez.history.parser.datamodel.DagInfo;
+import org.apache.tez.history.parser.datamodel.TaskAttemptInfo;
+import org.apache.tez.history.parser.datamodel.VertexInfo;
+
+import java.util.List;
+
+
+/**
+ * Get container reuse information at a per vertex level basis.
+ */
+public class ContainerReuseAnalyzer extends TezAnalyzerBase implements Analyzer {
+
+ private final Configuration config;
+
+ private static final String[] headers =
+ { "vertexName", "taskAttempts", "node", "containerId", "reuseCount" };
+
+ private final CSVResult csvResult;
+
+ public ContainerReuseAnalyzer(Configuration config) {
+ this.config = config;
+ this.csvResult = new CSVResult(headers);
+ }
+
+ @Override
+ public void analyze(DagInfo dagInfo) throws TezException {
+ for (VertexInfo vertexInfo : dagInfo.getVertices()) {
+ Multimap<Container, TaskAttemptInfo> containers = vertexInfo.getContainersMapping();
+ for (Container container : containers.keySet()) {
+ List<String> record = Lists.newLinkedList();
+ record.add(vertexInfo.getVertexName());
+ record.add(vertexInfo.getTaskAttempts().size() + "");
+ record.add(container.getHost());
+ record.add(container.getId());
+ record.add(Integer.toString(containers.get(container).size()));
+ csvResult.addRecord(record.toArray(new String[record.size()]));
+ }
+ }
+ }
+
+ @Override
+ public CSVResult getResult() throws TezException {
+ return csvResult;
+ }
+
+ @Override
+ public String getName() {
+ return "Container Reuse Analyzer";
+ }
+
+ @Override
+ public String getDescription() {
+ return "Get details on container reuse analysis";
+ }
+
+ @Override
+ public Configuration getConfiguration() {
+ return config;
+ }
+
+ public static void main(String[] args) throws Exception {
+ Configuration config = new Configuration();
+ ContainerReuseAnalyzer analyzer = new ContainerReuseAnalyzer(config);
+ int res = ToolRunner.run(config, analyzer, args);
+ analyzer.printResults();
+ System.exit(res);
+ }
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/8c8db7c5/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/CriticalPathAnalyzer.java
----------------------------------------------------------------------
diff --git a/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/CriticalPathAnalyzer.java b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/CriticalPathAnalyzer.java
new file mode 100644
index 0000000..d4efdf9
--- /dev/null
+++ b/tez-tools/analyzers/job-analyzer/src/main/java/org/apache/tez/analyzer/plugins/CriticalPathAnalyzer.java
@@ -0,0 +1,646 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.analyzer.plugins;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.StringInterner;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.tez.analyzer.Analyzer;
+import org.apache.tez.analyzer.CSVResult;
+import org.apache.tez.analyzer.plugins.CriticalPathAnalyzer.CriticalPathStep.EntityType;
+import org.apache.tez.analyzer.utils.SVGUtils;
+import org.apache.tez.dag.api.TezException;
+import org.apache.tez.dag.api.oldrecords.TaskAttemptState;
+import org.apache.tez.dag.records.TaskAttemptTerminationCause;
+import org.apache.tez.history.parser.datamodel.Container;
+import org.apache.tez.history.parser.datamodel.DagInfo;
+import org.apache.tez.history.parser.datamodel.TaskAttemptInfo;
+import org.apache.tez.history.parser.datamodel.VertexInfo;
+import org.apache.tez.history.parser.datamodel.TaskAttemptInfo.DataDependencyEvent;
+import org.apache.tez.history.parser.datamodel.TaskInfo;
+
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+public class CriticalPathAnalyzer extends TezAnalyzerBase implements Analyzer {
+
+ String succeededState = StringInterner.weakIntern(TaskAttemptState.SUCCEEDED.name());
+ String failedState = StringInterner.weakIntern(TaskAttemptState.FAILED.name());
+
+ public enum CriticalPathDependency {
+ DATA_DEPENDENCY,
+ INIT_DEPENDENCY,
+ COMMIT_DEPENDENCY,
+ RETRY_DEPENDENCY,
+ OUTPUT_RECREATE_DEPENDENCY
+ }
+
+ public static final String DRAW_SVG = "tez.critical-path-analyzer.draw-svg";
+
+ public static class CriticalPathStep {
+ public enum EntityType {
+ ATTEMPT,
+ VERTEX_INIT,
+ DAG_COMMIT
+ }
+
+ EntityType type;
+ TaskAttemptInfo attempt;
+ CriticalPathDependency reason; // reason linking this to the previous step on the critical path
+ long startCriticalPathTime; // time at which attempt is on critical path
+ long stopCriticalPathTime; // time at which attempt is off critical path
+ List<String> notes = Lists.newLinkedList();
+
+ public CriticalPathStep(TaskAttemptInfo attempt, EntityType type) {
+ this.type = type;
+ this.attempt = attempt;
+ }
+ public EntityType getType() {
+ return type;
+ }
+ public TaskAttemptInfo getAttempt() {
+ return attempt;
+ }
+ public long getStartCriticalTime() {
+ return startCriticalPathTime;
+ }
+ public long getStopCriticalTime() {
+ return stopCriticalPathTime;
+ }
+ public CriticalPathDependency getReason() {
+ return reason;
+ }
+ public List<String> getNotes() {
+ return notes;
+ }
+ }
+
+ List<CriticalPathStep> criticalPath = Lists.newLinkedList();
+
+ Map<String, TaskAttemptInfo> attempts = Maps.newHashMap();
+
+ int maxConcurrency = 0;
+ ArrayList<TimeInfo> concurrencyByTime = Lists.newArrayList();
+
+ public CriticalPathAnalyzer() {
+ }
+
+ @Override
+ public void analyze(DagInfo dagInfo) throws TezException {
+ // get all attempts in the dag and find the last failed/succeeded attempt.
+ // ignore killed attempt to handle kills that happen upon dag completion
+ TaskAttemptInfo lastAttempt = null;
+ long lastAttemptFinishTime = 0;
+ for (VertexInfo vertex : dagInfo.getVertices()) {
+ for (TaskInfo task : vertex.getTasks()) {
+ for (TaskAttemptInfo attempt : task.getTaskAttempts()) {
+ attempts.put(attempt.getTaskAttemptId(), attempt);
+ if (attempt.getStatus().equals(succeededState) ||
+ attempt.getStatus().equals(failedState)) {
+ if (lastAttemptFinishTime < attempt.getFinishTime()) {
+ lastAttempt = attempt;
+ lastAttemptFinishTime = attempt.getFinishTime();
+ }
+ }
+ }
+ }
+ }
+
+ if (lastAttempt == null) {
+ System.out.println("Cannot find last attempt to finish in DAG " + dagInfo.getDagId());
+ return;
+ }
+
+ createCriticalPath(dagInfo, lastAttempt, lastAttemptFinishTime, attempts);
+
+ analyzeCriticalPath(dagInfo);
+
+ if (getConf().getBoolean(DRAW_SVG, true)) {
+ saveCriticalPathAsSVG(dagInfo);
+ }
+ }
+
+ public List<CriticalPathStep> getCriticalPath() {
+ return criticalPath;
+ }
+
+ private void saveCriticalPathAsSVG(DagInfo dagInfo) {
+ SVGUtils svg = new SVGUtils();
+ String outputFileName = getOutputDir() + File.separator + dagInfo.getDagId() + ".svg";
+ System.out.println("Writing output to: " + outputFileName);
+ svg.saveCriticalPathAsSVG(dagInfo, outputFileName, criticalPath);
+ }
+
+ static class TimeInfo implements Comparable<TimeInfo> {
+ long timestamp;
+ int count;
+ boolean start;
+ TimeInfo(long timestamp, boolean start) {
+ this.timestamp = timestamp;
+ this.start = start;
+ }
+
+ @Override
+ public int compareTo(TimeInfo o) {
+ return Long.compare(this.timestamp, o.timestamp);
+ }
+
+ @Override
+ public int hashCode() {
+ return (int)((timestamp >> 32) ^ timestamp);
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if(o == null) {
+ return false;
+ }
+ if (o.getClass() == this.getClass()) {
+ TimeInfo other = (TimeInfo) o;
+ return (this.compareTo(other) == 0);
+ }
+ else {
+ return false;
+ }
+ }
+ }
+
+ private void determineConcurrency(DagInfo dag) {
+ ArrayList<TimeInfo> timeInfo = Lists.newArrayList();
+ for (VertexInfo v : dag.getVertices()) {
+ for (TaskInfo t : v.getTasks()) {
+ for (TaskAttemptInfo a : t.getTaskAttempts()) {
+ if (a.getStartTime() > 0) {
+ timeInfo.add(new TimeInfo(a.getStartTime(), true));
+ timeInfo.add(new TimeInfo(a.getFinishTime(), false));
+ }
+ }
+ }
+ }
+ Collections.sort(timeInfo);
+
+ int concurrency = 0;
+ TimeInfo lastTimeInfo = null;
+ for (TimeInfo t : timeInfo) {
+ concurrency += (t.start) ? 1 : -1;
+ maxConcurrency = (concurrency > maxConcurrency) ? concurrency : maxConcurrency;
+ if (lastTimeInfo == null || lastTimeInfo.timestamp < t.timestamp) {
+ lastTimeInfo = t;
+ lastTimeInfo.count = concurrency;
+ concurrencyByTime.add(lastTimeInfo);
+ } else {
+ // lastTimeInfo.timestamp == t.timestamp
+ lastTimeInfo.count = concurrency;
+ }
+ }
+// for (TimeInfo t : concurrencyByTime) {
+// System.out.println(t.timestamp + " " + t.count);
+// }
+ }
+
+ private int getIntervalMaxConcurrency(long begin, long end) {
+ int concurrency = 0;
+ for (TimeInfo timeInfo : concurrencyByTime) {
+ if (timeInfo.timestamp < begin) {
+ continue;
+ }
+ if (timeInfo.timestamp > end) {
+ break;
+ }
+ if (timeInfo.count > concurrency) {
+ concurrency = timeInfo.count;
+ }
+ }
+ return concurrency;
+ }
+
+ private void analyzeAllocationOverhead(DagInfo dag) {
+ List<TaskAttemptInfo> preemptedAttempts = Lists.newArrayList();
+ for (VertexInfo v : dag.getVertices()) {
+ for (TaskInfo t : v.getTasks()) {
+ for (TaskAttemptInfo a : t.getTaskAttempts()) {
+ if (a.getTerminationCause().equals(
+ TaskAttemptTerminationCause.INTERNAL_PREEMPTION.name())) {
+ System.out.println("Found preempted attempt " + a.getTaskAttemptId());
+ preemptedAttempts.add(a);
+ }
+ }
+ }
+ }
+ for (int i = 0; i < criticalPath.size(); ++i) {
+ CriticalPathStep step = criticalPath.get(i);
+ TaskAttemptInfo attempt = step.attempt;
+ if (step.getType() != EntityType.ATTEMPT) {
+ continue;
+ }
+
+ long creationTime = attempt.getCreationTime();
+ long allocationTime = attempt.getAllocationTime();
+ long finishTime = attempt.getFinishTime();
+ if (allocationTime < step.startCriticalPathTime) {
+ // allocated before it became critical
+ continue;
+ }
+
+ // the attempt is critical before allocation. So allocation overhead needs analysis
+ Container container = attempt.getContainer();
+ if (container != null) {
+ Collection<TaskAttemptInfo> attempts = dag.getContainerMapping().get(container);
+ if (attempts != null && !attempts.isEmpty()) {
+ // arrange attempts by allocation time
+ List<TaskAttemptInfo> attemptsList = Lists.newArrayList(attempts);
+ Collections.sort(attemptsList, TaskAttemptInfo.orderingOnAllocationTime());
+ // walk the list to record allocation time before the current attempt
+ long containerPreviousAllocatedTime = 0;
+ int reUsesForVertex = 1;
+ for (TaskAttemptInfo containerAttempt : attemptsList) {
+ if (containerAttempt.getTaskAttemptId().equals(attempt.getTaskAttemptId())) {
+ break;
+ }
+ if (containerAttempt.getTaskInfo().getVertexInfo().getVertexId().equals(
+ attempt.getTaskInfo().getVertexInfo().getVertexId())) {
+ // another task from the same vertex ran in this container. So there are multiple
+ // waves for this vertex on this container.
+ reUsesForVertex++;
+ }
+ long cAllocTime = containerAttempt.getAllocationTime();
+ long cFinishTime = containerAttempt.getFinishTime();
+ if (cFinishTime > creationTime) {
+ // for containerAttempts that used the container while this attempt was waiting
+ // add up time container was allocated to containerAttempt. Account for allocations
+ // that started before this attempt was created.
+ containerPreviousAllocatedTime +=
+ (cFinishTime - (cAllocTime > creationTime ? cAllocTime : creationTime));
+ }
+ }
+ int numVertexTasks = attempt.getTaskInfo().getVertexInfo().getNumTasks();
+ int intervalMaxConcurrency = getIntervalMaxConcurrency(creationTime, finishTime);
+ double numWaves = getWaves(numVertexTasks, intervalMaxConcurrency);
+
+ if (reUsesForVertex > 1) {
+ step.notes.add("Container ran multiple tasks for this vertex. ");
+ if (numWaves < 1) {
+ // less than 1 wave total but still ran more than 1 on this container
+ step.notes.add("Vertex potentially seeing contention from other branches in the DAG. ");
+ }
+ }
+ if (containerPreviousAllocatedTime == 0) {
+ step.notes.add("Container newly allocated.");
+ } else {
+ if (containerPreviousAllocatedTime >= attempt.getCreationToAllocationTimeInterval()) {
+ step.notes.add("Container was fully allocated");
+ } else {
+ step.notes.add("Container in use for " +
+ SVGUtils.getTimeStr(containerPreviousAllocatedTime) + " out of " +
+ SVGUtils.getTimeStr(attempt.getCreationToAllocationTimeInterval()) +
+ " of allocation wait time");
+ }
+ }
+ }
+ // look for internal preemptions while attempt was waiting for allocation
+ for (TaskAttemptInfo a : preemptedAttempts) {
+ if (a.getTaskInfo().getVertexInfo().getVertexId()
+ .equals(attempt.getTaskInfo().getVertexInfo().getVertexId())) {
+ // dont preempt same vertex task. ideally this should look at priority but we dont have it
+ continue;
+ }
+ if (a.getFinishTime() > creationTime && a.getFinishTime() < allocationTime) {
+ // found an attempt that was preempted within this time interval
+ step.notes.add("Potentially waited for preemption of " + a.getShortName());
+ }
+ }
+ }
+ }
+ }
+
+ private double getWaves(int numTasks, int concurrency) {
+ double numWaves = (numTasks*1.0) / concurrency;
+ numWaves = (double)Math.round(numWaves * 10d) / 10d; // convert to 1 decimal place
+ return numWaves;
+ }
+
+ private void analyzeWaves(DagInfo dag) {
+ for (int i = 0; i < criticalPath.size(); ++i) {
+ CriticalPathStep step = criticalPath.get(i);
+ TaskAttemptInfo attempt = step.attempt;
+ if (step.getType() != EntityType.ATTEMPT) {
+ continue;
+ }
+ long creationTime = attempt.getCreationTime();
+ long finishTime = attempt.getFinishTime();
+
+ int numVertexTasks = attempt.getTaskInfo().getVertexInfo().getNumTasks();
+ if (numVertexTasks <= 1) {
+ continue;
+ }
+ int intervalMaxConcurrency = getIntervalMaxConcurrency(creationTime, finishTime);
+ double numWaves = getWaves(numVertexTasks, intervalMaxConcurrency);
+
+ step.notes.add("Vertex ran " + numVertexTasks
+ + " tasks in " + numWaves
+ + " waves with available concurrency of " + intervalMaxConcurrency);
+ if (numWaves > 1) {
+ if (numWaves%1 < 0.5) {
+ // more than 1 wave needed and last wave is small
+ step.notes.add("Last partial wave did not use full concurrency. ");
+ }
+ }
+ }
+ }
+
+ private void analyzeStragglers(DagInfo dag) {
+ long dagStartTime = dag.getStartTime();
+ long dagTime = dag.getFinishTime() - dagStartTime;
+ long totalAttemptCriticalTime = 0;
+ for (int i = 0; i < criticalPath.size(); ++i) {
+ CriticalPathStep step = criticalPath.get(i);
+ totalAttemptCriticalTime += (step.stopCriticalPathTime - step.startCriticalPathTime);
+ TaskAttemptInfo attempt = step.attempt;
+ if (step.getType() == EntityType.ATTEMPT) {
+ // analyze execution overhead
+ if (attempt.getLastDataEvents().size() > 1) {
+ // there were read errors. that could have delayed the attempt. ignore this
+ continue;
+ }
+ long avgPostDataExecutionTime = attempt.getTaskInfo().getVertexInfo()
+ .getAvgPostDataExecutionTimeInterval();
+ if (avgPostDataExecutionTime <= 0) {
+ continue;
+ }
+ long attemptExecTime = attempt.getPostDataExecutionTimeInterval();
+ if (avgPostDataExecutionTime * 1.25 < attemptExecTime) {
+ step.notes
+ .add("Potential straggler. Post Data Execution time " +
+ SVGUtils.getTimeStr(attemptExecTime)
+ + " compared to vertex average of " +
+ SVGUtils.getTimeStr(avgPostDataExecutionTime));
+ }
+ }
+ }
+ System.out
+ .println("DAG time taken: " + dagTime + " TotalAttemptTime: " + totalAttemptCriticalTime
+ + " DAG finish time: " + dag.getFinishTime() + " DAG start time: " + dagStartTime);
+ }
+
+ private void analyzeCriticalPath(DagInfo dag) {
+ if (!criticalPath.isEmpty()) {
+ determineConcurrency(dag);
+ analyzeStragglers(dag);
+ analyzeWaves(dag);
+ analyzeAllocationOverhead(dag);
+ }
+ }
+
+ private void createCriticalPath(DagInfo dagInfo, TaskAttemptInfo lastAttempt,
+ long lastAttemptFinishTime, Map<String, TaskAttemptInfo> attempts) {
+ List<CriticalPathStep> tempCP = Lists.newLinkedList();
+ if (lastAttempt != null) {
+ TaskAttemptInfo currentAttempt = lastAttempt;
+ CriticalPathStep currentStep = new CriticalPathStep(currentAttempt, EntityType.DAG_COMMIT);
+ long currentAttemptStopCriticalPathTime = lastAttemptFinishTime;
+
+ // add the commit step
+ if (dagInfo.getFinishTime() > 0) {
+ currentStep.stopCriticalPathTime = dagInfo.getFinishTime();
+ } else {
+ // AM crashed and no dag finished written
+ currentStep.stopCriticalPathTime = currentAttemptStopCriticalPathTime;
+ }
+ currentStep.startCriticalPathTime = currentAttemptStopCriticalPathTime;
+ currentStep.reason = CriticalPathDependency.COMMIT_DEPENDENCY;
+ tempCP.add(currentStep);
+
+ while (true) {
+ Preconditions.checkState(currentAttempt != null);
+ Preconditions.checkState(currentAttemptStopCriticalPathTime > 0);
+ System.out.println(
+ "Step: " + tempCP.size() + " Attempt: " + currentAttempt.getTaskAttemptId());
+
+ currentStep = new CriticalPathStep(currentAttempt, EntityType.ATTEMPT);
+ currentStep.stopCriticalPathTime = currentAttemptStopCriticalPathTime;
+
+ // consider the last data event seen immediately preceding the current critical path
+ // stop time for this attempt
+ long currentStepLastDataEventTime = 0;
+ String currentStepLastDataTA = null;
+ DataDependencyEvent item = currentAttempt.getLastDataEventInfo(currentStep.stopCriticalPathTime);
+ if (item!=null) {
+ currentStepLastDataEventTime = item.getTimestamp();
+ currentStepLastDataTA = item.getTaskAttemptId();
+ }
+
+ // sanity check
+ for (CriticalPathStep previousStep : tempCP) {
+ if (previousStep.type == EntityType.ATTEMPT) {
+ if (previousStep.attempt.getTaskAttemptId().equals(currentAttempt.getTaskAttemptId())) {
+ // found loop.
+ // this should only happen for read errors in currentAttempt
+ List<DataDependencyEvent> dataEvents = currentAttempt.getLastDataEvents();
+ Preconditions.checkState(dataEvents.size() > 1); // received
+ // original and
+ // retry data events
+ Preconditions.checkState(currentStepLastDataEventTime < dataEvents
+ .get(dataEvents.size() - 1).getTimestamp()); // new event is
+ // earlier than
+ // last
+ }
+ }
+ }
+
+ tempCP.add(currentStep);
+
+ // find the next attempt on the critical path
+ boolean dataDependency = false;
+ // find out predecessor dependency
+ if (currentStepLastDataEventTime > currentAttempt.getCreationTime()) {
+ dataDependency = true;
+ }
+
+ long startCriticalPathTime = 0;
+ String nextAttemptId = null;
+ CriticalPathDependency reason = null;
+ if (dataDependency) {
+ // last data event was produced after the attempt was scheduled. use
+ // data dependency
+ // typically the case when scheduling ahead of time
+ System.out.println("Has data dependency");
+ if (!Strings.isNullOrEmpty(currentStepLastDataTA)) {
+ // there is a valid data causal TA. Use it.
+ nextAttemptId = currentStepLastDataTA;
+ reason = CriticalPathDependency.DATA_DEPENDENCY;
+ startCriticalPathTime = currentStepLastDataEventTime;
+ System.out.println("Using data dependency " + nextAttemptId);
+ } else {
+ // there is no valid data causal TA. This means data event came from the same vertex
+ VertexInfo vertex = currentAttempt.getTaskInfo().getVertexInfo();
+ Preconditions.checkState(!vertex.getAdditionalInputInfoList().isEmpty(),
+ "Vertex: " + vertex.getVertexId() + " has no external inputs but the last data event "
+ + "TA is null for " + currentAttempt.getTaskAttemptId());
+ nextAttemptId = null;
+ reason = CriticalPathDependency.INIT_DEPENDENCY;
+ System.out.println("Using init dependency");
+ }
+ } else {
+ // attempt was scheduled after last data event. use scheduling dependency
+ // typically happens for retries
+ System.out.println("Has scheduling dependency");
+ if (!Strings.isNullOrEmpty(currentAttempt.getCreationCausalTA())) {
+ // there is a scheduling causal TA. Use it.
+ nextAttemptId = currentAttempt.getCreationCausalTA();
+ reason = CriticalPathDependency.RETRY_DEPENDENCY;
+ TaskAttemptInfo nextAttempt = attempts.get(nextAttemptId);
+ if (nextAttemptId != null) {
+ VertexInfo currentVertex = currentAttempt.getTaskInfo().getVertexInfo();
+ VertexInfo nextVertex = nextAttempt.getTaskInfo().getVertexInfo();
+ if (!nextVertex.getVertexName().equals(currentVertex.getVertexName())){
+ // cause from different vertex. Might be rerun to re-generate outputs
+ for (VertexInfo outVertex : currentVertex.getOutputVertices()) {
+ if (nextVertex.getVertexName().equals(outVertex.getVertexName())) {
+ // next vertex is an output vertex
+ reason = CriticalPathDependency.OUTPUT_RECREATE_DEPENDENCY;
+ break;
+ }
+ }
+ }
+ }
+ if (reason == CriticalPathDependency.OUTPUT_RECREATE_DEPENDENCY) {
+ // rescheduled due to read error. start critical at read error report time.
+ // for now proxy own creation time for read error report time
+ startCriticalPathTime = currentAttempt.getCreationTime();
+ } else {
+ // rescheduled due to own previous attempt failure
+ // we are critical when the previous attempt fails
+ Preconditions.checkState(nextAttempt != null);
+ Preconditions.checkState(nextAttempt.getTaskInfo().getTaskId().equals(
+ currentAttempt.getTaskInfo().getTaskId()));
+ startCriticalPathTime = nextAttempt.getFinishTime();
+ }
+ System.out.println("Using scheduling dependency " + nextAttemptId);
+ } else {
+ // there is no scheduling causal TA.
+ if (!Strings.isNullOrEmpty(currentStepLastDataTA)) {
+ // there is a data event going to the vertex. Count the time between data event and
+ // creation time as Initializer/Manager overhead and follow data dependency
+ nextAttemptId = currentStepLastDataTA;
+ reason = CriticalPathDependency.DATA_DEPENDENCY;
+ startCriticalPathTime = currentStepLastDataEventTime;
+ long overhead = currentAttempt.getCreationTime() - currentStepLastDataEventTime;
+ currentStep.notes
+ .add("Initializer/VertexManager scheduling overhead " + SVGUtils.getTimeStr(overhead));
+ System.out.println("Using data dependency " + nextAttemptId);
+ } else {
+ // there is no scheduling causal TA and no data event casual TA.
+ // the vertex has external input that sent the last data events
+ // or the vertex has external input but does not use events
+ // or the vertex has no external inputs or edges
+ nextAttemptId = null;
+ reason = CriticalPathDependency.INIT_DEPENDENCY;
+ System.out.println("Using init dependency");
+ }
+ }
+ }
+
+ currentStep.startCriticalPathTime = startCriticalPathTime;
+ currentStep.reason = reason;
+
+ Preconditions.checkState(currentStep.stopCriticalPathTime >= currentStep.startCriticalPathTime);
+
+ if (Strings.isNullOrEmpty(nextAttemptId)) {
+ Preconditions.checkState(reason.equals(CriticalPathDependency.INIT_DEPENDENCY));
+ Preconditions.checkState(startCriticalPathTime == 0);
+ // no predecessor attempt found. this is the last step in the critical path
+ // assume attempts start critical path time is when its scheduled. before that is
+ // vertex initialization time
+ currentStep.startCriticalPathTime = currentStep.attempt.getCreationTime();
+
+ // add vertex init step
+ long initStepStopCriticalTime = currentStep.startCriticalPathTime;
+ currentStep = new CriticalPathStep(currentAttempt, EntityType.VERTEX_INIT);
+ currentStep.stopCriticalPathTime = initStepStopCriticalTime;
+ currentStep.startCriticalPathTime = dagInfo.getStartTime();
+ currentStep.reason = CriticalPathDependency.INIT_DEPENDENCY;
+ tempCP.add(currentStep);
+
+ if (!tempCP.isEmpty()) {
+ for (int i=tempCP.size() - 1; i>=0; --i) {
+ criticalPath.add(tempCP.get(i));
+ }
+ }
+ return;
+ }
+
+ currentAttempt = attempts.get(nextAttemptId);
+ currentAttemptStopCriticalPathTime = startCriticalPathTime;
+ }
+ }
+ }
+
+ @Override
+ public CSVResult getResult() throws TezException {
+ String[] headers = { "Entity", "PathReason", "Status", "CriticalStartTime",
+ "CriticalStopTime", "Notes" };
+
+ CSVResult csvResult = new CSVResult(headers);
+ for (CriticalPathStep step : criticalPath) {
+ String entity = (step.getType() == EntityType.ATTEMPT ? step.getAttempt().getTaskAttemptId()
+ : (step.getType() == EntityType.VERTEX_INIT
+ ? step.attempt.getTaskInfo().getVertexInfo().getVertexName() : "DAG COMMIT"));
+ String [] record = {entity, step.getReason().name(),
+ step.getAttempt().getDetailedStatus(), String.valueOf(step.getStartCriticalTime()),
+ String.valueOf(step.getStopCriticalTime()),
+ Joiner.on(";").join(step.getNotes())};
+ csvResult.addRecord(record);
+ }
+ return csvResult;
+ }
+
+ @Override
+ public String getName() {
+ return "CriticalPathAnalyzer";
+ }
+
+ @Override
+ public String getDescription() {
+ return "Analyze critical path of the DAG";
+ }
+
+ @Override
+ public Configuration getConfiguration() {
+ return getConf();
+ }
+
+ public static void main(String[] args) throws Exception {
+ int res = ToolRunner.run(new Configuration(), new CriticalPathAnalyzer(), args);
+ System.exit(res);
+ }
+
+}
[4/5] tez git commit: TEZ-2973. Backport Analyzers to branch-0.7
Posted by je...@apache.org.
http://git-wip-us.apache.org/repos/asf/tez/blob/8c8db7c5/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/DagInfo.java
----------------------------------------------------------------------
diff --git a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/DagInfo.java b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/DagInfo.java
new file mode 100644
index 0000000..5fb760c
--- /dev/null
+++ b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/DagInfo.java
@@ -0,0 +1,586 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.history.parser.datamodel;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Predicate;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.LinkedHashMultimap;
+import com.google.common.collect.LinkedListMultimap;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Multimap;
+import com.google.common.collect.Multimaps;
+import com.google.common.collect.Ordering;
+import org.apache.commons.collections.BidiMap;
+import org.apache.commons.collections.bidimap.DualHashBidiMap;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.util.StringInterner;
+import org.apache.tez.client.CallerContext;
+import org.apache.tez.dag.api.event.VertexState;
+import org.codehaus.jettison.json.JSONArray;
+import org.codehaus.jettison.json.JSONException;
+import org.codehaus.jettison.json.JSONObject;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.hadoop.classification.InterfaceAudience.Public;
+import static org.apache.hadoop.classification.InterfaceStability.Evolving;
+
+@Public
+@Evolving
+public class DagInfo extends BaseInfo {
+
+ private static final Log LOG = LogFactory.getLog(DagInfo.class);
+
+ //Fields populated via JSON
+ private final String name;
+ private final long startTime;
+ private final long endTime;
+ private final long submitTime;
+ private final int failedTasks;
+ private final String dagId;
+ private final int numVertices;
+ private final String status;
+ private final String diagnostics;
+ private VersionInfo versionInfo;
+ private CallerContext callerContext;
+
+ //VertexID --> VertexName & vice versa
+ private final BidiMap vertexNameIDMapping;
+
+ //edgeId to EdgeInfo mapping
+ private final Map<Integer, EdgeInfo> edgeInfoMap;
+
+ //Only for internal parsing (vertexname mapping)
+ private Map<String, BasicVertexInfo> basicVertexInfoMap;
+
+ //VertexName --> VertexInfo
+ private Map<String, VertexInfo> vertexNameMap;
+
+ private Multimap<Container, TaskAttemptInfo> containerMapping;
+
+ DagInfo(JSONObject jsonObject) throws JSONException {
+ super(jsonObject);
+
+ vertexNameMap = Maps.newHashMap();
+ vertexNameIDMapping = new DualHashBidiMap();
+ edgeInfoMap = Maps.newHashMap();
+ basicVertexInfoMap = Maps.newHashMap();
+ containerMapping = LinkedHashMultimap.create();
+
+ Preconditions.checkArgument(jsonObject.getString(Constants.ENTITY_TYPE).equalsIgnoreCase
+ (Constants.TEZ_DAG_ID));
+
+ dagId = StringInterner.weakIntern(jsonObject.getString(Constants.ENTITY));
+
+ //Parse additional Info
+ JSONObject otherInfoNode = jsonObject.getJSONObject(Constants.OTHER_INFO);
+ startTime = otherInfoNode.optLong(Constants.START_TIME);
+ endTime = otherInfoNode.optLong(Constants.FINISH_TIME);
+ //TODO: Not getting populated correctly for lots of jobs. Verify
+ submitTime = otherInfoNode.optLong(Constants.START_REQUESTED_TIME);
+ diagnostics = otherInfoNode.optString(Constants.DIAGNOSTICS);
+ failedTasks = otherInfoNode.optInt(Constants.NUM_FAILED_TASKS);
+ JSONObject dagPlan = otherInfoNode.optJSONObject(Constants.DAG_PLAN);
+ name = StringInterner.weakIntern((dagPlan != null) ? (dagPlan.optString(Constants.DAG_NAME)) : null);
+ if (dagPlan != null) {
+ JSONArray vertices = dagPlan.optJSONArray(Constants.VERTICES);
+ if (vertices != null) {
+ numVertices = vertices.length();
+ } else {
+ numVertices = 0;
+ }
+ parseDAGPlan(dagPlan);
+ } else {
+ numVertices = 0;
+ }
+ status = StringInterner.weakIntern(otherInfoNode.optString(Constants.STATUS));
+
+ //parse name id mapping
+ JSONObject vertexIDMappingJson = otherInfoNode.optJSONObject(Constants.VERTEX_NAME_ID_MAPPING);
+ if (vertexIDMappingJson != null) {
+ //get vertex name
+ for (Map.Entry<String, BasicVertexInfo> entry : basicVertexInfoMap.entrySet()) {
+ String vertexId = vertexIDMappingJson.optString(entry.getKey());
+ //vertexName --> vertexId
+ vertexNameIDMapping.put(entry.getKey(), vertexId);
+ }
+ }
+ }
+
+ public static DagInfo create(JSONObject jsonObject) throws JSONException {
+ DagInfo dagInfo = new DagInfo(jsonObject);
+ return dagInfo;
+ }
+
+ private void parseDAGPlan(JSONObject dagPlan) throws JSONException {
+ int version = dagPlan.optInt(Constants.VERSION, 1);
+ parseEdges(dagPlan.optJSONArray(Constants.EDGES));
+
+ JSONArray verticesInfo = dagPlan.optJSONArray(Constants.VERTICES);
+ parseBasicVertexInfo(verticesInfo);
+
+ if (version > 1) {
+ parseDAGContext(dagPlan.optJSONObject(Constants.DAG_CONTEXT));
+ }
+ }
+
+ private void parseDAGContext(JSONObject callerContextInfo) {
+ if (callerContextInfo == null) {
+ LOG.info("No DAG Caller Context available");
+ return;
+ }
+ String context = callerContextInfo.optString(Constants.CONTEXT);
+ String callerId = callerContextInfo.optString(Constants.CALLER_ID);
+ String callerType = callerContextInfo.optString(Constants.CALLER_TYPE);
+ String description = callerContextInfo.optString(Constants.DESCRIPTION);
+
+ this.callerContext = CallerContext.create(context, description);
+ if (callerId != null && !callerId.isEmpty() && callerType != null && !callerType.isEmpty()) {
+ this.callerContext.setCallerIdAndType(callerId, callerType);
+ } else {
+ LOG.info("No DAG Caller Context Id and Type available");
+ }
+
+ }
+
+ private void parseBasicVertexInfo(JSONArray verticesInfo) throws JSONException {
+ if (verticesInfo == null) {
+ LOG.info("No vertices available.");
+ return;
+ }
+
+ //Parse basic information available in DAG for vertex and edges
+ for (int i = 0; i < verticesInfo.length(); i++) {
+ BasicVertexInfo basicVertexInfo = new BasicVertexInfo();
+
+ JSONObject vJson = verticesInfo.getJSONObject(i);
+ basicVertexInfo.vertexName =
+ vJson.optString(Constants.VERTEX_NAME);
+ JSONArray inEdges = vJson.optJSONArray(Constants.IN_EDGE_IDS);
+ if (inEdges != null) {
+ String[] inEdgeIds = new String[inEdges.length()];
+ for (int j = 0; j < inEdges.length(); j++) {
+ inEdgeIds[j] = inEdges.get(j).toString();
+ }
+ basicVertexInfo.inEdgeIds = inEdgeIds;
+ }
+
+ JSONArray outEdges = vJson.optJSONArray(Constants.OUT_EDGE_IDS);
+ if (outEdges != null) {
+ String[] outEdgeIds = new String[outEdges.length()];
+ for (int j = 0; j < outEdges.length(); j++) {
+ outEdgeIds[j] = outEdges.get(j).toString();
+ }
+ basicVertexInfo.outEdgeIds = outEdgeIds;
+ }
+
+ JSONArray addInputsJson =
+ vJson.optJSONArray(Constants.ADDITIONAL_INPUTS);
+ basicVertexInfo.additionalInputs = parseAdditionalDetailsForVertex(addInputsJson);
+
+ JSONArray addOutputsJson =
+ vJson.optJSONArray(Constants.ADDITIONAL_OUTPUTS);
+ basicVertexInfo.additionalOutputs = parseAdditionalDetailsForVertex(addOutputsJson);
+
+ basicVertexInfoMap.put(basicVertexInfo.vertexName, basicVertexInfo);
+ }
+ }
+
+ /**
+ * get additional details available for every vertex in the dag
+ *
+ * @param jsonArray
+ * @return AdditionalInputOutputDetails[]
+ * @throws JSONException
+ */
+ private AdditionalInputOutputDetails[] parseAdditionalDetailsForVertex(JSONArray jsonArray) throws
+ JSONException {
+ if (jsonArray != null) {
+ AdditionalInputOutputDetails[]
+ additionalInputOutputDetails = new AdditionalInputOutputDetails[jsonArray.length()];
+ for (int j = 0; j < jsonArray.length(); j++) {
+ String name = jsonArray.getJSONObject(j).optString(
+ Constants.NAME);
+ String clazz = jsonArray.getJSONObject(j).optString(
+ Constants.CLASS);
+ String initializer =
+ jsonArray.getJSONObject(j).optString(Constants.INITIALIZER);
+ String userPayloadText = jsonArray.getJSONObject(j).optString(
+ Constants.USER_PAYLOAD_TEXT);
+
+ additionalInputOutputDetails[j] =
+ new AdditionalInputOutputDetails(name, clazz, initializer, userPayloadText);
+
+ }
+ return additionalInputOutputDetails;
+ }
+ return null;
+ }
+
+ /**
+ * Parse edge details in the DAG
+ *
+ * @param edgesArray
+ *
+ * @throws JSONException
+ */
+ private void parseEdges(JSONArray edgesArray) throws JSONException {
+ if (edgesArray == null) {
+ return;
+ }
+ for (int i = 0; i < edgesArray.length(); i++) {
+ JSONObject edge = edgesArray.getJSONObject(i);
+ Integer edgeId = edge.optInt(Constants.EDGE_ID);
+ String inputVertexName =
+ edge.optString(Constants.INPUT_VERTEX_NAME);
+ String outputVertexName =
+ edge.optString(Constants.OUTPUT_VERTEX_NAME);
+ String dataMovementType =
+ edge.optString(Constants.DATA_MOVEMENT_TYPE);
+ String edgeSourceClass =
+ edge.optString(Constants.EDGE_SOURCE_CLASS);
+ String edgeDestinationClass =
+ edge.optString(Constants.EDGE_DESTINATION_CLASS);
+ String inputUserPayloadAsText =
+ edge.optString(Constants.INPUT_PAYLOAD_TEXT);
+ String outputUserPayloadAsText =
+ edge.optString(Constants.OUTPUT_PAYLOAD_TEXT);
+ EdgeInfo edgeInfo = new EdgeInfo(inputVertexName, outputVertexName,
+ dataMovementType, edgeSourceClass, edgeDestinationClass, inputUserPayloadAsText,
+ outputUserPayloadAsText);
+ edgeInfoMap.put(edgeId, edgeInfo);
+ }
+ }
+
+ static class BasicVertexInfo {
+ String vertexName;
+ String[] inEdgeIds;
+ String[] outEdgeIds;
+ AdditionalInputOutputDetails[] additionalInputs;
+ AdditionalInputOutputDetails[] additionalOutputs;
+ }
+
+ void addVertexInfo(VertexInfo vertexInfo) {
+ BasicVertexInfo basicVertexInfo = basicVertexInfoMap.get(vertexInfo.getVertexName());
+
+ Preconditions.checkArgument(basicVertexInfo != null,
+ "VerteName " + vertexInfo.getVertexName()
+ + " not present in DAG's vertices " + basicVertexInfoMap.entrySet());
+
+ //populate additional information in VertexInfo
+ if (basicVertexInfo.additionalInputs != null) {
+ vertexInfo.setAdditionalInputInfoList(Arrays.asList(basicVertexInfo.additionalInputs));
+ }
+ if (basicVertexInfo.additionalOutputs != null) {
+ vertexInfo.setAdditionalOutputInfoList(Arrays.asList(basicVertexInfo.additionalOutputs));
+ }
+
+ //Populate edge information in vertex
+ if (basicVertexInfo.inEdgeIds != null) {
+ for (String edge : basicVertexInfo.inEdgeIds) {
+ EdgeInfo edgeInfo = edgeInfoMap.get(Integer.parseInt(edge));
+ Preconditions.checkState(edgeInfo != null, "EdgeId " + edge + " not present in DAG");
+ vertexInfo.addInEdge(edgeInfo);
+ }
+ }
+
+ if (basicVertexInfo.outEdgeIds != null) {
+ for (String edge : basicVertexInfo.outEdgeIds) {
+ EdgeInfo edgeInfo = edgeInfoMap.get(Integer.parseInt(edge));
+ Preconditions.checkState(edgeInfo != null, "EdgeId " + edge + " not present in DAG");
+ vertexInfo.addOutEdge(edgeInfo);
+ }
+ }
+
+ vertexNameMap.put(vertexInfo.getVertexName(), vertexInfo);
+ }
+
+ void setVersionInfo(VersionInfo versionInfo) {
+ this.versionInfo = versionInfo;
+ }
+
+ void addContainerMapping(Container container, TaskAttemptInfo taskAttemptInfo) {
+ this.containerMapping.put(container, taskAttemptInfo);
+ }
+
+ @Override
+ public String toString() {
+ StringBuilder sb = new StringBuilder();
+ sb.append("[");
+ sb.append("dagID=").append(getDagId()).append(", ");
+ sb.append("dagName=").append(getName()).append(", ");
+ sb.append("status=").append(getStatus()).append(", ");
+ sb.append("startTime=").append(getStartTimeInterval()).append(", ");
+ sb.append("submitTime=").append(getSubmitTime()).append(", ");
+ sb.append("endTime=").append(getFinishTimeInterval()).append(", ");
+ sb.append("timeTaken=").append(getTimeTaken()).append(", ");
+ sb.append("diagnostics=").append(getDiagnostics()).append(", ");
+ sb.append("vertexNameIDMapping=").append(getVertexNameIDMapping()).append(", ");
+ sb.append("failedTasks=").append(getFailedTaskCount()).append(", ");
+ sb.append("events=").append(getEvents()).append(", ");
+ sb.append("status=").append(getStatus());
+ sb.append("]");
+ return sb.toString();
+ }
+
+ public Multimap<Container, TaskAttemptInfo> getContainerMapping() {
+ return Multimaps.unmodifiableMultimap(containerMapping);
+ }
+
+ public final VersionInfo getVersionInfo() {
+ return versionInfo;
+ }
+
+ public final CallerContext getCallerContext() {
+ return callerContext;
+ }
+
+ public final String getName() {
+ return name;
+ }
+
+ public final Collection<EdgeInfo> getEdges() {
+ return Collections.unmodifiableCollection(edgeInfoMap.values());
+ }
+
+ public final long getSubmitTime() {
+ return submitTime;
+ }
+
+ public final long getStartTime() {
+ return startTime;
+ }
+
+ public final long getFinishTime() {
+ return endTime;
+ }
+
+ /**
+ * Reference start time for the DAG. Vertex, Task, TaskAttempt would map on to this.
+ * If absolute start time is needed, call getAbsStartTime().
+ *
+ * @return starting time w.r.t to dag
+ */
+ public final long getStartTimeInterval() {
+ return 0;
+ }
+
+ @Override
+ public final long getFinishTimeInterval() {
+ long dagEndTime = (endTime - startTime);
+ if (dagEndTime < 0) {
+ //probably dag is not complete or failed in middle. get the last task attempt time
+ for (VertexInfo vertexInfo : getVertices()) {
+ dagEndTime = (vertexInfo.getFinishTimeInterval() > dagEndTime) ? vertexInfo.getFinishTimeInterval() : dagEndTime;
+ }
+ }
+ return dagEndTime;
+ }
+
+ public final long getTimeTaken() {
+ return getFinishTimeInterval();
+ }
+
+ public final String getStatus() {
+ return status;
+ }
+
+ /**
+ * Get vertexInfo for a given vertexid
+ *
+ * @param vertexId
+ * @return VertexInfo
+ */
+ public VertexInfo getVertexFromId(String vertexId) {
+ return vertexNameMap.get(vertexNameIDMapping.getKey(vertexId));
+ }
+
+ /**
+ * Get vertexInfo for a given vertex name
+ *
+ * @param vertexName
+ * @return VertexInfo
+ */
+ public final VertexInfo getVertex(String vertexName) {
+ return vertexNameMap.get(vertexName);
+ }
+
+ public final String getDiagnostics() {
+ return diagnostics;
+ }
+
+ /**
+ * Get all vertices
+ *
+ * @return List<VertexInfo>
+ */
+ public final List<VertexInfo> getVertices() {
+ List<VertexInfo> vertices = Lists.newLinkedList(vertexNameMap.values());
+ Collections.sort(vertices, new Comparator<VertexInfo>() {
+
+ @Override public int compare(VertexInfo o1, VertexInfo o2) {
+ return (o1.getStartTimeInterval() < o2.getStartTimeInterval()) ? -1 :
+ ((o1.getStartTimeInterval() == o2.getStartTimeInterval()) ?
+ 0 : 1);
+ }
+ });
+ return Collections.unmodifiableList(vertices);
+ }
+
+ /**
+ * Get list of failed vertices
+ *
+ * @return List<VertexInfo>
+ */
+ public final List<VertexInfo> getFailedVertices() {
+ return getVertices(VertexState.FAILED);
+ }
+
+ /**
+ * Get list of killed vertices
+ *
+ * @return List<VertexInfo>
+ */
+ public final List<VertexInfo> getKilledVertices() {
+ return getVertices(VertexState.KILLED);
+ }
+
+ /**
+ * Get list of failed vertices
+ *
+ * @return List<VertexInfo>
+ */
+ public final List<VertexInfo> getSuccessfullVertices() {
+ return getVertices(VertexState.SUCCEEDED);
+ }
+
+ /**
+ * Get list of vertices belonging to a specific state
+ *
+ * @param state
+ * @return Collection<VertexInfo>
+ */
+ public final List<VertexInfo> getVertices(final VertexState state) {
+ return Collections.unmodifiableList(Lists.newLinkedList(Iterables.filter(Lists.newLinkedList
+ (vertexNameMap.values()), new Predicate<VertexInfo>() {
+ @Override public boolean apply(VertexInfo input) {
+ return input.getStatus() != null && input.getStatus().equals(state.toString());
+ }
+ }
+ )
+ )
+ );
+ }
+
+ public final Map<String, VertexInfo> getVertexMapping() {
+ return Collections.unmodifiableMap(vertexNameMap);
+ }
+
+ private Ordering<VertexInfo> getVertexOrdering() {
+ return Ordering.from(new Comparator<VertexInfo>() {
+ @Override public int compare(VertexInfo o1, VertexInfo o2) {
+ return (o1.getTimeTaken() < o2.getTimeTaken()) ? -1 :
+ ((o1.getTimeTaken() == o2.getTimeTaken()) ?
+ 0 : 1);
+ }
+ });
+ }
+
+ /**
+ * Get the slowest vertex in the DAG
+ *
+ * @return VertexInfo
+ */
+ public final VertexInfo getSlowestVertex() {
+ List<VertexInfo> vertexInfoList = getVertices();
+ if (vertexInfoList.size() == 0) {
+ return null;
+ }
+ return getVertexOrdering().max(vertexInfoList);
+ }
+
+ /**
+ * Get the slowest vertex in the DAG
+ *
+ * @return VertexInfo
+ */
+ public final VertexInfo getFastestVertex() {
+ List<VertexInfo> vertexInfoList = getVertices();
+ if (vertexInfoList.size() == 0) {
+ return null;
+ }
+ return getVertexOrdering().min(vertexInfoList);
+ }
+
+ /**
+ * Get node details for this DAG. Would be useful for analyzing node to tasks.
+ *
+ * @return Multimap<String, TaskAttemptInfo> taskAttempt details at every node
+ */
+ public final Multimap<String, TaskAttemptInfo> getNodeDetails() {
+ Multimap<String, TaskAttemptInfo> nodeDetails = LinkedListMultimap.create();
+ for (VertexInfo vertexInfo : getVertices()) {
+ Multimap<Container, TaskAttemptInfo> containerMapping = vertexInfo.getContainersMapping();
+ for (Map.Entry<Container, TaskAttemptInfo> entry : containerMapping.entries()) {
+ nodeDetails.put(entry.getKey().getHost(), entry.getValue());
+ }
+ }
+ return nodeDetails;
+ }
+
+ /**
+ * Get containers used for this DAG
+ *
+ * @return Multimap<Container, TaskAttemptInfo> task attempt details at every container
+ */
+ public final Multimap<Container, TaskAttemptInfo> getContainersToTaskAttemptMapping() {
+ List<VertexInfo> VertexInfoList = getVertices();
+ Multimap<Container, TaskAttemptInfo> containerMapping = LinkedHashMultimap.create();
+
+ for (VertexInfo vertexInfo : VertexInfoList) {
+ containerMapping.putAll(vertexInfo.getContainersMapping());
+ }
+ return Multimaps.unmodifiableMultimap(containerMapping);
+ }
+
+ public final Map getVertexNameIDMapping() {
+ return vertexNameIDMapping;
+ }
+
+ public final int getNumVertices() {
+ return numVertices;
+ }
+
+ public final String getDagId() {
+ return dagId;
+ }
+
+ public final int getFailedTaskCount() {
+ return failedTasks;
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/8c8db7c5/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/EdgeInfo.java
----------------------------------------------------------------------
diff --git a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/EdgeInfo.java b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/EdgeInfo.java
new file mode 100644
index 0000000..ab8e831
--- /dev/null
+++ b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/EdgeInfo.java
@@ -0,0 +1,112 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.history.parser.datamodel;
+
+import static org.apache.hadoop.classification.InterfaceAudience.Public;
+import static org.apache.hadoop.classification.InterfaceStability.Evolving;
+
+@Public
+@Evolving
+public class EdgeInfo {
+
+ private final String inputVertexName;
+ private final String outputVertexName;
+ private final String dataMovementType;
+ private final String edgeSourceClass;
+ private final String edgeDestinationClass;
+ private final String inputUserPayloadAsText;
+ private final String outputUserPayloadAsText;
+
+ private VertexInfo sourceVertex;
+ private VertexInfo destinationVertex;
+
+ public EdgeInfo(String inputVertexName, String outputVertexName, String dataMovementType,
+ String edgeSourceClass, String edgeDestinationClass, String inputUserPayloadAsText, String
+ outputUserPayloadAsText) {
+ this.inputVertexName = inputVertexName;
+ this.outputVertexName = outputVertexName;
+ this.dataMovementType = dataMovementType;
+ this.edgeSourceClass = edgeSourceClass;
+ this.edgeDestinationClass = edgeDestinationClass;
+ this.inputUserPayloadAsText = inputUserPayloadAsText;
+ this.outputUserPayloadAsText = outputUserPayloadAsText;
+ }
+
+ public final String getInputVertexName() {
+ return inputVertexName;
+ }
+
+ public final String getOutputVertexName() {
+ return outputVertexName;
+ }
+
+ public final String getDataMovementType() {
+ return dataMovementType;
+ }
+
+ public final String getEdgeSourceClass() {
+ return edgeSourceClass;
+ }
+
+ public final String getEdgeDestinationClass() {
+ return edgeDestinationClass;
+ }
+
+ public final String getInputUserPayloadAsText() {
+ return inputUserPayloadAsText;
+ }
+
+ public final String getOutputUserPayloadAsText() {
+ return outputUserPayloadAsText;
+ }
+
+ public final VertexInfo getSourceVertex() {
+ return sourceVertex;
+ }
+
+ public final void setSourceVertex(VertexInfo sourceVertex) {
+ this.sourceVertex = sourceVertex;
+ }
+
+ public final VertexInfo getDestinationVertex() {
+ return destinationVertex;
+ }
+
+ public final void setDestinationVertex(VertexInfo destinationVertex) {
+ this.destinationVertex = destinationVertex;
+ }
+
+ @Override
+ public String toString() {
+ StringBuilder sb = new StringBuilder();
+ sb.append("[");
+ sb.append("inputVertexName=").append(inputVertexName).append(", ");
+ sb.append("outputVertexName=").append(outputVertexName).append(", ");
+ sb.append("dataMovementType=").append(dataMovementType).append(", ");
+ sb.append("edgeSourceClass=").append(edgeSourceClass).append(", ");
+ sb.append("edgeDestinationClass=").append(edgeDestinationClass).append(", ");
+ sb.append("inputUserPayloadAsText=").append(inputUserPayloadAsText).append(",");
+ sb.append("outputUserPayloadAsText=").append(outputUserPayloadAsText).append(", ");
+ sb.append("sourceVertex=").append(sourceVertex.getVertexName()).append(", ");
+ sb.append("destinationVertex=").append(destinationVertex.getVertexName()).append(", ");
+ sb.append("outputUserPayloadAsText=").append(outputUserPayloadAsText);
+ sb.append("]");
+ return sb.toString();
+ }
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/8c8db7c5/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/Event.java
----------------------------------------------------------------------
diff --git a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/Event.java b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/Event.java
new file mode 100644
index 0000000..70310f3
--- /dev/null
+++ b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/Event.java
@@ -0,0 +1,63 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.history.parser.datamodel;
+
+import static org.apache.hadoop.classification.InterfaceAudience.Public;
+import static org.apache.hadoop.classification.InterfaceStability.Evolving;
+
+@Public
+@Evolving
+public class Event {
+ private final String info;
+ private final String type;
+ private final long time;
+
+ private long refTime; //typically dag start time.
+
+ public Event(String info, String type, long time) {
+ this.time = time;
+ this.type = type;
+ this.info = info;
+ }
+
+ void setReferenceTime(long refTime) {
+ this.refTime = refTime;
+ }
+
+ public final String getInfo() {
+ return info;
+ }
+
+ public final String getType() {
+ return type;
+ }
+
+ public final long getAbsoluteTime() {
+ return time;
+ }
+
+ public final long getTime() {
+ return time - refTime;
+ }
+
+ @Override
+ public String toString() {
+ return "[info=" + info + ", type=" + type + ", time=" + time + "]";
+ }
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/8c8db7c5/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/TaskAttemptInfo.java
----------------------------------------------------------------------
diff --git a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/TaskAttemptInfo.java b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/TaskAttemptInfo.java
new file mode 100644
index 0000000..d373513
--- /dev/null
+++ b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/TaskAttemptInfo.java
@@ -0,0 +1,379 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.history.parser.datamodel;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Ordering;
+
+import org.apache.hadoop.util.StringInterner;
+import org.apache.tez.common.ATSConstants;
+import org.apache.tez.common.counters.DAGCounter;
+import org.apache.tez.common.counters.TaskCounter;
+import org.apache.tez.common.counters.TezCounter;
+import org.apache.tez.dag.api.oldrecords.TaskAttemptState;
+import org.apache.tez.history.parser.utils.Utils;
+import org.codehaus.jettison.json.JSONException;
+import org.codehaus.jettison.json.JSONObject;
+
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.hadoop.classification.InterfaceStability.Evolving;
+import static org.apache.hadoop.classification.InterfaceAudience.Public;
+
+@Public
+@Evolving
+public class TaskAttemptInfo extends BaseInfo {
+
+ private static final String SUCCEEDED = StringInterner.weakIntern(TaskAttemptState.SUCCEEDED.name());
+
+ private final String taskAttemptId;
+ private final long startTime;
+ private final long endTime;
+ private final String diagnostics;
+
+ private final long creationTime;
+ private final long allocationTime;
+ private final String containerId;
+ private final String nodeId;
+ private final String status;
+ private final String logUrl;
+ private final String creationCausalTA;
+ private final String terminationCause;
+ private final long executionTimeInterval;
+ // this list is in time order - array list for easy walking
+ private final ArrayList<DataDependencyEvent> lastDataEvents = Lists.newArrayList();
+
+ private TaskInfo taskInfo;
+
+ private Container container;
+
+ public static class DataDependencyEvent {
+ String taId;
+ long timestamp;
+ public DataDependencyEvent(String id, long time) {
+ taId = id;
+ timestamp = time;
+ }
+ public long getTimestamp() {
+ return timestamp;
+ }
+ public String getTaskAttemptId() {
+ return taId;
+ }
+ }
+
+ TaskAttemptInfo(JSONObject jsonObject) throws JSONException {
+ super(jsonObject);
+
+ Preconditions.checkArgument(
+ jsonObject.getString(Constants.ENTITY_TYPE).equalsIgnoreCase
+ (Constants.TEZ_TASK_ATTEMPT_ID));
+
+ taskAttemptId = StringInterner.weakIntern(jsonObject.optString(Constants.ENTITY));
+
+ //Parse additional Info
+ final JSONObject otherInfoNode = jsonObject.getJSONObject(Constants.OTHER_INFO);
+ startTime = otherInfoNode.optLong(Constants.START_TIME);
+ endTime = otherInfoNode.optLong(Constants.FINISH_TIME);
+ diagnostics = otherInfoNode.optString(Constants.DIAGNOSTICS);
+ creationTime = otherInfoNode.optLong(Constants.CREATION_TIME);
+ creationCausalTA = StringInterner.weakIntern(
+ otherInfoNode.optString(Constants.CREATION_CAUSAL_ATTEMPT));
+ allocationTime = otherInfoNode.optLong(Constants.ALLOCATION_TIME);
+ containerId = StringInterner.weakIntern(otherInfoNode.optString(Constants.CONTAINER_ID));
+ String id = otherInfoNode.optString(Constants.NODE_ID);
+ nodeId = StringInterner.weakIntern((id != null) ? (id.split(":")[0]) : "");
+ logUrl = otherInfoNode.optString(Constants.COMPLETED_LOGS_URL);
+
+ status = StringInterner.weakIntern(otherInfoNode.optString(Constants.STATUS));
+ container = new Container(containerId, nodeId);
+ if (otherInfoNode.has(Constants.LAST_DATA_EVENTS)) {
+ List<DataDependencyEvent> eventInfo = Utils.parseDataEventDependencyFromJSON(
+ otherInfoNode.optJSONObject(Constants.LAST_DATA_EVENTS));
+ long lastTime = 0;
+ for (DataDependencyEvent item : eventInfo) {
+ // check these are in time order
+ Preconditions.checkState(lastTime < item.getTimestamp());
+ lastTime = item.getTimestamp();
+ lastDataEvents.add(item);
+ }
+ }
+ terminationCause = StringInterner
+ .weakIntern(otherInfoNode.optString(ATSConstants.TASK_ATTEMPT_ERROR_ENUM));
+ executionTimeInterval = (endTime > startTime) ? (endTime - startTime) : 0;
+ }
+
+ public static Ordering<TaskAttemptInfo> orderingOnAllocationTime() {
+ return Ordering.from(new Comparator<TaskAttemptInfo>() {
+ @Override
+ public int compare(TaskAttemptInfo o1, TaskAttemptInfo o2) {
+ return (o1.getAllocationTime() < o2.getAllocationTime() ? -1
+ : o1.getAllocationTime() > o2.getAllocationTime() ? 1 : 0);
+ }
+ });
+ }
+
+ void setTaskInfo(TaskInfo taskInfo) {
+ Preconditions.checkArgument(taskInfo != null, "Provide valid taskInfo");
+ this.taskInfo = taskInfo;
+ taskInfo.addTaskAttemptInfo(this);
+ }
+
+ @Override
+ public final long getStartTimeInterval() {
+ return startTime - (getTaskInfo().getVertexInfo().getDagInfo().getStartTime());
+ }
+
+ @Override
+ public final long getFinishTimeInterval() {
+ return endTime - (getTaskInfo().getVertexInfo().getDagInfo().getStartTime());
+ }
+
+ public final boolean isSucceeded() {
+ return status.equals(SUCCEEDED);
+ }
+
+ public final List<DataDependencyEvent> getLastDataEvents() {
+ return lastDataEvents;
+ }
+
+ public final long getExecutionTimeInterval() {
+ return executionTimeInterval;
+ }
+
+ public final long getPostDataExecutionTimeInterval() {
+ if (getStartTime() > 0 && getFinishTime() > 0) {
+ // start time defaults to the actual start time
+ long postDataStartTime = startTime;
+ if (getLastDataEvents() != null && !getLastDataEvents().isEmpty()) {
+ // if last data event is after the start time then use last data event time
+ long lastEventTime = getLastDataEvents().get(getLastDataEvents().size()-1).getTimestamp();
+ postDataStartTime = startTime > lastEventTime ? startTime : lastEventTime;
+ }
+ return (getFinishTime() - postDataStartTime);
+ }
+ return -1;
+ }
+
+ public final long getAllocationToEndTimeInterval() {
+ return (endTime - allocationTime);
+ }
+
+ public final long getAllocationToStartTimeInterval() {
+ return (startTime - allocationTime);
+ }
+
+ public final long getCreationToAllocationTimeInterval() {
+ return (allocationTime - creationTime);
+ }
+
+ public final long getStartTime() {
+ return startTime;
+ }
+
+ public final long getFinishTime() {
+ return endTime;
+ }
+
+ public final long getCreationTime() {
+ return creationTime;
+ }
+
+ public final DataDependencyEvent getLastDataEventInfo(long timeThreshold) {
+ for (int i=lastDataEvents.size()-1; i>=0; i--) {
+ // walk back in time until we get first event that happened before the threshold
+ DataDependencyEvent item = lastDataEvents.get(i);
+ if (item.getTimestamp() < timeThreshold) {
+ return item;
+ }
+ }
+ return null;
+ }
+
+ public final long getTimeTaken() {
+ return getFinishTimeInterval() - getStartTimeInterval();
+ }
+
+ public final long getCreationTimeInterval() {
+ return creationTime - (getTaskInfo().getVertexInfo().getDagInfo().getStartTime());
+ }
+
+ public final String getCreationCausalTA() {
+ return creationCausalTA;
+ }
+
+ public final long getAllocationTime() {
+ return allocationTime;
+ }
+
+ public final String getShortName() {
+ return getTaskInfo().getVertexInfo().getVertexName() + " : " +
+ taskAttemptId.substring(taskAttemptId.lastIndexOf('_', taskAttemptId.lastIndexOf('_') - 1) + 1);
+ }
+
+ @Override
+ public final String getDiagnostics() {
+ return diagnostics;
+ }
+
+ public final String getTerminationCause() {
+ return terminationCause;
+ }
+
+ public static TaskAttemptInfo create(JSONObject taskInfoObject) throws JSONException {
+ return new TaskAttemptInfo(taskInfoObject);
+ }
+
+ public final boolean isLocalityInfoAvailable() {
+ Map<String, TezCounter> dataLocalTask = getCounter(DAGCounter.class.getName(),
+ DAGCounter.DATA_LOCAL_TASKS.toString());
+ Map<String, TezCounter> rackLocalTask = getCounter(DAGCounter.class.getName(),
+ DAGCounter.RACK_LOCAL_TASKS.toString());
+
+ Map<String, TezCounter> otherLocalTask = getCounter(DAGCounter.class.getName(),
+ DAGCounter.OTHER_LOCAL_TASKS.toString());
+
+ if (!dataLocalTask.isEmpty() || !rackLocalTask.isEmpty() || !otherLocalTask.isEmpty()) {
+ return true;
+ }
+ return false;
+ }
+
+ public final String getDetailedStatus() {
+ if (!Strings.isNullOrEmpty(getTerminationCause())) {
+ return getStatus() + ":" + getTerminationCause();
+ }
+ return getStatus();
+ }
+
+ public final TezCounter getLocalityInfo() {
+ Map<String, TezCounter> dataLocalTask = getCounter(DAGCounter.class.getName(),
+ DAGCounter.DATA_LOCAL_TASKS.toString());
+ Map<String, TezCounter> rackLocalTask = getCounter(DAGCounter.class.getName(),
+ DAGCounter.RACK_LOCAL_TASKS.toString());
+ Map<String, TezCounter> otherLocalTask = getCounter(DAGCounter.class.getName(),
+ DAGCounter.OTHER_LOCAL_TASKS.toString());
+
+ if (!dataLocalTask.isEmpty()) {
+ return dataLocalTask.get(DAGCounter.class.getName());
+ }
+
+ if (!rackLocalTask.isEmpty()) {
+ return rackLocalTask.get(DAGCounter.class.getName());
+ }
+
+ if (!otherLocalTask.isEmpty()) {
+ return otherLocalTask.get(DAGCounter.class.getName());
+ }
+ return null;
+ }
+
+ public final TaskInfo getTaskInfo() {
+ return taskInfo;
+ }
+
+ public final String getTaskAttemptId() {
+ return taskAttemptId;
+ }
+
+ public final String getNodeId() {
+ return nodeId;
+ }
+
+ public final String getStatus() {
+ return status;
+ }
+
+ public final Container getContainer() {
+ return container;
+ }
+
+ public final String getLogURL() {
+ return logUrl;
+ }
+
+ /**
+ * Get merge counter per source. Available in case of reducer task
+ *
+ * @return Map<String, TezCounter> merge phase time at every counter group level
+ */
+ public final Map<String, TezCounter> getMergePhaseTime() {
+ return getCounter(null, TaskCounter.MERGE_PHASE_TIME.name());
+ }
+
+ /**
+ * Get shuffle counter per source. Available in case of shuffle
+ *
+ * @return Map<String, TezCounter> shuffle phase time at every counter group level
+ */
+ public final Map<String, TezCounter> getShufflePhaseTime() {
+ return getCounter(null, TaskCounter.SHUFFLE_PHASE_TIME.name());
+ }
+
+ /**
+ * Get OUTPUT_BYTES counter per source. Available in case of map outputs
+ *
+ * @return Map<String, TezCounter> output bytes counter at every counter group
+ */
+ public final Map<String, TezCounter> getTaskOutputBytes() {
+ return getCounter(null, TaskCounter.OUTPUT_BYTES.name());
+ }
+
+ /**
+ * Get number of spills per source. (SPILLED_RECORDS / OUTPUT_RECORDS)
+ *
+ * @return Map<String, Long> spill count details
+ */
+ public final Map<String, Float> getSpillCount() {
+ Map<String, TezCounter> outputRecords = getCounter(null, "OUTPUT_RECORDS");
+ Map<String, TezCounter> spilledRecords = getCounter(null, "SPILLED_RECORDS");
+ Map<String, Float> result = Maps.newHashMap();
+ for (Map.Entry<String, TezCounter> entry : spilledRecords.entrySet()) {
+ String source = entry.getKey();
+ long spilledVal = entry.getValue().getValue();
+ long outputVal = outputRecords.get(source).getValue();
+ result.put(source, (spilledVal * 1.0f) / (outputVal * 1.0f));
+ }
+ return result;
+ }
+
+ public String toString() {
+ StringBuilder sb = new StringBuilder();
+ sb.append("[");
+ sb.append("taskAttemptId=").append(getTaskAttemptId()).append(", ");
+ sb.append("creationTime=").append(getCreationTimeInterval()).append(", ");
+ sb.append("startTime=").append(getStartTimeInterval()).append(", ");
+ sb.append("finishTime=").append(getFinishTimeInterval()).append(", ");
+ sb.append("timeTaken=").append(getTimeTaken()).append(", ");
+ sb.append("events=").append(getEvents()).append(", ");
+ sb.append("diagnostics=").append(getDiagnostics()).append(", ");
+ sb.append("container=").append(getContainer()).append(", ");
+ sb.append("nodeId=").append(getNodeId()).append(", ");
+ sb.append("logURL=").append(getLogURL()).append(", ");
+ sb.append("status=").append(getStatus());
+ sb.append("]");
+ return sb.toString();
+ }
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/8c8db7c5/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/TaskInfo.java
----------------------------------------------------------------------
diff --git a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/TaskInfo.java b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/TaskInfo.java
new file mode 100644
index 0000000..c6f89d6
--- /dev/null
+++ b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/TaskInfo.java
@@ -0,0 +1,354 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.history.parser.datamodel;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Predicate;
+import com.google.common.base.Strings;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.LinkedHashMultimap;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Multimap;
+import com.google.common.collect.Multimaps;
+import com.google.common.collect.Ordering;
+
+import org.apache.hadoop.util.StringInterner;
+import org.apache.tez.dag.api.oldrecords.TaskAttemptState;
+import org.codehaus.jettison.json.JSONException;
+import org.codehaus.jettison.json.JSONObject;
+
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.hadoop.classification.InterfaceAudience.Public;
+import static org.apache.hadoop.classification.InterfaceStability.Evolving;
+
+@Public
+@Evolving
+public class TaskInfo extends BaseInfo {
+
+ private final long startTime;
+ private final long endTime;
+ private final String diagnostics;
+ private final String successfulAttemptId;
+ private final long scheduledTime;
+ private final String status;
+ private final String taskId;
+
+ private VertexInfo vertexInfo;
+
+ private Map<String, TaskAttemptInfo> attemptInfoMap = Maps
+ .newHashMap();
+
+ TaskInfo(JSONObject jsonObject) throws JSONException {
+ super(jsonObject);
+
+ Preconditions.checkArgument(
+ jsonObject.getString(Constants.ENTITY_TYPE).equalsIgnoreCase
+ (Constants.TEZ_TASK_ID));
+
+ taskId = StringInterner.weakIntern(jsonObject.optString(Constants.ENTITY));
+
+ //Parse additional Info
+ final JSONObject otherInfoNode = jsonObject.getJSONObject(Constants.OTHER_INFO);
+ startTime = otherInfoNode.optLong(Constants.START_TIME);
+ endTime = otherInfoNode.optLong(Constants.FINISH_TIME);
+ diagnostics = otherInfoNode.optString(Constants.DIAGNOSTICS);
+ successfulAttemptId = StringInterner.weakIntern(
+ otherInfoNode.optString(Constants.SUCCESSFUL_ATTEMPT_ID));
+ scheduledTime = otherInfoNode.optLong(Constants.SCHEDULED_TIME);
+ status = StringInterner.weakIntern(otherInfoNode.optString(Constants.STATUS));
+ }
+
+ @Override
+ public final long getStartTimeInterval() {
+ return startTime - (vertexInfo.getDagInfo().getStartTime());
+ }
+
+ public final long getStartTime() {
+ return startTime;
+ }
+
+ public final long getFinishTime() {
+ return endTime;
+ }
+
+ @Override
+ public final long getFinishTimeInterval() {
+ long taskFinishTime = endTime - (vertexInfo.getDagInfo().getStartTime());
+ if (taskFinishTime < 0) {
+ //probably vertex is not complete or failed in middle. get the last task attempt time
+ for (TaskAttemptInfo attemptInfo : getTaskAttempts()) {
+ taskFinishTime = (attemptInfo.getFinishTimeInterval() > taskFinishTime)
+ ? attemptInfo.getFinishTimeInterval() : taskFinishTime;
+ }
+ }
+ return taskFinishTime;
+ }
+
+ @Override
+ public final String getDiagnostics() {
+ return diagnostics;
+ }
+
+ public static TaskInfo create(JSONObject taskInfoObject) throws
+ JSONException {
+ return new TaskInfo(taskInfoObject);
+ }
+
+ void addTaskAttemptInfo(TaskAttemptInfo taskAttemptInfo) {
+ attemptInfoMap.put(taskAttemptInfo.getTaskAttemptId(), taskAttemptInfo);
+ }
+
+ void setVertexInfo(VertexInfo vertexInfo) {
+ Preconditions.checkArgument(vertexInfo != null, "Provide valid vertexInfo");
+ this.vertexInfo = vertexInfo;
+ //link it to vertex
+ vertexInfo.addTaskInfo(this);
+ }
+
+ public final VertexInfo getVertexInfo() {
+ return vertexInfo;
+ }
+
+ /**
+ * Get all task attempts
+ *
+ * @return list of task attempt info
+ */
+ public final List<TaskAttemptInfo> getTaskAttempts() {
+ List<TaskAttemptInfo> attemptsList = Lists.newLinkedList(attemptInfoMap.values());
+ Collections.sort(attemptsList, orderingOnAttemptStartTime());
+ return Collections.unmodifiableList(attemptsList);
+ }
+
+ /**
+ * Get list of failed tasks
+ *
+ * @return List<TaskAttemptInfo>
+ */
+ public final List<TaskAttemptInfo> getFailedTaskAttempts() {
+ return getTaskAttempts(TaskAttemptState.FAILED);
+ }
+
+ /**
+ * Get list of killed tasks
+ *
+ * @return List<TaskAttemptInfo>
+ */
+ public final List<TaskAttemptInfo> getKilledTaskAttempts() {
+ return getTaskAttempts(TaskAttemptState.KILLED);
+ }
+
+ /**
+ * Get list of failed tasks
+ *
+ * @return List<TaskAttemptInfo>
+ */
+ public final List<TaskAttemptInfo> getSuccessfulTaskAttempts() {
+ return getTaskAttempts(TaskAttemptState.SUCCEEDED);
+ }
+
+ /**
+ * Get list of tasks belonging to a specific state
+ *
+ * @param state
+ * @return Collection<TaskAttemptInfo>
+ */
+ public final List<TaskAttemptInfo> getTaskAttempts(final TaskAttemptState state) {
+ return Collections.unmodifiableList(Lists.newLinkedList(Iterables.filter(Lists.newLinkedList
+ (attemptInfoMap.values()), new Predicate<TaskAttemptInfo>() {
+ @Override
+ public boolean apply(TaskAttemptInfo input) {
+ return input.getStatus() != null && input.getStatus().equals(state.toString());
+ }
+ }
+ )
+ )
+ );
+ }
+
+ /**
+ * Get the set of containers on which the task attempts ran for this task
+ *
+ * @return Multimap<Container, TaskAttemptInfo> task attempt details at container level
+ */
+ public final Multimap<Container, TaskAttemptInfo> getContainersMapping() {
+ Multimap<Container, TaskAttemptInfo> containerMapping = LinkedHashMultimap.create();
+ for (TaskAttemptInfo attemptInfo : getTaskAttempts()) {
+ containerMapping.put(attemptInfo.getContainer(), attemptInfo);
+ }
+ return Multimaps.unmodifiableMultimap(containerMapping);
+ }
+
+ /**
+ * Get the successful task attempt
+ *
+ * @return TaskAttemptInfo
+ */
+ public final TaskAttemptInfo getSuccessfulTaskAttempt() {
+ if (!Strings.isNullOrEmpty(getSuccessfulAttemptId())) {
+ for (TaskAttemptInfo attemptInfo : getTaskAttempts()) {
+ if (attemptInfo.getTaskAttemptId().equals(getSuccessfulAttemptId())) {
+ return attemptInfo;
+ }
+ }
+ }
+ // fall back to checking status if successful attempt id is not available
+ for (TaskAttemptInfo attemptInfo : getTaskAttempts()) {
+ if (attemptInfo.getStatus().equalsIgnoreCase(TaskAttemptState.SUCCEEDED.toString())) {
+ return attemptInfo;
+ }
+ }
+ return null;
+ }
+
+ /**
+ * Get last task attempt to finish
+ *
+ * @return TaskAttemptInfo
+ */
+ public final TaskAttemptInfo getLastTaskAttemptToFinish() {
+ List<TaskAttemptInfo> attemptsList = getTaskAttempts();
+ if (attemptsList.isEmpty()) {
+ return null;
+ }
+
+ return Ordering.from(new Comparator<TaskAttemptInfo>() {
+ @Override public int compare(TaskAttemptInfo o1, TaskAttemptInfo o2) {
+ return (o1.getFinishTimeInterval() < o2.getFinishTimeInterval()) ? -1 :
+ ((o1.getFinishTimeInterval() == o2.getFinishTimeInterval()) ?
+ 0 : 1);
+ }
+ }).max(attemptsList);
+ }
+
+ /**
+ * Get average task attempt duration. Includes succesful and failed tasks
+ *
+ * @return float
+ */
+ public final float getAvgTaskAttemptDuration() {
+ float totalTaskDuration = 0;
+ List<TaskAttemptInfo> attemptsList = getTaskAttempts();
+ if (attemptsList.size() == 0) {
+ return 0;
+ }
+ for (TaskAttemptInfo attemptInfo : attemptsList) {
+ totalTaskDuration += attemptInfo.getTimeTaken();
+ }
+ return ((totalTaskDuration * 1.0f) / attemptsList.size());
+ }
+
+ private Ordering<TaskAttemptInfo> orderingOnTimeTaken() {
+ return Ordering.from(new Comparator<TaskAttemptInfo>() {
+ @Override public int compare(TaskAttemptInfo o1, TaskAttemptInfo o2) {
+ return (o1.getTimeTaken() < o2.getTimeTaken()) ? -1 :
+ ((o1.getTimeTaken() == o2.getTimeTaken()) ?
+ 0 : 1);
+ }
+ });
+ }
+
+ private Ordering<TaskAttemptInfo> orderingOnAttemptStartTime() {
+ return Ordering.from(new Comparator<TaskAttemptInfo>() {
+ @Override public int compare(TaskAttemptInfo o1, TaskAttemptInfo o2) {
+ return (o1.getStartTimeInterval() < o2.getStartTimeInterval()) ? -1 :
+ ((o1.getStartTimeInterval() == o2.getStartTimeInterval()) ? 0 : 1);
+ }
+ });
+ }
+
+ /**
+ * Get min task attempt duration. This includes successful/failed task attempts as well
+ *
+ * @return long
+ */
+ public final long getMinTaskAttemptDuration() {
+ List<TaskAttemptInfo> attemptsList = getTaskAttempts();
+ if (attemptsList.isEmpty()) {
+ return 0;
+ }
+
+ return orderingOnTimeTaken().min(attemptsList).getTimeTaken();
+ }
+
+ /**
+ * Get max task attempt duration. This includes successful/failed task attempts as well
+ *
+ * @return long
+ */
+ public final long getMaxTaskAttemptDuration() {
+ List<TaskAttemptInfo> attemptsList = getTaskAttempts();
+ if (attemptsList.isEmpty()) {
+ return 0;
+ }
+
+ return orderingOnTimeTaken().max(attemptsList).getTimeTaken();
+ }
+
+ public final int getNumberOfTaskAttempts() {
+ return getTaskAttempts().size();
+ }
+
+ public final String getStatus() {
+ return status;
+ }
+
+ public final String getTaskId() {
+ return taskId;
+ }
+
+ public final long getTimeTaken() {
+ return getFinishTimeInterval() - getStartTimeInterval();
+ }
+
+ public final String getSuccessfulAttemptId() {
+ return successfulAttemptId;
+ }
+
+ public final long getAbsoluteScheduleTime() {
+ return scheduledTime;
+ }
+
+ public final long getScheduledTime() {
+ return scheduledTime - this.getVertexInfo().getDagInfo().getStartTime();
+ }
+
+ @Override
+ public String toString() {
+ StringBuilder sb = new StringBuilder();
+ sb.append("[");
+ sb.append("taskId=").append(getTaskId()).append(", ");
+ sb.append("scheduledTime=").append(getAbsoluteScheduleTime()).append(", ");
+ sb.append("startTime=").append(getStartTimeInterval()).append(", ");
+ sb.append("finishTime=").append(getFinishTimeInterval()).append(", ");
+ sb.append("timeTaken=").append(getTimeTaken()).append(", ");
+ sb.append("events=").append(getEvents()).append(", ");
+ sb.append("diagnostics=").append(getDiagnostics()).append(", ");
+ sb.append("successfulAttempId=").append(getSuccessfulAttemptId()).append(", ");
+ sb.append("status=").append(getStatus()).append(", ");
+ sb.append("vertexName=").append(getVertexInfo().getVertexName());
+ sb.append("]");
+ return sb.toString();
+ }
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/8c8db7c5/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/VersionInfo.java
----------------------------------------------------------------------
diff --git a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/VersionInfo.java b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/VersionInfo.java
new file mode 100644
index 0000000..97d18cd
--- /dev/null
+++ b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/VersionInfo.java
@@ -0,0 +1,45 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.history.parser.datamodel;
+
+public class VersionInfo {
+
+ private final String buildTime;
+ private final String revision;
+ private final String version;
+
+ public VersionInfo(String buildTime, String revision, String version) {
+ this.buildTime = buildTime;
+ this.revision = revision;
+ this.version = version;
+ }
+
+ public String getBuildTime() {
+ return buildTime;
+ }
+
+ public String getRevision() {
+ return revision;
+ }
+
+ public String getVersion() {
+ return version;
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/8c8db7c5/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/VertexInfo.java
----------------------------------------------------------------------
diff --git a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/VertexInfo.java b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/VertexInfo.java
new file mode 100644
index 0000000..50647fe
--- /dev/null
+++ b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/datamodel/VertexInfo.java
@@ -0,0 +1,636 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.history.parser.datamodel;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Predicate;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.LinkedHashMultimap;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Multimap;
+import com.google.common.collect.Multimaps;
+import com.google.common.collect.Ordering;
+
+import org.apache.hadoop.util.StringInterner;
+import org.apache.tez.dag.api.oldrecords.TaskState;
+import org.codehaus.jettison.json.JSONException;
+import org.codehaus.jettison.json.JSONObject;
+
+import javax.annotation.Nullable;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.hadoop.classification.InterfaceAudience.Public;
+import static org.apache.hadoop.classification.InterfaceStability.Evolving;
+
+@Public
+@Evolving
+public class VertexInfo extends BaseInfo {
+
+ private final String vertexId;
+ private final String vertexName;
+ private final long finishTime;
+ private final long initTime;
+ private final long initRequestedTime;
+ private final long startTime;
+ private final long startRequestedTime;
+
+ private final String diagnostics;
+ private final String processorClass;
+
+ private final int numTasks;
+ private final int failedTasks;
+ private final int completedTasks;
+ private final int succeededTasks;
+ private final int killedTasks;
+ private final int numFailedTaskAttempts;
+
+ private final String status;
+
+ //TaskID --> TaskInfo for internal reference
+ private Map<String, TaskInfo> taskInfoMap;
+
+ private final List<EdgeInfo> inEdgeList;
+ private final List<EdgeInfo> outEdgeList;
+
+ private final List<AdditionalInputOutputDetails> additionalInputInfoList;
+ private final List<AdditionalInputOutputDetails> additionalOutputInfoList;
+
+ private long avgPostDataExecutionTimeInterval = -1;
+
+ private DagInfo dagInfo;
+
+ VertexInfo(JSONObject jsonObject) throws JSONException {
+ super(jsonObject);
+
+ Preconditions.checkArgument(
+ jsonObject.getString(Constants.ENTITY_TYPE).equalsIgnoreCase
+ (Constants.TEZ_VERTEX_ID));
+
+ vertexId = StringInterner.weakIntern(jsonObject.optString(Constants.ENTITY));
+ taskInfoMap = Maps.newHashMap();
+
+ inEdgeList = Lists.newLinkedList();
+ outEdgeList = Lists.newLinkedList();
+ additionalInputInfoList = Lists.newLinkedList();
+ additionalOutputInfoList = Lists.newLinkedList();
+
+ //Parse additional Info
+ JSONObject otherInfoNode = jsonObject.getJSONObject(Constants.OTHER_INFO);
+ initRequestedTime = otherInfoNode.optLong(Constants.INIT_REQUESTED_TIME);
+ startRequestedTime = otherInfoNode.optLong(Constants.START_REQUESTED_TIME);
+ startTime = otherInfoNode.optLong(Constants.START_TIME);
+ initTime = otherInfoNode.optLong(Constants.INIT_TIME);
+ finishTime = otherInfoNode.optLong(Constants.FINISH_TIME);
+ diagnostics = otherInfoNode.optString(Constants.DIAGNOSTICS);
+ numTasks = otherInfoNode.optInt(Constants.NUM_TASKS);
+ failedTasks = otherInfoNode.optInt(Constants.NUM_FAILED_TASKS);
+ succeededTasks =
+ otherInfoNode.optInt(Constants.NUM_SUCCEEDED_TASKS);
+ completedTasks =
+ otherInfoNode.optInt(Constants.NUM_COMPLETED_TASKS);
+ killedTasks = otherInfoNode.optInt(Constants.NUM_KILLED_TASKS);
+ numFailedTaskAttempts =
+ otherInfoNode.optInt(Constants.NUM_FAILED_TASKS_ATTEMPTS);
+ vertexName = StringInterner.weakIntern(otherInfoNode.optString(Constants.VERTEX_NAME));
+ processorClass = StringInterner.weakIntern(otherInfoNode.optString(Constants.PROCESSOR_CLASS_NAME));
+ status = StringInterner.weakIntern(otherInfoNode.optString(Constants.STATUS));
+ }
+
+ public static VertexInfo create(JSONObject vertexInfoObject) throws
+ JSONException {
+ return new VertexInfo(vertexInfoObject);
+ }
+
+ /**
+ * Update edge details with source and destination vertex objects.
+ */
+ private void updateEdgeInfo() {
+ if (dagInfo.getNumVertices() == dagInfo.getVertices().size()) {
+ //We can update EdgeInfo when all vertices are parsed
+ Map<String, VertexInfo> vertexMapping = dagInfo.getVertexMapping();
+ for (EdgeInfo edge : dagInfo.getEdges()) {
+ VertexInfo sourceVertex = vertexMapping.get(edge.getInputVertexName());
+ VertexInfo destinationVertex = vertexMapping.get(edge.getOutputVertexName());
+ edge.setSourceVertex(sourceVertex);
+ edge.setDestinationVertex(destinationVertex);
+ }
+ }
+ }
+
+ void addTaskInfo(TaskInfo taskInfo) {
+ this.taskInfoMap.put(taskInfo.getTaskId(), taskInfo);
+ }
+
+ void setAdditionalInputInfoList(List<AdditionalInputOutputDetails> additionalInputInfoList) {
+ this.additionalInputInfoList.clear();
+ this.additionalInputInfoList.addAll(additionalInputInfoList);
+ }
+
+ void setAdditionalOutputInfoList(List<AdditionalInputOutputDetails> additionalOutputInfoList) {
+ this.additionalOutputInfoList.clear();
+ this.additionalOutputInfoList.addAll(additionalOutputInfoList);
+ }
+
+ void addInEdge(EdgeInfo edgeInfo) {
+ this.inEdgeList.add(edgeInfo);
+ }
+
+ void addOutEdge(EdgeInfo edgeInfo) {
+ this.outEdgeList.add(edgeInfo);
+ }
+
+ void setDagInfo(DagInfo dagInfo) {
+ Preconditions.checkArgument(dagInfo != null, "Provide valid dagInfo");
+ this.dagInfo = dagInfo;
+ //link vertex to dagInfo
+ dagInfo.addVertexInfo(this);
+ updateEdgeInfo();
+ }
+
+ public List<AdditionalInputOutputDetails> getAdditionalInputInfoList() {
+ return Collections.unmodifiableList(additionalInputInfoList);
+ }
+
+ public List<AdditionalInputOutputDetails> getAdditionalOutputInfoList() {
+ return Collections.unmodifiableList(additionalOutputInfoList);
+ }
+
+ @Override
+ public final long getStartTimeInterval() {
+ return startTime - (dagInfo.getStartTime());
+ }
+
+ public final long getFirstTaskStartTimeInterval() {
+ TaskInfo firstTask = getFirstTaskToStart();
+ if (firstTask == null) {
+ return 0;
+ }
+ return firstTask.getStartTimeInterval();
+ }
+
+ public final long getLastTaskFinishTimeInterval() {
+ if (getLastTaskToFinish() == null || getLastTaskToFinish().getFinishTimeInterval() < 0) {
+ return dagInfo.getFinishTimeInterval();
+ }
+ return getLastTaskToFinish().getFinishTimeInterval();
+ }
+
+ public final long getAvgPostDataExecutionTimeInterval() {
+ if (avgPostDataExecutionTimeInterval == -1) {
+ long totalExecutionTime = 0;
+ long totalAttempts = 0;
+ for (TaskInfo task : getTasks()) {
+ TaskAttemptInfo attempt = task.getSuccessfulTaskAttempt();
+ if (attempt != null) {
+ // count only time after last data was received
+ long execTime = attempt.getPostDataExecutionTimeInterval();
+ if (execTime >= 0) {
+ totalExecutionTime += execTime;
+ totalAttempts++;
+ }
+ }
+ }
+ if (totalAttempts > 0) {
+ avgPostDataExecutionTimeInterval = Math.round(totalExecutionTime*1.0/totalAttempts);
+ }
+ }
+ return avgPostDataExecutionTimeInterval;
+ }
+
+ public final long getStartTime() {
+ return startTime;
+ }
+
+ public final long getFinishTime() {
+ return finishTime;
+ }
+
+ public final long getInitTime() {
+ return initTime;
+ }
+
+ public final long getInitRequestedTime() {
+ return initRequestedTime;
+ }
+
+ public final long getStartRequestedTime() {
+ return startRequestedTime;
+ }
+
+ @Override
+ public final long getFinishTimeInterval() {
+ long vertexEndTime = finishTime - (dagInfo.getStartTime());
+ if (vertexEndTime < 0) {
+ //probably vertex is not complete or failed in middle. get the last task attempt time
+ for (TaskInfo taskInfo : getTasks()) {
+ vertexEndTime = (taskInfo.getFinishTimeInterval() > vertexEndTime)
+ ? taskInfo.getFinishTimeInterval() : vertexEndTime;
+ }
+ }
+ return vertexEndTime;
+ }
+
+ @Override
+ public final String getDiagnostics() {
+ return diagnostics;
+ }
+
+ public final String getVertexName() {
+ return vertexName;
+ }
+
+ public final String getVertexId() {
+ return vertexId;
+ }
+
+ //Quite possible that getFinishTime is not yet recorded for failed vertices (or killed vertices)
+ //Start time of vertex infers that the dependencies are done and AM has inited it.
+ public final long getTimeTaken() {
+ return (getFinishTimeInterval() - getStartTimeInterval());
+ }
+
+ //Time taken for last task to finish - time taken for first task to start
+ public final long getTimeTakenForTasks() {
+ return (getLastTaskFinishTimeInterval() - getFirstTaskStartTimeInterval());
+ }
+
+ public final long getInitTimeInterval() {
+ return initTime - dagInfo.getStartTime();
+ }
+
+ public final int getNumTasks() {
+ return numTasks;
+ }
+
+ public final int getFailedTasksCount() {
+ return failedTasks;
+ }
+
+ public final int getKilledTasksCount() {
+ return killedTasks;
+ }
+
+ public final int getCompletedTasksCount() {
+ return completedTasks;
+ }
+
+ public final int getSucceededTasksCount() {
+ return succeededTasks;
+ }
+
+ public final int getNumFailedTaskAttemptsCount() {
+ return numFailedTaskAttempts;
+ }
+
+ public final String getProcessorClassName() {
+ return processorClass;
+
+ }
+
+
+ private List<TaskInfo> getTasksInternal() {
+ return Lists.newLinkedList(taskInfoMap.values());
+ }
+
+ /**
+ * Get all tasks
+ *
+ * @return list of taskInfo
+ */
+ public final List<TaskInfo> getTasks() {
+ return Collections.unmodifiableList(getTasksInternal());
+ }
+
+ /**
+ * Get all tasks in sorted order
+ *
+ * @param sorted
+ * @param ordering
+ * @return list of TaskInfo
+ */
+ public final List<TaskInfo> getTasks(boolean sorted, @Nullable Ordering<TaskInfo> ordering) {
+ List<TaskInfo> taskInfoList = getTasksInternal();
+ if (sorted) {
+ Collections.sort(taskInfoList, ((ordering == null) ? orderingOnStartTime() : ordering));
+ }
+ return Collections.unmodifiableList(taskInfoList);
+ }
+
+ /**
+ * Get list of failed tasks
+ *
+ * @return List<TaskAttemptInfo>
+ */
+ public final List<TaskInfo> getFailedTasks() {
+ return getTasks(TaskState.FAILED);
+ }
+
+ /**
+ * Get list of killed tasks
+ *
+ * @return List<TaskAttemptInfo>
+ */
+ public final List<TaskInfo> getKilledTasks() {
+ return getTasks(TaskState.KILLED);
+ }
+
+ /**
+ * Get list of failed tasks
+ *
+ * @return List<TaskAttemptInfo>
+ */
+ public final List<TaskInfo> getSuccessfulTasks() {
+ return getTasks(TaskState.SUCCEEDED);
+ }
+
+ /**
+ * Get list of tasks belonging to a specific state
+ *
+ * @param state
+ * @return List<TaskAttemptInfo>
+ */
+ public final List<TaskInfo> getTasks(final TaskState state) {
+ return Collections.unmodifiableList(Lists.newLinkedList(Iterables.filter(Lists.newLinkedList
+ (taskInfoMap.values()), new Predicate<TaskInfo>() {
+ @Override public boolean apply(TaskInfo input) {
+ return input.getStatus() != null && input.getStatus().equals(state.toString());
+ }
+ }
+ )
+ )
+ );
+ }
+
+ /**
+ * Get source vertices for this vertex
+ *
+ * @return List<VertexInfo> list of incoming vertices to this vertex
+ */
+ public final List<VertexInfo> getInputVertices() {
+ List<VertexInfo> inputVertices = Lists.newLinkedList();
+ for (EdgeInfo edge : inEdgeList) {
+ inputVertices.add(edge.getSourceVertex());
+ }
+ return Collections.unmodifiableList(inputVertices);
+ }
+
+ /**
+ * Get destination vertices for this vertex
+ *
+ * @return List<VertexInfo> list of output vertices
+ */
+ public final List<VertexInfo> getOutputVertices() {
+ List<VertexInfo> outputVertices = Lists.newLinkedList();
+ for (EdgeInfo edge : outEdgeList) {
+ outputVertices.add(edge.getDestinationVertex());
+ }
+ return Collections.unmodifiableList(outputVertices);
+ }
+
+ // expensive method to call for large DAGs as it creates big lists on every call
+ private List<TaskAttemptInfo> getTaskAttemptsInternal() {
+ List<TaskAttemptInfo> taskAttemptInfos = Lists.newLinkedList();
+ for (TaskInfo taskInfo : getTasks()) {
+ taskAttemptInfos.addAll(taskInfo.getTaskAttempts());
+ }
+ return taskAttemptInfos;
+ }
+
+ /**
+ * Get all task attempts
+ *
+ * @return List<TaskAttemptInfo> list of attempts
+ */
+ public List<TaskAttemptInfo> getTaskAttempts() {
+ return Collections.unmodifiableList(getTaskAttemptsInternal());
+ }
+
+ /**
+ * Get all task attempts in sorted order
+ *
+ * @param sorted
+ * @param ordering
+ * @return list of TaskAttemptInfo
+ */
+ public final List<TaskAttemptInfo> getTaskAttempts(boolean sorted,
+ @Nullable Ordering<TaskAttemptInfo> ordering) {
+ List<TaskAttemptInfo> taskAttemptInfos = getTaskAttemptsInternal();
+ if (sorted) {
+ Collections.sort(taskAttemptInfos, ((ordering == null) ? orderingOnAttemptStartTime() : ordering));
+ }
+ return Collections.unmodifiableList(taskAttemptInfos);
+ }
+
+ public final TaskInfo getTask(String taskId) {
+ return taskInfoMap.get(taskId);
+ }
+
+ /**
+ * Get incoming edge information for a specific vertex
+ *
+ * @return List<EdgeInfo> list of input edges on this vertex
+ */
+ public final List<EdgeInfo> getInputEdges() {
+ return Collections.unmodifiableList(inEdgeList);
+ }
+
+ /**
+ * Get outgoing edge information for a specific vertex
+ *
+ * @return List<EdgeInfo> list of output edges on this vertex
+ */
+ public final List<EdgeInfo> getOutputEdges() {
+ return Collections.unmodifiableList(outEdgeList);
+ }
+
+ public final Multimap<Container, TaskAttemptInfo> getContainersMapping() {
+ Multimap<Container, TaskAttemptInfo> containerMapping = LinkedHashMultimap.create();
+ for (TaskAttemptInfo attemptInfo : getTaskAttempts()) {
+ containerMapping.put(attemptInfo.getContainer(), attemptInfo);
+ }
+ return Multimaps.unmodifiableMultimap(containerMapping);
+ }
+
+ /**
+ * Get first task to start
+ *
+ * @return TaskInfo
+ */
+ public final TaskInfo getFirstTaskToStart() {
+ List<TaskInfo> taskInfoList = Lists.newLinkedList(taskInfoMap.values());
+ if (taskInfoList.size() == 0) {
+ return null;
+ }
+ Collections.sort(taskInfoList, new Comparator<TaskInfo>() {
+ @Override public int compare(TaskInfo o1, TaskInfo o2) {
+ return (o1.getStartTimeInterval() < o2.getStartTimeInterval()) ? -1 :
+ ((o1.getStartTimeInterval() == o2.getStartTimeInterval()) ?
+ 0 : 1);
+ }
+ });
+ return taskInfoList.get(0);
+ }
+
+ /**
+ * Get last task to finish
+ *
+ * @return TaskInfo
+ */
+ public final TaskInfo getLastTaskToFinish() {
+ List<TaskInfo> taskInfoList = Lists.newLinkedList(taskInfoMap.values());
+ if (taskInfoList.size() == 0) {
+ return null;
+ }
+ Collections.sort(taskInfoList, new Comparator<TaskInfo>() {
+ @Override public int compare(TaskInfo o1, TaskInfo o2) {
+ return (o1.getFinishTimeInterval() > o2.getFinishTimeInterval()) ? -1 :
+ ((o1.getStartTimeInterval() == o2.getStartTimeInterval()) ?
+ 0 : 1);
+ }
+ });
+ return taskInfoList.get(0);
+ }
+
+ /**
+ * Get average task duration
+ *
+ * @return long
+ */
+ public final float getAvgTaskDuration() {
+ float totalTaskDuration = 0;
+ List<TaskInfo> tasksList = getTasks();
+ if (tasksList.size() == 0) {
+ return 0;
+ }
+ for (TaskInfo taskInfo : tasksList) {
+ totalTaskDuration += taskInfo.getTimeTaken();
+ }
+ return ((totalTaskDuration * 1.0f) / tasksList.size());
+ }
+
+ /**
+ * Get min task duration in vertex
+ *
+ * @return long
+ */
+ public final long getMinTaskDuration() {
+ TaskInfo taskInfo = getMinTaskDurationTask();
+ return (taskInfo != null) ? taskInfo.getTimeTaken() : 0;
+ }
+
+ /**
+ * Get max task duration in vertex
+ *
+ * @return long
+ */
+ public final long getMaxTaskDuration() {
+ TaskInfo taskInfo = getMaxTaskDurationTask();
+ return (taskInfo != null) ? taskInfo.getTimeTaken() : 0;
+ }
+
+ private Ordering<TaskInfo> orderingOnTimeTaken() {
+ return Ordering.from(new Comparator<TaskInfo>() {
+ @Override public int compare(TaskInfo o1, TaskInfo o2) {
+ return (o1.getTimeTaken() < o2.getTimeTaken()) ? -1 :
+ ((o1.getTimeTaken() == o2.getTimeTaken()) ? 0 : 1);
+ }
+ });
+ }
+
+ private Ordering<TaskInfo> orderingOnStartTime() {
+ return Ordering.from(new Comparator<TaskInfo>() {
+ @Override public int compare(TaskInfo o1, TaskInfo o2) {
+ return (o1.getStartTimeInterval() < o2.getStartTimeInterval()) ? -1 :
+ ((o1.getStartTimeInterval() == o2.getStartTimeInterval()) ? 0 : 1);
+ }
+ });
+ }
+
+ private Ordering<TaskAttemptInfo> orderingOnAttemptStartTime() {
+ return Ordering.from(new Comparator<TaskAttemptInfo>() {
+ @Override public int compare(TaskAttemptInfo o1, TaskAttemptInfo o2) {
+ return (o1.getStartTimeInterval() < o2.getStartTimeInterval()) ? -1 :
+ ((o1.getStartTimeInterval() == o2.getStartTimeInterval()) ? 0 : 1);
+ }
+ });
+ }
+
+ /**
+ * Get min task duration in vertex
+ *
+ * @return TaskInfo
+ */
+ public final TaskInfo getMinTaskDurationTask() {
+ List<TaskInfo> taskInfoList = getTasks();
+ if (taskInfoList.size() == 0) {
+ return null;
+ }
+
+ return orderingOnTimeTaken().min(taskInfoList);
+ }
+
+ /**
+ * Get max task duration in vertex
+ *
+ * @return TaskInfo
+ */
+ public final TaskInfo getMaxTaskDurationTask() {
+ List<TaskInfo> taskInfoList = getTasks();
+ if (taskInfoList.size() == 0) {
+ return null;
+ }
+ return orderingOnTimeTaken().max(taskInfoList);
+ }
+
+ public final String getStatus() {
+ return status;
+ }
+
+ public final DagInfo getDagInfo() {
+ return dagInfo;
+ }
+
+ @Override
+ public String toString() {
+ StringBuilder sb = new StringBuilder();
+ sb.append("[");
+ sb.append("vertexName=").append(getVertexName()).append(", ");
+ sb.append("events=").append(getEvents()).append(", ");
+ sb.append("initTime=").append(getInitTimeInterval()).append(", ");
+ sb.append("startTime=").append(getStartTimeInterval()).append(", ");
+ sb.append("endTime=").append(getFinishTimeInterval()).append(", ");
+ sb.append("timeTaken=").append(getTimeTaken()).append(", ");
+ sb.append("diagnostics=").append(getDiagnostics()).append(", ");
+ sb.append("numTasks=").append(getNumTasks()).append(", ");
+ sb.append("processorClassName=").append(getProcessorClassName()).append(", ");
+ sb.append("numCompletedTasks=").append(getCompletedTasksCount()).append(", ");
+ sb.append("numFailedTaskAttempts=").append(getNumFailedTaskAttemptsCount()).append(", ");
+ sb.append("numSucceededTasks=").append(getSucceededTasksCount()).append(", ");
+ sb.append("numFailedTasks=").append(getFailedTasks()).append(", ");
+ sb.append("numKilledTasks=").append(getKilledTasks()).append(", ");
+ sb.append("tasksCount=").append(taskInfoMap.size()).append(", ");
+ sb.append("status=").append(getStatus());
+ sb.append("]");
+ return sb.toString();
+ }
+}
http://git-wip-us.apache.org/repos/asf/tez/blob/8c8db7c5/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/utils/Utils.java
----------------------------------------------------------------------
diff --git a/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/utils/Utils.java b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/utils/Utils.java
new file mode 100644
index 0000000..ffb854a
--- /dev/null
+++ b/tez-plugins/tez-history-parser/src/main/java/org/apache/tez/history/parser/utils/Utils.java
@@ -0,0 +1,139 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.tez.history.parser.utils;
+
+import com.google.common.base.Strings;
+import com.google.common.collect.Lists;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.util.StringInterner;
+import org.apache.log4j.ConsoleAppender;
+import org.apache.log4j.Level;
+import org.apache.log4j.Logger;
+import org.apache.log4j.PatternLayout;
+import org.apache.tez.common.counters.CounterGroup;
+import org.apache.tez.common.counters.TezCounter;
+import org.apache.tez.common.counters.TezCounters;
+import org.apache.tez.dag.history.logging.EntityTypes;
+import org.apache.tez.history.parser.datamodel.Constants;
+import org.apache.tez.history.parser.datamodel.Event;
+import org.apache.tez.history.parser.datamodel.TaskAttemptInfo.DataDependencyEvent;
+import org.codehaus.jettison.json.JSONArray;
+import org.codehaus.jettison.json.JSONException;
+import org.codehaus.jettison.json.JSONObject;
+
+import java.util.List;
+
+@InterfaceAudience.Private
+public class Utils {
+
+ private static final String LOG4J_CONFIGURATION = "log4j.configuration";
+
+ /**
+ * Parse tez counters from json
+ *
+ * @param jsonObject
+ * @return TezCounters
+ * @throws JSONException
+ */
+ public static TezCounters parseTezCountersFromJSON(JSONObject jsonObject)
+ throws JSONException {
+ TezCounters counters = new TezCounters();
+
+ if (jsonObject == null) {
+ return counters; //empty counters.
+ }
+
+ final JSONArray counterGroupNodes = jsonObject.optJSONArray(Constants.COUNTER_GROUPS);
+ if (counterGroupNodes != null) {
+ for (int i = 0; i < counterGroupNodes.length(); i++) {
+ JSONObject counterGroupNode = counterGroupNodes.optJSONObject(i);
+ final String groupName = counterGroupNode.optString(Constants.COUNTER_GROUP_NAME);
+ final String groupDisplayName = counterGroupNode.optString(
+ Constants.COUNTER_GROUP_DISPLAY_NAME, groupName);
+
+ CounterGroup group = counters.addGroup(groupName, groupDisplayName);
+
+ final JSONArray counterNodes = counterGroupNode.optJSONArray(Constants.COUNTERS);
+
+ //Parse counter nodes
+ for (int j = 0; j < counterNodes.length(); j++) {
+ JSONObject counterNode = counterNodes.optJSONObject(j);
+ final String counterName = counterNode.getString(Constants.COUNTER_NAME);
+ final String counterDisplayName =
+ counterNode.optString(Constants.COUNTER_DISPLAY_NAME, counterName);
+ final long counterValue = counterNode.getLong(Constants.COUNTER_VALUE);
+ TezCounter counter = group.findCounter(
+ counterName,
+ counterDisplayName);
+ counter.setValue(counterValue);
+ }
+ }
+ }
+ return counters;
+ }
+
+ public static List<DataDependencyEvent> parseDataEventDependencyFromJSON(JSONObject jsonObject)
+ throws JSONException {
+ List<DataDependencyEvent> events = Lists.newArrayList();
+ JSONArray fields = jsonObject.optJSONArray(Constants.LAST_DATA_EVENTS);
+ for (int i=0; i<fields.length(); i++) {
+ JSONObject eventMap = fields.getJSONObject(i);
+ events.add(new DataDependencyEvent(
+ StringInterner.weakIntern(eventMap.optString(EntityTypes.TEZ_TASK_ATTEMPT_ID.name())),
+ eventMap.optLong(Constants.TIMESTAMP)));
+ }
+ return events;
+ }
+
+ /**
+ * Parse events from json
+ *
+ * @param eventNodes
+ * @param eventList
+ * @throws JSONException
+ */
+ public static void parseEvents(JSONArray eventNodes, List<Event> eventList) throws
+ JSONException {
+ if (eventNodes == null) {
+ return;
+ }
+ for (int i = 0; i < eventNodes.length(); i++) {
+ JSONObject eventNode = eventNodes.optJSONObject(i);
+ final String eventInfo = eventNode.optString(Constants.EVENT_INFO);
+ final String eventType = eventNode.optString(Constants.EVENT_TYPE);
+ final long time = eventNode.optLong(Constants.EVENT_TIME_STAMP);
+
+ Event event = new Event(eventInfo, eventType, time);
+
+ eventList.add(event);
+
+ }
+ }
+
+ public static void setupRootLogger() {
+ if (Strings.isNullOrEmpty(System.getProperty(LOG4J_CONFIGURATION))) {
+ //By default print to console with INFO level
+ Logger.getRootLogger().
+ addAppender(new ConsoleAppender(new PatternLayout(PatternLayout.TTCC_CONVERSION_PATTERN)));
+ Logger.getRootLogger().setLevel(Level.INFO);
+ }
+ }
+
+}