You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@oozie.apache.org by vi...@apache.org on 2012/08/21 19:20:41 UTC

svn commit: r1375671 [1/2] - in /incubator/oozie/trunk: ./ client/src/main/java/org/apache/oozie/client/rest/ core/ core/src/main/java/org/apache/oozie/client/rest/ core/src/main/java/org/apache/oozie/servlet/ core/src/main/java/org/apache/oozie/util/ ...

Author: virag
Date: Tue Aug 21 17:20:40 2012
New Revision: 1375671

URL: http://svn.apache.org/viewvc?rev=1375671&view=rev
Log:
OOZIE-906 Show runtime job DAG visually in Oozie console/dashboard (vaidya via virag)

Added:
    incubator/oozie/trunk/core/src/main/java/org/apache/oozie/util/GraphGenerator.java
    incubator/oozie/trunk/core/src/test/java/org/apache/oozie/util/TestGraphGenerator.java
    incubator/oozie/trunk/core/src/test/resources/graphWF.xml
    incubator/oozie/trunk/core/src/test/resources/invalidGraphWF.xml
Modified:
    incubator/oozie/trunk/client/src/main/java/org/apache/oozie/client/rest/RestConstants.java
    incubator/oozie/trunk/core/pom.xml
    incubator/oozie/trunk/core/src/main/java/org/apache/oozie/client/rest/JsonWorkflowJob.java
    incubator/oozie/trunk/core/src/main/java/org/apache/oozie/servlet/BaseJobServlet.java
    incubator/oozie/trunk/core/src/main/java/org/apache/oozie/servlet/V0JobServlet.java
    incubator/oozie/trunk/core/src/main/java/org/apache/oozie/servlet/V1JobServlet.java
    incubator/oozie/trunk/core/src/test/java/org/apache/oozie/servlet/TestV0JobServlet.java
    incubator/oozie/trunk/core/src/test/java/org/apache/oozie/servlet/TestV1JobServlet.java
    incubator/oozie/trunk/docs/src/site/twiki/WebServicesAPI.twiki
    incubator/oozie/trunk/release-log.txt

Modified: incubator/oozie/trunk/client/src/main/java/org/apache/oozie/client/rest/RestConstants.java
URL: http://svn.apache.org/viewvc/incubator/oozie/trunk/client/src/main/java/org/apache/oozie/client/rest/RestConstants.java?rev=1375671&r1=1375670&r2=1375671&view=diff
==============================================================================
--- incubator/oozie/trunk/client/src/main/java/org/apache/oozie/client/rest/RestConstants.java (original)
+++ incubator/oozie/trunk/client/src/main/java/org/apache/oozie/client/rest/RestConstants.java Tue Aug 21 17:20:40 2012
@@ -38,6 +38,8 @@ public interface RestConstants {
 
     public static final String TEXT_CONTENT_TYPE = "text/plain";
 
+    public static final String PNG_IMAGE_CONTENT_TYPE = "image/png";
+
     public static final String ACTION_PARAM = "action";
 
     public static final String OFFSET_PARAM = "offset";
@@ -77,6 +79,10 @@ public interface RestConstants {
 
     public static final String JOB_SHOW_DEFINITION = "definition";
 
+    public static final String JOB_SHOW_GRAPH = "graph";
+
+    public static final String JOB_SHOW_KILL_PARAM = "show-kill";
+
     public static final String JOB_BUNDLE_RERUN_COORD_SCOPE_PARAM = "coord-scope";
 
     public static final String JOB_BUNDLE_RERUN_DATE_SCOPE_PARAM = "date-scope";

Modified: incubator/oozie/trunk/core/pom.xml
URL: http://svn.apache.org/viewvc/incubator/oozie/trunk/core/pom.xml?rev=1375671&r1=1375670&r2=1375671&view=diff
==============================================================================
--- incubator/oozie/trunk/core/pom.xml (original)
+++ incubator/oozie/trunk/core/pom.xml Tue Aug 21 17:20:40 2012
@@ -259,6 +259,37 @@
             <scope>compile</scope>
         </dependency>
 
+        <!-- For drawing runtime DAG -->
+        <dependency>
+            <groupId>net.sf.jung</groupId>
+            <artifactId>jung-graph-impl</artifactId>
+            <version>2.0.1</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>net.sf.jung</groupId>
+            <artifactId>jung-api</artifactId>
+            <version>2.0.1</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>net.sf.jung</groupId>
+            <artifactId>jung-visualization</artifactId>
+            <version>2.0.1</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>net.sf.jung</groupId>
+            <artifactId>jung-algorithms</artifactId>
+            <version>2.0.1</version>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>net.sourceforge.collections</groupId>
+            <artifactId>collections-generic</artifactId>
+            <version>4.01</version>
+            <scope>compile</scope>
+        </dependency>
     </dependencies>
 
     <build>

Modified: incubator/oozie/trunk/core/src/main/java/org/apache/oozie/client/rest/JsonWorkflowJob.java
URL: http://svn.apache.org/viewvc/incubator/oozie/trunk/core/src/main/java/org/apache/oozie/client/rest/JsonWorkflowJob.java?rev=1375671&r1=1375670&r2=1375671&view=diff
==============================================================================
--- incubator/oozie/trunk/core/src/main/java/org/apache/oozie/client/rest/JsonWorkflowJob.java (original)
+++ incubator/oozie/trunk/core/src/main/java/org/apache/oozie/client/rest/JsonWorkflowJob.java Tue Aug 21 17:20:40 2012
@@ -1,4 +1,4 @@
-/**
+    /**
  * 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

Modified: incubator/oozie/trunk/core/src/main/java/org/apache/oozie/servlet/BaseJobServlet.java
URL: http://svn.apache.org/viewvc/incubator/oozie/trunk/core/src/main/java/org/apache/oozie/servlet/BaseJobServlet.java?rev=1375671&r1=1375670&r2=1375671&view=diff
==============================================================================
--- incubator/oozie/trunk/core/src/main/java/org/apache/oozie/servlet/BaseJobServlet.java (original)
+++ incubator/oozie/trunk/core/src/main/java/org/apache/oozie/servlet/BaseJobServlet.java Tue Aug 21 17:20:40 2012
@@ -20,15 +20,11 @@ package org.apache.oozie.servlet;
 import java.io.IOException;
 import java.util.Arrays;
 
-import javax.servlet.ServletConfig;
 import javax.servlet.ServletException;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
 import org.apache.oozie.BaseEngineException;
 import org.apache.oozie.ErrorCode;
 import org.apache.oozie.client.OozieClient;
@@ -37,8 +33,6 @@ import org.apache.oozie.client.rest.Json
 import org.apache.oozie.client.rest.RestConstants;
 import org.apache.oozie.service.AuthorizationException;
 import org.apache.oozie.service.AuthorizationService;
-import org.apache.oozie.service.HadoopAccessorException;
-import org.apache.oozie.service.HadoopAccessorService;
 import org.apache.oozie.service.Services;
 import org.apache.oozie.service.XLogService;
 import org.apache.oozie.util.ConfigUtils;
@@ -253,6 +247,11 @@ public abstract class BaseJobServlet ext
             response.setStatus(HttpServletResponse.SC_OK);
             response.getWriter().write(wfDefinition);
         }
+        else if (show.equals(RestConstants.JOB_SHOW_GRAPH)) {
+            stopCron();
+            streamJobGraph(request, response);
+            startCron(); // -- should happen before you stream anything in response?
+        }
         else {
             throw new XServletException(HttpServletResponse.SC_BAD_REQUEST, ErrorCode.E0303,
                     RestConstants.JOB_SHOW_PARAM, show);
@@ -362,4 +361,14 @@ public abstract class BaseJobServlet ext
     abstract void streamJobLog(HttpServletRequest request, HttpServletResponse response) throws XServletException,
             IOException;
 
+    /**
+     * abstract method to create and stream image for runtime DAG -- workflow only
+     *
+     * @param request
+     * @param response
+     * @throws XServletException
+     * @throws IOException
+     */
+    abstract void streamJobGraph(HttpServletRequest request, HttpServletResponse response)
+            throws XServletException, IOException;
 }

Modified: incubator/oozie/trunk/core/src/main/java/org/apache/oozie/servlet/V0JobServlet.java
URL: http://svn.apache.org/viewvc/incubator/oozie/trunk/core/src/main/java/org/apache/oozie/servlet/V0JobServlet.java?rev=1375671&r1=1375670&r2=1375671&view=diff
==============================================================================
--- incubator/oozie/trunk/core/src/main/java/org/apache/oozie/servlet/V0JobServlet.java (original)
+++ incubator/oozie/trunk/core/src/main/java/org/apache/oozie/servlet/V0JobServlet.java Tue Aug 21 17:20:40 2012
@@ -200,4 +200,13 @@ public class V0JobServlet extends BaseJo
         }
     }
 
