OOZIE-2406 Completely rewrite GraphGenerator code (andras.piros via gezapeti)


Project: http://git-wip-us.apache.org/repos/asf/oozie/repo
Commit: http://git-wip-us.apache.org/repos/asf/oozie/commit/883c7556
Tree: http://git-wip-us.apache.org/repos/asf/oozie/tree/883c7556
Diff: http://git-wip-us.apache.org/repos/asf/oozie/diff/883c7556

Branch: refs/heads/master
Commit: 883c7556d2f302261eae5aec6e323a5b1de04f74
Parents: 6b6f42c
Author: Gezapeti Cseh <gezap...@apache.org>
Authored: Fri Oct 13 15:14:33 2017 +0200
Committer: Gezapeti Cseh <gezap...@apache.org>
Committed: Fri Oct 13 15:14:43 2017 +0200

----------------------------------------------------------------------
 .../apache/oozie/client/rest/RestConstants.java |   4 +
 core/pom.xml                                    |  26 +-
 .../apache/oozie/servlet/JsonRestServlet.java   |   4 +-
 .../org/apache/oozie/servlet/V1JobServlet.java  |  82 +-
 .../org/apache/oozie/util/GraphGenerator.java   | 742 -------------------
 .../apache/oozie/util/graph/GraphGenerator.java | 102 +++
 .../apache/oozie/util/graph/GraphRenderer.java  |  35 +
 .../oozie/util/graph/GraphvizRenderer.java      | 242 ++++++
 .../apache/oozie/util/graph/OutputFormat.java   |  23 +
 .../oozie/util/graph/WorkflowActionNode.java    | 149 ++++
 .../oozie/util/graph/WorkflowGraphHandler.java  | 257 +++++++
 core/src/main/resources/oozie-default.xml       |   9 +
 .../apache/oozie/servlet/TestV1JobServlet.java  |  37 +-
 .../apache/oozie/util/TestGraphGenerator.java   | 105 ---
 .../oozie/util/graph/TestGraphGenerator.java    | 227 ++++++
 .../graph-workflow-decision-fork-join.xml       |  64 ++
 .../test/resources/graph-workflow-invalid.xml   | 507 +++++++++++++
 .../resources/graph-workflow-many-actions.xml   | 267 +++++++
 .../test/resources/graph-workflow-simple.xml    | 115 +++
 core/src/test/resources/graphWF.xml             | 507 -------------
 core/src/test/resources/graphWF_26_actions.xml  | 264 -------
 core/src/test/resources/invalidGraphWF.xml      | 507 -------------
 docs/src/site/twiki/WebServicesAPI.twiki        |  41 +-
 pom.xml                                         |  37 +-
 release-log.txt                                 |   1 +
 webapp/src/main/webapp/oozie-console.js         |   2 +-
 26 files changed, 2154 insertions(+), 2202 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/oozie/blob/883c7556/client/src/main/java/org/apache/oozie/client/rest/RestConstants.java