-}
+    /*
+     * Not implemented in v0
+     */
+    @Override
+    protected void streamJobGraph(HttpServletRequest request, HttpServletResponse response)
+            throws XServletException, IOException {
+        // Should this error code be NOT_IMPLEMENTED?
+        throw new XServletException(HttpServletResponse.SC_BAD_REQUEST, ErrorCode.E0306);
+    }
+}
\ No newline at end of file

Modified: incubator/oozie/trunk/core/src/main/java/org/apache/oozie/servlet/V1JobServlet.java
URL: http://svn.apache.org/viewvc/incubator/oozie/trunk/core/src/main/java/org/apache/oozie/servlet/V1JobServlet.java?rev=1375671&r1=1375670&r2=1375671&view=diff
==============================================================================
--- incubator/oozie/trunk/core/src/main/java/org/apache/oozie/servlet/V1JobServlet.java (original)
+++ incubator/oozie/trunk/core/src/main/java/org/apache/oozie/servlet/V1JobServlet.java Tue Aug 21 17:20:40 2012
@@ -19,36 +19,23 @@ package org.apache.oozie.servlet;
 
 import java.io.IOException;
 import java.util.List;
-
 import javax.servlet.ServletInputStream;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
-
 import org.apache.hadoop.conf.Configuration;
-import org.apache.oozie.BaseEngineException;
-import org.apache.oozie.BundleEngine;
-import org.apache.oozie.BundleEngineException;
-import org.apache.oozie.CoordinatorActionBean;
-import org.apache.oozie.CoordinatorActionInfo;
-import org.apache.oozie.CoordinatorEngine;
-import org.apache.oozie.CoordinatorEngineException;
+import org.apache.oozie.*;
+import org.apache.oozie.client.rest.*;
 import org.apache.oozie.command.CommandException;
 import org.apache.oozie.command.coord.CoordRerunXCommand;
-import org.apache.oozie.coord.CoordUtils;
-import org.apache.oozie.DagEngine;
-import org.apache.oozie.DagEngineException;
-import org.apache.oozie.ErrorCode;
-import org.apache.oozie.client.rest.JsonBean;
-import org.apache.oozie.client.rest.JsonCoordinatorJob;
-import org.apache.oozie.client.rest.JsonTags;
-import org.apache.oozie.client.rest.RestConstants;
 import org.apache.oozie.service.BundleEngineService;
 import org.apache.oozie.service.CoordinatorEngineService;
 import org.apache.oozie.service.DagEngineService;
 import org.apache.oozie.service.Services;
+import org.apache.oozie.util.GraphGenerator;
 import org.apache.oozie.util.XLog;
 import org.json.simple.JSONObject;
 
+
 @SuppressWarnings("serial")
 public class V1JobServlet extends BaseJobServlet {
 
@@ -280,6 +267,31 @@ public class V1JobServlet extends BaseJo
         }
     }
 
+    @Override
+    protected void streamJobGraph(HttpServletRequest request, HttpServletResponse response)
+            throws XServletException, IOException {
+        String jobId = getResourceName(request);
+        if (jobId.endsWith("-W")) {
+            // Applicable only to worflow, for now
+            response.setContentType(RestConstants.PNG_IMAGE_CONTENT_TYPE);
+            try {
+                String showKill = request.getParameter(RestConstants.JOB_SHOW_KILL_PARAM);
+                boolean sK = showKill != null && (showKill.equalsIgnoreCase("yes") || showKill.equals("1") || showKill.equalsIgnoreCase("true"));
+
+                new GraphGenerator(
+                        getWorkflowJobDefinition(request, response),
+                        (JsonWorkflowJob)getWorkflowJob(request, response),
+                        sK).write(response.getOutputStream());
+            }
+            catch (Exception e) {
+                throw new XServletException(HttpServletResponse.SC_INTERNAL_SERVER_ERROR, ErrorCode.E0307, e);
+            }
+        }
+        else {
+            throw new XServletException(HttpServletResponse.SC_BAD_REQUEST, ErrorCode.E0306);
+        }
+    }
+
     /**
      * Start wf job
      *

Added: incubator/oozie/trunk/core/src/main/java/org/apache/oozie/util/GraphGenerator.java
URL: http://svn.apache.org/viewvc/incubator/oozie/trunk/core/src/main/java/org/apache/oozie/util/GraphGenerator.java?rev=1375671&view=auto
==============================================================================
--- incubator/oozie/trunk/core/src/main/java/org/apache/oozie/util/GraphGenerator.java (added)
+++ incubator/oozie/trunk/core/src/main/java/org/apache/oozie/util/GraphGenerator.java Tue Aug 21 17:20:40 2012
@@ -0,0 +1,716 @@
+/*
+ * Copyright 2012 Apache Software Foundation.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package 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 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;
+import javax.imageio.ImageIO;
+import javax.xml.parsers.SAXParser;
+import javax.xml.parsers.SAXParserFactory;
+import org.apache.commons.collections15.Transformer;
+import org.apache.oozie.client.WorkflowAction;
+import org.apache.oozie.client.WorkflowAction.Status;
+import org.apache.oozie.client.WorkflowJob;
+import org.apache.oozie.client.rest.JsonWorkflowJob;
+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;
+
+/**
+ * Class to generate and plot runtime workflow DAG
+ */
+public class GraphGenerator {
+
+    private String xml;
+    private JsonWorkflowJob job;
+    private boolean showKill = false;
+
+    /**
+     * 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, JsonWorkflowJob 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, JsonWorkflowJob job) {
+        this(xml, job, false);
+    }
+
+    /**
+     * Overridden to thwart finalizer attack
+     */
+    @Override
+    public 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.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);
+            }
+        }
+
+        @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()));
+                }
+            }
+        }
+
+        @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;
+                }
+            }
+        }
+    }
+}
\ No newline at end of file

Modified: incubator/oozie/trunk/core/src/test/java/org/apache/oozie/servlet/TestV0JobServlet.java
URL: http://svn.apache.org/viewvc/incubator/oozie/trunk/core/src/test/java/org/apache/oozie/servlet/TestV0JobServlet.java?rev=1375671&r1=1375670&r2=1375671&view=diff
==============================================================================
--- incubator/oozie/trunk/core/src/test/java/org/apache/oozie/servlet/TestV0JobServlet.java (original)
+++ incubator/oozie/trunk/core/src/test/java/org/apache/oozie/servlet/TestV0JobServlet.java Tue Aug 21 17:20:40 2012
@@ -36,6 +36,9 @@ import java.util.HashMap;
 import java.util.Map;
 import java.util.concurrent.Callable;
 
+import java.util.List;
+import org.apache.oozie.ErrorCode;
+
 public class TestV0JobServlet extends DagServletTestCase {
 
     static {
@@ -199,4 +202,24 @@ public class TestV0JobServlet extends Da
             }
         });
     }
-}
+
+    public void testGraph() throws Exception {
+        runTest("/v0/job/*", V0JobServlet.class, IS_SECURITY_ENABLED, new Callable<Void>() {
+            @Override
+            public Void call() throws Exception {
+
+                MockDagEngineService.reset();
+                Map<String, String> params = new HashMap<String, String>();
+                params.put(RestConstants.JOB_SHOW_PARAM, RestConstants.JOB_SHOW_GRAPH);
+                URL url = createURL(MockDagEngineService.JOB_ID + 1 + MockDagEngineService.JOB_ID_END, params);
+                HttpURLConnection conn = (HttpURLConnection) url.openConnection();
+                conn.setRequestMethod("GET");
+
+                assertEquals(HttpServletResponse.SC_BAD_REQUEST, conn.getResponseCode());
+                assertEquals(ErrorCode.E0306.name(), conn.getHeaderField(RestConstants.OOZIE_ERROR_CODE));
+
+                return null;
+            }
+        });
+    }
+}
\ No newline at end of file