----------------------------------------------------------------------
diff --git 
a/client/src/main/java/org/apache/oozie/client/rest/RestConstants.java 
b/client/src/main/java/org/apache/oozie/client/rest/RestConstants.java
index 74843dc..84998f6 100644
--- a/client/src/main/java/org/apache/oozie/client/rest/RestConstants.java
+++ b/client/src/main/java/org/apache/oozie/client/rest/RestConstants.java
@@ -41,6 +41,8 @@ public interface RestConstants {
 
     String PNG_IMAGE_CONTENT_TYPE = "image/png";
 
+    String SVG_IMAGE_CONTENT_TYPE = "image/svg+xml";
+
     String ACTION_PARAM = "action";
 
     String OFFSET_PARAM = "offset";
@@ -101,6 +103,8 @@ public interface RestConstants {
 
     String JOB_SHOW_KILL_PARAM = "show-kill";
 
+    String JOB_FORMAT_PARAM = "format";
+
     String JOB_SHOW_STATUS = "status";
 
     String JOB_SHOW_WF_ACTIONS_IN_COORD = "wf-actions";

http://git-wip-us.apache.org/repos/asf/oozie/blob/883c7556/core/pom.xml
----------------------------------------------------------------------
diff --git a/core/pom.xml b/core/pom.xml
index 6f9adb6..9f0689e 100644
--- a/core/pom.xml
+++ b/core/pom.xml
@@ -469,30 +469,10 @@
 
         <!-- For drawing runtime DAG -->
         <dependency>
-            <groupId>net.sf.jung</groupId>
-            <artifactId>jung-graph-impl</artifactId>
-            <scope>compile</scope>
-        </dependency>
-        <dependency>
-            <groupId>net.sf.jung</groupId>
-            <artifactId>jung-api</artifactId>
-            <scope>compile</scope>
-        </dependency>
-        <dependency>
-            <groupId>net.sf.jung</groupId>
-            <artifactId>jung-visualization</artifactId>
-            <scope>compile</scope>
-        </dependency>
-        <dependency>
-            <groupId>net.sf.jung</groupId>
-            <artifactId>jung-algorithms</artifactId>
-            <scope>compile</scope>
-        </dependency>
-        <dependency>
-            <groupId>net.sourceforge.collections</groupId>
-            <artifactId>collections-generic</artifactId>
-            <scope>compile</scope>
+            <groupId>guru.nidi</groupId>
+            <artifactId>graphviz-java</artifactId>
         </dependency>
+
         <dependency>
             <groupId>org.eclipse.jgit</groupId>
             <artifactId>org.eclipse.jgit</artifactId>

http://git-wip-us.apache.org/repos/asf/oozie/blob/883c7556/core/src/main/java/org/apache/oozie/servlet/JsonRestServlet.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/oozie/servlet/JsonRestServlet.java 
b/core/src/main/java/org/apache/oozie/servlet/JsonRestServlet.java
index 059d3cf..10307d1 100644
--- a/core/src/main/java/org/apache/oozie/servlet/JsonRestServlet.java
+++ b/core/src/main/java/org/apache/oozie/servlet/JsonRestServlet.java
@@ -189,7 +189,7 @@ public abstract class JsonRestServlet extends HttpServlet {
      * @param name name of the timer for the cron.
      * @param cron cron to add to a instrumentation timer.
      */
-    private void addCron(String name, Instrumentation.Cron cron) {
+    protected void addCron(String name, Instrumentation.Cron cron) {
         instrumentation.addCron(INSTRUMENTATION_GROUP, name, cron);
     }
 
@@ -228,7 +228,7 @@ public abstract class JsonRestServlet extends HttpServlet {
      * @param name counter name.
      * @param count count to increment the counter.
      */
-    private void incrCounter(String name, int count) {
+    protected void incrCounter(String name, int count) {
         if (instrumentation != null) {
             instrumentation.incr(INSTRUMENTATION_GROUP, name, count);
         }

http://git-wip-us.apache.org/repos/asf/oozie/blob/883c7556/core/src/main/java/org/apache/oozie/servlet/V1JobServlet.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/oozie/servlet/V1JobServlet.java 
b/core/src/main/java/org/apache/oozie/servlet/V1JobServlet.java
index 10812c6..0d275fd 100644
--- a/core/src/main/java/org/apache/oozie/servlet/V1JobServlet.java
+++ b/core/src/main/java/org/apache/oozie/servlet/V1JobServlet.java
@@ -20,11 +20,13 @@ package org.apache.oozie.servlet;
 
 import java.io.IOException;
 import java.util.List;
+import java.util.Locale;
 
 import javax.servlet.ServletInputStream;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 
+import com.google.common.base.Strings;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.oozie.*;
 import org.apache.oozie.client.WorkflowAction;
@@ -38,8 +40,12 @@ import org.apache.oozie.service.CoordinatorEngineService;
 import org.apache.oozie.service.DagEngineService;
 import org.apache.oozie.service.Services;
 import org.apache.oozie.service.UUIDService;
-import org.apache.oozie.util.GraphGenerator;
+import org.apache.oozie.util.Instrumentation;
+import org.apache.oozie.util.graph.GraphGenerator;
 import org.apache.oozie.util.XLog;
+import org.apache.oozie.util.graph.GraphRenderer;
+import org.apache.oozie.util.graph.GraphvizRenderer;
+import org.apache.oozie.util.graph.OutputFormat;
 import org.json.simple.JSONArray;
 import org.json.simple.JSONObject;
 
@@ -299,19 +305,32 @@ public class V1JobServlet extends BaseJobServlet {
         String jobId = getResourceName(request);
         if (jobId.endsWith("-W")) {
             try {
-                // Applicable only to worflow, for now
-                response.setContentType(RestConstants.PNG_IMAGE_CONTENT_TYPE);
 
-                String showKill = 
request.getParameter(RestConstants.JOB_SHOW_KILL_PARAM);
-                boolean sK = showKill != null && 
(showKill.equalsIgnoreCase("yes") || showKill.equals("1") || 
showKill.equalsIgnoreCase("true"));
+                final String showKillParameter = 
request.getParameter(RestConstants.JOB_SHOW_KILL_PARAM);
+                final boolean showKill = isShowKillSet(showKillParameter);
+
+                final String formatParameter = 
request.getParameter(RestConstants.JOB_FORMAT_PARAM);
+                final OutputFormat outputFormat = 
getOutputFormat(formatParameter);
+
+                final String contentType = getContentType(outputFormat);
+
+                response.setContentType(contentType);
+
+                final Instrumentation.Cron cron = new Instrumentation.Cron();
+                cron.start();
+
+                final GraphRenderer graphRenderer = new GraphvizRenderer();
 
                 new GraphGenerator(
-                        getWorkflowJobDefinition(request, response),
-                        (WorkflowJobBean)getWorkflowJob(request, response),
-                        sK).write(response.getOutputStream());
+                            getWorkflowJobDefinition(request, response),
+                            (WorkflowJobBean)getWorkflowJob(request, response),
+                            showKill,
+                            graphRenderer).write(response.getOutputStream(), 
outputFormat);
 
+                cron.stop();
+                instrument(outputFormat, cron);
             }
-            catch (Exception e) {
+            catch (final Exception e) {
                 throw new XServletException(HttpServletResponse.SC_NOT_FOUND, 
ErrorCode.E0307, e.getMessage(), e);
             }
         }
@@ -320,6 +339,51 @@ public class V1JobServlet extends BaseJobServlet {
         }
     }
 
+    private boolean isShowKillSet(final String showKillParameter) {
+        return showKillParameter != null &&
+                (showKillParameter.equalsIgnoreCase("yes") ||
+                        showKillParameter.equals("1") ||
+                        showKillParameter.equalsIgnoreCase("true"));
+    }
+
+    private OutputFormat getOutputFormat(final String formatParameter) {
+        final OutputFormat outputFormat;
+        if (Strings.isNullOrEmpty(formatParameter)) {
+            outputFormat = OutputFormat.PNG;
+        }
+        else {
+            outputFormat = 
OutputFormat.valueOf(formatParameter.toUpperCase(Locale.getDefault()));
+        }
+        return outputFormat;
+    }
+
+    private String getContentType(final OutputFormat outputFormat) {
+        final String contentType;
+
+        switch (outputFormat) {
+            case PNG:
+                contentType = RestConstants.PNG_IMAGE_CONTENT_TYPE;
+                break;
+            case DOT:
+                contentType = RestConstants.TEXT_CONTENT_TYPE;
+                break;
+            case SVG:
+                contentType = RestConstants.SVG_IMAGE_CONTENT_TYPE;
+                break;
+            default:
+                throw new IllegalArgumentException("Unknown output format, 
cannot get content type: " + outputFormat);
+        }
+
+        return contentType;
+    }
+
+    private void instrument(final OutputFormat outputFormat, final 
Instrumentation.Cron cron) {
+        addCron(INSTRUMENTATION_NAME + "-graph", cron);
+        incrCounter(INSTRUMENTATION_NAME + "-graph", 1);
+        addCron(INSTRUMENTATION_NAME + "-graph-" + 
outputFormat.toString().toLowerCase(Locale.getDefault()), cron);
+        incrCounter(INSTRUMENTATION_NAME + "-graph-" + 
outputFormat.toString().toLowerCase(Locale.getDefault()), 1);
+    }
+
     /**
      * Start wf job
      *

http://git-wip-us.apache.org/repos/asf/oozie/blob/883c7556/core/src/main/java/org/apache/oozie/util/GraphGenerator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/oozie/util/GraphGenerator.java 
b/core/src/main/java/org/apache/oozie/util/GraphGenerator.java
deleted file mode 100644
index 6ded2c6..0000000
--- a/core/src/main/java/org/apache/oozie/util/GraphGenerator.java
+++ /dev/null
@@ -1,742 +0,0 @@
-/**
- * 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.oozie.util;
-
-import edu.uci.ics.jung.algorithms.layout.StaticLayout;
-import edu.uci.ics.jung.graph.DirectedSparseGraph;
-import edu.uci.ics.jung.graph.Graph;
-import edu.uci.ics.jung.graph.util.Context;
-import edu.uci.ics.jung.visualization.VisualizationImageServer;
-import edu.uci.ics.jung.visualization.renderers.Renderer;
-import edu.uci.ics.jung.visualization.util.ArrowFactory;
-import org.apache.commons.collections15.Transformer;
-import org.apache.oozie.WorkflowJobBean;
-import org.apache.oozie.client.WorkflowAction;
-import org.apache.oozie.client.WorkflowAction.Status;
-import org.apache.oozie.client.WorkflowJob;
-import org.xml.sax.Attributes;
-import org.xml.sax.InputSource;
-import org.xml.sax.SAXException;
-import org.xml.sax.XMLReader;
-import org.xml.sax.helpers.DefaultHandler;
-
-import javax.imageio.ImageIO;
-import javax.xml.parsers.SAXParser;
-import javax.xml.parsers.SAXParserFactory;
-import java.awt.*;
-import java.awt.geom.Ellipse2D;
-import java.awt.geom.Point2D;
-import java.awt.image.BufferedImage;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.io.StringReader;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.LinkedHashMap;
-import java.util.Map;
-
-/**
- * Class to generate and plot runtime workflow DAG
- */
-public class GraphGenerator {
-
-    private String xml;
-    private WorkflowJobBean job;
-    private boolean showKill = false;
-    private final int actionsLimit = 25;
-
-    /**
-     * C'tor
-     * @param xml The workflow definition XML
-     * @param job Current status of the job
-     * @param showKill Flag to whether show 'kill' node
-     */
-    public GraphGenerator(String xml, WorkflowJobBean job, boolean showKill) {
-        if(job == null) {
-            throw new IllegalArgumentException("JsonWorkflowJob can't be 
null");
-        }
-        this.xml = xml;
-        this.job = job;
-        this.showKill = showKill;
-    }
-
-    /**
-     * C'tor
-     * @param xml
-     * @param job
-     */
-    public GraphGenerator(String xml, WorkflowJobBean job) {
-        this(xml, job, false);
-    }
-
-    /**
-     * Overridden to thwart finalizer attack
-     */
-    @Override
-    public final void finalize() {
-        // No-op; just to avoid finalizer attack
-        // as the constructor is throwing an exception
-    }
-
-    /**
-     * Stream the PNG file to client
-     * @param out
-     * @throws Exception
-     */
-    public void write(OutputStream out) throws Exception {
-        SAXParserFactory spf = SAXParserFactory.newInstance();
-        
spf.setFeature("http://xml.org/sax/features/external-general-entities";, false);
-        
spf.setFeature("http://xml.org/sax/features/external-parameter-entities";, 
false);
-        spf.setFeature("http://apache.org/xml/features/disallow-doctype-decl";, 
true);
-        spf.setNamespaceAware(true);
-        SAXParser saxParser = spf.newSAXParser();
-        XMLReader xmlReader = saxParser.getXMLReader();
-        xmlReader.setContentHandler(new XMLParser(out));
-        xmlReader.parse(new InputSource(new StringReader(xml)));
-    }
-
-    private class XMLParser extends DefaultHandler {
-
-        private OutputStream out;
-        private LinkedHashMap<String, OozieWFNode> tags;
-
-        private String action = null;
-        private String actionOK = null;
-        private String actionErr = null;
-        private String actionType = null;
-        private String fork;
-        private String decision;
-
-        public XMLParser(OutputStream out) {
-            this.out = out;
-        }
-
-        @Override
-        public void startDocument() throws SAXException {
-            tags = new LinkedHashMap();
-        }
-
-        @Override
-        public void endDocument() throws SAXException {
-
-            if(tags.isEmpty()) {
-                // Nothing to do here!
-                return;
-            }
-
-            int maxX = Integer.MIN_VALUE;
-            int maxY = Integer.MIN_VALUE;
-            int minX = Integer.MAX_VALUE;
-            int currX = 45;
-            int currY = 45;
-            final int xMargin = 205;
-            final int yMargin = 50;
-            final int xIncr = 215; // The widest element is 200 pixels 
(Rectangle)
-            final int yIncr = 255; // The tallest element is 150 pixels; 
(Diamond)
-            HashMap<String, WorkflowAction> actionMap = new HashMap<String, 
WorkflowAction>();
-
-            // Create a hashmap for faster lookups
-            // Also override showKill if there's any failed action
-            boolean found = false;
-            for(WorkflowAction wfAction : job.getActions()) {
-                actionMap.put(wfAction.getName(), wfAction);
-                if(!found) {
-                    switch(wfAction.getStatus()) {
-                        case KILLED:
-                        case ERROR:
-                        case FAILED:
-                            showKill = true; // Assuming on error the workflow 
eventually ends with kill node
-                            found = true;
-                    }
-                }
-            }
-
-            // Start building the graph
-            DirectedSparseGraph<OozieWFNode, String> dg = new 
DirectedSparseGraph<OozieWFNode, String>();
-            for(Map.Entry<String, OozieWFNode> entry : tags.entrySet()) {
-                String name = entry.getKey();
-                OozieWFNode node = entry.getValue();
-                if(actionMap.containsKey(name)) {
-                    node.setStatus(actionMap.get(name).getStatus());
-                }
-
-                // Set (x,y) coords of the vertices if not already set
-                if(node.getLocation().equals(new Point(0, 0))) {
-                    node.setLocation(currX, currY);
-                }
-
-                float childStep = showKill ? -(((float)node.getArcs().size() - 
1 ) / 2)
-                        : -((float)node.getArcs().size() / 2 - 1);
-                int nodeX = node.getLocation().x;
-                int nodeY = node.getLocation().y;
-                for(Map.Entry<String, Boolean> arc : 
node.getArcs().entrySet()) {
-                    if(!showKill && arc.getValue() && 
tags.get(arc.getKey()).getType().equals("kill")) {
-                        // Don't show kill node (assumption: only error goes 
to kill node;
-                        // No ok goes to kill node)
-                        continue;
-                    }
-                    OozieWFNode child = tags.get(arc.getKey());
-                    if(child == null) {
-                        continue; // or throw error?
-                    }
-                    dg.addEdge(name + "-->" + arc.getKey(), node, child);
-                    // TODO: Experimental -- should we set coords even if 
they're already set?
-                    //if(child.getLocation().equals(new Point(0, 0))) {
-                        int childX = (int)(nodeX + childStep * xIncr);
-                        int childY = nodeY + yIncr;
-                        child.setLocation(childX, childY);
-
-                        if(minX > childX) {
-                            minX = childX;
-                        }
-                        if(maxX < childX) {
-                            maxX = childX;
-                        }
-                        if(maxY < childY) {
-                            maxY = childY;
-                        }
-                    //}
-                    childStep += 1;
-                }
-
-                currY += yIncr;
-                currX = nodeX;
-                if(minX > nodeX) {
-                    minX = nodeX;
-                }
-                if(maxX < nodeX) {
-                    maxX = nodeX;
-                }
-                if(maxY < nodeY) {
-                    maxY = nodeY;
-                }
-            } // Done building graph
-
-            final int padX = minX < 0 ? -minX: 0;
-
-            Transformer<OozieWFNode, Point2D> locationInit = new 
Transformer<OozieWFNode, Point2D>() {
-
-                @Override
-                public Point2D transform(OozieWFNode node) {
-                    if(padX == 0) {
-                        return node.getLocation();
-                    } else {
-                        return new Point(node.getLocation().x + padX + 
xMargin, node.getLocation().y);
-                    }
-                }
-
-            };
-
-            StaticLayout<OozieWFNode, String> layout = new 
StaticLayout<OozieWFNode, String>(dg, locationInit, new Dimension(maxX + padX + 
xMargin, maxY));
-            layout.lock(true);
-            VisualizationImageServer<OozieWFNode, String> vis = new 
VisualizationImageServer<OozieWFNode, String>(layout, new Dimension(maxX + padX 
+ 2 * xMargin, maxY + yMargin));
-
-            vis.getRenderContext().setEdgeArrowTransformer(new 
ArrowShapeTransformer());
-            vis.getRenderContext().setArrowDrawPaintTransformer(new 
ArcPaintTransformer());
-            vis.getRenderContext().setEdgeDrawPaintTransformer(new 
ArcPaintTransformer());
-            vis.getRenderContext().setEdgeStrokeTransformer(new 
ArcStrokeTransformer());
-            vis.getRenderContext().setVertexShapeTransformer(new 
NodeShapeTransformer());
-            vis.getRenderContext().setVertexFillPaintTransformer(new 
NodePaintTransformer());
-            vis.getRenderContext().setVertexStrokeTransformer(new 
NodeStrokeTransformer());
-            vis.getRenderContext().setVertexLabelTransformer(new 
NodeLabelTransformer());
-            vis.getRenderContext().setVertexFontTransformer(new 
NodeFontTransformer());
-            
vis.getRenderer().getVertexLabelRenderer().setPosition(Renderer.VertexLabel.Position.CNTR);
-            vis.setBackground(Color.WHITE);
-
-            Dimension d = vis.getSize();
-            BufferedImage img = new BufferedImage(d.width, d.height, 
BufferedImage.TYPE_INT_RGB);
-            Graphics2D g = img.createGraphics();
-            vis.paintAll(g);
-
-            try {
-                ImageIO.write(img, "png", out);
-            }
-            catch (IOException ioe) {
-                throw new SAXException(ioe);
-            }
-            finally {
-                try {
-                    out.close(); //closing connection is imperative
-                                 //regardless of ImageIO.write throwing 
exception or not
-                                 //hence in finally block
-                }
-                catch (IOException e) {
-                    XLog.getLog(getClass()).trace("Exception while closing 
OutputStream");
-                }
-                out = null;
-                img.flush();
-                g.dispose();
-                vis.removeAll();
-            }
-        }
-
-        @Override
-        public void startElement(String namespaceURI,
-                                String localName,
-                                String qName,
-                                Attributes atts)
-            throws SAXException {
-            if(localName.equalsIgnoreCase("start")) {
-                String start = localName.toLowerCase();
-                if(!tags.containsKey(start)) {
-                    OozieWFNode v = new OozieWFNode(start, start);
-                    v.addArc(atts.getValue("to"));
-                    tags.put(start, v);
-                }
-            } else if(localName.equalsIgnoreCase("action")) {
-                action = atts.getValue("name");
-            } else if(action != null && actionType == null) {
-                actionType = localName.toLowerCase();
-            } else if(localName.equalsIgnoreCase("ok") && action != null && 
actionOK == null) {
-                    actionOK = atts.getValue("to");
-            } else if(localName.equalsIgnoreCase("error") && action != null && 
actionErr == null) {
-                    actionErr = atts.getValue("to");
-            } else if(localName.equalsIgnoreCase("fork")) {
-                fork = atts.getValue("name");
-                if(!tags.containsKey(fork)) {
-                    tags.put(fork, new OozieWFNode(fork, 
localName.toLowerCase()));
-                }
-            } else if(localName.equalsIgnoreCase("path")) {
-                tags.get(fork).addArc(atts.getValue("start"));
-            } else if(localName.equalsIgnoreCase("join")) {
-                String join = atts.getValue("name");
-                if(!tags.containsKey(join)) {
-                    OozieWFNode v = new OozieWFNode(join, 
localName.toLowerCase());
-                    v.addArc(atts.getValue("to"));
-                    tags.put(join, v);
-                }
-            } else if(localName.equalsIgnoreCase("decision")) {
-                decision = atts.getValue("name");
-                if(!tags.containsKey(decision)) {
-                    tags.put(decision, new OozieWFNode(decision, 
localName.toLowerCase()));
-                }
-            } else if(localName.equalsIgnoreCase("case")
-                    || localName.equalsIgnoreCase("default")) {
-                tags.get(decision).addArc(atts.getValue("to"));
-            } else if(localName.equalsIgnoreCase("kill")
-                    || localName.equalsIgnoreCase("end")) {
-                String name = atts.getValue("name");
-                if(!tags.containsKey(name)) {
-                    tags.put(name, new OozieWFNode(name, 
localName.toLowerCase()));
-                }
-            }
-            if (tags.size() > actionsLimit) {
-                tags.clear();
-                throw new SAXException("Can't display the graph. Number of 
actions are more than display limit " + actionsLimit);
-            }
-        }
-
-        @Override
-        public void endElement(String namespaceURI,
-                                String localName,
-                                String qName)
-                throws SAXException {
-            if(localName.equalsIgnoreCase("action")) {
-                tags.put(action, new OozieWFNode(action, actionType));
-                tags.get(action).addArc(this.actionOK);
-                tags.get(action).addArc(this.actionErr, true);
-                action = null;
-                actionOK = null;
-                actionErr = null;
-                actionType = null;
-            }
-        }
-
-        private class OozieWFNode {
-            private String name;
-            private String type;
-            private Point loc;
-            private HashMap<String, Boolean> arcs;
-            private Status status = null;
-
-            public OozieWFNode(String name,
-                    String type,
-                    HashMap<String, Boolean> arcs,
-                    Point loc,
-                    Status status) {
-                this.name = name;
-                this.type = type;
-                this.arcs = arcs;
-                this.loc = loc;
-                this.status = status;
-            }
-
-            public OozieWFNode(String name, String type, HashMap<String, 
Boolean> arcs) {
-                this(name, type, arcs, new Point(0, 0), null);
-            }
-
-            public OozieWFNode(String name, String type) {
-                this(name, type, new HashMap<String, Boolean>(), new Point(0, 
0), null);
-            }
-
-            public OozieWFNode(String name, String type, WorkflowAction.Status 
status) {
-                this(name, type, new HashMap<String, Boolean>(), new Point(0, 
0), status);
-            }
-
-            public void addArc(String arc, boolean isError) {
-                arcs.put(arc, isError);
-            }
-
-            public void addArc(String arc) {
-                addArc(arc, false);
-            }
-
-            public void setName(String name) {
-                this.name = name;
-            }
-
-            public void setType(String type) {
-                this.type = type;
-            }
-
-            public void setLocation(Point loc) {
-                this.loc = loc;
-            }
-
-            public void setLocation(double x, double y) {
-                loc.setLocation(x, y);
-            }
-
-            public void setStatus(WorkflowAction.Status status) {
-                this.status = status;
-            }
-
-            public String getName() {
-                return name;
-            }
-
-            public String getType() {
-                return type;
-            }
-
-            public HashMap<String, Boolean> getArcs() {
-                return arcs;
-            }
-
-            public Point getLocation() {
-                return loc;
-            }
-
-            public WorkflowAction.Status getStatus() {
-                return status;
-            }
-
-            @Override
-            public String toString() {
-                StringBuilder s = new StringBuilder();
-
-                s.append("Node: ").append(name).append("\t");
-                s.append("Type: ").append(type).append("\t");
-                s.append("Location: (").append(loc.getX()).append(", 
").append(loc.getY()).append(")\t");
-                s.append("Status: ").append(status).append("\n");
-                Iterator<Map.Entry<String, Boolean>> it = 
arcs.entrySet().iterator();
-                while(it.hasNext()) {
-                    Map.Entry<String, Boolean> entry = it.next();
-
-                    s.append("\t").append(entry.getKey());
-                    if(entry.getValue().booleanValue()) {
-                        s.append(" on error\n");
-                    } else {
-                        s.append("\n");
-                    }
-                }
-
-                return s.toString();
-            }
-        }
-
-        private class NodeFontTransformer implements Transformer<OozieWFNode, 
Font> {
-            private final Font font = new Font("Default", Font.BOLD, 15);
-
-            @Override
-            public Font transform(OozieWFNode node) {
-                return font;
-            }
-        }
-
-        private class ArrowShapeTransformer implements 
Transformer<Context<Graph<OozieWFNode, String>, String>,  Shape> {
-            private final Shape arrow = ArrowFactory.getWedgeArrow(10.0f, 
20.0f);
-
-            @Override
-            public Shape transform(Context<Graph<OozieWFNode, String>, String> 
i) {
-                return arrow;
-            }
-        }
-
-        private class ArcPaintTransformer implements Transformer<String, 
Paint> {
-            // Paint based on transition
-            @Override
-            public Paint transform(String arc) {
-                int sep = arc.indexOf("-->");
-                String source = arc.substring(0, sep);
-                String target = arc.substring(sep + 3);
-                OozieWFNode src = tags.get(source);
-                OozieWFNode tgt = tags.get(target);
-
-                if(src.getType().equals("start")) {
-                    if(tgt.getStatus() == null) {
-                        return Color.LIGHT_GRAY;
-                    } else {
-                        return Color.GREEN;
-                    }
-                }
-
-                if(src.getArcs().get(target)) {
-                    // Dealing with error transition (i.e. target is error)
-                    if(src.getStatus() == null) {
-                        return Color.LIGHT_GRAY;
-                    }
-                    switch(src.getStatus()) {
-                        case KILLED:
-                        case ERROR:
-                        case FAILED:
-                            return Color.RED;
-                        default:
-                            return Color.LIGHT_GRAY;
-                    }
-                } else {
-                    // Non-error
-                    if(src.getType().equals("decision")) {
-                        // Check for target too
-                        if(tgt.getStatus() != null) {
-                            return Color.GREEN;
-                        } else {
-                            return Color.LIGHT_GRAY;
-                        }
-                    } else {
-                        if(src.getStatus() == null) {
-                            return Color.LIGHT_GRAY;
-                        }
-                        switch(src.getStatus()) {
-                            case OK:
-                            case DONE:
-                            case END_RETRY:
-                            case END_MANUAL:
-                                return Color.GREEN;
-                            default:
-                                return Color.LIGHT_GRAY;
-                        }
-                    }
-                }
-            }
-        }
-
-        private class NodeStrokeTransformer implements 
Transformer<OozieWFNode, Stroke> {
-            private final Stroke stroke1 = new BasicStroke(2.0f);
-            private final Stroke stroke2 = new BasicStroke(4.0f);
-
-            @Override
-            public Stroke transform(OozieWFNode node) {
-                if(node.getType().equals("start")
-                        || node.getType().equals("end")
-                        || node.getType().equals("kill")) {
-                    return stroke2;
-                }
-                return stroke1;
-            }
-        }
-
-        private class NodeLabelTransformer implements Transformer<OozieWFNode, 
String> {
-            /*
-            * 20 chars in rectangle in 2 rows max
-            * 14 chars in diamond in 2 rows max
-            * 9 in triangle in 2 rows max
-            * 8 in invtriangle in 2 rows max
-            * 8 in circle in 2 rows max
-            */
-            @Override
-            public String transform(OozieWFNode node) {
-                //return node.getType();
-                String name = node.getName();
-                String type = node.getType();
-                StringBuilder s = new StringBuilder();
-                if(type.equals("decision")) {
-                    if(name.length() <= 14) {
-                        return name;
-                    } else {
-                        s.append("<html>").append(name.substring(0, 
12)).append("-<br />");
-                        if(name.substring(13).length() > 14) {
-                            s.append(name.substring(12, 25)).append("...");
-                        } else {
-                            s.append(name.substring(12));
-                        }
-                        s.append("</html>");
-                        return s.toString();
-                    }
-                } else if(type.equals("fork")) {
-                    if(name.length() <= 9) {
-                        return "<html><br />" + name + "</html>";
-                    } else {
-                        s.append("<html><br />").append(name.substring(0, 
7)).append("-<br />");
-                        if(name.substring(8).length() > 9) {
-                            s.append(name.substring(7, 15)).append("...");
-                        } else {
-                            s.append(name.substring(7));
-                        }
-                        s.append("</html>");
-                        return s.toString();
-                    }
-                } else if(type.equals("join")) {
-                    if(name.length() <= 8) {
-                        return "<html>" + name + "</html>";
-                    } else {
-                        s.append("<html>").append(name.substring(0, 
6)).append("-<br />");
-                        if(name.substring(7).length() > 8) {
-                            s.append(name.substring(6, 13)).append("...");
-                        } else {
-                            s.append(name.substring(6));
-                        }
-                        s.append("</html>");
-                        return s.toString();
-                    }
-                } else if(type.equals("start")
-                        || type.equals("end")
-                        || type.equals("kill")) {
-                    if(name.length() <= 8) {
-                        return "<html>" + name + "</html>";
-                    } else {
-                        s.append("<html>").append(name.substring(0, 
6)).append("-<br />");
-                        if(name.substring(7).length() > 8) {
-                            s.append(name.substring(6, 13)).append("...");
-                        } else {
-                            s.append(name.substring(6));
-                        }
-                        s.append("</html>");
-                        return s.toString();
-                    }
-                }else {
-                    if(name.length() <= 20) {
-                        return name;
-                    } else {
-                        s.append("<html>").append(name.substring(0, 
18)).append("-<br />");
-                        if(name.substring(19).length() > 20) {
-                            s.append(name.substring(18, 37)).append("...");
-                        } else {
-                            s.append(name.substring(18));
-                        }
-                        s.append("</html>");
-                        return s.toString();
-                    }
-                }
-            }
-        }
-
-        private class NodePaintTransformer implements Transformer<OozieWFNode, 
Paint> {
-            @Override
-            public Paint transform(OozieWFNode node) {
-                WorkflowJob.Status jobStatus = job.getStatus();
-                if(node.getType().equals("start")) {
-                    return Color.WHITE;
-                } else if(node.getType().equals("end")) {
-                    if(jobStatus == WorkflowJob.Status.SUCCEEDED) {
-                        return Color.GREEN;
-                    }
-                    return Color.BLACK;
-                } else if(node.getType().equals("kill")) {
-                    if(jobStatus == WorkflowJob.Status.FAILED
-                            || jobStatus == WorkflowJob.Status.KILLED) {
-                        return Color.RED;
-                    }
-                    return Color.WHITE;
-                }
-
-                // Paint based on status for rest
-                WorkflowAction.Status status = node.getStatus();
-                if(status == null) {
-                    return Color.LIGHT_GRAY;
-                }
-                switch(status) {
-                    case OK:
-                    case DONE:
-                    case END_RETRY:
-                    case END_MANUAL:
-                        return Color.GREEN;
-                    case PREP:
-                    case RUNNING:
-                    case USER_RETRY:
-                    case START_RETRY:
-                    case START_MANUAL:
-                        return Color.YELLOW;
-                    case KILLED:
-                    case ERROR:
-                    case FAILED:
-                        return Color.RED;
-                    default:
-                        return Color.LIGHT_GRAY;
-                }
-            }
-        }
-
-        private class NodeShapeTransformer implements Transformer<OozieWFNode, 
Shape> {
-            private final Ellipse2D.Double circle = new Ellipse2D.Double(-40, 
-40, 80, 80);
-            private final Rectangle rect = new Rectangle(-100, -30, 200, 60);
-            private final Polygon diamond = new Polygon(new int[]{-75, 0, 75, 
0}, new int[]{0, 75, 0, -75}, 4);
-            private final Polygon triangle = new Polygon(new int[]{-85, 85, 
0}, new int[]{0, 0, -148}, 3);
-            private final Polygon invtriangle = new Polygon(new int[]{-85, 85, 
0}, new int[]{0, 0, 148}, 3);
-
-            @Override
-            public Shape transform(OozieWFNode node) {
-                if("start".equals(node.getType())
-                    || "end".equals(node.getType())
-                    || "kill".equals(node.getType())) {
-                    return circle;
-                }
-                if("fork".equals(node.getType())) {
-                    return triangle;
-                }
-                if("join".equals(node.getType())) {
-                    return invtriangle;
-                }
-                if("decision".equals(node.getType())) {
-                    return diamond;
-                }
-                return rect; // All action nodes
-            }
-        }
-
-        private class ArcStrokeTransformer implements Transformer<String, 
Stroke> {
-            private final Stroke stroke1 = new BasicStroke(2.0f);
-            private final Stroke dashed = new BasicStroke(1.0f, 
BasicStroke.CAP_BUTT, BasicStroke.JOIN_MITER, 10.0f, new float[] {10.0f}, 0.0f);
-
-            // Draw based on transition
-            @Override
-            public Stroke transform(String arc) {
-                int sep = arc.indexOf("-->");
-                String source = arc.substring(0, sep);
-                String target = arc.substring(sep + 3);
-                OozieWFNode src = tags.get(source);
-                if(src.getArcs().get(target)) {
-                        if(src.getStatus() == null) {
-                            return dashed;
-                        }
-                        switch(src.getStatus()) {
-                            case KILLED:
-                            case ERROR:
-                            case FAILED:
-                                return stroke1;
-                            default:
-                                return dashed;
-                        }
-                } else {
-                    return stroke1;
-                }
-            }
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/oozie/blob/883c7556/core/src/main/java/org/apache/oozie/util/graph/GraphGenerator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/oozie/util/graph/GraphGenerator.java 
b/core/src/main/java/org/apache/oozie/util/graph/GraphGenerator.java
new file mode 100644
index 0000000..12848d2
--- /dev/null
+++ b/core/src/main/java/org/apache/oozie/util/graph/GraphGenerator.java
@@ -0,0 +1,102 @@
+/**
+ * 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.oozie.util.graph;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
+import org.apache.oozie.client.WorkflowJob;
+import org.xml.sax.InputSource;
+import org.xml.sax.SAXException;
+import org.xml.sax.XMLReader;
+
+import javax.xml.XMLConstants;
+import javax.xml.parsers.ParserConfigurationException;
+import javax.xml.parsers.SAXParser;
+import javax.xml.parsers.SAXParserFactory;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.io.StringReader;
+
+/**
+ * Class to generate and plot runtime workflow DAG.
+ * <p/>
+ * Since it delegates to {@link WorkflowGraphHandler} and a {@link 
GraphRenderer}, it is the single entry point when changing graph
+ * generation behavior.
+ */
+public class GraphGenerator {
+    public static final String SAX_FEATURE_EXTERNAL_GENERAL_ENTITIES = 
"http://xml.org/sax/features/external-general-entities";;
+    public static final String SAX_FEATURE_EXTERNAL_PARAMETER_ENTITIES = 
"http://xml.org/sax/features/external-parameter-entities";;
+    public static final String SAX_FEATURE_DISALLOW_DOCTYPE_DECL = 
"http://apache.org/xml/features/disallow-doctype-decl";;
+    private final GraphRenderer graphRenderer;
+    private final String xml;
+    private final WorkflowJob job;
+    private final boolean showKill;
+
+    /**
+     * Constructor Inversion of Control-style for better testability.
+     * @param xml The workflow definition XML
+     * @param job Current status of the job
+     * @param showKill Flag to whether show 'kill' node
+     * @param graphRenderer
+     */
+    public GraphGenerator(final String xml, final WorkflowJob job, final 
boolean showKill, final GraphRenderer graphRenderer) {
+        Preconditions.checkArgument(!Strings.isNullOrEmpty(xml), "xml can't be 
empty");
+        Preconditions.checkArgument(job != null, "WorkflowJob can't be null");
+
+        this.xml = xml;
+        this.job = job;
+        this.showKill = showKill;
+        this.graphRenderer = graphRenderer;
+    }
+
+    /**
+     * Stream the generated PNG, DOT or SVG stream to the caller. Note that 
closing the {@link OutputStream} is the responsibility
+     * of the caller.
+     *
+     * @param out the {@link OutputStream} to use on streaming
+     * @param outputFormat The output format to apply when rendering
+     * @throws Exception
+     */
+    public void write(final OutputStream out, final OutputFormat outputFormat)
+            throws ParserConfigurationException, SAXException, IOException {
+        final XMLReader xmlReader = newXmlReader();
+        xmlReader.setFeature(XMLConstants.FEATURE_SECURE_PROCESSING, true);
+
+        xmlReader.setContentHandler(
+                new WorkflowGraphHandler(out, outputFormat, job, showKill, 
graphRenderer));
+
+        try (final StringReader stringReader = new StringReader(xml)) {
+            xmlReader.parse(new InputSource(stringReader));
+        }
+    }
+
+    private XMLReader newXmlReader() throws ParserConfigurationException, 
SAXException {
+        final SAXParserFactory spf = SAXParserFactory.newInstance();
+        spf.setFeature(SAX_FEATURE_EXTERNAL_GENERAL_ENTITIES, false);
+        spf.setFeature(SAX_FEATURE_EXTERNAL_PARAMETER_ENTITIES, false);
+        spf.setFeature(SAX_FEATURE_DISALLOW_DOCTYPE_DECL, true);
+        spf.setNamespaceAware(true);
+
+        final SAXParser saxParser = spf.newSAXParser();
+        final XMLReader xmlReader = saxParser.getXMLReader();
+        xmlReader.setFeature(XMLConstants.FEATURE_SECURE_PROCESSING, true);
+
+        return xmlReader;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/oozie/blob/883c7556/core/src/main/java/org/apache/oozie/util/graph/GraphRenderer.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/oozie/util/graph/GraphRenderer.java 
b/core/src/main/java/org/apache/oozie/util/graph/GraphRenderer.java
new file mode 100644
index 0000000..8fd3f17
--- /dev/null
+++ b/core/src/main/java/org/apache/oozie/util/graph/GraphRenderer.java
@@ -0,0 +1,35 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.util.graph;
+
+import java.awt.image.BufferedImage;
+
+public interface GraphRenderer {
+    void addNode(WorkflowActionNode node);
+
+    void addEdge(WorkflowActionNode parent, WorkflowActionNode child);
+
+    void persist(WorkflowActionNode parent);
+
+    BufferedImage renderPng();
+
+    String renderDot();
+
+    String renderSvg();
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/883c7556/core/src/main/java/org/apache/oozie/util/graph/GraphvizRenderer.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/oozie/util/graph/GraphvizRenderer.java 
b/core/src/main/java/org/apache/oozie/util/graph/GraphvizRenderer.java
new file mode 100644
index 0000000..2651457
--- /dev/null
+++ b/core/src/main/java/org/apache/oozie/util/graph/GraphvizRenderer.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
+ *
+ *      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.oozie.util.graph;
+
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.Multimap;
+import guru.nidi.graphviz.attribute.Color;
+import guru.nidi.graphviz.attribute.RankDir;
+import guru.nidi.graphviz.attribute.Shape;
+import guru.nidi.graphviz.engine.Engine;
+import guru.nidi.graphviz.engine.Format;
+import guru.nidi.graphviz.engine.Graphviz;
+import guru.nidi.graphviz.engine.Rasterizer;
+import guru.nidi.graphviz.model.Factory;
+import guru.nidi.graphviz.model.Graph;
+import guru.nidi.graphviz.model.Node;
+import org.apache.oozie.client.WorkflowAction;
+import org.apache.oozie.service.ConfigurationService;
+
+import java.awt.image.BufferedImage;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+public class GraphvizRenderer implements GraphRenderer {
+
+    /**
+     * We need this single-thread executor because we have to make sure:
+     * <ul>
+     * <li>all GraphViz rendering operations happen on the same thread.
+     * This is because of {@link com.eclipsesource.v8.V8} thread handling</li>
+     * <li>GraphViz rendering operations don't timeout</li>
+     * <li>GraphViz rendering operations don't overlap</li>
+     * </ul>
+     */
+    private static final ExecutorService EXECUTOR_SERVICE = 
Executors.newSingleThreadExecutor();
+    private static final long GRAPHVIZ_TIMEOUT_SECONDS = 
ConfigurationService.getLong("oozie.graphviz.timeout.seconds");
+
+    private Graph graphvizGraph = 
Factory.graph().generalAttr().with(RankDir.TOP_TO_BOTTOM).directed();
+    private final Map<String, Node> graphvizNodes = new LinkedHashMap<>();
+    private final Multimap<String, String> edges = ArrayListMultimap.create();
+    ;
+    private int arcCount = 0;
+
+    @Override
+    public void addNode(final WorkflowActionNode node) {
+        final Shape shape = getShape(node.getType());
+        final Color color = getColor(node.getStatus());
+
+        final Node graphvizNode = 
Factory.node(node.getName()).with(shape).with(color);
+
+        graphvizNodes.put(node.getName(), graphvizNode);
+    }
+
+    private Shape getShape(final String type) {
+        final Shape shape;
+
+        switch (type) {
+            case "start":
+                shape = Shape.CIRCLE;
+                break;
+            case "end":
+                shape = Shape.DOUBLE_CIRCLE;
+                break;
+            case "kill":
+                shape = Shape.OCTAGON;
+                break;
+            case "decision":
+                shape = Shape.DIAMOND;
+                break;
+            case "fork":
+                shape = Shape.TRIANGLE;
+                break;
+            case "join":
+                shape = Shape.INV_TRIANGLE;
+                break;
+            default:
+                shape = Shape.RECTANGLE;
+                break;
+        }
+
+        return shape;
+    }
+
+    private Color getColor(final WorkflowAction.Status status) {
+        if (status == null) {
+            return Color.BLACK;
+        }
+
+        final Color color;
+
+        switch (status) {
+            case PREP:
+            case USER_RETRY:
+            case START_RETRY:
+            case START_MANUAL:
+                color = Color.GREY;
+                break;
+            case RUNNING:
+            case END_RETRY:
+            case END_MANUAL:
+                color = Color.YELLOW;
+                break;
+            case OK:
+            case DONE:
+                color = Color.GREEN;
+                break;
+            case ERROR:
+            case FAILED:
+            case KILLED:
+                color = Color.RED;
+                break;
+            default:
+                color = Color.BLACK;
+        }
+
+        return color;
+    }
+
+    private Node createOrGetGraphvizNode(final WorkflowActionNode node) {
+        if (graphvizNodes.containsKey(node.getName())) {
+            return graphvizNodes.get(node.getName());
+        }
+
+        addNode(node);
+
+        return graphvizNodes.get(node.getName());
+    }
+
+    @Override
+    public void addEdge(final WorkflowActionNode parent, final 
WorkflowActionNode child) {
+        if (edges.containsEntry(parent.getName(), child.getName())) {
+            return;
+        }
+
+        Node graphvizParent = createOrGetGraphvizNode(parent);
+
+        graphvizParent = graphvizParent.link(
+                
Factory.to(createOrGetGraphvizNode(child)).with(calculateEdgeColor(child.getStatus())));
+        graphvizNodes.put(parent.getName(), graphvizParent);
+
+        edges.put(parent.getName(), child.getName());
+        arcCount++;
+    }
+
+    private Color calculateEdgeColor(final WorkflowAction.Status childStatus) {
+        if (childStatus == null) {
+            return Color.BLACK;
+        }
+
+        if (childStatus.equals(WorkflowAction.Status.RUNNING)) {
+            return Color.GREEN;
+        }
+
+        return getColor(childStatus);
+    }
+
+    @Override
+    public void persist(final WorkflowActionNode node) {
+        final Node graphvizNode = graphvizNodes.get(node.getName());
+        graphvizGraph = graphvizGraph.with(graphvizNode);
+    }
+
+    @Override
+    public BufferedImage renderPng() {
+        final Future<BufferedImage> pngFuture = EXECUTOR_SERVICE.submit(new 
PngRenderer());
+
+        try {
+            return pngFuture.get(GRAPHVIZ_TIMEOUT_SECONDS, TimeUnit.SECONDS);
+        } catch (final InterruptedException | ExecutionException | 
TimeoutException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    private int calculateHeight(final int arcCount) {
+        return Math.min(arcCount * 100, 2000);
+    }
+
+    @Override
+    public String renderDot() {
+        return graphvizGraph.toString();
+    }
+
+    @Override
+    public String renderSvg() {
+        final Future<String> svgFuture = EXECUTOR_SERVICE.submit(new 
SvgRenderer());
+
+        try {
+            return svgFuture.get(GRAPHVIZ_TIMEOUT_SECONDS, TimeUnit.SECONDS);
+        } catch (final InterruptedException | ExecutionException | 
TimeoutException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    private class PngRenderer implements Callable<BufferedImage> {
+        @Override
+        public BufferedImage call() throws Exception {
+            final Graphviz graphviz = newGraphviz();
+
+            return graphviz.render(Format.PNG).toImage();
+        }
+    }
+
+    private class SvgRenderer implements Callable<String> {
+
+        @Override
+        public String call() throws Exception {
+            final Graphviz graphviz = newGraphviz();
+
+            return graphviz.render(Format.SVG).toString();
+        }
+    }
+
+    private Graphviz newGraphviz() {
+        return Graphviz.fromGraph(graphvizGraph)
+                .rasterizer(Rasterizer.BATIK)
+                .engine(Engine.DOT)
+                .height(calculateHeight(arcCount));
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/oozie/blob/883c7556/core/src/main/java/org/apache/oozie/util/graph/OutputFormat.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/oozie/util/graph/OutputFormat.java 
b/core/src/main/java/org/apache/oozie/util/graph/OutputFormat.java
new file mode 100644
index 0000000..67d35a5
--- /dev/null
+++ b/core/src/main/java/org/apache/oozie/util/graph/OutputFormat.java
@@ -0,0 +1,23 @@
+/**
+ * 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.oozie.util.graph;
+
+public enum OutputFormat {
+    PNG, DOT, SVG
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/883c7556/core/src/main/java/org/apache/oozie/util/graph/WorkflowActionNode.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/oozie/util/graph/WorkflowActionNode.java 
b/core/src/main/java/org/apache/oozie/util/graph/WorkflowActionNode.java
new file mode 100644
index 0000000..3d3f38b
--- /dev/null
+++ b/core/src/main/java/org/apache/oozie/util/graph/WorkflowActionNode.java
@@ -0,0 +1,149 @@
+/**
+ * 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.oozie.util.graph;
+
+import org.apache.oozie.client.WorkflowAction;
+
+import java.awt.Point;
+import java.util.HashMap;
+import java.util.Map;
+
+public class WorkflowActionNode {
+    private String name;
+    private String type;
+    private Point loc;
+    private final Map<String, Boolean> arcs;
+    private WorkflowAction.Status status = null;
+
+    private WorkflowActionNode(final String name,
+                               final String type,
+                               final HashMap<String, Boolean> arcs,
+                               final Point loc,
+                               final WorkflowAction.Status status) {
+        this.name = name;
+        this.type = type;
+        this.arcs = arcs;
+        this.loc = loc;
+        this.status = status;
+    }
+
+    WorkflowActionNode(final String name, final String type) {
+        this(name, type, new HashMap<String, Boolean>(), new Point(0, 0), 
null);
+    }
+
+    void addArc(final String arc, final boolean isError) {
+        arcs.put(arc, isError);
+    }
+
+    void addArc(final String arc) {
+        addArc(arc, false);
+    }
+
+    public void setName(final String name) {
+        this.name = name;
+    }
+
+    public void setType(final String type) {
+        this.type = type;
+    }
+
+    public void setLocation(final Point loc) {
+        this.loc = loc;
+    }
+
+    void setLocation(final double x, final double y) {
+        loc.setLocation(x, y);
+    }
+
+    public void setStatus(final WorkflowAction.Status status) {
+        this.status = status;
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public String getType() {
+        return type;
+    }
+
+    Map<String, Boolean> getArcs() {
+        return arcs;
+    }
+
+    public Point getLocation() {
+        return loc;
+    }
+
+    public WorkflowAction.Status getStatus() {
+        return status;
+    }
+
+    @Override
+    public String toString() {
+        final StringBuilder s = new StringBuilder();
+
+        s.append("Node: ").append(name).append("\t");
+        s.append("Type: ").append(type).append("\t");
+        s.append("Location: (").append(loc.getX()).append(", 
").append(loc.getY()).append(")\t");
+        s.append("Status: ").append(status).append("\n");
+
+        for (final Map.Entry<String, Boolean> entry : arcs.entrySet()) {
+            s.append("\t").append(entry.getKey());
+            if (entry.getValue()) {
+                s.append(" on error\n");
+            } else {
+                s.append("\n");
+            }
+        }
+
+        return s.toString();
+    }
+
+    @Override
+    public boolean equals(final Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+
+        final WorkflowActionNode that = (WorkflowActionNode) o;
+
+        if (!name.equals(that.name)) {
+            return false;
+        }
+        if (!type.equals(that.type)) {
+            return false;
+        }
+
+        return status == that.status;
+    }
+
+    @Override
+    public int hashCode() {
+        int result = name.hashCode();
+
+        result = 31 * result + type.hashCode();
+        result = 31 * result + (status != null ? status.hashCode() : 0);
+
+        return result;
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/883c7556/core/src/main/java/org/apache/oozie/util/graph/WorkflowGraphHandler.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/oozie/util/graph/WorkflowGraphHandler.java 
b/core/src/main/java/org/apache/oozie/util/graph/WorkflowGraphHandler.java
new file mode 100644
index 0000000..2a01587
--- /dev/null
+++ b/core/src/main/java/org/apache/oozie/util/graph/WorkflowGraphHandler.java
@@ -0,0 +1,257 @@
+/**
+ * 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.oozie.util.graph;
+
+import com.google.common.base.Charsets;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
+import org.apache.oozie.client.WorkflowAction;
+import org.apache.oozie.client.WorkflowJob;
+import org.apache.oozie.util.Instrumentation;
+import org.xml.sax.Attributes;
+import org.xml.sax.SAXException;
+import org.xml.sax.helpers.DefaultHandler;
+
+import javax.imageio.ImageIO;
+import java.awt.image.BufferedImage;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.LinkedHashMap;
+import java.util.Locale;
+import java.util.Map;
+
+public class WorkflowGraphHandler extends DefaultHandler {
+    private OutputStream out;
+    private final OutputFormat outputFormat;
+    private final WorkflowJob job;
+    private boolean showKill;
+    private final GraphRenderer graphRenderer;
+    private final Map<String, WorkflowActionNode> tags = new LinkedHashMap<>();
+    private final WorkflowParseState state = new WorkflowParseState();
+
+    WorkflowGraphHandler(final OutputStream out,
+                         final OutputFormat outputFormat,
+                         final WorkflowJob job,
+                         final boolean showKill,
+                         final GraphRenderer graphRenderer) {
+        this.out = out;
+        this.job = job;
+        this.showKill = showKill;
+        this.graphRenderer = graphRenderer;
+        this.outputFormat = outputFormat;
+    }
+
+    @Override
+    public void startDocument() throws SAXException {
+        // NOP
+    }
+
+    @Override
+    public void endDocument() throws SAXException {
+        final Instrumentation.Cron cron = new Instrumentation.Cron();
+        cron.start();
+
+        if (tags.isEmpty()) {
+            // Nothing to do here!
+            return;
+        }
+
+        final Map<String, WorkflowAction> workflowActions = 
fillWorkflowActions();
+        for (final Map.Entry<String, WorkflowActionNode> entry : 
tags.entrySet()) {
+            final String name = entry.getKey();
+            final WorkflowActionNode parent = entry.getValue();
+            if (workflowActions.containsKey(name)) {
+                parent.setStatus(workflowActions.get(name).getStatus());
+            }
+
+            graphRenderer.addNode(parent);
+
+            for (final Map.Entry<String, Boolean> arc : 
parent.getArcs().entrySet()) {
+                if (!showKill && arc.getValue() && 
tags.get(arc.getKey()).getType().equals("kill")) {
+                    // Don't show kill node (assumption: only error goes to 
kill node;
+                    // No ok goes to kill node)
+                    continue;
+                }
+
+                final WorkflowActionNode child = tags.get(arc.getKey());
+                if (child != null) {
+                    if (workflowActions.containsKey(arc.getKey())) {
+                        
child.setStatus(workflowActions.get(arc.getKey()).getStatus());
+                    }
+
+                    graphRenderer.addEdge(parent, child);
+                }
+            }
+
+            graphRenderer.persist(parent);
+        }
+
+        switch (outputFormat) {
+            case PNG:
+                renderAndWritePng();
+                break;
+            case DOT:
+                renderAndWriteDot();
+                break;
+            case SVG:
+                renderAndWriteSvg();
+                break;
+            default:
+                throw new IllegalArgumentException(String.format("Unknown 
outputFormat %s", outputFormat));
+        }
+    }
+
+    private void renderAndWritePng() throws SAXException {
+        final BufferedImage source = graphRenderer.renderPng();
+
+        try {
+            ImageIO.write(source, "png", out);
+        } catch (final IOException ioe) {
+            throw new SAXException(ioe);
+        } finally {
+            source.flush();
+        }
+    }
+
+    private void renderAndWriteDot() throws SAXException {
+        renderStringContent(graphRenderer.renderDot());
+    }
+
+    private void renderAndWriteSvg() throws SAXException {
+        renderStringContent(graphRenderer.renderSvg());
+    }
+
+    private void renderStringContent(final String content) throws SAXException 
{
+        Preconditions.checkState(!Strings.isNullOrEmpty(content), "No output 
generated from graph.");
+
+        try {
+            out.write(content.getBytes(Charsets.UTF_8));
+        } catch (final IOException ioe) {
+            throw new SAXException(ioe);
+        }
+    }
+
+    private Map<String, WorkflowAction> fillWorkflowActions() {
+        final Map<String, WorkflowAction> workflowActions = new 
LinkedHashMap<>();
+
+        boolean found = false;
+        for (final WorkflowAction wfAction : job.getActions()) {
+            workflowActions.put(wfAction.getName(), wfAction);
+            if (!found) {
+                switch (wfAction.getStatus()) {
+                    case KILLED:
+                    case ERROR:
+                    case FAILED:
+                        showKill = true; // Assuming on error the workflow 
eventually ends with kill node
+                        found = true;
+                        break;
+                    default:
+                        // Look further
+                        break;
+                }
+            }
+        }
+
+        return workflowActions;
+    }
+
+
+    @Override
+    public void startElement(final String namespaceURI,
+                             final String localName,
+                             final String qName,
+                             final Attributes atts)
+            throws SAXException {
+        if (localName.equalsIgnoreCase("start")) {
+            final String start = localName.toLowerCase(Locale.getDefault());
+            if (!tags.containsKey(start)) {
+                final WorkflowActionNode v = new WorkflowActionNode(start, 
start);
+                v.addArc(atts.getValue("to"));
+                tags.put(start, v);
+            }
+        } else if (localName.equalsIgnoreCase("action")) {
+            state.action = atts.getValue("name");
+        } else if (state.action != null && state.actionType == null) {
+            state.actionType = localName.toLowerCase(Locale.getDefault());
+        } else if (localName.equalsIgnoreCase("ok") && state.action != null && 
state.actionOK == null) {
+            state.actionOK = atts.getValue("to");
+        } else if (localName.equalsIgnoreCase("error") && state.action != null 
&& state.actionErr == null) {
+            state.actionErr = atts.getValue("to");
+        } else if (localName.equalsIgnoreCase("fork")) {
+            state.fork = atts.getValue("name");
+            if (!tags.containsKey(state.fork)) {
+                tags.put(state.fork, new WorkflowActionNode(state.fork, 
localName.toLowerCase(Locale.getDefault())));
+            }
+        } else if (localName.equalsIgnoreCase("path")) {
+            tags.get(state.fork).addArc(atts.getValue("start"));
+        } else if (localName.equalsIgnoreCase("join")) {
+            final String join = atts.getValue("name");
+            if (!tags.containsKey(join)) {
+                final WorkflowActionNode v = new WorkflowActionNode(join, 
localName.toLowerCase(Locale.getDefault()));
+                v.addArc(atts.getValue("to"));
+                tags.put(join, v);
+            }
+        } else if (localName.equalsIgnoreCase("decision")) {
+            state.decision = atts.getValue("name");
+            if (!tags.containsKey(state.decision)) {
+                tags.put(state.decision, new 
WorkflowActionNode(state.decision, localName.toLowerCase(Locale.getDefault())));
+            }
+        } else if (localName.equalsIgnoreCase("case")
+                || localName.equalsIgnoreCase("default")) {
+            tags.get(state.decision).addArc(atts.getValue("to"));
+        } else if (localName.equalsIgnoreCase("kill")
+                || localName.equalsIgnoreCase("end")) {
+            final String name = atts.getValue("name");
+            if (!tags.containsKey(name)) {
+                tags.put(name, new WorkflowActionNode(name, 
localName.toLowerCase(Locale.getDefault())));
+            }
+        }
+    }
+
+
+    @Override
+    public void endElement(final String namespaceURI,
+                           final String localName,
+                           final String qName)
+            throws SAXException {
+        if (localName.equalsIgnoreCase("action")) {
+            tags.put(state.action, new WorkflowActionNode(state.action, 
state.actionType));
+            tags.get(state.action).addArc(state.actionOK);
+            tags.get(state.action).addArc(state.actionErr, true);
+
+            state.reset();
+        }
+    }
+
+    private static class WorkflowParseState {
+        private String action;
+        private String actionOK;
+        private String actionErr;
+        private String actionType;
+        private String fork;
+        private String decision;
+
+        public void reset() {
+            action = null;
+            actionOK = null;
+            actionErr = null;
+            actionType = null;
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/oozie/blob/883c7556/core/src/main/resources/oozie-default.xml
----------------------------------------------------------------------
diff --git a/core/src/main/resources/oozie-default.xml 
b/core/src/main/resources/oozie-default.xml
index 2389b99..9ba8fd4 100644
--- a/core/src/main/resources/oozie-default.xml
+++ b/core/src/main/resources/oozie-default.xml
@@ -3075,6 +3075,15 @@ will be the requeue interval for the actions which are 
waiting for a long time w
             Most users should not have to change this.
         </description>
     </property>
+
+    <property>
+        <name>oozie.graphviz.timeout.seconds</name>
+        <value>60</value>
+        <description>
+            The default number of seconds Graphviz graph generation will 
timeout.
+        </description>
+    </property>
+
     <property>
         <name>oozie.launcher.default.vcores</name>
         <value>1</value>

http://git-wip-us.apache.org/repos/asf/oozie/blob/883c7556/core/src/test/java/org/apache/oozie/servlet/TestV1JobServlet.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/oozie/servlet/TestV1JobServlet.java 
b/core/src/test/java/org/apache/oozie/servlet/TestV1JobServlet.java
index ee9ab55..fe0778d 100644
--- a/core/src/test/java/org/apache/oozie/servlet/TestV1JobServlet.java
+++ b/core/src/test/java/org/apache/oozie/servlet/TestV1JobServlet.java
@@ -24,7 +24,7 @@ import org.apache.oozie.client.OozieClient;
 import org.apache.oozie.client.rest.RestConstants;
 import org.apache.oozie.client.rest.JsonTags;
 import org.apache.oozie.util.IOUtils;
-import org.apache.oozie.servlet.V1JobServlet;
+import org.apache.oozie.util.graph.OutputFormat;
 import org.json.simple.JSONObject;
 import org.json.simple.JSONValue;
 
@@ -35,6 +35,7 @@ import java.io.InputStreamReader;
 import java.net.HttpURLConnection;
 import java.net.URL;
 import java.util.HashMap;
+import java.util.Locale;
 import java.util.Map;
 import java.util.concurrent.Callable;
 
@@ -370,6 +371,7 @@ public class TestV1JobServlet extends DagServletTestCase {
             @Override
             public Void call() throws Exception {
 
+                // Without format param -> OutputFormat.PNG
                 MockDagEngineService.reset();
                 Map<String, String> params = new HashMap<String, String>();
                 params.put(RestConstants.JOB_SHOW_PARAM, 
RestConstants.JOB_SHOW_GRAPH);
@@ -378,9 +380,38 @@ public class TestV1JobServlet extends DagServletTestCase {
                 conn.setRequestMethod("GET");
                 assertEquals(HttpServletResponse.SC_OK, 
conn.getResponseCode());
                 
assertTrue(conn.getHeaderField("content-type").startsWith(RestConstants.PNG_IMAGE_CONTENT_TYPE));
-                //Assert.assertNotNull(ImageIO.read(conn.getInputStream())); 
// Can't check this as the XML is just <workflow/>
 
-                // Negative test..  should fail
+                // format=png -> OutputFormat.PNG
+                params.clear();
+                params.put(RestConstants.JOB_SHOW_PARAM, 
RestConstants.JOB_SHOW_GRAPH);
+                params.put(RestConstants.JOB_FORMAT_PARAM, 
OutputFormat.PNG.name().toLowerCase(Locale.getDefault()));
+                url = createURL(MockDagEngineService.JOB_ID + 1 + 
MockDagEngineService.JOB_ID_END, params);
+                conn = (HttpURLConnection) url.openConnection();
+                conn.setRequestMethod("GET");
+                assertEquals(HttpServletResponse.SC_OK, 
conn.getResponseCode());
+                
assertTrue(conn.getHeaderField("content-type").startsWith(RestConstants.PNG_IMAGE_CONTENT_TYPE));
+
+                // format=svg -> OutputFormat.SVG
+                params.clear();
+                params.put(RestConstants.JOB_SHOW_PARAM, 
RestConstants.JOB_SHOW_GRAPH);
+                params.put(RestConstants.JOB_FORMAT_PARAM, 
OutputFormat.SVG.name().toLowerCase(Locale.getDefault()));
+                url = createURL(MockDagEngineService.JOB_ID + 1 + 
MockDagEngineService.JOB_ID_END, params);
+                conn = (HttpURLConnection) url.openConnection();
+                conn.setRequestMethod("GET");
+                assertEquals(HttpServletResponse.SC_OK, 
conn.getResponseCode());
+                
assertTrue(conn.getHeaderField("content-type").startsWith(RestConstants.SVG_IMAGE_CONTENT_TYPE));
+
+                // format=dot -> OutputFormat.DOT
+                params.clear();
+                params.put(RestConstants.JOB_SHOW_PARAM, 
RestConstants.JOB_SHOW_GRAPH);
+                params.put(RestConstants.JOB_FORMAT_PARAM, 
OutputFormat.DOT.name().toLowerCase(Locale.getDefault()));
+                url = createURL(MockDagEngineService.JOB_ID + 1 + 
MockDagEngineService.JOB_ID_END, params);
+                conn = (HttpURLConnection) url.openConnection();
+                conn.setRequestMethod("GET");
+                assertEquals(HttpServletResponse.SC_OK, 
conn.getResponseCode());
+                
assertTrue(conn.getHeaderField("content-type").startsWith(RestConstants.TEXT_CONTENT_TYPE));
+
+                // Negative test, should fail
                 MockCoordinatorEngineService.reset();
                 params.clear();
                 params.put(RestConstants.JOB_SHOW_PARAM, 
RestConstants.JOB_SHOW_GRAPH);

http://git-wip-us.apache.org/repos/asf/oozie/blob/883c7556/core/src/test/java/org/apache/oozie/util/TestGraphGenerator.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/oozie/util/TestGraphGenerator.java 
b/core/src/test/java/org/apache/oozie/util/TestGraphGenerator.java
deleted file mode 100644
index 002e925..0000000
--- a/core/src/test/java/org/apache/oozie/util/TestGraphGenerator.java
+++ /dev/null
@@ -1,105 +0,0 @@
-/**
- * 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.oozie.util;
-
-import junit.framework.Assert;
-import org.apache.oozie.WorkflowJobBean;
-import org.apache.oozie.test.XTestCase;
-
-import javax.imageio.ImageIO;
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileOutputStream;
-import java.io.IOException;
-
-public class TestGraphGenerator extends XTestCase {
-
-    public void testConstructor() {
-        try {
-            new GraphGenerator(null, null);
-        }
-        catch (final IllegalArgumentException iae) {
-            Assert.assertTrue("Construction with illegal args failed as 
expected: " + iae.getMessage(), true);
-        }
-        try {
-            new GraphGenerator("<workflow></workflow>", null);
-        }
-        catch (final IllegalArgumentException iae) {
-            Assert.assertTrue("Construction with illegal args failed as 
expected: " + iae.getMessage(), true);
-        }
-        Assert.assertNotNull(new GraphGenerator("<workflow></workflow>", new 
WorkflowJobBean()));
-        Assert.assertNotNull(new GraphGenerator(null, new WorkflowJobBean()));
-        final WorkflowJobBean jsonWFJob = new WorkflowJobBean();
-        jsonWFJob.setAppName("My Test App");
-        jsonWFJob.setId("My Test ID");
-        Assert.assertNotNull(new GraphGenerator("<workflow></workflow>", 
jsonWFJob));
-        Assert.assertNotNull(new GraphGenerator("<workflow></workflow>", 
jsonWFJob, false));
-        Assert.assertNotNull(new GraphGenerator("<workflow></workflow>", 
jsonWFJob, true));
-    }
-
-    public void testWrite() {
-        final WorkflowJobBean jsonWFJob = new WorkflowJobBean();
-        jsonWFJob.setAppName("My Test App");
-        jsonWFJob.setId("My Test ID");
-
-        generateAndAssertPng(jsonWFJob, "graphWF.xml", false);
-
-        generateAndAssertPng(jsonWFJob, "graphWF.xml", true);
-
-        try {
-            final String content = 
IOUtils.getResourceAsString("invalidGraphWF.xml", -1);
-            final GraphGenerator g = new GraphGenerator(content, jsonWFJob, 
true);
-            g.write(new org.apache.hadoop.io.IOUtils.NullOutputStream());
-        }
-        catch (final Exception e) {
-            Assert.fail("Write PNG failed for invalidGraphWF.xml: " + 
e.getMessage());
-        }
-    }
-
-    private void generateAndAssertPng(final WorkflowJobBean workflowJob, final 
String path, final boolean showKill) {
-        try {
-            final File outputPng = File.createTempFile("graph-output", path);
-            final String content = IOUtils.getResourceAsString(path, -1);
-            final GraphGenerator g = new GraphGenerator(content, workflowJob);
-            g.write(new FileOutputStream(outputPng));
-            Assert.assertNotNull("PNG read error", ImageIO.read(new 
FileInputStream(outputPng)));
-        }
-        catch (final Exception e) {
-            Assert.fail(String.format("Read or write PNG without kill failed 
for %s: %s", path, e.getMessage()));
-        }
-    }
-
-    public void testJobDAGLimit_more() throws IOException {
-        final WorkflowJobBean jsonWFJob = new WorkflowJobBean();
-        jsonWFJob.setAppName("My Test App");
-        jsonWFJob.setId("My Test ID");
-
-        try {
-            final String content = 
IOUtils.getResourceAsString("graphWF_26_actions.xml", -1);
-            final GraphGenerator g = new GraphGenerator(content, jsonWFJob);
-            g.write(new FileOutputStream(File.createTempFile("graph-output", 
"over-limit")));
-            Assert.fail("This should not get executed");
-
-        }
-        catch (final Exception e) {
-            Assert.assertTrue(e.getMessage().startsWith(
-                    "Can't display the graph. Number of actions are more than 
display limit"));
-        }
-    }
-}

Reply via email to