Modified: incubator/oozie/trunk/core/src/test/java/org/apache/oozie/servlet/TestV1JobServlet.java
URL: http://svn.apache.org/viewvc/incubator/oozie/trunk/core/src/test/java/org/apache/oozie/servlet/TestV1JobServlet.java?rev=1375671&r1=1375670&r2=1375671&view=diff
==============================================================================
--- incubator/oozie/trunk/core/src/test/java/org/apache/oozie/servlet/TestV1JobServlet.java (original)
+++ incubator/oozie/trunk/core/src/test/java/org/apache/oozie/servlet/TestV1JobServlet.java Tue Aug 21 17:20:40 2012
@@ -34,7 +34,12 @@ import java.net.HttpURLConnection;
 import java.net.URL;
 import java.util.HashMap;
 import java.util.Map;
+import java.util.List;
+import java.util.Enumeration;
 import java.util.concurrent.Callable;
+import javax.imageio.ImageIO;
+import junit.framework.Assert;
+import org.apache.oozie.ErrorCode;
 
 public class TestV1JobServlet extends DagServletTestCase {
 
@@ -302,4 +307,34 @@ public class TestV1JobServlet extends Da
             }
         });
     }
+
+    public void testGraph() throws Exception {
+        runTest("/v1/job/*", V1JobServlet.class, IS_SECURITY_ENABLED, new Callable<Void>() {
+            @Override
+            public Void call() throws Exception {
+
+                MockDagEngineService.reset();
+                Map<String, String> params = new HashMap<String, String>();
+                params.put(RestConstants.JOB_SHOW_PARAM, RestConstants.JOB_SHOW_GRAPH);
+                URL url = createURL(MockDagEngineService.JOB_ID + 1 + MockDagEngineService.JOB_ID_END, params);
+                HttpURLConnection conn = (HttpURLConnection) url.openConnection();
+                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
+                MockCoordinatorEngineService.reset();
+                params.clear();
+                params.put(RestConstants.JOB_SHOW_PARAM, RestConstants.JOB_SHOW_GRAPH);
+                url = createURL(MockCoordinatorEngineService.JOB_ID + 1, params);
+                conn = (HttpURLConnection) url.openConnection();
+                conn.setRequestMethod("GET");
+                assertEquals(HttpServletResponse.SC_BAD_REQUEST, conn.getResponseCode());
+                assertEquals(ErrorCode.E0306.name(), conn.getHeaderField(RestConstants.OOZIE_ERROR_CODE));
+
+                return null;
+            }
+        });
+    }
 }

Added: incubator/oozie/trunk/core/src/test/java/org/apache/oozie/util/TestGraphGenerator.java
URL: http://svn.apache.org/viewvc/incubator/oozie/trunk/core/src/test/java/org/apache/oozie/util/TestGraphGenerator.java?rev=1375671&view=auto
==============================================================================
--- incubator/oozie/trunk/core/src/test/java/org/apache/oozie/util/TestGraphGenerator.java (added)
+++ incubator/oozie/trunk/core/src/test/java/org/apache/oozie/util/TestGraphGenerator.java Tue Aug 21 17:20:40 2012
@@ -0,0 +1,117 @@
+/*
+ * Copyright 2012 Apache Software Foundation.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.oozie.util;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.MappedByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.charset.Charset;
+import javax.imageio.ImageIO;
+import junit.framework.Assert;
+import org.apache.oozie.client.rest.JsonWorkflowJob;
+import org.apache.oozie.test.XTestCase;
+
+public class TestGraphGenerator extends XTestCase {
+
+    public void testConstructor() {
+        try {
+            GraphGenerator graphGenerator = new GraphGenerator(null, null);
+        } catch(IllegalArgumentException iae) {
+            Assert.assertTrue("Construction with illegal args failed as expected: " + iae.getMessage(), true);
+        }
+        try {
+            GraphGenerator graphGenerator1 = new GraphGenerator("<workflow></workflow>", null);
+        } catch(IllegalArgumentException iae) {
+            Assert.assertTrue("Construction with illegal args failed as expected: " + iae.getMessage(), true);
+        }
+        Assert.assertNotNull(new GraphGenerator("<workflow></workflow>", new JsonWorkflowJob()));
+        Assert.assertNotNull(new GraphGenerator(null, new JsonWorkflowJob()));
+        JsonWorkflowJob jsonWFJob = new JsonWorkflowJob();
+        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() {
+        JsonWorkflowJob jsonWFJob = new JsonWorkflowJob();
+        jsonWFJob.setAppName("My Test App");
+        jsonWFJob.setId("My Test ID");
+        String png1 = "src/test/resources/tmp1.png";
+        String png2 = "src/test/resources/tmp2.png";
+
+        try {
+            GraphGenerator g = new GraphGenerator(readFile("src/test/resources/graphWF.xml"), jsonWFJob);
+            g.write(new FileOutputStream(new File(png1)));
+        } catch(Exception e) {
+            Assert.fail("Write PNG failed for graphWF.xml: " + e.getMessage());
+        }
+
+        File f1 = new File(png1);
+        try {
+            // Check if a valid file was written
+            Assert.assertNotNull(ImageIO.read(f1));
+        } catch(IOException io) {
+            Assert.fail("Not a valid PNG: " + io.getMessage());
+        }
+
+        try {
+            GraphGenerator g = new GraphGenerator(readFile("src/test/resources/graphWF.xml"), jsonWFJob, true);
+            g.write(new FileOutputStream(new File(png2)));
+        } catch(Exception e) {
+            Assert.fail("Write PNG failed for graphWF.xml: " + e.getMessage());
+        }
+
+        File f2 = new File(png2);
+        try {
+            // Check if a valid file was written
+            Assert.assertNotNull(ImageIO.read(f2));
+        } catch(IOException io) {
+            Assert.fail("Not a valid PNG: " + io.getMessage());
+        }
+
+        Assert.assertTrue(f1.length() < f2.length());
+        f1.delete();
+        f2.delete();
+
+        try {
+            GraphGenerator g = new GraphGenerator(readFile("src/test/resources/invalidGraphWF.xml"), jsonWFJob, true);
+            g.write(new FileOutputStream(new File("src/test/resources/invalid.png")));
+        } catch(Exception e) {
+            Assert.fail("Write PNG failed for invalidGraphWF.xml: " + e.getMessage());
+        }
+        new File("src/test/resources/invalid.png").delete();
+    }
+
+    private static String readFile(String path) throws IOException {
+        File f = new File(path);
+        System.out.println(f.getAbsolutePath());
+        FileInputStream stream = new FileInputStream(f);
+        try {
+            FileChannel fc = stream.getChannel();
+            MappedByteBuffer bb = fc.map(FileChannel.MapMode.READ_ONLY, 0, fc.size());
+            /* Instead of using default, pass in a decoder. */
+            return Charset.defaultCharset().decode(bb).toString();
+        }
+        finally {
+            stream.close();
+        }
+    }
+}
\ No newline at end of file

Added: incubator/oozie/trunk/core/src/test/resources/graphWF.xml
URL: http://svn.apache.org/viewvc/incubator/oozie/trunk/core/src/test/resources/graphWF.xml?rev=1375671&view=auto
==============================================================================
--- incubator/oozie/trunk/core/src/test/resources/graphWF.xml (added)
+++ incubator/oozie/trunk/core/src/test/resources/graphWF.xml Tue Aug 21 17:20:40 2012
@@ -0,0 +1,490 @@
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+<workflow-app xmlns="uri:oozie:workflow:0.2" name="sm3-segment-3908-251483">
+    <start to="or_0_1"/>
+    <kill name="kill">
+        <message>killed message</message>
+    </kill>
+    <action name="or_0_1">
+        <pig>
+            <job-tracker>${jobTracker}</job-tracker>
+            <name-node>${nameNode}</name-node>
+            <prepare>
+                <delete path="${nameNode}/projects/sm3/root/request/251483/output/pig/or_0_1"/>
+            </prepare>
+            <configuration>
+                <property>
+                    <name>mapreduce.reduce.input.limit</name>
+                    <value>-1</value>
+                </property>
+                <property>
+                    <name>mapred.job.queue.name</name>
+                    <value>${queueName}</value>
+                </property>
+                <property>
+                    <name>mapreduce.job.acl-view-job</name>
+                    <value>${SM_H20S_VIEWJOB}</value>
+                </property>
+                <property>
+                    <name>oozie.launcher.mapreduce.job.acl-view-job</name>
+                    <value>${SM_H20S_VIEWJOB}</value>
+                </property>
+                <property>
+                    <name>mapred.map.tasks.speculative.execution</name>
+                    <value>false</value>
+                </property>
+                <property>
+                    <name>mapred.reduce.tasks.speculative.execution</name>
+                    <value>false</value>
+                </property>
+                <property>
+                    <name>oozie.launcher.mapred.child.java.opts</name>
+                    <value>${OOZIE_LAUNCHER_MAPRED_CHILD_JAVA_OPTS}</value>
+                </property>
+                <property>
+                    <name>SM3.DUMMYSID</name>
+                    <value>FALSE</value>
+                </property>
+            </configuration>
+            <script>script/pig/sm3-segment-3908-251483-or_0_1.pig</script>
+            <param>REDUCERS=15</param>
+            <param>OUTPUT=/projects/sm3/root/request/251483/output/pig/or_0_1</param>
+            <param>INPUT1=/projects/sm3/root/segment/235/2/20120308_20120308</param>
+            <param>USERTYPE1=yuid</param>
+            <param>INPUT2=/projects/sm3/root/segment/236/2/20120308_20120308</param>
+            <param>USERTYPE2=yuid</param>
+        </pig>
+        <ok to="or_or_0_1_2"/>
+        <error to="kill"/>
+    </action>
+    <action name="or_or_0_1_2">
+        <pig>
+            <job-tracker>${jobTracker}</job-tracker>
+            <name-node>${nameNode}</name-node>
+            <prepare>
+                <delete path="${nameNode}/projects/sm3/root/request/251483/output/pig/or_or_0_1_2"/>
+            </prepare>
+            <configuration>
+                <property>
+                    <name>mapreduce.reduce.input.limit</name>
+                    <value>-1</value>
+                </property>
+                <property>
+                    <name>mapred.job.queue.name</name>
+                    <value>${queueName}</value>
+                </property>
+                <property>
+                    <name>mapreduce.job.acl-view-job</name>
+                    <value>${SM_H20S_VIEWJOB}</value>
+                </property>
+                <property>
+                    <name>oozie.launcher.mapreduce.job.acl-view-job</name>
+                    <value>${SM_H20S_VIEWJOB}</value>
+                </property>
+                <property>
+                    <name>mapred.map.tasks.speculative.execution</name>
+                    <value>false</value>
+                </property>
+                <property>
+                    <name>mapred.reduce.tasks.speculative.execution</name>
+                    <value>false</value>
+                </property>
+                <property>
+                    <name>oozie.launcher.mapred.child.java.opts</name>
+                    <value>${OOZIE_LAUNCHER_MAPRED_CHILD_JAVA_OPTS}</value>
+                </property>
+                <property>
+                    <name>SM3.DUMMYSID</name>
+                    <value>FALSE</value>
+                </property>
+            </configuration>
+            <script>script/pig/sm3-segment-3908-251483-or_or_0_1_2.pig</script>
+            <param>REDUCERS=15</param>
+            <param>OUTPUT=/projects/sm3/root/request/251483/output/pig/or_or_0_1_2</param>
+            <param>INPUT1=/projects/sm3/root/request/251483/output/pig/or_0_1</param>
+            <param>USERTYPE1=yuid</param>
+            <param>INPUT2=/projects/sm3/root/segment/237/2/20120308_20120308</param>
+            <param>USERTYPE2=yuid</param>
+        </pig>
+        <ok to="or_or_0_1_3"/>
+        <error to="kill"/>
+    </action>
+    <action name="or_or_0_1_3">
+        <pig>
+            <job-tracker>${jobTracker}</job-tracker>
+            <name-node>${nameNode}</name-node>
+            <prepare>
+                <delete path="${nameNode}/projects/sm3/root/request/251483/output/pig/or_or_0_1_3"/>
+            </prepare>
+            <configuration>
+                <property>
+                    <name>mapreduce.reduce.input.limit</name>
+                    <value>-1</value>
+                </property>
+                <property>
+                    <name>mapred.job.queue.name</name>
+                    <value>${queueName}</value>
+                </property>
+                <property>
+                    <name>mapreduce.job.acl-view-job</name>
+                    <value>${SM_H20S_VIEWJOB}</value>
+                </property>
+                <property>
+                    <name>oozie.launcher.mapreduce.job.acl-view-job</name>
+                    <value>${SM_H20S_VIEWJOB}</value>
+                </property>
+                <property>
+                    <name>mapred.map.tasks.speculative.execution</name>
+                    <value>false</value>
+                </property>
+                <property>
+                    <name>mapred.reduce.tasks.speculative.execution</name>
+                    <value>false</value>
+                </property>
+                <property>
+                    <name>oozie.launcher.mapred.child.java.opts</name>
+                    <value>${OOZIE_LAUNCHER_MAPRED_CHILD_JAVA_OPTS}</value>
+                </property>
+                <property>
+                    <name>SM3.DUMMYSID</name>
+                    <value>FALSE</value>
+                </property>
+            </configuration>
+            <script>script/pig/sm3-segment-3908-251483-or_or_0_1_3.pig</script>
+            <param>REDUCERS=15</param>
+            <param>OUTPUT=/projects/sm3/root/request/251483/output/pig/or_or_0_1_3</param>
+            <param>INPUT1=/projects/sm3/root/request/251483/output/pig/or_or_0_1_2</param>
+            <param>USERTYPE1=yuid</param>
+            <param>INPUT2=/projects/sm3/root/segment/238/2/20120308_20120308</param>
+            <param>USERTYPE2=yuid</param>
+        </pig>
+        <ok to="or_or_0_1_4"/>
+        <error to="kill"/>
+    </action>
+    <action name="or_or_0_1_4">
+        <pig>
+            <job-tracker>${jobTracker}</job-tracker>
+            <name-node>${nameNode}</name-node>
+            <prepare>
+                <delete path="${nameNode}/projects/sm3/root/request/251483/output/pig/or_or_0_1_4"/>
+            </prepare>
+            <configuration>
+                <property>
+                    <name>mapreduce.reduce.input.limit</name>
+                    <value>-1</value>
+                </property>
+                <property>
+                    <name>mapred.job.queue.name</name>
+                    <value>${queueName}</value>
+                </property>
+                <property>
+                    <name>mapreduce.job.acl-view-job</name>
+                    <value>${SM_H20S_VIEWJOB}</value>
+                </property>
+                <property>
+                    <name>oozie.launcher.mapreduce.job.acl-view-job</name>
+                    <value>${SM_H20S_VIEWJOB}</value>
+                </property>
+                <property>
+                    <name>mapred.map.tasks.speculative.execution</name>
+                    <value>false</value>
+                </property>
+                <property>
+                    <name>mapred.reduce.tasks.speculative.execution</name>
+                    <value>false</value>
+                </property>
+                <property>
+                    <name>oozie.launcher.mapred.child.java.opts</name>
+                    <value>${OOZIE_LAUNCHER_MAPRED_CHILD_JAVA_OPTS}</value>
+                </property>
+                <property>
+                    <name>SM3.DUMMYSID</name>
+                    <value>FALSE</value>
+                </property>
+            </configuration>
+            <script>script/pig/sm3-segment-3908-251483-or_or_0_1_4.pig</script>
+            <param>REDUCERS=15</param>
+            <param>OUTPUT=/projects/sm3/root/request/251483/output/pig/or_or_0_1_4</param>
+            <param>INPUT1=/projects/sm3/root/request/251483/output/pig/or_or_0_1_3</param>
+            <param>USERTYPE1=yuid</param>
+            <param>INPUT2=/projects/sm3/root/segment/239/2/20120308_20120308</param>
+            <param>USERTYPE2=yuid</param>
+        </pig>
+        <ok to="or_or_0_1_5"/>
+        <error to="kill"/>
+    </action>
+    <action name="or_or_0_1_5">
+        <pig>
+            <job-tracker>${jobTracker}</job-tracker>
+            <name-node>${nameNode}</name-node>
+            <prepare>
+                <delete path="${nameNode}/projects/sm3/root/request/251483/output/pig/or_or_0_1_5"/>
+            </prepare>
+            <configuration>
+                <property>
+                    <name>mapreduce.reduce.input.limit</name>
+                    <value>-1</value>
+                </property>
+                <property>
+                    <name>mapred.job.queue.name</name>
+                    <value>${queueName}</value>
+                </property>
+                <property>
+                    <name>mapreduce.job.acl-view-job</name>
+                    <value>${SM_H20S_VIEWJOB}</value>
+                </property>
+                <property>
+                    <name>oozie.launcher.mapreduce.job.acl-view-job</name>
+                    <value>${SM_H20S_VIEWJOB}</value>
+                </property>
+                <property>
+                    <name>mapred.map.tasks.speculative.execution</name>
+                    <value>false</value>
+                </property>
+                <property>
+                    <name>mapred.reduce.tasks.speculative.execution</name>
+                    <value>false</value>
+                </property>
+                <property>
+                    <name>oozie.launcher.mapred.child.java.opts</name>
+                    <value>${OOZIE_LAUNCHER_MAPRED_CHILD_JAVA_OPTS}</value>
+                </property>
+                <property>
+                    <name>SM3.DUMMYSID</name>
+                    <value>FALSE</value>
+                </property>
+            </configuration>
+            <script>script/pig/sm3-segment-3908-251483-or_or_0_1_5.pig</script>
+            <param>REDUCERS=15</param>
+            <param>OUTPUT=/projects/sm3/root/request/251483/output/pig/or_or_0_1_5</param>
+            <param>INPUT1=/projects/sm3/root/request/251483/output/pig/or_or_0_1_4</param>
+            <param>USERTYPE1=yuid</param>
+            <param>INPUT2=/projects/sm3/root/segment/240/2/20120308_20120308</param>
+            <param>USERTYPE2=yuid</param>
+        </pig>
+        <ok to="or_or_0_1_6"/>
+        <error to="kill"/>
+    </action>
+    <action name="or_or_0_1_6">
+        <pig>
+            <job-tracker>${jobTracker}</job-tracker>
+            <name-node>${nameNode}</name-node>
+            <prepare>
+                <delete path="${nameNode}/projects/sm3/root/request/251483/output/pig/or_or_0_1_6"/>
+            </prepare>
+            <configuration>
+                <property>
+                    <name>mapreduce.reduce.input.limit</name>
+                    <value>-1</value>
+                </property>
+                <property>
+                    <name>mapred.job.queue.name</name>
+                    <value>${queueName}</value>
+                </property>
+                <property>
+                    <name>mapreduce.job.acl-view-job</name>
+                    <value>${SM_H20S_VIEWJOB}</value>
+                </property>
+                <property>
+                    <name>oozie.launcher.mapreduce.job.acl-view-job</name>
+                    <value>${SM_H20S_VIEWJOB}</value>
+                </property>
+                <property>
+                    <name>mapred.map.tasks.speculative.execution</name>
+                    <value>false</value>
+                </property>
+                <property>
+                    <name>mapred.reduce.tasks.speculative.execution</name>
+                    <value>false</value>
+                </property>
+                <property>
+                    <name>oozie.launcher.mapred.child.java.opts</name>
+                    <value>${OOZIE_LAUNCHER_MAPRED_CHILD_JAVA_OPTS}</value>
+                </property>
+                <property>
+                    <name>SM3.DUMMYSID</name>
+                    <value>FALSE</value>
+                </property>
+            </configuration>
+            <script>script/pig/sm3-segment-3908-251483-or_or_0_1_6.pig</script>
+            <param>REDUCERS=15</param>
+            <param>OUTPUT=/projects/sm3/root/request/251483/output/pig/or_or_0_1_6</param>
+            <param>INPUT1=/projects/sm3/root/request/251483/output/pig/or_or_0_1_5</param>
+            <param>USERTYPE1=yuid</param>
+            <param>INPUT2=/projects/sm3/root/segment/241/2/20120308_20120308</param>
+            <param>USERTYPE2=yuid</param>
+        </pig>
+        <ok to="or_or_0_1_7"/>
+        <error to="kill"/>
+    </action>
+    <action name="or_or_0_1_7">
+        <pig>
+            <job-tracker>${jobTracker}</job-tracker>
+            <name-node>${nameNode}</name-node>
+            <prepare>
+                <delete path="${nameNode}/projects/sm3/root/request/251483/output/pig/or_or_0_1_7"/>
+            </prepare>
+            <configuration>
+                <property>
+                    <name>mapreduce.reduce.input.limit</name>
+                    <value>-1</value>
+                </property>
+                <property>
+                    <name>mapred.job.queue.name</name>
+                    <value>${queueName}</value>
+                </property>
+                <property>
+                    <name>mapreduce.job.acl-view-job</name>
+                    <value>${SM_H20S_VIEWJOB}</value>
+                </property>
+                <property>
+                    <name>oozie.launcher.mapreduce.job.acl-view-job</name>
+                    <value>${SM_H20S_VIEWJOB}</value>
+                </property>
+                <property>
+                    <name>mapred.map.tasks.speculative.execution</name>
+                    <value>false</value>
+                </property>
+                <property>
+                    <name>mapred.reduce.tasks.speculative.execution</name>
+                    <value>false</value>
+                </property>
+                <property>
+                    <name>oozie.launcher.mapred.child.java.opts</name>
+                    <value>${OOZIE_LAUNCHER_MAPRED_CHILD_JAVA_OPTS}</value>
+                </property>
+                <property>
+                    <name>SM3.DUMMYSID</name>
+                    <value>FALSE</value>
+                </property>
+            </configuration>
+            <script>script/pig/sm3-segment-3908-251483-or_or_0_1_7.pig</script>
+            <param>REDUCERS=15</param>
+            <param>OUTPUT=/projects/sm3/root/request/251483/output/pig/or_or_0_1_7</param>
+            <param>INPUT1=/projects/sm3/root/request/251483/output/pig/or_or_0_1_6</param>
+            <param>USERTYPE1=yuid</param>
+            <param>INPUT2=/projects/sm3/root/segment/242/2/20120308_20120308</param>
+            <param>USERTYPE2=yuid</param>
+        </pig>
+        <ok to="or_or_0_1_8"/>
+        <error to="kill"/>
+    </action>
+    <action name="or_or_0_1_8">
+        <pig>
+            <job-tracker>${jobTracker}</job-tracker>
+            <name-node>${nameNode}</name-node>
+            <prepare>
+                <delete path="${nameNode}/projects/sm3/root/request/251483/output/pig/or_or_0_1_8"/>
+            </prepare>
+            <configuration>
+                <property>
+                    <name>mapreduce.reduce.input.limit</name>
+                    <value>-1</value>
+                </property>
+                <property>
+                    <name>mapred.job.queue.name</name>
+                    <value>${queueName}</value>
+                </property>
+                <property>
+                    <name>mapreduce.job.acl-view-job</name>
+                    <value>${SM_H20S_VIEWJOB}</value>
+                </property>
+                <property>
+                    <name>oozie.launcher.mapreduce.job.acl-view-job</name>
+                    <value>${SM_H20S_VIEWJOB}</value>
+                </property>
+                <property>
+                    <name>mapred.map.tasks.speculative.execution</name>
+                    <value>false</value>
+                </property>
+                <property>
+                    <name>mapred.reduce.tasks.speculative.execution</name>
+                    <value>false</value>
+                </property>
+                <property>
+                    <name>oozie.launcher.mapred.child.java.opts</name>
+                    <value>${OOZIE_LAUNCHER_MAPRED_CHILD_JAVA_OPTS}</value>
+                </property>
+                <property>
+                    <name>SM3.DUMMYSID</name>
+                    <value>FALSE</value>
+                </property>
+            </configuration>
+            <script>script/pig/sm3-segment-3908-251483-or_or_0_1_8.pig</script>
+            <param>REDUCERS=15</param>
+            <param>OUTPUT=/projects/sm3/root/request/251483/output/pig/or_or_0_1_8</param>
+            <param>INPUT1=/projects/sm3/root/request/251483/output/pig/or_or_0_1_7</param>
+            <param>USERTYPE1=yuid</param>
+            <param>INPUT2=/projects/sm3/root/segment/243/2/20120308_20120308</param>
+            <param>USERTYPE2=yuid</param>
+        </pig>
+        <ok to="or_or_0_1_9"/>
+        <error to="kill"/>
+    </action>
+    <action name="or_or_0_1_9">
+        <pig>
+            <job-tracker>${jobTracker}</job-tracker>
+            <name-node>${nameNode}</name-node>
+            <prepare>
+                <delete path="${nameNode}/projects/sm3/root/segment/3908/8/20120308_20120308"/>
+            </prepare>
+            <configuration>
+                <property>
+                    <name>mapreduce.reduce.input.limit</name>
+                    <value>-1</value>
+                </property>
+                <property>
+                    <name>mapred.job.queue.name</name>
+                    <value>${queueName}</value>
+                </property>
+                <property>
+                    <name>mapreduce.job.acl-view-job</name>
+                    <value>${SM_H20S_VIEWJOB}</value>
+                </property>
+                <property>
+                    <name>oozie.launcher.mapreduce.job.acl-view-job</name>
+                    <value>${SM_H20S_VIEWJOB}</value>
+                </property>
+                <property>
+                    <name>mapred.map.tasks.speculative.execution</name>
+                    <value>false</value>
+                </property>
+                <property>
+                    <name>mapred.reduce.tasks.speculative.execution</name>
+                    <value>false</value>
+                </property>
+                <property>
+                    <name>oozie.launcher.mapred.child.java.opts</name>
+                    <value>${OOZIE_LAUNCHER_MAPRED_CHILD_JAVA_OPTS}</value>
+                </property>
+                <property>
+                    <name>SM3.DUMMYSID</name>
+                    <value>FALSE</value>
+                </property>
+            </configuration>
+            <script>script/pig/sm3-segment-3908-251483-or_or_0_1_9.pig</script>
+            <param>REDUCERS=15</param>
+            <param>OUTPUT=/projects/sm3/root/segment/3908/8/20120308_20120308</param>
+            <param>INPUT1=/projects/sm3/root/request/251483/output/pig/or_or_0_1_8</param>
+            <param>USERTYPE1=yuid</param>
+            <param>INPUT2=/projects/sm3/root/segment/244/2/20120308_20120308</param>
+            <param>USERTYPE2=yuid</param>
+        </pig>
+        <ok to="cleanup"/>
+        <error to="kill"/>
+    </action>
+    <action name="cleanup">
+        <fs>
+            <delete path="${nameNode}/projects/sm3/root/request/251483/output/pig/or_0_1"/>
+            <delete path="${nameNode}/projects/sm3/root/request/251483/output/pig/or_or_0_1_2"/>
+            <delete path="${nameNode}/projects/sm3/root/request/251483/output/pig/or_or_0_1_3"/>
+            <delete path="${nameNode}/projects/sm3/root/request/251483/output/pig/or_or_0_1_4"/>
+            <delete path="${nameNode}/projects/sm3/root/request/251483/output/pig/or_or_0_1_5"/>
+            <delete path="${nameNode}/projects/sm3/root/request/251483/output/pig/or_or_0_1_6"/>
+            <delete path="${nameNode}/projects/sm3/root/request/251483/output/pig/or_or_0_1_7"/>
+            <delete path="${nameNode}/projects/sm3/root/request/251483/output/pig/or_or_0_1_8"/>
+        </fs>
+        <ok to="end"/>
+        <error to="kill"/>
+    </action>
+    <end name="end"/>
+</workflow-app>
\ No newline at end of file