azkaban-developers

Changes

build.xml 56(+44 -12)

src/less/azkaban-svg.less 133(+0 -133)

src/less/flow.less 81(+70 -11)

src/less/Makefile 27(+22 -5)

src/less/project.less 29(+19 -10)

src/less/tables.less 44(+34 -10)

src/tl/Makefile 3(+2 -1)

src/web/js/azkaban/view/flow-graph.js 312(+0 -312)

src/web/js/jquery.svg.js 1394(+1394 -0)

unit/build.xml 64(+49 -15)

Details

build.xml 56(+44 -12)

diff --git a/build.xml b/build.xml
index e1f77bb..63152e5 100644
--- a/build.xml
+++ b/build.xml
@@ -6,6 +6,7 @@
 	<property name="dist.jar.dir" value="${basedir}/dist/jars" />
 	<property name="dist.dust.dir" value="${basedir}/dist/dust" />
 	<property name="dist.less.dir" value="${basedir}/dist/less" />
+	<property name="dist.web.dir" value="${basedir}/dist/web" />
 	<property name="dist.classes.dir" value="${basedir}/dist/classes" />
 	<property name="dist.packages.dir" value="${basedir}/dist/packages" />
 	<property name="dist.web.package.dir" value="${dist.packages.dir}/azkaban-web-server" />
@@ -57,13 +58,29 @@
     </exec>
 	</target>
 
-	<target name="build" description="Compile main source tree java files">
-		<delete dir="${dist.classes.dir}" />
-		<mkdir dir="${dist.classes.dir}" />
+	<target name="dust" description="Compile Less css files.">
+		<!-- Compile dustjs templates -->
 		<delete dir="${dist.dust.dir}" />
 		<mkdir dir="${dist.dust.dir}" />
+		<exec dir="${dust.src.dir}" executable="make" failonerror="true"/>
+		<copy todir="${dist.dust.dir}">
+      		<fileset dir="${dust.src.dir}/obj" includes="*.js" />
+		</copy>
+	</target>
+	
+	<target name="less" description="Compile Less css files.">
+		<!-- Compile LESS to CSS -->
 		<delete dir="${dist.less.dir}" />
 		<mkdir dir="${dist.less.dir}" />
+    	<exec dir="${less.src.dir}" executable="make" failonerror="true"/>
+		<copy todir="${dist.less.dir}" >
+      		<fileset dir="${less.src.dir}/obj" includes="*.css" />
+    	</copy>
+	</target>
+	
+	<target name="build" description="Compile main source tree java files">
+		<delete dir="${dist.classes.dir}" />
+		<mkdir dir="${dist.classes.dir}" />
 		
 		<!-- copy non-java files to classes dir to load from classpath -->
 		<copy todir="${dist.classes.dir}">
@@ -78,17 +95,32 @@
 			<classpath refid="main.classpath" />
 		</javac>
 
-    <!-- Compile dustjs templates -->
-    <exec dir="${dust.src.dir}" executable="make" failonerror="true"/>
-		<copy todir="${dist.dust.dir}">
-      <fileset dir="${dust.src.dir}/obj" includes="*.js" />
+		<antcall target="dust"></antcall>
+		<antcall target="less"></antcall>
+	</target>
+	
+	<target name="webmin" description="Copies only the non compiled web resources to dist dir">
+		<copy todir="${dist.web.dir}" overwrite="true">
+			<fileset dir="${web.src.dir}" />
+		</copy>
+	</target>
+	
+	<target name="web" description="Creates web resourses in a dir. Useful for development">
+		<mkdir dir="${dist.web.dir}" />
+		
+		<antcall target="webmin"></antcall>
+		<antcall target="dust"></antcall>
+		<antcall target="less"></antcall>
+		
+		<!-- Copy compiled dust templates -->
+		<copy todir="${dist.web.dir}/js">
+			<fileset dir="${dist.dust.dir}" />
 		</copy>
 
-		<!-- Compile LESS to CSS -->
-    <exec dir="${less.src.dir}" executable="make" failonerror="true"/>
-		<copy todir="${dist.less.dir}" >
-      <fileset dir="${less.src.dir}/obj" includes="*.css" />
-    </copy>
+		<!-- Copy compiled less CSS -->
+		<copy todir="${dist.web.dir}/css">
+			<fileset dir="${dist.less.dir}" />
+		</copy>
 	</target>
 	
 	<target name="jars" depends="build" description="Create azkaban jar">
diff --git a/src/java/azkaban/execapp/AzkabanExecutorServer.java b/src/java/azkaban/execapp/AzkabanExecutorServer.java
index 41681c0..50be4f6 100644
--- a/src/java/azkaban/execapp/AzkabanExecutorServer.java
+++ b/src/java/azkaban/execapp/AzkabanExecutorServer.java
@@ -109,6 +109,12 @@ public class AzkabanExecutorServer {
 		runnerManager.setGlobalProps(executorGlobalProps);
 		
 		configureMBeanServer();
+
+    File statsDir = new File(props.getString("azkaban.stats.dir", "stats"));
+    if (!statsDir.exists()) {
+      statsDir.mkdir();
+    }
+    props.put("azkaban.stats.dir", statsDir.getCanonicalPath());
 		
 		try {
 			server.start();
diff --git a/src/java/azkaban/execapp/event/EventHandler.java b/src/java/azkaban/execapp/event/EventHandler.java
index 9aead0a..9887176 100644
--- a/src/java/azkaban/execapp/event/EventHandler.java
+++ b/src/java/azkaban/execapp/event/EventHandler.java
@@ -16,6 +16,7 @@
 
 package azkaban.execapp.event;
 
+import java.util.ArrayList;
 import java.util.HashSet;
 
 public class EventHandler {
@@ -29,6 +30,7 @@ public class EventHandler {
 	}
 	
 	public void fireEventListeners(Event event) {
+		ArrayList<EventListener> listeners = new ArrayList<EventListener>(this.listeners);
 		for (EventListener listener: listeners) {
 			listener.handleEvent(event);
 		}
diff --git a/src/java/azkaban/execapp/event/FlowWatcher.java b/src/java/azkaban/execapp/event/FlowWatcher.java
index dabb9f0..2dbf05f 100644
--- a/src/java/azkaban/execapp/event/FlowWatcher.java
+++ b/src/java/azkaban/execapp/event/FlowWatcher.java
@@ -53,7 +53,7 @@ public abstract class FlowWatcher {
 	 * Called to fire events to the JobRunner listeners
 	 * @param jobId
 	 */
-	protected synchronized void handleJobFinished(String jobId, Status status) {
+	protected synchronized void handleJobStatusChange(String jobId, Status status) {
 		BlockingStatus block = map.get(jobId);
 		if (block != null) {
 			block.changeStatus(status);
@@ -69,7 +69,8 @@ public abstract class FlowWatcher {
 			return null;
 		}
 		
-		ExecutableNode node = flow.getExecutableNode(jobId);
+		String[] split = jobId.split(":");
+		ExecutableNode node = flow.getExecutableNode(split);
 		if (node == null) {
 			return null;
 		}
@@ -84,7 +85,8 @@ public abstract class FlowWatcher {
 	}
 	
 	public Status peekStatus(String jobId) {
-		ExecutableNode node = flow.getExecutableNode(jobId);
+		String[] split = jobId.split(":");
+		ExecutableNode node = flow.getExecutableNode(split);
 		if (node != null) {
 			return node.getStatus();
 		}
@@ -92,17 +94,17 @@ public abstract class FlowWatcher {
 		return null;
 	}
 	
-	public synchronized void failAllWatches() {
-		logger.info("Failing all watches on " + execId);
+	public synchronized void unblockAllWatches() {
+		logger.info("Unblock all watches on " + execId);
 		cancelWatch = true;
 		
 		for(BlockingStatus status : map.values()) {
 			logger.info("Unblocking " + status.getJobId());
-			status.changeStatus(Status.KILLED);
+			status.changeStatus(Status.SKIPPED);
 			status.unblock();
 		}
 		
-		logger.info("Successfully failed all watches on " + execId);
+		logger.info("Successfully unblocked all watches on " + execId);
 	}
 	
 	public boolean isWatchCancelled() {
diff --git a/src/java/azkaban/execapp/event/LocalFlowWatcher.java b/src/java/azkaban/execapp/event/LocalFlowWatcher.java
index 1ae2cc9..fc85dc8 100644
--- a/src/java/azkaban/execapp/event/LocalFlowWatcher.java
+++ b/src/java/azkaban/execapp/event/LocalFlowWatcher.java
@@ -47,7 +47,7 @@ public class LocalFlowWatcher extends FlowWatcher {
 		runner = null;
 		
 		getLogger().info("Stopping watcher, and unblocking pipeline");
-		super.failAllWatches();
+		super.unblockAllWatches();
 	}
 
 	public class LocalFlowWatcherListener implements EventListener {
@@ -55,17 +55,19 @@ public class LocalFlowWatcher extends FlowWatcher {
 		public void handleEvent(Event event) {
 			if (event.getType() == Type.JOB_FINISHED) {
 				if (event.getRunner() instanceof FlowRunner) {
+					// The flow runner will finish a job without it running
 					Object data = event.getData();
 					if (data instanceof ExecutableNode) {
 						ExecutableNode node = (ExecutableNode)data;
-						handleJobFinished(node.getJobId(), node.getStatus());
+						handleJobStatusChange(node.getNestedId(), node.getStatus());
 					}
 				}
 				else if (event.getRunner() instanceof JobRunner) {
+					// A job runner is finished
 					JobRunner runner = (JobRunner)event.getRunner();
 					ExecutableNode node = runner.getNode();
-					
-					handleJobFinished(node.getJobId(), node.getStatus());
+					System.out.println(node + " looks like " + node.getStatus());
+					handleJobStatusChange(node.getNestedId(), node.getStatus());
 				}
 			}
 			else if (event.getType() == Type.FLOW_FINISHED) {
diff --git a/src/java/azkaban/execapp/event/RemoteFlowWatcher.java b/src/java/azkaban/execapp/event/RemoteFlowWatcher.java
index 6d66c7e..b266555 100644
--- a/src/java/azkaban/execapp/event/RemoteFlowWatcher.java
+++ b/src/java/azkaban/execapp/event/RemoteFlowWatcher.java
@@ -16,6 +16,9 @@
 
 package azkaban.execapp.event;
 
+import java.util.ArrayList;
+import java.util.Map;
+
 import azkaban.executor.ExecutableFlow;
 import azkaban.executor.ExecutableNode;
 import azkaban.executor.ExecutorLoader;
@@ -70,27 +73,24 @@ public class RemoteFlowWatcher extends FlowWatcher {
 					isShutdown = true;
 				}
 				
+				long updateTime = 0;
 				if (flow == null) {
 					flow = updateFlow;
 				}
 				else {
+					Map<String, Object> updateData = updateFlow.toUpdateObject(updateTime);
+					ArrayList<ExecutableNode> updatedNodes = new ArrayList<ExecutableNode>();
+					flow.applyUpdateObject(updateData, updatedNodes);
+
 					flow.setStatus(updateFlow.getStatus());
 					flow.setEndTime(updateFlow.getEndTime());
 					flow.setUpdateTime(updateFlow.getUpdateTime());
 					
-					for (ExecutableNode node : flow.getExecutableNodes()) {
-						String jobId = node.getJobId();
-						ExecutableNode newNode = updateFlow.getExecutableNode(jobId);
-						long updateTime = node.getUpdateTime();
-						node.setUpdateTime(newNode.getUpdateTime());
-						node.setStatus(newNode.getStatus());
-						node.setStartTime(newNode.getStartTime());
-						node.setEndTime(newNode.getEndTime());
-						
-						if (updateTime < newNode.getUpdateTime()) {
-							handleJobFinished(jobId, newNode.getStatus());
-						}
+					for (ExecutableNode node : updatedNodes) {
+						handleJobStatusChange(node.getNestedId(), node.getStatus());
 					}
+					
+					updateTime = flow.getUpdateTime();
 				}
 				
 				if (Status.isStatusFinished(flow.getStatus())) {
@@ -108,7 +108,7 @@ public class RemoteFlowWatcher extends FlowWatcher {
 		}
 		
 	}
-
+	
 	@Override
 	public synchronized void stopWatcher() {
 		if(isShutdown) {
@@ -118,7 +118,7 @@ public class RemoteFlowWatcher extends FlowWatcher {
 		if (thread != null) {
 			thread.interrupt();
 		}
-		super.failAllWatches();
+		super.unblockAllWatches();
 		loader = null;
 		flow = null;
 	}
diff --git a/src/java/azkaban/execapp/ExecutorServlet.java b/src/java/azkaban/execapp/ExecutorServlet.java
index b85c922..94fb4c2 100644
--- a/src/java/azkaban/execapp/ExecutorServlet.java
+++ b/src/java/azkaban/execapp/ExecutorServlet.java
@@ -32,7 +32,7 @@ import org.apache.log4j.Logger;
 import org.codehaus.jackson.map.ObjectMapper;
 
 import azkaban.executor.ConnectorParams;
-import azkaban.executor.ExecutableFlow;
+import azkaban.executor.ExecutableFlowBase;
 import azkaban.executor.ExecutorManagerException;
 import azkaban.utils.FileIOUtils.JobMetaData;
 import azkaban.utils.FileIOUtils.LogData;
@@ -57,8 +57,7 @@ public class ExecutorServlet extends HttpServlet implements ConnectorParams {
 		application = (AzkabanExecutorServer) config.getServletContext().getAttribute(AzkabanServletContextListener.AZKABAN_SERVLET_CONTEXT_KEY);
 
 		if (application == null) {
-			throw new IllegalStateException(
-					"No batch application is defined in the servlet context!");
+			throw new IllegalStateException("No batch application is defined in the servlet context!");
 		}
 
 		flowRunnerManager = application.getFlowRunnerManager();
@@ -231,7 +230,7 @@ public class ExecutorServlet extends HttpServlet implements ConnectorParams {
 			long updateTime = JSONUtils.getLongFromObject(updateTimesList.get(i));
 			int execId = (Integer)execIDList.get(i);
 			
-			ExecutableFlow flow = flowRunnerManager.getExecutableFlow(execId);
+			ExecutableFlowBase flow = flowRunnerManager.getExecutableFlow(execId);
 			if (flow == null) {
 				Map<String, Object> errorResponse = new HashMap<String,Object>();
 				errorResponse.put(RESPONSE_ERROR, "Flow does not exist");
@@ -259,7 +258,7 @@ public class ExecutorServlet extends HttpServlet implements ConnectorParams {
 	}
 	
 	private void handleAjaxFlowStatus(Map<String, Object> respMap, int execid) {
-		ExecutableFlow flow = flowRunnerManager.getExecutableFlow(execid);
+		ExecutableFlowBase flow = flowRunnerManager.getExecutableFlow(execid);
 		if (flow == null) {
 			respMap.put(STATUS_PARAM, RESPONSE_NOTFOUND);
 		}
diff --git a/src/java/azkaban/execapp/FlowRunner.java b/src/java/azkaban/execapp/FlowRunner.java
index d7cef1f..5f38882 100644
--- a/src/java/azkaban/execapp/FlowRunner.java
+++ b/src/java/azkaban/execapp/FlowRunner.java
@@ -1,5 +1,5 @@
 /*
- * Copyright 2012 LinkedIn Corp.
+ * Copyright 2013 LinkedIn Corp
  * 
  * 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
@@ -18,7 +18,7 @@ package azkaban.execapp;
 
 import java.io.File;
 import java.io.IOException;
-import java.util.ArrayList;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -40,6 +40,7 @@ import azkaban.execapp.event.EventHandler;
 import azkaban.execapp.event.EventListener;
 import azkaban.execapp.event.FlowWatcher;
 import azkaban.executor.ExecutableFlow;
+import azkaban.executor.ExecutableFlowBase;
 import azkaban.executor.ExecutableNode;
 import azkaban.executor.ExecutionOptions;
 import azkaban.executor.ExecutionOptions.FailureAction;
@@ -69,24 +70,22 @@ public class FlowRunner extends EventHandler implements Runnable {
 	
 	private int execId;
 	private File execDir;
-	private ExecutableFlow flow;
+	private final ExecutableFlow flow;
 	private Thread flowRunnerThread;
 	private int numJobThreads = 10;
+	private ExecutionOptions.FailureAction failureAction;
 	
 	// Sync object for queuing
 	private Object mainSyncObj = new Object();
 	
 	// Properties map
 	private Map<String, Props> sharedProps = new HashMap<String, Props>();
-	private Map<String, Props> jobOutputProps = new HashMap<String, Props>();
 	
 	private Props globalProps;
-	private Props commonProps;
 	private final JobTypeManager jobtypeManager;
 	
 	private JobRunnerEventListener listener = new JobRunnerEventListener();
-	private Map<String, JobRunner> jobRunners = new ConcurrentHashMap<String, JobRunner>();
-	private Map<String, JobRunner> activeJobRunners = new ConcurrentHashMap<String, JobRunner>();
+	private Set<JobRunner> activeJobRunners = Collections.newSetFromMap(new ConcurrentHashMap<JobRunner, Boolean>());
 	
 	// Used for pipelining
 	private Integer pipelineLevel = null;
@@ -107,6 +106,10 @@ public class FlowRunner extends EventHandler implements Runnable {
 	private boolean flowCancelled = false;
 	
 	public FlowRunner(ExecutableFlow flow, ExecutorLoader executorLoader, ProjectLoader projectLoader, JobTypeManager jobtypeManager) throws ExecutorManagerException {
+		this(flow, executorLoader, projectLoader, jobtypeManager, null);
+	}
+
+	public FlowRunner(ExecutableFlow flow, ExecutorLoader executorLoader, ProjectLoader projectLoader, JobTypeManager jobtypeManager, ExecutorService executorService) throws ExecutorManagerException {
 		this.execId = flow.getExecutionId();
 		this.flow = flow;
 		this.executorLoader = executorLoader;
@@ -117,8 +120,9 @@ public class FlowRunner extends EventHandler implements Runnable {
 		ExecutionOptions options = flow.getExecutionOptions();
 		this.pipelineLevel = options.getPipelineLevel();
 		this.pipelineExecId = options.getPipelineExecutionId();
-
+		this.failureAction = options.getFailureAction();
 		this.proxyUsers = flow.getProxyUsers();
+		this.executorService = executorService;
 	}
 
 	public FlowRunner setFlowWatcher(FlowWatcher watcher) {
@@ -191,13 +195,28 @@ public class FlowRunner extends EventHandler implements Runnable {
 		}
 	}
 	
+	@SuppressWarnings("unchecked")
 	private void setupFlowExecution() {
 		int projectId = flow.getProjectId();
 		int version = flow.getVersion();
 		String flowId = flow.getFlowId();
 		
 		// Add a bunch of common azkaban properties
-		commonProps = PropsUtils.addCommonFlowProperties(flow);
+		Props commonFlowProps = PropsUtils.addCommonFlowProperties(this.globalProps, flow);
+		
+		if (flow.getJobSource() != null) {
+			String source = flow.getJobSource();
+			Props flowProps = sharedProps.get(source);
+			flowProps.setParent(commonFlowProps);
+			commonFlowProps = flowProps;
+		}
+		
+		// If there are flow overrides, we apply them now.
+		Map<String,String> flowParam = flow.getExecutionOptions().getFlowParameters();
+		if (flowParam != null && !flowParam.isEmpty()) {
+			commonFlowProps = new Props(commonFlowProps, flowParam);
+		}
+		flow.setInputProps(commonFlowProps);
 		
 		// Create execution dir
 		createLogger(flowId);
@@ -214,7 +233,6 @@ public class FlowRunner extends EventHandler implements Runnable {
 		// The current thread is used for interrupting blocks
 		flowRunnerThread = Thread.currentThread();
 		flowRunnerThread.setName("FlowRunner-exec-" + flow.getExecutionId());
-
 	}
 	
 	private void updateFlowReference() throws ExecutorManagerException {
@@ -289,11 +307,6 @@ public class FlowRunner extends EventHandler implements Runnable {
 
 				props.setParent(inherits);
 			}
-			else {
-				String source = fprops.getSource();
-				Props props = sharedProps.get(source);
-				props.setParent(globalProps);
-			}
 		}
 	}
 	
@@ -318,51 +331,7 @@ public class FlowRunner extends EventHandler implements Runnable {
 					continue;
 				}
 				else {
-					List<ExecutableNode> jobsReadyToRun = findReadyJobsToRun();
-					
-					if (!jobsReadyToRun.isEmpty() && !flowCancelled) {
-						for (ExecutableNode node : jobsReadyToRun) {
-							long currentTime = System.currentTimeMillis();
-							
-							// Queue a job only if it's ready to run.
-							if (node.getStatus() == Status.READY) {
-								// Collect output props from the job's dependencies.
-								Props outputProps = collectOutputProps(node);
-								node.setStatus(Status.QUEUED);
-								JobRunner runner = createJobRunner(node, outputProps);
-								logger.info("Submitting job " + node.getJobId() + " to run.");
-								try {
-									executorService.submit(runner);
-									jobRunners.put(node.getJobId(), runner);
-									activeJobRunners.put(node.getJobId(), runner);
-								} catch (RejectedExecutionException e) {
-									logger.error(e);
-								};
-								
-							} // If killed, then auto complete and KILL
-							else if (node.getStatus() == Status.KILLED) {
-								logger.info("Killing " + node.getJobId() + " due to prior errors.");
-								node.setStartTime(currentTime);
-								node.setEndTime(currentTime);
-								fireEventListeners(Event.create(this, Type.JOB_FINISHED, node));
-							} // If disabled, then we auto skip
-							else if (node.getStatus() == Status.DISABLED) {
-								logger.info("Skipping disabled job " + node.getJobId() + ".");
-								node.setStartTime(currentTime);
-								node.setEndTime(currentTime);
-								node.setStatus(Status.SKIPPED);
-								fireEventListeners(Event.create(this, Type.JOB_FINISHED, node));
-							}
-						}
-						
-						updateFlow();
-					}
-					else {
-						if (isFlowFinished() || flowCancelled ) {
-							flowFinished = true;
-							break;
-						}
-					
+					if (!progressGraph()) {
 						try {
 							mainSyncObj.wait(CHECK_WAIT_MS);
 						} catch (InterruptedException e) {
@@ -372,91 +341,70 @@ public class FlowRunner extends EventHandler implements Runnable {
 			}
 		}
 		
-		if (flowCancelled) {
-			try {
-				logger.info("Flow was force cancelled cleaning up.");
-				for(JobRunner activeRunner : activeJobRunners.values()) {
-					activeRunner.cancel();
-				}
-				
-				for (ExecutableNode node: flow.getExecutableNodes()) {
-					if (Status.isStatusFinished(node.getStatus())) {
-						continue;
-					}
-					else if (node.getStatus() == Status.DISABLED) {
-						node.setStatus(Status.SKIPPED);
-					}
-					else {
-						node.setStatus(Status.KILLED);
-					}
-					fireEventListeners(Event.create(this, Type.JOB_FINISHED, node));
-				}
-			} catch (Exception e) {
-				logger.error(e);
-			}
-	
-			updateFlow();
-		}
-		
 		logger.info("Finishing up flow. Awaiting Termination");
 		executorService.shutdown();
 		
-		synchronized(mainSyncObj) {
-			switch(flow.getStatus()) {
-			case FAILED_FINISHING:
-				logger.info("Setting flow status to Failed.");
-				flow.setStatus(Status.FAILED);
-			case FAILED:
-			case KILLED:
-				logger.info("Flow is set to " + flow.getStatus().toString());
-				break;
-			default:
-				flow.setStatus(Status.SUCCEEDED);
-				logger.info("Flow is set to " + flow.getStatus().toString());
-			}
-		}
+		finalizeFlow(flow);
+		updateFlow();
+		logger.info("Finished Flow");
 	}
 	
-	private List<ExecutableNode> findReadyJobsToRun() {
-		ArrayList<ExecutableNode> jobsToRun = new ArrayList<ExecutableNode>();
-		for (ExecutableNode node : flow.getExecutableNodes()) {
-			if (Status.isStatusFinished(node.getStatus())) {
-				continue;
+	private boolean progressGraph() throws IOException {
+		List<ExecutableNode> jobsReadyToRun = flow.findNextJobsToRun();
+
+		// If its the current flow
+		if (jobsReadyToRun.size() == 1 && jobsReadyToRun.get(0) == flow) {
+			flowFinished = true;
+			return true;
+		}
+
+		for (ExecutableNode node: jobsReadyToRun) {
+			Status nextStatus = getImpliedStatus(node);
+			
+			if (node instanceof ExecutableFlowBase && ((ExecutableFlowBase)node).isFlowFinished()) {
+				finalizeFlow((ExecutableFlowBase)node);
+				fireEventListeners(Event.create(this, Type.JOB_FINISHED, node));
+			}
+			else if (nextStatus == Status.KILLED || isCancelled()) {
+				logger.info("Killing " + node.getId() + " due to prior errors.");
+				node.killNode(System.currentTimeMillis());
+				fireEventListeners(Event.create(this, Type.JOB_FINISHED, node));
+			}
+			else if (nextStatus == Status.DISABLED) {
+				logger.info("Skipping disabled job " + node.getId() + ".");
+				node.skipNode(System.currentTimeMillis());
+				fireEventListeners(Event.create(this, Type.JOB_FINISHED, node));
 			}
 			else {
-				// Check the dependencies to see if execution conditions are met,
-				// and what the status should be set to.
-				Status impliedStatus = getImpliedStatus(node);
-				if (getImpliedStatus(node) != null) {
-					node.setStatus(impliedStatus);
-					jobsToRun.add(node);
-				}
+				runExecutableNode(node);
 			}
 		}
-		
-		return jobsToRun;
-	}
-
-	private boolean isFlowFinished() {
-		if (!activeJobRunners.isEmpty()) {
-			return false;
+			
+		if (!jobsReadyToRun.isEmpty()) {
+			updateFlow();
+			return true;
 		}
-		
-		for (String end: flow.getEndNodes()) {
-			ExecutableNode node = flow.getExecutableNode(end);
-			if (!Status.isStatusFinished(node.getStatus()) ) {
-				return false;
-			}
+		else {
+			return false;
 		}
-		
-		return true;
 	}
 	
-	private Props collectOutputProps(ExecutableNode node) {
+	private void finalizeFlow(ExecutableFlowBase flow) {
+		String id = flow == this.flow ? "" : flow.getNestedId() + " ";
+
+		// If it's not the starting flow, we'll create set of output props
+		// for the finished flow.
+		boolean succeeded = true;
 		Props previousOutput = null;
-		// Iterate the in nodes again and create the dependencies
-		for (String dependency : node.getInNodes()) {
-			Props output = jobOutputProps.get(dependency);
+		
+		for(String end: flow.getEndNodes()) {
+			ExecutableNode node = flow.getExecutableNode(end);
+
+			if (node.getStatus() == Status.KILLED || node.getStatus() == Status.FAILED) {
+				succeeded = false;
+			}
+			
+			Props output = node.getOutputProps();
 			if (output != null) {
 				output = Props.clone(output);
 				output.setParent(previousOutput);
@@ -464,58 +412,195 @@ public class FlowRunner extends EventHandler implements Runnable {
 			}
 		}
 		
-		return previousOutput;
+		flow.setOutputProps(previousOutput);
+		if (!succeeded && (flow.getStatus() == Status.RUNNING)) {
+			flow.setStatus(Status.KILLED);
+		}
+		
+		flow.setEndTime(System.currentTimeMillis());
+		switch(flow.getStatus()) {
+		case FAILED_FINISHING:
+			logger.info("Setting flow " + id + "status to Failed.");
+			flow.setStatus(Status.FAILED);
+		case FAILED:
+		case KILLED:
+		case FAILED_SUCCEEDED:
+			logger.info("Flow " + id + "is set to " + flow.getStatus().toString());
+			break;
+		default:
+			flow.setStatus(Status.SUCCEEDED);
+			logger.info("Flow " + id + "is set to " + flow.getStatus().toString());
+		}
 	}
 	
-	private JobRunner createJobRunner(ExecutableNode node, Props previousOutput) {
-		String source = node.getJobPropsSource();
-		String propsSource = node.getPropsSource();
-
-		// If no properties are set, we just set the global properties.
-		Props parentProps = propsSource == null ? globalProps : sharedProps.get(propsSource);
-
-		// Set up overrides
-		ExecutionOptions options = flow.getExecutionOptions();
-		@SuppressWarnings("unchecked")
-		Props flowProps = new Props(null, options.getFlowParameters()); 
-		flowProps.putAll(commonProps);
-		flowProps.setParent(parentProps);
-		parentProps = flowProps;
-
-		// We add the previous job output and put into this props.
-		if (previousOutput != null) {
-			Props earliestParent = previousOutput.getEarliestAncestor();
-			earliestParent.setParent(parentProps);
+	@SuppressWarnings("unchecked")
+	private void prepareJobProperties(ExecutableNode node) throws IOException {
+		Props props = null;
+		// The following is the hiearchical ordering of dependency resolution
+		// 1. Parent Flow Properties
+		ExecutableFlowBase parentFlow = node.getParentFlow();
+		if (parentFlow != null) {
+			props = parentFlow.getInputProps();
+		}
+		
+		// 2. Shared Properties
+		String sharedProps = node.getPropsSource();
+		if (sharedProps != null) {
+			Props shared = this.sharedProps.get(sharedProps);
+			if (shared != null) {
+				// Clone because we may clobber
+				shared = Props.clone(shared);
+				shared.setEarliestAncestor(props);
+				props = shared;
+			}
+		}
 
-			parentProps = previousOutput;
+		// 3. Flow Override properties
+		Map<String,String> flowParam = flow.getExecutionOptions().getFlowParameters();
+		if (flowParam != null && !flowParam.isEmpty()) {
+			props = new Props(props, flowParam);
 		}
 		
-		// Load job file.
-		File path = new File(execDir, source);
-		Props prop = null;
+		// 4. Output Properties
+		Props outputProps = collectOutputProps(node);
+		if (outputProps != null) {
+			outputProps.setEarliestAncestor(props);
+			props = outputProps;
+		}
+		
+		// 5. The job source
+		Props jobSource = loadJobProps(node);
+		if (jobSource != null) {
+			jobSource.setParent(props);
+			props = jobSource;
+		}
+		
+		node.setInputProps(props);
+	}
+	
+	private Props loadJobProps(ExecutableNode node) throws IOException {
+		Props props = null;
+		String source = node.getJobSource();
+		if (source == null) {
+			return null;
+		}
 		
 		// load the override props if any
 		try {
-			prop = projectLoader.fetchProjectProperty(flow.getProjectId(), flow.getVersion(), node.getJobId()+".jor");
+			props = projectLoader.fetchProjectProperty(flow.getProjectId(), flow.getVersion(), node.getId()+".jor");
 		}
 		catch(ProjectManagerException e) {
 			e.printStackTrace();
-			logger.error("Error loading job override property for job " + node.getJobId());
+			logger.error("Error loading job override property for job " + node.getId());
 		}
-		if(prop == null) {
+		
+		File path = new File(execDir, source);
+		if(props == null) {
 			// if no override prop, load the original one on disk
 			try {
-				prop = new Props(null, path);				
+				props = new Props(null, path);				
 			} catch (IOException e) {
 				e.printStackTrace();
-				logger.error("Error loading job file " + source + " for job " + node.getJobId());
+				logger.error("Error loading job file " + source + " for job " + node.getId());
 			}
 		}
 		// setting this fake source as this will be used to determine the location of log files.
-		prop.setSource(path.getPath());
-		prop.setParent(parentProps);
+		if (path.getPath() != null) {
+			props.setSource(path.getPath());
+		}
+		return props;
+	}
+	
+	private void runExecutableNode(ExecutableNode node) throws IOException {
+		// Collect output props from the job's dependencies.
+		prepareJobProperties(node);
+		
+		if (node instanceof ExecutableFlowBase) {
+			node.setStatus(Status.RUNNING);
+			node.setStartTime(System.currentTimeMillis());
+			
+			logger.info("Starting subflow " + node.getNestedId() + ".");
+		}
+		else {
+			node.setStatus(Status.QUEUED);
+			JobRunner runner = createJobRunner(node);
+			logger.info("Submitting job " + node.getNestedId() + " to run.");
+			try {
+				executorService.submit(runner);
+				activeJobRunners.add(runner);
+			} catch (RejectedExecutionException e) {
+				logger.error(e);
+			};
+		}
+	}
+	
+	/**
+	 * Determines what the state of the next node should be.
+	 * 
+	 * @param node
+	 * @return
+	 */
+	public Status getImpliedStatus(ExecutableNode node) {
+		if (flowFailed && failureAction == ExecutionOptions.FailureAction.FINISH_CURRENTLY_RUNNING) {
+			return Status.KILLED;
+		}
+		else if (node.getStatus() == Status.DISABLED) {
+			return Status.DISABLED;
+		}
 		
-		JobRunner jobRunner = new JobRunner(node, prop, path.getParentFile(), executorLoader, jobtypeManager);
+		ExecutableFlowBase flow = node.getParentFlow();
+		boolean shouldKill = false;
+		for (String dependency: node.getInNodes()) {
+			ExecutableNode dependencyNode = flow.getExecutableNode(dependency);
+			Status depStatus = dependencyNode.getStatus();
+			
+			switch (depStatus) {
+			case FAILED:
+			case KILLED:
+				shouldKill = true;
+			case SKIPPED:
+			case SUCCEEDED:
+			case FAILED_SUCCEEDED:
+				continue;
+			default:
+				// Should never come here.
+				return null;
+			}
+		}
+
+		if (shouldKill) {
+			return Status.KILLED;
+		}
+		
+		// If it's disabled but ready to run, we want to make sure it continues being disabled.
+		if (node.getStatus() == Status.DISABLED) {
+			return Status.DISABLED;
+		}
+		
+		// All good to go, ready to run.
+		return Status.READY;
+	}
+	
+	private Props collectOutputProps(ExecutableNode node) {
+		Props previousOutput = null;
+		// Iterate the in nodes again and create the dependencies
+		for (String dependency : node.getInNodes()) {
+			Props output = node.getParentFlow().getExecutableNode(dependency).getOutputProps();
+			if (output != null) {
+				output = Props.clone(output);
+				output.setParent(previousOutput);
+				previousOutput = output;
+			}
+		}
+		
+		return previousOutput;
+	}
+	
+	private JobRunner createJobRunner(ExecutableNode node) {
+		// Load job file.
+		File path = new File(execDir, node.getJobSource());
+		
+		JobRunner jobRunner = new JobRunner(node, path.getParentFile(), executorLoader, jobtypeManager);
 		if (watcher != null) {
 			jobRunner.setPipeline(watcher, pipelineLevel);
 		}
@@ -592,80 +677,55 @@ public class FlowRunner extends EventHandler implements Runnable {
 			}
 			
 			logger.info("Cancelling " + activeJobRunners.size() + " jobs.");
-			for (JobRunner runner : activeJobRunners.values()) {
+			for (JobRunner runner : activeJobRunners) {
 				runner.cancel();
 			}
-			
-			if (flow.getStatus() != Status.FAILED && flow.getStatus() != Status.FAILED_FINISHING) {
-				logger.info("Setting flow status to " + Status.KILLED.toString());
-				flow.setStatus(Status.KILLED);
-			}
 		}
 	}
 	
 	public void retryFailures(String user) {
 		synchronized(mainSyncObj) {
 			logger.info("Retrying failures invoked by " + user);
-			ArrayList<String> failures = new ArrayList<String>();
-			for (ExecutableNode node: flow.getExecutableNodes()) {
-				if (node.getStatus() == Status.FAILED) {
-					failures.add(node.getJobId());
-				}
-				else if (node.getStatus() == Status.KILLED) {
-					node.setStartTime(-1);
-					node.setEndTime(-1);
-					node.setStatus(Status.READY);
-				}
-			}
+			retryFailures(flow);
+			
+			flow.setStatus(Status.RUNNING);
+			flow.setUpdateTime(System.currentTimeMillis());
+			flowFailed = false;
 			
-			retryJobs(failures, user);
+			updateFlow();
+			interrupt();
 		}
 	}
 	
-	public void retryJobs(List<String> jobIds, String user) {
-		synchronized(mainSyncObj) {
-			for (String jobId: jobIds) {
-				ExecutableNode node = flow.getExecutableNode(jobId);
-				if (node == null) {
-					logger.error("Job " + jobId + " doesn't exist in execution " + flow.getExecutionId() + ". Cannot retry.");
-					continue;
-				}
-				
-				if (Status.isStatusFinished(node.getStatus())) {
-					// Resets the status and increments the attempt number
-					node.resetForRetry();
-					reEnableDependents(node);
-					logger.info("Re-enabling job " + node.getJobId() + " attempt " + node.getAttempt());
-				}
-				else {
-					logger.error("Cannot retry job " + jobId + " since it hasn't run yet. User " + user);
-					continue;
+	private void retryFailures(ExecutableFlowBase flow) {
+		for (ExecutableNode node: flow.getExecutableNodes()) {
+			if (node instanceof ExecutableFlowBase) {
+				if (node.getStatus() == Status.FAILED || node.getStatus() == Status.FAILED_FINISHING || node.getStatus() == Status.KILLED) {
+					retryFailures((ExecutableFlowBase)node);
 				}
 			}
 			
-			boolean isFailureFound = false;
-			for (ExecutableNode node: flow.getExecutableNodes()) {
-				Status nodeStatus = node.getStatus();
-				if (nodeStatus == Status.FAILED || nodeStatus == Status.KILLED) {
-					isFailureFound = true;
-					break;
-				}
+			if (node.getStatus() == Status.FAILED) {
+				node.resetForRetry();
+				logger.info("Re-enabling job " + node.getNestedId() + " attempt " + node.getAttempt());
+				reEnableDependents(node);
 			}
-			
-			if (!isFailureFound) {
-				flow.setStatus(Status.RUNNING);
-				flow.setUpdateTime(System.currentTimeMillis());
-				flowFailed = false;
+			else if (node.getStatus() == Status.KILLED) {
+				node.setStartTime(-1);
+				node.setEndTime(-1);
+				node.setStatus(Status.READY);
+			}
+			else if (node.getStatus() == Status.FAILED_FINISHING) {
+				node.setStartTime(-1);
+				node.setEndTime(-1);
+				node.setStatus(Status.READY);
 			}
-			
-			updateFlow();
-			interrupt();
 		}
 	}
 	
 	private void reEnableDependents(ExecutableNode node) {
 		for(String dependent: node.getOutNodes()) {
-			ExecutableNode dependentNode = flow.getExecutableNode(dependent);
+			ExecutableNode dependentNode = node.getParentFlow().getExecutableNode(dependent);
 			
 			if (dependentNode.getStatus() == Status.KILLED) {
 				dependentNode.setStatus(Status.READY);
@@ -683,60 +743,11 @@ public class FlowRunner extends EventHandler implements Runnable {
 	private void interrupt() {
 		flowRunnerThread.interrupt();
 	}
-	
-	private Status getImpliedStatus(ExecutableNode node) {
-		switch(node.getStatus()) {
-		case FAILED:
-		case KILLED:
-		case SKIPPED:
-		case SUCCEEDED:
-		case QUEUED:
-		case RUNNING:
-			return null;
-		default:
-			break;
-		}
-		
-		boolean shouldKill = false;
-		for (String dependency : node.getInNodes()) {
-			ExecutableNode dependencyNode = flow.getExecutableNode(dependency);
-			
-			Status depStatus = dependencyNode.getStatus();
-			switch (depStatus) {
-			case FAILED:
-			case KILLED:
-				shouldKill = true;
-			case SKIPPED:
-			case SUCCEEDED:
-				continue;
-			case RUNNING:
-			case QUEUED:
-			case DISABLED:
-				return null;
-			default:
-				// Return null means it's not ready to run.
-				return null;
-			}
-		}
-		
-		ExecutionOptions options = flow.getExecutionOptions();
-		if (shouldKill || flowCancelled || (flowFailed && options.getFailureAction() != FailureAction.FINISH_ALL_POSSIBLE)) {
-			return Status.KILLED;
-		}
-		
-		// If it's disabled but ready to run, we want to make sure it continues being disabled.
-		if (node.getStatus() == Status.DISABLED) {
-			return Status.DISABLED;
-		}
-		
-		// All good to go, ready to run.
-		return Status.READY;
-	}
-	
+
 	private class JobRunnerEventListener implements EventListener {
 		public JobRunnerEventListener() {
 		}
-
+		
 		@Override
 		public synchronized void handleEvent(Event event) {
 			JobRunner runner = (JobRunner)event.getRunner();
@@ -747,54 +758,61 @@ public class FlowRunner extends EventHandler implements Runnable {
 			else if (event.getType() == Type.JOB_FINISHED) {
 				synchronized(mainSyncObj) {
 					ExecutableNode node = runner.getNode();
-					activeJobRunners.remove(node.getJobId());
+					activeJobRunners.remove(node.getId());
 					
-					logger.info("Job Finished " + node.getJobId() + " with status " + node.getStatus());
-					if (runner.getOutputProps() != null) {
-						logger.info("Job " + node.getJobId() + " had output props.");
-						jobOutputProps.put(node.getJobId(), runner.getOutputProps());
+					String id = node.getNestedId();
+					logger.info("Job Finished " + id + " with status " + node.getStatus());
+					if (node.getOutputProps() != null && node.getOutputProps().size() > 0) {
+						logger.info("Job " + id + " had output props.");
 					}
 					
-					updateFlow();
-					
 					if (node.getStatus() == Status.FAILED) {
-						// Retry failure if conditions are met.
-						if (!runner.isCancelled() && runner.getRetries() > node.getAttempt()) {
-							logger.info("Job " + node.getJobId() + " will be retried. Attempt " + node.getAttempt() + " of " + runner.getRetries());
+						if (runner.getRetries() > node.getAttempt()) {
+							logger.info("Job " + id + " will be retried. Attempt " + node.getAttempt() + " of " + runner.getRetries());
 							node.setDelayedExecution(runner.getRetryBackoff());
 							node.resetForRetry();
 						}
 						else {
-							if (!runner.isCancelled() && runner.getRetries() > 0) {
-					
-								logger.info("Job " + node.getJobId() + " has run out of retry attempts");
+							if (runner.getRetries() > 0) {
+								logger.info("Job " + id + " has run out of retry attempts");
 								// Setting delayed execution to 0 in case this is manually re-tried.
 								node.setDelayedExecution(0);
 							}
-
+							
 							flowFailed = true;
 							
-							ExecutionOptions options = flow.getExecutionOptions();
 							// The KILLED status occurs when cancel is invoked. We want to keep this
 							// status even in failure conditions.
-							if (flow.getStatus() != Status.KILLED && flow.getStatus() != Status.FAILED) {
-								flow.setStatus(Status.FAILED_FINISHING);
-								if (options.getFailureAction() == FailureAction.CANCEL_ALL && !flowCancelled) {
+							if (!flowCancelled) {
+								// During a failure, we propagate the failure to parent flows
+								propagateStatus(node.getParentFlow(), Status.FAILED_FINISHING);
+	
+								if (failureAction == FailureAction.CANCEL_ALL) {
 									logger.info("Flow failed. Failure option is Cancel All. Stopping execution.");
 									cancel();
 								}
 							}
 						}
+						
 					}
-					
+					updateFlow();
+
 					interrupt();
-	
 					fireEventListeners(event);
 				}
 			}
 		}
+		
+		private void propagateStatus(ExecutableFlowBase base, Status status) {
+			if (!Status.isStatusFinished(base.getStatus())) {
+				base.setStatus(status);
+				if (base.getParentFlow() != null) {
+					propagateStatus(base.getParentFlow(), status);
+				}
+			}
+		}
 	}
-	
+
 	public boolean isCancelled() {
 		return flowCancelled;
 	}
@@ -809,7 +827,7 @@ public class FlowRunner extends EventHandler implements Runnable {
 	
 	public File getJobLogFile(String jobId, int attempt) {
 		ExecutableNode node = flow.getExecutableNode(jobId);
-		File path = new File(execDir, node.getJobPropsSource());
+		File path = new File(execDir, node.getJobSource());
 		
 		String logFileName = JobRunner.createLogFileName(execId, jobId, attempt);
 		File logFile = new File(path.getParentFile(), logFileName);
@@ -823,7 +841,7 @@ public class FlowRunner extends EventHandler implements Runnable {
 	
 	public File getJobMetaDataFile(String jobId, int attempt) {
 		ExecutableNode node = flow.getExecutableNode(jobId);
-		File path = new File(execDir, node.getJobPropsSource());
+		File path = new File(execDir, node.getJobSource());
 		
 		String metaDataFileName = JobRunner.createMetaDataFileName(execId, jobId, attempt);
 		File metaDataFile = new File(path.getParentFile(), metaDataFileName);
diff --git a/src/java/azkaban/execapp/FlowRunnerManager.java b/src/java/azkaban/execapp/FlowRunnerManager.java
index 8c98086..ecc0847 100644
--- a/src/java/azkaban/execapp/FlowRunnerManager.java
+++ b/src/java/azkaban/execapp/FlowRunnerManager.java
@@ -25,7 +25,6 @@ import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
-import java.util.List;
 import java.util.Map;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.ConcurrentHashMap;
@@ -496,16 +495,6 @@ public class FlowRunnerManager implements EventListener {
 		runner.retryFailures(user);
 	}
 	
-	public void retryJobs(int execId, String user, List<String> jobId) throws ExecutorManagerException {
-		FlowRunner runner = runningFlows.get(execId);
-		
-		if (runner == null) {
-			throw new ExecutorManagerException("Execution " + execId + " is not running.");
-		}
-		
-		runner.retryJobs(jobId, user);
-	}
-	
 	public ExecutableFlow getExecutableFlow(int execId) {
 		FlowRunner runner = runningFlows.get(execId);
 		if (runner == null) {
@@ -648,7 +637,7 @@ public class FlowRunnerManager implements EventListener {
 	}
 	
 	public String getRunningFlowIds() {
-		List<Integer> ids = new ArrayList<Integer>(runningFlows.keySet());
+		ArrayList<Integer> ids = new ArrayList<Integer>(runningFlows.keySet());
 		Collections.sort(ids);
 		return ids.toString();
 	}
diff --git a/src/java/azkaban/execapp/JobRunner.java b/src/java/azkaban/execapp/JobRunner.java
index 6413974..99bd8ab 100644
--- a/src/java/azkaban/execapp/JobRunner.java
+++ b/src/java/azkaban/execapp/JobRunner.java
@@ -38,6 +38,7 @@ import azkaban.execapp.event.Event;
 import azkaban.execapp.event.Event.Type;
 import azkaban.execapp.event.EventHandler;
 import azkaban.execapp.event.FlowWatcher;
+import azkaban.executor.ExecutableFlowBase;
 import azkaban.executor.ExecutableNode;
 import azkaban.executor.ExecutorLoader;
 import azkaban.executor.ExecutorManagerException;
@@ -55,7 +56,6 @@ public class JobRunner extends EventHandler implements Runnable {
 	
 	private ExecutorLoader loader;
 	private Props props;
-	private Props outputProps;
 	private ExecutableNode node;
 	private File workingDir;
 
@@ -68,6 +68,7 @@ public class JobRunner extends EventHandler implements Runnable {
 	
 	private Job job;
 	private int executionId = -1;
+	private String jobId;
 	
 	private static final Object logCreatorLock = new Object();
 	private Object syncObject = new Object();
@@ -88,11 +89,13 @@ public class JobRunner extends EventHandler implements Runnable {
 	private boolean cancelled = false;
 	private BlockingStatus currentBlockStatus = null;
 	
-	public JobRunner(ExecutableNode node, Props props, File workingDir, ExecutorLoader loader, JobTypeManager jobtypeManager) {
-		this.props = props;
+	public JobRunner(ExecutableNode node, File workingDir, ExecutorLoader loader, JobTypeManager jobtypeManager) {
+		this.props = node.getInputProps();
 		this.node = node;
 		this.workingDir = workingDir;
-		this.executionId = node.getExecutionId();
+		
+		this.executionId = node.getParentFlow().getExecutionId();
+		this.jobId = node.getId();
 		this.loader = loader;
 		this.jobtypeManager = jobtypeManager;
 	}
@@ -116,14 +119,68 @@ public class JobRunner extends EventHandler implements Runnable {
 		this.pipelineLevel = pipelineLevel;
 
 		if (this.pipelineLevel == 1) {
-			pipelineJobs.add(node.getJobId());
+			pipelineJobs.add(node.getNestedId());
 		}
 		else if (this.pipelineLevel == 2) {
-			pipelineJobs.add(node.getJobId());
-			pipelineJobs.addAll(node.getOutNodes());
+			pipelineJobs.add(node.getNestedId());
+			ExecutableFlowBase parentFlow = node.getParentFlow();
+			
+			if (parentFlow.getEndNodes().contains(node.getId())) {
+				if (!parentFlow.getOutNodes().isEmpty()) {
+					ExecutableFlowBase grandParentFlow = parentFlow.getParentFlow();
+					for (String outNode: parentFlow.getOutNodes()) {
+						ExecutableNode nextNode = grandParentFlow.getExecutableNode(outNode);
+						
+						// If the next node is a nested flow, then we add the nested starting nodes 
+						if (nextNode instanceof ExecutableFlowBase) {
+							ExecutableFlowBase nextFlow = (ExecutableFlowBase)nextNode;
+							findAllStartingNodes(nextFlow, pipelineJobs);
+						}
+						else {
+							pipelineJobs.add(nextNode.getNestedId());
+						}
+					}
+				}
+			}
+			else {
+				for (String outNode : node.getOutNodes()) {
+					ExecutableNode nextNode = parentFlow.getExecutableNode(outNode);
+	
+					// If the next node is a nested flow, then we add the nested starting nodes 
+					if (nextNode instanceof ExecutableFlowBase) {
+						ExecutableFlowBase nextFlow = (ExecutableFlowBase)nextNode;
+						findAllStartingNodes(nextFlow, pipelineJobs);
+					}
+					else {
+						pipelineJobs.add(nextNode.getNestedId());
+					}
+				}
+			}
+		}
+	}
+	
+	private void findAllStartingNodes(ExecutableFlowBase flow, Set<String> pipelineJobs) {
+		for (String startingNode: flow.getStartNodes()) {
+			ExecutableNode node = flow.getExecutableNode(startingNode);
+			if (node instanceof ExecutableFlowBase) {
+				findAllStartingNodes((ExecutableFlowBase)node, pipelineJobs);
+			}
+			else {
+				pipelineJobs.add(node.getNestedId());
+			}
 		}
 	}
 	
+	/**
+	 * Returns a list of jobs that this JobRunner will wait upon to finish before starting.
+	 * It is only relevant if pipeline is turned on.
+	 * 
+	 * @return
+	 */
+	public Set<String> getPipelineWatchedJobs() {
+		return pipelineJobs;
+	}
+	
 	public void setDelayStart(long delayMS) {
 		delayStartMs = delayMS;
 	}
@@ -143,12 +200,18 @@ public class JobRunner extends EventHandler implements Runnable {
 	private void createLogger() {
 		// Create logger
 		synchronized (logCreatorLock) {
-			String loggerName = System.currentTimeMillis() + "." + executionId + "." + node.getJobId();
+			String loggerName = System.currentTimeMillis() + "." + this.executionId + "." + this.jobId;
 			logger = Logger.getLogger(loggerName);
 
 			// Create file appender
-			String logName = createLogFileName(node.getExecutionId(), node.getJobId(), node.getAttempt());
+			String id = this.jobId;
+			if (node.getExecutableFlow() != node.getParentFlow()) {
+				id = node.getPrintableId("._.");
+			}
+			
+			String logName = createLogFileName(this.executionId, id, node.getAttempt());
 			logFile = new File(workingDir, logName);
+			
 			String absolutePath = logFile.getAbsolutePath();
 
 			jobAppender = null;
@@ -160,7 +223,7 @@ public class JobRunner extends EventHandler implements Runnable {
 				logger.addAppender(jobAppender);
 				logger.setAdditivity(false);
 			} catch (IOException e) {
-				flowLogger.error("Could not open log file in " + workingDir + " for job " + node.getJobId(), e);
+				flowLogger.error("Could not open log file in " + workingDir + " for job " + this.jobId, e);
 			}
 		}
 	}
@@ -177,98 +240,161 @@ public class JobRunner extends EventHandler implements Runnable {
 			node.setUpdateTime(System.currentTimeMillis());
 			loader.updateExecutableNode(node);
 		} catch (ExecutorManagerException e) {
-			flowLogger.error("Could not update job properties in db for " + node.getJobId(), e);
+			flowLogger.error("Could not update job properties in db for " + this.jobId, e);
 		}
 	}
 	
-	@Override
-	public void run() {
-		Thread.currentThread().setName("JobRunner-" + node.getJobId() + "-" + executionId);
+	/**
+	 * Used to handle non-ready and special status's (i.e. KILLED). Returns true
+	 * if they handled anything.
+	 * 
+	 * @return
+	 */
+	private boolean handleNonReadyStatus() {
+		Status nodeStatus = node.getStatus();
+		boolean quickFinish = false;
+		long time = System.currentTimeMillis();
 		
-		if (node.getStatus() == Status.DISABLED) {
-			node.setStartTime(System.currentTimeMillis());
-			fireEvent(Event.create(this, Type.JOB_STARTED, null, false));
-			node.setStatus(Status.SKIPPED);
-			node.setEndTime(System.currentTimeMillis());
-			fireEvent(Event.create(this, Type.JOB_FINISHED));
-			return;
-		} else if (this.cancelled) {
-			node.setStartTime(System.currentTimeMillis());
-			fireEvent(Event.create(this, Type.JOB_STARTED, null, false));
-			node.setStatus(Status.FAILED);
-			node.setEndTime(System.currentTimeMillis());
-			fireEvent(Event.create(this, Type.JOB_FINISHED));
-		} else if (node.getStatus() == Status.FAILED || node.getStatus() == Status.KILLED) {
-			node.setStartTime(System.currentTimeMillis());
+		if (this.isCancelled() || Status.isStatusFinished(nodeStatus)) {
+			quickFinish = true;
+		}
+		else if (nodeStatus == Status.DISABLED) {
+			changeStatus(Status.SKIPPED, time);
+			quickFinish = true;
+		} 
+		else if (this.cancelled) {
+			changeStatus(Status.FAILED, time);
+			quickFinish = true;
+		} 
+		
+		if (quickFinish) {
+			node.setStartTime(time);
 			fireEvent(Event.create(this, Type.JOB_STARTED, null, false));
-			node.setEndTime(System.currentTimeMillis());
+			node.setEndTime(time);
 			fireEvent(Event.create(this, Type.JOB_FINISHED));
-			return;
+			return true;
 		}
-		else {
-			createLogger();
-			node.setUpdateTime(System.currentTimeMillis());
-
-			// For pipelining of jobs. Will watch other jobs.
-			if (!pipelineJobs.isEmpty()) {
-				String blockedList = "";
-				ArrayList<BlockingStatus> blockingStatus = new ArrayList<BlockingStatus>();
-				for (String waitingJobId : pipelineJobs) {
-					Status status = watcher.peekStatus(waitingJobId);
-					if (status != null && !Status.isStatusFinished(status)) {
-						BlockingStatus block = watcher.getBlockingStatus(waitingJobId);
-						blockingStatus.add(block);
-						blockedList += waitingJobId + ",";
-					}
+		
+		return false;
+	}
+	
+	/**
+	 * If pipelining is set, will block on another flow's jobs.
+	 */
+	private boolean blockOnPipeLine() {
+		if (this.isCancelled()) {
+			return true;
+		}
+		
+		// For pipelining of jobs. Will watch other jobs.
+		if (!pipelineJobs.isEmpty()) {
+			String blockedList = "";
+			ArrayList<BlockingStatus> blockingStatus = new ArrayList<BlockingStatus>();
+			for (String waitingJobId : pipelineJobs) {
+				Status status = watcher.peekStatus(waitingJobId);
+				if (status != null && !Status.isStatusFinished(status)) {
+					BlockingStatus block = watcher.getBlockingStatus(waitingJobId);
+					blockingStatus.add(block);
+					blockedList += waitingJobId + ",";
 				}
-				if (!blockingStatus.isEmpty()) {
-					logger.info("Pipeline job " + node.getJobId() + " waiting on " + blockedList + " in execution " + watcher.getExecId());
-					
-					for(BlockingStatus bStatus: blockingStatus) {
-						logger.info("Waiting on pipelined job " + bStatus.getJobId());
-						currentBlockStatus = bStatus;
-						bStatus.blockOnFinishedStatus();
+			}
+			if (!blockingStatus.isEmpty()) {
+				logger.info("Pipeline job " + this.jobId + " waiting on " + blockedList + " in execution " + watcher.getExecId());
+				
+				for(BlockingStatus bStatus: blockingStatus) {
+					logger.info("Waiting on pipelined job " + bStatus.getJobId());
+					currentBlockStatus = bStatus;
+					bStatus.blockOnFinishedStatus();
+					if (this.isCancelled()) {
+						logger.info("Job was cancelled while waiting on pipeline. Quiting.");
+						return true;
+					}
+					else {
 						logger.info("Pipelined job " + bStatus.getJobId() + " finished.");
-						if (watcher.isWatchCancelled()) {
-							break;
-						}
 					}
-					writeStatus();	
-					fireEvent(Event.create(this, Type.JOB_STATUS_CHANGED));
 				}
-				if (watcher.isWatchCancelled()) {
-					logger.info("Job was cancelled while waiting on pipeline. Quiting.");
-					node.setStartTime(System.currentTimeMillis());
-					node.setEndTime(System.currentTimeMillis());
-					node.setStatus(Status.FAILED);
-					fireEvent(Event.create(this, Type.JOB_FINISHED));
-					return;
+			}
+		}
+		
+		currentBlockStatus = null;
+		return false;
+	}
+	
+	private boolean delayExecution() {
+		if (this.isCancelled()) {
+			return true;
+		}
+		
+		long currentTime = System.currentTimeMillis();
+		if (delayStartMs > 0) {
+			logger.info("Delaying start of execution for " + delayStartMs + " milliseconds.");
+			synchronized(this) {
+				try {
+					this.wait(delayStartMs);
+					logger.info("Execution has been delayed for " + delayStartMs + " ms. Continuing with execution.");
+				} catch (InterruptedException e) {
+					logger.error("Job " + this.jobId + " was to be delayed for " + delayStartMs + ". Interrupted after " + (System.currentTimeMillis() - currentTime));
 				}
 			}
 			
-			currentBlockStatus = null;
-			long currentTime = System.currentTimeMillis();
-			if (delayStartMs > 0) {
-				logger.info("Delaying start of execution for " + delayStartMs + " milliseconds.");
-				synchronized(this) {
-					try {
-						this.wait(delayStartMs);
-						logger.info("Execution has been delayed for " + delayStartMs + " ms. Continuing with execution.");
-					} catch (InterruptedException e) {
-						logger.error("Job " + node.getJobId() + " was to be delayed for " + delayStartMs + ". Interrupted after " + (System.currentTimeMillis() - currentTime));
+			if (this.isCancelled()) {
+				logger.info("Job was cancelled while in delay. Quiting.");
+				return true;
+			}
+		}
+		
+		return false;
+	}
+	
+	private void finalizeLogFile() {
+		closeLogger();
+		
+		if (logFile != null) {
+			try {
+				File[] files = logFile.getParentFile().listFiles(new FilenameFilter() {
+					
+					@Override
+					public boolean accept(File dir, String name) {
+						return name.startsWith(logFile.getName());
 					}
-				}
+				} 
+				);
+				Arrays.sort(files, Collections.reverseOrder());
 				
-				if (cancelled) {
-					logger.info("Job was cancelled while in delay. Quiting.");
-					node.setStartTime(System.currentTimeMillis());
-					node.setEndTime(System.currentTimeMillis());
-					fireEvent(Event.create(this, Type.JOB_FINISHED));
-					return;
-				}
+				loader.uploadLogFile(executionId, this.node.getNestedId(), node.getAttempt(), files);
+			} catch (ExecutorManagerException e) {
+				flowLogger.error("Error writing out logs for job " + this.node.getNestedId(), e);
 			}
-			
-			node.setStartTime(System.currentTimeMillis());
+		}
+		else {
+			flowLogger.info("Log file for job " + this.jobId + " is null");
+		}
+	}
+	
+	/**
+	 * The main run thread.
+	 * 
+	 */
+	@Override
+	public void run() {
+		Thread.currentThread().setName("JobRunner-" + this.jobId + "-" + executionId);
+		
+		// If the job is cancelled, disabled, killed. No log is created in this case
+		if (handleNonReadyStatus()) {
+			return;
+		}
+
+		createLogger();
+		boolean errorFound = false;
+		// Delay execution if necessary. Will return a true if something went wrong.
+		errorFound |= delayExecution();
+
+		// For pipelining of jobs. Will watch other jobs. Will return true if something went wrong.
+		errorFound |= blockOnPipeLine();
+
+		// Start the node.
+		node.setStartTime(System.currentTimeMillis());
+		if (!errorFound && !isCancelled()) {
 			fireEvent(Event.create(this, Type.JOB_STARTED, null, false));
 			try {
 				loader.uploadExecutableNode(node, props);
@@ -277,56 +403,26 @@ public class JobRunner extends EventHandler implements Runnable {
 			}
 			
 			if (prepareJob()) {
+				// Writes status to the db
 				writeStatus();
 				fireEvent(Event.create(this, Type.JOB_STATUS_CHANGED), false);
 				runJob();
+				writeStatus();
 			}
 			else {
-				node.setStatus(Status.FAILED);
-				logError("Job run failed!");
-			}
-			
-			node.setEndTime(System.currentTimeMillis());
-
-			logInfo("Finishing job " + node.getJobId() + " at " + node.getEndTime());
-
-			closeLogger();
-			writeStatus();
-			
-			if (logFile != null) {
-				try {
-					File[] files = logFile.getParentFile().listFiles(new FilenameFilter() {
-						
-						@Override
-						public boolean accept(File dir, String name) {
-							return name.startsWith(logFile.getName());
-						}
-					} 
-					);
-					Arrays.sort(files, Collections.reverseOrder());
-					
-					
-					loader.uploadLogFile(executionId, node.getJobId(), node.getAttempt(), files);
-				} catch (ExecutorManagerException e) {
-					flowLogger.error("Error writing out logs for job " + node.getJobId(), e);
-				}
-			}
-			else {
-				flowLogger.info("Log file for job " + node.getJobId() + " is null");
+				changeStatus(Status.FAILED);
+				logError("Job run failed preparing the job.");
 			}
 		}
-		fireEvent(Event.create(this, Type.JOB_FINISHED));
-	}
-	
-	private void fireEvent(Event event) {
-		fireEvent(event, true);
-	}
-	
-	private void fireEvent(Event event, boolean updateTime) {
-		if (updateTime) {
-			node.setUpdateTime(System.currentTimeMillis());
+		node.setEndTime(System.currentTimeMillis());
+
+		if (isCancelled()) {
+			changeStatus(Status.FAILED);
 		}
-		this.fireEventListeners(event);
+		logInfo("Finishing job " + this.jobId + " at " + node.getEndTime());
+		
+		fireEvent(Event.create(this, Type.JOB_FINISHED), false);
+		finalizeLogFile();
 	}
 	
 	private boolean prepareJob() throws RuntimeException {
@@ -342,15 +438,22 @@ public class JobRunner extends EventHandler implements Runnable {
 			}
 
 			if (node.getAttempt() > 0) {
-				logInfo("Starting job " + node.getJobId() + " attempt " + node.getAttempt() + " at " + node.getStartTime());
+				logInfo("Starting job " + this.jobId + " attempt " + node.getAttempt() + " at " + node.getStartTime());
 			}
 			else {
-				logInfo("Starting job " + node.getJobId() + " at " + node.getStartTime());
+				logInfo("Starting job " + this.jobId + " at " + node.getStartTime());
 			}
+			
+			// If it's an embedded flow, we'll add the nested flow info to the job conf
+			if (node.getExecutableFlow() != node.getParentFlow()) {
+				String subFlow = node.getPrintableId(":");
+				props.put(CommonJobProperties.NESTED_FLOW_PATH, subFlow);
+			}
+			
 			props.put(CommonJobProperties.JOB_ATTEMPT, node.getAttempt());
-			props.put(CommonJobProperties.JOB_METADATA_FILE, createMetaDataFileName(executionId, node.getJobId(), node.getAttempt()));
-			node.setStatus(Status.RUNNING);
-
+			props.put(CommonJobProperties.JOB_METADATA_FILE, createMetaDataFileName(executionId, this.jobId, node.getAttempt()));
+			changeStatus(Status.RUNNING);
+			
 			// Ability to specify working directory
 			if (!props.containsKey(AbstractProcessJob.WORKING_DIR)) {
 				props.put(AbstractProcessJob.WORKING_DIR, workingDir.getAbsolutePath());
@@ -359,17 +462,16 @@ public class JobRunner extends EventHandler implements Runnable {
 			if(props.containsKey("user.to.proxy")) {
 				String jobProxyUser = props.getString("user.to.proxy");
 				if(proxyUsers != null && !proxyUsers.contains(jobProxyUser)) {
-					logger.error("User " + jobProxyUser + " has no permission to execute this job " + node.getJobId() + "!");
+					logger.error("User " + jobProxyUser + " has no permission to execute this job " + this.jobId + "!");
 					return false;
 				}
 			}
 			
-			//job = JobWrappingFactory.getJobWrappingFactory().buildJobExecutor(node.getJobId(), props, logger);
 			try {
-				job = jobtypeManager.buildJobExecutor(node.getJobId(), props, logger);
+				job = jobtypeManager.buildJobExecutor(this.jobId, props, logger);
 			}
 			catch (JobTypeManagerException e) {
-				logger.error("Failed to build job type, skipping this job");
+				logger.error("Failed to build job type");
 				return false;
 			}
 		}
@@ -382,20 +484,49 @@ public class JobRunner extends EventHandler implements Runnable {
 			job.run();
 		} catch (Exception e) {
 			e.printStackTrace();
-
-			node.setStatus(Status.FAILED);
-			logError("Job run failed!");
-			logError(e.getMessage() + e.getCause());
-			return;
+			
+			if (props.getBoolean("job.succeed.on.failure", false)) {
+				changeStatus(Status.FAILED_SUCCEEDED);
+				logError("Job run failed, but will treat it like success.");
+				logError(e.getMessage() + e.getCause());
+			}
+			else {
+				changeStatus(Status.FAILED);
+				logError("Job run failed!");
+				logError(e.getMessage() + e.getCause());
+			}
 		}
-
-		node.setStatus(Status.SUCCEEDED);
+		
 		if (job != null) {
-			outputProps = job.getJobGeneratedProperties();
-			node.setOutputProps(outputProps);
+			node.setOutputProps(job.getJobGeneratedProperties());
+		}
+		
+		// If the job is still running, set the status to Success.
+		if (!Status.isStatusFinished(node.getStatus())) {
+			changeStatus(Status.SUCCEEDED);
 		}
 	}
 	
+	private void changeStatus(Status status) {
+		changeStatus(status, System.currentTimeMillis());
+	}
+	
+	private void changeStatus(Status status, long time) {
+		node.setStatus(status);
+		node.setUpdateTime(time);
+	}
+	
+	private void fireEvent(Event event) {
+		fireEvent(event, true);
+	}
+	
+	private void fireEvent(Event event, boolean updateTime) {
+		if (updateTime) {
+			node.setUpdateTime(System.currentTimeMillis());
+		}
+		this.fireEventListeners(event);
+	}
+	
 	public void cancel() {
 		synchronized (syncObject) {
 			logError("Cancel has been called.");
@@ -433,10 +564,6 @@ public class JobRunner extends EventHandler implements Runnable {
 		return node.getStatus();
 	}
 
-	public Props getOutputProps() {
-		return outputProps;
-	}
-
 	private void logError(String message) {
 		if (logger != null) {
 			logger.error(message);
diff --git a/src/java/azkaban/executor/ConnectorParams.java b/src/java/azkaban/executor/ConnectorParams.java
index 4a24556..0c0c7e0 100644
--- a/src/java/azkaban/executor/ConnectorParams.java
+++ b/src/java/azkaban/executor/ConnectorParams.java
@@ -62,11 +62,11 @@ public interface ConnectorParams {
 	public static final int NODE_END_INDEX = 3;
 
 	public static final String UPDATE_TIME_LIST_PARAM = "updatetime";
-	public static final String EXEC_ID_LIST_PARAM = "execid";
+	public static final String EXEC_ID_LIST_PARAM = "executionId";
 	
 	public static final String FORCED_FAILED_MARKER = ".failed";
 	
-	public static final String UPDATE_MAP_EXEC_ID = "execId";
+	public static final String UPDATE_MAP_EXEC_ID = "executionId";
 	public static final String UPDATE_MAP_JOBID = "jobId";
 	public static final String UPDATE_MAP_UPDATE_TIME = "updateTime";
 	public static final String UPDATE_MAP_STATUS = "status";
diff --git a/src/java/azkaban/executor/ExecutableFlow.java b/src/java/azkaban/executor/ExecutableFlow.java
index eaa27a6..5fa2d89 100644
--- a/src/java/azkaban/executor/ExecutableFlow.java
+++ b/src/java/azkaban/executor/ExecutableFlow.java
@@ -1,5 +1,5 @@
 /*
- * Copyright 2012 LinkedIn Corp.
+ * Copyright 2013 LinkedIn Corp.
  * 
  * 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
@@ -13,7 +13,6 @@
  * License for the specific language governing permissions and limitations under
  * the License.
  */
-
 package azkaban.executor;
 
 import java.util.ArrayList;
@@ -24,77 +23,51 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
-import azkaban.executor.ExecutableNode.Attempt;
-import azkaban.flow.Edge;
 import azkaban.flow.Flow;
-import azkaban.flow.FlowProps;
-import azkaban.flow.Node;
-import azkaban.utils.JSONUtils;
-
-public class ExecutableFlow {
+import azkaban.project.Project;
+import azkaban.utils.TypedMapWrapper;
+
+public class ExecutableFlow extends ExecutableFlowBase {
+	public static final String EXECUTIONID_PARAM = "executionId";
+	public static final String EXECUTIONPATH_PARAM ="executionPath";
+	public static final String EXECUTIONOPTIONS_PARAM ="executionOptions";
+	public static final String PROJECTID_PARAM ="projectId";
+	public static final String SCHEDULEID_PARAM ="scheduleId";
+	public static final String SUBMITUSER_PARAM = "submitUser";
+	public static final String SUBMITTIME_PARAM = "submitTime";
+	public static final String VERSION_PARAM = "version";
+	public static final String PROXYUSERS_PARAM = "proxyUsers";
+	
 	private int executionId = -1;
-	private String flowId;
 	private int scheduleId = -1;
 	private int projectId;
 	private int version;
-
-	private String executionPath;
-
-	private HashMap<String, FlowProps> flowProps = new HashMap<String, FlowProps>();
-	private HashMap<String, ExecutableNode> executableNodes = new HashMap<String, ExecutableNode>();
-	private ArrayList<String> startNodes;
-	private ArrayList<String> endNodes;
-
 	private long submitTime = -1;
-	private long startTime = -1;
-	private long endTime = -1;
-	private long updateTime = -1;
-
-	private Status flowStatus = Status.READY;
 	private String submitUser;
-
+	private String executionPath;
+	
 	private HashSet<String> proxyUsers = new HashSet<String>();
 	private ExecutionOptions executionOptions;
-
-	public ExecutableFlow(Flow flow) {
-		this.projectId = flow.getProjectId();
-		this.scheduleId = -1;
-		this.flowId = flow.getId();
-		this.version = flow.getVersion();
-		this.setFlow(flow);
-	}
-
-	public ExecutableFlow(int executionId, Flow flow) {
-		this.projectId = flow.getProjectId();
+	
+	public ExecutableFlow(Project project, Flow flow) {
+		this.projectId = project.getId();
+		this.version = project.getVersion();
 		this.scheduleId = -1;
-		this.flowId = flow.getId();
-		this.version = flow.getVersion();
-		this.executionId = executionId;
 
-		this.setFlow(flow);
+		this.setFlow(project, flow);
 	}
 
 	public ExecutableFlow() {
 	}
 
-	public long getUpdateTime() {
-		return updateTime;
+	@Override
+	public String getId() {
+		return getFlowId();
 	}
-
-	public void setUpdateTime(long updateTime) {
-		this.updateTime = updateTime;
-	}
-
-	public List<ExecutableNode> getExecutableNodes() {
-		return new ArrayList<ExecutableNode>(executableNodes.values());
-	}
-
-	public ExecutableNode getExecutableNode(String id) {
-		return executableNodes.get(id);
-	}
-
-	public Collection<FlowProps> getFlowProps() {
-		return flowProps.values();
+	
+	@Override
+	public ExecutableFlow getExecutableFlow() {
+		return this;
 	}
 
 	public void addAllProxyUsers(Collection<String> proxyUsers) {
@@ -112,77 +85,17 @@ public class ExecutableFlow {
 	public ExecutionOptions getExecutionOptions() {
 		return executionOptions;
 	}
-
-	private void setFlow(Flow flow) {
+	
+	protected void setFlow(Project project, Flow flow) {
+		super.setFlow(project, flow);
 		executionOptions = new ExecutionOptions();
 
-		for (Node node: flow.getNodes()) {
-			String id = node.getId();
-			ExecutableNode exNode = new ExecutableNode(node, this);
-			executableNodes.put(id, exNode);
-		}
-
-		for (Edge edge: flow.getEdges()) {
-			ExecutableNode sourceNode = executableNodes.get(edge.getSourceId());
-			ExecutableNode targetNode = executableNodes.get(edge.getTargetId());
-
-			sourceNode.addOutNode(edge.getTargetId());
-			targetNode.addInNode(edge.getSourceId());
-		}
-
 		if (flow.getSuccessEmails() != null) {
 			executionOptions.setSuccessEmails(flow.getSuccessEmails());
 		}
 		if (flow.getFailureEmails() != null) {
 			executionOptions.setFailureEmails(flow.getFailureEmails());
 		}
-		executionOptions.setMailCreator(flow.getMailCreator());
-
-		flowProps.putAll(flow.getAllFlowProps());
-	}
-
-	public List<String> getStartNodes() {
-		if (startNodes == null) {
-			startNodes = new ArrayList<String>();
-			for (ExecutableNode node: executableNodes.values()) {
-				if (node.getInNodes().isEmpty()) {
-					startNodes.add(node.getJobId());
-				}
-			}
-		}
-
-		return startNodes;
-	}
-
-	public List<String> getEndNodes() {
-		if (endNodes == null) {
-			endNodes = new ArrayList<String>();
-			for (ExecutableNode node: executableNodes.values()) {
-				if (node.getOutNodes().isEmpty()) {
-					endNodes.add(node.getJobId());
-				}
-			}
-		}
-
-		return endNodes;
-	}
-
-	public boolean setNodeStatus(String nodeId, Status status) {
-		ExecutableNode exNode = executableNodes.get(nodeId);
-		if (exNode == null) {
-			return false;
-		}
-		exNode.setStatus(status);
-		return true;
-	}
-
-	public void setProxyNodes(int externalExecutionId, String nodeId) {
-		ExecutableNode exNode = executableNodes.get(nodeId);
-		if (exNode == null) {
-			return;
-		}
-
-		exNode.setExternalExecutionId(externalExecutionId);
 	}
 
 	public int getExecutionId() {
@@ -191,20 +104,9 @@ public class ExecutableFlow {
 
 	public void setExecutionId(int executionId) {
 		this.executionId = executionId;
-
-		for (ExecutableNode node: executableNodes.values()) {
-			node.setExecutionId(executionId);
-		}
-	}
-
-	public String getFlowId() {
-		return flowId;
-	}
-
-	public void setFlowId(String flowId) {
-		this.flowId = flowId;
 	}
 
+	@Override
 	public int getProjectId() {
 		return projectId;
 	}
@@ -229,265 +131,97 @@ public class ExecutableFlow {
 		this.executionPath = executionPath;
 	}
 
-	public long getStartTime() {
-		return startTime;
+	public String getSubmitUser() {
+		return submitUser;
 	}
 
-	public void setStartTime(long time) {
-		this.startTime = time;
+	public void setSubmitUser(String submitUser) {
+		this.submitUser = submitUser;
 	}
-
-	public long getEndTime() {
-		return endTime;
+	
+	@Override
+	public int getVersion() {
+		return version;
 	}
 
-	public void setEndTime(long time) {
-		this.endTime = time;
+	public void setVersion(int version) {
+		this.version = version;
 	}
 
 	public long getSubmitTime() {
 		return submitTime;
 	}
 
-	public void setSubmitTime(long time) {
-		this.submitTime = time;
-	}
-
-	public Status getStatus() {
-		return flowStatus;
-	}
-
-	public void setStatus(Status flowStatus) {
-		this.flowStatus = flowStatus;
+	public void setSubmitTime(long submitTime) {
+		this.submitTime = submitTime;
 	}
 
 	public Map<String, Object> toObject() {
 		HashMap<String, Object> flowObj = new HashMap<String, Object>();
-		flowObj.put("type", "executableflow");
-		flowObj.put("executionId", executionId);
-		flowObj.put("executionPath", executionPath);
-		flowObj.put("flowId", flowId);
-		flowObj.put("projectId", projectId);
-
-		if (scheduleId >= 0) {
-			flowObj.put("scheduleId", scheduleId);
-		}
-		flowObj.put("submitTime", submitTime);
-		flowObj.put("startTime", startTime);
-		flowObj.put("endTime", endTime);
-		flowObj.put("status", flowStatus.toString());
-		flowObj.put("submitUser", submitUser);
-		flowObj.put("version", version);
-
-		flowObj.put("executionOptions", this.executionOptions.toObject());
-		flowObj.put("version", version);
-
-		ArrayList<Object> props = new ArrayList<Object>();
-		for (FlowProps fprop: flowProps.values()) {
-			HashMap<String, Object> propObj = new HashMap<String, Object>();
-			String source = fprop.getSource();
-			String inheritedSource = fprop.getInheritedSource();
-
-			propObj.put("source", source);
-			if (inheritedSource != null) {
-				propObj.put("inherited", inheritedSource);
-			}
-			props.add(propObj);
-		}
-		flowObj.put("properties", props);
-
-		ArrayList<Object> nodes = new ArrayList<Object>();
-		for (ExecutableNode node: executableNodes.values()) {
-			nodes.add(node.toObject());
-		}
-		flowObj.put("nodes", nodes);
-
+		fillMapFromExecutable(flowObj);
+		
+		flowObj.put(EXECUTIONID_PARAM, executionId);
+		flowObj.put(EXECUTIONPATH_PARAM, executionPath);		
+		flowObj.put(PROJECTID_PARAM, projectId);
+		
+		if(scheduleId >= 0) {
+			flowObj.put(SCHEDULEID_PARAM, scheduleId);
+		}
+
+		flowObj.put(SUBMITUSER_PARAM, submitUser);
+		flowObj.put(VERSION_PARAM, version);
+		
+		flowObj.put(EXECUTIONOPTIONS_PARAM, this.executionOptions.toObject());
+		flowObj.put(VERSION_PARAM, version);
+		
 		ArrayList<String> proxyUserList = new ArrayList<String>(proxyUsers);
-		flowObj.put("proxyUsers", proxyUserList);
+		flowObj.put(PROXYUSERS_PARAM, proxyUserList);
 
+		flowObj.put(SUBMITTIME_PARAM, submitTime);
+		
 		return flowObj;
 	}
 
-	public Object toUpdateObject(long lastUpdateTime) {
-		Map<String, Object> updateData = new HashMap<String, Object>();
-		updateData.put("execId", this.executionId);
-		updateData.put("status", this.flowStatus.getNumVal());
-		updateData.put("startTime", this.startTime);
-		updateData.put("endTime", this.endTime);
-		updateData.put("updateTime", this.updateTime);
-
-		List<Map<String, Object>> updatedNodes = new ArrayList<Map<String, Object>>();
-		for (ExecutableNode node: executableNodes.values()) {
-
-			if (node.getUpdateTime() > lastUpdateTime) {
-				Map<String, Object> updatedNodeMap = new HashMap<String, Object>();
-				updatedNodeMap.put("jobId", node.getJobId());
-				updatedNodeMap.put("status", node.getStatus().getNumVal());
-				updatedNodeMap.put("startTime", node.getStartTime());
-				updatedNodeMap.put("endTime", node.getEndTime());
-				updatedNodeMap.put("updateTime", node.getUpdateTime());
-				updatedNodeMap.put("attempt", node.getAttempt());
-
-				if (node.getAttempt() > 0) {
-					ArrayList<Map<String, Object>> pastAttempts = new ArrayList<Map<String, Object>>();
-					for (Attempt attempt: node.getPastAttemptList()) {
-						pastAttempts.add(attempt.toObject());
-					}
-					updatedNodeMap.put("pastAttempts", pastAttempts);
-				}
-
-				updatedNodes.add(updatedNodeMap);
-			}
-		}
-
-		updateData.put("nodes", updatedNodes);
-		return updateData;
-	}
-
-	@SuppressWarnings("unchecked")
-	public void applyUpdateObject(Map<String, Object> updateData) {
-		List<Map<String, Object>> updatedNodes = (List<Map<String, Object>>)updateData.get("nodes");
-		for (Map<String, Object> node: updatedNodes) {
-			String jobId = (String)node.get("jobId");
-			Status status = Status.fromInteger((Integer)node.get("status"));
-			long startTime = JSONUtils.getLongFromObject(node.get("startTime"));
-			long endTime = JSONUtils.getLongFromObject(node.get("endTime"));
-			long updateTime = JSONUtils.getLongFromObject(node.get("updateTime"));
-
-			ExecutableNode exNode = executableNodes.get(jobId);
-			exNode.setEndTime(endTime);
-			exNode.setStartTime(startTime);
-			exNode.setUpdateTime(updateTime);
-			exNode.setStatus(status);
-
-			int attempt = 0;
-			if (node.containsKey("attempt")) {
-				attempt = (Integer)node.get("attempt");
-				if (attempt > 0) {
-					exNode.updatePastAttempts((List<Object>)node.get("pastAttempts"));
-				}
-			}
-
-			exNode.setAttempt(attempt);
-		}
-
-		this.flowStatus = Status.fromInteger((Integer)updateData.get("status"));
-
-		this.startTime = JSONUtils.getLongFromObject(updateData.get("startTime"));
-		this.endTime = JSONUtils.getLongFromObject(updateData.get("endTime"));
-		this.updateTime = JSONUtils.getLongFromObject(updateData.get("updateTime"));
-	}
-
 	@SuppressWarnings("unchecked")
 	public static ExecutableFlow createExecutableFlowFromObject(Object obj) {
 		ExecutableFlow exFlow = new ExecutableFlow();
-
-		HashMap<String, Object> flowObj = (HashMap<String, Object>)obj;
-		exFlow.executionId = (Integer)flowObj.get("executionId");
-		exFlow.executionPath = (String)flowObj.get("executionPath");
-		exFlow.flowId = (String)flowObj.get("flowId");
-		exFlow.projectId = (Integer)flowObj.get("projectId");
-		if (flowObj.containsKey("scheduleId")) {
-			exFlow.scheduleId = (Integer)flowObj.get("scheduleId");
-		}
-		exFlow.submitTime = JSONUtils.getLongFromObject(flowObj.get("submitTime"));
-		exFlow.startTime = JSONUtils.getLongFromObject(flowObj.get("startTime"));
-		exFlow.endTime = JSONUtils.getLongFromObject(flowObj.get("endTime"));
-		exFlow.flowStatus = Status.valueOf((String)flowObj.get("status"));
-		exFlow.submitUser = (String)flowObj.get("submitUser");
-		exFlow.version = (Integer)flowObj.get("version");
-
-		if (flowObj.containsKey("executionOptions")) {
-			exFlow.executionOptions = ExecutionOptions.createFromObject(flowObj.get("executionOptions"));
-		}
-		else {
-			// for backawards compatibility should remove in a few versions.
-			exFlow.executionOptions = ExecutionOptions.createFromObject(flowObj);
-		}
-
-		// Copy nodes
-		List<Object> nodes = (List<Object>)flowObj.get("nodes");
-		for (Object nodeObj: nodes) {
-			ExecutableNode node = ExecutableNode.createNodeFromObject(nodeObj, exFlow);
-			exFlow.executableNodes.put(node.getJobId(), node);
-		}
-
-		List<Object> properties = (List<Object>)flowObj.get("properties");
-		for (Object propNode: properties) {
-			HashMap<String, Object> fprop = (HashMap<String, Object>)propNode;
-			String source = (String)fprop.get("source");
-			String inheritedSource = (String)fprop.get("inherited");
-
-			FlowProps flowProps = new FlowProps(inheritedSource, source);
-			exFlow.flowProps.put(source, flowProps);
-		}
-
-		if (flowObj.containsKey("proxyUsers")) {
-			ArrayList<String> proxyUserList = (ArrayList<String>)flowObj.get("proxyUsers");
-			exFlow.addAllProxyUsers(proxyUserList);
-		}
-
+		HashMap<String, Object> flowObj = (HashMap<String,Object>)obj;
+		exFlow.fillExecutableFromMapObject(flowObj);
+		
 		return exFlow;
 	}
-
-	@SuppressWarnings("unchecked")
-	public void updateExecutableFlowFromObject(Object obj) {
-		HashMap<String, Object> flowObj = (HashMap<String, Object>)obj;
-
-		submitTime = JSONUtils.getLongFromObject(flowObj.get("submitTime"));
-		startTime = JSONUtils.getLongFromObject(flowObj.get("startTime"));
-		endTime = JSONUtils.getLongFromObject(flowObj.get("endTime"));
-		flowStatus = Status.valueOf((String)flowObj.get("status"));
-
-		List<Object> nodes = (List<Object>)flowObj.get("nodes");
-		for (Object nodeObj: nodes) {
-			HashMap<String, Object> nodeHash = (HashMap<String, Object>)nodeObj;
-			String nodeId = (String)nodeHash.get("id");
-			ExecutableNode node = executableNodes.get(nodeId);
-			if (nodeId == null) {
-				throw new RuntimeException("Node " + nodeId + " doesn't exist in flow.");
-			}
-
-			node.updateNodeFromObject(nodeObj);
+	
+	@Override
+	public void fillExecutableFromMapObject(TypedMapWrapper<String, Object> flowObj) {
+		super.fillExecutableFromMapObject(flowObj);
+		
+		this.executionId = flowObj.getInt(EXECUTIONID_PARAM);
+		this.executionPath = flowObj.getString(EXECUTIONPATH_PARAM);
+
+		this.projectId = flowObj.getInt(PROJECTID_PARAM);
+		this.scheduleId = flowObj.getInt(SCHEDULEID_PARAM);
+		this.submitUser = flowObj.getString(SUBMITUSER_PARAM);
+		this.version = flowObj.getInt(VERSION_PARAM);
+		this.submitTime = flowObj.getLong(SUBMITTIME_PARAM);
+		
+		if (flowObj.containsKey(EXECUTIONOPTIONS_PARAM)) {
+			this.executionOptions = ExecutionOptions.createFromObject(flowObj.getObject(EXECUTIONOPTIONS_PARAM));
 		}
-	}
-
-	public Set<String> getSources() {
-		HashSet<String> set = new HashSet<String>();
-		for (ExecutableNode exNode: executableNodes.values()) {
-			set.add(exNode.getJobPropsSource());
+		else {
+			// for backwards compatibility should remove in a few versions.
+			this.executionOptions = ExecutionOptions.createFromObject(flowObj);
 		}
+		
+		if(flowObj.containsKey(PROXYUSERS_PARAM)) {
+			List<String> proxyUserList = flowObj.<String>getList(PROXYUSERS_PARAM);
+			this.addAllProxyUsers(proxyUserList);
 
-		for (FlowProps props: flowProps.values()) {
-			set.add(props.getSource());
 		}
-		return set;
-	}
-
-	public String getSubmitUser() {
-		return submitUser;
-	}
-
-	public void setSubmitUser(String submitUser) {
-		this.submitUser = submitUser;
-	}
-
-	public int getVersion() {
-		return version;
-	}
-
-	public void setVersion(int version) {
-		this.version = version;
 	}
 	
-	public static boolean isFinished(ExecutableFlow flow) {
-		switch(flow.getStatus()) {
-		case SUCCEEDED:
-		case FAILED:
-		case KILLED:
-			return true;
-		default:
-			return false;
-		}
+	public Map<String, Object> toUpdateObject(long lastUpdateTime) {
+		Map<String, Object> updateData = super.toUpdateObject(lastUpdateTime);
+		updateData.put(EXECUTIONID_PARAM, this.executionId);
+		return updateData;
 	}
-}
+}
\ No newline at end of file
diff --git a/src/java/azkaban/executor/ExecutableFlowBase.java b/src/java/azkaban/executor/ExecutableFlowBase.java
new file mode 100644
index 0000000..040888b
--- /dev/null
+++ b/src/java/azkaban/executor/ExecutableFlowBase.java
@@ -0,0 +1,409 @@
+/*
+ * Copyright 2012 LinkedIn, Inc
+ * 
+ * Licensed under the Apache License, Version 2.0 (the "License"); you may not
+ * use this file except in compliance with the License. You may obtain a copy of
+ * the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package azkaban.executor;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import azkaban.flow.Edge;
+import azkaban.flow.Flow;
+import azkaban.flow.FlowProps;
+import azkaban.flow.Node;
+import azkaban.flow.SpecialJobTypes;
+import azkaban.project.Project;
+import azkaban.utils.TypedMapWrapper;
+
+public class ExecutableFlowBase extends ExecutableNode {
+	public static final String FLOW_ID_PARAM = "flowId";
+	public static final String NODES_PARAM = "nodes";
+	public static final String PROPERTIES_PARAM = "properties";
+	public static final String SOURCE_PARAM = "source";
+	public static final String INHERITED_PARAM = "inherited";
+	
+	private HashMap<String, ExecutableNode> executableNodes = new HashMap<String, ExecutableNode>();
+	private ArrayList<String> startNodes;
+	private ArrayList<String> endNodes;
+	
+	private HashMap<String, FlowProps> flowProps = new HashMap<String, FlowProps>();
+	private String flowId;
+	
+	public ExecutableFlowBase(Project project, Node node, Flow flow, ExecutableFlowBase parent) {
+		super(node, parent);
+
+		setFlow(project, flow);
+	}
+	
+	public ExecutableFlowBase() {
+	}
+	
+	public int getExecutionId() {
+		if (this.getParentFlow() != null) {
+			return this.getParentFlow().getExecutionId();
+		}
+		
+		return -1;
+ 	}
+	
+	public int getProjectId() {
+		if (this.getParentFlow() != null) {
+			return this.getParentFlow().getProjectId();
+		}
+		
+		return -1;
+	}
+	
+	public int getVersion() {
+		if (this.getParentFlow() != null) {
+			return this.getParentFlow().getVersion();
+		}
+		
+		return -1;
+	}
+	
+	public Collection<FlowProps> getFlowProps() {
+		return flowProps.values();
+	}
+	
+	public String getFlowId() {
+		return flowId;
+	}
+	
+	protected void setFlow(Project project, Flow flow) {
+		this.flowId = flow.getId();
+		flowProps.putAll(flow.getAllFlowProps());
+		
+		for (Node node: flow.getNodes()) {
+			String id = node.getId();
+			if (node.getType().equals(SpecialJobTypes.EMBEDDED_FLOW_TYPE)) {
+				String embeddedFlowId = node.getEmbeddedFlowId();
+				Flow subFlow = project.getFlow(embeddedFlowId);
+				
+				ExecutableFlowBase embeddedFlow = new ExecutableFlowBase(project, node, subFlow, this);
+				executableNodes.put(id, embeddedFlow);
+			}
+			else {
+				ExecutableNode exNode = new ExecutableNode(node, this);
+				executableNodes.put(id, exNode);
+			}
+		}
+		
+		for (Edge edge: flow.getEdges()) {
+			ExecutableNode sourceNode = executableNodes.get(edge.getSourceId());
+			ExecutableNode targetNode = executableNodes.get(edge.getTargetId());
+			
+			if (sourceNode == null) {
+				System.out.println("Source node " + edge.getSourceId() + " doesn't exist");
+			}
+			sourceNode.addOutNode(edge.getTargetId());
+			targetNode.addInNode(edge.getSourceId());
+		}
+	}
+	
+	public List<ExecutableNode> getExecutableNodes() {
+		return new ArrayList<ExecutableNode>(executableNodes.values());
+	}
+	
+	public ExecutableNode getExecutableNode(String id) {
+		return executableNodes.get(id);
+	}
+	
+	public ExecutableNode getExecutableNode(String ... ids) {
+		return getExecutableNode(this, ids, 0);
+	}
+	
+	private ExecutableNode getExecutableNode(ExecutableFlowBase flow, String[] ids, int currentIdIdx) {
+		ExecutableNode node = flow.getExecutableNode(ids[currentIdIdx]);
+		currentIdIdx++;
+		
+		if (node == null) {
+			return null;
+		}
+		
+		if (ids.length == currentIdIdx) {
+			return node;
+		}
+		else if (node instanceof ExecutableFlowBase) {
+			return getExecutableNode((ExecutableFlowBase)node, ids, currentIdIdx);
+		}
+		else {
+			return null;
+		}
+		
+	}
+	
+	public List<String> getStartNodes() {
+		if (startNodes == null) {
+			startNodes = new ArrayList<String>();
+			for (ExecutableNode node: executableNodes.values()) {
+				if (node.getInNodes().isEmpty()) {
+					startNodes.add(node.getId());
+				}
+			}
+		}
+		
+		return startNodes;
+	}
+	
+	public List<String> getEndNodes() {
+		if (endNodes == null) {
+			endNodes = new ArrayList<String>();
+			for (ExecutableNode node: executableNodes.values()) {
+				if (node.getOutNodes().isEmpty()) {
+					endNodes.add(node.getId());
+				}
+			}
+		}
+		
+		return endNodes;
+	}
+	
+	public Map<String,Object> toObject() {
+		Map<String,Object> mapObj = new HashMap<String,Object>();
+		fillMapFromExecutable(mapObj);
+		
+		return mapObj;
+	}
+	
+	protected void fillMapFromExecutable(Map<String,Object> flowObjMap) {
+		super.fillMapFromExecutable(flowObjMap);
+		
+		flowObjMap.put(FLOW_ID_PARAM, flowId);
+		
+		ArrayList<Object> nodes = new ArrayList<Object>();
+		for (ExecutableNode node: executableNodes.values()) {
+			nodes.add(node.toObject());
+		}
+		flowObjMap.put(NODES_PARAM, nodes);
+		
+		// Flow properties
+		ArrayList<Object> props = new ArrayList<Object>();
+		for (FlowProps fprop: flowProps.values()) {
+			HashMap<String, Object> propObj = new HashMap<String, Object>();
+			String source = fprop.getSource();
+			String inheritedSource = fprop.getInheritedSource();
+			
+			propObj.put(SOURCE_PARAM, source);
+			if (inheritedSource != null) {
+				propObj.put(INHERITED_PARAM, inheritedSource);
+			}
+			props.add(propObj);
+		}
+		flowObjMap.put(PROPERTIES_PARAM, props);
+	}
+
+	@Override
+	public void fillExecutableFromMapObject(TypedMapWrapper<String,Object> flowObjMap) {
+		super.fillExecutableFromMapObject(flowObjMap);
+		
+		this.flowId = flowObjMap.getString(FLOW_ID_PARAM);
+		List<Object> nodes = flowObjMap.<Object>getList(NODES_PARAM);
+		
+		if (nodes != null) {
+			for (Object nodeObj: nodes) {
+				@SuppressWarnings("unchecked")
+				Map<String,Object> nodeObjMap = (Map<String,Object>)nodeObj;
+				TypedMapWrapper<String,Object> wrapper = new TypedMapWrapper<String,Object>(nodeObjMap);
+				
+				String type = wrapper.getString(TYPE_PARAM);
+				if (type != null && type.equals(SpecialJobTypes.EMBEDDED_FLOW_TYPE)) {
+					ExecutableFlowBase exFlow = new ExecutableFlowBase();
+					exFlow.fillExecutableFromMapObject(wrapper);
+					exFlow.setParentFlow(this);
+					
+					executableNodes.put(exFlow.getId(), exFlow);
+				}
+				else {
+					ExecutableNode exJob = new ExecutableNode();
+					exJob.fillExecutableFromMapObject(nodeObjMap);
+					exJob.setParentFlow(this);
+					
+					executableNodes.put(exJob.getId(), exJob);
+				}
+			}
+		}
+		
+		List<Object> properties = flowObjMap.<Object>getList(PROPERTIES_PARAM);
+		for (Object propNode : properties) {
+			@SuppressWarnings("unchecked")
+			HashMap<String, Object> fprop = (HashMap<String, Object>)propNode;
+			String source = (String)fprop.get("source");
+			String inheritedSource = (String)fprop.get("inherited");
+			
+			FlowProps flowProps = new FlowProps(inheritedSource, source);
+			this.flowProps.put(source, flowProps);
+		}
+	}
+	
+	public Map<String, Object> toUpdateObject(long lastUpdateTime) {
+		Map<String, Object> updateData = super.toUpdateObject();
+		
+		List<Map<String,Object>> updatedNodes = new ArrayList<Map<String,Object>>();
+		for (ExecutableNode node: executableNodes.values()) {
+			if (node instanceof ExecutableFlowBase) {
+				Map<String, Object> updatedNodeMap = ((ExecutableFlowBase)node).toUpdateObject(lastUpdateTime);
+				// We add only flows to the list which either have a good update time, or has updated descendants.
+				if (node.getUpdateTime() > lastUpdateTime || updatedNodeMap.containsKey(NODES_PARAM)) {
+					updatedNodes.add(updatedNodeMap);
+				}
+			} 
+			else {
+				if (node.getUpdateTime() > lastUpdateTime) {
+					Map<String, Object> updatedNodeMap = node.toUpdateObject();
+					updatedNodes.add(updatedNodeMap);
+				}
+			}
+		}
+		
+		// if there are no updated nodes, we just won't add it to the list. This is good
+		// since if this is a nested flow, the parent is given the option to include or
+		// discard these subflows.
+		if (!updatedNodes.isEmpty()) {
+			updateData.put(NODES_PARAM, updatedNodes);
+		}
+		return updateData;
+	}
+	
+	public void applyUpdateObject(TypedMapWrapper<String, Object> updateData, List<ExecutableNode> updatedNodes) {
+		super.applyUpdateObject(updateData);
+		
+		if (updatedNodes != null) {
+			updatedNodes.add(this);
+		}
+
+		List<Map<String,Object>> nodes = (List<Map<String,Object>>)updateData.<Map<String,Object>>getList(NODES_PARAM);
+		if (nodes != null) {
+			for (Map<String,Object> node: nodes) {
+				TypedMapWrapper<String,Object> nodeWrapper = new TypedMapWrapper<String,Object>(node);
+				String id = nodeWrapper.getString(ID_PARAM);
+				if (id == null) {
+					// Legacy case
+					id = nodeWrapper.getString("jobId");				
+				}
+	
+				ExecutableNode exNode = executableNodes.get(id);
+				if (updatedNodes != null) {
+					updatedNodes.add(exNode);
+				}
+				
+				if (exNode instanceof ExecutableFlowBase) {
+					((ExecutableFlowBase)exNode).applyUpdateObject(nodeWrapper, updatedNodes);
+				}
+				else {
+					exNode.applyUpdateObject(nodeWrapper);
+				}
+			}
+		}
+	}
+	
+	public void applyUpdateObject(Map<String, Object> updateData, List<ExecutableNode> updatedNodes) {
+		TypedMapWrapper<String, Object> typedMapWrapper = new TypedMapWrapper<String,Object>(updateData);
+		applyUpdateObject(typedMapWrapper, updatedNodes);
+	}
+	
+	@Override
+	public void applyUpdateObject(Map<String, Object> updateData) {
+		TypedMapWrapper<String, Object> typedMapWrapper = new TypedMapWrapper<String,Object>(updateData);
+		applyUpdateObject(typedMapWrapper, null);
+	}
+	
+	public void reEnableDependents(ExecutableNode ... nodes) {
+		for(ExecutableNode node: nodes) {
+			for(String dependent: node.getOutNodes()) {
+				ExecutableNode dependentNode = getExecutableNode(dependent);
+				
+				if (dependentNode.getStatus() == Status.KILLED) {
+					dependentNode.setStatus(Status.READY);
+					dependentNode.setUpdateTime(System.currentTimeMillis());
+					reEnableDependents(dependentNode);
+	
+					if (dependentNode instanceof ExecutableFlowBase) {
+						
+						((ExecutableFlowBase)dependentNode).reEnableDependents();
+					}
+				}
+				else if (dependentNode.getStatus() == Status.SKIPPED) {
+					dependentNode.setStatus(Status.DISABLED);
+					dependentNode.setUpdateTime(System.currentTimeMillis());
+					reEnableDependents(dependentNode);
+				}
+			}
+		}
+	}
+	
+	/**
+	 * Only returns true if the status of all finished nodes is true.
+	 * @return
+	 */
+	public boolean isFlowFinished() {
+		for (String end: getEndNodes()) {
+			ExecutableNode node = getExecutableNode(end);
+			if (!Status.isStatusFinished(node.getStatus()) ) {
+				return false;
+			}
+		}
+		
+		return true;
+	}
+	
+	/**
+	 * Finds all jobs which are ready to run. This occurs when all of its 
+	 * dependency nodes are finished running.
+	 * 
+	 * It will also return any subflow that has been completed such that the
+	 * FlowRunner can properly handle them.
+	 * 
+	 * @param flow
+	 * @return
+	 */
+	public List<ExecutableNode> findNextJobsToRun() {
+		ArrayList<ExecutableNode> jobsToRun = new ArrayList<ExecutableNode>();
+		
+		if (isFlowFinished() && !Status.isStatusFinished(getStatus())) {
+			jobsToRun.add(this);
+		}
+		else {
+			nodeloop:
+			for (ExecutableNode node: executableNodes.values()) {
+				if(Status.isStatusFinished(node.getStatus())) {
+					continue;
+				}
+	
+				if ((node instanceof ExecutableFlowBase) && Status.isStatusRunning(node.getStatus())) {
+					// If the flow is still running, we traverse into the flow
+					jobsToRun.addAll(((ExecutableFlowBase)node).findNextJobsToRun());
+				}
+				else if (Status.isStatusRunning(node.getStatus())) {
+					continue;
+				}
+				else {
+					for (String dependency: node.getInNodes()) {
+						// We find that the outer-loop is unfinished.
+						if (!Status.isStatusFinished(getExecutableNode(dependency).getStatus())) {
+							continue nodeloop;
+						}
+					}
+	
+					jobsToRun.add(node);
+				}
+			}
+		}
+		
+		return jobsToRun;
+	}
+}
\ No newline at end of file
diff --git a/src/java/azkaban/executor/ExecutableNode.java b/src/java/azkaban/executor/ExecutableNode.java
index 1bea9b7..9d807ff 100644
--- a/src/java/azkaban/executor/ExecutableNode.java
+++ b/src/java/azkaban/executor/ExecutableNode.java
@@ -1,5 +1,5 @@
 /*
- * Copyright 2012 LinkedIn Corp.
+ * Copyright 2013 LinkedIn Corp.
  * 
  * 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
@@ -17,202 +17,122 @@
 package azkaban.executor;
 
 import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-
 import azkaban.flow.Node;
-import azkaban.utils.JSONUtils;
 import azkaban.utils.Props;
+import azkaban.utils.PropsUtils;
+import azkaban.utils.TypedMapWrapper;
 
+/**
+ * Base Executable that nodes and flows are based.
+ */
 public class ExecutableNode {
-	private String jobId;
-	private int executionId;
-	private String type;
-	private String jobPropsSource;
-	private String inheritPropsSource;
+	public static final String ID_PARAM = "id";
+	public static final String STATUS_PARAM = "status";
+	public static final String STARTTIME_PARAM = "startTime";
+	public static final String ENDTIME_PARAM = "endTime";
+	public static final String UPDATETIME_PARAM = "updateTime";
+	public static final String INNODES_PARAM = "inNodes";
+	public static final String OUTNODES_PARAM = "outNodes";
+	public static final String TYPE_PARAM = "type";
+	public static final String PROPS_SOURCE_PARAM = "propSource";
+	public static final String JOB_SOURCE_PARAM = "jobSource";
+	public static final String OUTPUT_PROPS_PARAM = "outputProps";
+	
+	private String id;
+	private String type = null;
 	private Status status = Status.READY;
 	private long startTime = -1;
 	private long endTime = -1;
 	private long updateTime = -1;
-	private int level = 0;
-	private ExecutableFlow flow;
-	private Props outputProps;
-	private int attempt = 0;
-	private boolean paused = false;
 	
-	private long delayExecution = 0;
-
+	// Path to Job File
+	private String jobSource; 
+	// Path to top level props file
+	private String propsSource;
 	private Set<String> inNodes = new HashSet<String>();
 	private Set<String> outNodes = new HashSet<String>();
 	
-	// Used if proxy node
-	private Integer externalExecutionId;
-	private ArrayList<Attempt> pastAttempts = null;
+	private Props inputProps;
+	private Props outputProps;
 	
-	public ExecutableNode(Node node, ExecutableFlow flow) {
-		jobId = node.getId();
-		executionId = flow.getExecutionId();
-		type = node.getType();
-		jobPropsSource = node.getJobSource();
-		inheritPropsSource = node.getPropsSource();
-		status = Status.READY;
-		level = node.getLevel();
-		this.flow = flow;
-	}
+	public static final String ATTEMPT_PARAM = "attempt";
+	public static final String PASTATTEMPTS_PARAM = "pastAttempts";
 	
-	public ExecutableNode() {
-	}
+	private int attempt = 0;
+	private long delayExecution = 0;
+	private ArrayList<ExecutionAttempt> pastAttempts = null;
 	
-	public void resetForRetry() {
-		Attempt pastAttempt = new Attempt(attempt, startTime, endTime, status);
-		attempt++;
-		
-		synchronized (this) {
-			if (pastAttempts == null) {
-				pastAttempts = new ArrayList<Attempt>();
-			}
-			
-			pastAttempts.add(pastAttempt);
-		}
-		startTime = -1;
-		endTime = -1;
-		updateTime = System.currentTimeMillis();
-		status = Status.READY;
-	}
+	// Transient. These values aren't saved, but rediscovered.
+	private ExecutableFlowBase parentFlow; 
 	
-	public void setExecutableFlow(ExecutableFlow flow) {
-		this.flow = flow;
+	public ExecutableNode(Node node) {
+		this.id = node.getId();
+		this.jobSource = node.getJobSource();
+		this.propsSource = node.getPropsSource();
 	}
 	
-	public void setExecutionId(int id) {
-		executionId = id;
+	public ExecutableNode(Node node, ExecutableFlowBase parent) {
+		this(node.getId(), node.getType(), node.getJobSource(), node.getPropsSource(), parent);
 	}
 
-	public int getExecutionId() {
-		return executionId;
+	public ExecutableNode(String id, String type, String jobSource, String propsSource, ExecutableFlowBase parent) {
+		this.id = id;
+		this.jobSource = jobSource;
+		this.propsSource = propsSource;
+		this.type = type;
+		setParentFlow(parent);
 	}
-
-	public String getJobId() {
-		return jobId;
+	
+	public ExecutableNode() {
 	}
-
-	public void setJobId(String id) {
-		this.jobId = id;
+	
+	public ExecutableFlow getExecutableFlow() {
+		if (parentFlow == null) {
+			return null;
+		}
+		
+		return parentFlow.getExecutableFlow();
 	}
-
-	public void addInNode(String exNode) {
-		inNodes.add(exNode);
+	
+	public void setParentFlow(ExecutableFlowBase flow) {
+		this.parentFlow = flow;
 	}
-
-	public void addOutNode(String exNode) {
-		outNodes.add(exNode);
+	
+	public ExecutableFlowBase getParentFlow() {
+		return parentFlow;
 	}
-
-	public Set<String> getOutNodes() {
-		return outNodes;
+	
+	public String getId() {
+		return id;
 	}
 	
-	public Set<String> getInNodes() {
-		return inNodes;
+	public void setId(String id) {
+		this.id = id;
 	}
 	
 	public Status getStatus() {
 		return status;
 	}
 
-	public void setStatus(Status status) {
-		this.status = status;
+	public String getType() {
+		return type;
 	}
-	
-	public long getDelayedExecution() {
-		return delayExecution;
+
+	public void setType(String type) {
+		this.type = type;
 	}
 	
-	public void setDelayedExecution(long delayMs) {
-		delayExecution = delayMs;
+	public void setStatus(Status status) {
+		this.status = status;
 	}
 	
-	public Object toObject() {
-		HashMap<String, Object> objMap = new HashMap<String, Object>();
-		objMap.put("id", jobId);
-		objMap.put("jobSource", jobPropsSource);
-		objMap.put("propSource", inheritPropsSource);
-		objMap.put("jobType", type);
-		objMap.put("status", status.toString());
-		objMap.put("inNodes", new ArrayList<String>(inNodes));
-		objMap.put("outNodes", new ArrayList<String>(outNodes));
-		objMap.put("startTime", startTime);
-		objMap.put("endTime", endTime);
-		objMap.put("updateTime", updateTime);
-		objMap.put("level", level);
-		objMap.put("externalExecutionId", externalExecutionId);
-		objMap.put("paused", paused);
-		
-		if (pastAttempts != null) {
-			ArrayList<Object> attemptsList = new ArrayList<Object>(pastAttempts.size());
-			for (Attempt attempts : pastAttempts) {
-				attemptsList.add(attempts.toObject());
-			}
-			objMap.put("pastAttempts", attemptsList);
-		}
-		
-		return objMap;
-	}
-
-	@SuppressWarnings("unchecked")
-	public static ExecutableNode createNodeFromObject(Object obj, ExecutableFlow flow) {
-		ExecutableNode exNode = new ExecutableNode();
-		
-		HashMap<String, Object> objMap = (HashMap<String,Object>)obj;
-		exNode.executionId = flow == null ? 0 : flow.getExecutionId();
-		exNode.jobId = (String)objMap.get("id");
-		exNode.jobPropsSource = (String)objMap.get("jobSource");
-		exNode.inheritPropsSource = (String)objMap.get("propSource");
-		exNode.type = (String)objMap.get("jobType");
-		exNode.status = Status.valueOf((String)objMap.get("status"));
-		
-		exNode.inNodes.addAll( (List<String>)objMap.get("inNodes") );
-		exNode.outNodes.addAll( (List<String>)objMap.get("outNodes") );
-		
-		exNode.startTime = JSONUtils.getLongFromObject(objMap.get("startTime"));
-		exNode.endTime = JSONUtils.getLongFromObject(objMap.get("endTime"));
-		exNode.updateTime = JSONUtils.getLongFromObject(objMap.get("updateTime"));
-		exNode.level = (Integer)objMap.get("level");
-		
-		exNode.externalExecutionId = (Integer)objMap.get("externalExecutionId");
-		
-		exNode.flow = flow;
-		Boolean paused = (Boolean)objMap.get("paused");
-		if (paused!=null) {
-			exNode.paused = paused;
-		}
-		
-		List<Object> pastAttempts = (List<Object>)objMap.get("pastAttempts");
-		if (pastAttempts!=null) {
-			ArrayList<Attempt> attempts = new ArrayList<Attempt>();
-			for (Object attemptObj: pastAttempts) {
-				Attempt attempt = Attempt.fromObject(attemptObj);
-				attempts.add(attempt);
-			}
-			
-			exNode.pastAttempts = attempts;
-		}
-		
-		return exNode;
-	}
-
-	@SuppressWarnings("unchecked")
-	public void updateNodeFromObject(Object obj) {
-		HashMap<String, Object> objMap = (HashMap<String,Object>)obj;
-		status = Status.valueOf((String)objMap.get("status"));
-
-		startTime = JSONUtils.getLongFromObject(objMap.get("startTime"));
-		endTime = JSONUtils.getLongFromObject(objMap.get("endTime"));
-	}
-
 	public long getStartTime() {
 		return startTime;
 	}
@@ -228,23 +148,7 @@ public class ExecutableNode {
 	public void setEndTime(long endTime) {
 		this.endTime = endTime;
 	}
-
-	public String getJobPropsSource() {
-		return jobPropsSource;
-	}
-
-	public String getPropsSource() {
-		return inheritPropsSource;
-	}
-
-	public int getLevel() {
-		return level;
-	}
-
-	public ExecutableFlow getFlow() {
-		return flow;
-	}
-
+	
 	public long getUpdateTime() {
 		return updateTime;
 	}
@@ -252,24 +156,64 @@ public class ExecutableNode {
 	public void setUpdateTime(long updateTime) {
 		this.updateTime = updateTime;
 	}
+	
+	public void addOutNode(String exNode) {
+		outNodes.add(exNode);
+	}
+	
+	public void addInNode(String exNode) {
+		inNodes.add(exNode);
+	}
 
+	public Set<String> getOutNodes() {
+		return outNodes;
+	}
+	
+	public Set<String> getInNodes() {
+		return inNodes;
+	}
+	
+	public boolean hasJobSource() {
+		return jobSource != null;
+	}
+	
+	public boolean hasPropsSource() {
+		return propsSource != null;
+	}
+	
+	public String getJobSource() {
+		return jobSource;
+	}
+	
+	public String getPropsSource() {
+		return propsSource;
+	}
+	
+	public void setInputProps(Props input) {
+		this.inputProps = input;
+	}
+	
 	public void setOutputProps(Props output) {
 		this.outputProps = output;
 	}
 
+	public Props getInputProps() {
+		return this.inputProps;
+	}
+	
 	public Props getOutputProps() {
 		return outputProps;
 	}
-
-	public Integer getExternalExecutionId() {
-		return externalExecutionId;
+	
+	public long getDelayedExecution() {
+		return delayExecution;
 	}
-
-	public void setExternalExecutionId(Integer externalExecutionId) {
-		this.externalExecutionId = externalExecutionId;
+	
+	public void setDelayedExecution(long delayMs) {
+		delayExecution = delayMs;
 	}
-
-	public List<Attempt> getPastAttemptList() {
+	
+	public List<ExecutionAttempt> getPastAttemptList() {
 		return pastAttempts;
 	}
 	
@@ -281,96 +225,207 @@ public class ExecutableNode {
 		this.attempt = attempt;
 	}
 	
-	public boolean isPaused() {
-		return paused;
-	}
-	
-	public void setPaused(boolean paused) {
-		this.paused = paused;
+	public void resetForRetry() {
+		ExecutionAttempt pastAttempt = new ExecutionAttempt(attempt, this);
+		attempt++;
+		
+		synchronized (this) {
+			if (pastAttempts == null) {
+				pastAttempts = new ArrayList<ExecutionAttempt>();
+			}
+			
+			pastAttempts.add(pastAttempt);
+		}
+		
+		this.setStartTime(-1);
+		this.setEndTime(-1);
+		this.setUpdateTime(System.currentTimeMillis());
+		this.setStatus(Status.READY);
 	}
 	
 	public List<Object> getAttemptObjects() {
 		ArrayList<Object> array = new ArrayList<Object>();
 		
-		for (Attempt attempt: pastAttempts) {
+		for (ExecutionAttempt attempt: pastAttempts) {
 			array.add(attempt.toObject());
 		}
 		
 		return array;
 	}
 	
+	public String getNestedId() {
+		return getPrintableId(":");
+	}
 	
-	public void updatePastAttempts(List<Object> pastAttemptsList) {
-		if (pastAttemptsList == null) {
-			return;
+	public String getPrintableId(String delimiter) {
+		if (this.getParentFlow() == null || this.getParentFlow() instanceof ExecutableFlow) {
+			return getId();
 		}
+		return getParentFlow().getPrintableId(delimiter) + delimiter + getId();
+	}
+	
+	public Map<String,Object> toObject() {
+		Map<String,Object> mapObj = new HashMap<String,Object>();
+		fillMapFromExecutable(mapObj);
 		
-		synchronized (this) {
-			if (this.pastAttempts == null) {
-				this.pastAttempts = new ArrayList<Attempt>();
-			}
-
-			// We just check size because past attempts don't change
-			if (pastAttemptsList.size() <= this.pastAttempts.size()) {
-				return;
+		return mapObj;
+	}
+	
+	protected void fillMapFromExecutable(Map<String,Object> objMap) {
+		objMap.put(ID_PARAM, this.id);
+		objMap.put(STATUS_PARAM, status.toString());
+		objMap.put(STARTTIME_PARAM, startTime);
+		objMap.put(ENDTIME_PARAM, endTime);
+		objMap.put(UPDATETIME_PARAM, updateTime);
+		objMap.put(TYPE_PARAM, type);
+		objMap.put(ATTEMPT_PARAM, attempt);
+		
+		if (inNodes != null && !inNodes.isEmpty()) {
+			objMap.put(INNODES_PARAM, inNodes);
+		}
+		if (outNodes != null && !outNodes.isEmpty()) {
+			objMap.put(OUTNODES_PARAM, outNodes);
+		}
+		
+		if (hasPropsSource()) {
+			objMap.put(PROPS_SOURCE_PARAM, this.propsSource);
+		}
+		if (hasJobSource()) {
+			objMap.put(JOB_SOURCE_PARAM, this.jobSource);
+		}
+		
+		if (outputProps != null && outputProps.size() > 0) {
+			objMap.put(OUTPUT_PROPS_PARAM, PropsUtils.toStringMap(outputProps, true));
+		}
+		
+		if (pastAttempts != null) {
+			ArrayList<Object> attemptsList = new ArrayList<Object>(pastAttempts.size());
+			for (ExecutionAttempt attempts : pastAttempts) {
+				attemptsList.add(attempts.toObject());
 			}
-
-			Object[] pastAttemptArray = pastAttemptsList.toArray();
-			for (int i = this.pastAttempts.size(); i < pastAttemptArray.length; ++i) {
-				Attempt attempt = Attempt.fromObject(pastAttemptArray[i]);
-				this.pastAttempts.add(attempt);
+			objMap.put(PASTATTEMPTS_PARAM, attemptsList);
+		}
+	}
+	
+	@SuppressWarnings("unchecked")
+	public void fillExecutableFromMapObject(TypedMapWrapper<String, Object> wrappedMap) {
+		this.id = wrappedMap.getString(ID_PARAM);
+		this.type = wrappedMap.getString(TYPE_PARAM);
+		this.status = Status.valueOf(wrappedMap.getString(STATUS_PARAM));
+		this.startTime = wrappedMap.getLong(STARTTIME_PARAM);
+		this.endTime = wrappedMap.getLong(ENDTIME_PARAM);
+		this.updateTime = wrappedMap.getLong(UPDATETIME_PARAM);
+		this.attempt = wrappedMap.getInt(ATTEMPT_PARAM, 0);
+
+		this.inNodes = new HashSet<String>();
+		this.inNodes.addAll(wrappedMap.getStringCollection(INNODES_PARAM, Collections.<String>emptySet()));
+		
+		this.outNodes = new HashSet<String>();
+		this.outNodes.addAll(wrappedMap.getStringCollection(OUTNODES_PARAM, Collections.<String>emptySet()));
+		
+		this.propsSource = wrappedMap.getString(PROPS_SOURCE_PARAM);
+		this.jobSource = wrappedMap.getString(JOB_SOURCE_PARAM);
+		
+		Map<String, String> outputProps = wrappedMap.<String,String>getMap(OUTPUT_PROPS_PARAM);
+		if (outputProps != null) {
+			this.outputProps = new Props(null, outputProps);
+		}
+		
+		Collection<Object> pastAttempts = wrappedMap.<Object>getCollection(PASTATTEMPTS_PARAM);
+		if (pastAttempts!=null) {
+			ArrayList<ExecutionAttempt> attempts = new ArrayList<ExecutionAttempt>();
+			for (Object attemptObj: pastAttempts) {
+				ExecutionAttempt attempt = ExecutionAttempt.fromObject(attemptObj);
+				attempts.add(attempt);
 			}
+			
+			this.pastAttempts = attempts;
 		}
+	}
 
+	public void fillExecutableFromMapObject(Map<String,Object> objMap) {
+		TypedMapWrapper<String, Object> wrapper = new TypedMapWrapper<String, Object>(objMap);
+		fillExecutableFromMapObject(wrapper);
 	}
 
-	public static class Attempt {
-		private int attempt = 0;
-		private long startTime = -1;
-		private long endTime = -1;
-		private Status status;
+	public Map<String, Object> toUpdateObject() {
+		Map<String, Object> updatedNodeMap = new HashMap<String,Object>();
+		updatedNodeMap.put(ID_PARAM, getId());
+		updatedNodeMap.put(STATUS_PARAM, getStatus().getNumVal());
+		updatedNodeMap.put(STARTTIME_PARAM, getStartTime());
+		updatedNodeMap.put(ENDTIME_PARAM, getEndTime());
+		updatedNodeMap.put(UPDATETIME_PARAM, getUpdateTime());
 		
-		public Attempt(int attempt, long startTime, long endTime, Status status) {
-			this.attempt = attempt;
-			this.startTime = startTime;
-			this.endTime = endTime;
-			this.status = status;
+		updatedNodeMap.put(ATTEMPT_PARAM, getAttempt());
+
+		if (getAttempt() > 0) {
+			ArrayList<Map<String,Object>> pastAttempts = new ArrayList<Map<String,Object>>();
+			for (ExecutionAttempt attempt: getPastAttemptList()) {
+				pastAttempts.add(attempt.toObject());
+			}
+			updatedNodeMap.put(PASTATTEMPTS_PARAM, pastAttempts);
 		}
 		
-		public long getStartTime() {
-			return startTime;
+		return updatedNodeMap;
+	}
+	
+	public void applyUpdateObject(TypedMapWrapper<String, Object> updateData) {
+		this.status = Status.fromInteger(updateData.getInt(STATUS_PARAM, this.status.getNumVal()));
+		this.startTime = updateData.getLong(STARTTIME_PARAM);
+		this.updateTime = updateData.getLong(UPDATETIME_PARAM);
+		this.endTime = updateData.getLong(ENDTIME_PARAM);
+		
+		if (updateData.containsKey(ATTEMPT_PARAM)) {
+			attempt = updateData.getInt(ATTEMPT_PARAM);
+			if (attempt > 0) {
+				updatePastAttempts(
+						updateData.<Object>getList(PASTATTEMPTS_PARAM, Collections.<Object>emptyList()));
+			}
 		}
+	}
 
-		public long getEndTime() {
-			return endTime;
-		}
-		
-		public Status getStatus() {
-			return status;
+	public void applyUpdateObject(Map<String, Object> updateData) {
+		TypedMapWrapper<String,Object> wrapper = new TypedMapWrapper<String,Object>(updateData);
+		applyUpdateObject(wrapper);
+	}
+	
+	public void killNode(long killTime) {
+		if (this.status == Status.DISABLED) {
+			skipNode(killTime);
 		}
-		
-		public int getAttempt() {
-			return attempt;
+		else {
+			this.setStatus(Status.KILLED);
+			this.setStartTime(killTime);
+			this.setEndTime(killTime);
 		}
-		
-		public static Attempt fromObject(Object obj) {
-			@SuppressWarnings("unchecked")
-			Map<String, Object> map = (Map<String, Object>)obj;
-			int attempt = (Integer)map.get("attempt");
-			long startTime = JSONUtils.getLongFromObject(map.get("startTime"));
-			long endTime = JSONUtils.getLongFromObject(map.get("endTime"));
-			Status status = Status.valueOf((String)map.get("status"));
-			
-			return new Attempt(attempt, startTime, endTime, status);
+	}
+	
+	public void skipNode(long skipTime) {
+		this.setStatus(Status.SKIPPED);
+		this.setStartTime(skipTime);
+		this.setEndTime(skipTime);
+	}
+	
+	private void updatePastAttempts(List<Object> pastAttemptsList) {
+		if (pastAttemptsList == null) {
+			return;
 		}
 		
-		public Map<String, Object> toObject() {
-			HashMap<String,Object> attempts = new HashMap<String,Object>();
-			attempts.put("attempt", attempt);
-			attempts.put("startTime", startTime);
-			attempts.put("endTime", endTime);
-			attempts.put("status", status.toString());
-			return attempts;
+		synchronized (this) {
+			if (this.pastAttempts == null) {
+				this.pastAttempts = new ArrayList<ExecutionAttempt>();
+			}
+
+			// We just check size because past attempts don't change
+			if (pastAttemptsList.size() <= this.pastAttempts.size()) {
+				return;
+			}
+
+			Object[] pastAttemptArray = pastAttemptsList.toArray();
+			for (int i = this.pastAttempts.size(); i < pastAttemptArray.length; ++i) {
+				ExecutionAttempt attempt = ExecutionAttempt.fromObject(pastAttemptArray[i]);
+				this.pastAttempts.add(attempt);
+			}
 		}
 	}
-}
+}
\ No newline at end of file
diff --git a/src/java/azkaban/executor/ExecutionAttempt.java b/src/java/azkaban/executor/ExecutionAttempt.java
new file mode 100644
index 0000000..7712010
--- /dev/null
+++ b/src/java/azkaban/executor/ExecutionAttempt.java
@@ -0,0 +1,69 @@
+package azkaban.executor;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import azkaban.utils.TypedMapWrapper;
+
+public class ExecutionAttempt {
+	public static final String ATTEMPT_PARAM = "attempt";
+	public static final String STATUS_PARAM = "status";
+	public static final String STARTTIME_PARAM = "startTime";
+	public static final String ENDTIME_PARAM = "endTime";
+	
+	private int attempt = 0;
+	private long startTime = -1;
+	private long endTime = -1;
+	private Status status;
+	
+	public ExecutionAttempt(int attempt, ExecutableNode executable) {
+		this.attempt = attempt;
+		this.startTime = executable.getStartTime();
+		this.endTime = executable.getEndTime();
+		this.status = executable.getStatus();
+	}
+	
+	public ExecutionAttempt(int attempt, long startTime, long endTime, Status status) {
+		this.attempt = attempt;
+		this.startTime = startTime;
+		this.endTime = endTime;
+		this.status = status;
+	}
+	
+	public long getStartTime() {
+		return startTime;
+	}
+
+	public long getEndTime() {
+		return endTime;
+	}
+	
+	public Status getStatus() {
+		return status;
+	}
+	
+	public int getAttempt() {
+		return attempt;
+	}
+	
+	public static ExecutionAttempt fromObject(Object obj) {
+		@SuppressWarnings("unchecked")
+		Map<String, Object> map = (Map<String, Object>)obj;
+		TypedMapWrapper<String, Object> wrapper = new TypedMapWrapper<String, Object>(map);
+		int attempt = wrapper.getInt(ATTEMPT_PARAM);
+		long startTime = wrapper.getLong(STARTTIME_PARAM);
+		long endTime = wrapper.getLong(ENDTIME_PARAM);
+		Status status = Status.valueOf(wrapper.getString(STATUS_PARAM));
+		
+		return new ExecutionAttempt(attempt, startTime, endTime, status);
+	}
+	
+	public Map<String, Object> toObject() {
+		HashMap<String,Object> attempts = new HashMap<String,Object>();
+		attempts.put(ATTEMPT_PARAM, attempt);
+		attempts.put(STARTTIME_PARAM, startTime);
+		attempts.put(ENDTIME_PARAM, endTime);
+		attempts.put(STATUS_PARAM, status.toString());
+		return attempts;
+	}
+}
\ No newline at end of file
diff --git a/src/java/azkaban/executor/ExecutionOptions.java b/src/java/azkaban/executor/ExecutionOptions.java
index 1461dcc..7913751 100644
--- a/src/java/azkaban/executor/ExecutionOptions.java
+++ b/src/java/azkaban/executor/ExecutionOptions.java
@@ -1,5 +1,5 @@
 /*
- * Copyright 2012 LinkedIn Corp.
+ * Copyright 2013 LinkedIn Corp.
  * 
  * 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
@@ -18,13 +18,13 @@ package azkaban.executor;
 
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
-import java.util.Set;
 
 import azkaban.executor.mail.DefaultMailCreator;
+import azkaban.utils.TypedMapWrapper;
 
 /**
  * Execution options for submitted flows and scheduled flows
@@ -34,6 +34,21 @@ public class ExecutionOptions {
 	public static final String CONCURRENT_OPTION_PIPELINE="pipeline";
 	public static final String CONCURRENT_OPTION_IGNORE="ignore";
 	
+	private static final String FLOW_PARAMETERS = "flowParameters";
+	private static final String NOTIFY_ON_FIRST_FAILURE = "notifyOnFirstFailure";
+	private static final String NOTIFY_ON_LAST_FAILURE = "notifyOnLastFailure";
+	private static final String SUCCESS_EMAILS = "successEmails";
+	private static final String FAILURE_EMAILS = "failureEmails";
+	private static final String FAILURE_ACTION = "failureAction";
+	private static final String PIPELINE_LEVEL = "pipelineLevel";
+	private static final String PIPELINE_EXECID = "pipelineExecId";
+	private static final String QUEUE_LEVEL = "queueLevel";
+	private static final String CONCURRENT_OPTION = "concurrentOption";
+	private static final String DISABLE = "disabled";
+	private static final String FAILURE_EMAILS_OVERRIDE = "failureEmailsOverride";
+	private static final String SUCCESS_EMAILS_OVERRIDE = "successEmailsOverride";
+	private static final String MAIL_CREATOR = "mailCreator";	
+
 	private boolean notifyOnFirstFailure = true;
 	private boolean notifyOnLastFailure = false;
 	private boolean failureEmailsOverride = false;
@@ -56,9 +71,9 @@ public class ExecutionOptions {
 	
 	private FailureAction failureAction = FailureAction.FINISH_CURRENTLY_RUNNING;
 	
-	private Set<String> initiallyDisabledJobs = new HashSet<String>();
+	private List<Object> initiallyDisabledJobs = new ArrayList<Object>();
 	
-	public void setFlowParameters(Map<String,String> flowParam) {
+	public void addAllFlowParameters(Map<String,String> flowParam) {
 		flowParameters.putAll(flowParam);
 	}
 	
@@ -158,31 +173,31 @@ public class ExecutionOptions {
 		return queueLevel;
 	}
 	
-	public List<String> getDisabledJobs() {
-		return new ArrayList<String>(initiallyDisabledJobs);
+	public List<Object> getDisabledJobs() {
+		return new ArrayList<Object>(initiallyDisabledJobs);
 	}
 	
-	public void setDisabledJobs(List<String> disabledJobs) {
-		initiallyDisabledJobs = new HashSet<String>(disabledJobs);
+	public void setDisabledJobs(List<Object> disabledJobs) {
+		initiallyDisabledJobs = disabledJobs;
 	}
 	
 	public Map<String,Object> toObject() {
 		HashMap<String,Object> flowOptionObj = new HashMap<String,Object>();
-		
-		flowOptionObj.put("flowParameters", this.flowParameters);
-		flowOptionObj.put("notifyOnFirstFailure", this.notifyOnFirstFailure);
-		flowOptionObj.put("notifyOnLastFailure", this.notifyOnLastFailure);
-		flowOptionObj.put("successEmails", successEmails);
-		flowOptionObj.put("failureEmails", failureEmails);
-		flowOptionObj.put("failureAction", failureAction.toString());
-		flowOptionObj.put("pipelineLevel", pipelineLevel);
-		flowOptionObj.put("pipelineExecId", pipelineExecId);
-		flowOptionObj.put("queueLevel", queueLevel);
-		flowOptionObj.put("concurrentOption", concurrentOption);
-		flowOptionObj.put("mailCreator", mailCreator);
-		flowOptionObj.put("disabled", initiallyDisabledJobs);
-		flowOptionObj.put("failureEmailsOverride", failureEmailsOverride);
-		flowOptionObj.put("successEmailsOverride", successEmailsOverride);
+
+		flowOptionObj.put(FLOW_PARAMETERS, this.flowParameters);
+		flowOptionObj.put(NOTIFY_ON_FIRST_FAILURE, this.notifyOnFirstFailure);
+		flowOptionObj.put(NOTIFY_ON_LAST_FAILURE, this.notifyOnLastFailure);
+		flowOptionObj.put(SUCCESS_EMAILS, successEmails);
+		flowOptionObj.put(FAILURE_EMAILS, failureEmails);
+		flowOptionObj.put(FAILURE_ACTION, failureAction.toString());
+		flowOptionObj.put(PIPELINE_LEVEL, pipelineLevel);
+		flowOptionObj.put(PIPELINE_EXECID, pipelineExecId);
+		flowOptionObj.put(QUEUE_LEVEL, queueLevel);
+		flowOptionObj.put(CONCURRENT_OPTION, concurrentOption);
+		flowOptionObj.put(DISABLE, initiallyDisabledJobs);
+		flowOptionObj.put(FAILURE_EMAILS_OVERRIDE, failureEmailsOverride);
+		flowOptionObj.put(SUCCESS_EMAILS_OVERRIDE, successEmailsOverride);
+		flowOptionObj.put(MAIL_CREATOR, mailCreator);
 		return flowOptionObj;
 	}
 	
@@ -193,52 +208,39 @@ public class ExecutionOptions {
 		}
 		
 		Map<String,Object> optionsMap = (Map<String,Object>)obj;
+		TypedMapWrapper<String,Object> wrapper = new TypedMapWrapper<String,Object>(optionsMap);
 		
 		ExecutionOptions options = new ExecutionOptions();
-		if (optionsMap.containsKey("flowParameters")) {
-			options.flowParameters = new HashMap<String, String>((Map<String,String>)optionsMap.get("flowParameters"));
+		if (optionsMap.containsKey(FLOW_PARAMETERS)) {
+			options.flowParameters = new HashMap<String, String>();
+			options.flowParameters.putAll(wrapper.<String,String>getMap(FLOW_PARAMETERS));
 		}
 		// Failure notification
-		if (optionsMap.containsKey("notifyOnFirstFailure")) {
-			options.notifyOnFirstFailure = (Boolean)optionsMap.get("notifyOnFirstFailure");
-		}
-		if (optionsMap.containsKey("notifyOnLastFailure")) {
-			options.notifyOnLastFailure = (Boolean)optionsMap.get("notifyOnLastFailure");
-		}
-		if (optionsMap.containsKey("concurrentOption")) {
-			options.concurrentOption = (String)optionsMap.get("concurrentOption");
-		}
-		if (optionsMap.containsKey("mailCreator")) {
-			options.mailCreator = (String)optionsMap.get("mailCreator");
-		}
-		if (optionsMap.containsKey("disabled")) {
-			options.initiallyDisabledJobs = new HashSet<String>((List<String>)optionsMap.get("disabled"));
-		}
+		options.notifyOnFirstFailure = wrapper.getBool(NOTIFY_ON_FIRST_FAILURE, options.notifyOnFirstFailure);
+		options.notifyOnLastFailure = wrapper.getBool(NOTIFY_ON_LAST_FAILURE, options.notifyOnLastFailure);
+		options.concurrentOption = wrapper.getString(CONCURRENT_OPTION, options.concurrentOption);
 		
-		// Failure action
-		if (optionsMap.containsKey("failureAction")) {
-			options.failureAction = FailureAction.valueOf((String)optionsMap.get("failureAction"));
+		if (wrapper.containsKey(DISABLE)) {
+			options.initiallyDisabledJobs = wrapper.<Object>getList(DISABLE);
 		}
-		options.pipelineLevel = (Integer)optionsMap.get("pipelineLevel");
-		options.pipelineExecId = (Integer)optionsMap.get("pipelineExecId");
-		options.queueLevel = (Integer)optionsMap.get("queueLevel");
 		
-		// Success emails
-		if (optionsMap.containsKey("successEmails")) {
-			options.setSuccessEmails((List<String>)optionsMap.get("successEmails"));
-		}
-		// Failure emails
-		if (optionsMap.containsKey("failureEmails")) {
-			options.setFailureEmails((List<String>)optionsMap.get("failureEmails"));
+		if (optionsMap.containsKey(MAIL_CREATOR)) {
+			options.mailCreator = (String)optionsMap.get(MAIL_CREATOR);
 		}
+
+		// Failure action
+		options.failureAction = FailureAction.valueOf(wrapper.getString(FAILURE_ACTION, options.failureAction.toString()));
+		options.pipelineLevel = wrapper.getInt(PIPELINE_LEVEL, options.pipelineLevel);
+		options.pipelineExecId = wrapper.getInt(PIPELINE_EXECID, options.pipelineExecId);
+		options.queueLevel = wrapper.getInt(QUEUE_LEVEL, options.queueLevel);
+
 		
-		if (optionsMap.containsKey("successEmailsOverride")) {
-			options.setSuccessEmailsOverridden((Boolean)optionsMap.get("successEmailsOverride"));
-		}
+		// Success emails
+		options.setSuccessEmails(wrapper.<String>getList(SUCCESS_EMAILS, Collections.<String>emptyList()));
+		options.setFailureEmails(wrapper.<String>getList(FAILURE_EMAILS, Collections.<String>emptyList()));
 		
-		if (optionsMap.containsKey("failureEmailsOverride")) {
-			options.setFailureEmailsOverridden((Boolean)optionsMap.get("failureEmailsOverride"));
-		}
+		options.setSuccessEmailsOverridden(wrapper.getBool(SUCCESS_EMAILS_OVERRIDE, false));
+		options.setFailureEmailsOverridden(wrapper.getBool(FAILURE_EMAILS_OVERRIDE, false));
 		
 		return options;
 	}
diff --git a/src/java/azkaban/executor/ExecutorManager.java b/src/java/azkaban/executor/ExecutorManager.java
index 06af3b7..cb763f5 100644
--- a/src/java/azkaban/executor/ExecutorManager.java
+++ b/src/java/azkaban/executor/ExecutorManager.java
@@ -1,5 +1,5 @@
 /*
- * Copyright 2012 LinkedIn Corp.
+ * Copyright 2014 LinkedIn Corp.
  * 
  * 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
@@ -419,6 +419,34 @@ public class ExecutorManager extends EventHandler implements ExecutorManagerAdap
 		}
 	}
 	
+	private void applyDisabledJobs(List<Object> disabledJobs, ExecutableFlowBase exflow) {
+		for (Object disabled: disabledJobs) {
+			if (disabled instanceof String) {
+				String nodeName = (String)disabled;
+				ExecutableNode node = exflow.getExecutableNode(nodeName);
+				if (node != null) {
+					node.setStatus(Status.DISABLED);
+				}
+			}
+			else if (disabled instanceof Map) {
+				@SuppressWarnings("unchecked")
+				Map<String,Object> nestedDisabled = (Map<String, Object>)disabled;
+				String nodeName = (String)nestedDisabled.get("id");
+				@SuppressWarnings("unchecked")
+				List<Object> subDisabledJobs = (List<Object>)nestedDisabled.get("children");
+				
+				if (nodeName == null || subDisabledJobs == null) {
+					return;
+				}
+				
+				ExecutableNode node = exflow.getExecutableNode(nodeName);
+				if (node != null && node instanceof ExecutableFlowBase) {
+					applyDisabledJobs(subDisabledJobs, (ExecutableFlowBase)node);
+				}
+			}
+		}
+	}
+	
 	@Override
 	public String submitExecutableFlow(ExecutableFlow exflow, String userId) throws ExecutorManagerException {
 		synchronized(exflow) {
@@ -426,6 +454,8 @@ public class ExecutorManager extends EventHandler implements ExecutorManagerAdap
 
 			int projectId = exflow.getProjectId();
 			String flowId = exflow.getFlowId();
+			exflow.setSubmitUser(userId);
+			exflow.setSubmitTime(System.currentTimeMillis());
 			
 			List<Integer> running = getRunningFlows(projectId, flowId);
 
@@ -434,29 +464,25 @@ public class ExecutorManager extends EventHandler implements ExecutorManagerAdap
 				options = new ExecutionOptions();
 			}
 			
+			String message = "";
 			if (options.getDisabledJobs() != null) {
-				// Disable jobs
-				for(String disabledId : options.getDisabledJobs()) {
-					ExecutableNode node = exflow.getExecutableNode(disabledId);
-					node.setStatus(Status.DISABLED);
-				}
+				applyDisabledJobs(options.getDisabledJobs(), exflow);
 			}
 			
-			String message = "";
 			if (!running.isEmpty()) {
 				if (options.getConcurrentOption().equals(ExecutionOptions.CONCURRENT_OPTION_PIPELINE)) {
 					Collections.sort(running);
 					Integer runningExecId = running.get(running.size() - 1);
 					
 					options.setPipelineExecutionId(runningExecId);
-					message = "Flow " + flowId + " is already running with exec id " + runningExecId +". Pipelining level " + options.getPipelineLevel() + ". ";
+					message = "Flow " + flowId + " is already running with exec id " + runningExecId +". Pipelining level " + options.getPipelineLevel() + ". \n";
 				}
 				else if (options.getConcurrentOption().equals(ExecutionOptions.CONCURRENT_OPTION_SKIP)) {
 					throw new ExecutorManagerException("Flow " + flowId + " is already running. Skipping execution.", ExecutorManagerException.Reason.SkippedExecution);
 				}
 				else {
 					// The settings is to run anyways.
-					message = "Flow " + flowId + " is already running with exec id " + StringUtils.join(running, ",") +". Will execute concurrently. ";
+					message = "Flow " + flowId + " is already running with exec id " + StringUtils.join(running, ",") +". Will execute concurrently. \n";
 				}
 			}
 			
@@ -653,9 +679,6 @@ public class ExecutorManager extends EventHandler implements ExecutorManagerAdap
 			while(!shutdown) {
 				try {
 					lastThreadCheckTime = System.currentTimeMillis();
-
-//					loadRunningFlows();
-
 					updaterStage = "Starting update all flows.";
 					
 					Map<ConnectionInfo, List<ExecutableFlow>> exFlowMap = getFlowToExecutorMap();
@@ -937,7 +960,7 @@ public class ExecutorManager extends EventHandler implements ExecutorManagerAdap
 		
 		Pair<ExecutionReference, ExecutableFlow> refPair = this.runningFlows.get(execId);
 		if (refPair == null) {
-			throw new ExecutorManagerException("No running flow found with the execution id.");
+			throw new ExecutorManagerException("No running flow found with the execution id. Removing " + execId);
 		}
 		
 		ExecutionReference ref = refPair.getFirst();
@@ -1166,8 +1189,4 @@ public class ExecutorManager extends EventHandler implements ExecutorManagerAdap
 			cleanOldExecutionLogs(DateTime.now().getMillis() - executionLogsRetentionMs);
 		}
 	}
-
-	
-
-	
 }
diff --git a/src/java/azkaban/executor/JdbcExecutorLoader.java b/src/java/azkaban/executor/JdbcExecutorLoader.java
index 6d44b57..70b73da 100644
--- a/src/java/azkaban/executor/JdbcExecutorLoader.java
+++ b/src/java/azkaban/executor/JdbcExecutorLoader.java
@@ -47,8 +47,10 @@ import azkaban.utils.Pair;
 import azkaban.utils.Props;
 import azkaban.utils.PropsUtils;
 
-public class JdbcExecutorLoader extends AbstractJdbcLoader implements ExecutorLoader {
-	private static final Logger logger = Logger.getLogger(JdbcExecutorLoader.class);
+public class JdbcExecutorLoader extends AbstractJdbcLoader 
+		implements ExecutorLoader {
+	private static final Logger logger = 
+			Logger.getLogger(JdbcExecutorLoader.class);
 
 	private EncodingType defaultEncodingType = EncodingType.GZIP;
 	
@@ -65,11 +67,13 @@ public class JdbcExecutorLoader extends AbstractJdbcLoader implements ExecutorLo
 	}
 	
 	@Override
-	public synchronized void uploadExecutableFlow(ExecutableFlow flow) throws ExecutorManagerException {
+	public synchronized void uploadExecutableFlow(ExecutableFlow flow) 
+			throws ExecutorManagerException {
 		Connection connection = getConnection();
 		try {
 			uploadExecutableFlow(connection, flow, defaultEncodingType);
-		} catch (IOException e) {
+		}
+		catch (IOException e) {
 			throw new ExecutorManagerException("Error uploading flow", e);
 		}
 		finally {
@@ -77,17 +81,32 @@ public class JdbcExecutorLoader extends AbstractJdbcLoader implements ExecutorLo
 		}
 	}
 	
-	private synchronized void uploadExecutableFlow(Connection connection, ExecutableFlow flow, EncodingType encType) throws ExecutorManagerException, IOException {
-		final String INSERT_EXECUTABLE_FLOW = "INSERT INTO execution_flows (project_id, flow_id, version, status, submit_time, submit_user, update_time) values (?,?,?,?,?,?,?)";
+	private synchronized void uploadExecutableFlow(Connection connection, 
+			ExecutableFlow flow, EncodingType encType) 
+			throws ExecutorManagerException, IOException {
+		final String INSERT_EXECUTABLE_FLOW = 
+				"INSERT INTO execution_flows " + 
+						"(project_id, flow_id, version, status, submit_time, submit_user, update_time) " + 
+						"values (?,?,?,?,?,?,?)";
 		QueryRunner runner = new QueryRunner();
 		long submitTime = System.currentTimeMillis();
 
 		long id;
 		try {
 			flow.setStatus(Status.PREPARING);
-			runner.update(connection, INSERT_EXECUTABLE_FLOW, flow.getProjectId(), flow.getFlowId(), flow.getVersion(), Status.PREPARING.getNumVal(), submitTime, flow.getSubmitUser(), submitTime);
+			runner.update(
+					connection, 
+					INSERT_EXECUTABLE_FLOW, 
+					flow.getProjectId(), 
+					flow.getFlowId(), 
+					flow.getVersion(), 
+					Status.PREPARING.getNumVal(), 
+					submitTime, 
+					flow.getSubmitUser(), 
+					submitTime);
 			connection.commit();
-			id = runner.query(connection, LastInsertID.LAST_INSERT_ID, new LastInsertID());
+			id = runner.query(
+					connection, LastInsertID.LAST_INSERT_ID, new LastInsertID());
 
 			if (id == -1l) {
 				throw new ExecutorManagerException("Execution id is not properly created.");
@@ -96,13 +115,15 @@ public class JdbcExecutorLoader extends AbstractJdbcLoader implements ExecutorLo
 			flow.setExecutionId((int)id);
 			
 			updateExecutableFlow(connection, flow, encType);
-		} catch (SQLException e) {
+		}
+		catch (SQLException e) {
 			throw new ExecutorManagerException("Error creating execution.", e);
 		}
 	}
 	
 	@Override
-	public void updateExecutableFlow(ExecutableFlow flow) throws ExecutorManagerException {
+	public void updateExecutableFlow(ExecutableFlow flow) 
+			throws ExecutorManagerException {
 		Connection connection = this.getConnection();
 		
 		try {
@@ -113,8 +134,13 @@ public class JdbcExecutorLoader extends AbstractJdbcLoader implements ExecutorLo
 		}
 	} 
 	
-	private void updateExecutableFlow(Connection connection, ExecutableFlow flow, EncodingType encType) throws ExecutorManagerException {
-		final String UPDATE_EXECUTABLE_FLOW_DATA = "UPDATE execution_flows SET status=?,update_time=?,start_time=?,end_time=?,enc_type=?,flow_data=? WHERE exec_id=?";
+	private void updateExecutableFlow(
+			Connection connection, ExecutableFlow flow, EncodingType encType) 
+			throws ExecutorManagerException {
+		final String UPDATE_EXECUTABLE_FLOW_DATA = 
+				"UPDATE execution_flows " + 
+						"SET status=?,update_time=?,start_time=?,end_time=?,enc_type=?,flow_data=? " + 
+						"WHERE exec_id=?";
 		QueryRunner runner = new QueryRunner();
 		
 		String json = JSONUtils.toJSON(flow.toObject());
@@ -132,36 +158,53 @@ public class JdbcExecutorLoader extends AbstractJdbcLoader implements ExecutorLo
 		}
 		
 		try {
-			runner.update(connection, UPDATE_EXECUTABLE_FLOW_DATA, flow.getStatus().getNumVal(), flow.getUpdateTime(), flow.getStartTime(), flow.getEndTime(), encType.getNumVal(), data, flow.getExecutionId());
+			runner.update(
+					connection, 
+					UPDATE_EXECUTABLE_FLOW_DATA, 
+					flow.getStatus().getNumVal(), 
+					flow.getUpdateTime(), 
+					flow.getStartTime(), 
+					flow.getEndTime(), 
+					encType.getNumVal(), 
+					data, 
+					flow.getExecutionId());
 			connection.commit();
 		}
-		catch(SQLException e) {
+		catch (SQLException e) {
 			throw new ExecutorManagerException("Error updating flow.", e);
 		}
 	}
 	
 	@Override
-	public ExecutableFlow fetchExecutableFlow(int id) throws ExecutorManagerException {
+	public ExecutableFlow fetchExecutableFlow(int id) 
+			throws ExecutorManagerException {
 		QueryRunner runner = createQueryRunner();
 		FetchExecutableFlows flowHandler = new FetchExecutableFlows();
 
 		try {
-			List<ExecutableFlow> properties = runner.query(FetchExecutableFlows.FETCH_EXECUTABLE_FLOW, flowHandler, id);
+			List<ExecutableFlow> properties = runner.query(
+					FetchExecutableFlows.FETCH_EXECUTABLE_FLOW, flowHandler, id);
 			return properties.get(0);
-		} catch (SQLException e) {
+		}
+		catch (SQLException e) {
 			throw new ExecutorManagerException("Error fetching flow id " + id, e);
 		}
 	}
 	
 	@Override
-	public Map<Integer, Pair<ExecutionReference, ExecutableFlow>> fetchActiveFlows() throws ExecutorManagerException {
+	public Map<Integer, Pair<ExecutionReference, ExecutableFlow>> fetchActiveFlows() 
+			throws ExecutorManagerException {
 		QueryRunner runner = createQueryRunner();
 		FetchActiveExecutableFlows flowHandler = new FetchActiveExecutableFlows();
 
 		try {
-			Map<Integer, Pair<ExecutionReference, ExecutableFlow>> properties = runner.query(FetchActiveExecutableFlows.FETCH_ACTIVE_EXECUTABLE_FLOW, flowHandler);
+			Map<Integer, Pair<ExecutionReference, ExecutableFlow>> properties = 
+					runner.query(
+							FetchActiveExecutableFlows.FETCH_ACTIVE_EXECUTABLE_FLOW, 
+							flowHandler);
 			return properties;
-		} catch (SQLException e) {
+		}
+		catch (SQLException e) {
 			throw new ExecutorManagerException("Error fetching active flows", e);
 		}
 	}
@@ -174,7 +217,8 @@ public class JdbcExecutorLoader extends AbstractJdbcLoader implements ExecutorLo
 		try {
 			int count = runner.query(IntHandler.NUM_EXECUTIONS, intHandler);
 			return count;
-		} catch (SQLException e) {
+		}
+		catch (SQLException e) {
 			throw new ExecutorManagerException("Error fetching num executions", e);
 		}
 	}
@@ -189,7 +233,8 @@ public class JdbcExecutorLoader extends AbstractJdbcLoader implements ExecutorLo
 			int count = runner.query(
 					IntHandler.NUM_FLOW_EXECUTIONS, intHandler, projectId, flowId);
 			return count;
-		} catch (SQLException e) {
+		}
+		catch (SQLException e) {
 			throw new ExecutorManagerException("Error fetching num executions", e);
 		}
 	}
@@ -204,7 +249,8 @@ public class JdbcExecutorLoader extends AbstractJdbcLoader implements ExecutorLo
 			int count = runner.query(
 					IntHandler.NUM_JOB_EXECUTIONS, intHandler, projectId, jobId);
 			return count;
-		} catch (SQLException e) {
+		}
+		catch (SQLException e) {
 			throw new ExecutorManagerException("Error fetching num executions", e);
 		}
 	}
@@ -224,7 +270,8 @@ public class JdbcExecutorLoader extends AbstractJdbcLoader implements ExecutorLo
 					skip, 
 					num);
 			return properties;
-		} catch (SQLException e) {
+		}
+		catch (SQLException e) {
 			throw new ExecutorManagerException("Error fetching active flows", e);
 		}
 	}
@@ -246,7 +293,8 @@ public class JdbcExecutorLoader extends AbstractJdbcLoader implements ExecutorLo
 					skip, 
 					num);
 			return properties;
-		} catch (SQLException e) {
+		}
+		catch (SQLException e) {
 			throw new ExecutorManagerException("Error fetching active flows", e);
 		}
 	}
@@ -265,7 +313,8 @@ public class JdbcExecutorLoader extends AbstractJdbcLoader implements ExecutorLo
 					skip, 
 					num);
 			return properties;
-		} catch (SQLException e) {
+		}
+		catch (SQLException e) {
 			throw new ExecutorManagerException("Error fetching active flows", e);
 		}
 	}
@@ -371,39 +420,56 @@ public class JdbcExecutorLoader extends AbstractJdbcLoader implements ExecutorLo
 	}
 	
 	@Override
-	public void addActiveExecutableReference(ExecutionReference reference) throws ExecutorManagerException {
-		final String INSERT = "INSERT INTO active_executing_flows (exec_id, host, port, update_time) values (?,?,?,?)";
+	public void addActiveExecutableReference(ExecutionReference reference)
+			throws ExecutorManagerException {
+		final String INSERT = 
+				"INSERT INTO active_executing_flows " + 
+					"(exec_id, host, port, update_time) values (?,?,?,?)";
 		QueryRunner runner = createQueryRunner();
 		
 		try {
-			runner.update(INSERT, reference.getExecId(), reference.getHost(), reference.getPort(), reference.getUpdateTime());
-		} catch (SQLException e) {
-			throw new ExecutorManagerException("Error updating active flow reference " + reference.getExecId(), e);
+			runner.update(
+					INSERT, 
+					reference.getExecId(), 
+					reference.getHost(), 
+					reference.getPort(), 
+					reference.getUpdateTime());
+		}
+		catch (SQLException e) {
+			throw new ExecutorManagerException(
+					"Error updating active flow reference " + reference.getExecId(), e);
 		}
 	}
 	
 	@Override
-	public void removeActiveExecutableReference(int execid) throws ExecutorManagerException {
+	public void removeActiveExecutableReference(int execid) 
+			throws ExecutorManagerException {
 		final String DELETE = "DELETE FROM active_executing_flows WHERE exec_id=?";
 		
 		QueryRunner runner = createQueryRunner();
 		try {
 			runner.update(DELETE, execid);
-		} catch (SQLException e) {
-			throw new ExecutorManagerException("Error deleting active flow reference " + execid, e);
+		}
+		catch (SQLException e) {
+			throw new ExecutorManagerException(
+					"Error deleting active flow reference " + execid, e);
 		}
 	}
 	
 	@Override
-	public boolean updateExecutableReference(int execId, long updateTime) throws ExecutorManagerException {
-		final String DELETE = "UPDATE active_executing_flows set update_time=? WHERE exec_id=?";
+	public boolean updateExecutableReference(int execId, long updateTime) 
+			throws ExecutorManagerException {
+		final String DELETE = 
+				"UPDATE active_executing_flows set update_time=? WHERE exec_id=?";
 		
 		QueryRunner runner = createQueryRunner();
 		int updateNum = 0;
 		try {
 			updateNum = runner.update(DELETE, updateTime, execId);
-		} catch (SQLException e) {
-			throw new ExecutorManagerException("Error deleting active flow reference " + execId, e);
+		}
+		catch (SQLException e) {
+			throw new ExecutorManagerException(
+					"Error deleting active flow reference " + execId, e);
 		}
 		
 		// Should be 1.
@@ -411,20 +477,34 @@ public class JdbcExecutorLoader extends AbstractJdbcLoader implements ExecutorLo
 	}
 
 	@Override
-	public void uploadExecutableNode(ExecutableNode node, Props inputProps) throws ExecutorManagerException {
-		final String INSERT_EXECUTION_NODE = "INSERT INTO execution_jobs (exec_id, project_id, version, flow_id, job_id, start_time, end_time, status, input_params, attempt) VALUES (?,?,?,?,?,?,?,?,?,?)";
+	public void uploadExecutableNode(ExecutableNode node, Props inputProps) 
+			throws ExecutorManagerException {
+		final String INSERT_EXECUTION_NODE = 
+			"INSERT INTO execution_jobs " + 
+					"(exec_id, project_id, version, flow_id, job_id, start_time, " + 
+					"end_time, status, input_params, attempt) VALUES (?,?,?,?,?,?,?,?,?,?)";
 		
 		byte[] inputParam = null;
 		if (inputProps != null) {
 			try {
-				String jsonString = JSONUtils.toJSON(PropsUtils.toHierarchicalMap(inputProps));
+				String jsonString =
+						JSONUtils.toJSON(PropsUtils.toHierarchicalMap(inputProps));
 				inputParam = GZIPUtils.gzipString(jsonString, "UTF-8");
-			} catch (IOException e) {
+			}
+			catch (IOException e) {
 				throw new ExecutorManagerException("Error encoding input params");
 			}
 		}
 		
-		ExecutableFlow flow = node.getFlow();
+		ExecutableFlow flow = node.getExecutableFlow();
+		String flowId = flow.getFlowId();
+		
+		// If the main flow is not the parent, then we'll create a composite key 
+		// for flowID.
+		if (flow != node.getParentFlow()) {
+			flowId = node.getParentFlow().getNestedId();
+		}
+		
 		QueryRunner runner = createQueryRunner();
 		try {
 			runner.update(
@@ -432,30 +512,36 @@ public class JdbcExecutorLoader extends AbstractJdbcLoader implements ExecutorLo
 					flow.getExecutionId(), 
 					flow.getProjectId(), 
 					flow.getVersion(), 
-					flow.getFlowId(), 
-					node.getJobId(),
+					flowId, 
+					node.getId(),
 					node.getStartTime(),
 					node.getEndTime(), 
 					node.getStatus().getNumVal(),
 					inputParam,
-					node.getAttempt()
-					);
+					node.getAttempt());
 		} catch (SQLException e) {
-			throw new ExecutorManagerException("Error writing job " + node.getJobId(), e);
+			throw new ExecutorManagerException(
+					"Error writing job " + node.getId(), e);
 		}
 	}
 	
 	@Override
-	public void updateExecutableNode(ExecutableNode node) throws ExecutorManagerException {
-		final String UPSERT_EXECUTION_NODE = "UPDATE execution_jobs SET start_time=?, end_time=?, status=?, output_params=? WHERE exec_id=? AND job_id=? AND attempt=?";
+	public void updateExecutableNode(ExecutableNode node)
+			throws ExecutorManagerException {
+		final String UPSERT_EXECUTION_NODE = 
+				"UPDATE execution_jobs " +
+						"SET start_time=?, end_time=?, status=?, output_params=? " + 
+						"WHERE exec_id=? AND flow_id=? AND job_id=? AND attempt=?";
 		
 		byte[] outputParam = null;
 		Props outputProps = node.getOutputProps();
 		if (outputProps != null) {
 			try {
-				String jsonString = JSONUtils.toJSON(PropsUtils.toHierarchicalMap(outputProps));
+				String jsonString =
+						JSONUtils.toJSON(PropsUtils.toHierarchicalMap(outputProps));
 				outputParam = GZIPUtils.gzipString(jsonString, "UTF-8");
-			} catch (IOException e) {
+			}
+			catch (IOException e) {
 				throw new ExecutorManagerException("Error encoding input params");
 			}
 		}
@@ -468,117 +554,168 @@ public class JdbcExecutorLoader extends AbstractJdbcLoader implements ExecutorLo
 					node.getEndTime(), 
 					node.getStatus().getNumVal(), 
 					outputParam,
-					node.getFlow().getExecutionId(),
-					node.getJobId(),
+					node.getExecutableFlow().getExecutionId(),
+					node.getParentFlow().getNestedId(),
+					node.getId(),
 					node.getAttempt());
 		} catch (SQLException e) {
-			throw new ExecutorManagerException("Error updating job " + node.getJobId(), e);
+			throw new ExecutorManagerException(
+					"Error updating job " + node.getId(), e);
 		}
 	}
 	
 	@Override
-	public List<ExecutableJobInfo> fetchJobInfoAttempts(int execId, String jobId) throws ExecutorManagerException {
+	public List<ExecutableJobInfo> fetchJobInfoAttempts(int execId, String jobId)
+			throws ExecutorManagerException {
 		QueryRunner runner = createQueryRunner();
 		
 		try {
-			List<ExecutableJobInfo> info = runner.query(FetchExecutableJobHandler.FETCH_EXECUTABLE_NODE_ATTEMPTS, new FetchExecutableJobHandler(), execId, jobId);
+			List<ExecutableJobInfo> info = runner.query(
+					FetchExecutableJobHandler.FETCH_EXECUTABLE_NODE_ATTEMPTS, 
+					new FetchExecutableJobHandler(), 
+					execId,
+					jobId);
 			if (info == null || info.isEmpty()) {
 				return null;
 			}
-			
 			return info;
-		} catch (SQLException e) {
+		}
+		catch (SQLException e) {
 			throw new ExecutorManagerException("Error querying job info " + jobId, e);
 		}
 	}
 	
 	@Override
-	public ExecutableJobInfo fetchJobInfo(int execId, String jobId, int attempts) throws ExecutorManagerException {
+	public ExecutableJobInfo fetchJobInfo(int execId, String jobId, int attempts)
+			throws ExecutorManagerException {
 		QueryRunner runner = createQueryRunner();
 		
 		try {
-			List<ExecutableJobInfo> info = runner.query(FetchExecutableJobHandler.FETCH_EXECUTABLE_NODE, new FetchExecutableJobHandler(), execId, jobId);
+			List<ExecutableJobInfo> info = runner.query(
+					FetchExecutableJobHandler.FETCH_EXECUTABLE_NODE, 
+					new FetchExecutableJobHandler(), 
+					execId, 
+					jobId);
 			if (info == null || info.isEmpty()) {
 				return null;
 			}
-			
 			return info.get(0);
-		} catch (SQLException e) {
+		}
+		catch (SQLException e) {
 			throw new ExecutorManagerException("Error querying job info " + jobId, e);
 		}
 	}
 	
 	@Override
-	public Props fetchExecutionJobInputProps(int execId, String jobId) throws ExecutorManagerException {
+	public Props fetchExecutionJobInputProps(int execId, String jobId)
+			throws ExecutorManagerException {
 		QueryRunner runner = createQueryRunner();
 		try {
-			Pair<Props, Props> props = runner.query(FetchExecutableJobPropsHandler.FETCH_INPUT_PARAM_EXECUTABLE_NODE, new FetchExecutableJobPropsHandler(), execId, jobId);
+			Pair<Props, Props> props = runner.query(
+					FetchExecutableJobPropsHandler.FETCH_INPUT_PARAM_EXECUTABLE_NODE, 
+					new FetchExecutableJobPropsHandler(), 
+					execId, 
+					jobId);
 			return props.getFirst();
 		}
 		catch (SQLException e) {
-			throw new ExecutorManagerException("Error querying job params " + execId + " " + jobId, e);
+			throw new ExecutorManagerException(
+					"Error querying job params " + execId + " " + jobId, e);
 		}
 	}
 	
 	@Override
-	public Props fetchExecutionJobOutputProps(int execId, String jobId) throws ExecutorManagerException {
+	public Props fetchExecutionJobOutputProps(int execId, String jobId) 
+			throws ExecutorManagerException {
 		QueryRunner runner = createQueryRunner();
 		try {
-			Pair<Props, Props> props = runner.query(FetchExecutableJobPropsHandler.FETCH_OUTPUT_PARAM_EXECUTABLE_NODE, new FetchExecutableJobPropsHandler(), execId, jobId);
+			Pair<Props, Props> props = runner.query(
+					FetchExecutableJobPropsHandler.FETCH_OUTPUT_PARAM_EXECUTABLE_NODE,
+					new FetchExecutableJobPropsHandler(),
+					execId,
+					jobId);
 			return props.getFirst();
 		}
 		catch (SQLException e) {
-			throw new ExecutorManagerException("Error querying job params " + execId + " " + jobId, e);
+			throw new ExecutorManagerException(
+					"Error querying job params " + execId + " " + jobId, e);
 		}
 	}
 	
 	@Override
-	public Pair<Props, Props> fetchExecutionJobProps(int execId, String jobId) throws ExecutorManagerException {
+	public Pair<Props, Props> fetchExecutionJobProps(int execId, String jobId)
+			throws ExecutorManagerException {
 		QueryRunner runner = createQueryRunner();
 		try {
-			Pair<Props, Props> props = runner.query(FetchExecutableJobPropsHandler.FETCH_INPUT_OUTPUT_PARAM_EXECUTABLE_NODE, new FetchExecutableJobPropsHandler(), execId, jobId);
+			Pair<Props, Props> props = runner.query(
+					FetchExecutableJobPropsHandler.FETCH_INPUT_OUTPUT_PARAM_EXECUTABLE_NODE, 
+					new FetchExecutableJobPropsHandler(), 
+					execId, 
+					jobId);
 			return props;
 		}
 		catch (SQLException e) {
-			throw new ExecutorManagerException("Error querying job params " + execId + " " + jobId, e);
+			throw new ExecutorManagerException(
+					"Error querying job params " + execId + " " + jobId, e);
 		}
 	}
 	
 	@Override
-	public List<ExecutableJobInfo> fetchJobHistory(int projectId, String jobId, int skip, int size) throws ExecutorManagerException {
+	public List<ExecutableJobInfo> fetchJobHistory(
+			int projectId, String jobId, int skip, int size) 
+			throws ExecutorManagerException {
 		QueryRunner runner = createQueryRunner();
 		
 		try {
-			List<ExecutableJobInfo> info = runner.query(FetchExecutableJobHandler.FETCH_PROJECT_EXECUTABLE_NODE, new FetchExecutableJobHandler(), projectId, jobId, skip, size);
+			List<ExecutableJobInfo> info = runner.query(
+					FetchExecutableJobHandler.FETCH_PROJECT_EXECUTABLE_NODE,
+					new FetchExecutableJobHandler(), 
+					projectId, 
+					jobId, 
+					skip, 
+					size);
 			if (info == null || info.isEmpty()) {
 				return null;
 			}
-			
 			return info;
-		} catch (SQLException e) {
+		}
+		catch (SQLException e) {
 			throw new ExecutorManagerException("Error querying job info " + jobId, e);
 		}
 	}
 	
 	@Override
-	public LogData fetchLogs(int execId, String name, int attempt, int startByte, int length) throws ExecutorManagerException {
+	public LogData fetchLogs(
+			int execId, String name, int attempt, int startByte, int length)
+			throws ExecutorManagerException {
 		QueryRunner runner = createQueryRunner();
 
 		FetchLogsHandler handler = new FetchLogsHandler(startByte, length + startByte);
-		
 		try {
-			LogData result = runner.query(FetchLogsHandler.FETCH_LOGS, handler, execId, name, attempt, startByte, startByte + length);
+			LogData result = runner.query(
+					FetchLogsHandler.FETCH_LOGS, 
+					handler, 
+					execId, 
+					name, 
+					attempt, 
+					startByte, 
+					startByte + length);
 			return result;
-		} catch (SQLException e) {
-			throw new ExecutorManagerException("Error fetching logs " + execId + " : " + name, e);
+		}
+		catch (SQLException e) {
+			throw new ExecutorManagerException(
+					"Error fetching logs " + execId + " : " + name, e);
 		}
 	}
 	
 	@Override
-	public void uploadLogFile(int execId, String name, int attempt, File ... files) throws ExecutorManagerException {
+	public void uploadLogFile(
+			int execId, String name, int attempt, File ... files)
+			throws ExecutorManagerException {
 		Connection connection = getConnection();
 		try {
-			uploadLogFile(connection, execId, name, attempt, files, defaultEncodingType);
+			uploadLogFile(
+					connection, execId, name, attempt, files, defaultEncodingType);
 			connection.commit();
 		}
 		catch (SQLException e) {
@@ -592,7 +729,13 @@ public class JdbcExecutorLoader extends AbstractJdbcLoader implements ExecutorLo
 		}
 	}
 	
-	private void uploadLogFile(Connection connection, int execId, String name, int attempt, File[] files, EncodingType encType) throws ExecutorManagerException, IOException {
+	private void uploadLogFile(
+			Connection connection, 
+			int execId, 
+			String name, 
+			int attempt, 
+			File[] files, 
+			EncodingType encType) throws ExecutorManagerException, IOException {
 		// 50K buffer... if logs are greater than this, we chunk.
 		// However, we better prevent large log files from being uploaded somehow
 		byte[] buffer = new byte[50*1024];
@@ -609,7 +752,16 @@ public class JdbcExecutorLoader extends AbstractJdbcLoader implements ExecutorLo
 				while (size >= 0) {
 					if (pos + size == buffer.length) {
 						// Flush here.
-						uploadLogPart(connection, execId, name, attempt, startByte, startByte + buffer.length, encType, buffer, buffer.length);
+						uploadLogPart(
+								connection, 
+								execId, 
+								name, 
+								attempt, 
+								startByte, 
+								startByte + buffer.length, 
+								encType, 
+								buffer, 
+								buffer.length);
 						
 						pos = 0;
 						length = buffer.length;
@@ -626,7 +778,16 @@ public class JdbcExecutorLoader extends AbstractJdbcLoader implements ExecutorLo
 			
 			// Final commit of buffer.
 			if (pos > 0) {
-				uploadLogPart(connection, execId, name, attempt, startByte, startByte + pos, encType, buffer, pos);
+				uploadLogPart(
+						connection, 
+						execId, 
+						name, 
+						attempt, 
+						startByte, 
+						startByte + pos, 
+						encType, 
+						buffer, 
+						pos);
 			}
 		}
 		catch (SQLException e) {
@@ -638,13 +799,24 @@ public class JdbcExecutorLoader extends AbstractJdbcLoader implements ExecutorLo
 		finally {
 			IOUtils.closeQuietly(bufferedStream);
 		}
-
 	}
 	
-	private void uploadLogPart(Connection connection, int execId, String name, int attempt, int startByte, int endByte, EncodingType encType, byte[] buffer, int length) throws SQLException, IOException {
-		final String INSERT_EXECUTION_LOGS = "INSERT INTO execution_logs (exec_id, name, attempt, enc_type, start_byte, end_byte, log, upload_time) VALUES (?,?,?,?,?,?,?,?)";
-		QueryRunner runner = new QueryRunner();
+	private void uploadLogPart(
+			Connection connection, 
+			int execId, 
+			String name, 
+			int attempt, 
+			int startByte, 
+			int endByte, 
+			EncodingType encType, 
+			byte[] buffer, 
+			int length) throws SQLException, IOException {
+		final String INSERT_EXECUTION_LOGS = 
+				"INSERT INTO execution_logs " + 
+						"(exec_id, name, attempt, enc_type, start_byte, end_byte, " + 
+						"log, upload_time) VALUES (?,?,?,?,?,?,?,?)";
 		
+		QueryRunner runner = new QueryRunner();
 		byte[] buf = buffer;
 		if (encType == EncodingType.GZIP) {
 			buf = GZIPUtils.gzipBytes(buf, 0, length);
@@ -653,38 +825,49 @@ public class JdbcExecutorLoader extends AbstractJdbcLoader implements ExecutorLo
 			buf = Arrays.copyOf(buffer, length);
 		}
 		
-		runner.update(connection, INSERT_EXECUTION_LOGS, execId, name, attempt, encType.getNumVal(), startByte, startByte + length, buf, DateTime.now().getMillis());
+		runner.update(
+				connection, 
+				INSERT_EXECUTION_LOGS, 
+				execId, 
+				name, 
+				attempt, 
+				encType.getNumVal(), 
+				startByte, 
+				startByte + length, 
+				buf, 
+				DateTime.now().getMillis());
 	}
 	
 	private Connection getConnection() throws ExecutorManagerException {
 		Connection connection = null;
 		try {
 			connection = super.getDBConnection(false);
-		} catch (Exception e) {
+		}
+		catch (Exception e) {
 			DbUtils.closeQuietly(connection);
 			throw new ExecutorManagerException("Error getting DB connection.", e);
 		}
-		
 		return connection;
 	}
 	
 	private static class LastInsertID implements ResultSetHandler<Long> {
 		private static String LAST_INSERT_ID = "SELECT LAST_INSERT_ID()";
-		
 		@Override
 		public Long handle(ResultSet rs) throws SQLException {
 			if (!rs.next()) {
 				return -1l;
 			}
-
 			long id = rs.getLong(1);
 			return id;
 		}
-		
 	}
 	
 	private static class FetchLogsHandler implements ResultSetHandler<LogData> {
-		private static String FETCH_LOGS = "SELECT exec_id, name, attempt, enc_type, start_byte, end_byte, log FROM execution_logs WHERE exec_id=? AND name=? AND attempt=? AND end_byte > ? AND start_byte <= ? ORDER BY start_byte";
+		private static String FETCH_LOGS =
+			"SELECT exec_id, name, attempt, enc_type, start_byte, end_byte, log " + 
+					"FROM execution_logs " + 
+					"WHERE exec_id=? AND name=? AND attempt=? AND end_byte > ? " + 
+					"AND start_byte <= ? ORDER BY start_byte";
 
 		private int startByte;
 		private int endByte;
@@ -713,8 +896,12 @@ public class JdbcExecutorLoader extends AbstractJdbcLoader implements ExecutorLo
 
 				byte[] data = rs.getBytes(7);
 
-				int offset = this.startByte > startByte ? this.startByte - startByte : 0;
-				int length = this.endByte < endByte ? this.endByte - startByte - offset: endByte - startByte - offset;
+				int offset = this.startByte > startByte 
+						? this.startByte - startByte 
+						: 0;
+				int length = this.endByte < endByte 
+						? this.endByte - startByte - offset
+						: endByte - startByte - offset;
 				try {
 					byte[] buffer = data;
 					if (encType == EncodingType.GZIP) {
@@ -722,22 +909,39 @@ public class JdbcExecutorLoader extends AbstractJdbcLoader implements ExecutorLo
 					}
 
 					byteStream.write(buffer, offset, length);
-				} catch (IOException e) {
+				}
+				catch (IOException e) {
 					throw new SQLException(e);
 				}
 			} while (rs.next());
 
 			byte[] buffer = byteStream.toByteArray();
-			Pair<Integer,Integer> result = FileIOUtils.getUtf8Range(buffer, 0, buffer.length);
+			Pair<Integer,Integer> result = FileIOUtils.getUtf8Range(
+					buffer, 0, buffer.length);
 		
-			return new LogData(startByte + result.getFirst(), result.getSecond(), new String(buffer, result.getFirst(), result.getSecond()));
+			return new LogData(
+					startByte + result.getFirst(), 
+					result.getSecond(), 
+					new String(buffer, result.getFirst(), result.getSecond()));
 		}
 	}
 	
-	private static class FetchExecutableJobHandler implements ResultSetHandler<List<ExecutableJobInfo>> {
-		private static String FETCH_EXECUTABLE_NODE = "SELECT exec_id, project_id, version, flow_id, job_id, start_time, end_time, status, attempt FROM execution_jobs WHERE exec_id=? AND job_id=? AND attempt_id=?";
-		private static String FETCH_EXECUTABLE_NODE_ATTEMPTS = "SELECT exec_id, project_id, version, flow_id, job_id, start_time, end_time, status, attempt FROM execution_jobs WHERE exec_id=? AND job_id=?";
-		private static String FETCH_PROJECT_EXECUTABLE_NODE = "SELECT exec_id, project_id, version, flow_id, job_id, start_time, end_time, status, attempt FROM execution_jobs WHERE project_id=? AND job_id=? ORDER BY exec_id DESC LIMIT ?, ? ";
+	private static class FetchExecutableJobHandler 
+			implements ResultSetHandler<List<ExecutableJobInfo>> {
+		private static String FETCH_EXECUTABLE_NODE = 
+				"SELECT exec_id, project_id, version, flow_id, job_id, " + 
+						"start_time, end_time, status, attempt " + 
+						"FROM execution_jobs WHERE exec_id=? " + 
+						"AND job_id=? AND attempt_id=?";
+		private static String FETCH_EXECUTABLE_NODE_ATTEMPTS = 
+				"SELECT exec_id, project_id, version, flow_id, job_id, " + 
+						"start_time, end_time, status, attempt FROM execution_jobs " + 
+						"WHERE exec_id=? AND job_id=?";
+		private static String FETCH_PROJECT_EXECUTABLE_NODE =
+				"SELECT exec_id, project_id, version, flow_id, job_id, " + 
+						"start_time, end_time, status, attempt FROM execution_jobs " +
+						"WHERE project_id=? AND job_id=? " + 
+						"ORDER BY exec_id DESC LIMIT ?, ? ";
 
 		@Override
 		public List<ExecutableJobInfo> handle(ResultSet rs) throws SQLException {
@@ -757,7 +961,16 @@ public class JdbcExecutorLoader extends AbstractJdbcLoader implements ExecutorLo
 				Status status = Status.fromInteger(rs.getInt(8));
 				int attempt = rs.getInt(9);
 				
-				ExecutableJobInfo info = new ExecutableJobInfo(execId, projectId, version, flowId, jobId, startTime, endTime, status, attempt);
+				ExecutableJobInfo info = new ExecutableJobInfo(
+						execId, 
+						projectId, 
+						version, 
+						flowId, 
+						jobId, 
+						startTime, 
+						endTime, 
+						status, 
+						attempt);
 				execNodes.add(info);
 			} while (rs.next());
 
@@ -765,10 +978,15 @@ public class JdbcExecutorLoader extends AbstractJdbcLoader implements ExecutorLo
 		}
 	}
 	
-	private static class FetchExecutableJobPropsHandler implements ResultSetHandler<Pair<Props, Props>> {
-		private static String FETCH_OUTPUT_PARAM_EXECUTABLE_NODE = "SELECT output_params FROM execution_jobs WHERE exec_id=? AND job_id=?";
-		private static String FETCH_INPUT_PARAM_EXECUTABLE_NODE = "SELECT input_params FROM execution_jobs WHERE exec_id=? AND job_id=?";
-		private static String FETCH_INPUT_OUTPUT_PARAM_EXECUTABLE_NODE = "SELECT input_params, output_params FROM execution_jobs WHERE exec_id=? AND job_id=?";
+	private static class FetchExecutableJobPropsHandler 
+			implements ResultSetHandler<Pair<Props, Props>> {
+		private static String FETCH_OUTPUT_PARAM_EXECUTABLE_NODE = 
+				"SELECT output_params FROM execution_jobs WHERE exec_id=? AND job_id=?";
+		private static String FETCH_INPUT_PARAM_EXECUTABLE_NODE = 
+				"SELECT input_params FROM execution_jobs WHERE exec_id=? AND job_id=?";
+		private static String FETCH_INPUT_OUTPUT_PARAM_EXECUTABLE_NODE = 
+				"SELECT input_params, output_params " +
+						"FROM execution_jobs WHERE exec_id=? AND job_id=?";
 		
 		@SuppressWarnings("unchecked")
 		@Override
@@ -793,9 +1011,10 @@ public class JdbcExecutorLoader extends AbstractJdbcLoader implements ExecutorLo
 					if (output != null) {
 						String jsonOutputString = GZIPUtils.unGzipString(output, "UTF-8");
 						outputProps = PropsUtils.fromHierarchicalMap(
-								(Map<String, Object>)JSONUtils.parseJSONFromString(jsonOutputString));
+								(Map<String, Object>) JSONUtils.parseJSONFromString(jsonOutputString));
 					}
-				} catch (IOException e) {
+				}
+				catch (IOException e) {
 					throw new SQLException("Error decoding param data", e);
 				}
 				
@@ -811,26 +1030,34 @@ public class JdbcExecutorLoader extends AbstractJdbcLoader implements ExecutorLo
 						props = PropsUtils.fromHierarchicalMap(
 								(Map<String, Object>)JSONUtils.parseJSONFromString(jsonProps));
 					}
-				} catch (IOException e) {
+				}
+				catch (IOException e) {
 					throw new SQLException("Error decoding param data", e);
 				}
 				
 				return new Pair<Props,Props>(props, null);
 			}
 		}
-		
 	}
 
-	private static class FetchActiveExecutableFlows implements ResultSetHandler<Map<Integer, Pair<ExecutionReference,ExecutableFlow>>> {
-		private static String FETCH_ACTIVE_EXECUTABLE_FLOW = "SELECT ex.exec_id exec_id, ex.enc_type enc_type, ex.flow_data flow_data, ax.host host, ax.port port, ax.update_time axUpdateTime FROM execution_flows ex INNER JOIN active_executing_flows ax ON ex.exec_id = ax.exec_id";
+	private static class FetchActiveExecutableFlows 
+			implements ResultSetHandler<Map<Integer, Pair<ExecutionReference,ExecutableFlow>>> {
+		private static String FETCH_ACTIVE_EXECUTABLE_FLOW = 
+				"SELECT ex.exec_id exec_id, ex.enc_type enc_type, ex.flow_data " + 
+						"flow_data, ax.host host, ax.port port, ax.update_time " + 
+						"axUpdateTime " + 
+						"FROM execution_flows ex " + 
+						"INNER JOIN active_executing_flows ax ON ex.exec_id = ax.exec_id";
 		
 		@Override
-		public Map<Integer, Pair<ExecutionReference,ExecutableFlow>> handle(ResultSet rs) throws SQLException {
+		public Map<Integer, Pair<ExecutionReference,ExecutableFlow>> handle(ResultSet rs) 
+				throws SQLException {
 			if (!rs.next()) {
 				return Collections.<Integer, Pair<ExecutionReference,ExecutableFlow>>emptyMap();
 			}
 
-			Map<Integer, Pair<ExecutionReference,ExecutableFlow>> execFlows = new HashMap<Integer, Pair<ExecutionReference,ExecutableFlow>>();
+			Map<Integer, Pair<ExecutionReference,ExecutableFlow>> execFlows = 
+					new HashMap<Integer, Pair<ExecutionReference,ExecutableFlow>>();
 			do {
 				int id = rs.getInt(1);
 				int encodingType = rs.getInt(2);
@@ -846,7 +1073,8 @@ public class JdbcExecutorLoader extends AbstractJdbcLoader implements ExecutorLo
 					EncodingType encType = EncodingType.fromInteger(encodingType);
 					Object flowObj;
 					try {
-						// Convoluted way to inflate strings. Should find common package or helper function.
+						// Convoluted way to inflate strings. Should find common package or
+						// helper function.
 						if (encType == EncodingType.GZIP) {
 							// Decompress the sucker.
 							String jsonString = GZIPUtils.unGzipString(data, "UTF-8");
@@ -857,12 +1085,15 @@ public class JdbcExecutorLoader extends AbstractJdbcLoader implements ExecutorLo
 							flowObj = JSONUtils.parseJSONFromString(jsonString);
 						}
 						
-						ExecutableFlow exFlow = ExecutableFlow.createExecutableFlowFromObject(flowObj);
+						ExecutableFlow exFlow = 
+								ExecutableFlow.createExecutableFlowFromObject(flowObj);
 						ExecutionReference ref = new ExecutionReference(id, host, port);
 						ref.setUpdateTime(updateTime);
 						
-						execFlows.put(id, new Pair<ExecutionReference, ExecutableFlow>(ref, exFlow));
-					} catch (IOException e) {
+						execFlows.put(
+								id, new Pair<ExecutionReference, ExecutableFlow>(ref, exFlow));
+					}
+					catch (IOException e) {
 						throw new SQLException("Error retrieving flow data " + id, e);
 					}
 				}
@@ -870,7 +1101,6 @@ public class JdbcExecutorLoader extends AbstractJdbcLoader implements ExecutorLo
 
 			return execFlows;
 		}
-		
 	}
 	
 	private static class FetchExecutableFlows 
@@ -927,7 +1157,8 @@ public class JdbcExecutorLoader extends AbstractJdbcLoader implements ExecutorLo
 						ExecutableFlow exFlow = 
 								ExecutableFlow.createExecutableFlowFromObject(flowObj);
 						execFlows.add(exFlow);
-					} catch (IOException e) {
+					}
+					catch (IOException e) {
 						throw new SQLException("Error retrieving flow data " + id, e);
 					}
 				}
@@ -935,35 +1166,40 @@ public class JdbcExecutorLoader extends AbstractJdbcLoader implements ExecutorLo
 
 			return execFlows;
 		}
-		
 	}
 	
 	private static class IntHandler implements ResultSetHandler<Integer> {
-		private static String NUM_EXECUTIONS = "SELECT COUNT(1) FROM execution_flows";
-		private static String NUM_FLOW_EXECUTIONS = "SELECT COUNT(1) FROM execution_flows WHERE project_id=? AND flow_id=?";
-		private static String NUM_JOB_EXECUTIONS = "SELECT COUNT(1) FROM execution_jobs WHERE project_id=? AND job_id=?";
+		private static String NUM_EXECUTIONS = 
+				"SELECT COUNT(1) FROM execution_flows";
+		private static String NUM_FLOW_EXECUTIONS = 
+				"SELECT COUNT(1) FROM execution_flows WHERE project_id=? AND flow_id=?";
+		private static String NUM_JOB_EXECUTIONS = 
+				"SELECT COUNT(1) FROM execution_jobs WHERE project_id=? AND job_id=?";
 		
 		@Override
 		public Integer handle(ResultSet rs) throws SQLException {
 			if (!rs.next()) {
 				return 0;
 			}
-			
 			return rs.getInt(1);
 		}
 	}
 
 	@Override
-	public int removeExecutionLogsByTime(long millis) throws ExecutorManagerException {
-		final String DELETE_BY_TIME = "DELETE FROM execution_logs WHERE upload_time < ?";
+	public int removeExecutionLogsByTime(long millis) 
+			throws ExecutorManagerException {
+		final String DELETE_BY_TIME = 
+				"DELETE FROM execution_logs WHERE upload_time < ?";
 		
 		QueryRunner runner = createQueryRunner();
 		int updateNum = 0;
 		try {
 			updateNum = runner.update(DELETE_BY_TIME, millis);
-		} catch (SQLException e) {
+		}
+		catch (SQLException e) {
 			e.printStackTrace();
-			throw new ExecutorManagerException("Error deleting old execution_logs before " + millis, e);			
+			throw new ExecutorManagerException(
+					"Error deleting old execution_logs before " + millis, e);			
 		}
 		
 		return updateNum;
diff --git a/src/java/azkaban/executor/mail/DefaultMailCreator.java b/src/java/azkaban/executor/mail/DefaultMailCreator.java
index 0802cae..831c809 100644
--- a/src/java/azkaban/executor/mail/DefaultMailCreator.java
+++ b/src/java/azkaban/executor/mail/DefaultMailCreator.java
@@ -52,7 +52,7 @@ public class DefaultMailCreator implements MailCreator {
 	public boolean createFirstErrorMessage(ExecutableFlow flow, EmailMessage message, String azkabanName, String clientHostname, String clientPortNumber, String... vars) {
 
 		ExecutionOptions option = flow.getExecutionOptions();
-		List<String> emailList = option.getDisabledJobs();
+		List<String> emailList = option.getFailureEmails();
 		int execId = flow.getExecutionId();
 
 		if (emailList != null && !emailList.isEmpty()) {
diff --git a/src/java/azkaban/executor/Status.java b/src/java/azkaban/executor/Status.java
index bdfe761..fa45066 100644
--- a/src/java/azkaban/executor/Status.java
+++ b/src/java/azkaban/executor/Status.java
@@ -1,5 +1,5 @@
 /*
- * Copyright 2012 LinkedIn Corp.
+ * Copyright 2014 LinkedIn Corp.
  * 
  * 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
@@ -17,7 +17,7 @@
 package azkaban.executor;
 
 public enum Status {
-	READY(10), PREPARING(20), RUNNING(30), PAUSED(40), SUCCEEDED(50), KILLED(60), FAILED(70), FAILED_FINISHING(80), SKIPPED(90), DISABLED(100), QUEUED(110);
+	READY(10), PREPARING(20), RUNNING(30), PAUSED(40), SUCCEEDED(50), KILLED(60), FAILED(70), FAILED_FINISHING(80), SKIPPED(90), DISABLED(100), QUEUED(110), FAILED_SUCCEEDED(120);
 	
 	private int numVal;
 
@@ -53,6 +53,8 @@ public enum Status {
 			return DISABLED;
 		case 110:
 			return QUEUED;
+		case 120:
+			return FAILED_SUCCEEDED;
 		default:
 			return READY;
 		}
@@ -64,9 +66,21 @@ public enum Status {
 		case KILLED:
 		case SUCCEEDED:
 		case SKIPPED:
+		case FAILED_SUCCEEDED:
 			return true;
 		default:
 			return false;
 		}
 	}
-}
+	
+	public static boolean isStatusRunning(Status status) {
+		switch (status) {
+		case RUNNING:
+		case FAILED_FINISHING:
+		case QUEUED:
+			return true;
+		default:
+			return false;
+		}
+	}
+}
\ No newline at end of file
diff --git a/src/java/azkaban/flow/CommonJobProperties.java b/src/java/azkaban/flow/CommonJobProperties.java
index 21f5b1b..7324205 100644
--- a/src/java/azkaban/flow/CommonJobProperties.java
+++ b/src/java/azkaban/flow/CommonJobProperties.java
@@ -28,6 +28,11 @@ public class CommonJobProperties {
 	public static final String JOB_TYPE = "type";
 	
 	/**
+	 * Force a node to be a root node in a flow, even if there are other jobs dependent on it.
+	 */
+	public static final String ROOT_NODE = "root.node";
+	
+	/**
 	 * Comma delimited list of job names which are dependencies
 	 */
 	public static final String DEPENDENCIES = "dependencies";
@@ -77,6 +82,11 @@ public class CommonJobProperties {
 	public static final String FLOW_ID = "azkaban.flow.flowid";
 	
 	/**
+	 * The nested flow id path
+	 */
+	public static final String NESTED_FLOW_PATH = "azkaban.flow.nested.path";
+	
+	/**
 	 * The execution id. This should be unique per flow, but may not be due to 
 	 * restarts.
 	 */
@@ -110,5 +120,4 @@ public class CommonJobProperties {
 	public static final String FLOW_START_SECOND = "azkaban.flow.start.second";
 	public static final String FLOW_START_MILLISSECOND = "azkaban.flow.start.milliseconds";
 	public static final String FLOW_START_TIMEZONE = "azkaban.flow.start.timezone";
-
 }
diff --git a/src/java/azkaban/flow/Flow.java b/src/java/azkaban/flow/Flow.java
index 3abb05b..d28a159 100644
--- a/src/java/azkaban/flow/Flow.java
+++ b/src/java/azkaban/flow/Flow.java
@@ -39,7 +39,7 @@ public class Flow {
 	private HashMap<String, Set<Edge>> outEdges = new HashMap<String, Set<Edge>>();
 	private HashMap<String, Set<Edge>> inEdges = new HashMap<String, Set<Edge>>();
 	private HashMap<String, FlowProps> flowProps = new HashMap<String, FlowProps>(); 
-
+	
 	private List<String> failureEmail = new ArrayList<String>();
 	private List<String> successEmail = new ArrayList<String>();
 	private String mailCreator = DefaultMailCreator.DEFAULT_MAIL_CREATOR;
@@ -84,7 +84,7 @@ public class Flow {
 			}
 		}
 	}
-
+	
 	private void recursiveSetLevels(Node node) {
 		Set<Edge> edges = outEdges.get(node.getId());
 		if (edges != null) {
@@ -159,13 +159,13 @@ public class Flow {
 	public void addNode(Node node) {
 		nodes.put(node.getId(), node);
 	}
-
+	
 	public void addAllFlowProperties(Collection<FlowProps> props) {
 		for (FlowProps prop : props) {
 			flowProps.put(prop.getSource(), prop);
 		}
 	}
-
+	
 	public String getId() {
 		return id;
 	}
diff --git a/src/java/azkaban/flow/Node.java b/src/java/azkaban/flow/Node.java
index d84a0f0..1a62b03 100644
--- a/src/java/azkaban/flow/Node.java
+++ b/src/java/azkaban/flow/Node.java
@@ -23,7 +23,6 @@ import java.util.Map;
 import azkaban.utils.Utils;
 
 public class Node {
-
 	private final String id;
 	private String jobSource;
 	private String propsSource;
@@ -33,6 +32,8 @@ public class Node {
 	private int expectedRunTimeSec = 1;
 	private String type;
 	
+	private String embeddedFlowId;
+	
 	public Node(String id) {
 		this.id = id;
 	}
@@ -103,6 +104,14 @@ public class Node {
 		return expectedRunTimeSec;
 	}
 	
+	public void setEmbeddedFlowId(String flowId) {
+		embeddedFlowId = flowId;
+	}
+	
+	public String getEmbeddedFlowId() {
+		return embeddedFlowId;
+	}
+	
 	@SuppressWarnings("unchecked")
 	public static Node fromObject(Object obj) {
 		Map<String,Object> mapObj = (Map<String,Object>)obj;
@@ -112,10 +121,13 @@ public class Node {
 		String jobSource = (String)mapObj.get("jobSource");
 		String propSource = (String)mapObj.get("propSource");
 		String jobType = (String)mapObj.get("jobType");
-		
+
+		String embeddedFlowId = (String)mapObj.get("embeddedFlowId");
+
 		node.setJobSource(jobSource);
 		node.setPropsSource(propSource);
 		node.setType(jobType);
+		node.setEmbeddedFlowId(embeddedFlowId);
 		
 		Integer expectedRuntime = (Integer)mapObj.get("expectedRuntime");
 		if (expectedRuntime != null) {
@@ -154,6 +166,9 @@ public class Node {
 		objMap.put("jobSource", jobSource);
 		objMap.put("propSource", propsSource);
 		objMap.put("jobType", type);
+		if (embeddedFlowId != null) {
+			objMap.put("embeddedFlowId", embeddedFlowId);
+		}
 		objMap.put("expectedRuntime", expectedRunTimeSec);
 
 		HashMap<String, Object> layoutInfo = new HashMap<String, Object>();
diff --git a/src/java/azkaban/flow/SpecialJobTypes.java b/src/java/azkaban/flow/SpecialJobTypes.java
new file mode 100644
index 0000000..dfbe03a
--- /dev/null
+++ b/src/java/azkaban/flow/SpecialJobTypes.java
@@ -0,0 +1,9 @@
+package azkaban.flow;
+
+public class SpecialJobTypes {
+	public static final String BRANCH_START_TYPE = "branch.start";
+	public static final String BRANCH_END_TYPE = "branch.end";
+
+	public static final String EMBEDDED_FLOW_TYPE = "flow";
+	public static final String FLOW_NAME = "flow.name";
+}
diff --git a/src/java/azkaban/jobExecutor/AbstractJob.java b/src/java/azkaban/jobExecutor/AbstractJob.java
index 4274727..75a6042 100644
--- a/src/java/azkaban/jobExecutor/AbstractJob.java
+++ b/src/java/azkaban/jobExecutor/AbstractJob.java
@@ -28,7 +28,6 @@ public abstract class AbstractJob implements Job {
 	public static final String JOB_FULLPATH = "job.fullpath";
 	public static final String JOB_ID = "job.id";
 	
-	
 	private final String _id;
 	private final Logger _log;
 	private volatile double _progress;
diff --git a/src/java/azkaban/jobExecutor/AbstractProcessJob.java b/src/java/azkaban/jobExecutor/AbstractProcessJob.java
index d106ea0..25f825b 100644
--- a/src/java/azkaban/jobExecutor/AbstractProcessJob.java
+++ b/src/java/azkaban/jobExecutor/AbstractProcessJob.java
@@ -52,7 +52,7 @@ public abstract class AbstractProcessJob extends AbstractJob {
 
 	protected String _cwd;
 
-	private volatile Props generatedPropeties;
+	private volatile Props generatedProperties;
 
 	protected AbstractProcessJob(String jobid, final Props sysProps, final Props jobProps, final Logger log) {
 		super(jobid, log);
@@ -83,7 +83,7 @@ public abstract class AbstractProcessJob extends AbstractJob {
 
 	@Override
 	public Props getJobGeneratedProperties() {
-		return generatedPropeties;
+		return generatedProperties;
 	}
 
 	/**
@@ -188,7 +188,7 @@ public abstract class AbstractProcessJob extends AbstractJob {
 	}
 
 	public void generateProperties(final File outputFile) {
-		generatedPropeties = loadOutputFileProps(outputFile);
+		generatedProperties = loadOutputFileProps(outputFile);
 	}
 
 }
diff --git a/src/java/azkaban/jobExecutor/ProcessJob.java b/src/java/azkaban/jobExecutor/ProcessJob.java
index 13272d4..fc9891c 100644
--- a/src/java/azkaban/jobExecutor/ProcessJob.java
+++ b/src/java/azkaban/jobExecutor/ProcessJob.java
@@ -48,20 +48,23 @@ public class ProcessJob extends AbstractProcessJob {
 			resolveProps();
 		}
 		catch (Exception e) {
-			error("Bad property definition! " + e.getMessage());
-			
+			handleError("Bad property definition! " + e.getMessage(), e);
 		}
 		
 		List<String> commands = null;
 		try {
-		commands = getCommandList();
+			commands = getCommandList();
 		}
 		catch (Exception e) {
-			error("Job set up failed " + e.getCause());
+			handleError("Job set up failed " + e.getCause(), e);
 		}
 
 		long startMs = System.currentTimeMillis();
 
+		if (commands == null) {
+			handleError("There are no commands to execute", null);
+		}
+		
 		info(commands.size() + " commands to execute.");
 		File[] propFiles = initPropsFiles();
 		Map<String, String> envVars = getEnvironmentVariables();
@@ -100,7 +103,16 @@ public class ProcessJob extends AbstractProcessJob {
 		generateProperties(propFiles[1]);
 	}
 
-
+	protected void handleError(String errorMsg, Exception e) throws Exception {
+		error(errorMsg);
+		if (e != null) {
+			throw new Exception(errorMsg, e);
+		}
+		else {
+			throw new Exception(errorMsg);
+		}
+	}
+	
 	protected List<String> getCommandList() {
 		List<String> commands = new ArrayList<String>();
 		commands.add(jobProps.getString(COMMAND));
diff --git a/src/java/azkaban/jobtype/JobTypeManager.java b/src/java/azkaban/jobtype/JobTypeManager.java
index c2ad137..1ef57ef 100644
--- a/src/java/azkaban/jobtype/JobTypeManager.java
+++ b/src/java/azkaban/jobtype/JobTypeManager.java
@@ -384,7 +384,6 @@ public class JobTypeManager
 //			logger.info("jobConf is " + jobConf);
 //			
 			job = (Job)Utils.callConstructor(executorClass, jobId, sysConf, jobConf, logger);
-			logger.info("job built.");
 		}
 		catch (Exception e) {
 			//job = new InitErrorJob(jobId, e);
diff --git a/src/java/azkaban/project/JdbcProjectLoader.java b/src/java/azkaban/project/JdbcProjectLoader.java
index 3d6362a..9553d48 100644
--- a/src/java/azkaban/project/JdbcProjectLoader.java
+++ b/src/java/azkaban/project/JdbcProjectLoader.java
@@ -93,6 +93,7 @@ public class JdbcProjectLoader extends AbstractJdbcLoader implements ProjectLoad
 			
 			for (Project project: projects) {
 				List<Triple<String, Boolean, Permission>> permissions = fetchPermissionsForProject(connection, project);
+				
 				for (Triple<String, Boolean, Permission> entry: permissions) {
 					if(entry.getSecond()) {
 						project.setGroupPermission(entry.getFirst(), entry.getThird());
@@ -102,7 +103,8 @@ public class JdbcProjectLoader extends AbstractJdbcLoader implements ProjectLoad
 					}
 				}
 			}
-		} catch (SQLException e) {
+		}
+		catch (SQLException e) {
 			throw new ProjectManagerException("Error retrieving all projects", e);
 		}
 		finally {
@@ -269,6 +271,7 @@ public class JdbcProjectLoader extends AbstractJdbcLoader implements ProjectLoad
 		}
 	}
 
+	@SuppressWarnings("resource")
 	private void uploadProjectFile(Connection connection, Project project, int version, String filetype, String filename, File localFile, String uploader) throws ProjectManagerException {
 		QueryRunner runner = new QueryRunner();
 		long updateTime = System.currentTimeMillis();
@@ -358,6 +361,7 @@ public class JdbcProjectLoader extends AbstractJdbcLoader implements ProjectLoad
 		return handler;
 	}
 	
+	@SuppressWarnings("resource")
 	private ProjectFileHandler getUploadedFile(Connection connection, int projectId, int version) throws ProjectManagerException {
 		QueryRunner runner = new QueryRunner();
 		ProjectVersionResultHandler pfHandler = new ProjectVersionResultHandler();
@@ -687,7 +691,8 @@ public class JdbcProjectLoader extends AbstractJdbcLoader implements ProjectLoad
 		}
 		catch (IOException e) {
 			throw new ProjectManagerException("Flow Upload failed.", e);
-		} catch (SQLException e) {
+		} 
+		catch (SQLException e) {
 			throw new ProjectManagerException("Flow Upload failed commit.", e);
 		}
 		finally {
diff --git a/src/java/azkaban/scheduler/ScheduleManager.java b/src/java/azkaban/scheduler/ScheduleManager.java
index f0325af..39e42e9 100644
--- a/src/java/azkaban/scheduler/ScheduleManager.java
+++ b/src/java/azkaban/scheduler/ScheduleManager.java
@@ -47,22 +47,10 @@ public class ScheduleManager implements TriggerAgent {
 	private final DateTimeFormatter _dateFormat = DateTimeFormat.forPattern("MM-dd-yyyy HH:mm:ss:SSS");
 	private ScheduleLoader loader;
 
-//	private Map<Pair<Integer, String>, Set<Schedule>> scheduleIdentityPairMap = new LinkedHashMap<Pair<Integer, String>, Set<Schedule>>();
 	private Map<Integer, Schedule> scheduleIDMap = new LinkedHashMap<Integer, Schedule>();
 	private Map<Pair<Integer, String>, Schedule> scheduleIdentityPairMap = new LinkedHashMap<Pair<Integer, String>, Schedule>();
-	
-//	private final ExecutorManagerAdapter executorManager;
-//	
-//	private ProjectManager projectManager = null;
-//	
+
 	// Used for mbeans to query Scheduler status
-//<<<<<<< HEAD
-//	
-//=======
-//	private long lastCheckTime = -1;
-//	private long nextWakupTime = -1;
-//	private String runnerStage = "not started";
-//>>>>>>> 10830aeb8ac819473873cac3bb4e07b4aeda67e8
 
 	/**
 	 * Give the schedule manager a loader class that will properly load the
@@ -72,15 +60,9 @@ public class ScheduleManager implements TriggerAgent {
 	 */
 	public ScheduleManager (ScheduleLoader loader) 
 	{
-//		this.executorManager = executorManager;
 		this.loader = loader;
-		
 	}
-	
-//	public void setProjectManager(ProjectManager projectManager) {
-//		this.projectManager = projectManager;
-//	}
-	
+
 	@Override
 	public void start() throws ScheduleManagerException {
 		List<Schedule> scheduleList = null;
@@ -178,7 +160,6 @@ public class ScheduleManager implements TriggerAgent {
 		return scheduleIDMap.get(scheduleId);
 	}
 
-
 	/**
 	 * Removes the flow from the schedule if it exists.
 	 * 
@@ -199,6 +180,7 @@ public class ScheduleManager implements TriggerAgent {
 			removeSchedule(sched);
 		}
 	}
+
 	/**
 	 * Removes the flow from the schedule if it exists.
 	 * 
@@ -219,15 +201,13 @@ public class ScheduleManager implements TriggerAgent {
 		}
 
 		scheduleIDMap.remove(sched.getScheduleId());
-		
+	
 		try {
 			loader.removeSchedule(sched);
 		} catch (ScheduleManagerException e) {
 			// TODO Auto-generated catch block
 			e.printStackTrace();
 		}
-
-		
 	}
 
 	// public synchronized void pauseScheduledFlow(String scheduleId){
@@ -339,12 +319,11 @@ public class ScheduleManager implements TriggerAgent {
 				// TODO Auto-generated catch block
 				e.printStackTrace();
 			}
-		}
-		else {
-			logger.error("The provided schedule is non-recurring and the scheduled time already passed. " + s.getScheduleName());
+		} else {
+			logger.error("The provided schedule is non-recurring and the scheduled time already passed. "
+					+ s.getScheduleName());
 		}
 	}
-	
 
 	@Override
 	public void loadTriggerFromProps(Props props) throws ScheduleManagerException {
@@ -354,9 +333,6 @@ public class ScheduleManager implements TriggerAgent {
 	/**
 	 * Thread that simply invokes the running of flows when the schedule is
 	 * ready.
-	 * 
-	 * @author Richard Park
-	 * 
 	 */
 //	public class ScheduleRunner extends Thread {
 //		private final PriorityBlockingQueue<Schedule> schedules;
@@ -612,6 +588,4 @@ public class ScheduleManager implements TriggerAgent {
 	public String getTriggerSource() {
 		return triggerSource;
 	}
-	
-
 }
diff --git a/src/java/azkaban/trigger/builtin/ExecuteFlowAction.java b/src/java/azkaban/trigger/builtin/ExecuteFlowAction.java
index ab1586a..caa07bf 100644
--- a/src/java/azkaban/trigger/builtin/ExecuteFlowAction.java
+++ b/src/java/azkaban/trigger/builtin/ExecuteFlowAction.java
@@ -216,7 +216,7 @@ public class ExecuteFlowAction implements TriggerAction {
 			throw new RuntimeException("Error finding the flow to execute " + flowName);
 		}
 		
-		ExecutableFlow exflow = new ExecutableFlow(flow);
+		ExecutableFlow exflow = new ExecutableFlow(project, flow);
 		exflow.setSubmitUser(submitUser);
 		exflow.addAllProxyUsers(project.getProxyUsers());
 		
diff --git a/src/java/azkaban/trigger/builtin/KillExecutionAction.java b/src/java/azkaban/trigger/builtin/KillExecutionAction.java
index dd53efe..3114fd2 100644
--- a/src/java/azkaban/trigger/builtin/KillExecutionAction.java
+++ b/src/java/azkaban/trigger/builtin/KillExecutionAction.java
@@ -23,6 +23,7 @@ import org.apache.log4j.Logger;
 
 import azkaban.executor.ExecutableFlow;
 import azkaban.executor.ExecutorManagerAdapter;
+import azkaban.executor.Status;
 import azkaban.trigger.TriggerAction;
 
 public class KillExecutionAction implements TriggerAction{
@@ -89,7 +90,7 @@ public class KillExecutionAction implements TriggerAction{
 	public void doAction() throws Exception {
 		ExecutableFlow exFlow = executorManager.getExecutableFlow(execId);
 		logger.info("ready to kill execution " + execId);
-		if(!ExecutableFlow.isFinished(exFlow)) {
+		if(!Status.isStatusFinished(exFlow.getStatus())) {
 			logger.info("Killing execution " + execId);
 			executorManager.cancelFlow(exFlow, "azkaban_sla");
 		}
diff --git a/src/java/azkaban/user/User.java b/src/java/azkaban/user/User.java
index 2262d92..8ea3f59 100644
--- a/src/java/azkaban/user/User.java
+++ b/src/java/azkaban/user/User.java
@@ -17,6 +17,7 @@
 package azkaban.user;
 
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
@@ -28,6 +29,8 @@ public class User {
 	private Set<String> groups = new HashSet<String>();
 	private UserPermissions userPermissions;
 
+	private HashMap<String,String> properties = new HashMap<String,String>();
+	
 	public User(String userid) {
 		this.userid = userid;
 	}
@@ -86,6 +89,10 @@ public class User {
 	public boolean hasRole(String role) {
 		return roles.contains(role);
 	}
+	
+	public String getProperty(String name) {
+		return properties.get(name);
+	}
 
 	public String toString() {
 		String groupStr = "[";
diff --git a/src/java/azkaban/utils/DirectoryFlowLoader.java b/src/java/azkaban/utils/DirectoryFlowLoader.java
index 35d0766..2bcd069 100644
--- a/src/java/azkaban/utils/DirectoryFlowLoader.java
+++ b/src/java/azkaban/utils/DirectoryFlowLoader.java
@@ -35,6 +35,7 @@ import azkaban.flow.Edge;
 import azkaban.flow.Flow;
 import azkaban.flow.FlowProps;
 import azkaban.flow.Node;
+import azkaban.flow.SpecialJobTypes;
 
 public class DirectoryFlowLoader {
 	private static final DirFilter DIR_FILTER = new DirFilter();
@@ -42,15 +43,20 @@ public class DirectoryFlowLoader {
 	private static final String JOB_SUFFIX = ".job";
 	
 	private final Logger logger;
+	private HashSet<String> rootNodes;
 	private HashMap<String, Flow> flowMap;
 	private HashMap<String, Node> nodeMap;
 	private HashMap<String, Map<String, Edge>> nodeDependencies;
 	private HashMap<String, Props> jobPropsMap;
+	
+	// Flow dependencies for embedded flows.
+	private HashMap<String, Set<String>> flowDependencies;
+	
 	private ArrayList<FlowProps> flowPropsList;
 	private ArrayList<Props> propsList;
 	private Set<String> errors;
 	private Set<String> duplicateJobs;
-	
+
 	public DirectoryFlowLoader(Logger logger) {
 		this.logger = logger;
 	}
@@ -80,16 +86,20 @@ public class DirectoryFlowLoader {
 		errors = new HashSet<String>();
 		duplicateJobs = new HashSet<String>();
 		nodeDependencies = new HashMap<String, Map<String, Edge>>();
-
+		rootNodes = new HashSet<String>();
+		flowDependencies = new HashMap<String, Set<String>>();
+		
 		// Load all the props files and create the Node objects
 		loadProjectFromDir(baseDirectory.getPath(), baseDirectory, null);
 		
 		// Create edges and find missing dependencies
 		resolveDependencies();
-
+		
 		// Create the flows.
 		buildFlowsFromDependencies();
-
+		
+		// Resolve embedded flows
+		resolveEmbeddedFlows();
 	}
 	
 	private void loadProjectFromDir(String base, File dir, Props parent) {
@@ -112,7 +122,6 @@ public class DirectoryFlowLoader {
 			propsList.add(parent);
 		}
 		
-		
 		// Load all Job files. If there's a duplicate name, then we don't load
 		File[] jobFiles = dir.listFiles(new SuffixFilter(JOB_SUFFIX));
 		for (File file: jobFiles) {
@@ -143,11 +152,15 @@ public class DirectoryFlowLoader {
 							node.setPropsSource(parent.getSource());
 						}
 
+						// Force root node
+						if(prop.getBoolean(CommonJobProperties.ROOT_NODE, false)) {
+							rootNodes.add(jobName);
+						}
+						
 						jobPropsMap.put(jobName, prop);
 						nodeMap.put(jobName, node);
 					}
 				}
-				
 			} catch (IOException e) {
 				errors.add("Error loading job file " + file.getName() + ":" + e.getMessage());
 			}
@@ -159,6 +172,37 @@ public class DirectoryFlowLoader {
 		}
 	}
 	
+	private void resolveEmbeddedFlows() {
+		for (String flowId: flowDependencies.keySet()) {
+			HashSet<String> visited = new HashSet<String>();
+			resolveEmbeddedFlow(flowId, visited);
+		}
+	}
+	
+	private void resolveEmbeddedFlow(String flowId, Set<String> visited) {
+		Set<String> embeddedFlow = flowDependencies.get(flowId);
+		if (embeddedFlow == null) {
+			return;
+		}
+		
+		visited.add(flowId);
+		for (String embeddedFlowId: embeddedFlow) {
+			if (visited.contains(embeddedFlowId)) {
+				errors.add("Embedded flow cycle found in " + flowId + "->" + embeddedFlowId);
+				return;
+			}
+			else if (!flowMap.containsKey(embeddedFlowId)) {
+				errors.add("Flow " + flowId + " depends on " + embeddedFlowId + " but can't be found.");
+				return;
+			}
+			else {
+				resolveEmbeddedFlow(embeddedFlowId, visited);
+			}
+		}
+		
+		visited.remove(flowId);
+	}
+	
 	private void resolveDependencies() {
 		// Add all the in edges and out edges. Catch bad dependencies and self referrals. Also collect list of nodes who are parents.
 		for (Node node: nodeMap.values()) {
@@ -227,7 +271,9 @@ public class DirectoryFlowLoader {
 		// Now create flows. Bad flows are marked invalid
 		Set<String> visitedNodes = new HashSet<String>();
 		for (Node base: nodeMap.values()) {
-			if (!nonRootNodes.contains(base.getId())) {
+			// Root nodes can be discovered when parsing jobs
+			if (rootNodes.contains(base.getId()) || !nonRootNodes.contains(base.getId())) {
+				rootNodes.add(base.getId());
 				Flow flow = new Flow(base.getId());
 				Props jobProp = jobPropsMap.get(base.getId());
 				
@@ -268,8 +314,20 @@ public class DirectoryFlowLoader {
 	private void constructFlow(Flow flow, Node node, Set<String> visited) {
 		visited.add(node.getId());
 
-		// Clone the node so each flow can operate on its own node
 		flow.addNode(node);
+		if (SpecialJobTypes.EMBEDDED_FLOW_TYPE.equals(node.getType())) {
+			Props props = jobPropsMap.get(node.getId());
+			String embeddedFlow = props.get(SpecialJobTypes.FLOW_NAME);
+			
+			Set<String> embeddedFlows = flowDependencies.get(flow.getId());
+			if (embeddedFlows == null) {
+				embeddedFlows = new HashSet<String>();
+				flowDependencies.put(flow.getId(), embeddedFlows);
+			}
+
+			node.setEmbeddedFlowId(embeddedFlow);
+			embeddedFlows.add(embeddedFlow);
+		}
 		Map<String, Edge> dependencies = nodeDependencies.get(node.getId());
 
 		if (dependencies != null) {
@@ -295,7 +353,7 @@ public class DirectoryFlowLoader {
 
 		visited.remove(node.getId());
 	}
-
+	
 	private String getNameWithoutExtension(File file) {
 		String filename = file.getName();
 		int index = filename.lastIndexOf('.');
diff --git a/src/java/azkaban/utils/Emailer.java b/src/java/azkaban/utils/Emailer.java
index dcc749c..a4e2ef8 100644
--- a/src/java/azkaban/utils/Emailer.java
+++ b/src/java/azkaban/utils/Emailer.java
@@ -163,7 +163,7 @@ public class Emailer extends AbstractMailer implements Alerter {
 		ArrayList<String> failedJobs = new ArrayList<String>();
 		for (ExecutableNode node : flow.getExecutableNodes()) {
 			if (node.getStatus() == Status.FAILED) {
-				failedJobs.add(node.getJobId());
+				failedJobs.add(node.getId());
 			}
 		}
 		return failedJobs;
diff --git a/src/java/azkaban/utils/JSONUtils.java b/src/java/azkaban/utils/JSONUtils.java
index e811b82..ad78b6c 100644
--- a/src/java/azkaban/utils/JSONUtils.java
+++ b/src/java/azkaban/utils/JSONUtils.java
@@ -88,6 +88,15 @@ public class JSONUtils {
 		stream.close();
 	}
 	
+	public static Object parseJSONFromStringQuiet(String json) {
+		try {
+			return parseJSONFromString(json);
+		} catch (IOException e) {
+			e.printStackTrace();
+			return null;
+		}
+	}
+	
 	public static Object parseJSONFromString(String json) throws IOException {
 		ObjectMapper mapper = new ObjectMapper();
 		JsonFactory factory = new JsonFactory();
diff --git a/src/java/azkaban/utils/Props.java b/src/java/azkaban/utils/Props.java
index 7c0df22..eb39dd6 100644
--- a/src/java/azkaban/utils/Props.java
+++ b/src/java/azkaban/utils/Props.java
@@ -156,7 +156,12 @@ public class Props {
 			putAll(props);
 		}
 	}
-
+	
+	public void setEarliestAncestor(Props parent) {
+		Props props = getEarliestAncestor();
+		props.setParent(parent);
+	}
+	
 	public Props getEarliestAncestor() {
 		if (_parent == null) {
 			return this;
diff --git a/src/java/azkaban/utils/PropsUtils.java b/src/java/azkaban/utils/PropsUtils.java
index 54993ee..37cd003 100644
--- a/src/java/azkaban/utils/PropsUtils.java
+++ b/src/java/azkaban/utils/PropsUtils.java
@@ -28,7 +28,7 @@ import java.util.UUID;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
-import azkaban.executor.ExecutableFlow;
+import azkaban.executor.ExecutableFlowBase;
 import azkaban.flow.CommonJobProperties;
 
 import org.apache.commons.lang.StringUtils;
@@ -144,8 +144,7 @@ public class PropsUtils {
 		return false;
 	}
 
-	private static final Pattern VARIABLE_PATTERN = Pattern
-			.compile("\\$\\{([a-zA-Z_.0-9]+)\\}");
+	private static final Pattern VARIABLE_PATTERN = Pattern.compile("\\$\\{([a-zA-Z_.0-9]+)\\}");
 
 	public static Props resolveProps(Props props) {
 		if (props == null) return null;
@@ -209,27 +208,27 @@ public class PropsUtils {
 		return buffer.toString();
 	}
 	
-	public static Props addCommonFlowProperties(final ExecutableFlow flow) {
-		Props parentProps = new Props();
+	public static Props addCommonFlowProperties(Props parentProps, final ExecutableFlowBase flow) {
+		Props props = new Props(parentProps);
 
-		parentProps.put(CommonJobProperties.FLOW_ID, flow.getFlowId());
-		parentProps.put(CommonJobProperties.EXEC_ID, flow.getExecutionId());
-		parentProps.put(CommonJobProperties.PROJECT_ID, flow.getProjectId());
-		parentProps.put(CommonJobProperties.PROJECT_VERSION, flow.getVersion());
-		parentProps.put(CommonJobProperties.FLOW_UUID, UUID.randomUUID().toString());
+		props.put(CommonJobProperties.FLOW_ID, flow.getFlowId());
+		props.put(CommonJobProperties.EXEC_ID, flow.getExecutionId());
+		props.put(CommonJobProperties.PROJECT_ID, flow.getProjectId());
+		props.put(CommonJobProperties.PROJECT_VERSION, flow.getVersion());
+		props.put(CommonJobProperties.FLOW_UUID, UUID.randomUUID().toString());
 
 		DateTime loadTime = new DateTime();
 
-		parentProps.put(CommonJobProperties.FLOW_START_TIMESTAMP, loadTime.toString());
-		parentProps.put(CommonJobProperties.FLOW_START_YEAR, loadTime.toString("yyyy"));
-		parentProps.put(CommonJobProperties.FLOW_START_MONTH, loadTime.toString("MM"));
-		parentProps.put(CommonJobProperties.FLOW_START_DAY, loadTime.toString("dd"));
-		parentProps.put(CommonJobProperties.FLOW_START_HOUR, loadTime.toString("HH"));
-		parentProps.put(CommonJobProperties.FLOW_START_MINUTE, loadTime.toString("mm"));
-		parentProps.put(CommonJobProperties.FLOW_START_SECOND, loadTime.toString("ss"));
-		parentProps.put(CommonJobProperties.FLOW_START_MILLISSECOND, loadTime.toString("SSS"));
-		parentProps.put(CommonJobProperties.FLOW_START_TIMEZONE, loadTime.toString("ZZZZ"));
-		return parentProps;
+		props.put(CommonJobProperties.FLOW_START_TIMESTAMP, loadTime.toString());
+		props.put(CommonJobProperties.FLOW_START_YEAR, loadTime.toString("yyyy"));
+		props.put(CommonJobProperties.FLOW_START_MONTH, loadTime.toString("MM"));
+		props.put(CommonJobProperties.FLOW_START_DAY, loadTime.toString("dd"));
+		props.put(CommonJobProperties.FLOW_START_HOUR, loadTime.toString("HH"));
+		props.put(CommonJobProperties.FLOW_START_MINUTE, loadTime.toString("mm"));
+		props.put(CommonJobProperties.FLOW_START_SECOND, loadTime.toString("ss"));
+		props.put(CommonJobProperties.FLOW_START_MILLISSECOND, loadTime.toString("SSS"));
+		props.put(CommonJobProperties.FLOW_START_TIMEZONE, loadTime.toString("ZZZZ"));
+		return props;
 	}
 
 	public static String toJSONString(Props props, boolean localOnly) {
diff --git a/src/java/azkaban/utils/TypedMapWrapper.java b/src/java/azkaban/utils/TypedMapWrapper.java
new file mode 100644
index 0000000..cce512b
--- /dev/null
+++ b/src/java/azkaban/utils/TypedMapWrapper.java
@@ -0,0 +1,141 @@
+package azkaban.utils;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+
+public class TypedMapWrapper<K, V> {
+	private Map<K,V> map;
+	public TypedMapWrapper(Map<K, V> map) {
+		this.map = map;
+	}
+	
+	public String getString(K key) {
+		return getString(key, null);
+	}
+	
+	public String getString(K key, String defaultVal) {
+		Object obj = map.get(key);
+		if (obj == null) {
+			return defaultVal;
+		}
+		if (obj instanceof String) {
+			return (String)obj;
+		}
+	
+		return obj.toString();
+	}
+	
+	public Boolean getBool(K key, Boolean defaultVal) {
+		Object obj = map.get(key);
+		if (obj == null) {
+			return defaultVal;
+		}
+		
+		return (Boolean)obj;
+	}
+	
+	public Integer getInt(K key) {
+		return getInt(key, -1);
+	}
+	
+	public Integer getInt(K key, Integer defaultVal) {
+		Object obj = map.get(key);
+		if (obj == null) {
+			return defaultVal;
+		}
+		if (obj instanceof Integer) {
+			return (Integer)obj;
+		}
+		else if (obj instanceof String) {
+			return Integer.valueOf((String)obj);
+		}
+		else {
+			return defaultVal;
+		}
+	}
+	
+	public Long getLong(K key) {
+		return getLong(key, -1l);
+	}
+	
+	public Long getLong(K key, Long defaultVal) {
+		Object obj = map.get(key);
+		if (obj == null) {
+			return defaultVal;
+		}
+		if (obj instanceof Long) {
+			return (Long)obj;
+		}
+		else if (obj instanceof Integer) {
+			return Long.valueOf((Integer)obj);
+		}
+		else if (obj instanceof String) {
+			return Long.valueOf((String)obj);
+		}
+		else {
+			return defaultVal;
+		}
+	}
+	
+	@SuppressWarnings("unchecked")
+	public Collection<String> getStringCollection(K key) {
+		Object obj = map.get(key);
+		return (Collection<String>)obj;
+	}
+	
+	@SuppressWarnings("unchecked")
+	public Collection<String> getStringCollection(K key, Collection<String> defaultVal) {
+		Object obj = map.get(key);
+		if (obj == null) {
+			return defaultVal;
+		}
+		
+		return (Collection<String>)obj;
+	}
+	
+	
+	@SuppressWarnings("unchecked")
+	public <C> Collection<C> getCollection(K key) {
+		Object obj = map.get(key);
+		if (obj instanceof Collection) {
+			return (Collection<C>)obj;
+		}
+		return null;
+	}
+	
+	@SuppressWarnings("unchecked")
+	public <L> List<L> getList(K key) {
+		Object obj = map.get(key);
+		if (obj instanceof List) {
+			return (List<L>)obj;
+		}
+		return null;
+	}
+	
+	@SuppressWarnings("unchecked")
+	public <L> List<L> getList(K key, List<L> defaultVal) {
+		Object obj = map.get(key);
+		if (obj instanceof List) {
+			return (List<L>)obj;
+		}
+		return defaultVal;
+	}
+	
+	public Object getObject(K key) {
+		return map.get(key);
+	}
+	
+	public Map<K, V> getMap() {
+		return map;
+	}
+	
+	@SuppressWarnings("unchecked")
+	public <S, T> Map<S,T> getMap(K key) {
+		return (Map<S,T>)map.get(key);
+	}
+	
+	public boolean containsKey(K key) {
+		return map.containsKey(key);
+	}
+}
diff --git a/src/java/azkaban/webapp/AzkabanWebServer.java b/src/java/azkaban/webapp/AzkabanWebServer.java
index f298763..21c6163 100644
--- a/src/java/azkaban/webapp/AzkabanWebServer.java
+++ b/src/java/azkaban/webapp/AzkabanWebServer.java
@@ -197,6 +197,12 @@ public class AzkabanWebServer extends AzkabanServer {
 		
 		tempDir = new File(props.getString("azkaban.temp.dir", "temp"));
 
+		File statsDir = new File(props.getString("azkaban.stats.dir", "stats"));
+		if (!statsDir.exists()) {
+			statsDir.mkdir();
+		}
+    props.put("azkaban.stats.dir", statsDir.getCanonicalPath());
+
 		// Setup time zone
 		if (props.containsKey(DEFAULT_TIMEZONE_ID)) {
 			String timezone = props.getString(DEFAULT_TIMEZONE_ID);
@@ -588,8 +594,8 @@ public class AzkabanWebServer extends AzkabanServer {
 	}
 
 	/**
-     * 
-     */
+		 * 
+		 */
 	public ExecutorManager getExecutorManager() {
 		return executorManager;
 	}
@@ -1098,10 +1104,11 @@ public class AzkabanWebServer extends AzkabanServer {
 				obj = constructor.newInstance(pluginProps);
 			} catch (Exception e) {
 				logger.error(e);
+				logger.error(e.getCause());
 			} 
 			
 			if (!(obj instanceof AbstractAzkabanServlet)) {
-				logger.error("The object is not an AbstractViewerServlet");
+				logger.error("The object is not an AbstractAzkabanServlet");
 				continue;
 			}
 			
diff --git a/src/java/azkaban/webapp/servlet/ExecutorServlet.java b/src/java/azkaban/webapp/servlet/ExecutorServlet.java
index 55f5d5b..309ffc8 100644
--- a/src/java/azkaban/webapp/servlet/ExecutorServlet.java
+++ b/src/java/azkaban/webapp/servlet/ExecutorServlet.java
@@ -29,7 +29,9 @@ import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 
 import azkaban.executor.ExecutableFlow;
+import azkaban.executor.ExecutableFlowBase;
 import azkaban.executor.ExecutableNode;
+import azkaban.executor.ExecutionAttempt;
 import azkaban.executor.ExecutionOptions;
 import azkaban.executor.ExecutorManagerAdapter;
 import azkaban.executor.ExecutionOptions.FailureAction;
@@ -117,9 +119,6 @@ public class ExecutorServlet extends LoginAbstractAzkabanServlet {
 				else if (ajaxName.equals("cancelFlow")) {
 					ajaxCancelFlow(req, resp, ret, session.getUser(), exFlow);
 				}
-				else if (ajaxName.equals("restartFlow")) {
-					ajaxRestartFlow(req, resp, ret, session.getUser(), exFlow);
-				}
 				else if (ajaxName.equals("pauseFlow")) {
 					ajaxPauseFlow(req, resp, ret, session.getUser(), exFlow);
 				}
@@ -307,7 +306,7 @@ public class ExecutorServlet extends LoginAbstractAzkabanServlet {
 		else {
 			return project;
 		}
-		
+
 		return null;
 	}	
 
@@ -372,7 +371,7 @@ public class ExecutorServlet extends LoginAbstractAzkabanServlet {
 			ret.put("error", e.getMessage());
 		}
 	}
-	
+
 	/**
 	 * Gets the logs through plain text stream to reduce memory overhead.
 	 * 
@@ -567,10 +566,10 @@ public class ExecutorServlet extends LoginAbstractAzkabanServlet {
 		ret.put("pipelineLevel", options.getPipelineLevel());
 		ret.put("pipelineExecution", options.getPipelineExecutionId());
 		ret.put("queueLevel", options.getQueueLevel());
-		
-		HashMap<String, String> nodeStatus = new HashMap<String,String>();
-		for(ExecutableNode node : exflow.getExecutableNodes()) {
-			nodeStatus.put(node.getJobId(), node.getStatus().toString());
+
+		HashMap<String, String> nodeStatus = new HashMap<String, String>();
+		for (ExecutableNode node : exflow.getExecutableNodes()) {
+			nodeStatus.put(node.getId(), node.getStatus().toString());
 		}
 		ret.put("nodeStatus", nodeStatus);
 		ret.put("disabled", options.getDisabledJobs());
@@ -581,7 +580,7 @@ public class ExecutorServlet extends LoginAbstractAzkabanServlet {
 		if (project == null) {
 			return;
 		}
-		
+
 		try {
 			executorManager.cancelFlow(exFlow, user.getUserId());
 		} catch (ExecutorManagerException e) {
@@ -600,13 +599,6 @@ public class ExecutorServlet extends LoginAbstractAzkabanServlet {
 			ret.put("execIds", refs);
 		}
 	}
-	
-	private void ajaxRestartFlow(HttpServletRequest req, HttpServletResponse resp, HashMap<String, Object> ret, User user, ExecutableFlow exFlow) throws ServletException{
-		Project project = getProjectAjaxByPermission(ret, exFlow.getProjectId(), user, Type.EXECUTE);
-		if (project == null) {
-			return;
-		}
-	}
 
 	private void ajaxPauseFlow(HttpServletRequest req, HttpServletResponse resp, HashMap<String, Object> ret, User user, ExecutableFlow exFlow) throws ServletException{
 		Project project = getProjectAjaxByPermission(ret, exFlow.getProjectId(), user, Type.EXECUTE);
@@ -634,45 +626,86 @@ public class ExecutorServlet extends LoginAbstractAzkabanServlet {
 		}
 	}
 	
-	private void ajaxFetchExecutableFlowUpdate(HttpServletRequest req, HttpServletResponse resp, HashMap<String, Object> ret, User user, ExecutableFlow exFlow) throws ServletException{
-		Long lastUpdateTime = Long.parseLong(getParam(req, "lastUpdateTime"));
-		System.out.println("Fetching " + exFlow.getExecutionId());
-		
-		Project project = getProjectAjaxByPermission(ret, exFlow.getProjectId(), user, Type.READ);
-		if (project == null) {
-			return;
-		}
-
-		// Just update the nodes and flow states
-		ArrayList<Map<String, Object>> nodeList = new ArrayList<Map<String, Object>>();
-		for (ExecutableNode node : exFlow.getExecutableNodes()) {
-			if (node.getUpdateTime() <= lastUpdateTime) {
-				continue;
+	private Map<String,Object> getExecutableFlowUpdateInfo(ExecutableNode node, long lastUpdateTime) {
+		HashMap<String, Object> nodeObj = new HashMap<String,Object>();
+		if (node instanceof ExecutableFlowBase) {
+			ExecutableFlowBase base = (ExecutableFlowBase)node;
+			ArrayList<Map<String, Object>> nodeList = new ArrayList<Map<String, Object>>();
+			
+			for (ExecutableNode subNode: base.getExecutableNodes()) {
+				Map<String,Object> subNodeObj = getExecutableFlowUpdateInfo(subNode, lastUpdateTime);
+				if (!subNodeObj.isEmpty()) {
+					nodeList.add(subNodeObj);
+				}
 			}
 			
-			HashMap<String, Object> nodeObj = new HashMap<String,Object>();
-			nodeObj.put("id", node.getJobId());
+			if (!nodeList.isEmpty()) {
+				nodeObj.put("flow", base.getFlowId());
+				nodeObj.put("nodes", nodeList);
+			}
+		}
+		
+		if (node.getUpdateTime() > lastUpdateTime || !nodeObj.isEmpty()) {
+			nodeObj.put("id", node.getId());
 			nodeObj.put("status", node.getStatus());
 			nodeObj.put("startTime", node.getStartTime());
 			nodeObj.put("endTime", node.getEndTime());
-			nodeObj.put("attempt", node.getAttempt());
+			nodeObj.put("updateTime", node.getUpdateTime());
 			
+			nodeObj.put("attempt", node.getAttempt());
 			if (node.getAttempt() > 0) {
 				nodeObj.put("pastAttempts", node.getAttemptObjects());
 			}
+		}
+		
+		return nodeObj;
+	}
+	
+	private Map<String,Object> getExecutableNodeInfo(ExecutableNode node) {
+		HashMap<String, Object> nodeObj = new HashMap<String,Object>();
+		nodeObj.put("id", node.getId());
+		nodeObj.put("status", node.getStatus());
+		nodeObj.put("startTime", node.getStartTime());
+		nodeObj.put("endTime", node.getEndTime());
+		nodeObj.put("updateTime", node.getUpdateTime());
+		nodeObj.put("type", node.getType());
+		nodeObj.put("nestedId", node.getNestedId());
+		
+		nodeObj.put("attempt", node.getAttempt());
+		if (node.getAttempt() > 0) {
+			nodeObj.put("pastAttempts", node.getAttemptObjects());
+		}
+		
+		if (node.getInNodes() != null && !node.getInNodes().isEmpty()) {
+			nodeObj.put("in", node.getInNodes());
+		}
+		
+		if (node instanceof ExecutableFlowBase) {
+			ExecutableFlowBase base = (ExecutableFlowBase)node;
+			ArrayList<Map<String, Object>> nodeList = new ArrayList<Map<String, Object>>();
+			
+			for (ExecutableNode subNode: base.getExecutableNodes()) {
+				Map<String,Object> subNodeObj = getExecutableNodeInfo(subNode);
+				if (!subNodeObj.isEmpty()) {
+					nodeList.add(subNodeObj);
+				}
+			}
 			
-			nodeList.add(nodeObj);
+			nodeObj.put("flow", base.getFlowId());
+			nodeObj.put("nodes", nodeList);
+			nodeObj.put("flowId", base.getFlowId());
 		}
-
-		ret.put("nodes", nodeList);
-		ret.put("status", exFlow.getStatus().toString());
-		ret.put("startTime", exFlow.getStartTime());
-		ret.put("endTime", exFlow.getEndTime());
-		ret.put("submitTime", exFlow.getSubmitTime());
-		ret.put("updateTime", exFlow.getUpdateTime());
+		
+		return nodeObj;
 	}
 	
-	private void ajaxFetchExecutableFlow(HttpServletRequest req, HttpServletResponse resp, HashMap<String, Object> ret, User user, ExecutableFlow exFlow) throws ServletException {
+	private void ajaxFetchExecutableFlowUpdate(
+			HttpServletRequest req,
+			HttpServletResponse resp, 
+			HashMap<String, Object> ret, 
+			User user,
+			ExecutableFlow exFlow) throws ServletException {
+		Long lastUpdateTime = Long.parseLong(getParam(req, "lastUpdateTime"));
 		System.out.println("Fetching " + exFlow.getExecutionId());
 
 		Project project = getProjectAjaxByPermission(ret, exFlow.getProjectId(), user, Type.READ);
@@ -680,46 +713,41 @@ public class ExecutorServlet extends LoginAbstractAzkabanServlet {
 			return;
 		}
 		
-		ArrayList<Map<String, Object>> nodeList = new ArrayList<Map<String, Object>>();
-		ArrayList<Map<String, Object>> edgeList = new ArrayList<Map<String,Object>>();
-		for (ExecutableNode node : exFlow.getExecutableNodes()) {
-			HashMap<String, Object> nodeObj = new HashMap<String,Object>();
-			nodeObj.put("id", node.getJobId());
-			nodeObj.put("level", node.getLevel());
-			nodeObj.put("status", node.getStatus());
-			nodeObj.put("startTime", node.getStartTime());
-			nodeObj.put("endTime", node.getEndTime());
-			
-			// Add past attempts
-			if (node.getPastAttemptList() != null) {
-				ArrayList<Object> pastAttempts = new ArrayList<Object>();
-				for (ExecutableNode.Attempt attempt: node.getPastAttemptList()) {
-					pastAttempts.add(attempt.toObject());
-				}
-				nodeObj.put("pastAttempts", pastAttempts);
-			}
-			
-			nodeList.add(nodeObj);
-			
-			// Add edges
-			for (String out: node.getOutNodes()) {
-				HashMap<String, Object> edgeObj = new HashMap<String,Object>();
-				edgeObj.put("from", node.getJobId());
-				edgeObj.put("target", out);
-				edgeList.add(edgeObj);
-			}
+		Map<String, Object> map = getExecutableFlowUpdateInfo(exFlow, lastUpdateTime);
+		map.put("status", exFlow.getStatus());
+		map.put("startTime", exFlow.getStartTime());
+		map.put("endTime",  exFlow.getEndTime());
+		map.put("updateTime", exFlow.getUpdateTime());
+		ret.putAll(map);
+	}
+
+	private void ajaxFetchExecutableFlow(
+			HttpServletRequest req, 
+			HttpServletResponse resp, 
+			HashMap<String, Object> ret, 
+			User user,
+			ExecutableFlow exFlow) throws ServletException {
+		System.out.println("Fetching " + exFlow.getExecutionId());
+
+		Project project = getProjectAjaxByPermission(ret,
+				exFlow.getProjectId(), user, Type.READ);
+		if (project == null) {
+			return;
 		}
 
-		ret.put("nodes", nodeList);
-		ret.put("edges", edgeList);
-		ret.put("status", exFlow.getStatus().toString());
-		ret.put("startTime", exFlow.getStartTime());
-		ret.put("endTime", exFlow.getEndTime());
 		ret.put("submitTime", exFlow.getSubmitTime());
 		ret.put("submitUser", exFlow.getSubmitUser());
+		ret.put("execid", exFlow.getExecutionId());
+		ret.put("projectId", exFlow.getProjectId());
+		ret.put("project", project.getName());
+		
+		Map<String,Object> flowObj = getExecutableNodeInfo(exFlow);
+		ret.putAll(flowObj);
 	}
-	
-	private void ajaxAttemptExecuteFlow(HttpServletRequest req, HttpServletResponse resp, HashMap<String, Object> ret, User user) throws ServletException {
+
+	private void ajaxAttemptExecuteFlow(HttpServletRequest req,
+			HttpServletResponse resp, HashMap<String, Object> ret, User user)
+			throws ServletException {
 		String projectName = getParam(req, "project");
 		String flowId = getParam(req, "flow");
 		
@@ -728,7 +756,7 @@ public class ExecutorServlet extends LoginAbstractAzkabanServlet {
 			ret.put("error", "Project '" + projectName + "' doesn't exist.");
 			return;
 		}
-		
+
 		ret.put("flow", flowId);
 		Flow flow = project.getFlow(flowId);
 		if (flow == null) {
@@ -748,7 +776,7 @@ public class ExecutorServlet extends LoginAbstractAzkabanServlet {
 			ret.put("error", "Project '" + projectName + "' doesn't exist.");
 			return;
 		}
-		
+
 		ret.put("flow", flowId);
 		Flow flow = project.getFlow(flowId);
 		if (flow == null) {
@@ -756,7 +784,7 @@ public class ExecutorServlet extends LoginAbstractAzkabanServlet {
 			return;
 		}
 		
-		ExecutableFlow exflow = new ExecutableFlow(flow);
+		ExecutableFlow exflow = new ExecutableFlow(project, flow);
 		exflow.setSubmitUser(user.getUserId());
 		exflow.addAllProxyUsers(project.getProxyUsers());
 
@@ -769,7 +797,7 @@ public class ExecutorServlet extends LoginAbstractAzkabanServlet {
 			options.setSuccessEmails(flow.getSuccessEmails());
 		}
 		options.setMailCreator(flow.getMailCreator());
-		
+
 		try {
 			String message = executorManager.submitExecutableFlow(exflow, user.getUserId());
 			ret.put("message", message);
diff --git a/src/java/azkaban/webapp/servlet/HttpRequestUtils.java b/src/java/azkaban/webapp/servlet/HttpRequestUtils.java
index 896d93b..6da9d88 100644
--- a/src/java/azkaban/webapp/servlet/HttpRequestUtils.java
+++ b/src/java/azkaban/webapp/servlet/HttpRequestUtils.java
@@ -19,6 +19,7 @@ package azkaban.webapp.servlet;
 import java.util.Arrays;
 import java.util.Enumeration;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 
 import javax.servlet.ServletException;
@@ -27,6 +28,7 @@ import javax.servlet.http.HttpServletRequest;
 import azkaban.executor.ExecutionOptions;
 import azkaban.executor.ExecutionOptions.FailureAction;
 import azkaban.executor.mail.DefaultMailCreator;
+import azkaban.utils.JSONUtils;
 
 public class HttpRequestUtils {
 	public static ExecutionOptions parseFlowOptions(HttpServletRequest req) throws ServletException {
@@ -96,13 +98,14 @@ public class HttpRequestUtils {
 		}
 		
 		Map<String, String> flowParamGroup = getParamGroup(req, "flowOverride");
-		execOptions.setFlowParameters(flowParamGroup);
+		execOptions.addAllFlowParameters(flowParamGroup);
 		
 		if (hasParam(req, "disabled")) {
 			String disabled = getParam(req, "disabled");
 			if (!disabled.isEmpty()) {
-				String[] disabledNodes = disabled.split("\\s*,\\s*");
-				execOptions.setDisabledJobs(Arrays.asList(disabledNodes));
+				@SuppressWarnings("unchecked")
+				List<Object> disabledList = (List<Object>)JSONUtils.parseJSONFromStringQuiet(disabled);
+				execOptions.setDisabledJobs(disabledList);
 			}
 		}
 		return execOptions;
diff --git a/src/java/azkaban/webapp/servlet/ProjectManagerServlet.java b/src/java/azkaban/webapp/servlet/ProjectManagerServlet.java
index 4fdcb5c..d173bd0 100644
--- a/src/java/azkaban/webapp/servlet/ProjectManagerServlet.java
+++ b/src/java/azkaban/webapp/servlet/ProjectManagerServlet.java
@@ -28,9 +28,9 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
-import java.util.HashSet;
 import java.util.Set;
 
 import javax.servlet.ServletConfig;
@@ -67,6 +67,7 @@ import azkaban.user.User;
 import azkaban.utils.JSONUtils;
 import azkaban.utils.Pair;
 import azkaban.utils.Props;
+import azkaban.utils.PropsUtils;
 import azkaban.utils.Utils;
 import azkaban.webapp.AzkabanWebServer;
 import azkaban.webapp.session.Session;
@@ -209,6 +210,11 @@ public class ProjectManagerServlet extends LoginAbstractAzkabanServlet {
 				ajaxFetchFlowGraph(project, ret, req);
 			}
 		}
+		else if (ajaxName.equals("fetchflownodedata")) {
+			if (handleAjaxPermission(project, user, Type.READ, ret)) {
+				ajaxFetchFlowNodeData(project, ret, req);
+			}
+		}
 		else if (ajaxName.equals("fetchprojectflows")) {
 			if (handleAjaxPermission(project, user, Type.READ, ret)) {
 				ajaxFetchProjectFlows(project, ret, req);
@@ -490,7 +496,6 @@ public class ProjectManagerServlet extends LoginAbstractAzkabanServlet {
 	}
 	
 	private void ajaxFetchJobInfo(Project project, HashMap<String, Object> ret, HttpServletRequest req) throws ServletException {
-
 		String flowName = getParam(req, "flowName");
 		String jobName = getParam(req, "jobName");
 		
@@ -582,34 +587,86 @@ public class ProjectManagerServlet extends LoginAbstractAzkabanServlet {
 	
 	private void ajaxFetchFlowGraph(Project project, HashMap<String, Object> ret, HttpServletRequest req) throws ServletException {
 		String flowId = getParam(req, "flow");
+		
+		fillFlowInfo(project, flowId, ret);
+	}
+	
+	private void fillFlowInfo(Project project, String flowId, HashMap<String, Object> ret) {
 		Flow flow = project.getFlow(flowId);
 		
-		//Collections.sort(flowNodes, NODE_LEVEL_COMPARATOR);
 		ArrayList<Map<String, Object>> nodeList = new ArrayList<Map<String, Object>>();
 		for (Node node: flow.getNodes()) {
-			HashMap<String, Object> nodeObj = new HashMap<String,Object>();
+			HashMap<String, Object> nodeObj = new HashMap<String, Object>();
 			nodeObj.put("id", node.getId());
-			nodeObj.put("level", node.getLevel());
-
+			nodeObj.put("type", node.getType());
+			if (node.getEmbeddedFlowId() != null) {
+				nodeObj.put("flowId", node.getEmbeddedFlowId());
+				//HashMap<String, Object> embeddedNodeObj = new HashMap<String, Object>();
+				fillFlowInfo(project, node.getEmbeddedFlowId(), nodeObj);
+				//nodeObj.put("flowData", embeddedNodeObj);
+			}
+			
 			nodeList.add(nodeObj);
+			Set<Edge> inEdges = flow.getInEdges(node.getId());
+			if (inEdges != null && !inEdges.isEmpty()) {
+				ArrayList<String> inEdgesList = new ArrayList<String>();
+				for (Edge edge: inEdges) {
+					inEdgesList.add(edge.getSourceId());
+				}
+				Collections.sort(inEdgesList);
+				nodeObj.put("in", inEdgesList);
+			}
 		}
 		
-		ArrayList<Map<String, Object>> edgeList = new ArrayList<Map<String, Object>>();
-		for (Edge edge: flow.getEdges()) {
-			HashMap<String, Object> edgeObj = new HashMap<String,Object>();
-			edgeObj.put("from", edge.getSourceId());
-			edgeObj.put("target", edge.getTargetId());
-			
-			if (edge.hasError()) {
-				edgeObj.put("error", edge.getError());
+		Collections.sort(nodeList, new Comparator<Map<String, Object>>() {
+			@Override
+			public int compare(Map<String, Object> o1, Map<String, Object> o2) {
+				String id = (String)o1.get("id");
+				return id.compareTo((String)o2.get("id"));
 			}
-			
-			edgeList.add(edgeObj);
+		});
+
+		ret.put("flow", flowId);
+		ret.put("nodes", nodeList);
+	}
+	
+	private void ajaxFetchFlowNodeData(Project project, HashMap<String, Object> ret, HttpServletRequest req) throws ServletException {
+		String flowId = getParam(req, "flow");
+		Flow flow = project.getFlow(flowId);
+		
+		String nodeId = getParam(req, "node");
+		Node node = flow.getNode(nodeId);
+		
+		if (node == null) {
+			ret.put("error", "Job " + nodeId + " doesn't exist.");
+			return;
 		}
 		
-		ret.put("flowId", flowId);
-		ret.put("nodes", nodeList);
-		ret.put("edges", edgeList);
+		ret.put("id", nodeId);
+		ret.put("flow", flowId);
+		ret.put("type", node.getType());
+		
+		Props props;
+		try {
+			props = projectManager.getProperties(project, node.getJobSource());
+		} catch (ProjectManagerException e) {
+			ret.put("error", "Failed to upload job override property for " + nodeId);
+			return;
+		}
+		
+		if (props == null) {
+			ret.put("error", "Properties for " + nodeId + " isn't found.");
+			return;
+		}
+		
+		Map<String,String> properties = PropsUtils.toStringMap(props, true);
+		ret.put("props", properties);
+		
+		if (node.getType().equals("flow")) {
+			if (node.getEmbeddedFlowId() != null) {
+				fillFlowInfo(project, node.getEmbeddedFlowId(), ret);
+			}
+		}
 	}
 	
 	private void ajaxFetchFlow(Project project, HashMap<String, Object> ret, HttpServletRequest req, HttpServletResponse resp) throws ServletException {
diff --git a/src/java/azkaban/webapp/servlet/ScheduleServlet.java b/src/java/azkaban/webapp/servlet/ScheduleServlet.java
index c373608..6cc913e 100644
--- a/src/java/azkaban/webapp/servlet/ScheduleServlet.java
+++ b/src/java/azkaban/webapp/servlet/ScheduleServlet.java
@@ -303,21 +303,12 @@ public class ScheduleServlet extends LoginAbstractAzkabanServlet {
 					}
 				}
 			}
-			
-			List<String> disabledJobs;
-			if(flowOptions != null) {
-				disabledJobs = flowOptions.getDisabledJobs() == null ? new ArrayList<String>() : flowOptions.getDisabledJobs();
-			}
-			else {
-				disabledJobs = new ArrayList<String>();
-			}
-				
+
 			List<String> allJobs = new ArrayList<String>();
 			for(Node n : flow.getNodes()) {
-				if(!disabledJobs.contains(n.getId())) {
-					allJobs.add(n.getId());
-				}
+				allJobs.add(n.getId());
 			}
+			
 			ret.put("allJobNames", allJobs);
 		} catch (ServletException e) {
 			ret.put("error", e);
diff --git a/src/java/azkaban/webapp/servlet/velocity/executingflowpage.vm b/src/java/azkaban/webapp/servlet/velocity/executingflowpage.vm
index cb6fe85..d944f4c 100644
--- a/src/java/azkaban/webapp/servlet/velocity/executingflowpage.vm
+++ b/src/java/azkaban/webapp/servlet/velocity/executingflowpage.vm
@@ -20,7 +20,7 @@
 
 #parse("azkaban/webapp/servlet/velocity/style.vm")
 #parse("azkaban/webapp/servlet/velocity/javascript.vm")
-
+#parse("azkaban/webapp/servlet/velocity/svgflowincludes.vm")
 		<script type="text/javascript" src="${context}/js/moment.min.js"></script>
     <script type="text/javascript" src="${context}/js/bootstrap-datetimepicker.min.js"></script>
     
@@ -28,17 +28,7 @@
 		<script type="text/javascript" src="${context}/js/flowstats.js"></script>
 		<script type="text/javascript" src="${context}/js/flowstats-no-data.js"></script>
 
-		<script type="text/javascript" src="${context}/js/azkaban/util/common.js"></script>
-		<script type="text/javascript" src="${context}/js/azkaban/util/date.js"></script>
-		<script type="text/javascript" src="${context}/js/azkaban/view/context-menu.js"></script>
-		<script type="text/javascript" src="${context}/js/azkaban/util/ajax.js"></script>
-		<script type="text/javascript" src="${context}/js/azkaban/util/job-status.js"></script>
-		<script type="text/javascript" src="${context}/js/azkaban/util/layout.js"></script>
 		<script type="text/javascript" src="${context}/js/azkaban/view/exflow.js"></script>
-		<script type="text/javascript" src="${context}/js/azkaban/view/flow-stats.js"></script>
-		<script type="text/javascript" src="${context}/js/azkaban/view/flow-job.js"></script>
-		<script type="text/javascript" src="${context}/js/azkaban/view/svg-graph.js"></script>
-		<script type="text/javascript" src="${context}/js/azkaban/util/svg-navigate.js"></script>
 		<script type="text/javascript">
 			var contextURL = "${context}";
 			var currentTime = ${currentTime};
@@ -50,8 +40,8 @@
 			var flowId = "${flowid}";
 			var execId = "${execid}";
 		</script>
-		<link rel="stylesheet" type="text/css" href="${context}/css/azkaban-svg.css">
-		<link rel="stylesheet" type="text/css" href="${context}/css/bootstrap-datetimepicker.css" />
+
+		<link rel="stylesheet" type="text/css" href="${context}/css/jquery-ui-1.10.1.custom.css" />
 	</head>
 	<body>
 
@@ -128,7 +118,7 @@
     <div class="container-full" id="jobListView">
 			<div class="row">
 				<div class="col-xs-12">
-					<table class="table table-striped table-bordered table-condensed table-hover executions-table">
+					<table class="table table-bordered table-condensed table-hover executions-table">
 						<thead>
 							<tr>
 								<th>Name</th>
diff --git a/src/java/azkaban/webapp/servlet/velocity/flowexecutionpanel.vm b/src/java/azkaban/webapp/servlet/velocity/flowexecutionpanel.vm
index 5c04043..6895f48 100644
--- a/src/java/azkaban/webapp/servlet/velocity/flowexecutionpanel.vm
+++ b/src/java/azkaban/webapp/servlet/velocity/flowexecutionpanel.vm
@@ -14,13 +14,6 @@
  * the License.
 *#
 
-			<script type="text/javascript" src="${context}/js/azkaban/util/layout.js"></script>
-			<script type="text/javascript" src="${context}/js/azkaban/util/svg-navigate.js"></script>
-			<script type="text/javascript" src="${context}/js/azkaban/view/context-menu.js"></script>
-			<script type="text/javascript" src="${context}/js/azkaban/util/common.js"></script>
-			<script type="text/javascript" src="${context}/js/azkaban/view/svg-graph.js"></script>
-			<script type="text/javascript" src="${context}/js/azkaban/view/flow-execute.js"></script>
-
 			<div class="modal modal-wide" id="execute-flow-panel">
 				<div class="modal-dialog">
 					<div class="modal-content">
@@ -220,4 +213,3 @@
 *#
 
 			<div id="contextMenu"></div>
-
diff --git a/src/java/azkaban/webapp/servlet/velocity/flowextendedpanel.vm b/src/java/azkaban/webapp/servlet/velocity/flowextendedpanel.vm
new file mode 100644
index 0000000..557d09c
--- /dev/null
+++ b/src/java/azkaban/webapp/servlet/velocity/flowextendedpanel.vm
@@ -0,0 +1,22 @@
+<div id="flowInfoBase" class="flowExtendedView" style="display:none">
+	<div class="flowExtendedViewHeader">
+		<h3 class="flowInfoTitle"><span class="nodeId"></span><span class="nodeType"></span></h3>
+		<a title="Close" class="modal-close closeInfoPanel">x</a>
+	</div>
+	<div class="dataContent">
+		<div class="dataFlow">
+		</div>
+		<div class="dataJobProperties">
+			<table class="dataPropertiesTable">
+				<thead class="dataPropertiesHead">
+					<tr>
+						<th>Name</th>
+						<th>Value</th>
+					</tr>
+				</thead>
+				<tbody class="dataPropertiesBody">
+				</tbody>
+			</table>
+		</div>
+	</div>
+</div>
diff --git a/src/java/azkaban/webapp/servlet/velocity/flowgraphview.vm b/src/java/azkaban/webapp/servlet/velocity/flowgraphview.vm
index 60cdb96..4ef618d 100644
--- a/src/java/azkaban/webapp/servlet/velocity/flowgraphview.vm
+++ b/src/java/azkaban/webapp/servlet/velocity/flowgraphview.vm
@@ -17,19 +17,20 @@
 
  	## Graph view.
 
-    <div class="container-full container-fill" id="graphView">
+		<div class="container-full container-fill" id="graphView">
 			<div class="row row-offcanvas row-offcanvas-left">
 				<div class="col-xs-6 col-sm-3 sidebar-offcanvas graph-sidebar">
-          <div class="panel panel-default" id="jobList">
-            <div class="panel-heading">
-              <input id="filter" type="text" placeholder="Job Filter" class="form-control">
-            </div>
-            <div id="list" class="list-group"></div>
-            <div class="panel-footer">
-              <button type="button" class="btn btn-sm btn-default" id="resetPanZoomBtn">Reset Pan Zoom</button>
-            </div>
-          </div><!-- /.panel -->
-        </div><!-- /.col-sidebar -->
+					<div class="panel panel-default" id="jobList">
+						<div class="panel-heading">
+							<input id="filter" type="text" placeholder="Job Filter" class="form-control">
+						</div>
+						<div id="joblist"></div>
+						<div class="panel-footer">
+							<button type="button" class="btn btn-sm btn-default" id="resetPanZoomBtn">Reset Pan Zoom</button>
+							<button type="button" class="btn btn-sm btn-default" id="autoPanZoomBtn" data-toggle="button">Auto Pan Zoom</button>
+						</div>
+					</div><!-- /.panel -->
+				</div><!-- /.col-sidebar -->
 				<div class="col-xs-12 col-sm-9 col-content">
 					<div id="svgDiv" class="well well-clear well-sm">
 						<svg id="flow-graph" xmlns="http://www.w3.org/2000/svg" version="1.1" shape-rendering="optimize-speed" text-rendering="optimize-speed">
@@ -37,4 +38,4 @@
 					</div>
 				</div>
 			</div>
-    </div>
+		</div>
diff --git a/src/java/azkaban/webapp/servlet/velocity/flowpage.vm b/src/java/azkaban/webapp/servlet/velocity/flowpage.vm
index 3ae46d6..ddc9551 100644
--- a/src/java/azkaban/webapp/servlet/velocity/flowpage.vm
+++ b/src/java/azkaban/webapp/servlet/velocity/flowpage.vm
@@ -20,26 +20,19 @@
 
 #parse("azkaban/webapp/servlet/velocity/style.vm")
 #parse("azkaban/webapp/servlet/velocity/javascript.vm")
-
+#parse("azkaban/webapp/servlet/velocity/svgflowincludes.vm")
 		<script type="text/javascript" src="${context}/js/moment.min.js"></script>
-		<script type="text/javascript" src="${context}/js/bootstrap-datetimepicker.min.js"></script>
-		<script type="text/javascript" src="${context}/js/d3.v3.min.js"></script>
+    <script type="text/javascript" src="${context}/js/bootstrap-datetimepicker.min.js"></script>
+    <script type="text/javascript" src="${context}/js/raphael.min.js"></script>
+    <script type="text/javascript" src="${context}/js/morris.min.js"></script>
 		
     <script type="text/javascript" src="${context}/js/dust-full-2.2.3.min.js"></script>
 		<script type="text/javascript" src="${context}/js/flowsummary.js"></script>
 		<script type="text/javascript" src="${context}/js/flowstats-no-data.js"></script>
 		<script type="text/javascript" src="${context}/js/flowstats.js"></script>
-
-		<script type="text/javascript" src="${context}/js/azkaban/util/date.js"></script>
-		<script type="text/javascript" src="${context}/js/azkaban/util/ajax.js"></script>
-		<script type="text/javascript" src="${context}/js/azkaban/util/common.js"></script>
-		<script type="text/javascript" src="${context}/js/azkaban/util/layout.js"></script>
+		
 		<script type="text/javascript" src="${context}/js/azkaban/view/time-graph.js"></script>
 		<script type="text/javascript" src="${context}/js/azkaban/view/flow.js"></script>
-		<script type="text/javascript" src="${context}/js/azkaban/view/flow-stats.js"></script>
-		<script type="text/javascript" src="${context}/js/azkaban/view/flow-job.js"></script>
-		<script type="text/javascript" src="${context}/js/azkaban/view/flow-graph.js"></script>
-		<script type="text/javascript" src="${context}/js/azkaban/util/svg-navigate.js"></script>
 		<script type="text/javascript">
 			var contextURL = "${context}";
 			var currentTime = ${currentTime};
@@ -52,25 +45,7 @@
 			var flowId = "${flowid}";
 			var execId = null;
 		</script>
-		<style>
-			.axis path,
-			.axis line {
-			  fill: none;
-			  stroke: #000;
-			  shape-rendering: crispEdges;
-			}
-			
-			.x.axis path {
-			  display: none;
-			}
-			
-			.line {
-			  fill: none;
-			  stroke: steelblue;
-			  stroke-width: 1.5px;
-			}
-		</style>
-		<link rel="stylesheet" type="text/css" href="${context}/css/azkaban-svg.css" />
+		<link rel="stylesheet" type="text/css" href="${context}/css/morris.css" />
 		<link rel="stylesheet" type="text/css" href="${context}/css/bootstrap-datetimepicker.css" />
 	</head>
 	<body>
@@ -129,8 +104,7 @@
     <div class="container-full" id="executionsView">
 			<div class="row">
 				<div class="col-xs-12">
-					<div class="well well-clear well-sm">
-            <div id="timeGraph"></div>
+					<div class="well well-clear well-sm" id="timeGraph">
           </div>
 
 					<table class="table table-striped table-bordered table-condensed table-hover" id="execTable">
diff --git a/src/java/azkaban/webapp/servlet/velocity/index.vm b/src/java/azkaban/webapp/servlet/velocity/index.vm
index 9cbe273..3e8eef8 100644
--- a/src/java/azkaban/webapp/servlet/velocity/index.vm
+++ b/src/java/azkaban/webapp/servlet/velocity/index.vm
@@ -102,15 +102,9 @@
               </div>
               <div class="clearfix"></div>
               <div class="project-flows" id="${project.name}-child">
-                <table class="table">
-                  <thead>
-                    <tr>
-                      <th class="tb-name">Flows</th>
-                    </tr>
-                  </thead>
-                  <tbody id="${project.name}-tbody">
-                  </tbody>
-                </table>
+                <h5>Flows</h5>
+                <div class="list-group" id="${project.name}-tbody">
+                </div>
               </div>
             </li>
   #end
diff --git a/src/java/azkaban/webapp/servlet/velocity/jobdetailspage.vm b/src/java/azkaban/webapp/servlet/velocity/jobdetailspage.vm
index 0a88c63..50858db 100644
--- a/src/java/azkaban/webapp/servlet/velocity/jobdetailspage.vm
+++ b/src/java/azkaban/webapp/servlet/velocity/jobdetailspage.vm
@@ -80,8 +80,8 @@
   ## Tabs
 
 			<ul class="nav nav-tabs" id="headertabs">
-				<li id="jobSummaryViewLink"><a href="#jobsummary">Summary</a></li>
-				<li id="jobLogViewLink"><a href="#joblog">Log</a></li>
+				<li id="jobLogViewLink"><a href="#logs">Job Logs</a></li>
+				<li id="jobSummaryViewLink"><a href="#summary">Summary</a></li>
 				<li><a href="${context}/pigvisualizer?execid=${execid}&jobid=${jobid}">Visualization</a></li>
 			</ul>
     </div>
diff --git a/src/java/azkaban/webapp/servlet/velocity/jobhistorypage.vm b/src/java/azkaban/webapp/servlet/velocity/jobhistorypage.vm
index 48b7394..bdeca48 100644
--- a/src/java/azkaban/webapp/servlet/velocity/jobhistorypage.vm
+++ b/src/java/azkaban/webapp/servlet/velocity/jobhistorypage.vm
@@ -21,7 +21,8 @@
 #parse("azkaban/webapp/servlet/velocity/style.vm")
 #parse("azkaban/webapp/servlet/velocity/javascript.vm")
 
-		<script type="text/javascript" src="${context}/js/d3.v3.min.js"></script>
+    <script type="text/javascript" src="${context}/js/raphael.min.js"></script>
+    <script type="text/javascript" src="${context}/js/morris.min.js"></script>
 		<script type="text/javascript" src="${context}/js/azkaban/util/date.js"></script>
 		<script type="text/javascript" src="${context}/js/azkaban/view/time-graph.js"></script>
 		<script type="text/javascript" src="${context}/js/azkaban/view/job-history.js"></script>
@@ -37,24 +38,7 @@
 			var jobName = "$jobid";
 			var dataSeries = $dataSeries;
 		</script>
-		<style>
-			.axis path,
-			.axis line {
-			  fill: none;
-			  stroke: #000;
-			  shape-rendering: crispEdges;
-			}
-			
-			.x.axis path {
-			  display: none;
-			}
-			
-			.line {
-			  fill: none;
-			  stroke: steelblue;
-			  stroke-width: 1.5px;
-			}
-		</style>
+		<link rel="stylesheet" type="text/css" href="${context}/css/morris.css" />
 	</head>
 	<body>
 
@@ -88,8 +72,7 @@
 
 			<div class="row">
 				<div class="col-xs-12">
-					<div class="well well-clear well-sm">
-            <div id="timeGraph"></div>
+					<div class="well well-clear well-sm" id="timeGraph">
           </div>
 
           <table id="all-jobs" class="table table-striped table-bordered table-condensed table-hover">
diff --git a/src/java/azkaban/webapp/servlet/velocity/projectpage.vm b/src/java/azkaban/webapp/servlet/velocity/projectpage.vm
index e6b5314..631ec0e 100644
--- a/src/java/azkaban/webapp/servlet/velocity/projectpage.vm
+++ b/src/java/azkaban/webapp/servlet/velocity/projectpage.vm
@@ -20,12 +20,10 @@
 
 #parse ("azkaban/webapp/servlet/velocity/style.vm")
 #parse ("azkaban/webapp/servlet/velocity/javascript.vm")
-
+#parse ("azkaban/webapp/servlet/velocity/svgflowincludes.vm")
 		<link rel="stylesheet" type="text/css" href="${context}/css/bootstrap-datetimepicker.css" />
-		<link rel="stylesheet" type="text/css" href="${context}/css/azkaban-svg.css" />
 		<script type="text/javascript" src="${context}/js/moment.min.js"></script>
 		<script type="text/javascript" src="${context}/js/bootstrap-datetimepicker.min.js"></script>
-		<script type="text/javascript" src="${context}/js/azkaban/util/ajax.js"></script>
 		<script type="text/javascript" src="${context}/js/azkaban/view/project.js"></script>
 		<script type="text/javascript" src="${context}/js/azkaban/view/project-modals.js"></script>
 		<script type="text/javascript">
diff --git a/src/java/azkaban/webapp/servlet/velocity/svgflowincludes.vm b/src/java/azkaban/webapp/servlet/velocity/svgflowincludes.vm
new file mode 100644
index 0000000..773416e
--- /dev/null
+++ b/src/java/azkaban/webapp/servlet/velocity/svgflowincludes.vm
@@ -0,0 +1,38 @@
+#*
+ * Copyright 2012 LinkedIn Corp.
+ * 
+ * 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.
+*#
+
+	<script type="text/javascript" src="${context}/js/jquery.svg.min.js"></script> 
+	<script type="text/javascript" src="${context}/js/jquery.svganim.min.js"></script> 
+	<script type="text/javascript" src="${context}/js/jquery.svgfilter.min.js"></script>
+	
+	<script type="text/javascript" src="${context}/js/azkaban/util/svgutils.js"></script>
+	<script type="text/javascript" src="${context}/js/azkaban/view/flow-execute.js"></script>
+	<script type="text/javascript" src="${context}/js/azkaban/util/flow-loader.js"></script>
+	<script type="text/javascript" src="${context}/js/azkaban/util/common.js"></script>
+	
+	<script type="text/javascript" src="${context}/js/azkaban/view/context-menu.js"></script>
+	<script type="text/javascript" src="${context}/js/azkaban/util/ajax.js"></script>
+	<script type="text/javascript" src="${context}/js/azkaban/util/job-status.js"></script>
+	<script type="text/javascript" src="${context}/js/azkaban/util/layout.js"></script>
+
+	<script type="text/javascript" src="${context}/js/azkaban/util/date.js"></script>	
+	<script type="text/javascript" src="${context}/js/azkaban/view/flow-stats.js"></script>
+	<script type="text/javascript" src="${context}/js/azkaban/view/flow-job.js"></script>
+	<script type="text/javascript" src="${context}/js/azkaban/view/flow-execution-list.js"></script>
+	<script type="text/javascript" src="${context}/js/azkaban/view/svg-graph.js"></script>
+	<script type="text/javascript" src="${context}/js/azkaban/util/svg-navigate.js"></script>
+
+	<link rel="stylesheet" type="text/css" href="${context}/css/azkaban-graph.css" />
\ No newline at end of file
diff --git a/src/java/azkaban/webapp/session/SessionCache.java b/src/java/azkaban/webapp/session/SessionCache.java
index 2d033d5..7591dbf 100644
--- a/src/java/azkaban/webapp/session/SessionCache.java
+++ b/src/java/azkaban/webapp/session/SessionCache.java
@@ -33,6 +33,7 @@ import azkaban.utils.cache.Cache.EjectionPolicy;
 public class SessionCache {
 	private static final int MAX_NUM_SESSIONS = 10000;
 	private static final long SESSION_TIME_TO_LIVE = 24*60*60*1000L;
+
 //	private CacheManager manager = CacheManager.create();
 	private Cache cache;
 
@@ -81,4 +82,4 @@ public class SessionCache {
 	public boolean removeSession(String id) {
 		return cache.remove(id);
 	}
-}
+}
\ No newline at end of file
diff --git a/src/less/azkaban-graph.less b/src/less/azkaban-graph.less
new file mode 100644
index 0000000..5cd548c
--- /dev/null
+++ b/src/less/azkaban-graph.less
@@ -0,0 +1,137 @@
+.nodebox {
+  text {
+    pointer-events: none;
+  }
+
+  image {
+    pointer-events: none;
+  }
+
+  > .border:hover {
+    fill-opacity: 0.7;
+  }
+
+  > .flowborder:hover {
+    stroke-opacity: 0.7;
+  }
+}
+
+/* Nodes */
+.node {
+  &:hover {
+    cursor: pointer;
+  }
+
+  &.selected > .nodebox .border {
+    stroke-width: 3;
+    stroke: #39b3d7;
+  }
+
+  &.selected > .nodebox .flowborder {
+    stroke-width: 3;
+    fill: #D9EDFF;
+  }
+}
+
+.border {
+	stroke-width: 1;
+}
+
+.flownode .nodebox .flowborder {
+	stroke-width: 1.25;
+	fill: #FFF;
+	fill-opacity: 0.8;
+}
+
+.READY > g > rect {
+	fill: #DDD;
+	stroke: #CCC;
+}
+
+.READY > g > text {
+	fill: #000;
+}
+
+.RUNNING > g > rect {
+	fill: #39b3d7;
+	stroke: #39b3d7;
+}
+
+.RUNNING > g > text {
+	fill: #FFF;
+}
+
+.SUCCEEDED > g > rect {
+	fill: #5cb85c;
+	stroke: #4cae4c;
+}
+
+.SUCCEEDED > g > text {
+	fill: #FFF;
+}
+
+.FAILED > g > rect {
+	fill: #d2322d;
+	stroke: #d2322d;
+}
+
+.FAILED > g > text {
+	fill: #FFF;
+}
+
+.KILLED > g > rect {
+	fill: #FF9999;
+	stroke: #FF9999;
+}
+
+.KILLED > g > text {
+	fill: #FFF;
+}
+
+.FAILED_FINISHING > g > rect {
+	fill: #ed9c28;
+	stroke: #ed9c28;
+}
+
+.FAILED_FINISHING > g > text {
+	fill: #FFF;
+}
+
+.DISABLED > g > rect {
+	fill: #DDD;
+	stroke: #CCC;
+}
+
+.DISABLED > g > rect {
+	fill: #DDD;
+	stroke: #CCC;
+}
+
+.nodeDisabled {
+	opacity: 0.25;
+}
+
+.SKIPPED > g > rect {
+	fill: #DDD;
+	stroke: #CCC;
+}
+
+.DISABLED {
+	opacity: 0.25;
+}
+
+.SKIPPED {
+	opacity: 0.25;
+}
+
+/* Edges */
+.edge {
+	stroke: #CCC;
+	stroke-width: 1.5;
+
+  &:hover {
+    stroke: #009FC9;
+    stroke-width: 1.5;
+  }
+}
+

src/less/flow.less 81(+70 -11)

diff --git a/src/less/flow.less b/src/less/flow.less
index 4c23e39..99f6be5 100644
--- a/src/less/flow.less
+++ b/src/less/flow.less
@@ -60,9 +60,21 @@
   &.QUEUED {
     background-color: #009fc9;
   }
+  
+  &.KILLED {
+    background-color: #ff9999;
+  }
 }
 
 td {
+	> .listExpand {
+		width: 16px;
+		height: 16px;
+		float:right;
+		margin-top: 5px;
+		font-size: 8pt;
+	}
+
   .status {
     -moz-border-radius: 2px;
     border-radius: 2px;
@@ -103,7 +115,7 @@ td {
     }
 
     &.KILLED {
-      background-color: #d9534f;
+      background-color: #ff9999;
     }
   }
 }
@@ -138,44 +150,80 @@ td {
 }
 
 // TODO: Rename this as #job-list
-#list {
+#joblist {
   height: 100%;
+}
+ul.tree-list {
+  list-style-type: none;
+  padding-left: 0px;
+  margin: 0;
+}
+  
+li.tree-list-item {
+  &.active > a {
+    background-color: #D9EDFF;
+  }
 
-  a {
+  ul.tree-list {
+    padding-left: 20px;
+  }
+
+  &.subFilter > a > .expandarrow {
+    color : #3398cc;
+  }
+
+  > a {
+    clear: both;
+    position: relative;
+    display: block;
+    border-bottom-width: 0;
+    padding: 10px 15px;
+
+    &:hover,
+    &:focus {
+      text-decoration: none;
+      background-color: #f5f5f5;
+      cursor: pointer;
+    }
+  
     &.nodedisabled,
     &.DISABLED {
       opacity: 0.3;
     }
-
+    
     &.DISABLED .icon {
       background-position: 16px 0px;
     }
-
+    
     &.READY .icon {
       background-position: 16px 0px;
     }
-
+    
     &.QUEUED .icon {
       opacity: 0.5;
       background-position: 32px 0px;
     }
-
+    
     &.RUNNING .icon {
       background-position: 32px 0px;
     }
-
+    
     &.SUCCEEDED .icon {
       background-position: 48px 0px;
     }
-
+    
     &.FAILED .icon {
       background-position: 0px 0px;
     }
-
+    
     &.KILLED .icon {
       background-position: 0px 0px;
     }
-
+    
+    &.FAILED_FINISHING .icon {
+      background-position: 0px 0px;
+    }
+    
     .icon {
       float: left;
       width: 16px;
@@ -184,5 +232,16 @@ td {
       background-image: url("./images/dot-icon.png");
       background-position: 16px 0px;
     }
+    
+    .expandarrow {
+      float: right;
+      width: 16px;
+      height: 16px;
+      font-size: 8pt;
+    }
+    
+    .filterHighlight {
+      background-color: #FFFF00;
+    }
   }
 }

src/less/Makefile 27(+22 -5)

diff --git a/src/less/Makefile b/src/less/Makefile
index 48f04c6..052a96c 100644
--- a/src/less/Makefile
+++ b/src/less/Makefile
@@ -1,16 +1,33 @@
+LESSC = lessc
 OBJ_DIR = obj
 OBJ = \
 	$(OBJ_DIR)/azkaban.css \
-	$(OBJ_DIR)/azkaban-svg.css
+	$(OBJ_DIR)/azkaban-graph.css
 
 all: $(OBJ)
 
-$(OBJ_DIR)/%.css: %.less
-	lessc $< $@
+azkaban_css_DEPS = \
+	azkaban.less \
+	base.less \
+	context-menu.less \
+	flow.less \
+	header.less \
+	job.less \
+	login.less \
+	log.less \
+	navbar.less \
+	non-responsive.less \
+	off-canvas.less \
+	project.less \
+	tables.less
+
+$(OBJ_DIR)/azkaban.css: $(azkaban_css_DEPS)
+	$(LESSC) $< $@
+
+$(OBJ_DIR)/azkaban-graph.css: azkaban-graph.less
+	$(LESSC) $< $@
 
 clean:
 	rm -rf $(OBJ_DIR)
 
-.SUFFIXES: .less .css
-
 .PHONY: all clean

src/less/project.less 29(+19 -10)

diff --git a/src/less/project.less b/src/less/project.less
index d7e1fb9..34f3ddf 100644
--- a/src/less/project.less
+++ b/src/less/project.less
@@ -1,6 +1,6 @@
 #project-list {
   padding: 0;
-  margin: 0;
+  margin: 0px 0px 40px 0px;
 
   li {
     list-style: none;
@@ -40,16 +40,25 @@
       }
     }
   }
+}
 
-  .project-flows {
-    display: none;
-    background-color: #f9f9f9;
-    table {
-      background: transparent;
-      margin-bottom: 0;
-      border-top: 1px solid #dddddd;
-    }
-  }
+.project-flows {
+	display: none;
+	background-color: #f9f9f9;
+	padding: 10px 15px 10px 15px;
+
+	h5 {
+		margin-top: 5px;
+	}
+
+	.list-group {
+		margin-bottom: 10px;
+	}
+
+	.list-group-item {
+		background: transparent;
+		padding: 7px 12px 7px 12px;
+	}
 }
 
 // Flow panel heading.

src/less/tables.less 44(+34 -10)

diff --git a/src/less/tables.less b/src/less/tables.less
index d67061b..d96815f 100644
--- a/src/less/tables.less
+++ b/src/less/tables.less
@@ -53,7 +53,26 @@ table.table-properties {
 
 // Table of executions.
 .executions-table {
-  th {
+  tr {
+    &.expanded {
+      opacity: 0.6;
+    }
+  }
+
+  td {
+  	&.subflowrow {
+  		padding: 0px 0px;
+  	
+  		table {
+  			margin: 0px;
+  			background-color: rgba(230, 230, 230, 0.75);
+  			
+  			td {
+  				background-color: none;
+  			}
+  		}
+  	}
+
     &.date {
       width: 160px;
     }
@@ -74,7 +93,7 @@ table.table-properties {
       width: 90px;
     }
 
-    &.status {
+    &.statustd {
       width: 100px;
     }
 
@@ -89,19 +108,24 @@ table.table-properties {
     &.logs {
       width: 30px;
     }
-  }
-
-  td {
-    &.timeline {
+    
+     &.timeline {
       width: 280px;
       padding: 0px;
       height: 100%;
       vertical-align: bottom;
       margin: 0px;
     }
-
-    &.execId {
-      font-weight: bold;
-    }
+    
+    &.startTime {
+  		width: 160px;
+  	}
+  	
+  	&.endTime {
+  		width: 160px;
+  	}
+    &.elapsedTime {
+  		width: 90px;
+  	}
   }
 }
diff --git a/src/package/execserver/conf/azkaban.properties b/src/package/execserver/conf/azkaban.properties
index 3f9beda..585e208 100644
--- a/src/package/execserver/conf/azkaban.properties
+++ b/src/package/execserver/conf/azkaban.properties
@@ -8,7 +8,7 @@ azkaban.jobtype.plugin.dir=plugins/jobtypes
 executor.global.properties=conf/global.properties
 azkaban.project.dir=projects
 
-azkaban.stats.dir=
+azkaban.stats.dir=stats
 
 database.type=mysql
 mysql.port=3306
diff --git a/src/package/soloserver/conf/azkaban.properties b/src/package/soloserver/conf/azkaban.properties
index d600ed9..9ca591e 100644
--- a/src/package/soloserver/conf/azkaban.properties
+++ b/src/package/soloserver/conf/azkaban.properties
@@ -21,7 +21,7 @@ h2.path=data/azkaban
 h2.create.tables=true
 
 # Stats
-azkaban.stats.dir=
+azkaban.stats.dir=stats
 
 # Velocity dev mode
 velocity.dev.mode=false
diff --git a/src/package/webserver/conf/azkaban.properties b/src/package/webserver/conf/azkaban.properties
index 3fe43a0..97e40ae 100644
--- a/src/package/webserver/conf/azkaban.properties
+++ b/src/package/webserver/conf/azkaban.properties
@@ -22,7 +22,7 @@ mysql.user=azkaban
 mysql.password=azkaban
 mysql.numconnections=100
 
-azkaban.stats.dir=
+azkaban.stats.dir=stats
 
 # Velocity dev mode
 velocity.dev.mode=false
diff --git a/src/tl/flowsummary.tl b/src/tl/flowsummary.tl
index 1c9154f..30c50de 100644
--- a/src/tl/flowsummary.tl
+++ b/src/tl/flowsummary.tl
@@ -14,7 +14,6 @@
           </table>
         </div>
       </div>
-      <hr>
 			
       <div class="row">
         <div class="col-xs-12">

src/tl/Makefile 3(+2 -1)

diff --git a/src/tl/Makefile b/src/tl/Makefile
index a9c8028..1670e6a 100644
--- a/src/tl/Makefile
+++ b/src/tl/Makefile
@@ -1,3 +1,4 @@
+DUSTC = dustc
 OBJ_DIR = obj
 OBJ = \
 	$(OBJ_DIR)/flowsummary.js \
@@ -7,7 +8,7 @@ OBJ = \
 all: $(OBJ)
 
 $(OBJ_DIR)/%.js: %.tl
-	mkdir -p $(OBJ_DIR) && dustc --name=$(basename $<) $< $@
+	mkdir -p $(OBJ_DIR) && $(DUSTC) --name=$(basename $<) $< $@
 
 clean:
 	rm -rf $(OBJ_DIR)
diff --git a/src/web/css/morris.css b/src/web/css/morris.css
new file mode 100644
index 0000000..5b313c2
--- /dev/null
+++ b/src/web/css/morris.css
@@ -0,0 +1,25 @@
+.morris-hover {
+  position: absolute;
+  z-index: 1000;
+}
+
+.morris-hover.morris-default-style {
+  border-radius: 10px;
+  padding: 6px;
+  color: #666;
+  background: rgba(255, 255, 255, 0.8);
+  border: solid 2px rgba(230, 230, 230, 0.8);
+  font-family: sans-serif;
+  font-size: 12px;
+  text-align: center;
+}
+
+.morris-hover.morris-default-style .morris-hover-row-label {
+  font-weight: bold;
+  margin: 0.25em 0;
+}
+
+.morris-hover.morris-default-style .morris-hover-point {
+  white-space: nowrap;
+  margin: 0.1em 0;
+}
diff --git a/src/web/images/graph-icon.png b/src/web/images/graph-icon.png
new file mode 100644
index 0000000..d315d97
Binary files /dev/null and b/src/web/images/graph-icon.png differ
diff --git a/src/web/js/azkaban.flow.extended.view.js b/src/web/js/azkaban.flow.extended.view.js
new file mode 100644
index 0000000..aa38d34
--- /dev/null
+++ b/src/web/js/azkaban.flow.extended.view.js
@@ -0,0 +1,62 @@
+azkaban.FlowExtendedViewPanel = Backbone.View.extend({
+	events: {
+		"click .closeInfoPanel" : "handleClosePanel" 
+	},
+	initialize: function(settings) {
+		//this.model.bind('change:flowinfo', this.changeFlowInfo, this);
+		$(this.el).show();
+		$(this.el).draggable({cancel: ".dataContent", containment: "document"});
+
+		this.render();
+		$(this.el).hide();
+	},
+	showExtendedView: function(evt) {
+		var event = evt;
+		
+		$(this.el).css({top: evt.pageY, left: evt.pageX});
+		$(this.el).show();
+	},
+	render: function(self) {
+		console.log("Changing title");
+		$(this.el).find(".nodeId").text(this.model.get("id"));
+		$(this.el).find(".nodeType").text(this.model.get("type"));
+		
+		var props = this.model.get("props");
+		var tableBody = $(this.el).find(".dataPropertiesBody");
+		
+		for (var key in props) {
+			var tr = document.createElement("tr");
+			var tdKey = document.createElement("td");
+			var tdValue = document.createElement("td");
+			
+			$(tdKey).text(key);
+			$(tdValue).text(props[key]);
+			
+			$(tr).append(tdKey);
+			$(tr).append(tdValue);
+			
+			$(tableBody).append(tr);
+			
+			var propsTable = $(this.el).find(".dataJobProperties");
+			$(propsTable).resizable({handler: "s"});
+		}
+		
+		if (this.model.get("type") == "flow") {
+			var svgns = "http://www.w3.org/2000/svg";
+			var svgDataFlow = $(this.el).find(".dataFlow");
+			
+			var svgGraph = document.createElementNS(svgns, "svg");
+			$(svgGraph).attr("class", "svgTiny");
+			$(svgDataFlow).append(svgGraph);
+			$(svgDataFlow).resizable();
+			
+			this.graphView = new azkaban.SvgGraphView({el: svgDataFlow, model: this.model, render: true, rightClick:  { "node": nodeClickCallback, "graph": graphClickCallback }})
+		}
+		else {
+			$(this.el).find(".dataFlow").hide();
+		}
+	},
+	handleClosePanel: function(self) {
+		$(this.el).hide();
+	}
+});
\ No newline at end of file
diff --git a/src/web/js/azkaban/model/log-data.js b/src/web/js/azkaban/model/log-data.js
index 2e21386..2fe3ac5 100644
--- a/src/web/js/azkaban/model/log-data.js
+++ b/src/web/js/azkaban/model/log-data.js
@@ -22,7 +22,7 @@ azkaban.LogDataModel = Backbone.Model.extend({
     HIVE_NUM_MAP_REDUCE_JOBS_STRING: "Total MapReduce jobs = ",
     HIVE_MAP_REDUCE_JOB_START: "Starting Job",
     HIVE_MAP_REDUCE_JOBS_SUMMARY: "MapReduce Jobs Launched:",
-    HIVE_MAP_REDUCE_SUMMARY_REGEX: /Job (\d+): Map: (\d+)  Reduce: (\d+)   HDFS Read: (\d+) HDFS Write: (\d+)/,
+    HIVE_MAP_REDUCE_SUMMARY_REGEX: /Job (\d+):\s+Map: (\d+)\s+Reduce: (\d+)\s+(?:Cumulative CPU: (.+?))?\s+HDFS Read: (\d+)\s+HDFS Write: (\d+)/,
 
     initialize: function() {
         this.set("offset", 0 );
@@ -232,7 +232,7 @@ azkaban.LogDataModel = Backbone.Model.extend({
             var numMRJobs = 0;
             while (i < numLines) {
                 line = lines[i];
-                if (line.contains(this.HIVE_NUM_MAP_REDUCE_JOBS_STRING)) {
+                if (line.indexOf(this.HIVE_NUM_MAP_REDUCE_JOBS_STRING) !== -1) {
                     // query involves map reduce jobs
                     var numMRJobs = parseInt(line.substring(this.HIVE_NUM_MAP_REDUCE_JOBS_STRING.length),10);
                     i++;
@@ -240,7 +240,7 @@ azkaban.LogDataModel = Backbone.Model.extend({
                     // get the map reduce jobs summary
                     while (i < numLines) {
                         line = lines[i];
-                        if (line.contains(this.HIVE_MAP_REDUCE_JOBS_SUMMARY)) {
+                        if (line.indexOf(this.HIVE_MAP_REDUCE_JOBS_SUMMARY) !== -1) {
                             // job summary table found
                             i++;
                             
@@ -264,6 +264,13 @@ azkaban.LogDataModel = Backbone.Model.extend({
                                     job.push(match[3]);
                                     job.push(match[4]);
                                     job.push(match[5]);
+                                    job.push(match[6]);
+
+                                    if (match[7]) {
+                                        this.set("hasCumulativeCPU", true);
+                                        job.push(match[7]);
+                                    }
+
                                     queryJobs.push(job);
                                     previousJob = currJob;
                                     numJobsSeen++;
@@ -281,7 +288,7 @@ azkaban.LogDataModel = Backbone.Model.extend({
                     } 
                     break;
                 }
-                else if (line.contains(this.HIVE_PARSING_START)) {
+                else if (line.indexOf(this.HIVE_PARSING_START) !== -1) {
                     if (numMRJobs === 0) {
                         hiveQueryJobs.push(null);
                     }
diff --git a/src/web/js/azkaban/util/flow-loader.js b/src/web/js/azkaban/util/flow-loader.js
new file mode 100644
index 0000000..ba193f0
--- /dev/null
+++ b/src/web/js/azkaban/util/flow-loader.js
@@ -0,0 +1,217 @@
+var statusStringMap = {
+	"FAILED": "Failed",
+	"SUCCEEDED": "Success",
+	"FAILED_FINISHING": "Running w/Failure",
+	"RUNNING": "Running",
+	"WAITING": "Waiting",
+	"KILLED": "Killed",
+	"DISABLED": "Disabled",
+	"READY": "Ready",
+	"UNKNOWN": "Unknown",
+	"QUEUED": "Queued",
+	"SKIPPED": "Skipped"
+};
+
+var extendedViewPanels = {};
+var extendedDataModels = {};
+var openJobDisplayCallback = function(nodeId, flowId, evt) {
+	console.log("Open up data");
+
+	/*
+	$("#flowInfoBase").before(cloneStuff);
+	var requestURL = contextURL + "/manager";
+	
+	$.get(
+      requestURL,
+      {"project": projectName, "ajax":"fetchflownodedata", "flow":flowId, "node": nodeId},
+      function(data) {
+  		var graphModel = new azkaban.GraphModel();
+  		graphModel.set({id: data.id, flow: data.flowData, type: data.type, props: data.props});
+
+  		var flowData = data.flowData;
+  		if (flowData) {
+  			createModelFromAjaxCall(flowData, graphModel);
+  		}
+  		
+  		var backboneView = new azkaban.FlowExtendedViewPanel({el:cloneStuff, model: graphModel});
+  		extendedViewPanels[nodeInfoPanelID] = backboneView;
+  		extendedDataModels[nodeInfoPanelID] = graphModel;
+  		backboneView.showExtendedView(evt);
+      },
+      "json"
+    );
+    */
+}
+
+var createNewPanel = function(node, model, evt) {
+	var parentPath = node.parentPath;
+	
+	var nodeInfoPanelID = parentPath ? parentPath + ":" + node.id + "-info" : node.id + "-info";
+	var cloneStuff = $("#flowInfoBase").clone();
+	cloneStuff.data = node;
+	$(cloneStuff).attr("id", nodeInfoPanelID);
+	$("#flowInfoBase").before(cloneStuff);
+	
+	var backboneView = new azkaban.FlowExtendedViewPanel({el:cloneStuff, model: model});
+	node.panel = backboneView;
+	backboneView.showExtendedView(evt);
+}
+
+/**
+ * Processes the flow data from Javascript
+ */
+var processFlowData = function(data) {
+	var nodes = {};
+	var edges = new Array();
+
+	//Create a node map
+	for (var i=0; i < data.nodes.length; ++i) {
+		var node = data.nodes[i];
+		nodes[node.id] = node;
+		if (!node.status) {
+			node.status = "READY";
+		}
+	}
+
+	// Create each node in and out nodes. Create an edge list.
+	for (var i=0; i < data.nodes.length; ++i) {
+		var node = data.nodes[i];
+		if (node.in) {
+			for (var j=0; j < node.in.length; ++j) {
+				var fromNode = nodes[node.in[j]];
+				if (!fromNode.outNodes) {
+					fromNode.outNodes = {};
+				}
+				if (!node.inNodes) {
+					node.inNodes = {};
+				}
+				
+				fromNode.outNodes[node.id] = node;
+				node.inNodes[fromNode.id] = fromNode;
+				edges.push({to: node.id, from: fromNode.id});
+			}
+		}
+	}
+
+	// Iterate over the nodes again. Parse the data if they're embedded flow data.
+	// Assign each nodes to the parent flow data.
+	for (var key in nodes) {
+		var node = nodes[key];
+		node.parent = data;
+		if (node.type == "flow") {
+			processFlowData(node);
+		}
+	}
+	
+	// Assign the node map and the edge list
+	data.nodeMap = nodes;
+	data.edges = edges;
+}
+
+var closeAllSubDisplays = function() {
+	$(".flowExtendedView").hide();
+}
+
+var nodeClickCallback = function(event, model, node) {
+	console.log("Node clicked callback");
+
+	var target = event.currentTarget;
+	var type = node.type;
+	var flowId = node.parent.flow;
+	var jobId = node.id;
+	
+	var requestURL = contextURL + "/manager?project=" + projectName + "&flow=" + flowId + "&job=" + jobId;
+	var menu = [];
+
+	if (type == "flow") {
+		var flowRequestURL = contextURL + "/manager?project=" + projectName + "&flow=" + node.flowId;
+		if (node.expanded) {
+			menu = [{title: "Collapse Flow...", callback: function() {model.trigger("collapseFlow", node);}}];
+		}
+		else {
+			menu = [{title: "Expand Flow...", callback: function() {model.trigger("expandFlow", node);}}];
+		}
+
+		$.merge(menu, [
+				{title: "View Properties...", callback: function() {openJobDisplayCallback(jobId, flowId, event)}},
+				{break: 1},
+				{title: "Open Flow...", callback: function() {window.location.href=flowRequestURL;}},
+				{title: "Open Flow in New Window...", callback: function() {window.open(flowRequestURL);}},
+				{break: 1},
+				{title: "Open Properties...", callback: function() {window.location.href=requestURL;}},
+				{title: "Open Properties in New Window...", callback: function() {window.open(requestURL);}},
+				{break: 1},
+				{title: "Center Flow", callback: function() {model.trigger("centerNode", node);}}
+		]);
+	}
+	else {
+		menu = [
+				{title: "View Properties...", callback: function() {openJobDisplayCallback(jobId, flowId, event)}},
+				{break: 1},
+				{title: "Open Job...", callback: function() {window.location.href=requestURL;}},
+				{title: "Open Job in New Window...", callback: function() {window.open(requestURL);}},
+				{break: 1},
+				{title: "Center Job", callback: function() {model.trigger("centerNode", node)}}
+		];
+	}
+	contextMenuView.show(event, menu);
+}
+
+var jobClickCallback = function(event, model, node) {
+	console.log("Node clicked callback");
+	var target = event.currentTarget;
+	var type = node.type;
+	var flowId = node.parent.flow;
+	var jobId = node.id;
+
+	var requestURL = contextURL + "/manager?project=" + projectName + "&flow=" + flowId + "&job=" + node.id;
+
+	var menu;
+	if (type == "flow") {
+		var flowRequestURL = contextURL + "/manager?project=" + projectName + "&flow=" + node.flowId;
+		menu = [
+				{title: "View Properties...", callback: function() {openJobDisplayCallback(jobId, flowId, event)}},
+				{break: 1},
+				{title: "Open Flow...", callback: function() {window.location.href=flowRequestURL;}},
+				{title: "Open Flow in New Window...", callback: function() {window.open(flowRequestURL);}},
+				{break: 1},
+				{title: "Open Properties...", callback: function() {window.location.href=requestURL;}},
+				{title: "Open Properties in New Window...", callback: function() {window.open(requestURL);}},
+				{break: 1},
+				{title: "Center Flow", callback: function() {model.trigger("centerNode", node)}}
+		];
+	}
+	else {
+		menu = [
+				{title: "View Job...", callback: function() {openJobDisplayCallback(jobId, flowId, event)}},
+				{break: 1},
+				{title: "Open Job...", callback: function() {window.location.href=requestURL;}},
+				{title: "Open Job in New Window...", callback: function() {window.open(requestURL);}},
+				{break: 1},
+				{title: "Center Job", callback: function() {graphModel.trigger("centerNode", node)}}
+		];
+	}
+	contextMenuView.show(event, menu);
+}
+
+var edgeClickCallback = function(event, model) {
+	console.log("Edge clicked callback");
+}
+
+var graphClickCallback = function(event, model) {
+	console.log("Graph clicked callback");
+
+	var jobId = event.currentTarget.jobid;
+	var flowId = model.get("flowId");
+	var requestURL = contextURL + "/manager?project=" + projectName + "&flow=" + flowId;
+
+	var menu = [	
+		{title: "Open Flow...", callback: function() {window.location.href=requestURL;}},
+		{title: "Open Flow in New Window...", callback: function() {window.open(requestURL);}},
+		{break: 1},
+		{title: "Center Graph", callback: function() {model.trigger("resetPanZoom");}}
+	];
+	
+	contextMenuView.show(event, menu);
+}
+
diff --git a/src/web/js/azkaban/util/layout.js b/src/web/js/azkaban/util/layout.js
index 6f6c2c5..96cdd21 100644
--- a/src/web/js/azkaban/util/layout.js
+++ b/src/web/js/azkaban/util/layout.js
@@ -20,36 +20,65 @@ var degreeRatio = 1/8;
 var maxHeight = 200;
 var cornerGap = 10;
 
-function layoutGraph(nodes, edges) {
-	var startLayer = [];
-	var numLayer = 0;
-	var nodeMap = {};
-	
+var idSort = function(a, b) {
+	if ( a.id < b.id ) {
+		return -1;
+	}
+	else if ( a.id > b.id ) {
+		return 1;
+	}
+	else {
+		return 0;
+	}
+}
+
+function prepareLayout(nodes, hmargin, layers, nodeMap) {
 	var maxLayer = 0;
-	var layers = {};
+	var nodeQueue = new Array();
+	// Find start layers first
+	for (var i=0; i < nodes.length; ++i) {
+		var node = nodes[i];
+		if (node.inNodes) {
+			// We sort here. Why? To keep the node drawing consistent
+			node.in.sort(idSort);
+		}
+		else {
+			// We sort here. Why? To keep it up and running.
+			nodeQueue.push(node);
+		}
+	}
+	// Sort here. To keep the node drawing consistent
+	nodes.sort(idSort);
 	
-	// Assign to layers
-	for (var i = 0; i < nodes.length; ++i) {
-		numLayer = Math.max(numLayer, nodes[i].level);
-		/*
-		if (nodes[i].id.length > maxTextSize) {
-			var label = nodes[i].id.substr(0, reductionSize) + "...";
-			nodes[i].label = label;
+	// calculate level
+	// breath first search the sucker
+	var index = 0;
+	while(index < nodeQueue.length) {
+		var node = nodeQueue[index];
+		if (node.inNodes) {
+			var level = 0;
+			for (var key in node.inNodes) {
+				level = Math.max(level, node.inNodes[key].level);
+			}
+			node.level = level + 1;
+		}
+		else {
+			node.level = 0;
 		}
-		else {*/
-			nodes[i].label = nodes[i].id;
-		//}
 		
-		var width = nodes[i].label.length * 10;
-		var node = {
-			id: nodes[i].id, 
-			node: nodes[i], 
-			level: nodes[i].level, 
-			in: [], 
-			out: [], 
-			width: width, 
-			x: 0 
-		};
+		if (node.outNodes) {
+			for (var key in node.outNodes) {
+				nodeQueue.push(node.outNodes[key]);
+			}
+		}
+		index++;
+	}
+	
+	// Assign to layers
+	for (var i = 0; i < nodes.length; ++i) {
+		var width = nodes[i].width ? nodes[i].width : nodes[i].label.length * 11.5 + 4;
+		var height = nodes[i].height ? nodes[i].height : 1;
+		var node = { id: nodes[i].id, node: nodes[i], level: nodes[i].level, in:[], out:[], width: width + hmargin, x:0, height:height };
 		nodeMap[nodes[i].id] = node;
 		maxLayer = Math.max(node.level, maxLayer);
 		if(!layers[node.level]) {
@@ -59,13 +88,32 @@ function layoutGraph(nodes, edges) {
 		layers[node.level].push(node);
 	}
 	
+	layers.maxLayer = maxLayer;
+}
+
+function respaceGraph(nodes, edges) {
+	
+}
+
+function layoutGraph(nodes, edges, hmargin) {
+	var startLayer = [];
+
+	var nodeMap = {};
+	var layers = {};
+	
+	if (!hmargin) {
+		hmargin = 8;
+	}
+	
+	prepareLayout(nodes, hmargin, layers, nodeMap);
+	var maxLayer = layers.maxLayer;
+	
 	// Create dummy nodes
 	var edgeDummies = {};
-	
 	for (var i=0; i < edges.length; ++i ) {
 		var edge = edges[i];
 		var src = edges[i].from;
-		var dest = edges[i].target;
+		var dest = edges[i].to;
 		
 		var edgeId = src + ">>" + dest;
 		
@@ -77,15 +125,7 @@ function layoutGraph(nodes, edges) {
 		var guides = [];
 		
 		for (var j = srcNode.level + 1; j < destNode.level; ++j) {
-			var dummyNode = {
-				level: j, 
-				in: [], 
-				x: lastNode.x, 
-				out: [], 
-				realSrc: srcNode, 
-				realDest: destNode, 
-				width: 10
-			};
+			var dummyNode = {level: j, in: [], x: lastNode.x, out: [], realSrc: srcNode, realDest: destNode, width: 10, height: 10};
 			layers[j].push(dummyNode);
 			dummyNode.in.push(lastNode);
 			lastNode.out.push(dummyNode);
@@ -140,12 +180,12 @@ function layoutGraph(nodes, edges) {
 		node.x = layerNode.x;
 		node.y = layerNode.y;
 	}
-	
+
 	// Dummy node for more points.
 	for (var i = 0; i < edges.length; ++i) {
 		var edge = edges[i];
 		var src = edges[i].from;
-		var dest = edges[i].target;
+		var dest = edges[i].to;
 		
 		var edgeId = src + ">>" + dest;
 		if (edgeDummies[edgeId] && edgeDummies[edgeId].length > 0) {
@@ -280,16 +320,22 @@ function spreadLayerSmart(layer) {
 function spaceVertically(layers, maxLayer) {
 	var startY = 0;
 	var startLayer = layers[0];
-	for (var i = 0; i < startLayer.length; ++i) {
+	var startMaxHeight = 1;
+	for (var i=0; i < startLayer.length; ++i) {
 		startLayer[i].y = startY;
+		startMaxHeight = Math.max(startMaxHeight, startLayer[i].height);
 	}
 	
-	var minHeight = 50;
-	for (var a = 1; a <= maxLayer; ++a) {
+	var minHeight = 40;
+	for (var a=1; a <= maxLayer; ++a) {
 		var maxDelta = 0;
 		var layer = layers[a];
-		for (var i = 0; i < layer.length; ++i) {
-			for (var j = 0; j < layer[i].in.length; ++j) {
+		
+		var layerMaxHeight = 1;
+		for (var i=0; i < layer.length; ++i) {
+			layerMaxHeight = Math.max(layerMaxHeight, layer[i].height);
+
+			for (var j=0; j < layer[i].in.length; ++j) {
 				var upper = layer[i].in[j];
 				var delta = Math.abs(upper.x - layer[i].x);
 				maxDelta = Math.max(maxDelta, delta);
@@ -299,10 +345,12 @@ function spaceVertically(layers, maxLayer) {
 		console.log("Max " + maxDelta);
 		var calcHeight = maxDelta*degreeRatio;
 		
-		calcHeight = Math.min(calcHeight, maxHeight); 
-		startY += Math.max(calcHeight, minHeight);
-		for (var i = 0; i < layer.length; ++i) {
-			layer[i].y = startY;
+		var newMinHeight = minHeight + startMaxHeight/2 + layerMaxHeight / 2;
+		startMaxHeight = layerMaxHeight;
+
+		startY += Math.max(calcHeight, newMinHeight);
+		for (var i=0; i < layer.length; ++i) {
+			layer[i].y=startY;
 		}
 	}
 }
diff --git a/src/web/js/azkaban/util/svg-navigate.js b/src/web/js/azkaban/util/svg-navigate.js
index b42f8c1..6fcb7cd 100644
--- a/src/web/js/azkaban/util/svg-navigate.js
+++ b/src/web/js/azkaban/util/svg-navigate.js
@@ -332,8 +332,19 @@
 			var aspectRatioDiv = divHeight/divWidth;
 
 			var scale = aspectRatioGraph > aspectRatioDiv ? (divHeight/height)*factor : (divWidth/width)*factor;
-			console.log("(" + x + "," + y + "," + width.toPrecision(4) + "," + height.toPrecision(4) + ")");
-			console.log("(rg:" + aspectRatioGraph.toPrecision(3) + ",rd:" + aspectRatioDiv.toPrecision(3) + "," + scale.toPrecision(3) + ")");
+			//console.log("(" + x + "," + y + "," + width.toPrecision(4) + "," + height.toPrecision(4) + ")");
+			//console.log("(rg:" + aspectRatioGraph.toPrecision(3) + ",rd:" + aspectRatioDiv.toPrecision(3) + "," + scale.toPrecision(3) + ")");
+			
+			if (arguments.maxScale) {
+				if (scale > arguments.maxScale) {
+					scale = arguments.maxScale;
+				}
+			}
+			if (arguments.minScale) {
+				if (scale < arguments.minScale) {
+					scale = arguments.minScale;
+				}
+			}
 			
 			// Center
 			var scaledWidth = width*scale;
diff --git a/src/web/js/azkaban/util/svgutils.js b/src/web/js/azkaban/util/svgutils.js
new file mode 100644
index 0000000..33dea78
--- /dev/null
+++ b/src/web/js/azkaban/util/svgutils.js
@@ -0,0 +1,70 @@
+function hasClass(el, name) 
+{
+	var classes = el.getAttribute("class");
+	if (classes == null) {
+		return false;
+	}
+   return new RegExp('(\\s|^)'+name+'(\\s|$)').test(classes);
+}
+
+function addClass(el, name)
+{
+   if (!hasClass(el, name)) { 
+   		var classes = el.getAttribute("class");
+   		if (classes) {
+   			classes += ' ' + name;
+   		}
+   		else {
+   			classes = name;
+   		}
+   		el.setAttribute("class", classes);
+   }
+}
+
+function removeClass(el, name)
+{
+   if (hasClass(el, name)) {
+      var classes = el.getAttribute("class");
+      el.setAttribute("class", classes.replace(new RegExp('(\\s|^)'+name+'(\\s|$)'),' ').replace(/^\s+|\s+$/g, ''));
+   }
+}
+
+function translateStr(x, y) {
+	return "translate(" + x + "," + y + ")"; 
+}
+
+function animatePolylineEdge(svg, edge, newPoints, time) {
+	var oldEdgeGuides = edge.oldpoints;
+	
+	var interval = 10;
+	var numsteps = time/interval;
+	
+	var deltaEdges = new Array();
+ 	for (var i=0; i < oldEdgeGuides.length; ++i) {
+		var startPoint = oldEdgeGuides[i];
+		var endPoint = newPoints[i];
+		
+		var deltaX = (endPoint[0] - startPoint[0])/numsteps;
+		var deltaY = (endPoint[1] - startPoint[1])/numsteps;
+		deltaEdges.push([deltaX, deltaY]);
+	}
+	
+	animatePolyLineLoop(svg, edge, oldEdgeGuides, deltaEdges, numsteps, 25);
+}
+
+function animatePolyLineLoop(svg, edge, lastPoints, deltaEdges, step, time) {
+	for (var i=0; i < deltaEdges.length; ++i) {
+		lastPoints[i][0] += deltaEdges[i][0];
+		lastPoints[i][1] += deltaEdges[i][1];
+	}
+	
+	svg.change(edge.line, {points: lastPoints});
+	if (step > 0) {
+		setTimeout(
+			function(){
+				animatePolyLineLoop(svg, edge, lastPoints, deltaEdges, step - 1);
+			}, 
+			time
+		);
+	}
+}
\ No newline at end of file
diff --git a/src/web/js/azkaban/view/exflow.js b/src/web/js/azkaban/view/exflow.js
index a35ab7f..eee5a1a 100644
--- a/src/web/js/azkaban/view/exflow.js
+++ b/src/web/js/azkaban/view/exflow.js
@@ -56,7 +56,7 @@ azkaban.StatusView = Backbone.View.extend({
 		var startTime = data.startTime;
 		var endTime = data.endTime;
 		
-		if (startTime == -1) {
+		if (!startTime || startTime == -1) {
 			$("#startTime").text("-");
 		}
 		else {
@@ -73,7 +73,7 @@ azkaban.StatusView = Backbone.View.extend({
 			$("#duration").text(durationString);
 		}
 		
-		if (endTime == -1) {
+		if (!endTime || endTime == -1) {
 			$("#endTime").text("-");
 		}
 		else {
@@ -238,14 +238,15 @@ azkaban.FlowTabView = Backbone.View.extend({
 	},
 	
 	handleRestartClick: function(evt) {
-    console.log("handleRestartClick");
+		console.log("handleRestartClick");
 		var data = graphModel.get("data");
-		var nodes = data.nodes;
+		
 		var executingData = {
 			project: projectName,
 			ajax: "executeFlow",
 			flow: flowId,
-			execid: execId
+			execid: execId,
+			exgraph: data
 		};
 		flowExecuteDialogView.show(executingData);
 	},
@@ -292,251 +293,6 @@ var showDialog = function(title, message) {
 var jobListView;
 var mainSvgGraphView;
 
-var executionListView;
-azkaban.ExecutionListView = Backbone.View.extend({
-	events: {
-		//"click .flow-progress-bar": "handleProgressBoxClick"
-	},
-	
-	initialize: function(settings) {
-		this.model.bind('change:graph', this.renderJobs, this);
-		this.model.bind('change:update', this.updateJobs, this);
-	},
-	
-	renderJobs: function(evt) {
-		var data = this.model.get("data");
-		var lastTime = data.endTime == -1 ? (new Date()).getTime() : data.endTime;
-		this.updateJobRow(data.nodes);
-		this.updateProgressBar(data);
-	},
-
-	/*handleProgressBoxClick: function(evt) {
-		var target = evt.currentTarget;
-		var job = target.job;
-		var attempt = target.attempt;
-		
-		var data = this.model.get("data");
-		var node = data.nodes[job];
-		
-		var jobId = event.currentTarget.jobid;
-		var requestURL = contextURL + "/manager?project=" + projectName + "&execid=" + execId + "&job=" + job + "&attempt=" + attempt;
-	
-		var menu = [	
-				{title: "Open Job...", callback: function() {window.location.href=requestURL;}},
-				{title: "Open Job in New Window...", callback: function() {window.open(requestURL);}}
-		];
-	
-		contextMenuView.show(evt, menu);
-	},*/
-	
-	updateJobs: function(evt) {
-		var data = this.model.get("update");
-		var lastTime = data.endTime == -1 ? (new Date()).getTime() : data.endTime;
-		
-		this.updateJobRow(data.nodes);
-		this.updateProgressBar(this.model.get("data"));
-	},
-	
-	updateJobRow: function(nodes) {
-		var executingBody = $("#executableBody");
-		nodes.sort(function(a,b) { return a.startTime - b.startTime; });
-		
-		for (var i = 0; i < nodes.length; ++i) {
-			var node = nodes[i];
-			if (node.startTime < 0) {
-        continue;
-      }
-      var nodeId = node.id.replace(".", "\\\\.");
-      var row = document.getElementById(nodeId + "-row");
-      if (!row) {
-        this.addNodeRow(node);
-      }
-      
-      var div = $("#" + nodeId + "-status-div");
-      div.text(statusStringMap[node.status]);
-      $(div).attr("class", "status " + node.status);
-      
-      var startdate = new Date(node.startTime);
-      $("#" + nodeId + "-start").text(getDateFormat(startdate));
-      
-      var endTime = node.endTime;
-      if (node.endTime == -1) {
-        $("#" + nodeId + "-end").text("-");
-        endTime = node.startTime + 1;
-      }
-      else {
-        var enddate = new Date(node.endTime);
-        $("#" + nodeId + "-end").text(getDateFormat(enddate));
-      }
-      
-      var progressBar = $("#" + nodeId + "-progressbar");
-      if (!progressBar.hasClass(node.status)) {
-        for (var j = 0; j < statusList.length; ++j) {
-          var status = statusList[j];
-          progressBar.removeClass(status);
-        }
-        progressBar.addClass(node.status);
-      }
-      
-      // Create past attempts
-      if (node.pastAttempts) {
-        for (var a = 0; a < node.pastAttempts.length; ++a) {
-          var attemptBarId = nodeId + "-progressbar-" + a;
-          var attempt = node.pastAttempts[a];
-          if ($("#" + attemptBarId).length == 0) {
-            var attemptBox = document.createElement("div");
-            $(attemptBox).attr("id", attemptBarId);
-            $(attemptBox).addClass("flow-progress-bar");
-            $(attemptBox).addClass("attempt");
-            $(attemptBox).addClass(attempt.status);
-            $(attemptBox).css("float","left");
-            $(attemptBox).bind("contextmenu", attemptRightClick);
-            $(progressBar).before(attemptBox);
-            attemptBox.job = nodeId;
-            attemptBox.attempt = a;
-          }
-        }
-      }
-      
-      if (node.endTime == -1) {
-        //$("#" + node.id + "-elapse").text("0 sec");
-        $("#" + nodeId + "-elapse").text(getDuration(node.startTime, (new Date()).getTime()));					
-      }
-      else {
-        $("#" + nodeId + "-elapse").text(getDuration(node.startTime, node.endTime));
-      }
-		}
-	},
-	
-	updateProgressBar: function(data) {
-		if (data.startTime == -1) {
-			return;
-		}
-		
-		var flowLastTime = data.endTime == -1 ? (new Date()).getTime() : data.endTime;
-		var flowStartTime = data.startTime;
-
-		var outerWidth = $(".flow-progress").css("width");
-		if (outerWidth) {
-			if (outerWidth.substring(outerWidth.length - 2, outerWidth.length) == "px") {
-				outerWidth = outerWidth.substring(0, outerWidth.length - 2);
-			}
-			outerWidth = parseInt(outerWidth);
-		}
-		
-		var nodes = data.nodes;
-		var diff = flowLastTime - flowStartTime;
-		var factor = outerWidth/diff;
-		for (var i = 0; i < nodes.length; ++i) {
-			var node = nodes[i];
-			var nodeId = node.id.replace(".", "\\\\.");
-			// calculate the progress
-
-			var elem = $("#" + node.id + "-progressbar");
-			var offsetLeft = 0;
-			var minOffset = 0;
-			elem.attempt = 0;
-			
-			// Add all the attempts
-			if (node.pastAttempts) {
-				var logURL = contextURL + "/executor?execid=" + execId + "&job=" + node.id + "&attempt=" +	node.pastAttempts.length;
-				var aId = node.id + "-log-link";
-				$("#" + aId).attr("href", logURL);
-				elem.attempt = node.pastAttempts.length;
-				
-				// Calculate the node attempt bars
-				for (var p = 0; p < node.pastAttempts.length; ++p) {
-					var pastAttempt = node.pastAttempts[p];
-					var pastAttemptBox = $("#" + nodeId + "-progressbar-" + p);
-					
-					var left = (pastAttempt.startTime - flowStartTime)*factor;
-					var width =	Math.max((pastAttempt.endTime - pastAttempt.startTime)*factor, 3);
-					
-					var margin = left - offsetLeft;
-					$(pastAttemptBox).css("margin-left", left - offsetLeft);
-					$(pastAttemptBox).css("width", width);
-					
-					$(pastAttemptBox).attr("title", "attempt:" + p + "	start:" + getHourMinSec(new Date(pastAttempt.startTime)) + "	end:" + getHourMinSec(new Date(pastAttempt.endTime)));
-					offsetLeft += width + margin;
-				}
-			}
-			
-			var nodeLastTime = node.endTime == -1 ? (new Date()).getTime() : node.endTime;
-			var left = Math.max((node.startTime-flowStartTime)*factor, minOffset);
-			var margin = left - offsetLeft;
-			var width = Math.max((nodeLastTime - node.startTime)*factor, 3);
-			width = Math.min(width, outerWidth);
-			
-			elem.css("margin-left", left)
-			elem.css("width", width);
-			elem.attr("title", "attempt:" + elem.attempt + "	start:" + getHourMinSec(new Date(node.startTime)) + "	end:" + getHourMinSec(new Date(node.endTime)));
-		}
-	},
-	
-	addNodeRow: function(node) {
-		var executingBody = $("#executableBody");
-		var tr = document.createElement("tr");
-		var tdName = document.createElement("td");
-		var tdTimeline = document.createElement("td");
-		var tdStart = document.createElement("td");
-		var tdEnd = document.createElement("td");
-		var tdElapse = document.createElement("td");
-		var tdStatus = document.createElement("td");
-		var tdDetails = document.createElement("td");
-		
-		$(tr).append(tdName);
-		$(tr).append(tdTimeline);
-		$(tr).append(tdStart);
-		$(tr).append(tdEnd);
-		$(tr).append(tdElapse);
-		$(tr).append(tdStatus);
-		$(tr).append(tdDetails);
-		$(tr).attr("id", node.id + "-row");
-		$(tdTimeline).attr("id", node.id + "-timeline");
-		$(tdStart).attr("id", node.id + "-start");
-		$(tdEnd).attr("id", node.id + "-end");
-		$(tdElapse).attr("id", node.id + "-elapse");
-		$(tdStatus).attr("id", node.id + "-status");
-
-		var outerProgressBar = document.createElement("div");
-		$(outerProgressBar).attr("id", node.id + "-outerprogressbar");
-		$(outerProgressBar).addClass("flow-progress");
-
-		var progressBox = document.createElement("div");
-		progressBox.job = node.id;
-		$(progressBox).attr("id", node.id + "-progressbar");
-		$(progressBox).addClass("flow-progress-bar");
-		$(outerProgressBar).append(progressBox);
-		$(tdTimeline).append(outerProgressBar);
-		$(tdTimeline).addClass("timeline");
-
-		var requestURL = contextURL + "/manager?project=" + projectName + "&job=" + node.id + "&history";
-		var a = document.createElement("a");
-		$(a).attr("href", requestURL);
-		$(a).text(node.id);
-		$(tdName).append(a);
-
-		var status = document.createElement("div");
-		$(status).addClass("status");
-		$(status).attr("id", node.id + "-status-div");
-		tdStatus.appendChild(status);
-
-		var logURL = contextURL + "/executor?execid=" + execId + "&job=" + node.id;
-		if (node.attempt) {
-			logURL += "&attempt=" + node.attempt;
-		}
-
-		var a = document.createElement("a");
-		$(a).attr("href", logURL);
-		$(a).attr("id", node.id + "-log-link");
-		$(a).text("Details");
-		$(tdDetails).addClass("details");
-		$(tdDetails).append(a);
-		
-		executingBody.append(tr);
-	}
-});
-
 var flowLogView;
 azkaban.FlowLogView = Backbone.View.extend({
 	events: {
@@ -626,44 +382,51 @@ azkaban.GraphModel = Backbone.Model.extend({});
 var logModel;
 azkaban.LogModel = Backbone.Model.extend({});
 
-var updateStatus = function() {
+var updateStatus = function(updateTime) {
 	var requestURL = contextURL + "/executor";
 	var oldData = graphModel.get("data");
 	var nodeMap = graphModel.get("nodeMap");
 	
+	if (!updateTime) {
+		updateTime = oldData.updateTime ? oldData.updateTime : 0;
+	}
+
 	var requestData = {
 		"execid": execId, 
 		"ajax": "fetchexecflowupdate", 
 		"lastUpdateTime": updateTime
 	};
-
+	
 	var successHandler = function(data) {
 		console.log("data updated");
-		updateTime = data.updateTime;
-		oldData.submitTime = data.submitTime;
-		oldData.startTime = data.startTime;
-		oldData.endTime = data.endTime;
-		oldData.status = data.status;
-		
-		for (var i = 0; i < data.nodes.length; ++i) {
-			var node = data.nodes[i];
-			var oldNode = nodeMap[node.id];
-			oldNode.startTime = node.startTime;
-			oldNode.updateTime = node.updateTime;
-			oldNode.endTime = node.endTime;
-			oldNode.status = node.status;
-			oldNode.attempt = node.attempt;
-			if (oldNode.attempt > 0) {
-				oldNode.pastAttempts = node.pastAttempts;
-			}
+		if (data.updateTime) {
+			updateGraph(oldData, data);
+	
+			graphModel.set({"update": data});
+			graphModel.trigger("change:update");
 		}
-
-		graphModel.set({"update": data});
-		graphModel.trigger("change:update");
 	};
 	ajaxCall(requestURL, requestData, successHandler);
 }
 
+var updateGraph = function(data, update) {
+	var nodeMap = data.nodeMap;
+	data.startTime = update.startTime;
+	data.endTime = update.endTime;
+	data.updateTime = update.updateTime;
+	data.status = update.status;
+	update.changedNode = data;
+	
+	if (update.nodes) {
+		for (var i = 0; i < update.nodes.length; ++i) {
+			var newNode = update.nodes[i];
+			var oldNode = nodeMap[newNode.id];
+			
+			updateGraph(oldNode, newNode);
+		}
+	}
+}
+
 var updateTime = -1;
 var updaterFunction = function() {
 	var oldData = graphModel.get("data");
@@ -687,6 +450,7 @@ var updaterFunction = function() {
 		}
 		else {
 			console.log("Flow finished, so no more updates");
+			setTimeout(function() {updateStatus(0);}, 500);
 		}
 	}
 	else {
@@ -758,23 +522,6 @@ var exGraphClickCallback = function(event) {
 	contextMenuView.show(event, menu);
 }
 
-var attemptRightClick = function(event) {
-	var target = event.currentTarget;
-	var job = target.job;
-	var attempt = target.attempt;
-	
-	var jobId = event.currentTarget.jobid;
-	var requestURL = contextURL + "/executor?project=" + projectName + "&execid=" + execId + "&job=" + job + "&attempt=" + attempt;
-
-	var menu = [	
-		{title: "Open Attempt Log...", callback: function() {window.location.href=requestURL;}},
-		{title: "Open Attempt Log in New Window...", callback: function() {window.open(requestURL);}}
-	];
-
-	contextMenuView.show(event, menu);
-	return false;
-}
-
 var flowStatsView;
 var flowStatsModel;
 
@@ -789,28 +536,28 @@ $(function() {
 		model: graphModel
 	});
 	
-  mainSvgGraphView = new azkaban.SvgGraphView({
+	mainSvgGraphView = new azkaban.SvgGraphView({
 		el: $('#svgDiv'), 
 		model: graphModel, 
 		rightClick:	{ 
-			"node": exNodeClickCallback, 
-			"edge": exEdgeClickCallback, 
-			"graph": exGraphClickCallback 
+			"node": nodeClickCallback, 
+			"edge": edgeClickCallback, 
+			"graph": graphClickCallback 
 		}
 	});
 	
-  jobsListView = new azkaban.JobListView({
+	jobsListView = new azkaban.JobListView({
 		el: $('#jobList'), 
 		model: graphModel, 
-		contextMenuCallback: exJobClickCallback
+		contextMenuCallback: jobClickCallback
 	});
 	
-  flowLogView = new azkaban.FlowLogView({
+	flowLogView = new azkaban.FlowLogView({
 		el: $('#flowLogView'), 
 		model: logModel
 	});
 	
-  statusView = new azkaban.StatusView({
+	statusView = new azkaban.StatusView({
 		el: $('#flow-status'), 
 		model: graphModel
 	});
@@ -835,38 +582,14 @@ $(function() {
 	var requestData = {"execid": execId, "ajax":"fetchexecflow"};
 	var successHandler = function(data) {
 		console.log("data fetched");
-		graphModel.set({data: data});
-		graphModel.set({disabled: {}});
+		processFlowData(data);
+		graphModel.set({data:data});
 		graphModel.trigger("change:graph");
 		
 		updateTime = Math.max(updateTime, data.submitTime);
 		updateTime = Math.max(updateTime, data.startTime);
 		updateTime = Math.max(updateTime, data.endTime);
 		
-		var nodeMap = {};
-		for (var i = 0; i < data.nodes.length; ++i) {
-			var node = data.nodes[i];
-			nodeMap[node.id] = node;
-			updateTime = Math.max(updateTime, node.startTime);
-			updateTime = Math.max(updateTime, node.endTime);
-		}
-		for (var i = 0; i < data.edges.length; ++i) {
-			var edge = data.edges[i];
-			 
-			if (!nodeMap[edge.target].in) {
-				nodeMap[edge.target].in = {};
-			}
-			var targetInMap = nodeMap[edge.target].in;
-			targetInMap[edge.from] = nodeMap[edge.from];
-			 
-			if (!nodeMap[edge.from].out) {
-				nodeMap[edge.from].out = {};
-			}
-			var sourceOutMap = nodeMap[edge.from].out;
-			sourceOutMap[edge.target] = nodeMap[edge.target];
-		}
-		
-		graphModel.set({nodeMap: nodeMap});
 		if (window.location.hash) {
 			var hash = window.location.hash;
 			if (hash == "#jobslist") {
@@ -875,9 +598,9 @@ $(function() {
 			else if (hash == "#log") {
 				flowTabView.handleLogLinkClick();
 			}
-      else if (hash == "#stats") {
-        flowTabView.handleStatsLinkClick();
-      }
+			else if (hash == "#stats") {
+				flowTabView.handleStatsLinkClick();
+			}
 		}
 		else {
 			flowTabView.handleGraphLinkClick();
diff --git a/src/web/js/azkaban/view/flow.js b/src/web/js/azkaban/view/flow.js
index ead141c..14ee620 100644
--- a/src/web/js/azkaban/view/flow.js
+++ b/src/web/js/azkaban/view/flow.js
@@ -16,19 +16,6 @@
 
 $.namespace('azkaban');
 
-var statusStringMap = {
-	"FAILED": "Failed",
-	"SUCCEEDED": "Success",
-	"FAILED_FINISHING": "Running w/Failure",
-	"RUNNING": "Running",
-	"WAITING": "Waiting",
-	"KILLED": "Killed",
-	"DISABLED": "Disabled",
-	"READY": "Ready",
-	"UNKNOWN": "Unknown",
-	"QUEUED": "Queued"
-};
-
 var handleJobMenuClick = function(action, el, pos) {
 	var jobid = el[0].jobid;
 	var requestURL = contextURL + "/manager?project=" + projectName + "&flow=" + 
@@ -303,26 +290,31 @@ azkaban.SummaryView = Backbone.View.extend({
 		this.model.bind('render', this.render, this);
 		
 		this.fetchDetails();
-    this.fetchSchedule();
+		this.fetchSchedule();
 		this.model.trigger('render');
 	},
 
-  fetchDetails: function() {
-    var requestURL = contextURL + "/manager";
-    var requestData = {
-      'ajax': 'fetchflowdetails',
-      'project': projectName,
-      'flow': flowId
-    };
+	fetchDetails: function() {
+		var requestURL = contextURL + "/manager";
+		var requestData = {
+			'ajax': 'fetchflowdetails',
+			'project': projectName,
+			'flow': flowId
+		};
+		
 		var model = this.model;
-    var successHandler = function(data) {
-      model.set({ 'jobTypes': data.jobTypes });
-      model.trigger('render');
-    };
-    $.get(requestURL, requestData, successHandler, 'json');
-  },
 
-	fetchSchedule: function() {
+		var successHandler = function(data) {
+			console.log(data);
+			model.set({
+				'jobTypes': data.jobTypes
+			});
+			model.trigger('render');
+		};
+		$.get(requestURL, requestData, successHandler, 'json');
+	},
+	
+  fetchSchedule: function() {
 		var requestURL = contextURL + "/schedule"
 		var requestData = {
 			'ajax': 'fetchSchedule',
@@ -331,8 +323,8 @@ azkaban.SummaryView = Backbone.View.extend({
 		};
 		var model = this.model;
 		var successHandler = function(data) {
-      model.set({'schedule': data.schedule});
-      model.trigger('render');
+			model.set({'schedule': data.schedule});
+			model.trigger('render');
 		};
 		$.get(requestURL, requestData, successHandler, 'json');
 	},
@@ -362,9 +354,9 @@ azkaban.SummaryView = Backbone.View.extend({
 
 	render: function(evt) {
 		var data = {
-      projectName: projectName,
+			projectName: projectName,
 			flowName: flowId,
-      jobTypes: this.model.get('jobTypes'),
+      		jobTypes: this.model.get('jobTypes'),
 			schedule: this.model.get('schedule'),
 		};
 		dust.render("flowsummary", data, function(err, out) {
@@ -373,52 +365,6 @@ azkaban.SummaryView = Backbone.View.extend({
 	},
 });
 
-var exNodeClickCallback = function(event) {
-	console.log("Node clicked callback");
-	var jobId = event.currentTarget.jobid;
-	var requestURL = contextURL + "/manager?project=" + projectName + "&flow=" + 
-			flowId + "&job=" + jobId;
-
-	var menu = [	
-		{title: "Open Job...", callback: function() {window.location.href=requestURL;}},
-		{title: "Open Job in New Window...", callback: function() {window.open(requestURL);}}
-	];
-
-	contextMenuView.show(event, menu);
-}
-
-var exJobClickCallback = function(event) {
-	console.log("Node clicked callback");
-	var jobId = event.currentTarget.jobid;
-	var requestURL = contextURL + "/manager?project=" + projectName + "&flow=" + 
-			flowId + "&job=" + jobId;
-
-	var menu = [	
-		{title: "Open Job...", callback: function() {window.location.href=requestURL;}},
-		{title: "Open Job in New Window...", callback: function() {window.open(requestURL);}}
-	];
-
-	contextMenuView.show(event, menu);
-}
-
-var exEdgeClickCallback = function(event) {
-	console.log("Edge clicked callback");
-}
-
-var exGraphClickCallback = function(event) {
-	console.log("Graph clicked callback");
-	var requestURL = contextURL + "/manager?project=" + projectName + "&flow=" + flowId;
-
-	var menu = [	
-		{title: "Open Flow...", callback: function() {window.location.href=requestURL;}},
-		{title: "Open Flow in New Window...", callback: function() {window.open(requestURL);}},
-		{break: 1},
-		{title: "Center Graph", callback: function() {graphModel.trigger("resetPanZoom");}}
-	];
-	
-	contextMenuView.show(event, menu);
-}
-
 var graphModel;
 azkaban.GraphModel = Backbone.Model.extend({});
 
@@ -466,16 +412,16 @@ $(function() {
 		el: $('#svgDiv'), 
 		model: graphModel, 
 		rightClick: { 
-			"node": exNodeClickCallback, 
-			"edge": exEdgeClickCallback, 
-			"graph": exGraphClickCallback 
+			"node": nodeClickCallback, 
+			"edge": edgeClickCallback, 
+			"graph": graphClickCallback 
 		}
 	});
 	
   jobsListView = new azkaban.JobListView({
 		el: $('#jobList'), 
 		model: graphModel, 
-		contextMenuCallback: exJobClickCallback
+		contextMenuCallback: jobClickCallback
 	});
 	
   executionsTimeGraphView = new azkaban.TimeGraphView({
@@ -505,32 +451,9 @@ $(function() {
 		"flow": flowId
 	};
 	var successHandler = function(data) {
-		// Create the nodes
-		var nodes = {};
-		for (var i = 0; i < data.nodes.length; ++i) {
-			var node = data.nodes[i];
-			nodes[node.id] = node;
-		}
-		for (var i = 0; i < data.edges.length; ++i) {
-			var edge = data.edges[i];
-			var fromNode = nodes[edge.from];
-			var toNode = nodes[edge.target];
-			
-			if (!fromNode.outNodes) {
-				fromNode.outNodes = {};
-			}
-			fromNode.outNodes[toNode.id] = toNode;
-			
-			if (!toNode.inNodes) {
-				toNode.inNodes = {};
-			}
-			toNode.inNodes[fromNode.id] = fromNode;
-		}
-	
 		console.log("data fetched");
-		graphModel.set({data: data});
-		graphModel.set({nodes: nodes});
-		graphModel.set({disabled: {}});
+		processFlowData(data);
+		graphModel.set({data:data});
 		graphModel.trigger("change:graph");
 		
 		// Handle the hash changes here so the graph finishes rendering first.
diff --git a/src/web/js/azkaban/view/flow-execute.js b/src/web/js/azkaban/view/flow-execute.js
index 2fc3b4b..e6d9cb1 100644
--- a/src/web/js/azkaban/view/flow-execute.js
+++ b/src/web/js/azkaban/view/flow-execute.js
@@ -16,26 +16,6 @@
 
 $.namespace('azkaban');
 
-function recurseAllAncestors(nodes, disabledMap, id, disable) {
-	var node = nodes[id];
-	if (node.in) {
-		for (var key in node.in) {
-			disabledMap[key] = disable;
-			recurseAllAncestors(nodes, disabledMap, key, disable);
-		}
-	}
-}
-
-function recurseAllDescendents(nodes, disabledMap, id, disable) {
-	var node = nodes[id];
-	if (node.out) {
-		for (var key in node.out) {
-			disabledMap[key] = disable;
-			recurseAllDescendents(nodes, disabledMap, key, disable);
-		}
-	}
-}
-
 var flowExecuteDialogView;
 azkaban.FlowExecuteDialogView = Backbone.View.extend({
 	events: {
@@ -90,20 +70,15 @@ azkaban.FlowExecuteDialogView = Backbone.View.extend({
 			}
 		}
 		
-		var disabled = "";
-		var disabledMap = this.model.get('disabled');
-		for (var dis in disabledMap) {
-			if (disabledMap[dis]) {
-				disabled += dis + ",";
-			}
-		}
+		var data = this.model.get("data");
+		var disabledList = gatherDisabledNodes(data);
 		
 		var executingData = {
 			projectId: projectId,
 			project: this.projectName,
 			ajax: "executeFlow",
 			flow: this.flowId,
-			disabled: disabled,
+			disabled: JSON.stringify(disabledList),
 			failureEmailsOverride:failureEmailsOverride,
 			successEmailsOverride:successEmailsOverride,
 			failureAction: failureAction,
@@ -128,7 +103,6 @@ azkaban.FlowExecuteDialogView = Backbone.View.extend({
 		
 		return executingData;
 	},
-	
 	changeFlowInfo: function() {
 		var successEmails = this.model.get("successEmails");
 		var failureEmails = this.model.get("failureEmails");
@@ -183,28 +157,7 @@ azkaban.FlowExecuteDialogView = Backbone.View.extend({
 		if (queueLevel) {
 			$('#queueLevel').val(queueLevel);
 		}
-		
-		if (nodeStatus) {
-			var nodeMap = this.model.get("nodeMap");
-			var disabled = {};
-			for (var key in nodeStatus) {
-				var status = nodeStatus[key];
-				
-				var node = nodeMap[key];
-				if (node) {
-					node.status = status;
-					if (node.status == "DISABLED" || node.status == "SKIPPED") {
-						node.status = "READY";
-						disabled[node.id] = true;
-					}
-					if (node.status == "SUCCEEDED" || node.status=="RUNNING") {
-						disabled[node.id] = true;
-					}
-				}
-			}
-			this.model.set({"disabled":disabled});
-		}
-		
+
 		if (flowParams) {
 			for (var key in flowParams) {
 				editTableView.handleAddRow({
@@ -222,20 +175,25 @@ azkaban.FlowExecuteDialogView = Backbone.View.extend({
 		
 		// ExecId is optional
 		var execId = data.execid;
-	
-		var loadedId = executableGraphModel.get("flowId");
-	
-		this.loadGraph(projectName, flowId);
-		this.loadFlowInfo(projectName, flowId, execId);
-
+		var exgraph = data.exgraph;
+		
 		this.projectName = projectName;
 		this.flowId = flowId;
-		if (jobId) {
-			this.showExecuteJob(projectName, flowId, jobId, data.withDep);
-		}
-		else {
-			this.showExecuteFlow(projectName, flowId);
+		
+		var self = this;
+		var loadCallback = function() {
+			if (jobId) {
+				self.showExecuteJob(projectName, flowId, jobId, data.withDep);
+			}
+			else {
+				self.showExecuteFlow(projectName, flowId);
+			}
 		}
+		
+		var loadedId = executableGraphModel.get("flowId");
+		this.loadGraph(projectName, flowId, exgraph, loadCallback);
+		this.loadFlowInfo(projectName, flowId, execId);
+
 	},
 	
 	showExecuteFlow: function(projectName, flowId) {
@@ -250,19 +208,17 @@ azkaban.FlowExecuteDialogView = Backbone.View.extend({
 		sideMenuDialogView.menuSelect($("#flow-option"));
 		$("#execute-flow-panel-title").text("Execute Flow " + flowId);
 		
-		var nodes = this.model.get("nodeMap");
+		var data = this.model.get("data");
 		var disabled = this.model.get("disabled");
 		
 		// Disable all, then re-enable those you want.
-		for (var key in nodes) {
-			disabled[key] = true;
-		}
+		disableAll();
 		
-		var jobNode = nodes[jobId];
-		disabled[jobId] = false;
+		var jobNode = data.nodeMap[jobId];
+		touchNode(jobNode, false);
 		
 		if (withDep) {
-			recurseAllAncestors(nodes, disabled, jobId, false);
+			recurseAllAncestors(jobNode, false);
 		}
 
 		this.showExecutionOptionPanel();
@@ -289,11 +245,62 @@ azkaban.FlowExecuteDialogView = Backbone.View.extend({
 		fetchFlowInfo(this.model, projectName, flowId, execId);
 	},
 	
-	loadGraph: function(projectName, flowId) {
+	loadGraph: function(projectName, flowId, exgraph, callback) {
 		console.log("Loading flow " + flowId);
-		fetchFlow(this.model, projectName, flowId, true);
+		var requestURL = contextURL + "/manager";
+		
+		var graphModel = executableGraphModel;
+		// fetchFlow(this.model, projectName, flowId, true);
+		var requestData = {
+				"project": projectName, 
+				"ajax": "fetchflowgraph", 
+				"flow": flowId
+			};
+		var self = this;
+		var successHandler = function(data) {
+			console.log("data fetched");
+			processFlowData(data);
+			graphModel.set({data:data});
+			
+			if (exgraph) {
+				self.assignInitialStatus(data, exgraph);
+			}
+			
+			// Auto disable jobs that are finished.
+			disableFinishedJobs(data);
+			executingSvgGraphView = new azkaban.SvgGraphView({
+				el: $('#flow-executing-graph'), 
+				model: graphModel,
+				render: false,
+				rightClick: { 
+					"node": expanelNodeClickCallback,
+					"edge": expanelEdgeClickCallback, 
+					"graph": expanelGraphClickCallback 
+				},
+				tooltipcontainer: "#svg-div-custom"
+			});
+			
+			if (callback) {
+				callback.call(this);
+			}
+		};
+		$.get(requestURL, requestData, successHandler, "json");
+	},
+	assignInitialStatus: function(data, statusData) {
+		// Copies statuses over from the previous execution if it exists.
+		var statusNodeMap = statusData.nodeMap;
+		var nodes = data.nodes;
+		for(var i=0; i<nodes.length; ++i) {
+			var node = nodes[i];
+			var statusNode = statusNodeMap[node.id];
+			if (statusNode) {
+				node.status = statusNode.status;
+				if (node.type == "flow" && statusNode.type == "flow") {
+					this.assignInitialStatus(node, statusNode);
+				}
+			}
+		}
 	},
-	
 	handleExecuteFlow: function(evt) {
 		console.log("click schedule button.");
 		var executeURL = contextURL + "/executor";
@@ -326,16 +333,16 @@ azkaban.EditTableView = Backbone.View.extend({
 	
 		var tr = document.createElement("tr");
 		var tdName = document.createElement("td");
-    $(tdName).addClass('property-key');
+		$(tdName).addClass('property-key');
 		var tdValue = document.createElement("td");
 		
 		var remove = document.createElement("div");
-    $(remove).addClass("pull-right").addClass('remove-btn');
-    var removeBtn = document.createElement("button");
-    $(removeBtn).attr('type', 'button');
-    $(removeBtn).addClass('btn').addClass('btn-xs').addClass('btn-danger');
-    $(removeBtn).text('Delete');
-    $(remove).append(removeBtn);
+		$(remove).addClass("pull-right").addClass('remove-btn');
+		var removeBtn = document.createElement("button");
+		$(removeBtn).attr('type', 'button');
+		$(removeBtn).addClass('btn').addClass('btn-xs').addClass('btn-danger');
+		$(removeBtn).text('Delete');
+		$(remove).append(removeBtn);
 
 		var nameData = document.createElement("span");
 		$(nameData).addClass("spanValue");
@@ -348,7 +355,7 @@ azkaban.EditTableView = Backbone.View.extend({
 		$(tdName).addClass("editable");
 		
 		$(tdValue).append(valueData);
-    $(tdValue).append(remove);
+		$(tdValue).append(remove);
 		$(tdValue).addClass("editable").addClass('value');
 		
 		$(tr).addClass("editRow");
@@ -367,7 +374,7 @@ azkaban.EditTableView = Backbone.View.extend({
 					
 		var input = document.createElement("input");
 		$(input).attr("type", "text");
-    $(input).addClass('form-control').addClass('input-sm');
+		$(input).addClass('form-control').addClass('input-sm');
 		$(input).css("width", "100%");
 		$(input).val(text);
 		$(curTarget).addClass("editing");
@@ -404,13 +411,13 @@ azkaban.EditTableView = Backbone.View.extend({
 		$(valueData).text(text);
 
 		if ($(parent).hasClass("value")) {
-      var remove = document.createElement("div");
-      $(remove).addClass("pull-right").addClass('remove-btn');
-      var removeBtn = document.createElement("button");
-      $(removeBtn).attr('type', 'button');
-      $(removeBtn).addClass('btn').addClass('btn-xs').addClass('btn-danger');
-      $(removeBtn).text('Delete');
-      $(remove).append(removeBtn);
+			var remove = document.createElement("div");
+			$(remove).addClass("pull-right").addClass('remove-btn');
+			var removeBtn = document.createElement("button");
+			$(removeBtn).attr('type', 'button');
+			$(removeBtn).addClass('btn').addClass('btn-xs').addClass('btn-danger');
+			$(removeBtn).text('Delete');
+			$(remove).append(removeBtn);
 			$(parent).append(remove);
 		}
 		
@@ -464,7 +471,7 @@ azkaban.SideMenuDialogView = Backbone.View.extend({
 var handleJobMenuClick = function(action, el, pos) {
 	var jobid = el[0].jobid;
 	
-	var requesgURL = contextURL + "/manager?project=" + projectName + "&flow=" + flowName + "&job=" + jobid;
+	var requestURL = contextURL + "/manager?project=" + projectName + "&flow=" + flowName + "&job=" + jobid;
 	if (action == "open") {
 		window.location.href = requestURL;
 	}
@@ -476,115 +483,198 @@ var handleJobMenuClick = function(action, el, pos) {
 var executableGraphModel;
 azkaban.GraphModel = Backbone.Model.extend({});
 
+/**
+ * Disable jobs that need to be disabled
+ */
+var disableFinishedJobs = function(data) {
+	for (var i=0; i < data.nodes.length; ++i) {
+		var node = data.nodes[i];
+		
+		if (node.status == "DISABLED" || node.status == "SKIPPED") {
+			node.status = "READY";
+			node.disabled = true;
+		}
+		else if (node.status == "SUCCEEDED" || node.status=="RUNNING") {
+			node.disabled = true;
+		}
+		else if (node.status == "KILLED") {
+			node.disabled = false;
+			node.status="READY";
+		}
+		else {
+			node.disabled = false;
+			if (node.flowData) {
+				disableFinishedJobs(node.flowData);
+			}
+		}
+	}
+}
+
+/**
+ * Enable all jobs. Recurse
+ */
 var enableAll = function() {
-	disabled = {};
-	executableGraphModel.set({disabled: disabled});
+	recurseTree(executableGraphModel.get("data"), false, false);
 	executableGraphModel.trigger("change:disabled");
 }
 
 var disableAll = function() {
-	var disabled = executableGraphModel.get("disabled");
-
-	var nodes = executableGraphModel.get("nodes");
-	for (var key in nodes) {
-		disabled[key] = true;
-	}
-
-	executableGraphModel.set({disabled: disabled});
+	recurseTree(executableGraphModel.get("data"), true, false);
 	executableGraphModel.trigger("change:disabled");
 }
 
-var touchNode = function(jobid, disable) {
-	var disabled = executableGraphModel.get("disabled");
+var recurseTree = function(data, disabled, recurse) {
+	for (var i=0; i < data.nodes.length; ++i) {
+		var node = data.nodes[i];
+		node.disabled = disabled;
+		
+		if (node.flowData && recurse) {
+			recurseTree(node.flowData, disabled);
+		}
+	}
+}
 
-	disabled[jobid] = disable;
-	executableGraphModel.set({disabled: disabled});
+var touchNode = function(node, disable) {
+	node.disabled = disable;
 	executableGraphModel.trigger("change:disabled");
 }
 
-var touchParents = function(jobid, disable) {
-	var disabled = executableGraphModel.get("disabled");
-	var nodes = executableGraphModel.get("nodes");
-	var inNodes = nodes[jobid].inNodes;
+var touchParents = function(node, disable) {
+	var inNodes = node.inNodes;
 
 	if (inNodes) {
 		for (var key in inNodes) {
-			disabled[key] = disable;
+			inNodes[key].disabled = disable;
 		}
 	}
-	
-	executableGraphModel.set({disabled: disabled});
+
 	executableGraphModel.trigger("change:disabled");
 }
 
-var touchChildren = function(jobid, disable) {
-	var disabledMap = executableGraphModel.get("disabled");
-	var nodes = executableGraphModel.get("nodes");
-	var outNodes = nodes[jobid].outNodes;
+var touchChildren = function(node, disable) {
+	var outNodes = node.outNodes;
 
 	if (outNodes) {
 		for (var key in outNodes) {
-			disabledMap[key] = disable;
+			outNodes[key].disabled = disable;
 		}
 	}
 	
-	executableGraphModel.set({disabled: disabledMap});
 	executableGraphModel.trigger("change:disabled");
 }
 
-var touchAncestors = function(jobid, disable) {
-	var disabled = executableGraphModel.get("disabled");
-	var nodes = executableGraphModel.get("nodes");
+var touchAncestors = function(node, disable) {
+	recurseAllAncestors(node, disable);
 	
-	recurseAllAncestors(nodes, disabled, jobid, disable);
-	
-	executableGraphModel.set({disabled: disabled});
 	executableGraphModel.trigger("change:disabled");
 }
 
 var touchDescendents = function(jobid, disable) {
-	var disabled = executableGraphModel.get("disabled");
-	var nodes = executableGraphModel.get("nodes");
-	
-	recurseAllDescendents(nodes, disabled, jobid, disable);
+	recurseAllDescendents(node, disable);
 	
-	executableGraphModel.set({disabled: disabled});
 	executableGraphModel.trigger("change:disabled");
 }
 
-var nodeClickCallback = function(event) {
+var gatherDisabledNodes = function(data) {
+	var nodes = data.nodes;
+	var disabled = [];
+	
+	for (var i = 0; i < nodes.length; ++i) {
+		var node = nodes[i];
+		if (node.disabled) {
+			disabled.push(node.id);
+		}
+		else {
+			if (node.flowData) {
+				var array = gatherDisabledNodes(node.flowData);
+				if (array && array.length > 0) {
+					disabled.push({id: node.id, children: array});
+				}
+			}
+		}
+	}
+	
+	return disabled;
+}
+
+function recurseAllAncestors(node, disable) {
+	var inNodes = node.inNodes;
+	if (inNodes) {
+		for (var key in inNodes) {
+			inNodes[key].disabled = disable;
+			recurseAllAncestors(inNodes[key], disable);
+		}
+	}
+}
+
+function recurseAllDescendents(node, disable) {
+	var outNodes = node.outNodes;
+	if (outNodes) {
+		for (var key in outNodes) {
+			outNodes[key].disabled = disable;
+			recurseAllDescendents(outNodes[key], disable);
+		}
+	}
+}
+
+var expanelNodeClickCallback = function(event, model, node) {
 	console.log("Node clicked callback");
-	var jobId = event.currentTarget.jobid;
+	var jobId = node.id;
 	var flowId = executableGraphModel.get("flowId");
-	var requestURL = contextURL + "/manager?project=" + projectName + "&flow=" + flowId + "&job=" + jobId;
+	var type = node.type;
+	
+	var menu;
+	if (type == "flow") {
+		var flowRequestURL = contextURL + "/manager?project=" + projectName + "&flow=" + node.flowId;
+		if (node.expanded) {
+			menu = [
+				{title: "Collapse Flow...", callback: function() {model.trigger("collapseFlow", node);}},
+				{title: "Open Flow in New Window...", callback: function() {window.open(flowRequestURL);}}
+			];
+	
+		}
+		else {
+			menu = [
+				{title: "Expand Flow...", callback: function() {model.trigger("expandFlow", node);}},
+				{title: "Open Flow in New Window...", callback: function() {window.open(flowRequestURL);}}
+			];
+		}
+	}
+	else {
+		var requestURL = contextURL + "/manager?project=" + projectName + "&flow=" + flowId + "&job=" + jobId;
+		menu = [
+				{title: "Open Job in New Window...", callback: function() {window.open(requestURL);}},
+			];
+	}
 
-	var menu = [
-		{title: "Open Job in New Window...", callback: function() {window.open(requestURL);}},
+	$.merge(menu, [
 		{break: 1},
-		{title: "Enable", callback: function() {touchNode(jobId, false);}, submenu: [
-			{title: "Parents", callback: function(){touchParents(jobId, false);}},
-			{title: "Ancestors", callback: function(){touchAncestors(jobId, false);}},
-			{title: "Children", callback: function(){touchChildren(jobId, false);}},
-			{title: "Descendents", callback: function(){touchDescendents(jobId, false);}},
+		{title: "Enable", callback: function() {touchNode(node, false);}, submenu: [
+			{title: "Parents", callback: function(){touchParents(node, false);}},
+			{title: "Ancestors", callback: function(){touchAncestors(node, false);}},
+			{title: "Children", callback: function(){touchChildren(node, false);}},
+			{title: "Descendents", callback: function(){touchDescendents(node, false);}},
 			{title: "Enable All", callback: function(){enableAll();}}
 		]},
-		{title: "Disable", callback: function() {touchNode(jobId, true)}, submenu: [
-			{title: "Parents", callback: function(){touchParents(jobId, true);}},
-			{title: "Ancestors", callback: function(){touchAncestors(jobId, true);}},
-			{title: "Children", callback: function(){touchChildren(jobId, true);}},
-			{title: "Descendents", callback: function(){touchDescendents(jobId, true);}},
+		{title: "Disable", callback: function() {touchNode(node, true)}, submenu: [
+			{title: "Parents", callback: function(){touchParents(node, true);}},
+			{title: "Ancestors", callback: function(){touchAncestors(node, true);}},
+			{title: "Children", callback: function(){touchChildren(node, true);}},
+			{title: "Descendents", callback: function(){touchDescendents(node, true);}},
 			{title: "Disable All", callback: function(){disableAll();}}
-		]}
-	];
+		]},
+		{title: "Center Job", callback: function() {model.trigger("centerNode", node);}}
+	]);
+
 
 	contextMenuView.show(event, menu);
 }
 
-var edgeClickCallback = function(event) {
+var expanelEdgeClickCallback = function(event) {
 	console.log("Edge clicked callback");
 }
 
-var graphClickCallback = function(event) {
+var expanelGraphClickCallback = function(event) {
 	console.log("Graph clicked callback");
 	var flowId = executableGraphModel.get("flowId");
 	var requestURL = contextURL + "/manager?project=" + projectName + "&flow=" + flowId;
@@ -608,17 +698,6 @@ $(function() {
 		el: $('#execute-flow-panel'), 
 		model: executableGraphModel
 	});
-	svgGraphView = new azkaban.SvgGraphView({
-		el: $('#svg-div-custom'), 
-		model: executableGraphModel, 
-		topGId: "topG", 
-		graphMargin: 10, 
-		rightClick: { 
-			"node": nodeClickCallback, 
-			"edge": edgeClickCallback, 
-			"graph": graphClickCallback 
-		}
-	});
 	
 	sideMenuDialogView = new azkaban.SideMenuDialogView({
 		el: $('#graph-options')
diff --git a/src/web/js/azkaban/view/flow-execution-list.js b/src/web/js/azkaban/view/flow-execution-list.js
new file mode 100644
index 0000000..69621a3
--- /dev/null
+++ b/src/web/js/azkaban/view/flow-execution-list.js
@@ -0,0 +1,359 @@
+var executionListView;
+azkaban.ExecutionListView = Backbone.View.extend({
+	events: {
+		//"contextmenu .flow-progress-bar": "handleProgressBoxClick"
+	},
+	
+	initialize: function(settings) {
+		this.model.bind('change:graph', this.renderJobs, this);
+		this.model.bind('change:update', this.updateJobs, this);
+		
+		// This is for tabbing. Blah, hacky
+		var executingBody = $("#executableBody")[0];
+		executingBody.level = 0;
+	},
+	
+	renderJobs: function(evt) {
+		var data = this.model.get("data");
+		var lastTime = data.endTime == -1 ? (new Date()).getTime() : data.endTime;
+		var executingBody = $("#executableBody");
+		this.updateJobRow(data.nodes, executingBody);
+		
+		var flowLastTime = data.endTime == -1 ? (new Date()).getTime() : data.endTime;
+		var flowStartTime = data.startTime;
+		this.updateProgressBar(data, flowStartTime, flowLastTime);
+	},
+//
+//	handleProgressBoxClick: function(evt) {
+//		var target = evt.currentTarget;
+//		var job = target.job;
+//		var attempt = target.attempt;
+//		
+//		var data = this.model.get("data");
+//		var node = data.nodes[job];
+//		
+//		var jobId = event.currentTarget.jobid;
+//		var requestURL = contextURL + "/manager?project=" + projectName + "&execid=" + execId + "&job=" + job + "&attempt=" + attempt;
+//	
+//		var menu = [	
+//				{title: "Open Job...", callback: function() {window.location.href=requestURL;}},
+//				{title: "Open Job in New Window...", callback: function() {window.open(requestURL);}}
+//		];
+//	
+//		contextMenuView.show(evt, menu);
+//	},
+	
+	updateJobs: function(evt) {
+		var update = this.model.get("update");
+		var lastTime = update.endTime == -1 ? (new Date()).getTime() : update.endTime;
+		var executingBody = $("#executableBody");
+		
+		if (update.nodes) {
+			this.updateJobRow(update.nodes, executingBody);
+		}
+		
+		var data = this.model.get("data");
+		var flowLastTime = data.endTime == -1 ? (new Date()).getTime() : data.endTime;
+		var flowStartTime = data.startTime;
+		this.updateProgressBar(data, flowStartTime, flowLastTime);
+	},
+	updateJobRow: function(nodes, body) {
+		if (!nodes) {
+			return;
+		}
+		
+		nodes.sort(function(a,b) { return a.startTime - b.startTime; });
+		
+		for (var i = 0; i < nodes.length; ++i) {
+			var node = nodes[i].changedNode ? nodes[i].changedNode : nodes[i];
+			
+			if (node.startTime < 0) {
+				continue;
+			}
+			//var nodeId = node.id.replace(".", "\\\\.");
+			var row = node.joblistrow;
+			if (!row) {
+				this.addNodeRow(node, body);
+			}
+			
+			row = node.joblistrow;
+			var statusDiv = $(row).find("> td.statustd > .status");
+			statusDiv.text(statusStringMap[node.status]);
+			$(statusDiv).attr("class", "status " + node.status);
+
+			var startTimeTd = $(row).find("> td.startTime");
+			var startdate = new Date(node.startTime);
+			$(startTimeTd).text(getDateFormat(startdate));
+	  
+			var endTimeTd = $(row).find("> td.endTime");
+			if (node.endTime == -1) {
+				$(endTimeTd).text("-");
+			}
+			else {
+				var enddate = new Date(node.endTime);
+				$(endTimeTd).text(getDateFormat(enddate));
+			}
+	  
+			var progressBar = $(row).find("> td.timeline > .flow-progress > .main-progress");
+			if (!progressBar.hasClass(node.status)) {
+				for (var j = 0; j < statusList.length; ++j) {
+					var status = statusList[j];
+					progressBar.removeClass(status);
+				}
+				progressBar.addClass(node.status);
+			}
+  
+			// Create past attempts
+			if (node.pastAttempts) {
+				for (var a = 0; a < node.pastAttempts.length; ++a) {
+					var attempt = node.pastAttempts[a];
+					var attemptBox = attempt.attemptBox;
+					
+					if (!attemptBox) {
+						var attemptBox = document.createElement("div");
+						attempt.attemptBox = attemptBox;
+						
+						$(attemptBox).addClass("flow-progress-bar");
+						$(attemptBox).addClass("attempt");
+						
+						$(attemptBox).css("float","left");
+						$(attemptBox).bind("contextmenu", attemptRightClick);
+						
+						$(progressBar).before(attemptBox);
+						attemptBox.job = nodeId;
+						attemptBox.attempt = a;
+					}
+				}
+			}
+  
+			var elapsedTime = $(row).find("> td.elapsedTime");
+			if (node.endTime == -1) {
+				$(elapsedTime).text(getDuration(node.startTime, (new Date()).getTime()));
+			}
+			else {
+				$(elapsedTime).text(getDuration(node.startTime, node.endTime));
+			}
+			
+			if (node.nodes) {
+				var subtableBody = $(row.subflowrow).find("> td > table");
+				subtableBody[0].level = $(body)[0].level + 1;
+				this.updateJobRow(node.nodes, subtableBody);
+			}
+		}
+	},
+	
+	updateProgressBar: function(data, flowStartTime, flowLastTime) {
+		if (data.startTime == -1) {
+			return;
+		}
+
+		var outerWidth = $(".flow-progress").css("width");
+		if (outerWidth) {
+			if (outerWidth.substring(outerWidth.length - 2, outerWidth.length) == "px") {
+				outerWidth = outerWidth.substring(0, outerWidth.length - 2);
+			}
+			outerWidth = parseInt(outerWidth);
+		}
+		
+		var nodes = data.nodes;
+		var diff = flowLastTime - flowStartTime;
+		var factor = outerWidth/diff;
+		for (var i = 0; i < nodes.length; ++i) {
+			var node = nodes[i];
+			
+			// calculate the progress
+			var tr = node.joblistrow;
+			
+			var progressBar = $(tr).find("> td.timeline > .flow-progress > .main-progress");
+			var offsetLeft = 0;
+			var minOffset = 0;
+			progressBar.attempt = 0;
+			
+			// Add all the attempts
+			if (node.pastAttempts) {
+				var logURL = contextURL + "/executor?execid=" + execId + "&job=" + node.id + "&attempt=" +	node.pastAttempts.length;
+				var anchor = $(tr).find("> td.details > a");
+				if (anchor.length != 0) {
+					$(anchor).attr("href", logURL);
+					progressBar.attempt = node.pastAttempts.length;
+				}
+				
+				// Calculate the node attempt bars
+				for (var p = 0; p < node.pastAttempts.length; ++p) {
+					var pastAttempt = node.pastAttempts[p];
+					var pastAttemptBox = pastAttempt.attemptBox;
+					
+					var left = (pastAttempt.startTime - flowStartTime)*factor;
+					var width =	Math.max((pastAttempt.endTime - pastAttempt.startTime)*factor, 3);
+					
+					var margin = left - offsetLeft;
+					$(pastAttemptBox).css("margin-left", left - offsetLeft);
+					$(pastAttemptBox).css("width", width);
+					
+					$(pastAttemptBox).attr("title", "attempt:" + p + "	start:" + getHourMinSec(new Date(pastAttempt.startTime)) + "	end:" + getHourMinSec(new Date(pastAttempt.endTime)));
+					offsetLeft += width + margin;
+				}
+			}
+			
+			var nodeLastTime = node.endTime == -1 ? (new Date()).getTime() : node.endTime;
+			var left = Math.max((node.startTime-flowStartTime)*factor, minOffset);
+			var margin = left - offsetLeft;
+			var width = Math.max((nodeLastTime - node.startTime)*factor, 3);
+			width = Math.min(width, outerWidth);
+			
+			progressBar.css("margin-left", left)
+			progressBar.css("width", width);
+			progressBar.attr("title", "attempt:" + progressBar.attempt + "	start:" + getHourMinSec(new Date(node.startTime)) + "	end:" + getHourMinSec(new Date(node.endTime)));
+		
+			if (node.nodes) {
+				this.updateProgressBar(node, flowStartTime, flowLastTime);
+			}
+		}
+	},
+	toggleExpandFlow: function(flow) {
+		console.log("Toggle Expand");
+		var tr = flow.joblistrow;
+		var subFlowRow = tr.subflowrow;
+		var expandIcon = $(tr).find("> td > .listExpand");
+		if (tr.expanded) {
+			tr.expanded = false;
+			$(expandIcon).removeClass("glyphicon-chevron-up");
+			$(expandIcon).addClass("glyphicon-chevron-down");
+			
+			$(tr).removeClass("expanded");
+			$(subFlowRow).hide();
+		}
+		else {
+			tr.expanded = true;
+			$(expandIcon).addClass("glyphicon-chevron-up");
+			$(expandIcon).removeClass("glyphicon-chevron-down");
+			$(tr).addClass("expanded");
+			$(subFlowRow).show();
+		}
+	},
+	expandFlow: function(flow) {
+		for (var i = 0; i < flow.nodes.length; ++i) {
+			var node = flow.nodes[i];
+			///@TODO Expand.
+		}
+	},
+	addNodeRow: function(node, body) {
+		var self = this;
+		var tr = document.createElement("tr");
+		var tdName = document.createElement("td");
+		var tdTimeline = document.createElement("td");
+		var tdStart = document.createElement("td");
+		var tdEnd = document.createElement("td");
+		var tdElapse = document.createElement("td");
+		var tdStatus = document.createElement("td");
+		var tdDetails = document.createElement("td");
+		node.joblistrow = tr;
+		tr.node = node;
+		var padding = 15*$(body)[0].level;
+		
+		$(tr).append(tdName);
+		$(tr).append(tdTimeline);
+		$(tr).append(tdStart);
+		$(tr).append(tdEnd);
+		$(tr).append(tdElapse);
+		$(tr).append(tdStatus);
+		$(tr).append(tdDetails);
+		$(tr).addClass("jobListRow");
+		
+		$(tdName).addClass("jobname");
+		if (padding) {
+			$(tdName).css("padding-left", padding);
+		}
+		$(tdTimeline).addClass("timeline");
+		$(tdStart).addClass("startTime");
+		$(tdEnd).addClass("endTime");
+		$(tdElapse).addClass("elapsedTime");
+		$(tdStatus).addClass("statustd");
+		$(tdDetails).addClass("details");
+		
+		var outerProgressBar = document.createElement("div");
+		//$(outerProgressBar).attr("id", node.id + "-outerprogressbar");
+		$(outerProgressBar).addClass("flow-progress");
+		
+		var progressBox = document.createElement("div");
+		progressBox.job = node.id;
+		//$(progressBox).attr("id", node.id + "-progressbar");
+		$(progressBox).addClass("flow-progress-bar");
+		$(progressBox).addClass("main-progress");
+		$(outerProgressBar).append(progressBox);
+		$(tdTimeline).append(outerProgressBar);
+
+		var requestURL = contextURL + "/manager?project=" + projectName + "&job=" + node.id + "&history";
+		var a = document.createElement("a");
+		$(a).attr("href", requestURL);
+		$(a).text(node.id);
+		$(tdName).append(a);
+		if (node.type=="flow") {
+			var expandIcon = document.createElement("div");
+			$(expandIcon).addClass("listExpand");
+			$(tdName).append(expandIcon);
+			$(expandIcon).addClass("expandarrow glyphicon glyphicon-chevron-down");
+			$(expandIcon).click(function(evt) {
+				var parent = $(evt.currentTarget).parents("tr")[0];
+				self.toggleExpandFlow(parent.node);
+			});
+		}
+
+		var status = document.createElement("div");
+		$(status).addClass("status");
+		//$(status).attr("id", node.id + "-status-div");
+		tdStatus.appendChild(status);
+
+		var logURL = contextURL + "/executor?execid=" + execId + "&job=" + node.nestedId;
+		if (node.attempt) {
+			logURL += "&attempt=" + node.attempt;
+		}
+
+		if (node.type != 'flow' && node.status != 'SKIPPED') {
+			var a = document.createElement("a");
+			$(a).attr("href", logURL);
+			//$(a).attr("id", node.id + "-log-link");
+			$(a).text("Details");
+			$(tdDetails).append(a);
+		}
+
+		$(body).append(tr);
+		if (node.type=="flow") {
+			var subFlowRow = document.createElement("tr");
+			var subFlowCell = document.createElement("td");
+			$(subFlowCell).addClass("subflowrow");
+			
+			var numColumn = $(tr).children("td").length;
+			$(subFlowCell).attr("colspan", numColumn);
+			tr.subflowrow = subFlowRow;
+			
+			$(subFlowRow).append(subFlowCell);
+			$(body).append(subFlowRow);
+			$(subFlowRow).hide();
+			var subtable = document.createElement("table");
+			var parentClasses = $(body).closest("table").attr("class");
+			
+			$(subtable).attr("class", parentClasses);
+			$(subtable).addClass("subtable");
+			$(subFlowCell).append(subtable);
+		}
+	}
+});
+
+var attemptRightClick = function(event) {
+	var target = event.currentTarget;
+	var job = target.job;
+	var attempt = target.attempt;
+	
+	var jobId = event.currentTarget.jobid;
+	var requestURL = contextURL + "/executor?project=" + projectName + "&execid=" + execId + "&job=" + job + "&attempt=" + attempt;
+
+	var menu = [	
+		{title: "Open Attempt Log...", callback: function() {window.location.href=requestURL;}},
+		{title: "Open Attempt Log in New Window...", callback: function() {window.open(requestURL);}}
+	];
+
+	contextMenuView.show(event, menu);
+	return false;
+}
+
diff --git a/src/web/js/azkaban/view/flow-job.js b/src/web/js/azkaban/view/flow-job.js
index fe8cf6d..e7f0875 100644
--- a/src/web/js/azkaban/view/flow-job.js
+++ b/src/web/js/azkaban/view/flow-job.js
@@ -17,9 +17,11 @@
 azkaban.JobListView = Backbone.View.extend({
 	events: {
 		"keyup input": "filterJobs",
-		"click .job": "handleJobClick",
-		"click .resetPanZoomBtn": "handleResetPanZoom",
-		"contextmenu li": "handleContextMenuClick"
+		"click li.listElement": "handleJobClick",
+		"click #resetPanZoomBtn": "handleResetPanZoom",
+		"click #autoPanZoomBtn": "handleAutoPanZoom",
+		"contextmenu li.listElement": "handleContextMenuClick",
+		"click .expandarrow": "handleToggleMenuExpand"
 	},
 	
 	initialize: function(settings) {
@@ -29,96 +31,118 @@ azkaban.JobListView = Backbone.View.extend({
 		this.model.bind('change:update', this.handleStatusUpdate, this);
 		
 		this.filterInput = $(this.el).find("#filter");
-		this.list = $(this.el).find("#list");
+		this.list = $(this.el).find("#joblist");
 		this.contextMenu = settings.contextMenuCallback;
 		this.listNodes = {};
 	},
 	
 	filterJobs: function(self) {
 		var filter = this.filterInput.val();
-		if (filter && filter.trim() != "") {
-			filter = filter.trim();
-			if (filter == "") {
-				if (this.filter) {
-					this.jobs.children().each(function(){
-						var a = $(this).find("a");
-						$(a).html(this.jobid);
-						$(this).show();
-					});
-				}
-				this.filter = null;
-				return;
-			}
-		}
-		else {
-			if (this.filter) {
-				this.jobs.children().each(function(){
-					var a = $(this).find("a");
-					$(a).html(this.jobid);
-					$(this).show();
-				});
-			}
-				
-			this.filter = null;
+		// Clear all filters first
+		if (!filter || filter.trim() == "") {
+			this.unfilterAll(self);
 			return;
 		}
 		
-		this.jobs.children().each(function() {
-			var jobid = this.jobid;
-			var index = jobid.indexOf(filter);
-			if (index != -1) {
-				var a = $(this).find("a");
-				var endIndex = index + filter.length;
-				var newHTML = jobid.substring(0, index) + "<span>" + 
-						jobid.substring(index, endIndex) + "</span>" + 
-						jobid.substring(endIndex, jobid.length);
-				
-				$(a).html(newHTML);
-				$(this).show();
+		this.hideAll(self);
+		var showList = {};
+		
+		// find the jobs that need to be exposed.
+		for (var key in this.listNodes) {
+			var li = this.listNodes[key];
+			var node = li.node;
+			var nodeName = node.id;
+			node.listElement = li;
+
+			var index = nodeName.indexOf(filter);
+			if (index == -1) {
+				continue;
 			}
-			else {
-				$(this).hide();
+
+			var spanlabel = $(li).find("> a > span");
+			
+			var endIndex = index + filter.length;
+			var newHTML = nodeName.substring(0, index) + "<span class=\"filterHighlight\">" + 
+					nodeName.substring(index, endIndex) + "</span>" + 
+					nodeName.substring(endIndex, nodeName.length);
+			$(spanlabel).html(newHTML);
+			
+			// Apply classes to all the included embedded flows.
+			var pIndex = key.length;
+			while ((pIndex = key.lastIndexOf(":", pIndex - 1)) > 0) {
+				var parentId = key.substr(0, pIndex);
+				var parentLi = this.listNodes[parentId];
+				$(parentLi).show();
+				$(parentLi).addClass("subFilter");
 			}
-		});
 			
-		this.filter = filter;
+			$(li).show();
+		}
 	},
 	
-	handleStatusUpdate: function(evt) {
-		var updateData = this.model.get("update");
-		if (updateData.nodes) {
-			for (var i = 0; i < updateData.nodes.length; ++i) {
-				var updateNode = updateData.nodes[i];
-				var job = this.listNodes[updateNode.id];
-				$(job).removeClass();
-				$(job).addClass("list-group-item");
-				$(job).addClass(updateNode.status);
-			}
+	hideAll: function(self) {
+		for (var key in this.listNodes) {
+			var li = this.listNodes[key];
+			var label = $(li).find("> a > span");
+			$(label).text(li.node.id);
+			$(li).removeClass("subFilter");
+			$(li).hide();
 		}
 	},
 	
-	assignInitialStatus: function(evt) {
+	unfilterAll: function(self) {
+		for (var key in this.listNodes) {
+			var li = this.listNodes[key];
+			var label = $(li).find("> a > span");
+			$(label).text(li.node.id);
+			$(li).removeClass("subFilter");
+			$(li).show();
+		}
+	},
+	
+	handleStatusUpdate: function(evt) {
 		var data = this.model.get("data");
+		this.changeStatuses(data);
+	},
+	
+	changeStatuses: function(data) {
 		for (var i = 0; i < data.nodes.length; ++i) {
-			var updateNode = data.nodes[i];
-			var job = this.listNodes[updateNode.id];
-      if (!$(job).hasClass("list-group-item")) {
-        $(job).addClass("list-group-item");
-      }
-			$(job).addClass(updateNode.status);
+			var node = data.nodes[i];
+
+			// Confused? In updates, a node reference is given to the update node.
+			var liElement = node.listElement;
+			var child = $(liElement).children("a");
+			if (!$(child).hasClass(node.status)) {
+				$(child).removeClass(statusList.join(' '));
+				$(child).addClass(node.status);
+				$(child).attr("title", node.status + " (" + node.type + ")");
+			}
+			if (node.nodes) {
+				this.changeStatuses(node);
+			}
 		}
 	},
 	
 	render: function(self) {
 		var data = this.model.get("data");
 		var nodes = data.nodes;
-		var edges = data.edges;
 		
-		this.listNodes = {}; 
+		this.renderTree(this.list, data);
+		
+		//this.assignInitialStatus(self);
+		this.handleDisabledChange(self);
+		this.changeStatuses(data);
+	},
+	
+	renderTree: function(el, data, prefix) {
+		var nodes = data.nodes;
 		if (nodes.length == 0) {
 			console.log("No results");
 			return;
 		};
+		if (!prefix) {
+			prefix = "";
+		}
 	
 		var nodeArray = nodes.slice(0);
 		nodeArray.sort(function(a, b) {
@@ -131,63 +155,128 @@ azkaban.JobListView = Backbone.View.extend({
 			}
 		});
 		
-		var list = this.list;
-		this.jobs = $(list);
+		var ul = document.createElement('ul');
+		$(ul).addClass("tree-list");
 		for (var i = 0; i < nodeArray.length; ++i) {
+			var li = document.createElement("li");
+			$(li).addClass("listElement");
+			$(li).addClass("tree-list-item");
+			
+			// This is used for the filter step.
+			var listNodeName = prefix + nodeArray[i].id;
+			this.listNodes[listNodeName]=li;
+			li.node = nodeArray[i];
+			li.node.listElement = li;
+
 			var a = document.createElement("a");
-			$(a).addClass('list-group-item').addClass('job');
-      $(a).attr('href', '#');
-      
-      var iconDiv = document.createElement('div');
-      $(iconDiv).addClass('icon');
-      $(a).append(iconDiv);
-			$(a).append(nodeArray[i].id);
-			$(list).append(a);
-			a.jobid = nodeArray[i].id;
-			this.listNodes[nodeArray[i].id] = a;
+			var iconDiv = document.createElement('div');
+			$(iconDiv).addClass('icon');
+			
+			$(a).append(iconDiv);
+			
+			var span = document.createElement("span");
+			$(span).text(nodeArray[i].id);
+			$(span).addClass("jobname");
+			$(a).append(span);
+			$(li).append(a);
+			$(ul).append(li);
+			
+			if (nodeArray[i].type == "flow") {
+				// Add the up down
+				var expandDiv = document.createElement("div");
+				$(expandDiv).addClass("expandarrow glyphicon glyphicon-chevron-down");
+				$(a).append(expandDiv);
+				
+				// Create subtree
+				var subul = this.renderTree(li, nodeArray[i], listNodeName + ":");
+				$(subul).hide();
+			}
 		}
 		
-		this.assignInitialStatus(self);
-		this.handleDisabledChange(self);
+		$(el).append(ul);
+		return ul;
+	},
+	
+	handleMenuExpand: function(li) {
+		var expandArrow = $(li).find("> a > .expandarrow");
+		var submenu = $(li).find("> ul");
+		
+		$(expandArrow).removeClass("glyphicon-chevron-down");
+		$(expandArrow).addClass("glyphicon-chevron-up");
+		$(submenu).slideDown();
+	},
+	
+	handleMenuCollapse: function(li) {
+		var expandArrow = $(li).find("> a > .expandarrow");
+		var submenu = $(li).find("> ul");
+		
+		$(expandArrow).removeClass("glyphicon-chevron-up");
+		$(expandArrow).addClass("glyphicon-chevron-down");
+		$(submenu).slideUp();
+	},
+	
+	handleToggleMenuExpand: function(evt) {
+		var expandarrow = evt.currentTarget;
+		var li = $(evt.currentTarget).closest("li.listElement");
+		var submenu = $(li).find("> ul");
+
+		if ($(submenu).is(":visible")) {
+			this.handleMenuCollapse(li);
+		}
+		else {
+			this.handleMenuExpand(li);
+		}
+		
+		evt.stopImmediatePropagation();
 	},
 	
 	handleContextMenuClick: function(evt) {
 		if (this.contextMenu) {
-			this.contextMenu(evt);
+			this.contextMenu(evt, this.model, evt.currentTarget.node);
 			return false;
 		}
 	},
 	
 	handleJobClick: function(evt) {
-		var jobid = evt.currentTarget.jobid;
-		if (!evt.currentTarget.jobid) {
+		console.log("Job clicked");
+		var li = $(evt.currentTarget).closest("li.listElement");
+		var node = li[0].node;
+		if (!node) {
 			return;
 		}
 		
 		if (this.model.has("selected")) {
 			var selected = this.model.get("selected");
-			if (selected == jobid) {
+			if (selected == node) {
 				this.model.unset("selected");
 			}
 			else {
-				this.model.set({"selected": jobid});
+				this.model.set({"selected": node});
 			}
 		}
 		else {
-			this.model.set({"selected": jobid});
+			this.model.set({"selected": node});
 		}
+		
+		evt.stopPropagation();
+		evt.cancelBubble = true;
 	},
 	
 	handleDisabledChange: function(evt) {
-		var disabledMap = this.model.get("disabled");
-		var nodes = this.model.get("nodes");
-		
-		for(var id in nodes) {
-			if (disabledMap[id]) {
-				$(this.listNodes[id]).addClass("nodedisabled");
+		this.changeDisabled(this.model.get('data'));
+	},
+	
+	changeDisabled: function(data) {
+		for (var i =0; i < data.nodes; ++i) {
+			var node = data.nodes[i];
+			if (node.disabled = true) {
+				removeClass(node.listElement, "nodedisabled");
+				if (node.type=='flow') {
+					this.changeDisabled(node);
+				}
 			}
 			else {
-				$(this.listNodes[id]).removeClass("nodedisabled");
+				addClass(node.listElement, "nodedisabled");
 			}
 		}
 	},
@@ -199,17 +288,42 @@ azkaban.JobListView = Backbone.View.extend({
 		
 		var previous = this.model.previous("selected");
 		var current = this.model.get("selected");
-		
+
 		if (previous) {
-			$(this.listNodes[previous]).removeClass("active");
+			$(previous.listElement).removeClass("active");
 		}
 		
 		if (current) {
-			$(this.listNodes[current]).addClass("active");
+			$(current.listElement).addClass("active");
+			this.propagateExpansion(current.listElement);
+		}
+	},
+	
+	propagateExpansion: function(li) {
+		var li = $(li).parent().closest("li.listElement")[0];
+		if (li) {
+			this.propagateExpansion(li);
+			this.handleMenuExpand(li);
 		}
 	},
 	
 	handleResetPanZoom: function(evt) {
 		this.model.trigger("resetPanZoom");
+	},
+	
+	handleAutoPanZoom: function(evt) {
+		var target = evt.currentTarget;
+		if ($(target).hasClass('btn-default')) {
+			$(target).removeClass('btn-default');
+			$(target).addClass('btn-info');
+		}
+		else if ($(target).hasClass('btn-info')) {
+			$(target).removeClass('btn-info');
+			$(target).addClass('btn-default');
+		}
+	 
+		// Using $().hasClass('active') does not use here because it appears that
+		// this is called before the Bootstrap toggle completes.
+		this.model.set({"autoPanZoom": $(target).hasClass('btn-info')});
 	}
 });
diff --git a/src/web/js/azkaban/view/job-details.js b/src/web/js/azkaban/view/job-details.js
index f191f37..ab77045 100644
--- a/src/web/js/azkaban/view/job-details.js
+++ b/src/web/js/azkaban/view/job-details.js
@@ -136,9 +136,15 @@ azkaban.JobSummaryView = Backbone.View.extend({
 		// Set up table column headers
 		var header = $("#hiveTableHeader");
 		var tr = document.createElement("tr");
-		var headers = ["Query","Job","Map","Reduce","HDFS Read","HDFS Write"];
+
+		var headers;
+		if (this.model.get("hasCumulativeCPU")) {
+			headers = ["Query","Job","Map","Reduce","Cumulative CPU","HDFS Read","HDFS Write"];
+		} else {
+			headers = ["Query","Job","Map","Reduce","HDFS Read","HDFS Write"];
+		}
+
 		var i;
-		
 		for (i = 0; i < headers.length; i++) {
 			var th = document.createElement("th");
 			$(th).text(headers[i]);
@@ -206,11 +212,11 @@ azkaban.JobTabView = Backbone.View.extend({
 
 	initialize: function(settings) {
 		var selectedView = settings.selectedView;
-		if (selectedView == 'joblog') {
-			this.handleJobLogViewLinkClick();
+		if (selectedView == 'summary') {
+			this.handleJobSummaryViewLinkClick();
 		}
 		else {
-			this.handleJobSummaryViewLinkClick();
+			this.handleJobLogViewLinkClick();
 		}
 	},
 
@@ -266,10 +272,10 @@ $(function() {
 
 	if (window.location.hash) {
 		var hash = window.location.hash;
-		if (hash == '#joblog') {
+		if (hash == '#logs') {
 			jobTabView.handleJobLogViewLinkClick();
 		}
-		else if (hash == '#jobsummary') {
+		else if (hash == '#summary') {
 			jobTabView.handleJobSummaryViewLinkClick();
 		}
 	}
diff --git a/src/web/js/azkaban/view/job-history.js b/src/web/js/azkaban/view/job-history.js
index a205d22..b07c27e 100644
--- a/src/web/js/azkaban/view/job-history.js
+++ b/src/web/js/azkaban/view/job-history.js
@@ -14,6 +14,7 @@
  * the License.
  */
 
+
 $.namespace('azkaban');
 
 var jobHistoryView;
@@ -35,4 +36,4 @@ $(function() {
 		model: dataModel,
     modelField: "data"
 	});
-});
+});
\ No newline at end of file
diff --git a/src/web/js/azkaban/view/main.js b/src/web/js/azkaban/view/main.js
index b6aac48..bf5e9a9 100644
--- a/src/web/js/azkaban/view/main.js
+++ b/src/web/js/azkaban/view/main.js
@@ -96,18 +96,12 @@ azkaban.ProjectTableView = Backbone.View.extend({
 		var requestURL = contextURL + "/manager?project=" + data.project + "&flow=";
 		for (var i = 0; i < flows.length; ++i) {
 			var id = flows[i].flowId;
-			var tr = document.createElement("tr");
-			var idtd = document.createElement("td");
-			$(idtd).addClass("tb-name");
-			
 			var ida = document.createElement("a");
 			ida.project = data.project;
 			$(ida).text(id);
 			$(ida).attr("href", requestURL + id);
-			
-			$(idtd).append(ida);
-			$(tr).append(idtd);
-			$(innerTable).append(tr);
+      $(ida).addClass('list-group-item');
+			$(innerTable).append(ida);
 		}
 	}
 });
diff --git a/src/web/js/azkaban/view/project.js b/src/web/js/azkaban/view/project.js
index ef01a0e..ccfeeb5 100644
--- a/src/web/js/azkaban/view/project.js
+++ b/src/web/js/azkaban/view/project.js
@@ -81,8 +81,8 @@ azkaban.FlowTableView = Backbone.View.extend({
 			var level = job.level;
 			var nodeId = flowId + "-" + name;
 	
-      var li = document.createElement('li');
-      $(li).addClass("list-group-item");
+			var li = document.createElement('li');
+			$(li).addClass("list-group-item");
 			$(li).attr("id", nodeId);
 			li.flowId = flowId;
 			li.dependents = job.dependents;
diff --git a/src/web/js/azkaban/view/svg-graph.js b/src/web/js/azkaban/view/svg-graph.js
index 67d2530..6910edc 100644
--- a/src/web/js/azkaban/view/svg-graph.js
+++ b/src/web/js/azkaban/view/svg-graph.js
@@ -13,115 +13,121 @@
  * License for the specific language governing permissions and limitations under
  * the License.
  */
+$.namespace('azkaban');
 
 azkaban.SvgGraphView = Backbone.View.extend({
 	events: {
-		"click g" : "clickGraph",
-		"contextmenu" : "handleRightClick",
-		"contextmenu g" : "handleRightClick",
-		"contextmenu polyline": "handleRightClick"
+		
 	},
-	
-  initialize: function(settings) {
+	initialize: function(settings) {
 		this.model.bind('change:selected', this.changeSelected, this);
+		this.model.bind('centerNode', this.centerNode, this);
 		this.model.bind('change:graph', this.render, this);
 		this.model.bind('resetPanZoom', this.resetPanZoom, this);
 		this.model.bind('change:update', this.handleStatusUpdate, this);
 		this.model.bind('change:disabled', this.handleDisabledChange, this);
 		this.model.bind('change:updateAll', this.handleUpdateAllStatus, this);
-		
-		this.graphMargin = settings.graphMargin ? settings.graphMargin : 200;
+		this.model.bind('expandFlow', this.expandFlow, this);
+		this.model.bind('collapseFlow', this.collapseFlow, this);
+
+		this.graphMargin = settings.graphMargin ? settings.graphMargin : 25;
 		this.svgns = "http://www.w3.org/2000/svg";
 		this.xlinksn = "http://www.w3.org/1999/xlink";
 		
 		var graphDiv = this.el[0];
 		var svg = $(this.el).find('svg')[0];
+		if (!svg) {
+			svg = this.el;
+		}
+
 		this.svgGraph = svg;
+		$(this.svgGraph).svg();
+		this.svg = $(svg).svg('get');
+
+		$(this.svgGraph).empty();
 		
+		// Create mainG node
 		var gNode = document.createElementNS(this.svgns, 'g');
+		gNode.setAttribute("class", "main graph");
 		svg.appendChild(gNode);
 		this.mainG = gNode;
+		
 		if (settings.rightClick) {
 			this.rightClick = settings.rightClick;
 		}
 
 		$(svg).svgNavigate();
-	},
-	
-  initializeDefs: function(self) {
-		var def = document.createElementNS(svgns, 'defs');
-		def.setAttributeNS(null, "id", "buttonDefs");
-
-		// ArrowHead
-		var arrowHeadMarker = document.createElementNS(svgns, 'marker');
-		arrowHeadMarker.setAttribute("id", "triangle");
-		arrowHeadMarker.setAttribute("viewBox", "0 0 10 10");
-		arrowHeadMarker.setAttribute("refX", "5");
-		arrowHeadMarker.setAttribute("refY", "5");
-		arrowHeadMarker.setAttribute("markerUnits", "strokeWidth");
-		arrowHeadMarker.setAttribute("markerWidth", "4");
-		arrowHeadMarker.setAttribute("markerHeight", "3");
-		arrowHeadMarker.setAttribute("orient", "auto");
-		var path = document.createElementNS(svgns, 'polyline');
-		arrowHeadMarker.appendChild(path);
-		path.setAttribute("points", "0,0 10,5 0,10 1,5");
-
-		def.appendChild(arrowHeadMarker);
-		
-		this.svgGraph.appendChild(def);
-	},
-	
-  render: function(self) {
-		console.log("graph render");
-
-		// Clean everything
-		while (this.mainG.lastChild) {
-			this.mainG.removeChild(this.mainG.lastChild);
+		
+		var self = this;
+		if (self.rightClick && self.rightClick.graph) {
+			$(svg).on("contextmenu", function(evt) {
+				console.log("graph click");
+				var currentTarget = evt.currentTarget;
+				
+				self.rightClick.graph(evt, self.model, currentTarget.data);
+				return false;
+			});
 		}
 
-		var data = this.model.get("data");
+		this.tooltipcontainer = settings.tooltipcontainer ? settings.tooltipcontainer : "body";
+		if (settings.render) {
+			this.render();
+		}
+	},
+	render: function() {
+		console.log("graph render");
+		$(this.mainG).empty();
+		
+		this.graphBounds = this.renderGraph(this.model.get("data"), this.mainG);
+		this.resetPanZoom(0);
+	},
+	renderGraph: function(data, g) {
+		g.data = data;
 		var nodes = data.nodes;
 		var edges = data.edges;
+		var nodeMap = data.nodeMap;
+
+		// Create a g node for edges, so that they're forced in the back.
+		var edgeG = this.svg.group(g);
 		if (nodes.length == 0) {
 			console.log("No results");
 			return;
 		};
-	
-		nodes.sort();
-		edges.sort();
-		// layout
-		layoutGraph(nodes, edges);
-		
-		var bounds = {};
-		this.nodes = {};
+
+		// Assign labels
 		for (var i = 0; i < nodes.length; ++i) {
-			this.nodes[nodes[i].id] = nodes[i];
+			nodes[i].label = nodes[i].id;
 		}
 		
-		for (var i = 0; i < edges.length; ++i) {
-			var inNodes = this.nodes[edges[i].target].inNodes;
-			if (!inNodes) {
-				inNodes = {};
-				this.nodes[edges[i].target].inNodes = inNodes;
-			}
-			inNodes[edges[i].from] = this.nodes[edges[i].from];
-			
-			var outNodes = this.nodes[edges[i].from].outNodes;
-			if (!outNodes) {
-				outNodes = {};
-				this.nodes[edges[i].from].outNodes = outNodes;
-			}
-			outNodes[edges[i].target] = this.nodes[edges[i].target];
-
-			this.drawEdge(this, edges[i]);
+		var self = this;
+		for (var i = 0; i < nodes.length; ++i) {
+			this.drawNode(this, nodes[i], g);
+			$(nodes[i].gNode).click(function(evt) {
+				var selected = self.model.get("selected");
+				if (selected == evt.currentTarget.data) {
+					self.model.unset("selected");
+				}
+				else {
+					self.model.set({"selected":evt.currentTarget.data});
+				}
+				
+				evt.stopPropagation();
+				evt.cancelBubble = true;
+			});
 		}
+
+		// layout
+		layoutGraph(nodes, edges, 10);
+		var bounds = this.calculateBounds(nodes);
+		this.moveNodes(nodes);
 		
-		this.gNodes = {};
-		for (var i = 0; i < nodes.length; ++i) {
-			this.drawNode(this, nodes[i], bounds);
+		for (var i = 0; i < edges.length; ++i) {
+			edges[i].toNode = nodeMap[edges[i].to];
+			edges[i].fromNode = nodeMap[edges[i].from];
+			this.drawEdge(this, edges[i], edgeG);
 		}
 		
-		this.model.set({"nodes": this.nodes, "edges": edges});
+		this.model.set({"flowId":data.flowId, "edges": edges});
 		
 		var margin = this.graphMargin;
 		bounds.minX = bounds.minX ? bounds.minX - margin : -margin;
@@ -129,82 +135,135 @@ azkaban.SvgGraphView = Backbone.View.extend({
 		bounds.maxX = bounds.maxX ? bounds.maxX + margin : margin;
 		bounds.maxY = bounds.maxY ? bounds.maxY + margin : margin;
 		
-		this.assignInitialStatus(self);
-		this.handleDisabledChange(self);
-		this.graphBounds = bounds;
-		this.resetPanZoom(0);
+		this.assignInitialStatus(this, data);
+
+		if (self.rightClick) {
+			if (self.rightClick.node) {
+				// Proper children selectors don't work properly on svg
+				for (var i = 0; i < nodes.length; ++i) {
+					$(nodes[i].gNode).on("contextmenu", function(evt) {
+						console.log("node click");
+						var currentTarget = evt.currentTarget;
+						self.rightClick.node(evt, self.model, currentTarget.data);
+						return false;
+					});
+				}
+			}
+			if (this.rightClick.graph) {
+				$(g).on("contextmenu", function(evt) {
+					console.log("graph click");
+					var currentTarget = evt.currentTarget;
+				
+					self.rightClick.graph(evt, self.model, currentTarget.data);
+					return false;
+				});
+			}
+		};
+		
+		$(".node").each( 
+				function(d,i){$(this).tooltip({container:self.tooltipcontainer, delay: {show: 500, hide: 100}});
+		});
+
+		return bounds;
 	},
-	
-  handleDisabledChange: function(evt) {
-		var disabledMap = this.model.get("disabled");
-
-		for(var id in this.nodes) {
-			 var g = this.gNodes[id];
-			if (disabledMap[id]) {
-				this.nodes[id].disabled = true;
-				addClass(g, "disabled");
+	handleDisabledChange: function(evt) {
+		this.changeDisabled(this.model.get('data'));
+	},
+	changeDisabled: function(data) {
+		for (var i =0; i < data.nodes.length; ++i) {
+			var node = data.nodes[i];
+			if (node.disabled) {
+				if (node.gNode) {
+					addClass(node.gNode, "nodeDisabled");
+					$(node.gNode).attr("title", "DISABLED (" + node.type + ")").tooltip('fixTitle');
+				}
 			}
 			else {
-				this.nodes[id].disabled = false;
-				removeClass(g, "disabled");
+				if (node.gNode) {
+					removeClass(node.gNode, "nodeDisabled");
+					$(node.gNode).attr("title", node.status + " (" + node.type + ")").tooltip('fixTitle');
+				}
+				if (node.type=='flow') {
+					this.changeDisabled(node);
+				}
 			}
 		}
 	},
-	
-  assignInitialStatus: function(evt) {
-		var data = this.model.get("data");
+	assignInitialStatus: function(evt, data) {
 		for (var i = 0; i < data.nodes.length; ++i) {
 			var updateNode = data.nodes[i];
-			var g = this.gNodes[updateNode.id];
-			addClass(g, updateNode.status);
+			var g = updateNode.gNode;
+			var initialStatus = updateNode.status ? updateNode.status : "READY";
+			
+			addClass(g, initialStatus);
+			var title = initialStatus + " (" + updateNode.type + ")";
+			
+			if (updateNode.disabled) {
+				addClass(g, "nodeDisabled");
+				title = "DISABLED (" + updateNode.type + ")";
+			}
+			$(g).attr("title", title);
 		}
 	},
-	
-  changeSelected: function(self) {
+	changeSelected: function(self) {
 		console.log("change selected");
 		var selected = this.model.get("selected");
 		var previous = this.model.previous("selected");
 		
 		if (previous) {
 			// Unset previous
-			var g = this.gNodes[previous];
-			removeClass(g, "selected");
+			removeClass(previous.gNode, "selected");
 		}
 		
 		if (selected) {
-			var g = this.gNodes[selected];
-			var node = this.nodes[selected];
-			
+			this.propagateExpansion(selected);
+			var g = selected.gNode;
 			addClass(g, "selected");
 			
-			var offset = 200;
-			var widthHeight = offset*2;
-			var x = node.x - offset;
-			var y = node.y - offset;
-			
-			$(this.svgGraph).svgNavigate("transformToBox", {
-				x: x, 
-				y: y, 
-				width: widthHeight, 
-				height: widthHeight
-			});
+			console.log(this.model.get("autoPanZoom"));
+			if (this.model.get("autoPanZoom")) {
+				this.centerNode(selected);
+			}
 		}
 	},
-	
-  handleStatusUpdate: function(evt) {
+	propagateExpansion: function(node) {
+		if (node.parent.type) {
+			this.propagateExpansion(node.parent);
+			this.expandFlow(node.parent);
+		}
+	},
+	handleStatusUpdate: function(evt) {
 		var updateData = this.model.get("update");
+		var data = this.model.get("data");
+		this.updateStatusChanges(updateData, data);
+	},
+	updateStatusChanges: function(updateData, data) {
+		// Assumes all changes have been applied.
 		if (updateData.nodes) {
+			var nodeMap = data.nodeMap;
 			for (var i = 0; i < updateData.nodes.length; ++i) {
-				var updateNode = updateData.nodes[i];
-				
-				var g = this.gNodes[updateNode.id];
-				this.handleRemoveAllStatus(g);
+				var node = updateData.nodes[i];
+				var nodeToUpdate = nodeMap[node.id];
 				
-				addClass(g, updateNode.status);
+				var g = nodeToUpdate.gNode;
+				if (g) {
+					this.handleRemoveAllStatus(g);
+					addClass(g, nodeToUpdate.status);
+					
+					var title = nodeToUpdate.status + " (" + nodeToUpdate.type + ")";
+					if (nodeToUpdate.disabled) {
+						addClass(g, "nodeDisabled");
+						title = "DISABLED (" + nodeToUpdate.type + ")";
+					}
+					$(g).attr("title", title).tooltip('fixTitle');
+					
+					if (node.nodes) {
+						this.updateStatusChanges(node, nodeToUpdate);
+					}
+				}
 			}
 		}
 	},
-	
   handleRemoveAllStatus: function(gNode) {
 		for (var j = 0; j < statusList.length; ++j) {
 			var status = statusList[j];
@@ -212,150 +271,361 @@ azkaban.SvgGraphView = Backbone.View.extend({
 		}
 	},
 	
-  clickGraph: function(self) {
-		console.log("click");
-		if (self.currentTarget.jobid) {
-			this.model.set({"selected": self.currentTarget.jobid});
-		}
-	},
-	
   handleRightClick: function(self) {
 		if (this.rightClick) {
 			var callbacks = this.rightClick;
 			var currentTarget = self.currentTarget;
 			if (callbacks.node && currentTarget.jobid) {
-				callbacks.node(self);
+				callbacks.node(self, this.model, currentTarget.nodeobj);
 			}
-			else if (callbacks.edge && 
-					(currentTarget.nodeName == "polyline" || 
-					 currentTarget.nodeName == "line")) {
-				callbacks.edge(self);
+			else if (callbacks.edge && (currentTarget.nodeName == "polyline" || currentTarget.nodeName == "line")) {
+				callbacks.edge(self, this.model);
 			}
 			else if (callbacks.graph) {
-				callbacks.graph(self);
+				callbacks.graph(self, this.model);
 			}
 			return false;
 		}
 	
 		return true;
 	},	
-	
-  drawEdge: function(self, edge) {
-		var svg = self.svgGraph;
+	drawEdge: function(self, edge, g) {
+		var svg = this.svg;
 		var svgns = self.svgns;
 		
-		var startNode = this.nodes[edge.from];
-		var endNode = this.nodes[edge.target];
+		var startNode = edge.fromNode;
+		var endNode = edge.toNode;
 		
-		if (edge.guides) {
-			var pointString = "" + startNode.x + "," + startNode.y + " ";
+		var startPointY = startNode.y + startNode.height/2;
+		var endPointY = endNode.y - endNode.height/2;
 
+		if (edge.guides) {
+			// Create guide array
+			var pointArray = new Array();
+			pointArray.push([startNode.x, startPointY]);
 			for (var i = 0; i < edge.guides.length; ++i ) {
-				edgeGuidePoint = edge.guides[i];
-				pointString += edgeGuidePoint.x + "," + edgeGuidePoint.y + " ";
+				var edgeGuidePoint = edge.guides[i];
+				pointArray.push([edgeGuidePoint.x, edgeGuidePoint.y]);
 			}
-			
-			pointString += endNode.x + "," + endNode.y;
-			var polyLine = document.createElementNS(svgns, "polyline");
-			polyLine.setAttributeNS(null, "class", "edge");
-			polyLine.setAttributeNS(null, "points", pointString);
-			polyLine.setAttributeNS(null, "style", "fill:none;");
-			self.mainG.appendChild(polyLine);
+			pointArray.push([endNode.x, endPointY]);
+		
+			edge.line = svg.polyline(g, pointArray, {class:"edge", fill:"none"});
+			edge.line.data = edge;
+			edge.oldpoints = pointArray;
 		}
 		else { 
-			var line = document.createElementNS(svgns, 'line');
-			line.setAttributeNS(null, "class", "edge");
-			line.setAttributeNS(null, "x1", startNode.x);
-			line.setAttributeNS(null, "y1", startNode.y);
-			line.setAttributeNS(null, "x2", endNode.x);
-			line.setAttributeNS(null, "y2", endNode.y);
+			edge.line = svg.line(g, startNode.x, startPointY, endNode.x, endPointY, {class:"edge"});
+			edge.line.data = edge;
+		}
+	},
+	drawNode: function(self, node, g) {
+		if (node.type == 'flow') {
+			this.drawFlowNode(self, node, g);
+		}
+		else {
+			this.drawBoxNode(self, node, g);
+		}
+	},
+	moveNodes: function(nodes) {
+		var svg = this.svg;
+		for (var i = 0; i < nodes.length; ++i) {
+			var node = nodes[i];
+			var gNode = node.gNode;
 			
-			self.mainG.appendChild(line);
+			svg.change(gNode, {"transform": translateStr(node.x, node.y)});
 		}
 	},
+	expandFlow: function(node) {
+		var svg = this.svg;
+		var gnode = node.gNode;
+		node.expanded = true;
+
+		var innerG = gnode.innerG;
+		var borderRect = innerG.borderRect;
+		var labelG = innerG.labelG;
+		
+		var bbox;
+		if (!innerG.expandedFlow) {
+			var topmargin= 30, bottommargin=5;
+			var hmargin = 10;
+		
+			var expandedFlow = svg.group(innerG, "", {class: "expandedGraph"});
+			this.renderGraph(node, expandedFlow);
+			innerG.expandedFlow = expandedFlow;
+			removeClass(innerG, "collapsed");
+			addClass(innerG, "expanded");
+			node.expandedWidth = node.width;
+			node.expandedHeight = node.height;
+		}
+		else {
+			$(innerG.expandedFlow).show();
+			removeClass(innerG, "collapsed");
+			addClass(innerG, "expanded");
+			node.width = node.expandedWidth;
+			node.height = node.expandedHeight;		
+		}
+		
+		this.relayoutFlow(node);
+		
+		var bounds = this.calculateBounds(this.model.get("data").nodes);
+
+		var margin = this.graphMargin;
+		bounds.minX = bounds.minX ? bounds.minX - margin : -margin;
+		bounds.minY = bounds.minY ? bounds.minY - margin : -margin;
+		bounds.maxX = bounds.maxX ? bounds.maxX + margin : margin;
+		bounds.maxY = bounds.maxY ? bounds.maxY + margin : margin;
+		this.graphBounds = bounds;
+	},
+	collapseFlow: function(node) {
+		console.log("Collapsing flow");
+		var svg = this.svg;
+		var gnode = node.gNode;
+		node.expanded = false;
 	
-  drawNode: function(self, node, bounds) {
-		var svg = self.svgGraph;
-		var svgns = self.svgns;
+		var innerG = gnode.innerG;
+		var borderRect = innerG.borderRect;
+		var labelG = innerG.labelG;
+
+		removeClass(innerG, "expanded");
+		addClass(innerG, "collapsed");
+
+		node.height = node.collapsedHeight;
+		node.width = node.collapsedWidth;
 
-		var xOffset = 10;
-		var yOffset = 10;
-		
-		var nodeG = document.createElementNS(svgns, "g");
-		nodeG.setAttributeNS(null, "class", "jobnode");
-		nodeG.setAttributeNS(null, "font-family", "helvetica");
-		nodeG.setAttributeNS(null, "transform", "translate(" + node.x + "," + node.y + ")");
-		this.gNodes[node.id] = nodeG;
-		
-		var innerG = document.createElementNS(svgns, "g");
-		innerG.setAttributeNS(null, "transform", "translate(-10,-10)");
-		
-		var circle = document.createElementNS(svgns, 'circle');
-		circle.setAttributeNS(null, "cy", 10);
-		circle.setAttributeNS(null, "cx", 10);
-		circle.setAttributeNS(null, "r", 12);
-		circle.setAttributeNS(null, "style", "width:inherit;stroke-opacity:1");
-		
-		var text = document.createElementNS(svgns, 'text');
-		var textLabel = document.createTextNode(node.label);
-		text.appendChild(textLabel);
-		text.setAttributeNS(null, "x", 4);
-		text.setAttributeNS(null, "y", 15);
-		text.setAttributeNS(null, "height", 10); 
+		$(innerG.expandedFlow).hide();
+		this.relayoutFlow(node);
+
+		var bounds = this.calculateBounds(this.model.get("data").nodes);
+
+		var margin = this.graphMargin;
+		bounds.minX = bounds.minX ? bounds.minX - margin : -margin;
+		bounds.minY = bounds.minY ? bounds.minY - margin : -margin;
+		bounds.maxX = bounds.maxX ? bounds.maxX + margin : margin;
+		bounds.maxY = bounds.maxY ? bounds.maxY + margin : margin;
+		this.graphBounds = bounds;
+
+	},
+	relayoutFlow: function(node) {
+		if (node.expanded) {
+			this.layoutExpandedFlowNode(node);
+		}
+
+		var parent = node.parent;
+		if (parent) {
+			layoutGraph(parent.nodes, parent.edges, 10);
+			this.relayoutFlow(parent);
+			// Move all points again.
+			this.moveNodeEdges(parent.nodes, parent.edges);
+			this.animateExpandedFlowNode(node, 250);
+		}
+	},
+	moveNodeEdges: function(nodes, edges) {
+		var svg = this.svg;
+		
+		for (var i = 0; i < nodes.length; ++i) {
+			var node = nodes[i];
+			var gNode = node.gNode;
+			
+			$(gNode).animate({"svgTransform": translateStr(node.x, node.y)}, 250);
+		}
+			
+		for (var j = 0; j < edges.length; ++j) {
+			var edge = edges[j];
+			var startNode = edge.fromNode;
+			var endNode = edge.toNode;
+			var line = edge.line;
 				
-		this.addBounds(bounds, {
-			minX: node.x - xOffset, 
-			minY: node.y - yOffset, 
-			maxX: node.x + xOffset, 
-			maxY: node.y + yOffset
-		});
+			var startPointY = startNode.y + startNode.height/2;
+			var endPointY = endNode.y - endNode.height/2;
+			
+			if (edge.guides) {
+				// Create guide array
+				var pointArray = new Array();
+				pointArray.push([startNode.x, startPointY]);
+				for (var i = 0; i < edge.guides.length; ++i ) {
+					var edgeGuidePoint = edge.guides[i];
+					pointArray.push([edgeGuidePoint.x, edgeGuidePoint.y]);
+				}
+				pointArray.push([endNode.x, endPointY]);
+				
+				animatePolylineEdge(svg, edge, pointArray, 250);
+				edge.oldpoints = pointArray;
+			}
+			else {
+				$(line).animate({svgX1: startNode.x, svgY1: startPointY, svgX2: endNode.x, svgY2: endPointY});
+			}
+		}
+
+	},
+	calculateBounds: function(nodes) {
+		var bounds = {};
+		var node = nodes[0];
+		bounds.minX = node.x - 10;
+		bounds.minY = node.y - 10;
+		bounds.maxX = node.x + 10;
+		bounds.maxY = node.y + 10;
+
+		for (var i = 0; i < nodes.length; ++i) {
+			node = nodes[i];
+			var centerX = node.width/2;
+			var centerY = node.height/2;
+
+			var minX = node.x - centerX;
+			var minY = node.y - centerY;
+			var maxX = node.x + centerX;
+			var maxY = node.y + centerY; 
+			
+			bounds.minX = Math.min(bounds.minX, minX);
+			bounds.minY = Math.min(bounds.minY, minY);
+			bounds.maxX = Math.max(bounds.maxX, maxX);
+			bounds.maxY = Math.max(bounds.maxY, maxY);
+		}
+		bounds.width = bounds.maxX - bounds.minX;
+		bounds.height = bounds.maxY - bounds.minY;
 		
-		var backRect = document.createElementNS(svgns, 'rect');
-		backRect.setAttributeNS(null, "x", 0);
-		backRect.setAttributeNS(null, "y", 2);
-		backRect.setAttributeNS(null, "class", "backboard");
-		backRect.setAttributeNS(null, "width", 10);
-		backRect.setAttributeNS(null, "height", 15);
-		
-		innerG.appendChild(circle);
-		innerG.appendChild(backRect);
-		innerG.appendChild(text);
-		innerG.jobid = node.id;
-
-		nodeG.appendChild(innerG);
-		self.mainG.appendChild(nodeG);
-
-		// Need to get text width after attaching to SVG.
-		var computeText = text.getComputedTextLength();
-		var halfWidth = computeText/2;
-		text.setAttributeNS(null, "x", -halfWidth + 10);
-		backRect.setAttributeNS(null, "x", -halfWidth);
-		backRect.setAttributeNS(null, "width", computeText + 20);
-
-		nodeG.setAttributeNS(null, "class", "node");
-		nodeG.jobid=node.id;
+		return bounds;
 	},
-	
-  addBounds: function(toBounds, addBounds) {
-		toBounds.minX = toBounds.minX 
-				? Math.min(toBounds.minX, addBounds.minX) : addBounds.minX;
-		toBounds.minY = toBounds.minY 
-				? Math.min(toBounds.minY, addBounds.minY) : addBounds.minY;
-		toBounds.maxX = toBounds.maxX 
-				? Math.max(toBounds.maxX, addBounds.maxX) : addBounds.maxX;
-		toBounds.maxY = toBounds.maxY 
-				? Math.max(toBounds.maxY, addBounds.maxY) : addBounds.maxY;
+	drawBoxNode: function(self, node, g) {
+		var svg = this.svg;
+		var horizontalMargin = 8;
+		var verticalMargin = 2;
+		
+		var nodeG = svg.group(g, "", {class:"node jobnode"});
+		
+		var innerG = svg.group(nodeG, "", {class:"nodebox"});
+		var borderRect = svg.rect(innerG, 0, 0, 10, 10, 3, 3, {class: "border"});
+		var jobNameText = svg.text(innerG, horizontalMargin, 16, node.label);
+		nodeG.innerG = innerG;
+		innerG.borderRect = borderRect;		
+
+		var labelBBox = jobNameText.getBBox();
+		var totalWidth = labelBBox.width + 2*horizontalMargin;
+		var totalHeight = labelBBox.height + 2*verticalMargin;
+		svg.change(borderRect, {width: totalWidth, height: totalHeight});
+		svg.change(jobNameText, {y: (totalHeight + labelBBox.height)/2 - 3});
+		svg.change(innerG, {transform: translateStr(-totalWidth/2, -totalHeight/2)});
+		
+		node.width=totalWidth;
+		node.height=totalHeight;
+		
+		node.gNode = nodeG;
+		nodeG.data = node;
 	},
-	
-  resetPanZoom: function(duration) {
+	drawFlowNode: function(self, node, g) {
+		var svg = this.svg;
+		
+		// Base flow node
+		var nodeG = svg.group(g, "", {"class": "node flownode"});
+		
+		// Create all the elements
+		var innerG = svg.group(nodeG, "", {class: "nodebox collapsed"});
+		var borderRect = svg.rect(innerG, 0, 0, 10, 10, 3, 3, {class: "flowborder"});
+		
+		// Create label
+		var labelG = svg.group(innerG);
+		var iconHeight = 20;
+		var iconWidth = 21;
+		var textOffset = iconWidth + 4;
+		var jobNameText = svg.text(labelG, textOffset, 1, node.label);
+		var flowIdText = svg.text(labelG, textOffset, 11, node.flowId, {"font-size": 8})
+		var tempLabelG = labelG.getBBox();
+		var iconImage = svg.image(labelG, 0, -iconHeight/2, iconWidth, iconHeight, contextURL + "/images/graph-icon.png", {}); 
+
+		// Assign key values to make searching quicker
+		node.gNode=nodeG;
+		nodeG.data=node;
+
+		// Do this because jquery svg selectors don't work
+		nodeG.innerG = innerG;
+		innerG.borderRect = borderRect;
+		innerG.labelG = labelG;
+
+		// Layout everything in the node
+		this.layoutFlowNode(self, node);
+	},
+	layoutFlowNode: function(self, node) {
+		var svg = this.svg;
+		var horizontalMargin = 8;
+		var verticalMargin = 2;
+
+		var gNode = node.gNode;
+		var innerG = gNode.innerG;
+		var borderRect = innerG.borderRect;
+		var labelG = innerG.labelG;
+
+		var labelBBox = labelG.getBBox();
+		var totalWidth = labelBBox.width + 2*horizontalMargin;
+		var totalHeight = labelBBox.height + 2*verticalMargin;
+
+		svg.change(labelG, {transform: translateStr(horizontalMargin, labelBBox.height/2 + verticalMargin)});
+		svg.change(innerG, {transform: translateStr(-totalWidth/2, -totalHeight/2)});
+		svg.change(borderRect, {width: totalWidth, height: totalHeight});
+
+		node.height = totalHeight;
+		node.width = totalWidth;
+		node.collapsedHeight = totalHeight;
+		node.collapsedWidth = totalWidth;
+	},
+	layoutExpandedFlowNode: function(node) {
+		var svg = this.svg;
+		var topmargin= 30, bottommargin=5;
+		var hmargin = 10;
+		
+		var gNode = node.gNode;
+		var innerG = gNode.innerG;
+		var borderRect = innerG.borderRect;
+		var labelG = innerG.labelG;
+		var expandedFlow = innerG.expandedFlow;
+		
+		var bound = this.calculateBounds(node.nodes);
+		
+		node.height = bound.height + topmargin + bottommargin;
+		node.width = bound.width + hmargin*2;
+		svg.change(expandedFlow, {transform: translateStr(-bound.minX + hmargin, -bound.minY + topmargin)});
+		//$(innerG).animate({svgTransform: translateStr(-node.width/2, -node.height/2)}, 50);
+		//$(borderRect).animate({svgWidth: node.width, svgHeight: node.height}, 50);
+	},
+	animateExpandedFlowNode: function(node, time) {
+		var gNode = node.gNode;
+		var innerG = gNode.innerG;
+		var borderRect = innerG.borderRect;
+
+		$(innerG).animate({svgTransform: translateStr(-node.width/2, -node.height/2)}, time);
+		$(borderRect).animate({svgWidth: node.width, svgHeight: node.height}, time);
+		$(borderRect).animate({svgFill: 'white'}, time);
+	},
+	resetPanZoom: function(duration) {
 		var bounds = this.graphBounds;
-		var param = {
-			x: bounds.minX, 
-			y: bounds.minY, 
-			width: (bounds.maxX - bounds.minX), 
-			height: (bounds.maxY - bounds.minY), 
-			duration: duration 
-		};
-		$(this.svgGraph).svgNavigate("transformToBox", param);
+		var param = {x: bounds.minX, y: bounds.minY, width: (bounds.maxX - bounds.minX), height: (bounds.maxY - bounds.minY), duration: duration };
+
+		this.panZoom(param);
+	},
+	centerNode: function(node) {
+		// The magic of affine transformation. 
+		// Multiply the inverse root matrix with the current matrix to get the node position.
+		// Rather do this than to traverse backwards through the scene graph.
+		var ctm = node.gNode.getCTM();
+		var transform = node.gNode.getTransformToElement();
+		var globalCTM = this.mainG.getCTM().inverse();
+		var otherTransform = globalCTM.multiply(ctm);
+		// Also a beauty of affine transformation. The translate is always the left most column of the matrix.
+		var x = otherTransform.e - node.width/2;
+		var y = otherTransform.f - node.height/2;
+
+		this.panZoom({x: x, y: y, width: node.width, height: node.height});
+	},
+	globalNodePosition: function(gNode) {
+		if (node.parent) {
+		
+			var parentPos = this.globalNodePosition(node.parent);
+			return {x: parentPos.x + node.x, y: parentPos.y + node.y};
+		}
+		else {
+			return {x: node.x, y: node.y};		
+		}
+	},
+	panZoom: function(params) {
+		params.maxScale = 2;
+		$(this.svgGraph).svgNavigate("transformToBox", params);
 	}
 });
diff --git a/src/web/js/azkaban/view/time-graph.js b/src/web/js/azkaban/view/time-graph.js
index 122cdad..c56a39f 100644
--- a/src/web/js/azkaban/view/time-graph.js
+++ b/src/web/js/azkaban/view/time-graph.js
@@ -24,102 +24,100 @@ azkaban.TimeGraphView = Backbone.View.extend({
 		this.model.bind('render', this.render, this);
 		this.model.bind('change:page', this.render, this);
     this.modelField = settings.modelField;
+    this.graphContainer = settings.el;
     this.render();
 	},
-	
+
 	render: function(self) {
-		var data = this.model.get(this.modelField);
-    if (data == null) {
+		var series = this.model.get(this.modelField);
+    if (series == null) {
       return;
     }
-	
-		var margin = {
-			top: 20, 
-			right: 20, 
-			bottom: 30, 
-			left: 70
-		};
-	  var width = $(this.el).width() - margin.left - margin.right;
-	  var height = 300 - margin.top - margin.bottom;
-	    
-		var x = d3.time.scale()
-		    .range([0, width]);
-		
-		var y = d3.scale.linear()
-		    .range([height, 0]);
-	    
-		var xAxis = d3.svg.axis()
-				.scale(x)
-				.orient("bottom");
 
-		var yAxis = d3.svg.axis()
-		    .scale(y)
-		    .orient("left");
-		yAxis.tickFormat(
-			function(d) {
-				return formatDuration(d, 1);
-			}
-		);
-		
-		var line = d3.svg.line()
-		    .x(function(d) { return x(d.startTime); })
-		    .y(function(d) { return y(d.endTime - d.startTime); });
-		 
-		var svg = d3.select("#timeGraph").append("svg")
-		    .attr("width", width + margin.left + margin.right)
-		    .attr("height", height + margin.top + margin.bottom)
-		    .append("g")
-		    .attr("transform", "translate(" + margin.left + "," + margin.top + ")");
-		  
-		var xextent = d3.extent(data, function(d) {
-			return d.startTime;
-		});
-		var diff = (xextent[1] - xextent[0])*0.05;
-		
-		xextent[0] -= diff;
-		xextent[1] += diff;
-		x.domain(xextent);
-		
-		var yextent = d3.extent(data, function(d) {
-			return d.endTime - d.startTime;
-		});
-		var upperYbound = yextent[1]*1.25;
-		y.domain([0, upperYbound]);
-	
-		svg.append("g")
-				.attr("class", "x axis")
-				.attr("transform", "translate(0," + height + ")")
-				.call(xAxis);
-	
-		svg.append("g")
-				.attr("class", "y axis")
-				.call(yAxis)
-				.append("text")
-				.attr("transform", "rotate(-90)")
-				.attr("y", 6)
-				.attr("dy", ".71em")
-				.style("text-anchor", "end")
-				.text("Duration");
-	
-		svg.append("path")
-				.datum(data)
-				.attr("class", "line")
-				.attr("d", line);
-				
-		var node = svg.selectAll("g.node")
-				.data(data)
-				.attr("class", "node")
-				.enter().append("g")
-				.attr("transform",  function(d) {
-			return "translate(" + x(d.startTime) + "," + y(d.endTime-d.startTime) + ")";
-		});
-		
-		node.append("circle")
-				.attr("r", 5)
-				.attr("class", function(d) {return d.status;})
-				.append("svg:title")
-				.text(function(d) {
-			return d.execId + ":" + d.flowId + " ran in " + getDuration(d.startTime, d.endTime);
-		});
+    // Array of points to be passed to Morris.
+    var data = [];
+    
+    // Map of y value to index for faster look-up in the lineColorsCallback to
+    // get the status for each point.
+    var indexMap = {};
+	  for (var i = 0; i < series.length; ++i) {
+      if (series[i].startTime == null || series[i].endTime == null) {
+        console.log("Each element in series must have startTime and endTime");
+        return;
+      }
+      var startTime = series[i].startTime;
+      var endTime = series[i].endTime;
+      if (endTime == -1) {
+        endTime = new Date().getTime();
+      }
+      var duration = endTime - startTime;
+      data.push({ 
+        time: endTime,
+        duration: duration
+      });
+
+      indexMap[endTime.toString()] = i;
+    }
+
+		if (data.length == 0) {
+			$(this.graphContainer).hide();
+			return;
+		}
+
+		var graphDiv = document.createElement('div');
+		$(this.graphContainer).append(graphDiv);
+
+    var lineColorsCallback = function(row, sidx, type) {
+      if (type != 'point') {
+        return "#000000";
+      }
+      var i = indexMap[row.x.toString()];
+      var status = series[i].status;
+      if (status == 'SKIPPED') {
+        return '#aaa';
+      }
+      else if (status == 'SUCCEEDED') {
+        return '#4e911e';
+      }
+      else if (status == 'RUNNING') {
+        return '#009fc9';
+      }
+      else if (status == 'PAUSED') {
+        return '#c92123';
+      }
+      else if (status == 'FAILED' || 
+          status == 'FAILED_FINISHING' || 
+          status == 'KILLED') {
+        return '#cc0000';
+      }
+      else {
+        return '#ccc';
+      }
+    };
+
+    var yLabelFormatCallback = function(y) {
+      var seconds = y / 1000.0;
+      return seconds.toString() + " s";
+    };
+
+    var hoverCallback = function(index, options, content) {
+      // Note: series contains the data points in descending order and index
+      // is the index into Morris's internal array of data sorted in ascending
+      // x order.
+      var status = series[options.data.length - index - 1].status;
+      return content + 
+          '<div class="morris-hover-point">Status: ' + status + '</div>';
+    };
+
+    Morris.Line({
+      element: graphDiv,
+      data: data,
+      xkey: 'time',
+      ykeys: ['duration'],
+      labels: ['Duration'],
+      lineColors: lineColorsCallback,
+      yLabelFormat: yLabelFormatCallback,
+      hoverCallback: hoverCallback
+    });
 	}
 });

src/web/js/jquery.svg.js 1394(+1394 -0)

diff --git a/src/web/js/jquery.svg.js b/src/web/js/jquery.svg.js
new file mode 100644
index 0000000..dcbf95f
--- /dev/null
+++ b/src/web/js/jquery.svg.js
@@ -0,0 +1,1394 @@
+/* http://keith-wood.name/svg.html
+   SVG for jQuery v1.4.5.
+   Written by Keith Wood (kbwood{at}iinet.com.au) August 2007.
+   Dual licensed under the GPL (http://dev.jquery.com/browser/trunk/jquery/GPL-LICENSE.txt) and 
+   MIT (http://dev.jquery.com/browser/trunk/jquery/MIT-LICENSE.txt) licenses. 
+   Please attribute the author if you use it. */
+
+(function($) { // Hide scope, no $ conflict
+
+/* SVG manager.
+   Use the singleton instance of this class, $.svg, 
+   to interact with the SVG functionality. */
+function SVGManager() {
+	this._settings = []; // Settings to be remembered per SVG object
+	this._extensions = []; // List of SVG extensions added to SVGWrapper
+		// for each entry [0] is extension name, [1] is extension class (function)
+		// the function takes one parameter - the SVGWrapper instance
+	this.regional = []; // Localisations, indexed by language, '' for default (English)
+	this.regional[''] = {errorLoadingText: 'Error loading',
+		notSupportedText: 'This browser does not support SVG'};
+	this.local = this.regional['']; // Current localisation
+	this._uuid = new Date().getTime();
+	this._renesis = detectActiveX('RenesisX.RenesisCtrl');
+}
+
+/* Determine whether a given ActiveX control is available.
+   @param  classId  (string) the ID for the ActiveX control
+   @return  (boolean) true if found, false if not */
+function detectActiveX(classId) {
+	try {
+		return !!(window.ActiveXObject && new ActiveXObject(classId));
+	}
+	catch (e) {
+		return false;
+	}
+}
+
+var PROP_NAME = 'svgwrapper';
+
+$.extend(SVGManager.prototype, {
+	/* Class name added to elements to indicate already configured with SVG. */
+	markerClassName: 'hasSVG',
+
+	/* SVG namespace. */
+	svgNS: 'http://www.w3.org/2000/svg',
+	/* XLink namespace. */
+	xlinkNS: 'http://www.w3.org/1999/xlink',
+
+	/* SVG wrapper class. */
+	_wrapperClass: SVGWrapper,
+
+	/* Camel-case versions of attribute names containing dashes or are reserved words. */
+	_attrNames: {class_: 'class', in_: 'in',
+		alignmentBaseline: 'alignment-baseline', baselineShift: 'baseline-shift',
+		clipPath: 'clip-path', clipRule: 'clip-rule',
+		colorInterpolation: 'color-interpolation',
+		colorInterpolationFilters: 'color-interpolation-filters',
+		colorRendering: 'color-rendering', dominantBaseline: 'dominant-baseline',
+		enableBackground: 'enable-background', fillOpacity: 'fill-opacity',
+		fillRule: 'fill-rule', floodColor: 'flood-color',
+		floodOpacity: 'flood-opacity', fontFamily: 'font-family',
+		fontSize: 'font-size', fontSizeAdjust: 'font-size-adjust',
+		fontStretch: 'font-stretch', fontStyle: 'font-style',
+		fontVariant: 'font-variant', fontWeight: 'font-weight',
+		glyphOrientationHorizontal: 'glyph-orientation-horizontal',
+		glyphOrientationVertical: 'glyph-orientation-vertical',
+		horizAdvX: 'horiz-adv-x', horizOriginX: 'horiz-origin-x',
+		imageRendering: 'image-rendering', letterSpacing: 'letter-spacing',
+		lightingColor: 'lighting-color', markerEnd: 'marker-end',
+		markerMid: 'marker-mid', markerStart: 'marker-start',
+		stopColor: 'stop-color', stopOpacity: 'stop-opacity',
+		strikethroughPosition: 'strikethrough-position',
+		strikethroughThickness: 'strikethrough-thickness',
+		strokeDashArray: 'stroke-dasharray', strokeDashOffset: 'stroke-dashoffset',
+		strokeLineCap: 'stroke-linecap', strokeLineJoin: 'stroke-linejoin',
+		strokeMiterLimit: 'stroke-miterlimit', strokeOpacity: 'stroke-opacity',
+		strokeWidth: 'stroke-width', textAnchor: 'text-anchor',
+		textDecoration: 'text-decoration', textRendering: 'text-rendering',
+		underlinePosition: 'underline-position', underlineThickness: 'underline-thickness',
+		vertAdvY: 'vert-adv-y', vertOriginY: 'vert-origin-y',
+		wordSpacing: 'word-spacing', writingMode: 'writing-mode'},
+
+	/* Add the SVG object to its container. */
+	_attachSVG: function(container, settings) {
+		var svg = (container.namespaceURI == this.svgNS ? container : null);
+		var container = (svg ? null : container);
+		if ($(container || svg).hasClass(this.markerClassName)) {
+			return;
+		}
+		if (typeof settings == 'string') {
+			settings = {loadURL: settings};
+		}
+		else if (typeof settings == 'function') {
+			settings = {onLoad: settings};
+		}
+		$(container || svg).addClass(this.markerClassName);
+		try {
+			if (!svg) {
+				svg = document.createElementNS(this.svgNS, 'svg');
+				svg.setAttribute('version', '1.1');
+				if (container.clientWidth > 0) {
+					svg.setAttribute('width', container.clientWidth);
+				}
+				if (container.clientHeight > 0) {
+					svg.setAttribute('height', container.clientHeight);
+				}
+				container.appendChild(svg);
+			}
+			this._afterLoad(container, svg, settings || {});
+		}
+		catch (e) {
+			if ($.browser.msie) {
+				if (!container.id) {
+					container.id = 'svg' + (this._uuid++);
+				}
+				this._settings[container.id] = settings;
+				container.innerHTML = '<embed type="image/svg+xml" width="100%" ' +
+					'height="100%" src="' + (settings.initPath || '') + 'blank.svg" ' +
+					'pluginspage="http://www.adobe.com/svg/viewer/install/main.html"/>';
+			}
+			else {
+				container.innerHTML = '<p class="svg_error">' +
+					this.local.notSupportedText + '</p>';
+			}
+		}
+	},
+
+	/* SVG callback after loading - register SVG root. */
+	_registerSVG: function() {
+		for (var i = 0; i < document.embeds.length; i++) { // Check all
+			var container = document.embeds[i].parentNode;
+			if (!$(container).hasClass($.svg.markerClassName) || // Not SVG
+					$.data(container, PROP_NAME)) { // Already done
+				continue;
+			}
+			var svg = null;
+			try {
+				svg = document.embeds[i].getSVGDocument();
+			}
+			catch(e) {
+				setTimeout($.svg._registerSVG, 250); // Renesis takes longer to load
+				return;
+			}
+			svg = (svg ? svg.documentElement : null);
+			if (svg) {
+				$.svg._afterLoad(container, svg);
+			}
+		}
+	},
+
+	/* Post-processing once loaded. */
+	_afterLoad: function(container, svg, settings) {
+		var settings = settings || this._settings[container.id];
+		this._settings[container ? container.id : ''] = null;
+		var wrapper = new this._wrapperClass(svg, container);
+		$.data(container || svg, PROP_NAME, wrapper);
+		try {
+			if (settings.loadURL) { // Load URL
+				wrapper.load(settings.loadURL, settings);
+			}
+			if (settings.settings) { // Additional settings
+				wrapper.configure(settings.settings);
+			}
+			if (settings.onLoad && !settings.loadURL) { // Onload callback
+				settings.onLoad.apply(container || svg, [wrapper]);
+			}
+		}
+		catch (e) {
+			alert(e);
+		}
+	},
+
+	/* Return the SVG wrapper created for a given container.
+	   @param  container  (string) selector for the container or
+	                      (element) the container for the SVG object or
+	                      jQuery collection - first entry is the container
+	   @return  (SVGWrapper) the corresponding SVG wrapper element, or null if not attached */
+	_getSVG: function(container) {
+		container = (typeof container == 'string' ? $(container)[0] :
+			(container.jquery ? container[0] : container));
+		return $.data(container, PROP_NAME);
+	},
+
+	/* Remove the SVG functionality from a div.
+	   @param  container  (element) the container for the SVG object */
+	_destroySVG: function(container) {
+		var $container = $(container);
+		if (!$container.hasClass(this.markerClassName)) {
+			return;
+		}
+		$container.removeClass(this.markerClassName);
+		if (container.namespaceURI != this.svgNS) {
+			$container.empty();
+		}
+		$.removeData(container, PROP_NAME);
+	},
+
+	/* Extend the SVGWrapper object with an embedded class.
+	   The constructor function must take a single parameter that is
+	   a reference to the owning SVG root object. This allows the 
+	   extension to access the basic SVG functionality.
+	   @param  name      (string) the name of the SVGWrapper attribute to access the new class
+	   @param  extClass  (function) the extension class constructor */
+	addExtension: function(name, extClass) {
+		this._extensions.push([name, extClass]);
+	},
+
+	/* Does this node belong to SVG?
+	   @param  node  (element) the node to be tested
+	   @return  (boolean) true if an SVG node, false if not */
+	isSVGElem: function(node) {
+		return (node.nodeType == 1 && node.namespaceURI == $.svg.svgNS);
+	}
+});
+
+/* The main SVG interface, which encapsulates the SVG element.
+   Obtain a reference from $().svg('get') */
+function SVGWrapper(svg, container) {
+	this._svg = svg; // The SVG root node
+	this._container = container; // The containing div
+	for (var i = 0; i < $.svg._extensions.length; i++) {
+		var extension = $.svg._extensions[i];
+		this[extension[0]] = new extension[1](this);
+	}
+}
+
+$.extend(SVGWrapper.prototype, {
+
+	/* Retrieve the width of the SVG object. */
+	_width: function() {
+		return (this._container ? this._container.clientWidth : this._svg.width);
+	},
+
+	/* Retrieve the height of the SVG object. */
+	_height: function() {
+		return (this._container ? this._container.clientHeight : this._svg.height);
+	},
+
+	/* Retrieve the root SVG element.
+	   @return  the top-level SVG element */
+	root: function() {
+		return this._svg;
+	},
+
+	/* Configure a SVG node.
+	   @param  node      (element, optional) the node to configure
+	   @param  settings  (object) additional settings for the root
+	   @param  clear     (boolean) true to remove existing attributes first,
+	                     false to add to what is already there (optional)
+	   @return  (SVGWrapper) this root */
+	configure: function(node, settings, clear) {
+		if (!node.nodeName) {
+			clear = settings;
+			settings = node;
+			node = this._svg;
+		}
+		if (clear) {
+			for (var i = node.attributes.length - 1; i >= 0; i--) {
+				var attr = node.attributes.item(i);
+				if (!(attr.nodeName == 'onload' || attr.nodeName == 'version' || 
+						attr.nodeName.substring(0, 5) == 'xmlns')) {
+					node.attributes.removeNamedItem(attr.nodeName);
+				}
+			}
+		}
+		for (var attrName in settings) {
+			node.setAttribute($.svg._attrNames[attrName] || attrName, settings[attrName]);
+		}
+		return this;
+	},
+
+	/* Locate a specific element in the SVG document.
+	   @param  id  (string) the element's identifier
+	   @return  (element) the element reference, or null if not found */
+	getElementById: function(id) {
+		return this._svg.ownerDocument.getElementById(id);
+	},
+
+	/* Change the attributes for a SVG node.
+	   @param  element   (SVG element) the node to change
+	   @param  settings  (object) the new settings
+	   @return  (SVGWrapper) this root */
+	change: function(element, settings) {
+		if (element) {
+			for (var name in settings) {
+				if (settings[name] == null) {
+					element.removeAttribute($.svg._attrNames[name] || name);
+				}
+				else {
+					element.setAttribute($.svg._attrNames[name] || name, settings[name]);
+				}
+			}
+		}
+		return this;
+	},
+
+	/* Check for parent being absent and adjust arguments accordingly. */
+	_args: function(values, names, optSettings) {
+		names.splice(0, 0, 'parent');
+		names.splice(names.length, 0, 'settings');
+		var args = {};
+		var offset = 0;
+		if (values[0] != null && values[0].jquery) {
+			values[0] = values[0][0];
+		}
+		if (values[0] != null && !(typeof values[0] == 'object' && values[0].nodeName)) {
+			args['parent'] = null;
+			offset = 1;
+		}
+		for (var i = 0; i < values.length; i++) {
+			args[names[i + offset]] = values[i];
+		}
+		if (optSettings) {
+			$.each(optSettings, function(i, value) {
+				if (typeof args[value] == 'object') {
+					args.settings = args[value];
+					args[value] = null;
+				}
+			});
+		}
+		return args;
+	},
+
+	/* Add a title.
+	   @param  parent    (element or jQuery) the parent node for the new title (optional)
+	   @param  text      (string) the text of the title
+	   @param  settings  (object) additional settings for the title (optional)
+	   @return  (element) the new title node */
+	title: function(parent, text, settings) {
+		var args = this._args(arguments, ['text']);
+		var node = this._makeNode(args.parent, 'title', args.settings || {});
+		node.appendChild(this._svg.ownerDocument.createTextNode(args.text));
+		return node;
+	},
+
+	/* Add a description.
+	   @param  parent    (element or jQuery) the parent node for the new description (optional)
+	   @param  text      (string) the text of the description
+	   @param  settings  (object) additional settings for the description (optional)
+	   @return  (element) the new description node */
+	describe: function(parent, text, settings) {
+		var args = this._args(arguments, ['text']);
+		var node = this._makeNode(args.parent, 'desc', args.settings || {});
+		node.appendChild(this._svg.ownerDocument.createTextNode(args.text));
+		return node;
+	},
+
+	/* Add a definitions node.
+	   @param  parent    (element or jQuery) the parent node for the new definitions (optional)
+	   @param  id        (string) the ID of this definitions (optional)
+	   @param  settings  (object) additional settings for the definitions (optional)
+	   @return  (element) the new definitions node */
+	defs: function(parent, id, settings) {
+		var args = this._args(arguments, ['id'], ['id']);
+		return this._makeNode(args.parent, 'defs', $.extend(
+			(args.id ? {id: args.id} : {}), args.settings || {}));
+	},
+
+	/* Add a symbol definition.
+	   @param  parent    (element or jQuery) the parent node for the new symbol (optional)
+	   @param  id        (string) the ID of this symbol
+	   @param  x1        (number) the left coordinate for this symbol
+	   @param  y1        (number) the top coordinate for this symbol
+	   @param  width     (number) the width of this symbol
+	   @param  height    (number) the height of this symbol
+	   @param  settings  (object) additional settings for the symbol (optional)
+	   @return  (element) the new symbol node */
+	symbol: function(parent, id, x1, y1, width, height, settings) {
+		var args = this._args(arguments, ['id', 'x1', 'y1', 'width', 'height']);
+		return this._makeNode(args.parent, 'symbol', $.extend({id: args.id,
+			viewBox: args.x1 + ' ' + args.y1 + ' ' + args.width + ' ' + args.height},
+			args.settings || {}));
+	},
+
+	/* Add a marker definition.
+	   @param  parent    (element or jQuery) the parent node for the new marker (optional)
+	   @param  id        (string) the ID of this marker
+	   @param  refX      (number) the x-coordinate for the reference point
+	   @param  refY      (number) the y-coordinate for the reference point
+	   @param  mWidth    (number) the marker viewport width
+	   @param  mHeight   (number) the marker viewport height
+	   @param  orient    (string or int) 'auto' or angle (degrees) (optional)
+	   @param  settings  (object) additional settings for the marker (optional)
+	   @return  (element) the new marker node */
+	marker: function(parent, id, refX, refY, mWidth, mHeight, orient, settings) {
+		var args = this._args(arguments, ['id', 'refX', 'refY',
+			'mWidth', 'mHeight', 'orient'], ['orient']);
+		return this._makeNode(args.parent, 'marker', $.extend(
+			{id: args.id, refX: args.refX, refY: args.refY, markerWidth: args.mWidth, 
+			markerHeight: args.mHeight, orient: args.orient || 'auto'}, args.settings || {}));
+	},
+
+	/* Add a style node.
+	   @param  parent    (element or jQuery) the parent node for the new node (optional)
+	   @param  styles    (string) the CSS styles
+	   @param  settings  (object) additional settings for the node (optional)
+	   @return  (element) the new style node */
+	style: function(parent, styles, settings) {
+		var args = this._args(arguments, ['styles']);
+		var node = this._makeNode(args.parent, 'style', $.extend(
+			{type: 'text/css'}, args.settings || {}));
+		node.appendChild(this._svg.ownerDocument.createTextNode(args.styles));
+		if ($.browser.opera) {
+			$('head').append('<style type="text/css">' + args.styles + '</style>');
+		}
+		return node;
+	},
+
+	/* Add a script node.
+	   @param  parent    (element or jQuery) the parent node for the new node (optional)
+	   @param  script    (string) the JavaScript code
+	   @param  type      (string) the MIME type for the code (optional, default 'text/javascript')
+	   @param  settings  (object) additional settings for the node (optional)
+	   @return  (element) the new script node */
+	script: function(parent, script, type, settings) {
+		var args = this._args(arguments, ['script', 'type'], ['type']);
+		var node = this._makeNode(args.parent, 'script', $.extend(
+			{type: args.type || 'text/javascript'}, args.settings || {}));
+		node.appendChild(this._svg.ownerDocument.createTextNode(args.script));
+		if (!$.browser.mozilla) {
+			$.globalEval(args.script);
+		}
+		return node;
+	},
+
+	/* Add a linear gradient definition.
+	   Specify all of x1, y1, x2, y2 or none of them.
+	   @param  parent    (element or jQuery) the parent node for the new gradient (optional)
+	   @param  id        (string) the ID for this gradient
+	   @param  stops     (string[][]) the gradient stops, each entry is
+	                     [0] is offset (0.0-1.0 or 0%-100%), [1] is colour, 
+						 [2] is opacity (optional)
+	   @param  x1        (number) the x-coordinate of the gradient start (optional)
+	   @param  y1        (number) the y-coordinate of the gradient start (optional)
+	   @param  x2        (number) the x-coordinate of the gradient end (optional)
+	   @param  y2        (number) the y-coordinate of the gradient end (optional)
+	   @param  settings  (object) additional settings for the gradient (optional)
+	   @return  (element) the new gradient node */
+	linearGradient: function(parent, id, stops, x1, y1, x2, y2, settings) {
+		var args = this._args(arguments,
+			['id', 'stops', 'x1', 'y1', 'x2', 'y2'], ['x1']);
+		var sets = $.extend({id: args.id}, 
+			(args.x1 != null ? {x1: args.x1, y1: args.y1, x2: args.x2, y2: args.y2} : {}));
+		return this._gradient(args.parent, 'linearGradient', 
+			$.extend(sets, args.settings || {}), args.stops);
+	},
+
+	/* Add a radial gradient definition.
+	   Specify all of cx, cy, r, fx, fy or none of them.
+	   @param  parent    (element or jQuery) the parent node for the new gradient (optional)
+	   @param  id        (string) the ID for this gradient
+	   @param  stops     (string[][]) the gradient stops, each entry
+	                     [0] is offset, [1] is colour, [2] is opacity (optional)
+	   @param  cx        (number) the x-coordinate of the largest circle centre (optional)
+	   @param  cy        (number) the y-coordinate of the largest circle centre (optional)
+	   @param  r         (number) the radius of the largest circle (optional)
+	   @param  fx        (number) the x-coordinate of the gradient focus (optional)
+	   @param  fy        (number) the y-coordinate of the gradient focus (optional)
+	   @param  settings  (object) additional settings for the gradient (optional)
+	   @return  (element) the new gradient node */
+	radialGradient: function(parent, id, stops, cx, cy, r, fx, fy, settings) {
+		var args = this._args(arguments,
+			['id', 'stops', 'cx', 'cy', 'r', 'fx', 'fy'], ['cx']);
+		var sets = $.extend({id: args.id}, (args.cx != null ?
+			{cx: args.cx, cy: args.cy, r: args.r, fx: args.fx, fy: args.fy} : {}));
+		return this._gradient(args.parent, 'radialGradient', 
+			$.extend(sets, args.settings || {}), args.stops);
+	},
+
+	/* Add a gradient node. */
+	_gradient: function(parent, name, settings, stops) {
+		var node = this._makeNode(parent, name, settings);
+		for (var i = 0; i < stops.length; i++) {
+			var stop = stops[i];
+			this._makeNode(node, 'stop', $.extend(
+				{offset: stop[0], stopColor: stop[1]}, 
+				(stop[2] != null ? {stopOpacity: stop[2]} : {})));
+		}
+		return node;
+	},
+
+	/* Add a pattern definition.
+	   Specify all of vx, vy, xwidth, vheight or none of them.
+	   @param  parent    (element or jQuery) the parent node for the new pattern (optional)
+	   @param  id        (string) the ID for this pattern
+	   @param  x         (number) the x-coordinate for the left edge of the pattern
+	   @param  y         (number) the y-coordinate for the top edge of the pattern
+	   @param  width     (number) the width of the pattern
+	   @param  height    (number) the height of the pattern
+	   @param  vx        (number) the minimum x-coordinate for view box (optional)
+	   @param  vy        (number) the minimum y-coordinate for the view box (optional)
+	   @param  vwidth    (number) the width of the view box (optional)
+	   @param  vheight   (number) the height of the view box (optional)
+	   @param  settings  (object) additional settings for the pattern (optional)
+	   @return  (element) the new pattern node */
+	pattern: function(parent, id, x, y, width, height, vx, vy, vwidth, vheight, settings) {
+		var args = this._args(arguments, ['id', 'x', 'y', 'width', 'height',
+			'vx', 'vy', 'vwidth', 'vheight'], ['vx']);
+		var sets = $.extend({id: args.id, x: args.x, y: args.y,
+			width: args.width, height: args.height}, (args.vx != null ?
+			{viewBox: args.vx + ' ' + args.vy + ' ' + args.vwidth + ' ' + args.vheight} : {}));
+		return this._makeNode(args.parent, 'pattern', $.extend(sets, args.settings || {}));
+	},
+
+	/* Add a clip path definition.
+	   @param  parent  (element) the parent node for the new element (optional)
+	   @param  id      (string) the ID for this path
+	   @param  units   (string) either 'userSpaceOnUse' (default) or 'objectBoundingBox' (optional)
+	   @return  (element) the new clipPath node */
+	clipPath: function(parent, id, units, settings) {
+		var args = this._args(arguments, ['id', 'units']);
+		args.units = args.units || 'userSpaceOnUse';
+		return this._makeNode(args.parent, 'clipPath', $.extend(
+			{id: args.id, clipPathUnits: args.units}, args.settings || {}));
+	},
+
+	/* Add a mask definition.
+	   @param  parent    (element or jQuery) the parent node for the new mask (optional)
+	   @param  id        (string) the ID for this mask
+	   @param  x         (number) the x-coordinate for the left edge of the mask
+	   @param  y         (number) the y-coordinate for the top edge of the mask
+	   @param  width     (number) the width of the mask
+	   @param  height    (number) the height of the mask
+	   @param  settings  (object) additional settings for the mask (optional)
+	   @return  (element) the new mask node */
+	mask: function(parent, id, x, y, width, height, settings) {
+		var args = this._args(arguments, ['id', 'x', 'y', 'width', 'height']);
+		return this._makeNode(args.parent, 'mask', $.extend(
+			{id: args.id, x: args.x, y: args.y, width: args.width, height: args.height},
+			args.settings || {}));
+	},
+
+	/* Create a new path object.
+	   @return  (SVGPath) a new path object */
+	createPath: function() {
+		return new SVGPath();
+	},
+
+	/* Create a new text object.
+	   @return  (SVGText) a new text object */
+	createText: function() {
+		return new SVGText();
+	},
+
+	/* Add an embedded SVG element.
+	   Specify all of vx, vy, vwidth, vheight or none of them.
+	   @param  parent    (element or jQuery) the parent node for the new node (optional)
+	   @param  x         (number) the x-coordinate for the left edge of the node
+	   @param  y         (number) the y-coordinate for the top edge of the node
+	   @param  width     (number) the width of the node
+	   @param  height    (number) the height of the node
+	   @param  vx        (number) the minimum x-coordinate for view box (optional)
+	   @param  vy        (number) the minimum y-coordinate for the view box (optional)
+	   @param  vwidth    (number) the width of the view box (optional)
+	   @param  vheight   (number) the height of the view box (optional)
+	   @param  settings  (object) additional settings for the node (optional)
+	   @return  (element) the new node */
+	svg: function(parent, x, y, width, height, vx, vy, vwidth, vheight, settings) {
+		var args = this._args(arguments, ['x', 'y', 'width', 'height',
+			'vx', 'vy', 'vwidth', 'vheight'], ['vx']);
+		var sets = $.extend({x: args.x, y: args.y, width: args.width, height: args.height}, 
+			(args.vx != null ? {viewBox: args.vx + ' ' + args.vy + ' ' +
+			args.vwidth + ' ' + args.vheight} : {}));
+		return this._makeNode(args.parent, 'svg', $.extend(sets, args.settings || {}));
+	},
+
+	/* Create a group.
+	   @param  parent    (element or jQuery) the parent node for the new group (optional)
+	   @param  id        (string) the ID of this group (optional)
+	   @param  settings  (object) additional settings for the group (optional)
+	   @return  (element) the new group node */
+	group: function(parent, id, settings) {
+		var args = this._args(arguments, ['id'], ['id']);
+		return this._makeNode(args.parent, 'g', $.extend({id: args.id}, args.settings || {}));
+	},
+
+	/* Add a usage reference.
+	   Specify all of x, y, width, height or none of them.
+	   @param  parent    (element or jQuery) the parent node for the new node (optional)
+	   @param  x         (number) the x-coordinate for the left edge of the node (optional)
+	   @param  y         (number) the y-coordinate for the top edge of the node (optional)
+	   @param  width     (number) the width of the node (optional)
+	   @param  height    (number) the height of the node (optional)
+	   @param  ref       (string) the ID of the definition node
+	   @param  settings  (object) additional settings for the node (optional)
+	   @return  (element) the new node */
+	use: function(parent, x, y, width, height, ref, settings) {
+		var args = this._args(arguments, ['x', 'y', 'width', 'height', 'ref']);
+		if (typeof args.x == 'string') {
+			args.ref = args.x;
+			args.settings = args.y;
+			args.x = args.y = args.width = args.height = null;
+		}
+		var node = this._makeNode(args.parent, 'use', $.extend(
+			{x: args.x, y: args.y, width: args.width, height: args.height},
+			args.settings || {}));
+		node.setAttributeNS($.svg.xlinkNS, 'href', args.ref);
+		return node;
+	},
+
+	/* Add a link, which applies to all child elements.
+	   @param  parent    (element or jQuery) the parent node for the new link (optional)
+	   @param  ref       (string) the target URL
+	   @param  settings  (object) additional settings for the link (optional)
+	   @return  (element) the new link node */
+	link: function(parent, ref, settings) {
+		var args = this._args(arguments, ['ref']);
+		var node = this._makeNode(args.parent, 'a', args.settings);
+		node.setAttributeNS($.svg.xlinkNS, 'href', args.ref);
+		return node;
+	},
+
+	/* Add an image.
+	   @param  parent    (element or jQuery) the parent node for the new image (optional)
+	   @param  x         (number) the x-coordinate for the left edge of the image
+	   @param  y         (number) the y-coordinate for the top edge of the image
+	   @param  width     (number) the width of the image
+	   @param  height    (number) the height of the image
+	   @param  ref       (string) the path to the image
+	   @param  settings  (object) additional settings for the image (optional)
+	   @return  (element) the new image node */
+	image: function(parent, x, y, width, height, ref, settings) {
+		var args = this._args(arguments, ['x', 'y', 'width', 'height', 'ref']);
+		var node = this._makeNode(args.parent, 'image', $.extend(
+			{x: args.x, y: args.y, width: args.width, height: args.height},
+			args.settings || {}));
+		node.setAttributeNS($.svg.xlinkNS, 'href', args.ref);
+		return node;
+	},
+
+	/* Draw a path.
+	   @param  parent    (element or jQuery) the parent node for the new shape (optional)
+	   @param  path      (string or SVGPath) the path to draw
+	   @param  settings  (object) additional settings for the shape (optional)
+	   @return  (element) the new shape node */
+	path: function(parent, path, settings) {
+		var args = this._args(arguments, ['path']);
+		return this._makeNode(args.parent, 'path', $.extend(
+			{d: (args.path.path ? args.path.path() : args.path)}, args.settings || {}));
+	},
+
+	/* Draw a rectangle.
+	   Specify both of rx and ry or neither.
+	   @param  parent    (element or jQuery) the parent node for the new shape (optional)
+	   @param  x         (number) the x-coordinate for the left edge of the rectangle
+	   @param  y         (number) the y-coordinate for the top edge of the rectangle
+	   @param  width     (number) the width of the rectangle
+	   @param  height    (number) the height of the rectangle
+	   @param  rx        (number) the x-radius of the ellipse for the rounded corners (optional)
+	   @param  ry        (number) the y-radius of the ellipse for the rounded corners (optional)
+	   @param  settings  (object) additional settings for the shape (optional)
+	   @return  (element) the new shape node */
+	rect: function(parent, x, y, width, height, rx, ry, settings) {
+		var args = this._args(arguments, ['x', 'y', 'width', 'height', 'rx', 'ry'], ['rx']);
+		return this._makeNode(args.parent, 'rect', $.extend(
+			{x: args.x, y: args.y, width: args.width, height: args.height},
+			(args.rx ? {rx: args.rx, ry: args.ry} : {}), args.settings || {}));
+	},
+
+	/* Draw a circle.
+	   @param  parent    (element or jQuery) the parent node for the new shape (optional)
+	   @param  cx        (number) the x-coordinate for the centre of the circle
+	   @param  cy        (number) the y-coordinate for the centre of the circle
+	   @param  r         (number) the radius of the circle
+	   @param  settings  (object) additional settings for the shape (optional)
+	   @return  (element) the new shape node */
+	circle: function(parent, cx, cy, r, settings) {
+		var args = this._args(arguments, ['cx', 'cy', 'r']);
+		return this._makeNode(args.parent, 'circle', $.extend(
+			{cx: args.cx, cy: args.cy, r: args.r}, args.settings || {}));
+	},
+
+	/* Draw an ellipse.
+	   @param  parent    (element or jQuery) the parent node for the new shape (optional)
+	   @param  cx        (number) the x-coordinate for the centre of the ellipse
+	   @param  cy        (number) the y-coordinate for the centre of the ellipse
+	   @param  rx        (number) the x-radius of the ellipse
+	   @param  ry        (number) the y-radius of the ellipse
+	   @param  settings  (object) additional settings for the shape (optional)
+	   @return  (element) the new shape node */
+	ellipse: function(parent, cx, cy, rx, ry, settings) {
+		var args = this._args(arguments, ['cx', 'cy', 'rx', 'ry']);
+		return this._makeNode(args.parent, 'ellipse', $.extend(
+			{cx: args.cx, cy: args.cy, rx: args.rx, ry: args.ry}, args.settings || {}));
+	},
+
+	/* Draw a line.
+	   @param  parent    (element or jQuery) the parent node for the new shape (optional)
+	   @param  x1        (number) the x-coordinate for the start of the line
+	   @param  y1        (number) the y-coordinate for the start of the line
+	   @param  x2        (number) the x-coordinate for the end of the line
+	   @param  y2        (number) the y-coordinate for the end of the line
+	   @param  settings  (object) additional settings for the shape (optional)
+	   @return  (element) the new shape node */
+	line: function(parent, x1, y1, x2, y2, settings) {
+		var args = this._args(arguments, ['x1', 'y1', 'x2', 'y2']);
+		return this._makeNode(args.parent, 'line', $.extend(
+			{x1: args.x1, y1: args.y1, x2: args.x2, y2: args.y2}, args.settings || {}));
+	},
+
+	/* Draw a polygonal line.
+	   @param  parent    (element or jQuery) the parent node for the new shape (optional)
+	   @param  points    (number[][]) the x-/y-coordinates for the points on the line
+	   @param  settings  (object) additional settings for the shape (optional)
+	   @return  (element) the new shape node */
+	polyline: function(parent, points, settings) {
+		var args = this._args(arguments, ['points']);
+		return this._poly(args.parent, 'polyline', args.points, args.settings);
+	},
+
+	/* Draw a polygonal shape.
+	   @param  parent    (element or jQuery) the parent node for the new shape (optional)
+	   @param  points    (number[][]) the x-/y-coordinates for the points on the shape
+	   @param  settings  (object) additional settings for the shape (optional)
+	   @return  (element) the new shape node */
+	polygon: function(parent, points, settings) {
+		var args = this._args(arguments, ['points']);
+		return this._poly(args.parent, 'polygon', args.points, args.settings);
+	},
+
+	/* Draw a polygonal line or shape. */
+	_poly: function(parent, name, points, settings) {
+		var ps = '';
+		for (var i = 0; i < points.length; i++) {
+			ps += points[i].join() + ' ';
+		}
+		return this._makeNode(parent, name, $.extend(
+			{points: $.trim(ps)}, settings || {}));
+	},
+
+	/* Draw text.
+	   Specify both of x and y or neither of them.
+	   @param  parent    (element or jQuery) the parent node for the text (optional)
+	   @param  x         (number or number[]) the x-coordinate(s) for the text (optional)
+	   @param  y         (number or number[]) the y-coordinate(s) for the text (optional)
+	   @param  value     (string) the text content or
+	                     (SVGText) text with spans and references
+	   @param  settings  (object) additional settings for the text (optional)
+	   @return  (element) the new text node */
+	text: function(parent, x, y, value, settings) {
+		var args = this._args(arguments, ['x', 'y', 'value']);
+		if (typeof args.x == 'string' && arguments.length < 4) {
+			args.value = args.x;
+			args.settings = args.y;
+			args.x = args.y = null;
+		}
+		return this._text(args.parent, 'text', args.value, $.extend(
+			{x: (args.x && isArray(args.x) ? args.x.join(' ') : args.x),
+			y: (args.y && isArray(args.y) ? args.y.join(' ') : args.y)}, 
+			args.settings || {}));
+	},
+
+	/* Draw text along a path.
+	   @param  parent    (element or jQuery) the parent node for the text (optional)
+	   @param  path      (string) the ID of the path
+	   @param  value     (string) the text content or
+	                     (SVGText) text with spans and references
+	   @param  settings  (object) additional settings for the text (optional)
+	   @return  (element) the new text node */
+	textpath: function(parent, path, value, settings) {
+		var args = this._args(arguments, ['path', 'value']);
+		var node = this._text(args.parent, 'textPath', args.value, args.settings || {});
+		node.setAttributeNS($.svg.xlinkNS, 'href', args.path);
+		return node;
+	},
+
+	/* Draw text. */
+	_text: function(parent, name, value, settings) {
+		var node = this._makeNode(parent, name, settings);
+		if (typeof value == 'string') {
+			node.appendChild(node.ownerDocument.createTextNode(value));
+		}
+		else {
+			for (var i = 0; i < value._parts.length; i++) {
+				var part = value._parts[i];
+				if (part[0] == 'tspan') {
+					var child = this._makeNode(node, part[0], part[2]);
+					child.appendChild(node.ownerDocument.createTextNode(part[1]));
+					node.appendChild(child);
+				}
+				else if (part[0] == 'tref') {
+					var child = this._makeNode(node, part[0], part[2]);
+					child.setAttributeNS($.svg.xlinkNS, 'href', part[1]);
+					node.appendChild(child);
+				}
+				else if (part[0] == 'textpath') {
+					var set = $.extend({}, part[2]);
+					set.href = null;
+					var child = this._makeNode(node, part[0], set);
+					child.setAttributeNS($.svg.xlinkNS, 'href', part[2].href);
+					child.appendChild(node.ownerDocument.createTextNode(part[1]));
+					node.appendChild(child);
+				}
+				else { // straight text
+					node.appendChild(node.ownerDocument.createTextNode(part[1]));
+				}
+			}
+		}
+		return node;
+	},
+
+	/* Add a custom SVG element.
+	   @param  parent    (element or jQuery) the parent node for the new element (optional)
+	   @param  name      (string) the name of the element
+	   @param  settings  (object) additional settings for the element (optional)
+	   @return  (element) the new custom node */
+	other: function(parent, name, settings) {
+		var args = this._args(arguments, ['name']);
+		return this._makeNode(args.parent, args.name, args.settings || {});
+	},
+
+	/* Create a shape node with the given settings. */
+	_makeNode: function(parent, name, settings) {
+		parent = parent || this._svg;
+		var node = this._svg.ownerDocument.createElementNS($.svg.svgNS, name);
+		for (var name in settings) {
+			var value = settings[name];
+			if (value != null && value != null && 
+					(typeof value != 'string' || value != '')) {
+				node.setAttribute($.svg._attrNames[name] || name, value);
+			}
+		}
+		parent.appendChild(node);
+		return node;
+	},
+
+	/* Add an existing SVG node to the diagram.
+	   @param  parent  (element or jQuery) the parent node for the new node (optional)
+	   @param  node    (element) the new node to add or
+	                   (string) the jQuery selector for the node or
+	                   (jQuery collection) set of nodes to add
+	   @return  (SVGWrapper) this wrapper */
+	add: function(parent, node) {
+		var args = this._args((arguments.length == 1 ? [null, parent] : arguments), ['node']);
+		var svg = this;
+		args.parent = args.parent || this._svg;
+		args.node = (args.node.jquery ? args.node : $(args.node));
+		try {
+			if ($.svg._renesis) {
+				throw 'Force traversal';
+			}
+			args.parent.appendChild(args.node.cloneNode(true));
+		}
+		catch (e) {
+			args.node.each(function() {
+				var child = svg._cloneAsSVG(this);
+				if (child) {
+					args.parent.appendChild(child);
+				}
+			});
+		}
+		return this;
+	},
+
+	/* Clone an existing SVG node and add it to the diagram.
+	   @param  parent  (element or jQuery) the parent node for the new node (optional)
+	   @param  node    (element) the new node to add or
+	                   (string) the jQuery selector for the node or
+	                   (jQuery collection) set of nodes to add
+	   @return  (element[]) collection of new nodes */
+	clone: function(parent, node) {
+		var svg = this;
+		var args = this._args((arguments.length == 1 ? [null, parent] : arguments), ['node']);
+		args.parent = args.parent || this._svg;
+		args.node = (args.node.jquery ? args.node : $(args.node));
+		var newNodes = [];
+		args.node.each(function() {
+			var child = svg._cloneAsSVG(this);
+			if (child) {
+				child.id = '';
+				args.parent.appendChild(child);
+				newNodes.push(child);
+			}
+		});
+		return newNodes;
+	},
+
+	/* SVG nodes must belong to the SVG namespace, so clone and ensure this is so.
+	   @param  node  (element) the SVG node to clone
+	   @return  (element) the cloned node */
+	_cloneAsSVG: function(node) {
+		var newNode = null;
+		if (node.nodeType == 1) { // element
+			newNode = this._svg.ownerDocument.createElementNS(
+				$.svg.svgNS, this._checkName(node.nodeName));
+			for (var i = 0; i < node.attributes.length; i++) {
+				var attr = node.attributes.item(i);
+				if (attr.nodeName != 'xmlns' && attr.nodeValue) {
+					if (attr.prefix == 'xlink') {
+						newNode.setAttributeNS($.svg.xlinkNS,
+							attr.localName || attr.baseName, attr.nodeValue);
+					}
+					else {
+						newNode.setAttribute(this._checkName(attr.nodeName), attr.nodeValue);
+					}
+				}
+			}
+			for (var i = 0; i < node.childNodes.length; i++) {
+				var child = this._cloneAsSVG(node.childNodes[i]);
+				if (child) {
+					newNode.appendChild(child);
+				}
+			}
+		}
+		else if (node.nodeType == 3) { // text
+			if ($.trim(node.nodeValue)) {
+				newNode = this._svg.ownerDocument.createTextNode(node.nodeValue);
+			}
+		}
+		else if (node.nodeType == 4) { // CDATA
+			if ($.trim(node.nodeValue)) {
+				try {
+					newNode = this._svg.ownerDocument.createCDATASection(node.nodeValue);
+				}
+				catch (e) {
+					newNode = this._svg.ownerDocument.createTextNode(
+						node.nodeValue.replace(/&/g, '&amp;').
+						replace(/</g, '&lt;').replace(/>/g, '&gt;'));
+				}
+			}
+		}
+		return newNode;
+	},
+
+	/* Node names must be lower case and without SVG namespace prefix. */
+	_checkName: function(name) {
+		name = (name.substring(0, 1) >= 'A' && name.substring(0, 1) <= 'Z' ?
+			name.toLowerCase() : name);
+		return (name.substring(0, 4) == 'svg:' ? name.substring(4) : name);
+	},
+
+	/* Load an external SVG document.
+	   @param  url       (string) the location of the SVG document or
+	                     the actual SVG content
+	   @param  settings  (boolean) see addTo below or
+	                     (function) see onLoad below or
+	                     (object) additional settings for the load with attributes below:
+	                       addTo       (boolean) true to add to what's already there,
+	                                   or false to clear the canvas first
+						   changeSize  (boolean) true to allow the canvas size to change,
+	                                   or false to retain the original
+	                       onLoad      (function) callback after the document has loaded,
+	                                   'this' is the container, receives SVG object and
+	                                   optional error message as a parameter
+	                       parent      (string or element or jQuery) the parent to load
+	                                   into, defaults to top-level svg element
+	   @return  (SVGWrapper) this root */
+	load: function(url, settings) {
+		settings = (typeof settings == 'boolean' ? {addTo: settings} :
+			(typeof settings == 'function' ? {onLoad: settings} :
+			(typeof settings == 'string' ? {parent: settings} : 
+			(typeof settings == 'object' && settings.nodeName ? {parent: settings} :
+			(typeof settings == 'object' && settings.jquery ? {parent: settings} :
+			settings || {})))));
+		if (!settings.parent && !settings.addTo) {
+			this.clear(false);
+		}
+		var size = [this._svg.getAttribute('width'), this._svg.getAttribute('height')];
+		var wrapper = this;
+		// Report a problem with the load
+		var reportError = function(message) {
+			message = $.svg.local.errorLoadingText + ': ' + message;
+			if (settings.onLoad) {
+				settings.onLoad.apply(wrapper._container || wrapper._svg, [wrapper, message]);
+			}
+			else {
+				wrapper.text(null, 10, 20, message);
+			}
+		};
+		// Create a DOM from SVG content
+		var loadXML4IE = function(data) {
+			var xml = new ActiveXObject('Microsoft.XMLDOM');
+			xml.validateOnParse = false;
+			xml.resolveExternals = false;
+			xml.async = false;
+			xml.loadXML(data);
+			if (xml.parseError.errorCode != 0) {
+				reportError(xml.parseError.reason);
+				return null;
+			}
+			return xml;
+		};
+		// Load the SVG DOM
+		var loadSVG = function(data) {
+			if (!data) {
+				return;
+			}
+			if (data.documentElement.nodeName != 'svg') {
+				var errors = data.getElementsByTagName('parsererror');
+				var messages = (errors.length ? errors[0].getElementsByTagName('div') : []); // Safari
+				reportError(!errors.length ? '???' :
+					(messages.length ? messages[0] : errors[0]).firstChild.nodeValue);
+				return;
+			}
+			var parent = (settings.parent ? $(settings.parent)[0] : wrapper._svg);
+			var attrs = {};
+			for (var i = 0; i < data.documentElement.attributes.length; i++) {
+				var attr = data.documentElement.attributes.item(i);
+				if (!(attr.nodeName == 'version' || attr.nodeName.substring(0, 5) == 'xmlns')) {
+					attrs[attr.nodeName] = attr.nodeValue;
+				}
+			}
+			wrapper.configure(parent, attrs, !settings.parent);
+			var nodes = data.documentElement.childNodes;
+			for (var i = 0; i < nodes.length; i++) {
+				try {
+					if ($.svg._renesis) {
+						throw 'Force traversal';
+					}
+					parent.appendChild(wrapper._svg.ownerDocument.importNode(nodes[i], true));
+					if (nodes[i].nodeName == 'script') {
+						$.globalEval(nodes[i].textContent);
+					}
+				}
+				catch (e) {
+					wrapper.add(parent, nodes[i]);
+				}
+			}
+			if (!settings.changeSize) {
+				wrapper.configure(parent, {width: size[0], height: size[1]});
+			}
+			if (settings.onLoad) {
+				settings.onLoad.apply(wrapper._container || wrapper._svg, [wrapper]);
+			}
+		};
+		if (url.match('<svg')) { // Inline SVG
+			loadSVG($.browser.msie ? loadXML4IE(url) :
+				new DOMParser().parseFromString(url, 'text/xml'));
+		}
+		else { // Remote SVG
+			$.ajax({url: url, dataType: ($.browser.msie ? 'text' : 'xml'),
+				success: function(xml) {
+					loadSVG($.browser.msie ? loadXML4IE(xml) : xml);
+				}, error: function(http, message, exc) {
+					reportError(message + (exc ? ' ' + exc.message : ''));
+				}});
+		}
+		return this;
+	},
+
+	/* Delete a specified node.
+	   @param  node  (element or jQuery) the drawing node to remove
+	   @return  (SVGWrapper) this root */
+	remove: function(node) {
+		node = (node.jquery ? node[0] : node);
+		node.parentNode.removeChild(node);
+		return this;
+	},
+
+	/* Delete everything in the current document.
+	   @param  attrsToo  (boolean) true to clear any root attributes as well,
+	                     false to leave them (optional)
+	   @return  (SVGWrapper) this root */
+	clear: function(attrsToo) {
+		if (attrsToo) {
+			this.configure({}, true);
+		}
+		while (this._svg.firstChild) {
+			this._svg.removeChild(this._svg.firstChild);
+		}
+		return this;
+	},
+
+	/* Serialise the current diagram into an SVG text document.
+	   @param  node  (SVG element) the starting node (optional)
+	   @return  (string) the SVG as text */
+	toSVG: function(node) {
+		node = node || this._svg;
+		return (typeof XMLSerializer == 'undefined' ? this._toSVG(node) :
+			new XMLSerializer().serializeToString(node));
+	},
+
+	/* Serialise one node in the SVG hierarchy. */
+	_toSVG: function(node) {
+		var svgDoc = '';
+		if (!node) {
+			return svgDoc;
+		}
+		if (node.nodeType == 3) { // Text
+			svgDoc = node.nodeValue;
+		}
+		else if (node.nodeType == 4) { // CDATA
+			svgDoc = '<![CDATA[' + node.nodeValue + ']]>';
+		}
+		else { // Element
+			svgDoc = '<' + node.nodeName;
+			if (node.attributes) {
+				for (var i = 0; i < node.attributes.length; i++) {
+					var attr = node.attributes.item(i);
+					if (!($.trim(attr.nodeValue) == '' || attr.nodeValue.match(/^\[object/) ||
+							attr.nodeValue.match(/^function/))) {
+						svgDoc += ' ' + (attr.namespaceURI == $.svg.xlinkNS ? 'xlink:' : '') + 
+							attr.nodeName + '="' + attr.nodeValue + '"';
+					}
+				}
+			}	
+			if (node.firstChild) {
+				svgDoc += '>';
+				var child = node.firstChild;
+				while (child) {
+					svgDoc += this._toSVG(child);
+					child = child.nextSibling;
+				}
+				svgDoc += '</' + node.nodeName + '>';
+			}
+				else {
+				svgDoc += '/>';
+			}
+		}
+		return svgDoc;
+	}
+});
+
+/* Helper to generate an SVG path.
+   Obtain an instance from the SVGWrapper object.
+   String calls together to generate the path and use its value:
+   var path = root.createPath();
+   root.path(null, path.move(100, 100).line(300, 100).line(200, 300).close(), {fill: 'red'});
+   or
+   root.path(null, path.move(100, 100).line([[300, 100], [200, 300]]).close(), {fill: 'red'}); */
+function SVGPath() {
+	this._path = '';
+}
+
+$.extend(SVGPath.prototype, {
+	/* Prepare to create a new path.
+	   @return  (SVGPath) this path */
+	reset: function() {
+		this._path = '';
+		return this;
+	},
+
+	/* Move the pointer to a position.
+	   @param  x         (number) x-coordinate to move to or
+	                     (number[][]) x-/y-coordinates to move to
+	   @param  y         (number) y-coordinate to move to (omitted if x is array)
+	   @param  relative  (boolean) true for coordinates relative to the current point,
+	                     false for coordinates being absolute
+	   @return  (SVGPath) this path */
+	move: function(x, y, relative) {
+		relative = (isArray(x) ? y : relative);
+		return this._coords((relative ? 'm' : 'M'), x, y);
+	},
+
+	/* Draw a line to a position.
+	   @param  x         (number) x-coordinate to move to or
+	                     (number[][]) x-/y-coordinates to move to
+	   @param  y         (number) y-coordinate to move to (omitted if x is array)
+	   @param  relative  (boolean) true for coordinates relative to the current point,
+	                     false for coordinates being absolute
+	   @return  (SVGPath) this path */
+	line: function(x, y, relative) {
+		relative = (isArray(x) ? y : relative);
+		return this._coords((relative ? 'l' : 'L'), x, y);
+	},
+
+	/* Draw a horizontal line to a position.
+	   @param  x         (number) x-coordinate to draw to or
+	                     (number[]) x-coordinates to draw to
+	   @param  relative  (boolean) true for coordinates relative to the current point,
+	                     false for coordinates being absolute
+	   @return  (SVGPath) this path */
+	horiz: function(x, relative) {
+		this._path += (relative ? 'h' : 'H') + (isArray(x) ? x.join(' ') : x);
+		return this;
+	},
+
+	/* Draw a vertical line to a position.
+	   @param  y         (number) y-coordinate to draw to or
+	                     (number[]) y-coordinates to draw to
+	   @param  relative  (boolean) true for coordinates relative to the current point,
+	                     false for coordinates being absolute
+	   @return  (SVGPath) this path */
+	vert: function(y, relative) {
+		this._path += (relative ? 'v' : 'V') + (isArray(y) ? y.join(' ') : y);
+		return this;
+	},
+
+	/* Draw a cubic Bézier curve.
+	   @param  x1        (number) x-coordinate of beginning control point or
+	                     (number[][]) x-/y-coordinates of control and end points to draw to
+	   @param  y1        (number) y-coordinate of beginning control point (omitted if x1 is array)
+	   @param  x2        (number) x-coordinate of ending control point (omitted if x1 is array)
+	   @param  y2        (number) y-coordinate of ending control point (omitted if x1 is array)
+	   @param  x         (number) x-coordinate of curve end (omitted if x1 is array)
+	   @param  y         (number) y-coordinate of curve end (omitted if x1 is array)
+	   @param  relative  (boolean) true for coordinates relative to the current point,
+	                     false for coordinates being absolute
+	   @return  (SVGPath) this path */
+	curveC: function(x1, y1, x2, y2, x, y, relative) {
+		relative = (isArray(x1) ? y1 : relative);
+		return this._coords((relative ? 'c' : 'C'), x1, y1, x2, y2, x, y);
+	},
+
+	/* Continue a cubic Bézier curve.
+	   Starting control point is the reflection of the previous end control point.
+	   @param  x2        (number) x-coordinate of ending control point or
+	                     (number[][]) x-/y-coordinates of control and end points to draw to
+	   @param  y2        (number) y-coordinate of ending control point (omitted if x2 is array)
+	   @param  x         (number) x-coordinate of curve end (omitted if x2 is array)
+	   @param  y         (number) y-coordinate of curve end (omitted if x2 is array)
+	   @param  relative  (boolean) true for coordinates relative to the current point,
+	                     false for coordinates being absolute
+	   @return  (SVGPath) this path */
+	smoothC: function(x2, y2, x, y, relative) {
+		relative = (isArray(x2) ? y2 : relative);
+		return this._coords((relative ? 's' : 'S'), x2, y2, x, y);
+	},
+
+	/* Draw a quadratic Bézier curve.
+	   @param  x1        (number) x-coordinate of control point or
+	                     (number[][]) x-/y-coordinates of control and end points to draw to
+	   @param  y1        (number) y-coordinate of control point (omitted if x1 is array)
+	   @param  x         (number) x-coordinate of curve end (omitted if x1 is array)
+	   @param  y         (number) y-coordinate of curve end (omitted if x1 is array)
+	   @param  relative  (boolean) true for coordinates relative to the current point,
+	                     false for coordinates being absolute
+	   @return  (SVGPath) this path */
+	curveQ: function(x1, y1, x, y, relative) {
+		relative = (isArray(x1) ? y1 : relative);
+		return this._coords((relative ? 'q' : 'Q'), x1, y1, x, y);
+	},
+
+	/* Continue a quadratic Bézier curve.
+	   Control point is the reflection of the previous control point.
+	   @param  x         (number) x-coordinate of curve end or
+	                     (number[][]) x-/y-coordinates of points to draw to
+	   @param  y         (number) y-coordinate of curve end (omitted if x is array)
+	   @param  relative  (boolean) true for coordinates relative to the current point,
+	                     false for coordinates being absolute
+	   @return  (SVGPath) this path */
+	smoothQ: function(x, y, relative) {
+		relative = (isArray(x) ? y : relative);
+		return this._coords((relative ? 't' : 'T'), x, y);
+	},
+
+	/* Generate a path command with (a list of) coordinates. */
+	_coords: function(cmd, x1, y1, x2, y2, x3, y3) {
+		if (isArray(x1)) {
+			for (var i = 0; i < x1.length; i++) {
+				var cs = x1[i];
+				this._path += (i == 0 ? cmd : ' ') + cs[0] + ',' + cs[1] +
+					(cs.length < 4 ? '' : ' ' + cs[2] + ',' + cs[3] +
+					(cs.length < 6 ? '': ' ' + cs[4] + ',' + cs[5]));
+			}
+		}
+		else {
+			this._path += cmd + x1 + ',' + y1 + 
+				(x2 == null ? '' : ' ' + x2 + ',' + y2 +
+				(x3 == null ? '' : ' ' + x3 + ',' + y3));
+		}
+		return this;
+	},
+
+	/* Draw an arc to a position.
+	   @param  rx         (number) x-radius of arc or
+	                      (number/boolean[][]) x-/y-coordinates and flags for points to draw to
+	   @param  ry         (number) y-radius of arc (omitted if rx is array)
+	   @param  xRotate    (number) x-axis rotation (degrees, clockwise) (omitted if rx is array)
+	   @param  large      (boolean) true to draw the large part of the arc,
+	                      false to draw the small part (omitted if rx is array)
+	   @param  clockwise  (boolean) true to draw the clockwise arc,
+	                      false to draw the anti-clockwise arc (omitted if rx is array)
+	   @param  x          (number) x-coordinate of arc end (omitted if rx is array)
+	   @param  y          (number) y-coordinate of arc end (omitted if rx is array)
+	   @param  relative   (boolean) true for coordinates relative to the current point,
+	                      false for coordinates being absolute
+	   @return  (SVGPath) this path */
+	arc: function(rx, ry, xRotate, large, clockwise, x, y, relative) {
+		relative = (isArray(rx) ? ry : relative);
+		this._path += (relative ? 'a' : 'A');
+		if (isArray(rx)) {
+			for (var i = 0; i < rx.length; i++) {
+				var cs = rx[i];
+				this._path += (i == 0 ? '' : ' ') + cs[0] + ',' + cs[1] + ' ' +
+					cs[2] + ' ' + (cs[3] ? '1' : '0') + ',' +
+					(cs[4] ? '1' : '0') + ' ' + cs[5] + ',' + cs[6];
+			}
+		}
+		else {
+			this._path += rx + ',' + ry + ' ' + xRotate + ' ' +
+				(large ? '1' : '0') + ',' + (clockwise ? '1' : '0') + ' ' + x + ',' + y;
+		}
+		return this;
+	},
+
+	/* Close the current path.
+	   @return  (SVGPath) this path */
+	close: function() {
+		this._path += 'z';
+		return this;
+	},
+
+	/* Return the string rendering of the specified path.
+	   @return  (string) stringified path */
+	path: function() {
+		return this._path;
+	}
+});
+
+SVGPath.prototype.moveTo = SVGPath.prototype.move;
+SVGPath.prototype.lineTo = SVGPath.prototype.line;
+SVGPath.prototype.horizTo = SVGPath.prototype.horiz;
+SVGPath.prototype.vertTo = SVGPath.prototype.vert;
+SVGPath.prototype.curveCTo = SVGPath.prototype.curveC;
+SVGPath.prototype.smoothCTo = SVGPath.prototype.smoothC;
+SVGPath.prototype.curveQTo = SVGPath.prototype.curveQ;
+SVGPath.prototype.smoothQTo = SVGPath.prototype.smoothQ;
+SVGPath.prototype.arcTo = SVGPath.prototype.arc;
+
+/* Helper to generate an SVG text object.
+   Obtain an instance from the SVGWrapper object.
+   String calls together to generate the text and use its value:
+   var text = root.createText();
+   root.text(null, x, y, text.string('This is ').
+     span('red', {fill: 'red'}).string('!'), {fill: 'blue'}); */
+function SVGText() {
+	this._parts = []; // The components of the text object
+}
+
+$.extend(SVGText.prototype, {
+	/* Prepare to create a new text object.
+	   @return  (SVGText) this text */
+	reset: function() {
+		this._parts = [];
+		return this;
+	},
+
+	/* Add a straight string value.
+	   @param  value  (string) the actual text
+	   @return  (SVGText) this text object */
+	string: function(value) {
+		this._parts[this._parts.length] = ['text', value];
+		return this;
+	},
+
+	/* Add a separate text span that has its own settings.
+	   @param  value     (string) the actual text
+	   @param  settings  (object) the settings for this text
+	   @return  (SVGText) this text object */
+	span: function(value, settings) {
+		this._parts[this._parts.length] = ['tspan', value, settings];
+		return this;
+	},
+
+	/* Add a reference to a previously defined text string.
+	   @param  id        (string) the ID of the actual text
+	   @param  settings  (object) the settings for this text
+	   @return  (SVGText) this text object */
+	ref: function(id, settings) {
+		this._parts[this._parts.length] = ['tref', id, settings];
+		return this;
+	},
+
+	/* Add text drawn along a path.
+	   @param  id        (string) the ID of the path
+	   @param  value     (string) the actual text
+	   @param  settings  (object) the settings for this text
+	   @return  (SVGText) this text object */
+	path: function(id, value, settings) {
+		this._parts[this._parts.length] = ['textpath', value, 
+			$.extend({href: id}, settings || {})];
+		return this;
+	}
+});
+
+/* Attach the SVG functionality to a jQuery selection.
+   @param  command  (string) the command to run (optional, default 'attach')
+   @param  options  (object) the new settings to use for these SVG instances
+   @return jQuery (object) for chaining further calls */
+$.fn.svg = function(options) {
+	var otherArgs = Array.prototype.slice.call(arguments, 1);
+	if (typeof options == 'string' && options == 'get') {
+		return $.svg['_' + options + 'SVG'].apply($.svg, [this[0]].concat(otherArgs));
+	}
+	return this.each(function() {
+		if (typeof options == 'string') {
+			$.svg['_' + options + 'SVG'].apply($.svg, [this].concat(otherArgs));
+		}
+		else {
+			$.svg._attachSVG(this, options || {});
+		} 
+	});
+};
+
+/* Determine whether an object is an array. */
+function isArray(a) {
+	return (a && a.constructor == Array);
+}
+
+// Singleton primary SVG interface
+$.svg = new SVGManager();
+
+})(jQuery);
diff --git a/src/web/js/jquery.svg.min.js b/src/web/js/jquery.svg.min.js
new file mode 100644
index 0000000..5b922fb
--- /dev/null
+++ b/src/web/js/jquery.svg.min.js
@@ -0,0 +1,7 @@
+/* http://keith-wood.name/svg.html
+   SVG for jQuery v1.4.5.
+   Written by Keith Wood (kbwood{at}iinet.com.au) August 2007.
+   Dual licensed under the GPL (http://dev.jquery.com/browser/trunk/jquery/GPL-LICENSE.txt) and 
+   MIT (http://dev.jquery.com/browser/trunk/jquery/MIT-LICENSE.txt) licenses. 
+   Please attribute the author if you use it. */
+(function($){function SVGManager(){this._settings=[];this._extensions=[];this.regional=[];this.regional['']={errorLoadingText:'Error loading',notSupportedText:'This browser does not support SVG'};this.local=this.regional[''];this._uuid=new Date().getTime();this._renesis=detectActiveX('RenesisX.RenesisCtrl')}function detectActiveX(a){try{return!!(window.ActiveXObject&&new ActiveXObject(a))}catch(e){return false}}var q='svgwrapper';$.extend(SVGManager.prototype,{markerClassName:'hasSVG',svgNS:'http://www.w3.org/2000/svg',xlinkNS:'http://www.w3.org/1999/xlink',_wrapperClass:SVGWrapper,_attrNames:{class_:'class',in_:'in',alignmentBaseline:'alignment-baseline',baselineShift:'baseline-shift',clipPath:'clip-path',clipRule:'clip-rule',colorInterpolation:'color-interpolation',colorInterpolationFilters:'color-interpolation-filters',colorRendering:'color-rendering',dominantBaseline:'dominant-baseline',enableBackground:'enable-background',fillOpacity:'fill-opacity',fillRule:'fill-rule',floodColor:'flood-color',floodOpacity:'flood-opacity',fontFamily:'font-family',fontSize:'font-size',fontSizeAdjust:'font-size-adjust',fontStretch:'font-stretch',fontStyle:'font-style',fontVariant:'font-variant',fontWeight:'font-weight',glyphOrientationHorizontal:'glyph-orientation-horizontal',glyphOrientationVertical:'glyph-orientation-vertical',horizAdvX:'horiz-adv-x',horizOriginX:'horiz-origin-x',imageRendering:'image-rendering',letterSpacing:'letter-spacing',lightingColor:'lighting-color',markerEnd:'marker-end',markerMid:'marker-mid',markerStart:'marker-start',stopColor:'stop-color',stopOpacity:'stop-opacity',strikethroughPosition:'strikethrough-position',strikethroughThickness:'strikethrough-thickness',strokeDashArray:'stroke-dasharray',strokeDashOffset:'stroke-dashoffset',strokeLineCap:'stroke-linecap',strokeLineJoin:'stroke-linejoin',strokeMiterLimit:'stroke-miterlimit',strokeOpacity:'stroke-opacity',strokeWidth:'stroke-width',textAnchor:'text-anchor',textDecoration:'text-decoration',textRendering:'text-rendering',underlinePosition:'underline-position',underlineThickness:'underline-thickness',vertAdvY:'vert-adv-y',vertOriginY:'vert-origin-y',wordSpacing:'word-spacing',writingMode:'writing-mode'},_attachSVG:function(a,b){var c=(a.namespaceURI==this.svgNS?a:null);var a=(c?null:a);if($(a||c).hasClass(this.markerClassName)){return}if(typeof b=='string'){b={loadURL:b}}else if(typeof b=='function'){b={onLoad:b}}$(a||c).addClass(this.markerClassName);try{if(!c){c=document.createElementNS(this.svgNS,'svg');c.setAttribute('version','1.1');if(a.clientWidth>0){c.setAttribute('width',a.clientWidth)}if(a.clientHeight>0){c.setAttribute('height',a.clientHeight)}a.appendChild(c)}this._afterLoad(a,c,b||{})}catch(e){if($.browser.msie){if(!a.id){a.id='svg'+(this._uuid++)}this._settings[a.id]=b;a.innerHTML='<embed type="image/svg+xml" width="100%" '+'height="100%" src="'+(b.initPath||'')+'blank.svg" '+'pluginspage="http://www.adobe.com/svg/viewer/install/main.html"/>'}else{a.innerHTML='<p class="svg_error">'+this.local.notSupportedText+'</p>'}}},_registerSVG:function(){for(var i=0;i<document.embeds.length;i++){var a=document.embeds[i].parentNode;if(!$(a).hasClass($.svg.markerClassName)||$.data(a,q)){continue}var b=null;try{b=document.embeds[i].getSVGDocument()}catch(e){setTimeout($.svg._registerSVG,250);return}b=(b?b.documentElement:null);if(b){$.svg._afterLoad(a,b)}}},_afterLoad:function(a,b,c){var c=c||this._settings[a.id];this._settings[a?a.id:'']=null;var d=new this._wrapperClass(b,a);$.data(a||b,q,d);try{if(c.loadURL){d.load(c.loadURL,c)}if(c.settings){d.configure(c.settings)}if(c.onLoad&&!c.loadURL){c.onLoad.apply(a||b,[d])}}catch(e){alert(e)}},_getSVG:function(a){a=(typeof a=='string'?$(a)[0]:(a.jquery?a[0]:a));return $.data(a,q)},_destroySVG:function(a){var b=$(a);if(!b.hasClass(this.markerClassName)){return}b.removeClass(this.markerClassName);if(a.namespaceURI!=this.svgNS){b.empty()}$.removeData(a,q)},addExtension:function(a,b){this._extensions.push([a,b])},isSVGElem:function(a){return(a.nodeType==1&&a.namespaceURI==$.svg.svgNS)}});function SVGWrapper(a,b){this._svg=a;this._container=b;for(var i=0;i<$.svg._extensions.length;i++){var c=$.svg._extensions[i];this[c[0]]=new c[1](this)}}$.extend(SVGWrapper.prototype,{_width:function(){return(this._container?this._container.clientWidth:this._svg.width)},_height:function(){return(this._container?this._container.clientHeight:this._svg.height)},root:function(){return this._svg},configure:function(a,b,c){if(!a.nodeName){c=b;b=a;a=this._svg}if(c){for(var i=a.attributes.length-1;i>=0;i--){var d=a.attributes.item(i);if(!(d.nodeName=='onload'||d.nodeName=='version'||d.nodeName.substring(0,5)=='xmlns')){a.attributes.removeNamedItem(d.nodeName)}}}for(var e in b){a.setAttribute($.svg._attrNames[e]||e,b[e])}return this},getElementById:function(a){return this._svg.ownerDocument.getElementById(a)},change:function(a,b){if(a){for(var c in b){if(b[c]==null){a.removeAttribute($.svg._attrNames[c]||c)}else{a.setAttribute($.svg._attrNames[c]||c,b[c])}}}return this},_args:function(b,c,d){c.splice(0,0,'parent');c.splice(c.length,0,'settings');var e={};var f=0;if(b[0]!=null&&b[0].jquery){b[0]=b[0][0]}if(b[0]!=null&&!(typeof b[0]=='object'&&b[0].nodeName)){e['parent']=null;f=1}for(var i=0;i<b.length;i++){e[c[i+f]]=b[i]}if(d){$.each(d,function(i,a){if(typeof e[a]=='object'){e.settings=e[a];e[a]=null}})}return e},title:function(a,b,c){var d=this._args(arguments,['text']);var e=this._makeNode(d.parent,'title',d.settings||{});e.appendChild(this._svg.ownerDocument.createTextNode(d.text));return e},describe:function(a,b,c){var d=this._args(arguments,['text']);var e=this._makeNode(d.parent,'desc',d.settings||{});e.appendChild(this._svg.ownerDocument.createTextNode(d.text));return e},defs:function(a,b,c){var d=this._args(arguments,['id'],['id']);return this._makeNode(d.parent,'defs',$.extend((d.id?{id:d.id}:{}),d.settings||{}))},symbol:function(a,b,c,d,e,f,g){var h=this._args(arguments,['id','x1','y1','width','height']);return this._makeNode(h.parent,'symbol',$.extend({id:h.id,viewBox:h.x1+' '+h.y1+' '+h.width+' '+h.height},h.settings||{}))},marker:function(a,b,c,d,e,f,g,h){var i=this._args(arguments,['id','refX','refY','mWidth','mHeight','orient'],['orient']);return this._makeNode(i.parent,'marker',$.extend({id:i.id,refX:i.refX,refY:i.refY,markerWidth:i.mWidth,markerHeight:i.mHeight,orient:i.orient||'auto'},i.settings||{}))},style:function(a,b,c){var d=this._args(arguments,['styles']);var e=this._makeNode(d.parent,'style',$.extend({type:'text/css'},d.settings||{}));e.appendChild(this._svg.ownerDocument.createTextNode(d.styles));if($.browser.opera){$('head').append('<style type="text/css">'+d.styles+'</style>')}return e},script:function(a,b,c,d){var e=this._args(arguments,['script','type'],['type']);var f=this._makeNode(e.parent,'script',$.extend({type:e.type||'text/javascript'},e.settings||{}));f.appendChild(this._svg.ownerDocument.createTextNode(e.script));if(!$.browser.mozilla){$.globalEval(e.script)}return f},linearGradient:function(a,b,c,d,e,f,g,h){var i=this._args(arguments,['id','stops','x1','y1','x2','y2'],['x1']);var j=$.extend({id:i.id},(i.x1!=null?{x1:i.x1,y1:i.y1,x2:i.x2,y2:i.y2}:{}));return this._gradient(i.parent,'linearGradient',$.extend(j,i.settings||{}),i.stops)},radialGradient:function(a,b,c,d,e,r,f,g,h){var i=this._args(arguments,['id','stops','cx','cy','r','fx','fy'],['cx']);var j=$.extend({id:i.id},(i.cx!=null?{cx:i.cx,cy:i.cy,r:i.r,fx:i.fx,fy:i.fy}:{}));return this._gradient(i.parent,'radialGradient',$.extend(j,i.settings||{}),i.stops)},_gradient:function(a,b,c,d){var e=this._makeNode(a,b,c);for(var i=0;i<d.length;i++){var f=d[i];this._makeNode(e,'stop',$.extend({offset:f[0],stopColor:f[1]},(f[2]!=null?{stopOpacity:f[2]}:{})))}return e},pattern:function(a,b,x,y,c,d,e,f,g,h,i){var j=this._args(arguments,['id','x','y','width','height','vx','vy','vwidth','vheight'],['vx']);var k=$.extend({id:j.id,x:j.x,y:j.y,width:j.width,height:j.height},(j.vx!=null?{viewBox:j.vx+' '+j.vy+' '+j.vwidth+' '+j.vheight}:{}));return this._makeNode(j.parent,'pattern',$.extend(k,j.settings||{}))},clipPath:function(a,b,c,d){var e=this._args(arguments,['id','units']);e.units=e.units||'userSpaceOnUse';return this._makeNode(e.parent,'clipPath',$.extend({id:e.id,clipPathUnits:e.units},e.settings||{}))},mask:function(a,b,x,y,c,d,e){var f=this._args(arguments,['id','x','y','width','height']);return this._makeNode(f.parent,'mask',$.extend({id:f.id,x:f.x,y:f.y,width:f.width,height:f.height},f.settings||{}))},createPath:function(){return new SVGPath()},createText:function(){return new SVGText()},svg:function(a,x,y,b,c,d,e,f,g,h){var i=this._args(arguments,['x','y','width','height','vx','vy','vwidth','vheight'],['vx']);var j=$.extend({x:i.x,y:i.y,width:i.width,height:i.height},(i.vx!=null?{viewBox:i.vx+' '+i.vy+' '+i.vwidth+' '+i.vheight}:{}));return this._makeNode(i.parent,'svg',$.extend(j,i.settings||{}))},group:function(a,b,c){var d=this._args(arguments,['id'],['id']);return this._makeNode(d.parent,'g',$.extend({id:d.id},d.settings||{}))},use:function(a,x,y,b,c,d,e){var f=this._args(arguments,['x','y','width','height','ref']);if(typeof f.x=='string'){f.ref=f.x;f.settings=f.y;f.x=f.y=f.width=f.height=null}var g=this._makeNode(f.parent,'use',$.extend({x:f.x,y:f.y,width:f.width,height:f.height},f.settings||{}));g.setAttributeNS($.svg.xlinkNS,'href',f.ref);return g},link:function(a,b,c){var d=this._args(arguments,['ref']);var e=this._makeNode(d.parent,'a',d.settings);e.setAttributeNS($.svg.xlinkNS,'href',d.ref);return e},image:function(a,x,y,b,c,d,e){var f=this._args(arguments,['x','y','width','height','ref']);var g=this._makeNode(f.parent,'image',$.extend({x:f.x,y:f.y,width:f.width,height:f.height},f.settings||{}));g.setAttributeNS($.svg.xlinkNS,'href',f.ref);return g},path:function(a,b,c){var d=this._args(arguments,['path']);return this._makeNode(d.parent,'path',$.extend({d:(d.path.path?d.path.path():d.path)},d.settings||{}))},rect:function(a,x,y,b,c,d,e,f){var g=this._args(arguments,['x','y','width','height','rx','ry'],['rx']);return this._makeNode(g.parent,'rect',$.extend({x:g.x,y:g.y,width:g.width,height:g.height},(g.rx?{rx:g.rx,ry:g.ry}:{}),g.settings||{}))},circle:function(a,b,c,r,d){var e=this._args(arguments,['cx','cy','r']);return this._makeNode(e.parent,'circle',$.extend({cx:e.cx,cy:e.cy,r:e.r},e.settings||{}))},ellipse:function(a,b,c,d,e,f){var g=this._args(arguments,['cx','cy','rx','ry']);return this._makeNode(g.parent,'ellipse',$.extend({cx:g.cx,cy:g.cy,rx:g.rx,ry:g.ry},g.settings||{}))},line:function(a,b,c,d,e,f){var g=this._args(arguments,['x1','y1','x2','y2']);return this._makeNode(g.parent,'line',$.extend({x1:g.x1,y1:g.y1,x2:g.x2,y2:g.y2},g.settings||{}))},polyline:function(a,b,c){var d=this._args(arguments,['points']);return this._poly(d.parent,'polyline',d.points,d.settings)},polygon:function(a,b,c){var d=this._args(arguments,['points']);return this._poly(d.parent,'polygon',d.points,d.settings)},_poly:function(a,b,c,d){var e='';for(var i=0;i<c.length;i++){e+=c[i].join()+' '}return this._makeNode(a,b,$.extend({points:$.trim(e)},d||{}))},text:function(a,x,y,b,c){var d=this._args(arguments,['x','y','value']);if(typeof d.x=='string'&&arguments.length<4){d.value=d.x;d.settings=d.y;d.x=d.y=null}return this._text(d.parent,'text',d.value,$.extend({x:(d.x&&isArray(d.x)?d.x.join(' '):d.x),y:(d.y&&isArray(d.y)?d.y.join(' '):d.y)},d.settings||{}))},textpath:function(a,b,c,d){var e=this._args(arguments,['path','value']);var f=this._text(e.parent,'textPath',e.value,e.settings||{});f.setAttributeNS($.svg.xlinkNS,'href',e.path);return f},_text:function(a,b,c,d){var e=this._makeNode(a,b,d);if(typeof c=='string'){e.appendChild(e.ownerDocument.createTextNode(c))}else{for(var i=0;i<c._parts.length;i++){var f=c._parts[i];if(f[0]=='tspan'){var g=this._makeNode(e,f[0],f[2]);g.appendChild(e.ownerDocument.createTextNode(f[1]));e.appendChild(g)}else if(f[0]=='tref'){var g=this._makeNode(e,f[0],f[2]);g.setAttributeNS($.svg.xlinkNS,'href',f[1]);e.appendChild(g)}else if(f[0]=='textpath'){var h=$.extend({},f[2]);h.href=null;var g=this._makeNode(e,f[0],h);g.setAttributeNS($.svg.xlinkNS,'href',f[2].href);g.appendChild(e.ownerDocument.createTextNode(f[1]));e.appendChild(g)}else{e.appendChild(e.ownerDocument.createTextNode(f[1]))}}}return e},other:function(a,b,c){var d=this._args(arguments,['name']);return this._makeNode(d.parent,d.name,d.settings||{})},_makeNode:function(a,b,c){a=a||this._svg;var d=this._svg.ownerDocument.createElementNS($.svg.svgNS,b);for(var b in c){var e=c[b];if(e!=null&&e!=null&&(typeof e!='string'||e!='')){d.setAttribute($.svg._attrNames[b]||b,e)}}a.appendChild(d);return d},add:function(b,c){var d=this._args((arguments.length==1?[null,b]:arguments),['node']);var f=this;d.parent=d.parent||this._svg;d.node=(d.node.jquery?d.node:$(d.node));try{if($.svg._renesis){throw'Force traversal';}d.parent.appendChild(d.node.cloneNode(true))}catch(e){d.node.each(function(){var a=f._cloneAsSVG(this);if(a){d.parent.appendChild(a)}})}return this},clone:function(b,c){var d=this;var e=this._args((arguments.length==1?[null,b]:arguments),['node']);e.parent=e.parent||this._svg;e.node=(e.node.jquery?e.node:$(e.node));var f=[];e.node.each(function(){var a=d._cloneAsSVG(this);if(a){a.id='';e.parent.appendChild(a);f.push(a)}});return f},_cloneAsSVG:function(a){var b=null;if(a.nodeType==1){b=this._svg.ownerDocument.createElementNS($.svg.svgNS,this._checkName(a.nodeName));for(var i=0;i<a.attributes.length;i++){var c=a.attributes.item(i);if(c.nodeName!='xmlns'&&c.nodeValue){if(c.prefix=='xlink'){b.setAttributeNS($.svg.xlinkNS,c.localName||c.baseName,c.nodeValue)}else{b.setAttribute(this._checkName(c.nodeName),c.nodeValue)}}}for(var i=0;i<a.childNodes.length;i++){var d=this._cloneAsSVG(a.childNodes[i]);if(d){b.appendChild(d)}}}else if(a.nodeType==3){if($.trim(a.nodeValue)){b=this._svg.ownerDocument.createTextNode(a.nodeValue)}}else if(a.nodeType==4){if($.trim(a.nodeValue)){try{b=this._svg.ownerDocument.createCDATASection(a.nodeValue)}catch(e){b=this._svg.ownerDocument.createTextNode(a.nodeValue.replace(/&/g,'&amp;').replace(/</g,'&lt;').replace(/>/g,'&gt;'))}}}return b},_checkName:function(a){a=(a.substring(0,1)>='A'&&a.substring(0,1)<='Z'?a.toLowerCase():a);return(a.substring(0,4)=='svg:'?a.substring(4):a)},load:function(j,k){k=(typeof k=='boolean'?{addTo:k}:(typeof k=='function'?{onLoad:k}:(typeof k=='string'?{parent:k}:(typeof k=='object'&&k.nodeName?{parent:k}:(typeof k=='object'&&k.jquery?{parent:k}:k||{})))));if(!k.parent&&!k.addTo){this.clear(false)}var l=[this._svg.getAttribute('width'),this._svg.getAttribute('height')];var m=this;var n=function(a){a=$.svg.local.errorLoadingText+': '+a;if(k.onLoad){k.onLoad.apply(m._container||m._svg,[m,a])}else{m.text(null,10,20,a)}};var o=function(a){var b=new ActiveXObject('Microsoft.XMLDOM');b.validateOnParse=false;b.resolveExternals=false;b.async=false;b.loadXML(a);if(b.parseError.errorCode!=0){n(b.parseError.reason);return null}return b};var p=function(a){if(!a){return}if(a.documentElement.nodeName!='svg'){var b=a.getElementsByTagName('parsererror');var c=(b.length?b[0].getElementsByTagName('div'):[]);n(!b.length?'???':(c.length?c[0]:b[0]).firstChild.nodeValue);return}var d=(k.parent?$(k.parent)[0]:m._svg);var f={};for(var i=0;i<a.documentElement.attributes.length;i++){var g=a.documentElement.attributes.item(i);if(!(g.nodeName=='version'||g.nodeName.substring(0,5)=='xmlns')){f[g.nodeName]=g.nodeValue}}m.configure(d,f,!k.parent);var h=a.documentElement.childNodes;for(var i=0;i<h.length;i++){try{if($.svg._renesis){throw'Force traversal';}d.appendChild(m._svg.ownerDocument.importNode(h[i],true));if(h[i].nodeName=='script'){$.globalEval(h[i].textContent)}}catch(e){m.add(d,h[i])}}if(!k.changeSize){m.configure(d,{width:l[0],height:l[1]})}if(k.onLoad){k.onLoad.apply(m._container||m._svg,[m])}};if(j.match('<svg')){p($.browser.msie?o(j):new DOMParser().parseFromString(j,'text/xml'))}else{$.ajax({url:j,dataType:($.browser.msie?'text':'xml'),success:function(a){p($.browser.msie?o(a):a)},error:function(a,b,c){n(b+(c?' '+c.message:''))}})}return this},remove:function(a){a=(a.jquery?a[0]:a);a.parentNode.removeChild(a);return this},clear:function(a){if(a){this.configure({},true)}while(this._svg.firstChild){this._svg.removeChild(this._svg.firstChild)}return this},toSVG:function(a){a=a||this._svg;return(typeof XMLSerializer=='undefined'?this._toSVG(a):new XMLSerializer().serializeToString(a))},_toSVG:function(a){var b='';if(!a){return b}if(a.nodeType==3){b=a.nodeValue}else if(a.nodeType==4){b='<![CDATA['+a.nodeValue+']]>'}else{b='<'+a.nodeName;if(a.attributes){for(var i=0;i<a.attributes.length;i++){var c=a.attributes.item(i);if(!($.trim(c.nodeValue)==''||c.nodeValue.match(/^\[object/)||c.nodeValue.match(/^function/))){b+=' '+(c.namespaceURI==$.svg.xlinkNS?'xlink:':'')+c.nodeName+'="'+c.nodeValue+'"'}}}if(a.firstChild){b+='>';var d=a.firstChild;while(d){b+=this._toSVG(d);d=d.nextSibling}b+='</'+a.nodeName+'>'}else{b+='/>'}}return b}});function SVGPath(){this._path=''}$.extend(SVGPath.prototype,{reset:function(){this._path='';return this},move:function(x,y,a){a=(isArray(x)?y:a);return this._coords((a?'m':'M'),x,y)},line:function(x,y,a){a=(isArray(x)?y:a);return this._coords((a?'l':'L'),x,y)},horiz:function(x,a){this._path+=(a?'h':'H')+(isArray(x)?x.join(' '):x);return this},vert:function(y,a){this._path+=(a?'v':'V')+(isArray(y)?y.join(' '):y);return this},curveC:function(a,b,c,d,x,y,e){e=(isArray(a)?b:e);return this._coords((e?'c':'C'),a,b,c,d,x,y)},smoothC:function(a,b,x,y,c){c=(isArray(a)?b:c);return this._coords((c?'s':'S'),a,b,x,y)},curveQ:function(a,b,x,y,c){c=(isArray(a)?b:c);return this._coords((c?'q':'Q'),a,b,x,y)},smoothQ:function(x,y,a){a=(isArray(x)?y:a);return this._coords((a?'t':'T'),x,y)},_coords:function(a,b,c,d,e,f,g){if(isArray(b)){for(var i=0;i<b.length;i++){var h=b[i];this._path+=(i==0?a:' ')+h[0]+','+h[1]+(h.length<4?'':' '+h[2]+','+h[3]+(h.length<6?'':' '+h[4]+','+h[5]))}}else{this._path+=a+b+','+c+(d==null?'':' '+d+','+e+(f==null?'':' '+f+','+g))}return this},arc:function(a,b,c,d,e,x,y,f){f=(isArray(a)?b:f);this._path+=(f?'a':'A');if(isArray(a)){for(var i=0;i<a.length;i++){var g=a[i];this._path+=(i==0?'':' ')+g[0]+','+g[1]+' '+g[2]+' '+(g[3]?'1':'0')+','+(g[4]?'1':'0')+' '+g[5]+','+g[6]}}else{this._path+=a+','+b+' '+c+' '+(d?'1':'0')+','+(e?'1':'0')+' '+x+','+y}return this},close:function(){this._path+='z';return this},path:function(){return this._path}});SVGPath.prototype.moveTo=SVGPath.prototype.move;SVGPath.prototype.lineTo=SVGPath.prototype.line;SVGPath.prototype.horizTo=SVGPath.prototype.horiz;SVGPath.prototype.vertTo=SVGPath.prototype.vert;SVGPath.prototype.curveCTo=SVGPath.prototype.curveC;SVGPath.prototype.smoothCTo=SVGPath.prototype.smoothC;SVGPath.prototype.curveQTo=SVGPath.prototype.curveQ;SVGPath.prototype.smoothQTo=SVGPath.prototype.smoothQ;SVGPath.prototype.arcTo=SVGPath.prototype.arc;function SVGText(){this._parts=[]}$.extend(SVGText.prototype,{reset:function(){this._parts=[];return this},string:function(a){this._parts[this._parts.length]=['text',a];return this},span:function(a,b){this._parts[this._parts.length]=['tspan',a,b];return this},ref:function(a,b){this._parts[this._parts.length]=['tref',a,b];return this},path:function(a,b,c){this._parts[this._parts.length]=['textpath',b,$.extend({href:a},c||{})];return this}});$.fn.svg=function(a){var b=Array.prototype.slice.call(arguments,1);if(typeof a=='string'&&a=='get'){return $.svg['_'+a+'SVG'].apply($.svg,[this[0]].concat(b))}return this.each(function(){if(typeof a=='string'){$.svg['_'+a+'SVG'].apply($.svg,[this].concat(b))}else{$.svg._attachSVG(this,a||{})}})};function isArray(a){return(a&&a.constructor==Array)}$.svg=new SVGManager()})(jQuery);
\ No newline at end of file
diff --git a/src/web/js/jquery.svganim.min.js b/src/web/js/jquery.svganim.min.js
new file mode 100644
index 0000000..3cc4020
--- /dev/null
+++ b/src/web/js/jquery.svganim.min.js
@@ -0,0 +1,7 @@
+/* http://keith-wood.name/svg.html
+   SVG attribute animations for jQuery v1.4.5.
+   Written by Keith Wood (kbwood{at}iinet.com.au) June 2008.
+   Dual licensed under the GPL (http://dev.jquery.com/browser/trunk/jquery/GPL-LICENSE.txt) and 
+   MIT (http://dev.jquery.com/browser/trunk/jquery/MIT-LICENSE.txt) licenses. 
+   Please attribute the author if you use it. */
+(function($){$.each(['x','y','width','height','rx','ry','cx','cy','r','x1','y1','x2','y2','stroke-width','strokeWidth','opacity','fill-opacity','fillOpacity','stroke-opacity','strokeOpacity','stroke-dashoffset','strokeDashOffset','font-size','fontSize','font-weight','fontWeight','letter-spacing','letterSpacing','word-spacing','wordSpacing'],function(i,f){var g=f.charAt(0).toUpperCase()+f.substr(1);if($.cssProps){$.cssProps['svg'+g]=$.cssProps['svg-'+f]=f}$.fx.step['svg'+g]=$.fx.step['svg-'+f]=function(a){var b=$.svg._attrNames[f]||f;var c=a.elem.attributes.getNamedItem(b);if(!a.set){a.start=(c?parseFloat(c.nodeValue):0);var d=($.fn.jquery>='1.6'?'':a.options.curAnim['svg'+g]||a.options.curAnim['svg-'+f]);if(/^[+-]=/.exec(d)){a.end=a.start+parseFloat(d.replace(/=/,''))}$(a.elem).css(b,'');a.set=true}var e=(a.pos*(a.end-a.start)+a.start)+(a.unit=='%'?'%':'');(c?c.nodeValue=e:a.elem.setAttribute(b,e))}});$.fx.step['svgStrokeDashArray']=$.fx.step['svg-strokeDashArray']=$.fx.step['svgStroke-dasharray']=$.fx.step['svg-stroke-dasharray']=function(a){var b=a.elem.attributes.getNamedItem('stroke-dasharray');if(!a.set){a.start=parseDashArray(b?b.nodeValue:'');var c=($.fn.jquery>='1.6'?a.end:a.options.curAnim['svgStrokeDashArray']||a.options.curAnim['svg-strokeDashArray']||a.options.curAnim['svgStroke-dasharray']||a.options.curAnim['svg-stroke-dasharray']);a.end=parseDashArray(c);if(/^[+-]=/.exec(c)){c=c.split(/[, ]+/);if(c.length%2==1){var d=c.length;for(var i=0;i<d;i++){c.push(c[i])}}for(var i=0;i<c.length;i++){if(/^[+-]=/.exec(c[i])){a.end[i]=a.start[i]+parseFloat(c[i].replace(/=/,''))}}}a.set=true}var e=$.map(a.start,function(n,i){return(a.pos*(a.end[i]-n)+n)}).join(',');(b?b.nodeValue=e:a.elem.setAttribute('stroke-dasharray',e))};function parseDashArray(a){var b=a.split(/[, ]+/);for(var i=0;i<b.length;i++){b[i]=parseFloat(b[i]);if(isNaN(b[i])){b[i]=0}}if(b.length%2==1){var c=b.length;for(var i=0;i<c;i++){b.push(b[i])}}return b}$.fx.step['svgViewBox']=$.fx.step['svg-viewBox']=function(a){var b=a.elem.attributes.getNamedItem('viewBox');if(!a.set){a.start=parseViewBox(b?b.nodeValue:'');var c=($.fn.jquery>='1.6'?a.end:a.options.curAnim['svgViewBox']||a.options.curAnim['svg-viewBox']);a.end=parseViewBox(c);if(/^[+-]=/.exec(c)){c=c.split(/[, ]+/);while(c.length<4){c.push('0')}for(var i=0;i<4;i++){if(/^[+-]=/.exec(c[i])){a.end[i]=a.start[i]+parseFloat(c[i].replace(/=/,''))}}}a.set=true}var d=$.map(a.start,function(n,i){return(a.pos*(a.end[i]-n)+n)}).join(' ');(b?b.nodeValue=d:a.elem.setAttribute('viewBox',d))};function parseViewBox(a){var b=a.split(/[, ]+/);for(var i=0;i<b.length;i++){b[i]=parseFloat(b[i]);if(isNaN(b[i])){b[i]=0}}while(b.length<4){b.push(0)}return b}$.fx.step['svgTransform']=$.fx.step['svg-transform']=function(a){var b=a.elem.attributes.getNamedItem('transform');if(!a.set){a.start=parseTransform(b?b.nodeValue:'');a.end=parseTransform(a.end,a.start);a.set=true}var c='';for(var i=0;i<a.end.order.length;i++){switch(a.end.order.charAt(i)){case't':c+=' translate('+(a.pos*(a.end.translateX-a.start.translateX)+a.start.translateX)+','+(a.pos*(a.end.translateY-a.start.translateY)+a.start.translateY)+')';break;case's':c+=' scale('+(a.pos*(a.end.scaleX-a.start.scaleX)+a.start.scaleX)+','+(a.pos*(a.end.scaleY-a.start.scaleY)+a.start.scaleY)+')';break;case'r':c+=' rotate('+(a.pos*(a.end.rotateA-a.start.rotateA)+a.start.rotateA)+','+(a.pos*(a.end.rotateX-a.start.rotateX)+a.start.rotateX)+','+(a.pos*(a.end.rotateY-a.start.rotateY)+a.start.rotateY)+')';break;case'x':c+=' skewX('+(a.pos*(a.end.skewX-a.start.skewX)+a.start.skewX)+')';case'y':c+=' skewY('+(a.pos*(a.end.skewY-a.start.skewY)+a.start.skewY)+')';break;case'm':var d='';for(var j=0;j<6;j++){d+=','+(a.pos*(a.end.matrix[j]-a.start.matrix[j])+a.start.matrix[j])}c+=' matrix('+d.substr(1)+')';break}}(b?b.nodeValue=c:a.elem.setAttribute('transform',c))};function parseTransform(a,b){a=a||'';if(typeof a=='object'){a=a.nodeValue}var c=$.extend({translateX:0,translateY:0,scaleX:0,scaleY:0,rotateA:0,rotateX:0,rotateY:0,skewX:0,skewY:0,matrix:[0,0,0,0,0,0]},b||{});c.order='';var d=/([a-zA-Z]+)\(\s*([+-]?[\d\.]+)\s*(?:[\s,]\s*([+-]?[\d\.]+)\s*(?:[\s,]\s*([+-]?[\d\.]+)\s*(?:[\s,]\s*([+-]?[\d\.]+)\s*[\s,]\s*([+-]?[\d\.]+)\s*[\s,]\s*([+-]?[\d\.]+)\s*)?)?)?\)/g;var e=d.exec(a);while(e){switch(e[1]){case'translate':c.order+='t';c.translateX=parseFloat(e[2]);c.translateY=(e[3]?parseFloat(e[3]):0);break;case'scale':c.order+='s';c.scaleX=parseFloat(e[2]);c.scaleY=(e[3]?parseFloat(e[3]):c.scaleX);break;case'rotate':c.order+='r';c.rotateA=parseFloat(e[2]);c.rotateX=(e[3]?parseFloat(e[3]):0);c.rotateY=(e[4]?parseFloat(e[4]):0);break;case'skewX':c.order+='x';c.skewX=parseFloat(e[2]);break;case'skewY':c.order+='y';c.skewY=parseFloat(e[2]);break;case'matrix':c.order+='m';c.matrix=[parseFloat(e[2]),parseFloat(e[3]),parseFloat(e[4]),parseFloat(e[5]),parseFloat(e[6]),parseFloat(e[7])];break}e=d.exec(a)}if(c.order=='m'&&Math.abs(c.matrix[0])==Math.abs(c.matrix[3])&&c.matrix[1]!=0&&Math.abs(c.matrix[1])==Math.abs(c.matrix[2])){var f=Math.acos(c.matrix[0])*180/Math.PI;f=(c.matrix[1]<0?360-f:f);c.order='rt';c.rotateA=f;c.rotateX=c.rotateY=0;c.translateX=c.matrix[4];c.translateY=c.matrix[5]}return c}$.each(['fill','stroke'],function(i,e){var f=e.charAt(0).toUpperCase()+e.substr(1);$.fx.step['svg'+f]=$.fx.step['svg-'+e]=function(a){if(!a.set){a.start=$.svg._getColour(a.elem,e);var b=(a.end=='none');a.end=(b?$.svg._getColour(a.elem.parentNode,e):$.svg._getRGB(a.end));a.end[3]=b;$(a.elem).css(e,'');a.set=true}var c=a.elem.attributes.getNamedItem(e);var d='rgb('+[Math.min(Math.max(parseInt((a.pos*(a.end[0]-a.start[0]))+a.start[0],10),0),255),Math.min(Math.max(parseInt((a.pos*(a.end[1]-a.start[1]))+a.start[1],10),0),255),Math.min(Math.max(parseInt((a.pos*(a.end[2]-a.start[2]))+a.start[2],10),0),255)].join(',')+')';d=(a.end[3]&&a.state==1?'none':d);(c?c.nodeValue=d:a.elem.setAttribute(e,d))}});$.svg._getColour=function(a,b){a=$(a);var c;do{c=a.attr(b)||a.css(b);if((c!=''&&c!='none')||a.hasClass($.svg.markerClassName)){break}}while(a=a.parent());return $.svg._getRGB(c)};$.svg._getRGB=function(a){var b;if(a&&a.constructor==Array){return(a.length==3||a.length==4?a:h['none'])}if(b=/^rgb\(\s*([0-9]{1,3})\s*,\s*([0-9]{1,3})\s*,\s*([0-9]{1,3})\s*\)$/.exec(a)){return[parseInt(b[1],10),parseInt(b[2],10),parseInt(b[3],10)]}if(b=/^rgb\(\s*([0-9]+(?:\.[0-9]+)?)\%\s*,\s*([0-9]+(?:\.[0-9]+)?)\%\s*,\s*([0-9]+(?:\.[0-9]+)?)\%\s*\)$/.exec(a)){return[parseFloat(b[1])*2.55,parseFloat(b[2])*2.55,parseFloat(b[3])*2.55]}if(b=/^#([a-fA-F0-9]{2})([a-fA-F0-9]{2})([a-fA-F0-9]{2})$/.exec(a)){return[parseInt(b[1],16),parseInt(b[2],16),parseInt(b[3],16)]}if(b=/^#([a-fA-F0-9])([a-fA-F0-9])([a-fA-F0-9])$/.exec(a)){return[parseInt(b[1]+b[1],16),parseInt(b[2]+b[2],16),parseInt(b[3]+b[3],16)]}return h[$.trim(a).toLowerCase()]||h['none']};var h={'':[255,255,255,1],none:[255,255,255,1],aliceblue:[240,248,255],antiquewhite:[250,235,215],aqua:[0,255,255],aquamarine:[127,255,212],azure:[240,255,255],beige:[245,245,220],bisque:[255,228,196],black:[0,0,0],blanchedalmond:[255,235,205],blue:[0,0,255],blueviolet:[138,43,226],brown:[165,42,42],burlywood:[222,184,135],cadetblue:[95,158,160],chartreuse:[127,255,0],chocolate:[210,105,30],coral:[255,127,80],cornflowerblue:[100,149,237],cornsilk:[255,248,220],crimson:[220,20,60],cyan:[0,255,255],darkblue:[0,0,139],darkcyan:[0,139,139],darkgoldenrod:[184,134,11],darkgray:[169,169,169],darkgreen:[0,100,0],darkgrey:[169,169,169],darkkhaki:[189,183,107],darkmagenta:[139,0,139],darkolivegreen:[85,107,47],darkorange:[255,140,0],darkorchid:[153,50,204],darkred:[139,0,0],darksalmon:[233,150,122],darkseagreen:[143,188,143],darkslateblue:[72,61,139],darkslategray:[47,79,79],darkslategrey:[47,79,79],darkturquoise:[0,206,209],darkviolet:[148,0,211],deeppink:[255,20,147],deepskyblue:[0,191,255],dimgray:[105,105,105],dimgrey:[105,105,105],dodgerblue:[30,144,255],firebrick:[178,34,34],floralwhite:[255,250,240],forestgreen:[34,139,34],fuchsia:[255,0,255],gainsboro:[220,220,220],ghostwhite:[248,248,255],gold:[255,215,0],goldenrod:[218,165,32],gray:[128,128,128],grey:[128,128,128],green:[0,128,0],greenyellow:[173,255,47],honeydew:[240,255,240],hotpink:[255,105,180],indianred:[205,92,92],indigo:[75,0,130],ivory:[255,255,240],khaki:[240,230,140],lavender:[230,230,250],lavenderblush:[255,240,245],lawngreen:[124,252,0],lemonchiffon:[255,250,205],lightblue:[173,216,230],lightcoral:[240,128,128],lightcyan:[224,255,255],lightgoldenrodyellow:[250,250,210],lightgray:[211,211,211],lightgreen:[144,238,144],lightgrey:[211,211,211],lightpink:[255,182,193],lightsalmon:[255,160,122],lightseagreen:[32,178,170],lightskyblue:[135,206,250],lightslategray:[119,136,153],lightslategrey:[119,136,153],lightsteelblue:[176,196,222],lightyellow:[255,255,224],lime:[0,255,0],limegreen:[50,205,50],linen:[250,240,230],magenta:[255,0,255],maroon:[128,0,0],mediumaquamarine:[102,205,170],mediumblue:[0,0,205],mediumorchid:[186,85,211],mediumpurple:[147,112,219],mediumseagreen:[60,179,113],mediumslateblue:[123,104,238],mediumspringgreen:[0,250,154],mediumturquoise:[72,209,204],mediumvioletred:[199,21,133],midnightblue:[25,25,112],mintcream:[245,255,250],mistyrose:[255,228,225],moccasin:[255,228,181],navajowhite:[255,222,173],navy:[0,0,128],oldlace:[253,245,230],olive:[128,128,0],olivedrab:[107,142,35],orange:[255,165,0],orangered:[255,69,0],orchid:[218,112,214],palegoldenrod:[238,232,170],palegreen:[152,251,152],paleturquoise:[175,238,238],palevioletred:[219,112,147],papayawhip:[255,239,213],peachpuff:[255,218,185],peru:[205,133,63],pink:[255,192,203],plum:[221,160,221],powderblue:[176,224,230],purple:[128,0,128],red:[255,0,0],rosybrown:[188,143,143],royalblue:[65,105,225],saddlebrown:[139,69,19],salmon:[250,128,114],sandybrown:[244,164,96],seagreen:[46,139,87],seashell:[255,245,238],sienna:[160,82,45],silver:[192,192,192],skyblue:[135,206,235],slateblue:[106,90,205],slategray:[112,128,144],slategrey:[112,128,144],snow:[255,250,250],springgreen:[0,255,127],steelblue:[70,130,180],tan:[210,180,140],teal:[0,128,128],thistle:[216,191,216],tomato:[255,99,71],turquoise:[64,224,208],violet:[238,130,238],wheat:[245,222,179],white:[255,255,255],whitesmoke:[245,245,245],yellow:[255,255,0],yellowgreen:[154,205,50]}})(jQuery);
\ No newline at end of file
diff --git a/src/web/js/jquery.svgdom.js b/src/web/js/jquery.svgdom.js
new file mode 100644
index 0000000..b5ad8af
--- /dev/null
+++ b/src/web/js/jquery.svgdom.js
@@ -0,0 +1,406 @@
+/* http://keith-wood.name/svg.html
+   jQuery DOM compatibility for jQuery SVG v1.4.5.
+   Written by Keith Wood (kbwood{at}iinet.com.au) April 2009.
+   Dual licensed under the GPL (http://dev.jquery.com/browser/trunk/jquery/GPL-LICENSE.txt) and 
+   MIT (http://dev.jquery.com/browser/trunk/jquery/MIT-LICENSE.txt) licenses. 
+   Please attribute the author if you use it. */
+
+(function($) { // Hide scope, no $ conflict
+
+/* Support adding class names to SVG nodes. */
+$.fn.addClass = function(origAddClass) {
+	return function(classNames) {
+		classNames = classNames || '';
+		return this.each(function() {
+			if ($.svg.isSVGElem(this)) {
+				var node = this;
+				$.each(classNames.split(/\s+/), function(i, className) {
+					var classes = (node.className ? node.className.baseVal : node.getAttribute('class'));
+					if ($.inArray(className, classes.split(/\s+/)) == -1) {
+						classes += (classes ? ' ' : '') + className;
+						(node.className ? node.className.baseVal = classes :
+							node.setAttribute('class',  classes));
+					}
+				});
+			}
+			else {
+				origAddClass.apply($(this), [classNames]);
+			}
+		});
+	};
+}($.fn.addClass);
+
+/* Support removing class names from SVG nodes. */
+$.fn.removeClass = function(origRemoveClass) {
+	return function(classNames) {
+		classNames = classNames || '';
+		return this.each(function() {
+			if ($.svg.isSVGElem(this)) {
+				var node = this;
+				$.each(classNames.split(/\s+/), function(i, className) {
+					var classes = (node.className ? node.className.baseVal : node.getAttribute('class'));
+					classes = $.grep(classes.split(/\s+/), function(n, i) { return n != className; }).
+						join(' ');
+					(node.className ? node.className.baseVal = classes :
+						node.setAttribute('class', classes));
+				});
+			}
+			else {
+				origRemoveClass.apply($(this), [classNames]);
+			}
+		});
+	};
+}($.fn.removeClass);
+
+/* Support toggling class names on SVG nodes. */
+$.fn.toggleClass = function(origToggleClass) {
+	return function(className, state) {
+		return this.each(function() {
+			if ($.svg.isSVGElem(this)) {
+				if (typeof state !== 'boolean') {
+					state = !$(this).hasClass(className);
+				}
+				$(this)[(state ? 'add' : 'remove') + 'Class'](className);
+			}
+			else {
+				origToggleClass.apply($(this), [className, state]);
+			}
+		});
+	};
+}($.fn.toggleClass);
+
+/* Support checking class names on SVG nodes. */
+$.fn.hasClass = function(origHasClass) {
+	return function(className) {
+		className = className || '';
+		var found = false;
+		this.each(function() {
+			if ($.svg.isSVGElem(this)) {
+				var classes = (this.className ? this.className.baseVal :
+					this.getAttribute('class')).split(/\s+/);
+				found = ($.inArray(className, classes) > -1);
+			}
+			else {
+				found = (origHasClass.apply($(this), [className]));
+			}
+			return !found;
+		});
+		return found;
+	};
+}($.fn.hasClass);
+
+/* Support attributes on SVG nodes. */
+$.fn.attr = function(origAttr) {
+	return function(name, value, type) {
+		if (typeof name === 'string' && value === undefined) {
+			var val = origAttr.apply(this, [name]);
+			if (val && val.baseVal && val.baseVal.numberOfItems != null) { // Multiple values
+				value = '';
+				val = val.baseVal;
+				if (name == 'transform') {
+					for (var i = 0; i < val.numberOfItems; i++) {
+						var item = val.getItem(i);
+						switch (item.type) {
+							case 1: value += ' matrix(' + item.matrix.a + ',' + item.matrix.b + ',' +
+										item.matrix.c + ',' + item.matrix.d + ',' +
+										item.matrix.e + ',' + item.matrix.f + ')';
+									break;
+							case 2: value += ' translate(' + item.matrix.e + ',' + item.matrix.f + ')'; break;
+							case 3: value += ' scale(' + item.matrix.a + ',' + item.matrix.d + ')'; break;
+							case 4: value += ' rotate(' + item.angle + ')'; break; // Doesn't handle new origin
+							case 5: value += ' skewX(' + item.angle + ')'; break;
+							case 6: value += ' skewY(' + item.angle + ')'; break;
+						}
+					}
+					val = value.substring(1);
+				}
+				else {
+					val = val.getItem(0).valueAsString;
+				}
+			}
+			return (val && val.baseVal ? val.baseVal.valueAsString : val);
+		}
+
+		var options = name;
+		if (typeof name === 'string') {
+			options = {};
+			options[name] = value;
+		}
+		return this.each(function() {
+			if ($.svg.isSVGElem(this)) {
+				for (var n in options) {
+					var val = ($.isFunction(options[n]) ? options[n]() : options[n]);
+					(type ? this.style[n] = val : this.setAttribute(n, val));
+				}
+			}
+			else {
+				origAttr.apply($(this), [name, value, type]);
+			}
+		});
+	};
+}($.fn.attr);
+
+/* Support removing attributes on SVG nodes. */
+$.fn.removeAttr = function(origRemoveAttr) {
+	return function(name) {
+		return this.each(function() {
+			if ($.svg.isSVGElem(this)) {
+				(this[name] && this[name].baseVal ? this[name].baseVal.value = '' :
+					this.setAttribute(name, ''));
+			}
+			else {
+				origRemoveAttr.apply($(this), [name]);
+			}
+		});
+	};
+}($.fn.removeAttr);
+
+/* Add numeric only properties. */
+$.extend($.cssNumber, {
+	'stopOpacity': true,
+	'strokeMitrelimit': true,
+	'strokeOpacity': true
+});
+
+/* Support retrieving CSS/attribute values on SVG nodes. */
+if ($.cssProps) {
+	$.css = function(origCSS) {
+		return function(elem, name, extra) {
+			var value = (name.match(/^svg.*/) ? $(elem).attr($.cssProps[name] || name) : '');
+			return value || origCSS(elem, name, extra);
+		};
+	}($.css);
+}
+  
+/* Determine if any nodes are SVG nodes. */
+function anySVG(checkSet) {
+	for (var i = 0; i < checkSet.length; i++) {
+		if (checkSet[i].nodeType == 1 && checkSet[i].namespaceURI == $.svg.svgNS) {
+			return true;
+		}
+	}
+	return false;
+}
+
+/* Update Sizzle selectors. */
+
+$.expr.relative['+'] = function(origRelativeNext) {
+	return function(checkSet, part, isXML) {
+		origRelativeNext(checkSet, part, isXML || anySVG(checkSet));
+	};
+}($.expr.relative['+']);
+
+$.expr.relative['>'] = function(origRelativeChild) {
+	return function(checkSet, part, isXML) {
+		origRelativeChild(checkSet, part, isXML || anySVG(checkSet));
+	};
+}($.expr.relative['>']);
+
+$.expr.relative[''] = function(origRelativeDescendant) {
+	return function(checkSet, part, isXML) {
+		origRelativeDescendant(checkSet, part, isXML || anySVG(checkSet));
+	};
+}($.expr.relative['']);
+
+$.expr.relative['~'] = function(origRelativeSiblings) {
+	return function(checkSet, part, isXML) {
+		origRelativeSiblings(checkSet, part, isXML || anySVG(checkSet));
+	};
+}($.expr.relative['~']);
+
+$.expr.find.ID = function(origFindId) {
+	return function(match, context, isXML) {
+		return ($.svg.isSVGElem(context) ?
+			[context.ownerDocument.getElementById(match[1])] :
+			origFindId(match, context, isXML));
+	};
+}($.expr.find.ID);
+
+var div = document.createElement('div');
+div.appendChild(document.createComment(''));
+if (div.getElementsByTagName('*').length > 0) { // Make sure no comments are found
+	$.expr.find.TAG = function(match, context) {
+		var results = context.getElementsByTagName(match[1]);
+		if (match[1] === '*') { // Filter out possible comments
+			var tmp = [];
+			for (var i = 0; results[i] || results.item(i); i++) {
+				if ((results[i] || results.item(i)).nodeType === 1) {
+					tmp.push(results[i] || results.item(i));
+				}
+			}
+			results = tmp;
+		}
+		return results;
+	};
+}
+
+$.expr.preFilter.CLASS = function(match, curLoop, inplace, result, not, isXML) {
+	match = ' ' + match[1].replace(/\\/g, '') + ' ';
+	if (isXML) {
+		return match;
+	}
+	for (var i = 0, elem = {}; elem != null; i++) {
+		elem = curLoop[i];
+		if (!elem) {
+			try {
+				elem = curLoop.item(i);
+			}
+			catch (e) {
+				// Ignore
+			}
+		}
+		if (elem) {
+			var className = (!$.svg.isSVGElem(elem) ? elem.className :
+				(elem.className ? elem.className.baseVal : '') || elem.getAttribute('class'));
+			if (not ^ (className && (' ' + className + ' ').indexOf(match) > -1)) {
+				if (!inplace)
+					result.push(elem);
+			}
+			else if (inplace) {
+				curLoop[i] = false;
+			}
+		}
+	}
+	return false;
+};
+
+$.expr.filter.CLASS = function(elem, match) {
+	var className = (!$.svg.isSVGElem(elem) ? elem.className :
+		(elem.className ? elem.className.baseVal : elem.getAttribute('class')));
+	return (' ' + className + ' ').indexOf(match) > -1;
+};
+
+$.expr.filter.ATTR = function(origFilterAttr) {
+	return function(elem, match) {
+		var handler = null;
+		if ($.svg.isSVGElem(elem)) {
+			handler = match[1];
+			$.expr.attrHandle[handler] = function(elem){
+				var attr = elem.getAttribute(handler);
+				return attr && attr.baseVal || attr;
+			};
+		}
+		var filter = origFilterAttr(elem, match);
+		if (handler) {
+			$.expr.attrHandle[handler] = null;
+		}
+		return filter;
+	};
+}($.expr.filter.ATTR);
+
+/*
+	In the removeData function (line 1881, v1.7.2):
+
+				if ( jQuery.support.deleteExpando ) {
+					delete elem[ internalKey ];
+				} else {
+					try { // SVG
+						elem.removeAttribute( internalKey );
+					} catch (e) {
+						elem[ internalKey ] = null;
+					}
+				}
+
+	In the event.add function (line 2985, v1.7.2):
+
+				if ( !special.setup || special.setup.call( elem, data, namespaces, eventHandle ) === false ) {
+					// Bind the global event handler to the element
+					try { // SVG
+						elem.addEventListener( type, eventHandle, false );
+					} catch(e) {
+						if ( elem.attachEvent ) {
+							elem.attachEvent( "on" + type, eventHandle );
+						}
+					}
+				}
+
+	In the event.remove function (line 3074, v1.7.2):
+
+			if ( !special.teardown || special.teardown.call( elem, namespaces ) === false ) {
+				try { // SVG
+					elem.removeEventListener(type, elemData.handle, false);
+				}
+				catch (e) {
+					if (elem.detachEvent) {
+						elem.detachEvent("on" + type, elemData.handle);
+					}
+				}
+			}
+
+	In the event.fix function (line 3394, v1.7.2):
+
+		if (event.target.namespaceURI == 'http://www.w3.org/2000/svg') { // SVG
+			event.button = [1, 4, 2][event.button];
+		}
+
+		// Add which for click: 1 === left; 2 === middle; 3 === right
+		// Note: button is not normalized, so don't use it
+		if ( !event.which && button !== undefined ) {
+			event.which = ( button & 1 ? 1 : ( button & 2 ? 3 : ( button & 4 ? 2 : 0 ) ) );
+		}
+
+	In the Sizzle function (line 4083, v1.7.2):
+
+	if ( toString.call(checkSet) === "[object Array]" ) {
+		if ( !prune ) {
+			results.push.apply( results, checkSet );
+
+		} else if ( context && context.nodeType === 1 ) {
+			for ( i = 0; checkSet[i] != null; i++ ) {
+				if ( checkSet[i] && (checkSet[i] === true || checkSet[i].nodeType === 1 && Sizzle.contains(context, checkSet[i])) ) {
+				results.push( set[i] || set.item(i) ); // SVG
+				}
+			}
+
+		} else {
+			for ( i = 0; checkSet[i] != null; i++ ) {
+				if ( checkSet[i] && checkSet[i].nodeType === 1 ) {
+					results.push( set[i] || set.item(i) ); // SVG
+				}
+			}
+		}
+	} else {...
+
+	In the fallback for the Sizzle makeArray function (line 4877, v1.7.2):
+
+	if ( toString.call(array) === "[object Array]" ) {
+		Array.prototype.push.apply( ret, array );
+
+	} else {
+		if ( typeof array.length === "number" ) {
+			for ( var l = array.length; i &lt; l; i++ ) {
+				ret.push( array[i] || array.item(i) ); // SVG
+			}
+
+		} else {
+			for ( ; array[i]; i++ ) {
+				ret.push( array[i] );
+			}
+		}
+	}
+
+	In the jQuery.cleandata function (line 6538, v1.7.2):
+
+				if ( deleteExpando ) {
+					delete elem[ jQuery.expando ];
+
+				} else {
+					try { // SVG
+						elem.removeAttribute( jQuery.expando );
+					} catch (e) {
+						// Ignore
+					}
+				}
+
+	In the fallback getComputedStyle function (line 6727, v1.7.2):
+
+		defaultView = (elem.ownerDocument ? elem.ownerDocument.defaultView : elem.defaultView); // SVG
+		if ( defaultView &&
+		(computedStyle = defaultView.getComputedStyle( elem, null )) ) {
+
+			ret = computedStyle.getPropertyValue( name );
+			...
+
+*/
+
+})(jQuery);
diff --git a/src/web/js/jquery.svgdom.min.js b/src/web/js/jquery.svgdom.min.js
new file mode 100644
index 0000000..3c280a5
--- /dev/null
+++ b/src/web/js/jquery.svgdom.min.js
@@ -0,0 +1,7 @@
+/* http://keith-wood.name/svg.html
+   jQuery DOM compatibility for jQuery SVG v1.4.5.
+   Written by Keith Wood (kbwood{at}iinet.com.au) April 2009.
+   Dual licensed under the GPL (http://dev.jquery.com/browser/trunk/jquery/GPL-LICENSE.txt) and 
+   MIT (http://dev.jquery.com/browser/trunk/jquery/MIT-LICENSE.txt) licenses. 
+   Please attribute the author if you use it. */
+(function($){$.fn.addClass=function(e){return function(d){d=d||'';return this.each(function(){if($.svg.isSVGElem(this)){var c=this;$.each(d.split(/\s+/),function(i,a){var b=(c.className?c.className.baseVal:c.getAttribute('class'));if($.inArray(a,b.split(/\s+/))==-1){b+=(b?' ':'')+a;(c.className?c.className.baseVal=b:c.setAttribute('class',b))}})}else{e.apply($(this),[d])}})}}($.fn.addClass);$.fn.removeClass=function(e){return function(d){d=d||'';return this.each(function(){if($.svg.isSVGElem(this)){var c=this;$.each(d.split(/\s+/),function(i,a){var b=(c.className?c.className.baseVal:c.getAttribute('class'));b=$.grep(b.split(/\s+/),function(n,i){return n!=a}).join(' ');(c.className?c.className.baseVal=b:c.setAttribute('class',b))})}else{e.apply($(this),[d])}})}}($.fn.removeClass);$.fn.toggleClass=function(c){return function(a,b){return this.each(function(){if($.svg.isSVGElem(this)){if(typeof b!=='boolean'){b=!$(this).hasClass(a)}$(this)[(b?'add':'remove')+'Class'](a)}else{c.apply($(this),[a,b])}})}}($.fn.toggleClass);$.fn.hasClass=function(d){return function(b){b=b||'';var c=false;this.each(function(){if($.svg.isSVGElem(this)){var a=(this.className?this.className.baseVal:this.getAttribute('class')).split(/\s+/);c=($.inArray(b,a)>-1)}else{c=(d.apply($(this),[b]))}return!c});return c}}($.fn.hasClass);$.fn.attr=function(h){return function(b,c,d){if(typeof b==='string'&&c===undefined){var e=h.apply(this,[b]);if(e&&e.baseVal&&e.baseVal.numberOfItems!=null){c='';e=e.baseVal;if(b=='transform'){for(var i=0;i<e.numberOfItems;i++){var f=e.getItem(i);switch(f.type){case 1:c+=' matrix('+f.matrix.a+','+f.matrix.b+','+f.matrix.c+','+f.matrix.d+','+f.matrix.e+','+f.matrix.f+')';break;case 2:c+=' translate('+f.matrix.e+','+f.matrix.f+')';break;case 3:c+=' scale('+f.matrix.a+','+f.matrix.d+')';break;case 4:c+=' rotate('+f.angle+')';break;case 5:c+=' skewX('+f.angle+')';break;case 6:c+=' skewY('+f.angle+')';break}}e=c.substring(1)}else{e=e.getItem(0).valueAsString}}return(e&&e.baseVal?e.baseVal.valueAsString:e)}var g=b;if(typeof b==='string'){g={};g[b]=c}return this.each(function(){if($.svg.isSVGElem(this)){for(var n in g){var a=($.isFunction(g[n])?g[n]():g[n]);(d?this.style[n]=a:this.setAttribute(n,a))}}else{h.apply($(this),[b,c,d])}})}}($.fn.attr);$.fn.removeAttr=function(b){return function(a){return this.each(function(){if($.svg.isSVGElem(this)){(this[a]&&this[a].baseVal?this[a].baseVal.value='':this.setAttribute(a,''))}else{b.apply($(this),[a])}})}}($.fn.removeAttr);$.extend($.cssNumber,{'stopOpacity':true,'strokeMitrelimit':true,'strokeOpacity':true});if($.cssProps){$.css=function(e){return function(a,b,c){var d=(b.match(/^svg.*/)?$(a).attr($.cssProps[b]||b):'');return d||e(a,b,c)}}($.css)}function anySVG(a){for(var i=0;i<a.length;i++){if(a[i].nodeType==1&&a[i].namespaceURI==$.svg.svgNS){return true}}return false}$.expr.relative['+']=function(d){return function(a,b,c){d(a,b,c||anySVG(a))}}($.expr.relative['+']);$.expr.relative['>']=function(d){return function(a,b,c){d(a,b,c||anySVG(a))}}($.expr.relative['>']);$.expr.relative['']=function(d){return function(a,b,c){d(a,b,c||anySVG(a))}}($.expr.relative['']);$.expr.relative['~']=function(d){return function(a,b,c){d(a,b,c||anySVG(a))}}($.expr.relative['~']);$.expr.find.ID=function(d){return function(a,b,c){return($.svg.isSVGElem(b)?[b.ownerDocument.getElementById(a[1])]:d(a,b,c))}}($.expr.find.ID);var j=document.createElement('div');j.appendChild(document.createComment(''));if(j.getElementsByTagName('*').length>0){$.expr.find.TAG=function(a,b){var c=b.getElementsByTagName(a[1]);if(a[1]==='*'){var d=[];for(var i=0;c[i]||c.item(i);i++){if((c[i]||c.item(i)).nodeType===1){d.push(c[i]||c.item(i))}}c=d}return c}}$.expr.preFilter.CLASS=function(a,b,c,d,f,g){a=' '+a[1].replace(/\\/g,'')+' ';if(g){return a}for(var i=0,elem={};elem!=null;i++){elem=b[i];if(!elem){try{elem=b.item(i)}catch(e){}}if(elem){var h=(!$.svg.isSVGElem(elem)?elem.className:(elem.className?elem.className.baseVal:'')||elem.getAttribute('class'));if(f^(h&&(' '+h+' ').indexOf(a)>-1)){if(!c)d.push(elem)}else if(c){b[i]=false}}}return false};$.expr.filter.CLASS=function(a,b){var c=(!$.svg.isSVGElem(a)?a.className:(a.className?a.className.baseVal:a.getAttribute('class')));return(' '+c+' ').indexOf(b)>-1};$.expr.filter.ATTR=function(g){return function(c,d){var e=null;if($.svg.isSVGElem(c)){e=d[1];$.expr.attrHandle[e]=function(a){var b=a.getAttribute(e);return b&&b.baseVal||b}}var f=g(c,d);if(e){$.expr.attrHandle[e]=null}return f}}($.expr.filter.ATTR)})(jQuery);
\ No newline at end of file
diff --git a/src/web/js/jquery.svgfilter.min.js b/src/web/js/jquery.svgfilter.min.js
new file mode 100644
index 0000000..551bdc9
--- /dev/null
+++ b/src/web/js/jquery.svgfilter.min.js
@@ -0,0 +1,7 @@
+/* http://keith-wood.name/svg.html
+   SVG filters for jQuery v1.4.5.
+   Written by Keith Wood (kbwood{at}iinet.com.au) August 2007.
+   Dual licensed under the GPL (http://dev.jquery.com/browser/trunk/jquery/GPL-LICENSE.txt) and
+   MIT (http://dev.jquery.com/browser/trunk/jquery/MIT-LICENSE.txt) licenses.
+   Please attribute the author if you use it. */
+(function($){$.svg.addExtension('filters',SVGFilter);$.extend($.svg._wrapperClass.prototype,{filter:function(a,b,x,y,c,d,e){var f=this._args(arguments,['id','x','y','width','height']);return this._makeNode(f.parent,'filter',$.extend({id:f.id,x:f.x,y:f.y,width:f.width,height:f.height},f.settings||{}))}});function SVGFilter(a){this._wrapper=a}$.extend(SVGFilter.prototype,{distantLight:function(a,b,c,d,e){var f=this._wrapper._args(arguments,['result','azimuth','elevation']);return this._wrapper._makeNode(f.parent,'feDistantLight',$.extend({result:f.result,azimuth:f.azimuth,elevation:f.elevation},f.settings||{}))},pointLight:function(a,b,x,y,z,c){var d=this._wrapper._args(arguments,['result','x','y','z']);return this._wrapper._makeNode(d.parent,'fePointLight',$.extend({result:d.result,x:d.x,y:d.y,z:d.z},d.settings||{}))},spotLight:function(a,b,x,y,z,c,d,e,f){var g=this._wrapper._args(arguments,['result','x','y','z','toX','toY','toZ'],['toX']);var h=$.extend({result:g.result,x:g.x,y:g.y,z:g.z},(g.toX!=null?{pointsAtX:g.toX,pointsAtY:g.toY,pointsAtZ:g.toZ}:{}));return this._wrapper._makeNode(g.parent,'feSpotLight',$.extend(h,g.settings||{}))},blend:function(a,b,c,d,e,f){var g=this._wrapper._args(arguments,['result','mode','in1','in2']);return this._wrapper._makeNode(g.parent,'feBlend',$.extend({result:g.result,mode:g.mode,in_:g.in1,in2:g.in2},g.settings||{}))},colorMatrix:function(a,b,c,d,e,f){var g=this._wrapper._args(arguments,['result','in1','type','values']);if(isArray(g.values)){var h='';for(var i=0;i<g.values.length;i++){h+=(i==0?'':' ')+g.values[i].join(' ')}g.values=h}else if(typeof g.values=='object'){g.settings=g.values;g.values=null}var j=$.extend({result:g.result,in_:g.in1,type:g.type},(g.values!=null?{values:g.values}:{}));return this._wrapper._makeNode(g.parent,'feColorMatrix',$.extend(j,g.settings||{}))},componentTransfer:function(a,b,c,d){var e=this._wrapper._args(arguments,['result','functions']);var f=this._wrapper._makeNode(e.parent,'feComponentTransfer',$.extend({result:e.result},e.settings||{}));var g=['R','G','B','A'];for(var i=0;i<Math.min(4,e.functions.length);i++){var h=e.functions[i];var j=$.extend({type:h[0]},(h[0]=='table'||h[0]=='discrete'?{tableValues:h[1].join(' ')}:(h[0]=='linear'?{slope:h[1],intercept:h[2]}:(h[0]=='gamma'?{amplitude:h[1],exponent:h[2],offset:h[3]}:{}))));this._wrapper._makeNode(f,'feFunc'+g[i],j)}return f},composite:function(a,b,c,d,e,f,g,h,i,j){var k=this._wrapper._args(arguments,['result','operator','in1','in2','k1','k2','k3','k4'],['k1']);var l=$.extend({result:k.result,operator:k.operator,'in':k.in1,in2:k.in2},(k.k1!=null?{k1:k.k1,k2:k.k2,k3:k.k3,k4:k.k4}:{}));return this._wrapper._makeNode(k.parent,'feComposite',$.extend(l,k.settings||{}))},convolveMatrix:function(a,b,c,d,e){var f=this._wrapper._args(arguments,['result','order','matrix']);var g='';for(var i=0;i<f.matrix.length;i++){g+=(i==0?'':' ')+f.matrix[i].join(' ')}f.matrix=g;return this._wrapper._makeNode(f.parent,'feConvolveMatrix',$.extend({result:f.result,order:f.order,kernelMatrix:f.matrix},f.settings||{}))},diffuseLighting:function(a,b,c,d){var e=this._wrapper._args(arguments,['result','colour'],['colour']);return this._wrapper._makeNode(e.parent,'feDiffuseLighting',$.extend($.extend({result:e.result},(e.colour?{lightingColor:e.colour}:{})),e.settings||{}))},displacementMap:function(a,b,c,d,e){var f=this._wrapper._args(arguments,['result','in1','in2']);return this._wrapper._makeNode(f.parent,'feDisplacementMap',$.extend({result:f.result,in_:f.in1,in2:f.in2},f.settings||{}))},flood:function(a,b,x,y,c,d,e,f,g){var h=this._wrapper._args(arguments,['result','x','y','width','height','colour','opacity']);if(arguments.length<6){h.colour=h.x;h.opacity=h.y;h.settings=h.width;h.x=null}var i=$.extend({result:h.result,floodColor:h.colour,floodOpacity:h.opacity},(h.x!=null?{x:h.x,y:h.y,width:h.width,height:h.height}:{}));return this._wrapper._makeNode(h.parent,'feFlood',$.extend(i,h.settings||{}))},gaussianBlur:function(a,b,c,d,e,f){var g=this._wrapper._args(arguments,['result','in1','stdDevX','stdDevY'],['stdDevY']);return this._wrapper._makeNode(g.parent,'feGaussianBlur',$.extend({result:g.result,in_:g.in1,stdDeviation:g.stdDevX+(g.stdDevY?' '+g.stdDevY:'')},g.settings||{}))},image:function(a,b,c,d){var e=this._wrapper._args(arguments,['result','href']);var f=this._wrapper._makeNode(e.parent,'feImage',$.extend({result:e.result},e.settings||{}));f.setAttributeNS($.svg.xlinkNS,'href',e.href);return f},merge:function(a,b,c,d){var e=this._wrapper._args(arguments,['result','refs']);var f=this._wrapper._makeNode(e.parent,'feMerge',$.extend({result:e.result},e.settings||{}));for(var i=0;i<e.refs.length;i++){this._wrapper._makeNode(f,'feMergeNode',{in_:e.refs[i]})}return f},morphology:function(a,b,c,d,e,f,g){var h=this._wrapper._args(arguments,['result','in1','operator','radiusX','radiusY'],['radiusY']);return this._wrapper._makeNode(h.parent,'feMorphology',$.extend({result:h.result,in_:h.in1,operator:h.operator,radius:h.radiusX+(h.radiusY?' '+h.radiusY:'')},h.settings||{}))},offset:function(a,b,c,d,e,f){var g=this._wrapper._args(arguments,['result','in1','dx','dy']);return this._wrapper._makeNode(g.parent,'feOffset',$.extend({result:g.result,in_:g.in1,dx:g.dx,dy:g.dy},g.settings||{}))},specularLighting:function(a,b,c,d,e,f,g){var h=this._wrapper._args(arguments,['result','in1','surfaceScale','specularConstant','specularExponent'],['surfaceScale','specularConstant','specularExponent']);return this._wrapper._makeNode(h.parent,'feSpecularLighting',$.extend({result:h.result,in_:h.in1,surfaceScale:h.surfaceScale,specularConstant:h.specularConstant,specularExponent:h.specularExponent},h.settings||{}))},tile:function(a,b,c,x,y,d,e,f){var g=this._wrapper._args(arguments,['result','in1','x','y','width','height']);return this._wrapper._makeNode(g.parent,'feTile',$.extend({result:g.result,in_:g.in1,x:g.x,y:g.y,width:g.width,height:g.height},g.settings||{}))},turbulence:function(a,b,c,d,e,f){var g=this._wrapper._args(arguments,['result','type','baseFreq','octaves'],['octaves']);return this._wrapper._makeNode(g.parent,'feTurbulence',$.extend({result:g.result,type:g.type,baseFrequency:g.baseFreq,numOctaves:g.octaves},g.settings||{}))}});function isArray(a){return(a&&a.constructor==Array)}})(jQuery)
\ No newline at end of file
diff --git a/src/web/js/morris.min.js b/src/web/js/morris.min.js
new file mode 100644
index 0000000..edc5d7b
--- /dev/null
+++ b/src/web/js/morris.min.js
@@ -0,0 +1 @@
+(function(){var e,t,n,r,i=[].slice,s={}.hasOwnProperty,o=function(e,t){function r(){this.constructor=e}for(var n in t)s.call(t,n)&&(e[n]=t[n]);return r.prototype=t.prototype,e.prototype=new r,e.__super__=t.prototype,e},u=function(e,t){return function(){return e.apply(t,arguments)}},a=[].indexOf||function(e){for(var t=0,n=this.length;t<n;t++)if(t in this&&this[t]===e)return t;return-1};t=window.Morris={},e=jQuery,t.EventEmitter=function(){function e(){}return e.prototype.on=function(e,t){return this.handlers==null&&(this.handlers={}),this.handlers[e]==null&&(this.handlers[e]=[]),this.handlers[e].push(t),this},e.prototype.fire=function(){var e,t,n,r,s,o,u;n=arguments[0],e=2<=arguments.length?i.call(arguments,1):[];if(this.handlers!=null&&this.handlers[n]!=null){o=this.handlers[n],u=[];for(r=0,s=o.length;r<s;r++)t=o[r],u.push(t.apply(null,e));return u}},e}(),t.commas=function(e){var t,n,r,i;return e!=null?(r=e<0?"-":"",t=Math.abs(e),n=Math.floor(t).toFixed(0),r+=n.replace(/(?=(?:\d{3})+$)(?!^)/g,","),i=t.toString(),i.length>n.length&&(r+=i.slice(n.length)),r):"-"},t.pad2=function(e){return(e<10?"0":"")+e},t.Grid=function(n){function r(t){var n=this;typeof t.element=="string"?this.el=e(document.getElementById(t.element)):this.el=e(t.element);if(this.el==null||this.el.length===0)throw new Error("Graph container element not found");this.el.css("position")==="static"&&this.el.css("position","relative"),this.options=e.extend({},this.gridDefaults,this.defaults||{},t),typeof this.options.units=="string"&&(this.options.postUnits=t.units),this.raphael=new Raphael(this.el[0]),this.elementWidth=null,this.elementHeight=null,this.dirty=!1,this.init&&this.init(),this.setData(this.options.data),this.el.bind("mousemove",function(e){var t;return t=n.el.offset(),n.fire("hovermove",e.pageX-t.left,e.pageY-t.top)}),this.el.bind("mouseout",function(e){return n.fire("hoverout")}),this.el.bind("touchstart touchmove touchend",function(e){var t,r;return r=e.originalEvent.touches[0]||e.originalEvent.changedTouches[0],t=n.el.offset(),n.fire("hover",r.pageX-t.left,r.pageY-t.top),r}),this.el.bind("click",function(e){var t;return t=n.el.offset(),n.fire("gridclick",e.pageX-t.left,e.pageY-t.top)}),this.postInit&&this.postInit()}return o(r,n),r.prototype.gridDefaults={dateFormat:null,axes:!0,grid:!0,gridLineColor:"#aaa",gridStrokeWidth:.5,gridTextColor:"#888",gridTextSize:12,gridTextFamily:"sans-serif",gridTextWeight:"normal",hideHover:!1,yLabelFormat:null,xLabelAngle:0,numLines:5,padding:25,parseTime:!0,postUnits:"",preUnits:"",ymax:"auto",ymin:"auto 0",goals:[],goalStrokeWidth:1,goalLineColors:["#666633","#999966","#cc6666","#663333"],events:[],eventStrokeWidth:1,eventLineColors:["#005a04","#ccffbb","#3a5f0b","#005502"]},r.prototype.setData=function(e,n){var r,i,s,o,u,a,f,l,c,h,p,d,v,m;n==null&&(n=!0),this.options.data=e;if(e==null||e.length===0){this.data=[],this.raphael.clear(),this.hover!=null&&this.hover.hide();return}d=this.cumulative?0:null,v=this.cumulative?0:null,this.options.goals.length>0&&(u=Math.min.apply(null,this.options.goals),o=Math.max.apply(null,this.options.goals),v=v!=null?Math.min(v,u):u,d=d!=null?Math.max(d,o):o),this.data=function(){var n,r,o;o=[];for(s=n=0,r=e.length;n<r;s=++n)f=e[s],a={},a.label=f[this.options.xkey],this.options.parseTime?(a.x=t.parseDate(a.label),this.options.dateFormat?a.label=this.options.dateFormat(a.x):typeof a.label=="number"&&(a.label=(new Date(a.label)).toString())):(a.x=s,this.options.xLabelFormat&&(a.label=this.options.xLabelFormat(a))),c=0,a.y=function(){var e,t,n,r;n=this.options.ykeys,r=[];for(i=e=0,t=n.length;e<t;i=++e)p=n[i],m=f[p],typeof m=="string"&&(m=parseFloat(m)),m!=null&&typeof m!="number"&&(m=null),m!=null&&(this.cumulative?c+=m:d!=null?(d=Math.max(m,d),v=Math.min(m,v)):d=v=m),this.cumulative&&c!=null&&(d=Math.max(c,d),v=Math.min(c,v)),r.push(m);return r}.call(this),o.push(a);return o}.call(this),this.options.parseTime&&(this.data=this.data.sort(function(e,t){return(e.x>t.x)-(t.x>e.x)})),this.xmin=this.data[0].x,this.xmax=this.data[this.data.length-1].x,this.events=[],this.options.parseTime&&this.options.events.length>0&&(this.events=function(){var e,n,i,s;i=this.options.events,s=[];for(e=0,n=i.length;e<n;e++)r=i[e],s.push(t.parseDate(r));return s}.call(this),this.xmax=Math.max(this.xmax,Math.max.apply(null,this.events)),this.xmin=Math.min(this.xmin,Math.min.apply(null,this.events))),this.xmin===this.xmax&&(this.xmin-=1,this.xmax+=1),this.ymin=this.yboundary("min",v),this.ymax=this.yboundary("max",d),this.ymin===this.ymax&&(v&&(this.ymin-=1),this.ymax+=1);if(this.options.axes===!0||this.options.grid===!0)this.options.ymax===this.gridDefaults.ymax&&this.options.ymin===this.gridDefaults.ymin?(this.grid=this.autoGridLines(this.ymin,this.ymax,this.options.numLines),this.ymin=Math.min(this.ymin,this.grid[0]),this.ymax=Math.max(this.ymax,this.grid[this.grid.length-1])):(l=(this.ymax-this.ymin)/(this.options.numLines-1),this.grid=function(){var e,t,n,r;r=[];for(h=e=t=this.ymin,n=this.ymax;t<=n?e<=n:e>=n;h=e+=l)r.push(h);return r}.call(this));this.dirty=!0;if(n)return this.redraw()},r.prototype.yboundary=function(e,t){var n,r;return n=this.options["y"+e],typeof n=="string"?n.slice(0,4)==="auto"?n.length>5?(r=parseInt(n.slice(5),10),t==null?r:Math[e](t,r)):t!=null?t:0:parseInt(n,10):n},r.prototype.autoGridLines=function(e,t,n){var r,i,s,o,u,a,f,l,c;return u=t-e,c=Math.floor(Math.log(u)/Math.log(10)),f=Math.pow(10,c),i=Math.floor(e/f)*f,r=Math.ceil(t/f)*f,a=(r-i)/(n-1),f===1&&a>1&&Math.ceil(a)!==a&&(a=Math.ceil(a),r=i+a*(n-1)),i<0&&r>0&&(i=Math.floor(e/a)*a,r=Math.ceil(t/a)*a),a<1?(o=Math.floor(Math.log(a)/Math.log(10)),s=function(){var e,t;t=[];for(l=e=i;i<=r?e<=r:e>=r;l=e+=a)t.push(parseFloat(l.toFixed(1-o)));return t}()):s=function(){var e,t;t=[];for(l=e=i;i<=r?e<=r:e>=r;l=e+=a)t.push(l);return t}(),s},r.prototype._calc=function(){var e,t,n,r,i,s;i=this.el.width(),n=this.el.height();if(this.elementWidth!==i||this.elementHeight!==n||this.dirty){this.elementWidth=i,this.elementHeight=n,this.dirty=!1,this.left=this.options.padding,this.right=this.elementWidth-this.options.padding,this.top=this.options.padding,this.bottom=this.elementHeight-this.options.padding,this.options.axes&&(s=function(){var e,n,r,i;r=this.grid,i=[];for(e=0,n=r.length;e<n;e++)t=r[e],i.push(this.measureText(this.yAxisFormat(t)).width);return i}.call(this),this.left+=Math.max.apply(Math,s),e=function(){var e,t,n;n=[];for(r=e=0,t=this.data.length;0<=t?e<t:e>t;r=0<=t?++e:--e)n.push(this.measureText(this.data[r].text,-this.options.xLabelAngle).height);return n}.call(this),this.bottom-=Math.max.apply(Math,e)),this.width=Math.max(1,this.right-this.left),this.height=Math.max(1,this.bottom-this.top),this.dx=this.width/(this.xmax-this.xmin),this.dy=this.height/(this.ymax-this.ymin);if(this.calc)return this.calc()}},r.prototype.transY=function(e){return this.bottom-(e-this.ymin)*this.dy},r.prototype.transX=function(e){return this.data.length===1?(this.left+this.right)/2:this.left+(e-this.xmin)*this.dx},r.prototype.redraw=function(){this.raphael.clear(),this._calc(),this.drawGrid(),this.drawGoals(),this.drawEvents();if(this.draw)return this.draw()},r.prototype.measureText=function(e,t){var n,r;return t==null&&(t=0),r=this.raphael.text(100,100,e).attr("font-size",this.options.gridTextSize).attr("font-family",this.options.gridTextFamily).attr("font-weight",this.options.gridTextWeight).rotate(t),n=r.getBBox(),r.remove(),n},r.prototype.yAxisFormat=function(e){return this.yLabelFormat(e)},r.prototype.yLabelFormat=function(e){return typeof this.options.yLabelFormat=="function"?this.options.yLabelFormat(e):""+this.options.preUnits+t.commas(e)+this.options.postUnits},r.prototype.updateHover=function(e,t){var n,r;n=this.hitTest(e,t);if(n!=null)return(r=this.hover).update.apply(r,n)},r.prototype.drawGrid=function(){var e,t,n,r,i,s;if(this.options.grid===!1&&this.options.axes===!1)return;i=this.grid,s=[];for(n=0,r=i.length;n<r;n++)e=i[n],t=this.transY(e),this.options.axes&&this.drawYAxisLabel(this.left-this.options.padding/2,t,this.yAxisFormat(e)),this.options.grid?s.push(this.drawGridLine("M"+this.left+","+t+"H"+(this.left+this.width))):s.push(void 0);return s},r.prototype.drawGoals=function(){var e,t,n,r,i,s,o;s=this.options.goals,o=[];for(n=r=0,i=s.length;r<i;n=++r)t=s[n],e=this.options.goalLineColors[n%this.options.goalLineColors.length],o.push(this.drawGoal(t,e));return o},r.prototype.drawEvents=function(){var e,t,n,r,i,s,o;s=this.events,o=[];for(n=r=0,i=s.length;r<i;n=++r)t=s[n],e=this.options.eventLineColors[n%this.options.eventLineColors.length],o.push(this.drawEvent(t,e));return o},r.prototype.drawGoal=function(e,t){return this.raphael.path("M"+this.left+","+this.transY(e)+"H"+this.right).attr("stroke",t).attr("stroke-width",this.options.goalStrokeWidth)},r.prototype.drawEvent=function(e,t){return this.raphael.path("M"+this.transX(e)+","+this.bottom+"V"+this.top).attr("stroke",t).attr("stroke-width",this.options.eventStrokeWidth)},r.prototype.drawYAxisLabel=function(e,t,n){return this.raphael.text(e,t,n).attr("font-size",this.options.gridTextSize).attr("font-family",this.options.gridTextFamily).attr("font-weight",this.options.gridTextWeight).attr("fill",this.options.gridTextColor).attr("text-anchor","end")},r.prototype.drawGridLine=function(e){return this.raphael.path(e).attr("stroke",this.options.gridLineColor).attr("stroke-width",this.options.gridStrokeWidth)},r}(t.EventEmitter),t.parseDate=function(e){var t,n,r,i,s,o,u,a,f,l,c;return typeof e=="number"?e:(n=e.match(/^(\d+) Q(\d)$/),i=e.match(/^(\d+)-(\d+)$/),s=e.match(/^(\d+)-(\d+)-(\d+)$/),u=e.match(/^(\d+) W(\d+)$/),a=e.match(/^(\d+)-(\d+)-(\d+)[ T](\d+):(\d+)(Z|([+-])(\d\d):?(\d\d))?$/),f=e.match(/^(\d+)-(\d+)-(\d+)[ T](\d+):(\d+):(\d+(\.\d+)?)(Z|([+-])(\d\d):?(\d\d))?$/),n?(new Date(parseInt(n[1],10),parseInt(n[2],10)*3-1,1)).getTime():i?(new Date(parseInt(i[1],10),parseInt(i[2],10)-1,1)).getTime():s?(new Date(parseInt(s[1],10),parseInt(s[2],10)-1,parseInt(s[3],10))).getTime():u?(l=new Date(parseInt(u[1],10),0,1),l.getDay()!==4&&l.setMonth(0,1+(4-l.getDay()+7)%7),l.getTime()+parseInt(u[2],10)*6048e5):a?a[6]?(o=0,a[6]!=="Z"&&(o=parseInt(a[8],10)*60+parseInt(a[9],10),a[7]==="+"&&(o=0-o)),Date.UTC(parseInt(a[1],10),parseInt(a[2],10)-1,parseInt(a[3],10),parseInt(a[4],10),parseInt(a[5],10)+o)):(new Date(parseInt(a[1],10),parseInt(a[2],10)-1,parseInt(a[3],10),parseInt(a[4],10),parseInt(a[5],10))).getTime():f?(c=parseFloat(f[6]),t=Math.floor(c),r=Math.round((c-t)*1e3),f[8]?(o=0,f[8]!=="Z"&&(o=parseInt(f[10],10)*60+parseInt(f[11],10),f[9]==="+"&&(o=0-o)),Date.UTC(parseInt(f[1],10),parseInt(f[2],10)-1,parseInt(f[3],10),parseInt(f[4],10),parseInt(f[5],10)+o,t,r)):(new Date(parseInt(f[1],10),parseInt(f[2],10)-1,parseInt(f[3],10),parseInt(f[4],10),parseInt(f[5],10),t,r)).getTime()):(new Date(parseInt(e,10),0,1)).getTime())},t.Hover=function(){function n(n){n==null&&(n={}),this.options=e.extend({},t.Hover.defaults,n),this.el=e("<div class='"+this.options["class"]+"'></div>"),this.el.hide(),this.options.parent.append(this.el)}return n.defaults={"class":"morris-hover morris-default-style"},n.prototype.update=function(e,t,n){return this.html(e),this.show(),this.moveTo(t,n)},n.prototype.html=function(e){return this.el.html(e)},n.prototype.moveTo=function(e,t){var n,r,i,s,o,u;return o=this.options.parent.innerWidth(),s=this.options.parent.innerHeight(),r=this.el.outerWidth(),n=this.el.outerHeight(),i=Math.min(Math.max(0,e-r/2),o-r),t!=null?(u=t-n-10,u<0&&(u=t+10,u+n>s&&(u=s/2-n/2))):u=s/2-n/2,this.el.css({left:i+"px",top:parseInt(u)+"px"})},n.prototype.show=function(){return this.el.show()},n.prototype.hide=function(){return this.el.hide()},n}(),t.Line=function(e){function n(e){this.hilight=u(this.hilight,this),this.onHoverOut=u(this.onHoverOut,this),this.onHoverMove=u(this.onHoverMove,this),this.onGridClick=u(this.onGridClick,this);if(!(this instanceof t.Line))return new t.Line(e);n.__super__.constructor.call(this,e)}return o(n,e),n.prototype.init=function(){this.pointGrow=Raphael.animation({r:this.options.pointSize+3},25,"linear"),this.pointShrink=Raphael.animation({r:this.options.pointSize},25,"linear");if(this.options.hideHover!=="always")return this.hover=new t.Hover({parent:this.el}),this.on("hovermove",this.onHoverMove),this.on("hoverout",this.onHoverOut),this.on("gridclick",this.onGridClick)},n.prototype.defaults={lineWidth:3,pointSize:4,lineColors:["#0b62a4","#7A92A3","#4da74d","#afd8f8","#edc240","#cb4b4b","#9440ed"],pointWidths:[1],pointStrokeColors:["#ffffff"],pointFillColors:[],smooth:!0,xLabels:"auto",xLabelFormat:null,xLabelMargin:24,continuousLine:!0,hideHover:!1},n.prototype.calc=function(){return this.calcPoints(),this.generatePaths()},n.prototype.calcPoints=function(){var e,t,n,r,i,s;i=this.data,s=[];for(n=0,r=i.length;n<r;n++)e=i[n],e._x=this.transX(e.x),e._y=function(){var n,r,i,s;i=e.y,s=[];for(n=0,r=i.length;n<r;n++)t=i[n],t!=null?s.push(this.transY(t)):s.push(t);return s}.call(this),s.push(e._ymax=Math.min.apply(null,[this.bottom].concat(function(){var n,r,i,s;i=e._y,s=[];for(n=0,r=i.length;n<r;n++)t=i[n],t!=null&&s.push(t);return s}())));return s},n.prototype.hitTest=function(e,t){var n,r,i,s,o;if(this.data.length===0)return null;o=this.data.slice(1);for(n=i=0,s=o.length;i<s;n=++i){r=o[n];if(e<(r._x+this.data[n]._x)/2)break}return n},n.prototype.onGridClick=function(e,t){var n;return n=this.hitTest(e,t),this.fire("click",n,this.options.data[n],e,t)},n.prototype.onHoverMove=function(e,t){var n;return n=this.hitTest(e,t),this.displayHoverForRow(n)},n.prototype.onHoverOut=function(){if(this.options.hideHover!==!1)return this.displayHoverForRow(null)},n.prototype.displayHoverForRow=function(e){var t;return e!=null?((t=this.hover).update.apply(t,this.hoverContentForRow(e)),this.hilight(e)):(this.hover.hide(),this.hilight())},n.prototype.hoverContentForRow=function(e){var t,n,r,i,s,o,u;r=this.data[e],t="<div class='morris-hover-row-label'>"+r.label+"</div>",u=r.y;for(n=s=0,o=u.length;s<o;n=++s)i=u[n],t+="<div class='morris-hover-point' style='color: "+this.colorFor(r,n,"label")+"'>\n  "+this.options.labels[n]+":\n  "+this.yLabelFormat(i)+"\n</div>";return typeof this.options.hoverCallback=="function"&&(t=this.options.hoverCallback(e,this.options,t)),[t,r._x,r._ymax]},n.prototype.generatePaths=function(){var e,n,r,i,s;return this.paths=function(){var o,u,f,l;l=[];for(r=o=0,u=this.options.ykeys.length;0<=u?o<u:o>u;r=0<=u?++o:--o)s=this.options.smooth===!0||(f=this.options.ykeys[r],a.call(this.options.smooth,f)>=0),n=function(){var e,t,n,s;n=this.data,s=[];for(e=0,t=n.length;e<t;e++)i=n[e],i._y[r]!==void 0&&s.push({x:i._x,y:i._y[r]});return s}.call(this),this.options.continuousLine&&(n=function(){var t,r,i;i=[];for(t=0,r=n.length;t<r;t++)e=n[t],e.y!==null&&i.push(e);return i}()),n.length>1?l.push(t.Line.createPath(n,s,this.bottom)):l.push(null);return l}.call(this)},n.prototype.draw=function(){this.options.axes&&this.drawXAxis(),this.drawSeries();if(this.options.hideHover===!1)return this.displayHoverForRow(this.data.length-1)},n.prototype.drawXAxis=function(){var e,n,r,i,s,o,u,a,f,l,c=this;u=this.bottom+this.options.padding/2,s=null,i=null,e=function(e,t){var n,r,o,a,f;return n=c.drawXAxisLabel(c.transX(t),u,e),f=n.getBBox(),n.transform("r"+ -c.options.xLabelAngle),r=n.getBBox(),n.transform("t0,"+r.height/2+"..."),c.options.xLabelAngle!==0&&(a=-0.5*f.width*Math.cos(c.options.xLabelAngle*Math.PI/180),n.transform("t"+a+",0...")),r=n.getBBox(),(s==null||s>=r.x+r.width||i!=null&&i>=r.x)&&r.x>=0&&r.x+r.width<c.el.width()?(c.options.xLabelAngle!==0&&(o=1.25*c.options.gridTextSize/Math.sin(c.options.xLabelAngle*Math.PI/180),i=r.x-o),s=r.x-c.options.xLabelMargin):n.remove()},this.options.parseTime?this.data.length===1&&this.options.xLabels==="auto"?r=[[this.data[0].label,this.data[0].x]]:r=t.labelSeries(this.xmin,this.xmax,this.width,this.options.xLabels,this.options.xLabelFormat):r=function(){var e,t,n,r;n=this.data,r=[];for(e=0,t=n.length;e<t;e++)o=n[e],r.push([o.label,o.x]);return r}.call(this),r.reverse(),l=[];for(a=0,f=r.length;a<f;a++)n=r[a],l.push(e(n[0],n[1]));return l},n.prototype.drawSeries=function(){var e,t,n,r,i,s;this.seriesPoints=[];for(e=t=r=this.options.ykeys.length-1;r<=0?t<=0:t>=0;e=r<=0?++t:--t)this._drawLineFor(e);s=[];for(e=n=i=this.options.ykeys.length-1;i<=0?n<=0:n>=0;e=i<=0?++n:--n)s.push(this._drawPointFor(e));return s},n.prototype._drawPointFor=function(e){var t,n,r,i,s,o;this.seriesPoints[e]=[],s=this.data,o=[];for(r=0,i=s.length;r<i;r++)n=s[r],t=null,n._y[e]!=null&&(t=this.drawLinePoint(n._x,n._y[e],this.options.pointSize,this.colorFor(n,e,"point"),e)),o.push(this.seriesPoints[e].push(t));return o},n.prototype._drawLineFor=function(e){var t;t=this.paths[e];if(t!==null)return this.drawLinePath(t,this.colorFor(null,e,"line"))},n.createPath=function(e,n,r){var i,s,o,u,a,f,l,c,h,p,d,v,m,g;l="",n&&(o=t.Line.gradients(e)),c={y:null};for(u=m=0,g=e.length;m<g;u=++m){i=e[u];if(i.y!=null)if(c.y!=null)n?(s=o[u],f=o[u-1],a=(i.x-c.x)/4,h=c.x+a,d=Math.min(r,c.y+a*f),p=i.x-a,v=Math.min(r,i.y-a*s),l+="C"+h+","+d+","+p+","+v+","+i.x+","+i.y):l+="L"+i.x+","+i.y;else if(!n||o[u]!=null)l+="M"+i.x+","+i.y;c=i}return l},n.gradients=function(e){var t,n,r,i,s,o,u,a;n=function(e,t){return(e.y-t.y)/(e.x-t.x)},a=[];for(r=o=0,u=e.length;o<u;r=++o)t=e[r],t.y!=null?(i=e[r+1]||{y:null},s=e[r-1]||{y:null},s.y!=null&&i.y!=null?a.push(n(s,i)):s.y!=null?a.push(n(s,t)):i.y!=null?a.push(n(t,i)):a.push(null)):a.push(null);return a},n.prototype.hilight=function(e){var t,n,r,i,s;if(this.prevHilight!==null&&this.prevHilight!==e)for(t=n=0,i=this.seriesPoints.length-1;0<=i?n<=i:n>=i;t=0<=i?++n:--n)this.seriesPoints[t][this.prevHilight]&&this.seriesPoints[t][this.prevHilight].animate(this.pointShrink);if(e!==null&&this.prevHilight!==e)for(t=r=0,s=this.seriesPoints.length-1;0<=s?r<=s:r>=s;t=0<=s?++r:--r)this.seriesPoints[t][e]&&this.seriesPoints[t][e].animate(this.pointGrow);return this.prevHilight=e},n.prototype.colorFor=function(e,t,n){return typeof this.options.lineColors=="function"?this.options.lineColors.call(this,e,t,n):n==="point"?this.options.pointFillColors[t%this.options.pointFillColors.length]||this.options.lineColors[t%this.options.lineColors.length]:this.options.lineColors[t%this.options.lineColors.length]},n.prototype.drawXAxisLabel=function(e,t,n){return this.raphael.text(e,t,n).attr("font-size",this.options.gridTextSize).attr("font-family",this.options.gridTextFamily).attr("font-weight",this.options.gridTextWeight).attr("fill",this.options.gridTextColor)},n.prototype.drawLinePath=function(e,t){return this.raphael.path(e).attr("stroke",t).attr("stroke-width",this.options.lineWidth)},n.prototype.drawLinePoint=function(e,t,n,r,i){return this.raphael.circle(e,t,n).attr("fill",r).attr("stroke-width",this.strokeWidthForSeries(i)).attr("stroke",this.strokeForSeries(i))},n.prototype.strokeWidthForSeries=function(e){return this.options.pointWidths[e%this.options.pointWidths.length]},n.prototype.strokeForSeries=function(e){return this.options.pointStrokeColors[e%this.options.pointStrokeColors.length]},n}(t.Grid),t.labelSeries=function(n,r,i,s,o){var u,a,f,l,c,h,p,d,v,m,g;f=200*(r-n)/i,a=new Date(n),p=t.LABEL_SPECS[s];if(p===void 0){g=t.AUTO_LABEL_ORDER;for(v=0,m=g.length;v<m;v++){l=g[v],h=t.LABEL_SPECS[l];if(f>=h.span){p=h;break}}}p===void 0&&(p=t.LABEL_SPECS.second),o&&(p=e.extend({},p,{fmt:o})),u=p.start(a),c=[];while((d=u.getTime())<=r)d>=n&&c.push([p.fmt(u),d]),p.incr(u);return c},n=function(e){return{span:e*60*1e3,start:function(e){return new Date(e.getFullYear(),e.getMonth(),e.getDate(),e.getHours())},fmt:function(e){return""+t.pad2(e.getHours())+":"+t.pad2(e.getMinutes())},incr:function(t){return t.setUTCMinutes(t.getUTCMinutes()+e)}}},r=function(e){return{span:e*1e3,start:function(e){return new Date(e.getFullYear(),e.getMonth(),e.getDate(),e.getHours(),e.getMinutes())},fmt:function(e){return""+t.pad2(e.getHours())+":"+t.pad2(e.getMinutes())+":"+t.pad2(e.getSeconds())},incr:function(t){return t.setUTCSeconds(t.getUTCSeconds()+e)}}},t.LABEL_SPECS={decade:{span:1728e8,start:function(e){return new Date(e.getFullYear()-e.getFullYear()%10,0,1)},fmt:function(e){return""+e.getFullYear()},incr:function(e){return e.setFullYear(e.getFullYear()+10)}},year:{span:1728e7,start:function(e){return new Date(e.getFullYear(),0,1)},fmt:function(e){return""+e.getFullYear()},incr:function(e){return e.setFullYear(e.getFullYear()+1)}},month:{span:24192e5,start:function(e){return new Date(e.getFullYear(),e.getMonth(),1)},fmt:function(e){return""+e.getFullYear()+"-"+t.pad2(e.getMonth()+1)},incr:function(e){return e.setMonth(e.getMonth()+1)}},day:{span:864e5,start:function(e){return new Date(e.getFullYear(),e.getMonth(),e.getDate())},fmt:function(e){return""+e.getFullYear()+"-"+t.pad2(e.getMonth()+1)+"-"+t.pad2(e.getDate())},incr:function(e){return e.setDate(e.getDate()+1)}},hour:n(60),"30min":n(30),"15min":n(15),"10min":n(10),"5min":n(5),minute:n(1),"30sec":r(30),"15sec":r(15),"10sec":r(10),"5sec":r(5),second:r(1)},t.AUTO_LABEL_ORDER=["decade","year","month","day","hour","30min","15min","10min","5min","minute","30sec","15sec","10sec","5sec","second"],t.Area=function(n){function i(n){var s;if(!(this instanceof t.Area))return new t.Area(n);s=e.extend({},r,n),this.cumulative=!s.behaveLikeLine,s.fillOpacity==="auto"&&(s.fillOpacity=s.behaveLikeLine?.8:1),i.__super__.constructor.call(this,s)}var r;return o(i,n),r={fillOpacity:"auto",behaveLikeLine:!1},i.prototype.calcPoints=function(){var e,t,n,r,i,s,o;s=this.data,o=[];for(r=0,i=s.length;r<i;r++)e=s[r],e._x=this.transX(e.x),t=0,e._y=function(){var r,i,s,o;s=e.y,o=[];for(r=0,i=s.length;r<i;r++)n=s[r],this.options.behaveLikeLine?o.push(this.transY(n)):(t+=n||0,o.push(this.transY(t)));return o}.call(this),o.push(e._ymax=Math.max.apply(Math,e._y));return o},i.prototype.drawSeries=function(){var e,t,n,r,i,s,o,u,a,f,l;this.seriesPoints=[],this.options.behaveLikeLine?t=function(){a=[];for(var e=0,t=this.options.ykeys.length-1;0<=t?e<=t:e>=t;0<=t?e++:e--)a.push(e);return a}.apply(this):t=function(){f=[];for(var e=u=this.options.ykeys.length-1;u<=0?e<=0:e>=0;u<=0?e++:e--)f.push(e);return f}.apply(this),l=[];for(i=0,s=t.length;i<s;i++)e=t[i],this._drawFillFor(e),this._drawLineFor(e),l.push(this._drawPointFor(e));return l},i.prototype._drawFillFor=function(e){var t;t=this.paths[e];if(t!==null)return t+="L"+this.transX(this.xmax)+","+this.bottom+"L"+this.transX(this.xmin)+","+this.bottom+"Z",this.drawFilledPath(t,this.fillForSeries(e))},i.prototype.fillForSeries=function(e){var t;return t=Raphael.rgb2hsl(this.colorFor(this.data[e],e,"line")),Raphael.hsl(t.h,this.options.behaveLikeLine?t.s*.9:t.s*.75,Math.min(.98,this.options.behaveLikeLine?t.l*1.2:t.l*1.25))},i.prototype.drawFilledPath=function(e,t){return this.raphael.path(e).attr("fill",t).attr("fill-opacity",this.options.fillOpacity).attr("stroke-width",0)},i}(t.Line),t.Bar=function(n){function r(n){this.onHoverOut=u(this.onHoverOut,this),this.onHoverMove=u(this.onHoverMove,this),this.onGridClick=u(this.onGridClick,this);if(!(this instanceof t.Bar))return new t.Bar(n);r.__super__.constructor.call(this,e.extend({},n,{parseTime:!1}))}return o(r,n),r.prototype.init=function(){this.cumulative=this.options.stacked;if(this.options.hideHover!=="always")return this.hover=new t.Hover({parent:this.el}),this.on("hovermove",this.onHoverMove),this.on("hoverout",this.onHoverOut),this.on("gridclick",this.onGridClick)},r.prototype.defaults={barSizeRatio:.75,barGap:3,barColors:["#0b62a4","#7a92a3","#4da74d","#afd8f8","#edc240","#cb4b4b","#9440ed"],xLabelMargin:50},r.prototype.calc=function(){var e;this.calcBars();if(this.options.hideHover===!1)return(e=this.hover).update.apply(e,this.hoverContentForRow(this.data.length-1))},r.prototype.calcBars=function(){var e,t,n,r,i,s,o;s=this.data,o=[];for(e=r=0,i=s.length;r<i;e=++r)t=s[e],t._x=this.left+this.width*(e+.5)/this.data.length,o.push(t._y=function(){var e,r,i,s;i=t.y,s=[];for(e=0,r=i.length;e<r;e++)n=i[e],n!=null?s.push(this.transY(n)):s.push(null);return s}.call(this));return o},r.prototype.draw=function(){return this.options.axes&&this.drawXAxis(),this.drawSeries()},r.prototype.drawXAxis=function(){var e,t,n,r,i,s,o,u,a,f,l,c,h;f=this.bottom+this.options.padding/2,o=null,s=null,h=[];for(e=l=0,c=this.data.length;0<=c?l<c:l>c;e=0<=c?++l:--l)u=this.data[this.data.length-1-e],t=this.drawXAxisLabel(u._x,f,u.label),a=t.getBBox(),t.transform("r"+ -this.options.xLabelAngle),n=t.getBBox(),t.transform("t0,"+n.height/2+"..."),this.options.xLabelAngle!==0&&(i=-0.5*a.width*Math.cos(this.options.xLabelAngle*Math.PI/180),t.transform("t"+i+",0...")),(o==null||o>=n.x+n.width||s!=null&&s>=n.x)&&n.x>=0&&n.x+n.width<this.el.width()?(this.options.xLabelAngle!==0&&(r=1.25*this.options.gridTextSize/Math.sin(this.options.xLabelAngle*Math.PI/180),s=n.x-r),h.push(o=n.x-this.options.xLabelMargin)):h.push(t.remove());return h},r.prototype.drawSeries=function(){var e,t,n,r,i,s,o,u,a,f,l,c,h,p;return n=this.width/this.options.data.length,u=this.options.stacked!=null?1:this.options.ykeys.length,e=(n*this.options.barSizeRatio-this.options.barGap*(u-1))/u,o=n*(1-this.options.barSizeRatio)/2,p=this.ymin<=0&&this.ymax>=0?this.transY(0):null,this.bars=function(){var u,d,v,m;v=this.data,m=[];for(r=u=0,d=v.length;u<d;r=++u)a=v[r],i=0,m.push(function(){var u,d,v,m;v=a._y,m=[];for(f=u=0,d=v.length;u<d;f=++u)h=v[f],h!==null?(p?(c=Math.min(h,p),t=Math.max(h,p)):(c=h,t=this.bottom),s=this.left+r*n+o,this.options.stacked||(s+=f*(e+this.options.barGap)),l=t-c,this.options.stacked&&(c-=i),this.drawBar(s,c,e,l,this.colorFor(a,f,"bar")),m.push(i+=l)):m.push(null);return m}.call(this));return m}.call(this)},r.prototype.colorFor=function(e,t,n){var r,i;return typeof this.options.barColors=="function"?(r={x:e.x,y:e.y[t],label:e.label},i={index:t,key:this.options.ykeys[t],label:this.options.labels[t]},this.options.barColors.call(this,r,i,n)):this.options.barColors[t%this.options.barColors.length]},r.prototype.hitTest=function(e,t){return this.data.length===0?null:(e=Math.max(Math.min(e,this.right),this.left),Math.min(this.data.length-1,Math.floor((e-this.left)/(this.width/this.data.length))))},r.prototype.onGridClick=function(e,t){var n;return n=this.hitTest(e,t),this.fire("click",n,this.options.data[n],e,t)},r.prototype.onHoverMove=function(e,t){var n,r;return n=this.hitTest(e,t),(r=this.hover).update.apply(r,this.hoverContentForRow(n))},r.prototype.onHoverOut=function(){if(this.options.hideHover!==!1)return this.hover.hide()},r.prototype.hoverContentForRow=function(e){var t,n,r,i,s,o,u,a;r=this.data[e],t="<div class='morris-hover-row-label'>"+r.label+"</div>",a=r.y;for(n=o=0,u=a.length;o<u;n=++o)s=a[n],t+="<div class='morris-hover-point' style='color: "+this.colorFor(r,n,"label")+"'>\n  "+this.options.labels[n]+":\n  "+this.yLabelFormat(s)+"\n</div>";return typeof this.options.hoverCallback=="function"&&(t=this.options.hoverCallback(e,this.options,t)),i=this.left+(e+.5)*this.width/this.data.length,[t,i]},r.prototype.drawXAxisLabel=function(e,t,n){var r;return r=this.raphael.text(e,t,n).attr("font-size",this.options.gridTextSize).attr("font-family",this.options.gridTextFamily).attr("font-weight",this.options.gridTextWeight).attr("fill",this.options.gridTextColor)},r.prototype.drawBar=function(e,t,n,r,i){return this.raphael.rect(e,t,n,r).attr("fill",i).attr("stroke-width",0)},r}(t.Grid),t.Donut=function(n){function r(n){this.select=u(this.select,this),this.click=u(this.click,this);var r;if(!(this instanceof t.Donut))return new t.Donut(n);typeof n.element=="string"?this.el=e(document.getElementById(n.element)):this.el=e(n.element),this.options=e.extend({},this.defaults,n);if(this.el===null||this.el.length===0)throw new Error("Graph placeholder not found.");if(n.data===void 0||n.data.length===0)return;this.data=n.data,this.values=function(){var e,t,n,i;n=this.data,i=[];for(e=0,t=n.length;e<t;e++)r=n[e],i.push(parseFloat(r.value));return i}.call(this),this.redraw()}return o(r,n),r.prototype.defaults={colors:["#0B62A4","#3980B5","#679DC6","#95BBD7","#B0CCE1","#095791","#095085","#083E67","#052C48","#042135"],backgroundColor:"#FFFFFF",labelColor:"#000000",formatter:t.commas},r.prototype.redraw=function(){var e,n,r,i,s,o,u,a,f,l,c,h,p,d,v,m,g,y,b,w,E,S,x;this.el.empty(),this.raphael=new Raphael(this.el[0]),n=this.el.width()/2,r=this.el.height()/2,p=(Math.min(n,r)-10)/3,c=0,w=this.values;for(d=0,g=w.length;d<g;d++)h=w[d],c+=h;a=5/(2*p),e=1.9999*Math.PI-a*this.data.length,o=0,s=0,this.segments=[],E=this.values;for(i=v=0,y=E.length;v<y;i=++v)h=E[i],f=o+a+e*(h/c),l=new t.DonutSegment(n,r,p*2,p,o,f,this.options.colors[s%this.options.colors.length],this.options.backgroundColor,s,this.raphael),l.render(),this.segments.push(l),l.on("hover",this.select),l.on("click",this.click),o=f,s+=1;this.text1=this.drawEmptyDonutLabel(n,r-10,this.options.labelColor,15,800),this.text2=this.drawEmptyDonutLabel(n,r+10,this.options.labelColor,14),u=Math.max.apply(null,function(){var e,t,n,r;n=this.values,r=[];for(e=0,t=n.length;e<t;e++)h=n[e],r.push(h);return r}.call(this)),s=0,S=this.values,x=[];for(m=0,b=S.length;m<b;m++){h=S[m];if(h===u){this.select(s);break}x.push(s+=1)}return x},r.prototype.click=function(e){return this.fire("click",e,this.data[e])},r.prototype.select=function(e){var t,n,r,i,s,o;o=this.segments;for(i=0,s=o.length;i<s;i++)n=o[i],n.deselect();return r=this.segments[e],r.select(),t=this.data[e],this.setLabels(t.label,this.options.formatter(t.value,t))},r.prototype.setLabels=function(e,t){var n,r,i,s,o,u,a,f;return n=(Math.min(this.el.width()/2,this.el.height()/2)-10)*2/3,s=1.8*n,i=n/2,r=n/3,this.text1.attr({text:e,transform:""}),o=this.text1.getBBox(),u=Math.min(s/o.width,i/o.height),this.text1.attr({transform:"S"+u+","+u+","+(o.x+o.width/2)+","+(o.y+o.height)}),this.text2.attr({text:t,transform:""}),a=this.text2.getBBox(),f=Math.min(s/a.width,r/a.height),this.text2.attr({transform:"S"+f+","+f+","+(a.x+a.width/2)+","+a.y})},r.prototype.drawEmptyDonutLabel=function(e,t,n,r,i){var s;return s=this.raphael.text(e,t,"").attr("font-size",r).attr("fill",n),i!=null&&s.attr("font-weight",i),s},r}(t.EventEmitter),t.DonutSegment=function(e){function t(e,t,n,r,i,s,o,a,f,l){this.cx=e,this.cy=t,this.inner=n,this.outer=r,this.color=o,this.backgroundColor=a,this.index=f,this.raphael=l,this.deselect=u(this.deselect,this),this.select=u(this.select,this),this.sin_p0=Math.sin(i),this.cos_p0=Math.cos(i),this.sin_p1=Math.sin(s),this.cos_p1=Math.cos(s),this.is_long=s-i>Math.PI?1:0,this.path=this.calcSegment(this.inner+3,this.inner+this.outer-5),this.selectedPath=this.calcSegment(this.inner+3,this.inner+this.outer),this.hilight=this.calcArc(this.inner)}return o(t,e),t.prototype.calcArcPoints=function(e){return[this.cx+e*this.sin_p0,this.cy+e*this.cos_p0,this.cx+e*this.sin_p1,this.cy+e*this.cos_p1]},t.prototype.calcSegment=function(e,t){var n,r,i,s,o,u,a,f,l,c;return l=this.calcArcPoints(e),n=l[0],i=l[1],r=l[2],s=l[3],c=this.calcArcPoints(t),o=c[0],a=c[1],u=c[2],f=c[3],"M"+n+","+i+("A"+e+","+e+",0,"+this.is_long+",0,"+r+","+s)+("L"+u+","+f)+("A"+t+","+t+",0,"+this.is_long+",1,"+o+","+a)+"Z"},t.prototype.calcArc=function(e){var t,n,r,i,s;return s=this.calcArcPoints(e),t=s[0],r=s[1],n=s[2],i=s[3],"M"+t+","+r+("A"+e+","+e+",0,"+this.is_long+",0,"+n+","+i)},t.prototype.render=function(){var e=this;return this.arc=this.drawDonutArc(this.hilight,this.color),this.seg=this.drawDonutSegment(this.path,this.color,this.backgroundColor,function(){return e.fire("hover",e.index)},function(){return e.fire("click",e.index)})},t.prototype.drawDonutArc=function(e,t){return this.raphael.path(e).attr({stroke:t,"stroke-width":2,opacity:0})},t.prototype.drawDonutSegment=function(e,t,n,r,i){return this.raphael.path(e).attr({fill:t,stroke:n,"stroke-width":3}).hover(r).click(i)},t.prototype.select=function(){if(!this.selected)return this.seg.animate({path:this.selectedPath},150,"<>"),this.arc.animate({opacity:1},150,"<>"),this.selected=!0},t.prototype.deselect=function(){if(this.selected)return this.seg.animate({path:this.path},150,"<>"),this.arc.animate({opacity:0},150,"<>"),this.selected=!1},t}(t.EventEmitter)}).call(this);
\ No newline at end of file
diff --git a/src/web/js/raphael.min.js b/src/web/js/raphael.min.js
new file mode 100644
index 0000000..404f8b2
--- /dev/null
+++ b/src/web/js/raphael.min.js
@@ -0,0 +1,11 @@
+// ┌────────────────────────────────────────────────────────────────────┐ \\
+// │ Raphaël 2.1.2 - JavaScript Vector Library                          │ \\
+// ├────────────────────────────────────────────────────────────────────┤ \\
+// │ Copyright © 2008-2012 Dmitry Baranovskiy (http://raphaeljs.com)    │ \\
+// │ Copyright © 2008-2012 Sencha Labs (http://sencha.com)              │ \\
+// ├────────────────────────────────────────────────────────────────────┤ \\
+// │ Licensed under the MIT (http://raphaeljs.com/license.html) license.│ \\
+// └────────────────────────────────────────────────────────────────────┘ \\
+!function(a){var b,c,d="0.4.2",e="hasOwnProperty",f=/[\.\/]/,g="*",h=function(){},i=function(a,b){return a-b},j={n:{}},k=function(a,d){a=String(a);var e,f=c,g=Array.prototype.slice.call(arguments,2),h=k.listeners(a),j=0,l=[],m={},n=[],o=b;b=a,c=0;for(var p=0,q=h.length;q>p;p++)"zIndex"in h[p]&&(l.push(h[p].zIndex),h[p].zIndex<0&&(m[h[p].zIndex]=h[p]));for(l.sort(i);l[j]<0;)if(e=m[l[j++]],n.push(e.apply(d,g)),c)return c=f,n;for(p=0;q>p;p++)if(e=h[p],"zIndex"in e)if(e.zIndex==l[j]){if(n.push(e.apply(d,g)),c)break;do if(j++,e=m[l[j]],e&&n.push(e.apply(d,g)),c)break;while(e)}else m[e.zIndex]=e;else if(n.push(e.apply(d,g)),c)break;return c=f,b=o,n.length?n:null};k._events=j,k.listeners=function(a){var b,c,d,e,h,i,k,l,m=a.split(f),n=j,o=[n],p=[];for(e=0,h=m.length;h>e;e++){for(l=[],i=0,k=o.length;k>i;i++)for(n=o[i].n,c=[n[m[e]],n[g]],d=2;d--;)b=c[d],b&&(l.push(b),p=p.concat(b.f||[]));o=l}return p},k.on=function(a,b){if(a=String(a),"function"!=typeof b)return function(){};for(var c=a.split(f),d=j,e=0,g=c.length;g>e;e++)d=d.n,d=d.hasOwnProperty(c[e])&&d[c[e]]||(d[c[e]]={n:{}});for(d.f=d.f||[],e=0,g=d.f.length;g>e;e++)if(d.f[e]==b)return h;return d.f.push(b),function(a){+a==+a&&(b.zIndex=+a)}},k.f=function(a){var b=[].slice.call(arguments,1);return function(){k.apply(null,[a,null].concat(b).concat([].slice.call(arguments,0)))}},k.stop=function(){c=1},k.nt=function(a){return a?new RegExp("(?:\\.|\\/|^)"+a+"(?:\\.|\\/|$)").test(b):b},k.nts=function(){return b.split(f)},k.off=k.unbind=function(a,b){if(!a)return k._events=j={n:{}},void 0;var c,d,h,i,l,m,n,o=a.split(f),p=[j];for(i=0,l=o.length;l>i;i++)for(m=0;m<p.length;m+=h.length-2){if(h=[m,1],c=p[m].n,o[i]!=g)c[o[i]]&&h.push(c[o[i]]);else for(d in c)c[e](d)&&h.push(c[d]);p.splice.apply(p,h)}for(i=0,l=p.length;l>i;i++)for(c=p[i];c.n;){if(b){if(c.f){for(m=0,n=c.f.length;n>m;m++)if(c.f[m]==b){c.f.splice(m,1);break}!c.f.length&&delete c.f}for(d in c.n)if(c.n[e](d)&&c.n[d].f){var q=c.n[d].f;for(m=0,n=q.length;n>m;m++)if(q[m]==b){q.splice(m,1);break}!q.length&&delete c.n[d].f}}else{delete c.f;for(d in c.n)c.n[e](d)&&c.n[d].f&&delete c.n[d].f}c=c.n}},k.once=function(a,b){var c=function(){return k.unbind(a,c),b.apply(this,arguments)};return k.on(a,c)},k.version=d,k.toString=function(){return"You are running Eve "+d},"undefined"!=typeof module&&module.exports?module.exports=k:"undefined"!=typeof define?define("eve",[],function(){return k}):a.eve=k}(this),function(a,b){"function"==typeof define&&define.amd?define(["eve"],function(c){return b(a,c)}):b(a,a.eve)}(this,function(a,b){function c(a){if(c.is(a,"function"))return u?a():b.on("raphael.DOMload",a);if(c.is(a,V))return c._engine.create[D](c,a.splice(0,3+c.is(a[0],T))).add(a);var d=Array.prototype.slice.call(arguments,0);if(c.is(d[d.length-1],"function")){var e=d.pop();return u?e.call(c._engine.create[D](c,d)):b.on("raphael.DOMload",function(){e.call(c._engine.create[D](c,d))})}return c._engine.create[D](c,arguments)}function d(a){if("function"==typeof a||Object(a)!==a)return a;var b=new a.constructor;for(var c in a)a[z](c)&&(b[c]=d(a[c]));return b}function e(a,b){for(var c=0,d=a.length;d>c;c++)if(a[c]===b)return a.push(a.splice(c,1)[0])}function f(a,b,c){function d(){var f=Array.prototype.slice.call(arguments,0),g=f.join("␀"),h=d.cache=d.cache||{},i=d.count=d.count||[];return h[z](g)?(e(i,g),c?c(h[g]):h[g]):(i.length>=1e3&&delete h[i.shift()],i.push(g),h[g]=a[D](b,f),c?c(h[g]):h[g])}return d}function g(){return this.hex}function h(a,b){for(var c=[],d=0,e=a.length;e-2*!b>d;d+=2){var f=[{x:+a[d-2],y:+a[d-1]},{x:+a[d],y:+a[d+1]},{x:+a[d+2],y:+a[d+3]},{x:+a[d+4],y:+a[d+5]}];b?d?e-4==d?f[3]={x:+a[0],y:+a[1]}:e-2==d&&(f[2]={x:+a[0],y:+a[1]},f[3]={x:+a[2],y:+a[3]}):f[0]={x:+a[e-2],y:+a[e-1]}:e-4==d?f[3]=f[2]:d||(f[0]={x:+a[d],y:+a[d+1]}),c.push(["C",(-f[0].x+6*f[1].x+f[2].x)/6,(-f[0].y+6*f[1].y+f[2].y)/6,(f[1].x+6*f[2].x-f[3].x)/6,(f[1].y+6*f[2].y-f[3].y)/6,f[2].x,f[2].y])}return c}function i(a,b,c,d,e){var f=-3*b+9*c-9*d+3*e,g=a*f+6*b-12*c+6*d;return a*g-3*b+3*c}function j(a,b,c,d,e,f,g,h,j){null==j&&(j=1),j=j>1?1:0>j?0:j;for(var k=j/2,l=12,m=[-.1252,.1252,-.3678,.3678,-.5873,.5873,-.7699,.7699,-.9041,.9041,-.9816,.9816],n=[.2491,.2491,.2335,.2335,.2032,.2032,.1601,.1601,.1069,.1069,.0472,.0472],o=0,p=0;l>p;p++){var q=k*m[p]+k,r=i(q,a,c,e,g),s=i(q,b,d,f,h),t=r*r+s*s;o+=n[p]*N.sqrt(t)}return k*o}function k(a,b,c,d,e,f,g,h,i){if(!(0>i||j(a,b,c,d,e,f,g,h)<i)){var k,l=1,m=l/2,n=l-m,o=.01;for(k=j(a,b,c,d,e,f,g,h,n);Q(k-i)>o;)m/=2,n+=(i>k?1:-1)*m,k=j(a,b,c,d,e,f,g,h,n);return n}}function l(a,b,c,d,e,f,g,h){if(!(O(a,c)<P(e,g)||P(a,c)>O(e,g)||O(b,d)<P(f,h)||P(b,d)>O(f,h))){var i=(a*d-b*c)*(e-g)-(a-c)*(e*h-f*g),j=(a*d-b*c)*(f-h)-(b-d)*(e*h-f*g),k=(a-c)*(f-h)-(b-d)*(e-g);if(k){var l=i/k,m=j/k,n=+l.toFixed(2),o=+m.toFixed(2);if(!(n<+P(a,c).toFixed(2)||n>+O(a,c).toFixed(2)||n<+P(e,g).toFixed(2)||n>+O(e,g).toFixed(2)||o<+P(b,d).toFixed(2)||o>+O(b,d).toFixed(2)||o<+P(f,h).toFixed(2)||o>+O(f,h).toFixed(2)))return{x:l,y:m}}}}function m(a,b,d){var e=c.bezierBBox(a),f=c.bezierBBox(b);if(!c.isBBoxIntersect(e,f))return d?0:[];for(var g=j.apply(0,a),h=j.apply(0,b),i=O(~~(g/5),1),k=O(~~(h/5),1),m=[],n=[],o={},p=d?0:[],q=0;i+1>q;q++){var r=c.findDotsAtSegment.apply(c,a.concat(q/i));m.push({x:r.x,y:r.y,t:q/i})}for(q=0;k+1>q;q++)r=c.findDotsAtSegment.apply(c,b.concat(q/k)),n.push({x:r.x,y:r.y,t:q/k});for(q=0;i>q;q++)for(var s=0;k>s;s++){var t=m[q],u=m[q+1],v=n[s],w=n[s+1],x=Q(u.x-t.x)<.001?"y":"x",y=Q(w.x-v.x)<.001?"y":"x",z=l(t.x,t.y,u.x,u.y,v.x,v.y,w.x,w.y);if(z){if(o[z.x.toFixed(4)]==z.y.toFixed(4))continue;o[z.x.toFixed(4)]=z.y.toFixed(4);var A=t.t+Q((z[x]-t[x])/(u[x]-t[x]))*(u.t-t.t),B=v.t+Q((z[y]-v[y])/(w[y]-v[y]))*(w.t-v.t);A>=0&&1.001>=A&&B>=0&&1.001>=B&&(d?p++:p.push({x:z.x,y:z.y,t1:P(A,1),t2:P(B,1)}))}}return p}function n(a,b,d){a=c._path2curve(a),b=c._path2curve(b);for(var e,f,g,h,i,j,k,l,n,o,p=d?0:[],q=0,r=a.length;r>q;q++){var s=a[q];if("M"==s[0])e=i=s[1],f=j=s[2];else{"C"==s[0]?(n=[e,f].concat(s.slice(1)),e=n[6],f=n[7]):(n=[e,f,e,f,i,j,i,j],e=i,f=j);for(var t=0,u=b.length;u>t;t++){var v=b[t];if("M"==v[0])g=k=v[1],h=l=v[2];else{"C"==v[0]?(o=[g,h].concat(v.slice(1)),g=o[6],h=o[7]):(o=[g,h,g,h,k,l,k,l],g=k,h=l);var w=m(n,o,d);if(d)p+=w;else{for(var x=0,y=w.length;y>x;x++)w[x].segment1=q,w[x].segment2=t,w[x].bez1=n,w[x].bez2=o;p=p.concat(w)}}}}}return p}function o(a,b,c,d,e,f){null!=a?(this.a=+a,this.b=+b,this.c=+c,this.d=+d,this.e=+e,this.f=+f):(this.a=1,this.b=0,this.c=0,this.d=1,this.e=0,this.f=0)}function p(){return this.x+H+this.y+H+this.width+" × "+this.height}function q(a,b,c,d,e,f){function g(a){return((l*a+k)*a+j)*a}function h(a,b){var c=i(a,b);return((o*c+n)*c+m)*c}function i(a,b){var c,d,e,f,h,i;for(e=a,i=0;8>i;i++){if(f=g(e)-a,Q(f)<b)return e;if(h=(3*l*e+2*k)*e+j,Q(h)<1e-6)break;e-=f/h}if(c=0,d=1,e=a,c>e)return c;if(e>d)return d;for(;d>c;){if(f=g(e),Q(f-a)<b)return e;a>f?c=e:d=e,e=(d-c)/2+c}return e}var j=3*b,k=3*(d-b)-j,l=1-j-k,m=3*c,n=3*(e-c)-m,o=1-m-n;return h(a,1/(200*f))}function r(a,b){var c=[],d={};if(this.ms=b,this.times=1,a){for(var e in a)a[z](e)&&(d[_(e)]=a[e],c.push(_(e)));c.sort(lb)}this.anim=d,this.top=c[c.length-1],this.percents=c}function s(a,d,e,f,g,h){e=_(e);var i,j,k,l,m,n,p=a.ms,r={},s={},t={};if(f)for(v=0,x=ic.length;x>v;v++){var u=ic[v];if(u.el.id==d.id&&u.anim==a){u.percent!=e?(ic.splice(v,1),k=1):j=u,d.attr(u.totalOrigin);break}}else f=+s;for(var v=0,x=a.percents.length;x>v;v++){if(a.percents[v]==e||a.percents[v]>f*a.top){e=a.percents[v],m=a.percents[v-1]||0,p=p/a.top*(e-m),l=a.percents[v+1],i=a.anim[e];break}f&&d.attr(a.anim[a.percents[v]])}if(i){if(j)j.initstatus=f,j.start=new Date-j.ms*f;else{for(var y in i)if(i[z](y)&&(db[z](y)||d.paper.customAttributes[z](y)))switch(r[y]=d.attr(y),null==r[y]&&(r[y]=cb[y]),s[y]=i[y],db[y]){case T:t[y]=(s[y]-r[y])/p;break;case"colour":r[y]=c.getRGB(r[y]);var A=c.getRGB(s[y]);t[y]={r:(A.r-r[y].r)/p,g:(A.g-r[y].g)/p,b:(A.b-r[y].b)/p};break;case"path":var B=Kb(r[y],s[y]),C=B[1];for(r[y]=B[0],t[y]=[],v=0,x=r[y].length;x>v;v++){t[y][v]=[0];for(var D=1,F=r[y][v].length;F>D;D++)t[y][v][D]=(C[v][D]-r[y][v][D])/p}break;case"transform":var G=d._,H=Pb(G[y],s[y]);if(H)for(r[y]=H.from,s[y]=H.to,t[y]=[],t[y].real=!0,v=0,x=r[y].length;x>v;v++)for(t[y][v]=[r[y][v][0]],D=1,F=r[y][v].length;F>D;D++)t[y][v][D]=(s[y][v][D]-r[y][v][D])/p;else{var K=d.matrix||new o,L={_:{transform:G.transform},getBBox:function(){return d.getBBox(1)}};r[y]=[K.a,K.b,K.c,K.d,K.e,K.f],Nb(L,s[y]),s[y]=L._.transform,t[y]=[(L.matrix.a-K.a)/p,(L.matrix.b-K.b)/p,(L.matrix.c-K.c)/p,(L.matrix.d-K.d)/p,(L.matrix.e-K.e)/p,(L.matrix.f-K.f)/p]}break;case"csv":var M=I(i[y])[J](w),N=I(r[y])[J](w);if("clip-rect"==y)for(r[y]=N,t[y]=[],v=N.length;v--;)t[y][v]=(M[v]-r[y][v])/p;s[y]=M;break;default:for(M=[][E](i[y]),N=[][E](r[y]),t[y]=[],v=d.paper.customAttributes[y].length;v--;)t[y][v]=((M[v]||0)-(N[v]||0))/p}var O=i.easing,P=c.easing_formulas[O];if(!P)if(P=I(O).match(Z),P&&5==P.length){var Q=P;P=function(a){return q(a,+Q[1],+Q[2],+Q[3],+Q[4],p)}}else P=nb;if(n=i.start||a.start||+new Date,u={anim:a,percent:e,timestamp:n,start:n+(a.del||0),status:0,initstatus:f||0,stop:!1,ms:p,easing:P,from:r,diff:t,to:s,el:d,callback:i.callback,prev:m,next:l,repeat:h||a.times,origin:d.attr(),totalOrigin:g},ic.push(u),f&&!j&&!k&&(u.stop=!0,u.start=new Date-p*f,1==ic.length))return kc();k&&(u.start=new Date-u.ms*f),1==ic.length&&jc(kc)}b("raphael.anim.start."+d.id,d,a)}}function t(a){for(var b=0;b<ic.length;b++)ic[b].el.paper==a&&ic.splice(b--,1)}c.version="2.1.2",c.eve=b;var u,v,w=/[, ]+/,x={circle:1,rect:1,path:1,ellipse:1,text:1,image:1},y=/\{(\d+)\}/g,z="hasOwnProperty",A={doc:document,win:a},B={was:Object.prototype[z].call(A.win,"Raphael"),is:A.win.Raphael},C=function(){this.ca=this.customAttributes={}},D="apply",E="concat",F="ontouchstart"in A.win||A.win.DocumentTouch&&A.doc instanceof DocumentTouch,G="",H=" ",I=String,J="split",K="click dblclick mousedown mousemove mouseout mouseover mouseup touchstart touchmove touchend touchcancel"[J](H),L={mousedown:"touchstart",mousemove:"touchmove",mouseup:"touchend"},M=I.prototype.toLowerCase,N=Math,O=N.max,P=N.min,Q=N.abs,R=N.pow,S=N.PI,T="number",U="string",V="array",W=Object.prototype.toString,X=(c._ISURL=/^url\(['"]?([^\)]+?)['"]?\)$/i,/^\s*((#[a-f\d]{6})|(#[a-f\d]{3})|rgba?\(\s*([\d\.]+%?\s*,\s*[\d\.]+%?\s*,\s*[\d\.]+%?(?:\s*,\s*[\d\.]+%?)?)\s*\)|hsba?\(\s*([\d\.]+(?:deg|\xb0|%)?\s*,\s*[\d\.]+%?\s*,\s*[\d\.]+(?:%?\s*,\s*[\d\.]+)?)%?\s*\)|hsla?\(\s*([\d\.]+(?:deg|\xb0|%)?\s*,\s*[\d\.]+%?\s*,\s*[\d\.]+(?:%?\s*,\s*[\d\.]+)?)%?\s*\))\s*$/i),Y={NaN:1,Infinity:1,"-Infinity":1},Z=/^(?:cubic-)?bezier\(([^,]+),([^,]+),([^,]+),([^\)]+)\)/,$=N.round,_=parseFloat,ab=parseInt,bb=I.prototype.toUpperCase,cb=c._availableAttrs={"arrow-end":"none","arrow-start":"none",blur:0,"clip-rect":"0 0 1e9 1e9",cursor:"default",cx:0,cy:0,fill:"#fff","fill-opacity":1,font:'10px "Arial"',"font-family":'"Arial"',"font-size":"10","font-style":"normal","font-weight":400,gradient:0,height:0,href:"http://raphaeljs.com/","letter-spacing":0,opacity:1,path:"M0,0",r:0,rx:0,ry:0,src:"",stroke:"#000","stroke-dasharray":"","stroke-linecap":"butt","stroke-linejoin":"butt","stroke-miterlimit":0,"stroke-opacity":1,"stroke-width":1,target:"_blank","text-anchor":"middle",title:"Raphael",transform:"",width:0,x:0,y:0},db=c._availableAnimAttrs={blur:T,"clip-rect":"csv",cx:T,cy:T,fill:"colour","fill-opacity":T,"font-size":T,height:T,opacity:T,path:"path",r:T,rx:T,ry:T,stroke:"colour","stroke-opacity":T,"stroke-width":T,transform:"transform",width:T,x:T,y:T},eb=/[\x09\x0a\x0b\x0c\x0d\x20\xa0\u1680\u180e\u2000\u2001\u2002\u2003\u2004\u2005\u2006\u2007\u2008\u2009\u200a\u202f\u205f\u3000\u2028\u2029]*,[\x09\x0a\x0b\x0c\x0d\x20\xa0\u1680\u180e\u2000\u2001\u2002\u2003\u2004\u2005\u2006\u2007\u2008\u2009\u200a\u202f\u205f\u3000\u2028\u2029]*/,fb={hs:1,rg:1},gb=/,?([achlmqrstvxz]),?/gi,hb=/([achlmrqstvz])[\x09\x0a\x0b\x0c\x0d\x20\xa0\u1680\u180e\u2000\u2001\u2002\u2003\u2004\u2005\u2006\u2007\u2008\u2009\u200a\u202f\u205f\u3000\u2028\u2029,]*((-?\d*\.?\d*(?:e[\-+]?\d+)?[\x09\x0a\x0b\x0c\x0d\x20\xa0\u1680\u180e\u2000\u2001\u2002\u2003\u2004\u2005\u2006\u2007\u2008\u2009\u200a\u202f\u205f\u3000\u2028\u2029]*,?[\x09\x0a\x0b\x0c\x0d\x20\xa0\u1680\u180e\u2000\u2001\u2002\u2003\u2004\u2005\u2006\u2007\u2008\u2009\u200a\u202f\u205f\u3000\u2028\u2029]*)+)/gi,ib=/([rstm])[\x09\x0a\x0b\x0c\x0d\x20\xa0\u1680\u180e\u2000\u2001\u2002\u2003\u2004\u2005\u2006\u2007\u2008\u2009\u200a\u202f\u205f\u3000\u2028\u2029,]*((-?\d*\.?\d*(?:e[\-+]?\d+)?[\x09\x0a\x0b\x0c\x0d\x20\xa0\u1680\u180e\u2000\u2001\u2002\u2003\u2004\u2005\u2006\u2007\u2008\u2009\u200a\u202f\u205f\u3000\u2028\u2029]*,?[\x09\x0a\x0b\x0c\x0d\x20\xa0\u1680\u180e\u2000\u2001\u2002\u2003\u2004\u2005\u2006\u2007\u2008\u2009\u200a\u202f\u205f\u3000\u2028\u2029]*)+)/gi,jb=/(-?\d*\.?\d*(?:e[\-+]?\d+)?)[\x09\x0a\x0b\x0c\x0d\x20\xa0\u1680\u180e\u2000\u2001\u2002\u2003\u2004\u2005\u2006\u2007\u2008\u2009\u200a\u202f\u205f\u3000\u2028\u2029]*,?[\x09\x0a\x0b\x0c\x0d\x20\xa0\u1680\u180e\u2000\u2001\u2002\u2003\u2004\u2005\u2006\u2007\u2008\u2009\u200a\u202f\u205f\u3000\u2028\u2029]*/gi,kb=(c._radial_gradient=/^r(?:\(([^,]+?)[\x09\x0a\x0b\x0c\x0d\x20\xa0\u1680\u180e\u2000\u2001\u2002\u2003\u2004\u2005\u2006\u2007\u2008\u2009\u200a\u202f\u205f\u3000\u2028\u2029]*,[\x09\x0a\x0b\x0c\x0d\x20\xa0\u1680\u180e\u2000\u2001\u2002\u2003\u2004\u2005\u2006\u2007\u2008\u2009\u200a\u202f\u205f\u3000\u2028\u2029]*([^\)]+?)\))?/,{}),lb=function(a,b){return _(a)-_(b)},mb=function(){},nb=function(a){return a},ob=c._rectPath=function(a,b,c,d,e){return e?[["M",a+e,b],["l",c-2*e,0],["a",e,e,0,0,1,e,e],["l",0,d-2*e],["a",e,e,0,0,1,-e,e],["l",2*e-c,0],["a",e,e,0,0,1,-e,-e],["l",0,2*e-d],["a",e,e,0,0,1,e,-e],["z"]]:[["M",a,b],["l",c,0],["l",0,d],["l",-c,0],["z"]]},pb=function(a,b,c,d){return null==d&&(d=c),[["M",a,b],["m",0,-d],["a",c,d,0,1,1,0,2*d],["a",c,d,0,1,1,0,-2*d],["z"]]},qb=c._getPath={path:function(a){return a.attr("path")},circle:function(a){var b=a.attrs;return pb(b.cx,b.cy,b.r)},ellipse:function(a){var b=a.attrs;return pb(b.cx,b.cy,b.rx,b.ry)},rect:function(a){var b=a.attrs;return ob(b.x,b.y,b.width,b.height,b.r)},image:function(a){var b=a.attrs;return ob(b.x,b.y,b.width,b.height)},text:function(a){var b=a._getBBox();return ob(b.x,b.y,b.width,b.height)},set:function(a){var b=a._getBBox();return ob(b.x,b.y,b.width,b.height)}},rb=c.mapPath=function(a,b){if(!b)return a;var c,d,e,f,g,h,i;for(a=Kb(a),e=0,g=a.length;g>e;e++)for(i=a[e],f=1,h=i.length;h>f;f+=2)c=b.x(i[f],i[f+1]),d=b.y(i[f],i[f+1]),i[f]=c,i[f+1]=d;return a};if(c._g=A,c.type=A.win.SVGAngle||A.doc.implementation.hasFeature("http://www.w3.org/TR/SVG11/feature#BasicStructure","1.1")?"SVG":"VML","VML"==c.type){var sb,tb=A.doc.createElement("div");if(tb.innerHTML='<v:shape adj="1"/>',sb=tb.firstChild,sb.style.behavior="url(#default#VML)",!sb||"object"!=typeof sb.adj)return c.type=G;tb=null}c.svg=!(c.vml="VML"==c.type),c._Paper=C,c.fn=v=C.prototype=c.prototype,c._id=0,c._oid=0,c.is=function(a,b){return b=M.call(b),"finite"==b?!Y[z](+a):"array"==b?a instanceof Array:"null"==b&&null===a||b==typeof a&&null!==a||"object"==b&&a===Object(a)||"array"==b&&Array.isArray&&Array.isArray(a)||W.call(a).slice(8,-1).toLowerCase()==b},c.angle=function(a,b,d,e,f,g){if(null==f){var h=a-d,i=b-e;return h||i?(180+180*N.atan2(-i,-h)/S+360)%360:0}return c.angle(a,b,f,g)-c.angle(d,e,f,g)},c.rad=function(a){return a%360*S/180},c.deg=function(a){return 180*a/S%360},c.snapTo=function(a,b,d){if(d=c.is(d,"finite")?d:10,c.is(a,V)){for(var e=a.length;e--;)if(Q(a[e]-b)<=d)return a[e]}else{a=+a;var f=b%a;if(d>f)return b-f;if(f>a-d)return b-f+a}return b},c.createUUID=function(a,b){return function(){return"xxxxxxxx-xxxx-4xxx-yxxx-xxxxxxxxxxxx".replace(a,b).toUpperCase()}}(/[xy]/g,function(a){var b=0|16*N.random(),c="x"==a?b:8|3&b;return c.toString(16)}),c.setWindow=function(a){b("raphael.setWindow",c,A.win,a),A.win=a,A.doc=A.win.document,c._engine.initWin&&c._engine.initWin(A.win)};var ub=function(a){if(c.vml){var b,d=/^\s+|\s+$/g;try{var e=new ActiveXObject("htmlfile");e.write("<body>"),e.close(),b=e.body}catch(g){b=createPopup().document.body}var h=b.createTextRange();ub=f(function(a){try{b.style.color=I(a).replace(d,G);var c=h.queryCommandValue("ForeColor");return c=(255&c)<<16|65280&c|(16711680&c)>>>16,"#"+("000000"+c.toString(16)).slice(-6)}catch(e){return"none"}})}else{var i=A.doc.createElement("i");i.title="Raphaël Colour Picker",i.style.display="none",A.doc.body.appendChild(i),ub=f(function(a){return i.style.color=a,A.doc.defaultView.getComputedStyle(i,G).getPropertyValue("color")})}return ub(a)},vb=function(){return"hsb("+[this.h,this.s,this.b]+")"},wb=function(){return"hsl("+[this.h,this.s,this.l]+")"},xb=function(){return this.hex},yb=function(a,b,d){if(null==b&&c.is(a,"object")&&"r"in a&&"g"in a&&"b"in a&&(d=a.b,b=a.g,a=a.r),null==b&&c.is(a,U)){var e=c.getRGB(a);a=e.r,b=e.g,d=e.b}return(a>1||b>1||d>1)&&(a/=255,b/=255,d/=255),[a,b,d]},zb=function(a,b,d,e){a*=255,b*=255,d*=255;var f={r:a,g:b,b:d,hex:c.rgb(a,b,d),toString:xb};return c.is(e,"finite")&&(f.opacity=e),f};c.color=function(a){var b;return c.is(a,"object")&&"h"in a&&"s"in a&&"b"in a?(b=c.hsb2rgb(a),a.r=b.r,a.g=b.g,a.b=b.b,a.hex=b.hex):c.is(a,"object")&&"h"in a&&"s"in a&&"l"in a?(b=c.hsl2rgb(a),a.r=b.r,a.g=b.g,a.b=b.b,a.hex=b.hex):(c.is(a,"string")&&(a=c.getRGB(a)),c.is(a,"object")&&"r"in a&&"g"in a&&"b"in a?(b=c.rgb2hsl(a),a.h=b.h,a.s=b.s,a.l=b.l,b=c.rgb2hsb(a),a.v=b.b):(a={hex:"none"},a.r=a.g=a.b=a.h=a.s=a.v=a.l=-1)),a.toString=xb,a},c.hsb2rgb=function(a,b,c,d){this.is(a,"object")&&"h"in a&&"s"in a&&"b"in a&&(c=a.b,b=a.s,a=a.h,d=a.o),a*=360;var e,f,g,h,i;return a=a%360/60,i=c*b,h=i*(1-Q(a%2-1)),e=f=g=c-i,a=~~a,e+=[i,h,0,0,h,i][a],f+=[h,i,i,h,0,0][a],g+=[0,0,h,i,i,h][a],zb(e,f,g,d)},c.hsl2rgb=function(a,b,c,d){this.is(a,"object")&&"h"in a&&"s"in a&&"l"in a&&(c=a.l,b=a.s,a=a.h),(a>1||b>1||c>1)&&(a/=360,b/=100,c/=100),a*=360;var e,f,g,h,i;return a=a%360/60,i=2*b*(.5>c?c:1-c),h=i*(1-Q(a%2-1)),e=f=g=c-i/2,a=~~a,e+=[i,h,0,0,h,i][a],f+=[h,i,i,h,0,0][a],g+=[0,0,h,i,i,h][a],zb(e,f,g,d)},c.rgb2hsb=function(a,b,c){c=yb(a,b,c),a=c[0],b=c[1],c=c[2];var d,e,f,g;return f=O(a,b,c),g=f-P(a,b,c),d=0==g?null:f==a?(b-c)/g:f==b?(c-a)/g+2:(a-b)/g+4,d=60*((d+360)%6)/360,e=0==g?0:g/f,{h:d,s:e,b:f,toString:vb}},c.rgb2hsl=function(a,b,c){c=yb(a,b,c),a=c[0],b=c[1],c=c[2];var d,e,f,g,h,i;return g=O(a,b,c),h=P(a,b,c),i=g-h,d=0==i?null:g==a?(b-c)/i:g==b?(c-a)/i+2:(a-b)/i+4,d=60*((d+360)%6)/360,f=(g+h)/2,e=0==i?0:.5>f?i/(2*f):i/(2-2*f),{h:d,s:e,l:f,toString:wb}},c._path2string=function(){return this.join(",").replace(gb,"$1")},c._preload=function(a,b){var c=A.doc.createElement("img");c.style.cssText="position:absolute;left:-9999em;top:-9999em",c.onload=function(){b.call(this),this.onload=null,A.doc.body.removeChild(this)},c.onerror=function(){A.doc.body.removeChild(this)},A.doc.body.appendChild(c),c.src=a},c.getRGB=f(function(a){if(!a||(a=I(a)).indexOf("-")+1)return{r:-1,g:-1,b:-1,hex:"none",error:1,toString:g};if("none"==a)return{r:-1,g:-1,b:-1,hex:"none",toString:g};!(fb[z](a.toLowerCase().substring(0,2))||"#"==a.charAt())&&(a=ub(a));var b,d,e,f,h,i,j=a.match(X);return j?(j[2]&&(e=ab(j[2].substring(5),16),d=ab(j[2].substring(3,5),16),b=ab(j[2].substring(1,3),16)),j[3]&&(e=ab((h=j[3].charAt(3))+h,16),d=ab((h=j[3].charAt(2))+h,16),b=ab((h=j[3].charAt(1))+h,16)),j[4]&&(i=j[4][J](eb),b=_(i[0]),"%"==i[0].slice(-1)&&(b*=2.55),d=_(i[1]),"%"==i[1].slice(-1)&&(d*=2.55),e=_(i[2]),"%"==i[2].slice(-1)&&(e*=2.55),"rgba"==j[1].toLowerCase().slice(0,4)&&(f=_(i[3])),i[3]&&"%"==i[3].slice(-1)&&(f/=100)),j[5]?(i=j[5][J](eb),b=_(i[0]),"%"==i[0].slice(-1)&&(b*=2.55),d=_(i[1]),"%"==i[1].slice(-1)&&(d*=2.55),e=_(i[2]),"%"==i[2].slice(-1)&&(e*=2.55),("deg"==i[0].slice(-3)||"°"==i[0].slice(-1))&&(b/=360),"hsba"==j[1].toLowerCase().slice(0,4)&&(f=_(i[3])),i[3]&&"%"==i[3].slice(-1)&&(f/=100),c.hsb2rgb(b,d,e,f)):j[6]?(i=j[6][J](eb),b=_(i[0]),"%"==i[0].slice(-1)&&(b*=2.55),d=_(i[1]),"%"==i[1].slice(-1)&&(d*=2.55),e=_(i[2]),"%"==i[2].slice(-1)&&(e*=2.55),("deg"==i[0].slice(-3)||"°"==i[0].slice(-1))&&(b/=360),"hsla"==j[1].toLowerCase().slice(0,4)&&(f=_(i[3])),i[3]&&"%"==i[3].slice(-1)&&(f/=100),c.hsl2rgb(b,d,e,f)):(j={r:b,g:d,b:e,toString:g},j.hex="#"+(16777216|e|d<<8|b<<16).toString(16).slice(1),c.is(f,"finite")&&(j.opacity=f),j)):{r:-1,g:-1,b:-1,hex:"none",error:1,toString:g}},c),c.hsb=f(function(a,b,d){return c.hsb2rgb(a,b,d).hex}),c.hsl=f(function(a,b,d){return c.hsl2rgb(a,b,d).hex}),c.rgb=f(function(a,b,c){return"#"+(16777216|c|b<<8|a<<16).toString(16).slice(1)}),c.getColor=function(a){var b=this.getColor.start=this.getColor.start||{h:0,s:1,b:a||.75},c=this.hsb2rgb(b.h,b.s,b.b);return b.h+=.075,b.h>1&&(b.h=0,b.s-=.2,b.s<=0&&(this.getColor.start={h:0,s:1,b:b.b})),c.hex},c.getColor.reset=function(){delete this.start},c.parsePathString=function(a){if(!a)return null;var b=Ab(a);if(b.arr)return Cb(b.arr);var d={a:7,c:6,h:1,l:2,m:2,r:4,q:4,s:4,t:2,v:1,z:0},e=[];return c.is(a,V)&&c.is(a[0],V)&&(e=Cb(a)),e.length||I(a).replace(hb,function(a,b,c){var f=[],g=b.toLowerCase();if(c.replace(jb,function(a,b){b&&f.push(+b)}),"m"==g&&f.length>2&&(e.push([b][E](f.splice(0,2))),g="l",b="m"==b?"l":"L"),"r"==g)e.push([b][E](f));else for(;f.length>=d[g]&&(e.push([b][E](f.splice(0,d[g]))),d[g]););}),e.toString=c._path2string,b.arr=Cb(e),e},c.parseTransformString=f(function(a){if(!a)return null;var b=[];return c.is(a,V)&&c.is(a[0],V)&&(b=Cb(a)),b.length||I(a).replace(ib,function(a,c,d){var e=[];M.call(c),d.replace(jb,function(a,b){b&&e.push(+b)}),b.push([c][E](e))}),b.toString=c._path2string,b});var Ab=function(a){var b=Ab.ps=Ab.ps||{};return b[a]?b[a].sleep=100:b[a]={sleep:100},setTimeout(function(){for(var c in b)b[z](c)&&c!=a&&(b[c].sleep--,!b[c].sleep&&delete b[c])}),b[a]};c.findDotsAtSegment=function(a,b,c,d,e,f,g,h,i){var j=1-i,k=R(j,3),l=R(j,2),m=i*i,n=m*i,o=k*a+3*l*i*c+3*j*i*i*e+n*g,p=k*b+3*l*i*d+3*j*i*i*f+n*h,q=a+2*i*(c-a)+m*(e-2*c+a),r=b+2*i*(d-b)+m*(f-2*d+b),s=c+2*i*(e-c)+m*(g-2*e+c),t=d+2*i*(f-d)+m*(h-2*f+d),u=j*a+i*c,v=j*b+i*d,w=j*e+i*g,x=j*f+i*h,y=90-180*N.atan2(q-s,r-t)/S;return(q>s||t>r)&&(y+=180),{x:o,y:p,m:{x:q,y:r},n:{x:s,y:t},start:{x:u,y:v},end:{x:w,y:x},alpha:y}},c.bezierBBox=function(a,b,d,e,f,g,h,i){c.is(a,"array")||(a=[a,b,d,e,f,g,h,i]);var j=Jb.apply(null,a);return{x:j.min.x,y:j.min.y,x2:j.max.x,y2:j.max.y,width:j.max.x-j.min.x,height:j.max.y-j.min.y}},c.isPointInsideBBox=function(a,b,c){return b>=a.x&&b<=a.x2&&c>=a.y&&c<=a.y2},c.isBBoxIntersect=function(a,b){var d=c.isPointInsideBBox;return d(b,a.x,a.y)||d(b,a.x2,a.y)||d(b,a.x,a.y2)||d(b,a.x2,a.y2)||d(a,b.x,b.y)||d(a,b.x2,b.y)||d(a,b.x,b.y2)||d(a,b.x2,b.y2)||(a.x<b.x2&&a.x>b.x||b.x<a.x2&&b.x>a.x)&&(a.y<b.y2&&a.y>b.y||b.y<a.y2&&b.y>a.y)},c.pathIntersection=function(a,b){return n(a,b)},c.pathIntersectionNumber=function(a,b){return n(a,b,1)},c.isPointInsidePath=function(a,b,d){var e=c.pathBBox(a);return c.isPointInsideBBox(e,b,d)&&1==n(a,[["M",b,d],["H",e.x2+10]],1)%2},c._removedFactory=function(a){return function(){b("raphael.log",null,"Raphaël: you are calling to method “"+a+"” of removed object",a)}};var Bb=c.pathBBox=function(a){var b=Ab(a);if(b.bbox)return d(b.bbox);if(!a)return{x:0,y:0,width:0,height:0,x2:0,y2:0};a=Kb(a);for(var c,e=0,f=0,g=[],h=[],i=0,j=a.length;j>i;i++)if(c=a[i],"M"==c[0])e=c[1],f=c[2],g.push(e),h.push(f);else{var k=Jb(e,f,c[1],c[2],c[3],c[4],c[5],c[6]);g=g[E](k.min.x,k.max.x),h=h[E](k.min.y,k.max.y),e=c[5],f=c[6]}var l=P[D](0,g),m=P[D](0,h),n=O[D](0,g),o=O[D](0,h),p=n-l,q=o-m,r={x:l,y:m,x2:n,y2:o,width:p,height:q,cx:l+p/2,cy:m+q/2};return b.bbox=d(r),r},Cb=function(a){var b=d(a);return b.toString=c._path2string,b},Db=c._pathToRelative=function(a){var b=Ab(a);if(b.rel)return Cb(b.rel);c.is(a,V)&&c.is(a&&a[0],V)||(a=c.parsePathString(a));var d=[],e=0,f=0,g=0,h=0,i=0;"M"==a[0][0]&&(e=a[0][1],f=a[0][2],g=e,h=f,i++,d.push(["M",e,f]));for(var j=i,k=a.length;k>j;j++){var l=d[j]=[],m=a[j];if(m[0]!=M.call(m[0]))switch(l[0]=M.call(m[0]),l[0]){case"a":l[1]=m[1],l[2]=m[2],l[3]=m[3],l[4]=m[4],l[5]=m[5],l[6]=+(m[6]-e).toFixed(3),l[7]=+(m[7]-f).toFixed(3);break;case"v":l[1]=+(m[1]-f).toFixed(3);break;case"m":g=m[1],h=m[2];default:for(var n=1,o=m.length;o>n;n++)l[n]=+(m[n]-(n%2?e:f)).toFixed(3)}else{l=d[j]=[],"m"==m[0]&&(g=m[1]+e,h=m[2]+f);for(var p=0,q=m.length;q>p;p++)d[j][p]=m[p]}var r=d[j].length;switch(d[j][0]){case"z":e=g,f=h;break;case"h":e+=+d[j][r-1];break;case"v":f+=+d[j][r-1];break;default:e+=+d[j][r-2],f+=+d[j][r-1]}}return d.toString=c._path2string,b.rel=Cb(d),d},Eb=c._pathToAbsolute=function(a){var b=Ab(a);if(b.abs)return Cb(b.abs);if(c.is(a,V)&&c.is(a&&a[0],V)||(a=c.parsePathString(a)),!a||!a.length)return[["M",0,0]];var d=[],e=0,f=0,g=0,i=0,j=0;"M"==a[0][0]&&(e=+a[0][1],f=+a[0][2],g=e,i=f,j++,d[0]=["M",e,f]);for(var k,l,m=3==a.length&&"M"==a[0][0]&&"R"==a[1][0].toUpperCase()&&"Z"==a[2][0].toUpperCase(),n=j,o=a.length;o>n;n++){if(d.push(k=[]),l=a[n],l[0]!=bb.call(l[0]))switch(k[0]=bb.call(l[0]),k[0]){case"A":k[1]=l[1],k[2]=l[2],k[3]=l[3],k[4]=l[4],k[5]=l[5],k[6]=+(l[6]+e),k[7]=+(l[7]+f);break;case"V":k[1]=+l[1]+f;break;case"H":k[1]=+l[1]+e;break;case"R":for(var p=[e,f][E](l.slice(1)),q=2,r=p.length;r>q;q++)p[q]=+p[q]+e,p[++q]=+p[q]+f;d.pop(),d=d[E](h(p,m));break;case"M":g=+l[1]+e,i=+l[2]+f;default:for(q=1,r=l.length;r>q;q++)k[q]=+l[q]+(q%2?e:f)}else if("R"==l[0])p=[e,f][E](l.slice(1)),d.pop(),d=d[E](h(p,m)),k=["R"][E](l.slice(-2));else for(var s=0,t=l.length;t>s;s++)k[s]=l[s];switch(k[0]){case"Z":e=g,f=i;break;case"H":e=k[1];break;case"V":f=k[1];break;case"M":g=k[k.length-2],i=k[k.length-1];default:e=k[k.length-2],f=k[k.length-1]}}return d.toString=c._path2string,b.abs=Cb(d),d},Fb=function(a,b,c,d){return[a,b,c,d,c,d]},Gb=function(a,b,c,d,e,f){var g=1/3,h=2/3;return[g*a+h*c,g*b+h*d,g*e+h*c,g*f+h*d,e,f]},Hb=function(a,b,c,d,e,g,h,i,j,k){var l,m=120*S/180,n=S/180*(+e||0),o=[],p=f(function(a,b,c){var d=a*N.cos(c)-b*N.sin(c),e=a*N.sin(c)+b*N.cos(c);return{x:d,y:e}});if(k)y=k[0],z=k[1],w=k[2],x=k[3];else{l=p(a,b,-n),a=l.x,b=l.y,l=p(i,j,-n),i=l.x,j=l.y;var q=(N.cos(S/180*e),N.sin(S/180*e),(a-i)/2),r=(b-j)/2,s=q*q/(c*c)+r*r/(d*d);s>1&&(s=N.sqrt(s),c=s*c,d=s*d);var t=c*c,u=d*d,v=(g==h?-1:1)*N.sqrt(Q((t*u-t*r*r-u*q*q)/(t*r*r+u*q*q))),w=v*c*r/d+(a+i)/2,x=v*-d*q/c+(b+j)/2,y=N.asin(((b-x)/d).toFixed(9)),z=N.asin(((j-x)/d).toFixed(9));y=w>a?S-y:y,z=w>i?S-z:z,0>y&&(y=2*S+y),0>z&&(z=2*S+z),h&&y>z&&(y-=2*S),!h&&z>y&&(z-=2*S)}var A=z-y;if(Q(A)>m){var B=z,C=i,D=j;z=y+m*(h&&z>y?1:-1),i=w+c*N.cos(z),j=x+d*N.sin(z),o=Hb(i,j,c,d,e,0,h,C,D,[z,B,w,x])}A=z-y;var F=N.cos(y),G=N.sin(y),H=N.cos(z),I=N.sin(z),K=N.tan(A/4),L=4/3*c*K,M=4/3*d*K,O=[a,b],P=[a+L*G,b-M*F],R=[i+L*I,j-M*H],T=[i,j];if(P[0]=2*O[0]-P[0],P[1]=2*O[1]-P[1],k)return[P,R,T][E](o);o=[P,R,T][E](o).join()[J](",");for(var U=[],V=0,W=o.length;W>V;V++)U[V]=V%2?p(o[V-1],o[V],n).y:p(o[V],o[V+1],n).x;return U},Ib=function(a,b,c,d,e,f,g,h,i){var j=1-i;return{x:R(j,3)*a+3*R(j,2)*i*c+3*j*i*i*e+R(i,3)*g,y:R(j,3)*b+3*R(j,2)*i*d+3*j*i*i*f+R(i,3)*h}},Jb=f(function(a,b,c,d,e,f,g,h){var i,j=e-2*c+a-(g-2*e+c),k=2*(c-a)-2*(e-c),l=a-c,m=(-k+N.sqrt(k*k-4*j*l))/2/j,n=(-k-N.sqrt(k*k-4*j*l))/2/j,o=[b,h],p=[a,g];return Q(m)>"1e12"&&(m=.5),Q(n)>"1e12"&&(n=.5),m>0&&1>m&&(i=Ib(a,b,c,d,e,f,g,h,m),p.push(i.x),o.push(i.y)),n>0&&1>n&&(i=Ib(a,b,c,d,e,f,g,h,n),p.push(i.x),o.push(i.y)),j=f-2*d+b-(h-2*f+d),k=2*(d-b)-2*(f-d),l=b-d,m=(-k+N.sqrt(k*k-4*j*l))/2/j,n=(-k-N.sqrt(k*k-4*j*l))/2/j,Q(m)>"1e12"&&(m=.5),Q(n)>"1e12"&&(n=.5),m>0&&1>m&&(i=Ib(a,b,c,d,e,f,g,h,m),p.push(i.x),o.push(i.y)),n>0&&1>n&&(i=Ib(a,b,c,d,e,f,g,h,n),p.push(i.x),o.push(i.y)),{min:{x:P[D](0,p),y:P[D](0,o)},max:{x:O[D](0,p),y:O[D](0,o)}}}),Kb=c._path2curve=f(function(a,b){var c=!b&&Ab(a);if(!b&&c.curve)return Cb(c.curve);for(var d=Eb(a),e=b&&Eb(b),f={x:0,y:0,bx:0,by:0,X:0,Y:0,qx:null,qy:null},g={x:0,y:0,bx:0,by:0,X:0,Y:0,qx:null,qy:null},h=(function(a,b,c){var d,e;if(!a)return["C",b.x,b.y,b.x,b.y,b.x,b.y];switch(!(a[0]in{T:1,Q:1})&&(b.qx=b.qy=null),a[0]){case"M":b.X=a[1],b.Y=a[2];break;case"A":a=["C"][E](Hb[D](0,[b.x,b.y][E](a.slice(1))));break;case"S":"C"==c||"S"==c?(d=2*b.x-b.bx,e=2*b.y-b.by):(d=b.x,e=b.y),a=["C",d,e][E](a.slice(1));break;case"T":"Q"==c||"T"==c?(b.qx=2*b.x-b.qx,b.qy=2*b.y-b.qy):(b.qx=b.x,b.qy=b.y),a=["C"][E](Gb(b.x,b.y,b.qx,b.qy,a[1],a[2]));break;case"Q":b.qx=a[1],b.qy=a[2],a=["C"][E](Gb(b.x,b.y,a[1],a[2],a[3],a[4]));break;case"L":a=["C"][E](Fb(b.x,b.y,a[1],a[2]));break;case"H":a=["C"][E](Fb(b.x,b.y,a[1],b.y));break;case"V":a=["C"][E](Fb(b.x,b.y,b.x,a[1]));break;case"Z":a=["C"][E](Fb(b.x,b.y,b.X,b.Y))}return a}),i=function(a,b){if(a[b].length>7){a[b].shift();for(var c=a[b];c.length;)a.splice(b++,0,["C"][E](c.splice(0,6)));a.splice(b,1),l=O(d.length,e&&e.length||0)}},j=function(a,b,c,f,g){a&&b&&"M"==a[g][0]&&"M"!=b[g][0]&&(b.splice(g,0,["M",f.x,f.y]),c.bx=0,c.by=0,c.x=a[g][1],c.y=a[g][2],l=O(d.length,e&&e.length||0))},k=0,l=O(d.length,e&&e.length||0);l>k;k++){d[k]=h(d[k],f),i(d,k),e&&(e[k]=h(e[k],g)),e&&i(e,k),j(d,e,f,g,k),j(e,d,g,f,k);var m=d[k],n=e&&e[k],o=m.length,p=e&&n.length;f.x=m[o-2],f.y=m[o-1],f.bx=_(m[o-4])||f.x,f.by=_(m[o-3])||f.y,g.bx=e&&(_(n[p-4])||g.x),g.by=e&&(_(n[p-3])||g.y),g.x=e&&n[p-2],g.y=e&&n[p-1]}return e||(c.curve=Cb(d)),e?[d,e]:d},null,Cb),Lb=(c._parseDots=f(function(a){for(var b=[],d=0,e=a.length;e>d;d++){var f={},g=a[d].match(/^([^:]*):?([\d\.]*)/);if(f.color=c.getRGB(g[1]),f.color.error)return null;f.color=f.color.hex,g[2]&&(f.offset=g[2]+"%"),b.push(f)}for(d=1,e=b.length-1;e>d;d++)if(!b[d].offset){for(var h=_(b[d-1].offset||0),i=0,j=d+1;e>j;j++)if(b[j].offset){i=b[j].offset;break}i||(i=100,j=e),i=_(i);for(var k=(i-h)/(j-d+1);j>d;d++)h+=k,b[d].offset=h+"%"}return b}),c._tear=function(a,b){a==b.top&&(b.top=a.prev),a==b.bottom&&(b.bottom=a.next),a.next&&(a.next.prev=a.prev),a.prev&&(a.prev.next=a.next)}),Mb=(c._tofront=function(a,b){b.top!==a&&(Lb(a,b),a.next=null,a.prev=b.top,b.top.next=a,b.top=a)},c._toback=function(a,b){b.bottom!==a&&(Lb(a,b),a.next=b.bottom,a.prev=null,b.bottom.prev=a,b.bottom=a)},c._insertafter=function(a,b,c){Lb(a,c),b==c.top&&(c.top=a),b.next&&(b.next.prev=a),a.next=b.next,a.prev=b,b.next=a},c._insertbefore=function(a,b,c){Lb(a,c),b==c.bottom&&(c.bottom=a),b.prev&&(b.prev.next=a),a.prev=b.prev,b.prev=a,a.next=b},c.toMatrix=function(a,b){var c=Bb(a),d={_:{transform:G},getBBox:function(){return c}};return Nb(d,b),d.matrix}),Nb=(c.transformPath=function(a,b){return rb(a,Mb(a,b))},c._extractTransform=function(a,b){if(null==b)return a._.transform;b=I(b).replace(/\.{3}|\u2026/g,a._.transform||G);var d=c.parseTransformString(b),e=0,f=0,g=0,h=1,i=1,j=a._,k=new o;if(j.transform=d||[],d)for(var l=0,m=d.length;m>l;l++){var n,p,q,r,s,t=d[l],u=t.length,v=I(t[0]).toLowerCase(),w=t[0]!=v,x=w?k.invert():0;"t"==v&&3==u?w?(n=x.x(0,0),p=x.y(0,0),q=x.x(t[1],t[2]),r=x.y(t[1],t[2]),k.translate(q-n,r-p)):k.translate(t[1],t[2]):"r"==v?2==u?(s=s||a.getBBox(1),k.rotate(t[1],s.x+s.width/2,s.y+s.height/2),e+=t[1]):4==u&&(w?(q=x.x(t[2],t[3]),r=x.y(t[2],t[3]),k.rotate(t[1],q,r)):k.rotate(t[1],t[2],t[3]),e+=t[1]):"s"==v?2==u||3==u?(s=s||a.getBBox(1),k.scale(t[1],t[u-1],s.x+s.width/2,s.y+s.height/2),h*=t[1],i*=t[u-1]):5==u&&(w?(q=x.x(t[3],t[4]),r=x.y(t[3],t[4]),k.scale(t[1],t[2],q,r)):k.scale(t[1],t[2],t[3],t[4]),h*=t[1],i*=t[2]):"m"==v&&7==u&&k.add(t[1],t[2],t[3],t[4],t[5],t[6]),j.dirtyT=1,a.matrix=k}a.matrix=k,j.sx=h,j.sy=i,j.deg=e,j.dx=f=k.e,j.dy=g=k.f,1==h&&1==i&&!e&&j.bbox?(j.bbox.x+=+f,j.bbox.y+=+g):j.dirtyT=1}),Ob=function(a){var b=a[0];switch(b.toLowerCase()){case"t":return[b,0,0];case"m":return[b,1,0,0,1,0,0];case"r":return 4==a.length?[b,0,a[2],a[3]]:[b,0];case"s":return 5==a.length?[b,1,1,a[3],a[4]]:3==a.length?[b,1,1]:[b,1]}},Pb=c._equaliseTransform=function(a,b){b=I(b).replace(/\.{3}|\u2026/g,a),a=c.parseTransformString(a)||[],b=c.parseTransformString(b)||[];for(var d,e,f,g,h=O(a.length,b.length),i=[],j=[],k=0;h>k;k++){if(f=a[k]||Ob(b[k]),g=b[k]||Ob(f),f[0]!=g[0]||"r"==f[0].toLowerCase()&&(f[2]!=g[2]||f[3]!=g[3])||"s"==f[0].toLowerCase()&&(f[3]!=g[3]||f[4]!=g[4]))return;for(i[k]=[],j[k]=[],d=0,e=O(f.length,g.length);e>d;d++)d in f&&(i[k][d]=f[d]),d in g&&(j[k][d]=g[d])
+}return{from:i,to:j}};c._getContainer=function(a,b,d,e){var f;return f=null!=e||c.is(a,"object")?a:A.doc.getElementById(a),null!=f?f.tagName?null==b?{container:f,width:f.style.pixelWidth||f.offsetWidth,height:f.style.pixelHeight||f.offsetHeight}:{container:f,width:b,height:d}:{container:1,x:a,y:b,width:d,height:e}:void 0},c.pathToRelative=Db,c._engine={},c.path2curve=Kb,c.matrix=function(a,b,c,d,e,f){return new o(a,b,c,d,e,f)},function(a){function b(a){return a[0]*a[0]+a[1]*a[1]}function d(a){var c=N.sqrt(b(a));a[0]&&(a[0]/=c),a[1]&&(a[1]/=c)}a.add=function(a,b,c,d,e,f){var g,h,i,j,k=[[],[],[]],l=[[this.a,this.c,this.e],[this.b,this.d,this.f],[0,0,1]],m=[[a,c,e],[b,d,f],[0,0,1]];for(a&&a instanceof o&&(m=[[a.a,a.c,a.e],[a.b,a.d,a.f],[0,0,1]]),g=0;3>g;g++)for(h=0;3>h;h++){for(j=0,i=0;3>i;i++)j+=l[g][i]*m[i][h];k[g][h]=j}this.a=k[0][0],this.b=k[1][0],this.c=k[0][1],this.d=k[1][1],this.e=k[0][2],this.f=k[1][2]},a.invert=function(){var a=this,b=a.a*a.d-a.b*a.c;return new o(a.d/b,-a.b/b,-a.c/b,a.a/b,(a.c*a.f-a.d*a.e)/b,(a.b*a.e-a.a*a.f)/b)},a.clone=function(){return new o(this.a,this.b,this.c,this.d,this.e,this.f)},a.translate=function(a,b){this.add(1,0,0,1,a,b)},a.scale=function(a,b,c,d){null==b&&(b=a),(c||d)&&this.add(1,0,0,1,c,d),this.add(a,0,0,b,0,0),(c||d)&&this.add(1,0,0,1,-c,-d)},a.rotate=function(a,b,d){a=c.rad(a),b=b||0,d=d||0;var e=+N.cos(a).toFixed(9),f=+N.sin(a).toFixed(9);this.add(e,f,-f,e,b,d),this.add(1,0,0,1,-b,-d)},a.x=function(a,b){return a*this.a+b*this.c+this.e},a.y=function(a,b){return a*this.b+b*this.d+this.f},a.get=function(a){return+this[I.fromCharCode(97+a)].toFixed(4)},a.toString=function(){return c.svg?"matrix("+[this.get(0),this.get(1),this.get(2),this.get(3),this.get(4),this.get(5)].join()+")":[this.get(0),this.get(2),this.get(1),this.get(3),0,0].join()},a.toFilter=function(){return"progid:DXImageTransform.Microsoft.Matrix(M11="+this.get(0)+", M12="+this.get(2)+", M21="+this.get(1)+", M22="+this.get(3)+", Dx="+this.get(4)+", Dy="+this.get(5)+", sizingmethod='auto expand')"},a.offset=function(){return[this.e.toFixed(4),this.f.toFixed(4)]},a.split=function(){var a={};a.dx=this.e,a.dy=this.f;var e=[[this.a,this.c],[this.b,this.d]];a.scalex=N.sqrt(b(e[0])),d(e[0]),a.shear=e[0][0]*e[1][0]+e[0][1]*e[1][1],e[1]=[e[1][0]-e[0][0]*a.shear,e[1][1]-e[0][1]*a.shear],a.scaley=N.sqrt(b(e[1])),d(e[1]),a.shear/=a.scaley;var f=-e[0][1],g=e[1][1];return 0>g?(a.rotate=c.deg(N.acos(g)),0>f&&(a.rotate=360-a.rotate)):a.rotate=c.deg(N.asin(f)),a.isSimple=!(+a.shear.toFixed(9)||a.scalex.toFixed(9)!=a.scaley.toFixed(9)&&a.rotate),a.isSuperSimple=!+a.shear.toFixed(9)&&a.scalex.toFixed(9)==a.scaley.toFixed(9)&&!a.rotate,a.noRotation=!+a.shear.toFixed(9)&&!a.rotate,a},a.toTransformString=function(a){var b=a||this[J]();return b.isSimple?(b.scalex=+b.scalex.toFixed(4),b.scaley=+b.scaley.toFixed(4),b.rotate=+b.rotate.toFixed(4),(b.dx||b.dy?"t"+[b.dx,b.dy]:G)+(1!=b.scalex||1!=b.scaley?"s"+[b.scalex,b.scaley,0,0]:G)+(b.rotate?"r"+[b.rotate,0,0]:G)):"m"+[this.get(0),this.get(1),this.get(2),this.get(3),this.get(4),this.get(5)]}}(o.prototype);var Qb=navigator.userAgent.match(/Version\/(.*?)\s/)||navigator.userAgent.match(/Chrome\/(\d+)/);v.safari="Apple Computer, Inc."==navigator.vendor&&(Qb&&Qb[1]<4||"iP"==navigator.platform.slice(0,2))||"Google Inc."==navigator.vendor&&Qb&&Qb[1]<8?function(){var a=this.rect(-99,-99,this.width+99,this.height+99).attr({stroke:"none"});setTimeout(function(){a.remove()})}:mb;for(var Rb=function(){this.returnValue=!1},Sb=function(){return this.originalEvent.preventDefault()},Tb=function(){this.cancelBubble=!0},Ub=function(){return this.originalEvent.stopPropagation()},Vb=function(a){var b=A.doc.documentElement.scrollTop||A.doc.body.scrollTop,c=A.doc.documentElement.scrollLeft||A.doc.body.scrollLeft;return{x:a.clientX+c,y:a.clientY+b}},Wb=function(){return A.doc.addEventListener?function(a,b,c,d){var e=function(a){var b=Vb(a);return c.call(d,a,b.x,b.y)};if(a.addEventListener(b,e,!1),F&&L[b]){var f=function(b){for(var e=Vb(b),f=b,g=0,h=b.targetTouches&&b.targetTouches.length;h>g;g++)if(b.targetTouches[g].target==a){b=b.targetTouches[g],b.originalEvent=f,b.preventDefault=Sb,b.stopPropagation=Ub;break}return c.call(d,b,e.x,e.y)};a.addEventListener(L[b],f,!1)}return function(){return a.removeEventListener(b,e,!1),F&&L[b]&&a.removeEventListener(L[b],e,!1),!0}}:A.doc.attachEvent?function(a,b,c,d){var e=function(a){a=a||A.win.event;var b=A.doc.documentElement.scrollTop||A.doc.body.scrollTop,e=A.doc.documentElement.scrollLeft||A.doc.body.scrollLeft,f=a.clientX+e,g=a.clientY+b;return a.preventDefault=a.preventDefault||Rb,a.stopPropagation=a.stopPropagation||Tb,c.call(d,a,f,g)};a.attachEvent("on"+b,e);var f=function(){return a.detachEvent("on"+b,e),!0};return f}:void 0}(),Xb=[],Yb=function(a){for(var c,d=a.clientX,e=a.clientY,f=A.doc.documentElement.scrollTop||A.doc.body.scrollTop,g=A.doc.documentElement.scrollLeft||A.doc.body.scrollLeft,h=Xb.length;h--;){if(c=Xb[h],F&&a.touches){for(var i,j=a.touches.length;j--;)if(i=a.touches[j],i.identifier==c.el._drag.id){d=i.clientX,e=i.clientY,(a.originalEvent?a.originalEvent:a).preventDefault();break}}else a.preventDefault();var k,l=c.el.node,m=l.nextSibling,n=l.parentNode,o=l.style.display;A.win.opera&&n.removeChild(l),l.style.display="none",k=c.el.paper.getElementByPoint(d,e),l.style.display=o,A.win.opera&&(m?n.insertBefore(l,m):n.appendChild(l)),k&&b("raphael.drag.over."+c.el.id,c.el,k),d+=g,e+=f,b("raphael.drag.move."+c.el.id,c.move_scope||c.el,d-c.el._drag.x,e-c.el._drag.y,d,e,a)}},Zb=function(a){c.unmousemove(Yb).unmouseup(Zb);for(var d,e=Xb.length;e--;)d=Xb[e],d.el._drag={},b("raphael.drag.end."+d.el.id,d.end_scope||d.start_scope||d.move_scope||d.el,a);Xb=[]},$b=c.el={},_b=K.length;_b--;)!function(a){c[a]=$b[a]=function(b,d){return c.is(b,"function")&&(this.events=this.events||[],this.events.push({name:a,f:b,unbind:Wb(this.shape||this.node||A.doc,a,b,d||this)})),this},c["un"+a]=$b["un"+a]=function(b){for(var d=this.events||[],e=d.length;e--;)d[e].name!=a||!c.is(b,"undefined")&&d[e].f!=b||(d[e].unbind(),d.splice(e,1),!d.length&&delete this.events);return this}}(K[_b]);$b.data=function(a,d){var e=kb[this.id]=kb[this.id]||{};if(0==arguments.length)return e;if(1==arguments.length){if(c.is(a,"object")){for(var f in a)a[z](f)&&this.data(f,a[f]);return this}return b("raphael.data.get."+this.id,this,e[a],a),e[a]}return e[a]=d,b("raphael.data.set."+this.id,this,d,a),this},$b.removeData=function(a){return null==a?kb[this.id]={}:kb[this.id]&&delete kb[this.id][a],this},$b.getData=function(){return d(kb[this.id]||{})},$b.hover=function(a,b,c,d){return this.mouseover(a,c).mouseout(b,d||c)},$b.unhover=function(a,b){return this.unmouseover(a).unmouseout(b)};var ac=[];$b.drag=function(a,d,e,f,g,h){function i(i){(i.originalEvent||i).preventDefault();var j=i.clientX,k=i.clientY,l=A.doc.documentElement.scrollTop||A.doc.body.scrollTop,m=A.doc.documentElement.scrollLeft||A.doc.body.scrollLeft;if(this._drag.id=i.identifier,F&&i.touches)for(var n,o=i.touches.length;o--;)if(n=i.touches[o],this._drag.id=n.identifier,n.identifier==this._drag.id){j=n.clientX,k=n.clientY;break}this._drag.x=j+m,this._drag.y=k+l,!Xb.length&&c.mousemove(Yb).mouseup(Zb),Xb.push({el:this,move_scope:f,start_scope:g,end_scope:h}),d&&b.on("raphael.drag.start."+this.id,d),a&&b.on("raphael.drag.move."+this.id,a),e&&b.on("raphael.drag.end."+this.id,e),b("raphael.drag.start."+this.id,g||f||this,i.clientX+m,i.clientY+l,i)}return this._drag={},ac.push({el:this,start:i}),this.mousedown(i),this},$b.onDragOver=function(a){a?b.on("raphael.drag.over."+this.id,a):b.unbind("raphael.drag.over."+this.id)},$b.undrag=function(){for(var a=ac.length;a--;)ac[a].el==this&&(this.unmousedown(ac[a].start),ac.splice(a,1),b.unbind("raphael.drag.*."+this.id));!ac.length&&c.unmousemove(Yb).unmouseup(Zb),Xb=[]},v.circle=function(a,b,d){var e=c._engine.circle(this,a||0,b||0,d||0);return this.__set__&&this.__set__.push(e),e},v.rect=function(a,b,d,e,f){var g=c._engine.rect(this,a||0,b||0,d||0,e||0,f||0);return this.__set__&&this.__set__.push(g),g},v.ellipse=function(a,b,d,e){var f=c._engine.ellipse(this,a||0,b||0,d||0,e||0);return this.__set__&&this.__set__.push(f),f},v.path=function(a){a&&!c.is(a,U)&&!c.is(a[0],V)&&(a+=G);var b=c._engine.path(c.format[D](c,arguments),this);return this.__set__&&this.__set__.push(b),b},v.image=function(a,b,d,e,f){var g=c._engine.image(this,a||"about:blank",b||0,d||0,e||0,f||0);return this.__set__&&this.__set__.push(g),g},v.text=function(a,b,d){var e=c._engine.text(this,a||0,b||0,I(d));return this.__set__&&this.__set__.push(e),e},v.set=function(a){!c.is(a,"array")&&(a=Array.prototype.splice.call(arguments,0,arguments.length));var b=new mc(a);return this.__set__&&this.__set__.push(b),b.paper=this,b.type="set",b},v.setStart=function(a){this.__set__=a||this.set()},v.setFinish=function(){var a=this.__set__;return delete this.__set__,a},v.setSize=function(a,b){return c._engine.setSize.call(this,a,b)},v.setViewBox=function(a,b,d,e,f){return c._engine.setViewBox.call(this,a,b,d,e,f)},v.top=v.bottom=null,v.raphael=c;var bc=function(a){var b=a.getBoundingClientRect(),c=a.ownerDocument,d=c.body,e=c.documentElement,f=e.clientTop||d.clientTop||0,g=e.clientLeft||d.clientLeft||0,h=b.top+(A.win.pageYOffset||e.scrollTop||d.scrollTop)-f,i=b.left+(A.win.pageXOffset||e.scrollLeft||d.scrollLeft)-g;return{y:h,x:i}};v.getElementByPoint=function(a,b){var c=this,d=c.canvas,e=A.doc.elementFromPoint(a,b);if(A.win.opera&&"svg"==e.tagName){var f=bc(d),g=d.createSVGRect();g.x=a-f.x,g.y=b-f.y,g.width=g.height=1;var h=d.getIntersectionList(g,null);h.length&&(e=h[h.length-1])}if(!e)return null;for(;e.parentNode&&e!=d.parentNode&&!e.raphael;)e=e.parentNode;return e==c.canvas.parentNode&&(e=d),e=e&&e.raphael?c.getById(e.raphaelid):null},v.getElementsByBBox=function(a){var b=this.set();return this.forEach(function(d){c.isBBoxIntersect(d.getBBox(),a)&&b.push(d)}),b},v.getById=function(a){for(var b=this.bottom;b;){if(b.id==a)return b;b=b.next}return null},v.forEach=function(a,b){for(var c=this.bottom;c;){if(a.call(b,c)===!1)return this;c=c.next}return this},v.getElementsByPoint=function(a,b){var c=this.set();return this.forEach(function(d){d.isPointInside(a,b)&&c.push(d)}),c},$b.isPointInside=function(a,b){var d=this.realPath=qb[this.type](this);return this.attr("transform")&&this.attr("transform").length&&(d=c.transformPath(d,this.attr("transform"))),c.isPointInsidePath(d,a,b)},$b.getBBox=function(a){if(this.removed)return{};var b=this._;return a?((b.dirty||!b.bboxwt)&&(this.realPath=qb[this.type](this),b.bboxwt=Bb(this.realPath),b.bboxwt.toString=p,b.dirty=0),b.bboxwt):((b.dirty||b.dirtyT||!b.bbox)&&((b.dirty||!this.realPath)&&(b.bboxwt=0,this.realPath=qb[this.type](this)),b.bbox=Bb(rb(this.realPath,this.matrix)),b.bbox.toString=p,b.dirty=b.dirtyT=0),b.bbox)},$b.clone=function(){if(this.removed)return null;var a=this.paper[this.type]().attr(this.attr());return this.__set__&&this.__set__.push(a),a},$b.glow=function(a){if("text"==this.type)return null;a=a||{};var b={width:(a.width||10)+(+this.attr("stroke-width")||1),fill:a.fill||!1,opacity:a.opacity||.5,offsetx:a.offsetx||0,offsety:a.offsety||0,color:a.color||"#000"},c=b.width/2,d=this.paper,e=d.set(),f=this.realPath||qb[this.type](this);f=this.matrix?rb(f,this.matrix):f;for(var g=1;c+1>g;g++)e.push(d.path(f).attr({stroke:b.color,fill:b.fill?b.color:"none","stroke-linejoin":"round","stroke-linecap":"round","stroke-width":+(b.width/c*g).toFixed(3),opacity:+(b.opacity/c).toFixed(3)}));return e.insertBefore(this).translate(b.offsetx,b.offsety)};var cc=function(a,b,d,e,f,g,h,i,l){return null==l?j(a,b,d,e,f,g,h,i):c.findDotsAtSegment(a,b,d,e,f,g,h,i,k(a,b,d,e,f,g,h,i,l))},dc=function(a,b){return function(d,e,f){d=Kb(d);for(var g,h,i,j,k,l="",m={},n=0,o=0,p=d.length;p>o;o++){if(i=d[o],"M"==i[0])g=+i[1],h=+i[2];else{if(j=cc(g,h,i[1],i[2],i[3],i[4],i[5],i[6]),n+j>e){if(b&&!m.start){if(k=cc(g,h,i[1],i[2],i[3],i[4],i[5],i[6],e-n),l+=["C"+k.start.x,k.start.y,k.m.x,k.m.y,k.x,k.y],f)return l;m.start=l,l=["M"+k.x,k.y+"C"+k.n.x,k.n.y,k.end.x,k.end.y,i[5],i[6]].join(),n+=j,g=+i[5],h=+i[6];continue}if(!a&&!b)return k=cc(g,h,i[1],i[2],i[3],i[4],i[5],i[6],e-n),{x:k.x,y:k.y,alpha:k.alpha}}n+=j,g=+i[5],h=+i[6]}l+=i.shift()+i}return m.end=l,k=a?n:b?m:c.findDotsAtSegment(g,h,i[0],i[1],i[2],i[3],i[4],i[5],1),k.alpha&&(k={x:k.x,y:k.y,alpha:k.alpha}),k}},ec=dc(1),fc=dc(),gc=dc(0,1);c.getTotalLength=ec,c.getPointAtLength=fc,c.getSubpath=function(a,b,c){if(this.getTotalLength(a)-c<1e-6)return gc(a,b).end;var d=gc(a,c,1);return b?gc(d,b).end:d},$b.getTotalLength=function(){var a=this.getPath();if(a)return this.node.getTotalLength?this.node.getTotalLength():ec(a)},$b.getPointAtLength=function(a){var b=this.getPath();if(b)return fc(b,a)},$b.getPath=function(){var a,b=c._getPath[this.type];if("text"!=this.type&&"set"!=this.type)return b&&(a=b(this)),a},$b.getSubpath=function(a,b){var d=this.getPath();if(d)return c.getSubpath(d,a,b)};var hc=c.easing_formulas={linear:function(a){return a},"<":function(a){return R(a,1.7)},">":function(a){return R(a,.48)},"<>":function(a){var b=.48-a/1.04,c=N.sqrt(.1734+b*b),d=c-b,e=R(Q(d),1/3)*(0>d?-1:1),f=-c-b,g=R(Q(f),1/3)*(0>f?-1:1),h=e+g+.5;return 3*(1-h)*h*h+h*h*h},backIn:function(a){var b=1.70158;return a*a*((b+1)*a-b)},backOut:function(a){a-=1;var b=1.70158;return a*a*((b+1)*a+b)+1},elastic:function(a){return a==!!a?a:R(2,-10*a)*N.sin((a-.075)*2*S/.3)+1},bounce:function(a){var b,c=7.5625,d=2.75;return 1/d>a?b=c*a*a:2/d>a?(a-=1.5/d,b=c*a*a+.75):2.5/d>a?(a-=2.25/d,b=c*a*a+.9375):(a-=2.625/d,b=c*a*a+.984375),b}};hc.easeIn=hc["ease-in"]=hc["<"],hc.easeOut=hc["ease-out"]=hc[">"],hc.easeInOut=hc["ease-in-out"]=hc["<>"],hc["back-in"]=hc.backIn,hc["back-out"]=hc.backOut;var ic=[],jc=a.requestAnimationFrame||a.webkitRequestAnimationFrame||a.mozRequestAnimationFrame||a.oRequestAnimationFrame||a.msRequestAnimationFrame||function(a){setTimeout(a,16)},kc=function(){for(var a=+new Date,d=0;d<ic.length;d++){var e=ic[d];if(!e.el.removed&&!e.paused){var f,g,h=a-e.start,i=e.ms,j=e.easing,k=e.from,l=e.diff,m=e.to,n=(e.t,e.el),o={},p={};if(e.initstatus?(h=(e.initstatus*e.anim.top-e.prev)/(e.percent-e.prev)*i,e.status=e.initstatus,delete e.initstatus,e.stop&&ic.splice(d--,1)):e.status=(e.prev+(e.percent-e.prev)*(h/i))/e.anim.top,!(0>h))if(i>h){var q=j(h/i);for(var r in k)if(k[z](r)){switch(db[r]){case T:f=+k[r]+q*i*l[r];break;case"colour":f="rgb("+[lc($(k[r].r+q*i*l[r].r)),lc($(k[r].g+q*i*l[r].g)),lc($(k[r].b+q*i*l[r].b))].join(",")+")";break;case"path":f=[];for(var t=0,u=k[r].length;u>t;t++){f[t]=[k[r][t][0]];for(var v=1,w=k[r][t].length;w>v;v++)f[t][v]=+k[r][t][v]+q*i*l[r][t][v];f[t]=f[t].join(H)}f=f.join(H);break;case"transform":if(l[r].real)for(f=[],t=0,u=k[r].length;u>t;t++)for(f[t]=[k[r][t][0]],v=1,w=k[r][t].length;w>v;v++)f[t][v]=k[r][t][v]+q*i*l[r][t][v];else{var x=function(a){return+k[r][a]+q*i*l[r][a]};f=[["m",x(0),x(1),x(2),x(3),x(4),x(5)]]}break;case"csv":if("clip-rect"==r)for(f=[],t=4;t--;)f[t]=+k[r][t]+q*i*l[r][t];break;default:var y=[][E](k[r]);for(f=[],t=n.paper.customAttributes[r].length;t--;)f[t]=+y[t]+q*i*l[r][t]}o[r]=f}n.attr(o),function(a,c,d){setTimeout(function(){b("raphael.anim.frame."+a,c,d)})}(n.id,n,e.anim)}else{if(function(a,d,e){setTimeout(function(){b("raphael.anim.frame."+d.id,d,e),b("raphael.anim.finish."+d.id,d,e),c.is(a,"function")&&a.call(d)})}(e.callback,n,e.anim),n.attr(m),ic.splice(d--,1),e.repeat>1&&!e.next){for(g in m)m[z](g)&&(p[g]=e.totalOrigin[g]);e.el.attr(p),s(e.anim,e.el,e.anim.percents[0],null,e.totalOrigin,e.repeat-1)}e.next&&!e.stop&&s(e.anim,e.el,e.next,null,e.totalOrigin,e.repeat)}}}c.svg&&n&&n.paper&&n.paper.safari(),ic.length&&jc(kc)},lc=function(a){return a>255?255:0>a?0:a};$b.animateWith=function(a,b,d,e,f,g){var h=this;if(h.removed)return g&&g.call(h),h;var i=d instanceof r?d:c.animation(d,e,f,g);s(i,h,i.percents[0],null,h.attr());for(var j=0,k=ic.length;k>j;j++)if(ic[j].anim==b&&ic[j].el==a){ic[k-1].start=ic[j].start;break}return h},$b.onAnimation=function(a){return a?b.on("raphael.anim.frame."+this.id,a):b.unbind("raphael.anim.frame."+this.id),this},r.prototype.delay=function(a){var b=new r(this.anim,this.ms);return b.times=this.times,b.del=+a||0,b},r.prototype.repeat=function(a){var b=new r(this.anim,this.ms);return b.del=this.del,b.times=N.floor(O(a,0))||1,b},c.animation=function(a,b,d,e){if(a instanceof r)return a;(c.is(d,"function")||!d)&&(e=e||d||null,d=null),a=Object(a),b=+b||0;var f,g,h={};for(g in a)a[z](g)&&_(g)!=g&&_(g)+"%"!=g&&(f=!0,h[g]=a[g]);return f?(d&&(h.easing=d),e&&(h.callback=e),new r({100:h},b)):new r(a,b)},$b.animate=function(a,b,d,e){var f=this;if(f.removed)return e&&e.call(f),f;var g=a instanceof r?a:c.animation(a,b,d,e);return s(g,f,g.percents[0],null,f.attr()),f},$b.setTime=function(a,b){return a&&null!=b&&this.status(a,P(b,a.ms)/a.ms),this},$b.status=function(a,b){var c,d,e=[],f=0;if(null!=b)return s(a,this,-1,P(b,1)),this;for(c=ic.length;c>f;f++)if(d=ic[f],d.el.id==this.id&&(!a||d.anim==a)){if(a)return d.status;e.push({anim:d.anim,status:d.status})}return a?0:e},$b.pause=function(a){for(var c=0;c<ic.length;c++)ic[c].el.id!=this.id||a&&ic[c].anim!=a||b("raphael.anim.pause."+this.id,this,ic[c].anim)!==!1&&(ic[c].paused=!0);return this},$b.resume=function(a){for(var c=0;c<ic.length;c++)if(ic[c].el.id==this.id&&(!a||ic[c].anim==a)){var d=ic[c];b("raphael.anim.resume."+this.id,this,d.anim)!==!1&&(delete d.paused,this.status(d.anim,d.status))}return this},$b.stop=function(a){for(var c=0;c<ic.length;c++)ic[c].el.id!=this.id||a&&ic[c].anim!=a||b("raphael.anim.stop."+this.id,this,ic[c].anim)!==!1&&ic.splice(c--,1);return this},b.on("raphael.remove",t),b.on("raphael.clear",t),$b.toString=function(){return"Raphaël’s object"};var mc=function(a){if(this.items=[],this.length=0,this.type="set",a)for(var b=0,c=a.length;c>b;b++)!a[b]||a[b].constructor!=$b.constructor&&a[b].constructor!=mc||(this[this.items.length]=this.items[this.items.length]=a[b],this.length++)},nc=mc.prototype;nc.push=function(){for(var a,b,c=0,d=arguments.length;d>c;c++)a=arguments[c],!a||a.constructor!=$b.constructor&&a.constructor!=mc||(b=this.items.length,this[b]=this.items[b]=a,this.length++);return this},nc.pop=function(){return this.length&&delete this[this.length--],this.items.pop()},nc.forEach=function(a,b){for(var c=0,d=this.items.length;d>c;c++)if(a.call(b,this.items[c],c)===!1)return this;return this};for(var oc in $b)$b[z](oc)&&(nc[oc]=function(a){return function(){var b=arguments;return this.forEach(function(c){c[a][D](c,b)})}}(oc));return nc.attr=function(a,b){if(a&&c.is(a,V)&&c.is(a[0],"object"))for(var d=0,e=a.length;e>d;d++)this.items[d].attr(a[d]);else for(var f=0,g=this.items.length;g>f;f++)this.items[f].attr(a,b);return this},nc.clear=function(){for(;this.length;)this.pop()},nc.splice=function(a,b){a=0>a?O(this.length+a,0):a,b=O(0,P(this.length-a,b));var c,d=[],e=[],f=[];for(c=2;c<arguments.length;c++)f.push(arguments[c]);for(c=0;b>c;c++)e.push(this[a+c]);for(;c<this.length-a;c++)d.push(this[a+c]);var g=f.length;for(c=0;c<g+d.length;c++)this.items[a+c]=this[a+c]=g>c?f[c]:d[c-g];for(c=this.items.length=this.length-=b-g;this[c];)delete this[c++];return new mc(e)},nc.exclude=function(a){for(var b=0,c=this.length;c>b;b++)if(this[b]==a)return this.splice(b,1),!0},nc.animate=function(a,b,d,e){(c.is(d,"function")||!d)&&(e=d||null);var f,g,h=this.items.length,i=h,j=this;if(!h)return this;e&&(g=function(){!--h&&e.call(j)}),d=c.is(d,U)?d:g;var k=c.animation(a,b,d,g);for(f=this.items[--i].animate(k);i--;)this.items[i]&&!this.items[i].removed&&this.items[i].animateWith(f,k,k),this.items[i]&&!this.items[i].removed||h--;return this},nc.insertAfter=function(a){for(var b=this.items.length;b--;)this.items[b].insertAfter(a);return this},nc.getBBox=function(){for(var a=[],b=[],c=[],d=[],e=this.items.length;e--;)if(!this.items[e].removed){var f=this.items[e].getBBox();a.push(f.x),b.push(f.y),c.push(f.x+f.width),d.push(f.y+f.height)}return a=P[D](0,a),b=P[D](0,b),c=O[D](0,c),d=O[D](0,d),{x:a,y:b,x2:c,y2:d,width:c-a,height:d-b}},nc.clone=function(a){a=this.paper.set();for(var b=0,c=this.items.length;c>b;b++)a.push(this.items[b].clone());return a},nc.toString=function(){return"Raphaël‘s set"},nc.glow=function(a){var b=this.paper.set();return this.forEach(function(c){var d=c.glow(a);null!=d&&d.forEach(function(a){b.push(a)})}),b},nc.isPointInside=function(a,b){var c=!1;return this.forEach(function(d){return d.isPointInside(a,b)?(console.log("runned"),c=!0,!1):void 0}),c},c.registerFont=function(a){if(!a.face)return a;this.fonts=this.fonts||{};var b={w:a.w,face:{},glyphs:{}},c=a.face["font-family"];for(var d in a.face)a.face[z](d)&&(b.face[d]=a.face[d]);if(this.fonts[c]?this.fonts[c].push(b):this.fonts[c]=[b],!a.svg){b.face["units-per-em"]=ab(a.face["units-per-em"],10);for(var e in a.glyphs)if(a.glyphs[z](e)){var f=a.glyphs[e];if(b.glyphs[e]={w:f.w,k:{},d:f.d&&"M"+f.d.replace(/[mlcxtrv]/g,function(a){return{l:"L",c:"C",x:"z",t:"m",r:"l",v:"c"}[a]||"M"})+"z"},f.k)for(var g in f.k)f[z](g)&&(b.glyphs[e].k[g]=f.k[g])}}return a},v.getFont=function(a,b,d,e){if(e=e||"normal",d=d||"normal",b=+b||{normal:400,bold:700,lighter:300,bolder:800}[b]||400,c.fonts){var f=c.fonts[a];if(!f){var g=new RegExp("(^|\\s)"+a.replace(/[^\w\d\s+!~.:_-]/g,G)+"(\\s|$)","i");for(var h in c.fonts)if(c.fonts[z](h)&&g.test(h)){f=c.fonts[h];break}}var i;if(f)for(var j=0,k=f.length;k>j&&(i=f[j],i.face["font-weight"]!=b||i.face["font-style"]!=d&&i.face["font-style"]||i.face["font-stretch"]!=e);j++);return i}},v.print=function(a,b,d,e,f,g,h,i){g=g||"middle",h=O(P(h||0,1),-1),i=O(P(i||1,3),1);var j,k=I(d)[J](G),l=0,m=0,n=G;if(c.is(e,"string")&&(e=this.getFont(e)),e){j=(f||16)/e.face["units-per-em"];for(var o=e.face.bbox[J](w),p=+o[0],q=o[3]-o[1],r=0,s=+o[1]+("baseline"==g?q+ +e.face.descent:q/2),t=0,u=k.length;u>t;t++){if("\n"==k[t])l=0,x=0,m=0,r+=q*i;else{var v=m&&e.glyphs[k[t-1]]||{},x=e.glyphs[k[t]];l+=m?(v.w||e.w)+(v.k&&v.k[k[t]]||0)+e.w*h:0,m=1}x&&x.d&&(n+=c.transformPath(x.d,["t",l*j,r*j,"s",j,j,p,s,"t",(a-p)/j,(b-s)/j]))}}return this.path(n).attr({fill:"#000",stroke:"none"})},v.add=function(a){if(c.is(a,"array"))for(var b,d=this.set(),e=0,f=a.length;f>e;e++)b=a[e]||{},x[z](b.type)&&d.push(this[b.type]().attr(b));return d},c.format=function(a,b){var d=c.is(b,V)?[0][E](b):arguments;return a&&c.is(a,U)&&d.length-1&&(a=a.replace(y,function(a,b){return null==d[++b]?G:d[b]})),a||G},c.fullfill=function(){var a=/\{([^\}]+)\}/g,b=/(?:(?:^|\.)(.+?)(?=\[|\.|$|\()|\[('|")(.+?)\2\])(\(\))?/g,c=function(a,c,d){var e=d;return c.replace(b,function(a,b,c,d,f){b=b||d,e&&(b in e&&(e=e[b]),"function"==typeof e&&f&&(e=e()))}),e=(null==e||e==d?a:e)+""};return function(b,d){return String(b).replace(a,function(a,b){return c(a,b,d)})}}(),c.ninja=function(){return B.was?A.win.Raphael=B.is:delete Raphael,c},c.st=nc,function(a,b,d){function e(){/in/.test(a.readyState)?setTimeout(e,9):c.eve("raphael.DOMload")}null==a.readyState&&a.addEventListener&&(a.addEventListener(b,d=function(){a.removeEventListener(b,d,!1),a.readyState="complete"},!1),a.readyState="loading"),e()}(document,"DOMContentLoaded"),b.on("raphael.DOMload",function(){u=!0}),function(){if(c.svg){var a="hasOwnProperty",b=String,d=parseFloat,e=parseInt,f=Math,g=f.max,h=f.abs,i=f.pow,j=/[, ]+/,k=c.eve,l="",m=" ",n="http://www.w3.org/1999/xlink",o={block:"M5,0 0,2.5 5,5z",classic:"M5,0 0,2.5 5,5 3.5,3 3.5,2z",diamond:"M2.5,0 5,2.5 2.5,5 0,2.5z",open:"M6,1 1,3.5 6,6",oval:"M2.5,0A2.5,2.5,0,0,1,2.5,5 2.5,2.5,0,0,1,2.5,0z"},p={};c.toString=function(){return"Your browser supports SVG.\nYou are running Raphaël "+this.version};var q=function(d,e){if(e){"string"==typeof d&&(d=q(d));for(var f in e)e[a](f)&&("xlink:"==f.substring(0,6)?d.setAttributeNS(n,f.substring(6),b(e[f])):d.setAttribute(f,b(e[f])))}else d=c._g.doc.createElementNS("http://www.w3.org/2000/svg",d),d.style&&(d.style.webkitTapHighlightColor="rgba(0,0,0,0)");return d},r=function(a,e){var j="linear",k=a.id+e,m=.5,n=.5,o=a.node,p=a.paper,r=o.style,s=c._g.doc.getElementById(k);if(!s){if(e=b(e).replace(c._radial_gradient,function(a,b,c){if(j="radial",b&&c){m=d(b),n=d(c);var e=2*(n>.5)-1;i(m-.5,2)+i(n-.5,2)>.25&&(n=f.sqrt(.25-i(m-.5,2))*e+.5)&&.5!=n&&(n=n.toFixed(5)-1e-5*e)}return l}),e=e.split(/\s*\-\s*/),"linear"==j){var t=e.shift();if(t=-d(t),isNaN(t))return null;var u=[0,0,f.cos(c.rad(t)),f.sin(c.rad(t))],v=1/(g(h(u[2]),h(u[3]))||1);u[2]*=v,u[3]*=v,u[2]<0&&(u[0]=-u[2],u[2]=0),u[3]<0&&(u[1]=-u[3],u[3]=0)}var w=c._parseDots(e);if(!w)return null;if(k=k.replace(/[\(\)\s,\xb0#]/g,"_"),a.gradient&&k!=a.gradient.id&&(p.defs.removeChild(a.gradient),delete a.gradient),!a.gradient){s=q(j+"Gradient",{id:k}),a.gradient=s,q(s,"radial"==j?{fx:m,fy:n}:{x1:u[0],y1:u[1],x2:u[2],y2:u[3],gradientTransform:a.matrix.invert()}),p.defs.appendChild(s);for(var x=0,y=w.length;y>x;x++)s.appendChild(q("stop",{offset:w[x].offset?w[x].offset:x?"100%":"0%","stop-color":w[x].color||"#fff"}))}}return q(o,{fill:"url(#"+k+")",opacity:1,"fill-opacity":1}),r.fill=l,r.opacity=1,r.fillOpacity=1,1},s=function(a){var b=a.getBBox(1);q(a.pattern,{patternTransform:a.matrix.invert()+" translate("+b.x+","+b.y+")"})},t=function(d,e,f){if("path"==d.type){for(var g,h,i,j,k,m=b(e).toLowerCase().split("-"),n=d.paper,r=f?"end":"start",s=d.node,t=d.attrs,u=t["stroke-width"],v=m.length,w="classic",x=3,y=3,z=5;v--;)switch(m[v]){case"block":case"classic":case"oval":case"diamond":case"open":case"none":w=m[v];break;case"wide":y=5;break;case"narrow":y=2;break;case"long":x=5;break;case"short":x=2}if("open"==w?(x+=2,y+=2,z+=2,i=1,j=f?4:1,k={fill:"none",stroke:t.stroke}):(j=i=x/2,k={fill:t.stroke,stroke:"none"}),d._.arrows?f?(d._.arrows.endPath&&p[d._.arrows.endPath]--,d._.arrows.endMarker&&p[d._.arrows.endMarker]--):(d._.arrows.startPath&&p[d._.arrows.startPath]--,d._.arrows.startMarker&&p[d._.arrows.startMarker]--):d._.arrows={},"none"!=w){var A="raphael-marker-"+w,B="raphael-marker-"+r+w+x+y;c._g.doc.getElementById(A)?p[A]++:(n.defs.appendChild(q(q("path"),{"stroke-linecap":"round",d:o[w],id:A})),p[A]=1);var C,D=c._g.doc.getElementById(B);D?(p[B]++,C=D.getElementsByTagName("use")[0]):(D=q(q("marker"),{id:B,markerHeight:y,markerWidth:x,orient:"auto",refX:j,refY:y/2}),C=q(q("use"),{"xlink:href":"#"+A,transform:(f?"rotate(180 "+x/2+" "+y/2+") ":l)+"scale("+x/z+","+y/z+")","stroke-width":(1/((x/z+y/z)/2)).toFixed(4)}),D.appendChild(C),n.defs.appendChild(D),p[B]=1),q(C,k);var E=i*("diamond"!=w&&"oval"!=w);f?(g=d._.arrows.startdx*u||0,h=c.getTotalLength(t.path)-E*u):(g=E*u,h=c.getTotalLength(t.path)-(d._.arrows.enddx*u||0)),k={},k["marker-"+r]="url(#"+B+")",(h||g)&&(k.d=c.getSubpath(t.path,g,h)),q(s,k),d._.arrows[r+"Path"]=A,d._.arrows[r+"Marker"]=B,d._.arrows[r+"dx"]=E,d._.arrows[r+"Type"]=w,d._.arrows[r+"String"]=e}else f?(g=d._.arrows.startdx*u||0,h=c.getTotalLength(t.path)-g):(g=0,h=c.getTotalLength(t.path)-(d._.arrows.enddx*u||0)),d._.arrows[r+"Path"]&&q(s,{d:c.getSubpath(t.path,g,h)}),delete d._.arrows[r+"Path"],delete d._.arrows[r+"Marker"],delete d._.arrows[r+"dx"],delete d._.arrows[r+"Type"],delete d._.arrows[r+"String"];for(k in p)if(p[a](k)&&!p[k]){var F=c._g.doc.getElementById(k);F&&F.parentNode.removeChild(F)}}},u={"":[0],none:[0],"-":[3,1],".":[1,1],"-.":[3,1,1,1],"-..":[3,1,1,1,1,1],". ":[1,3],"- ":[4,3],"--":[8,3],"- .":[4,3,1,3],"--.":[8,3,1,3],"--..":[8,3,1,3,1,3]},v=function(a,c,d){if(c=u[b(c).toLowerCase()]){for(var e=a.attrs["stroke-width"]||"1",f={round:e,square:e,butt:0}[a.attrs["stroke-linecap"]||d["stroke-linecap"]]||0,g=[],h=c.length;h--;)g[h]=c[h]*e+(h%2?1:-1)*f;q(a.node,{"stroke-dasharray":g.join(",")})}},w=function(d,f){var i=d.node,k=d.attrs,m=i.style.visibility;i.style.visibility="hidden";for(var o in f)if(f[a](o)){if(!c._availableAttrs[a](o))continue;var p=f[o];switch(k[o]=p,o){case"blur":d.blur(p);break;case"href":case"title":var u=q("title"),w=c._g.doc.createTextNode(p);u.appendChild(w),i.appendChild(u);break;case"target":var x=i.parentNode;if("a"!=x.tagName.toLowerCase()){var u=q("a");x.insertBefore(u,i),u.appendChild(i),x=u}"target"==o?x.setAttributeNS(n,"show","blank"==p?"new":p):x.setAttributeNS(n,o,p);break;case"cursor":i.style.cursor=p;break;case"transform":d.transform(p);break;case"arrow-start":t(d,p);break;case"arrow-end":t(d,p,1);break;case"clip-rect":var z=b(p).split(j);if(4==z.length){d.clip&&d.clip.parentNode.parentNode.removeChild(d.clip.parentNode);var A=q("clipPath"),B=q("rect");A.id=c.createUUID(),q(B,{x:z[0],y:z[1],width:z[2],height:z[3]}),A.appendChild(B),d.paper.defs.appendChild(A),q(i,{"clip-path":"url(#"+A.id+")"}),d.clip=B}if(!p){var C=i.getAttribute("clip-path");if(C){var D=c._g.doc.getElementById(C.replace(/(^url\(#|\)$)/g,l));D&&D.parentNode.removeChild(D),q(i,{"clip-path":l}),delete d.clip}}break;case"path":"path"==d.type&&(q(i,{d:p?k.path=c._pathToAbsolute(p):"M0,0"}),d._.dirty=1,d._.arrows&&("startString"in d._.arrows&&t(d,d._.arrows.startString),"endString"in d._.arrows&&t(d,d._.arrows.endString,1)));break;case"width":if(i.setAttribute(o,p),d._.dirty=1,!k.fx)break;o="x",p=k.x;case"x":k.fx&&(p=-k.x-(k.width||0));case"rx":if("rx"==o&&"rect"==d.type)break;case"cx":i.setAttribute(o,p),d.pattern&&s(d),d._.dirty=1;break;case"height":if(i.setAttribute(o,p),d._.dirty=1,!k.fy)break;o="y",p=k.y;case"y":k.fy&&(p=-k.y-(k.height||0));case"ry":if("ry"==o&&"rect"==d.type)break;case"cy":i.setAttribute(o,p),d.pattern&&s(d),d._.dirty=1;break;case"r":"rect"==d.type?q(i,{rx:p,ry:p}):i.setAttribute(o,p),d._.dirty=1;break;case"src":"image"==d.type&&i.setAttributeNS(n,"href",p);break;case"stroke-width":(1!=d._.sx||1!=d._.sy)&&(p/=g(h(d._.sx),h(d._.sy))||1),d.paper._vbSize&&(p*=d.paper._vbSize),i.setAttribute(o,p),k["stroke-dasharray"]&&v(d,k["stroke-dasharray"],f),d._.arrows&&("startString"in d._.arrows&&t(d,d._.arrows.startString),"endString"in d._.arrows&&t(d,d._.arrows.endString,1));break;case"stroke-dasharray":v(d,p,f);break;case"fill":var E=b(p).match(c._ISURL);if(E){A=q("pattern");var F=q("image");A.id=c.createUUID(),q(A,{x:0,y:0,patternUnits:"userSpaceOnUse",height:1,width:1}),q(F,{x:0,y:0,"xlink:href":E[1]}),A.appendChild(F),function(a){c._preload(E[1],function(){var b=this.offsetWidth,c=this.offsetHeight;q(a,{width:b,height:c}),q(F,{width:b,height:c}),d.paper.safari()})}(A),d.paper.defs.appendChild(A),q(i,{fill:"url(#"+A.id+")"}),d.pattern=A,d.pattern&&s(d);break}var G=c.getRGB(p);if(G.error){if(("circle"==d.type||"ellipse"==d.type||"r"!=b(p).charAt())&&r(d,p)){if("opacity"in k||"fill-opacity"in k){var H=c._g.doc.getElementById(i.getAttribute("fill").replace(/^url\(#|\)$/g,l));if(H){var I=H.getElementsByTagName("stop");q(I[I.length-1],{"stop-opacity":("opacity"in k?k.opacity:1)*("fill-opacity"in k?k["fill-opacity"]:1)})}}k.gradient=p,k.fill="none";break}}else delete f.gradient,delete k.gradient,!c.is(k.opacity,"undefined")&&c.is(f.opacity,"undefined")&&q(i,{opacity:k.opacity}),!c.is(k["fill-opacity"],"undefined")&&c.is(f["fill-opacity"],"undefined")&&q(i,{"fill-opacity":k["fill-opacity"]});G[a]("opacity")&&q(i,{"fill-opacity":G.opacity>1?G.opacity/100:G.opacity});case"stroke":G=c.getRGB(p),i.setAttribute(o,G.hex),"stroke"==o&&G[a]("opacity")&&q(i,{"stroke-opacity":G.opacity>1?G.opacity/100:G.opacity}),"stroke"==o&&d._.arrows&&("startString"in d._.arrows&&t(d,d._.arrows.startString),"endString"in d._.arrows&&t(d,d._.arrows.endString,1));break;case"gradient":("circle"==d.type||"ellipse"==d.type||"r"!=b(p).charAt())&&r(d,p);break;case"opacity":k.gradient&&!k[a]("stroke-opacity")&&q(i,{"stroke-opacity":p>1?p/100:p});case"fill-opacity":if(k.gradient){H=c._g.doc.getElementById(i.getAttribute("fill").replace(/^url\(#|\)$/g,l)),H&&(I=H.getElementsByTagName("stop"),q(I[I.length-1],{"stop-opacity":p}));break}default:"font-size"==o&&(p=e(p,10)+"px");var J=o.replace(/(\-.)/g,function(a){return a.substring(1).toUpperCase()});i.style[J]=p,d._.dirty=1,i.setAttribute(o,p)}}y(d,f),i.style.visibility=m},x=1.2,y=function(d,f){if("text"==d.type&&(f[a]("text")||f[a]("font")||f[a]("font-size")||f[a]("x")||f[a]("y"))){var g=d.attrs,h=d.node,i=h.firstChild?e(c._g.doc.defaultView.getComputedStyle(h.firstChild,l).getPropertyValue("font-size"),10):10;
+if(f[a]("text")){for(g.text=f.text;h.firstChild;)h.removeChild(h.firstChild);for(var j,k=b(f.text).split("\n"),m=[],n=0,o=k.length;o>n;n++)j=q("tspan"),n&&q(j,{dy:i*x,x:g.x}),j.appendChild(c._g.doc.createTextNode(k[n])),h.appendChild(j),m[n]=j}else for(m=h.getElementsByTagName("tspan"),n=0,o=m.length;o>n;n++)n?q(m[n],{dy:i*x,x:g.x}):q(m[0],{dy:0});q(h,{x:g.x,y:g.y}),d._.dirty=1;var p=d._getBBox(),r=g.y-(p.y+p.height/2);r&&c.is(r,"finite")&&q(m[0],{dy:r})}},z=function(a,b){this[0]=this.node=a,a.raphael=!0,this.id=c._oid++,a.raphaelid=this.id,this.matrix=c.matrix(),this.realPath=null,this.paper=b,this.attrs=this.attrs||{},this._={transform:[],sx:1,sy:1,deg:0,dx:0,dy:0,dirty:1},!b.bottom&&(b.bottom=this),this.prev=b.top,b.top&&(b.top.next=this),b.top=this,this.next=null},A=c.el;z.prototype=A,A.constructor=z,c._engine.path=function(a,b){var c=q("path");b.canvas&&b.canvas.appendChild(c);var d=new z(c,b);return d.type="path",w(d,{fill:"none",stroke:"#000",path:a}),d},A.rotate=function(a,c,e){if(this.removed)return this;if(a=b(a).split(j),a.length-1&&(c=d(a[1]),e=d(a[2])),a=d(a[0]),null==e&&(c=e),null==c||null==e){var f=this.getBBox(1);c=f.x+f.width/2,e=f.y+f.height/2}return this.transform(this._.transform.concat([["r",a,c,e]])),this},A.scale=function(a,c,e,f){if(this.removed)return this;if(a=b(a).split(j),a.length-1&&(c=d(a[1]),e=d(a[2]),f=d(a[3])),a=d(a[0]),null==c&&(c=a),null==f&&(e=f),null==e||null==f)var g=this.getBBox(1);return e=null==e?g.x+g.width/2:e,f=null==f?g.y+g.height/2:f,this.transform(this._.transform.concat([["s",a,c,e,f]])),this},A.translate=function(a,c){return this.removed?this:(a=b(a).split(j),a.length-1&&(c=d(a[1])),a=d(a[0])||0,c=+c||0,this.transform(this._.transform.concat([["t",a,c]])),this)},A.transform=function(b){var d=this._;if(null==b)return d.transform;if(c._extractTransform(this,b),this.clip&&q(this.clip,{transform:this.matrix.invert()}),this.pattern&&s(this),this.node&&q(this.node,{transform:this.matrix}),1!=d.sx||1!=d.sy){var e=this.attrs[a]("stroke-width")?this.attrs["stroke-width"]:1;this.attr({"stroke-width":e})}return this},A.hide=function(){return!this.removed&&this.paper.safari(this.node.style.display="none"),this},A.show=function(){return!this.removed&&this.paper.safari(this.node.style.display=""),this},A.remove=function(){if(!this.removed&&this.node.parentNode){var a=this.paper;a.__set__&&a.__set__.exclude(this),k.unbind("raphael.*.*."+this.id),this.gradient&&a.defs.removeChild(this.gradient),c._tear(this,a),"a"==this.node.parentNode.tagName.toLowerCase()?this.node.parentNode.parentNode.removeChild(this.node.parentNode):this.node.parentNode.removeChild(this.node);for(var b in this)this[b]="function"==typeof this[b]?c._removedFactory(b):null;this.removed=!0}},A._getBBox=function(){if("none"==this.node.style.display){this.show();var a=!0}var b={};try{b=this.node.getBBox()}catch(c){}finally{b=b||{}}return a&&this.hide(),b},A.attr=function(b,d){if(this.removed)return this;if(null==b){var e={};for(var f in this.attrs)this.attrs[a](f)&&(e[f]=this.attrs[f]);return e.gradient&&"none"==e.fill&&(e.fill=e.gradient)&&delete e.gradient,e.transform=this._.transform,e}if(null==d&&c.is(b,"string")){if("fill"==b&&"none"==this.attrs.fill&&this.attrs.gradient)return this.attrs.gradient;if("transform"==b)return this._.transform;for(var g=b.split(j),h={},i=0,l=g.length;l>i;i++)b=g[i],h[b]=b in this.attrs?this.attrs[b]:c.is(this.paper.customAttributes[b],"function")?this.paper.customAttributes[b].def:c._availableAttrs[b];return l-1?h:h[g[0]]}if(null==d&&c.is(b,"array")){for(h={},i=0,l=b.length;l>i;i++)h[b[i]]=this.attr(b[i]);return h}if(null!=d){var m={};m[b]=d}else null!=b&&c.is(b,"object")&&(m=b);for(var n in m)k("raphael.attr."+n+"."+this.id,this,m[n]);for(n in this.paper.customAttributes)if(this.paper.customAttributes[a](n)&&m[a](n)&&c.is(this.paper.customAttributes[n],"function")){var o=this.paper.customAttributes[n].apply(this,[].concat(m[n]));this.attrs[n]=m[n];for(var p in o)o[a](p)&&(m[p]=o[p])}return w(this,m),this},A.toFront=function(){if(this.removed)return this;"a"==this.node.parentNode.tagName.toLowerCase()?this.node.parentNode.parentNode.appendChild(this.node.parentNode):this.node.parentNode.appendChild(this.node);var a=this.paper;return a.top!=this&&c._tofront(this,a),this},A.toBack=function(){if(this.removed)return this;var a=this.node.parentNode;return"a"==a.tagName.toLowerCase()?a.parentNode.insertBefore(this.node.parentNode,this.node.parentNode.parentNode.firstChild):a.firstChild!=this.node&&a.insertBefore(this.node,this.node.parentNode.firstChild),c._toback(this,this.paper),this.paper,this},A.insertAfter=function(a){if(this.removed)return this;var b=a.node||a[a.length-1].node;return b.nextSibling?b.parentNode.insertBefore(this.node,b.nextSibling):b.parentNode.appendChild(this.node),c._insertafter(this,a,this.paper),this},A.insertBefore=function(a){if(this.removed)return this;var b=a.node||a[0].node;return b.parentNode.insertBefore(this.node,b),c._insertbefore(this,a,this.paper),this},A.blur=function(a){var b=this;if(0!==+a){var d=q("filter"),e=q("feGaussianBlur");b.attrs.blur=a,d.id=c.createUUID(),q(e,{stdDeviation:+a||1.5}),d.appendChild(e),b.paper.defs.appendChild(d),b._blur=d,q(b.node,{filter:"url(#"+d.id+")"})}else b._blur&&(b._blur.parentNode.removeChild(b._blur),delete b._blur,delete b.attrs.blur),b.node.removeAttribute("filter");return b},c._engine.circle=function(a,b,c,d){var e=q("circle");a.canvas&&a.canvas.appendChild(e);var f=new z(e,a);return f.attrs={cx:b,cy:c,r:d,fill:"none",stroke:"#000"},f.type="circle",q(e,f.attrs),f},c._engine.rect=function(a,b,c,d,e,f){var g=q("rect");a.canvas&&a.canvas.appendChild(g);var h=new z(g,a);return h.attrs={x:b,y:c,width:d,height:e,r:f||0,rx:f||0,ry:f||0,fill:"none",stroke:"#000"},h.type="rect",q(g,h.attrs),h},c._engine.ellipse=function(a,b,c,d,e){var f=q("ellipse");a.canvas&&a.canvas.appendChild(f);var g=new z(f,a);return g.attrs={cx:b,cy:c,rx:d,ry:e,fill:"none",stroke:"#000"},g.type="ellipse",q(f,g.attrs),g},c._engine.image=function(a,b,c,d,e,f){var g=q("image");q(g,{x:c,y:d,width:e,height:f,preserveAspectRatio:"none"}),g.setAttributeNS(n,"href",b),a.canvas&&a.canvas.appendChild(g);var h=new z(g,a);return h.attrs={x:c,y:d,width:e,height:f,src:b},h.type="image",h},c._engine.text=function(a,b,d,e){var f=q("text");a.canvas&&a.canvas.appendChild(f);var g=new z(f,a);return g.attrs={x:b,y:d,"text-anchor":"middle",text:e,font:c._availableAttrs.font,stroke:"none",fill:"#000"},g.type="text",w(g,g.attrs),g},c._engine.setSize=function(a,b){return this.width=a||this.width,this.height=b||this.height,this.canvas.setAttribute("width",this.width),this.canvas.setAttribute("height",this.height),this._viewBox&&this.setViewBox.apply(this,this._viewBox),this},c._engine.create=function(){var a=c._getContainer.apply(0,arguments),b=a&&a.container,d=a.x,e=a.y,f=a.width,g=a.height;if(!b)throw new Error("SVG container not found.");var h,i=q("svg"),j="overflow:hidden;";return d=d||0,e=e||0,f=f||512,g=g||342,q(i,{height:g,version:1.1,width:f,xmlns:"http://www.w3.org/2000/svg"}),1==b?(i.style.cssText=j+"position:absolute;left:"+d+"px;top:"+e+"px",c._g.doc.body.appendChild(i),h=1):(i.style.cssText=j+"position:relative",b.firstChild?b.insertBefore(i,b.firstChild):b.appendChild(i)),b=new c._Paper,b.width=f,b.height=g,b.canvas=i,b.clear(),b._left=b._top=0,h&&(b.renderfix=function(){}),b.renderfix(),b},c._engine.setViewBox=function(a,b,c,d,e){k("raphael.setViewBox",this,this._viewBox,[a,b,c,d,e]);var f,h,i=g(c/this.width,d/this.height),j=this.top,l=e?"meet":"xMinYMin";for(null==a?(this._vbSize&&(i=1),delete this._vbSize,f="0 0 "+this.width+m+this.height):(this._vbSize=i,f=a+m+b+m+c+m+d),q(this.canvas,{viewBox:f,preserveAspectRatio:l});i&&j;)h="stroke-width"in j.attrs?j.attrs["stroke-width"]:1,j.attr({"stroke-width":h}),j._.dirty=1,j._.dirtyT=1,j=j.prev;return this._viewBox=[a,b,c,d,!!e],this},c.prototype.renderfix=function(){var a,b=this.canvas,c=b.style;try{a=b.getScreenCTM()||b.createSVGMatrix()}catch(d){a=b.createSVGMatrix()}var e=-a.e%1,f=-a.f%1;(e||f)&&(e&&(this._left=(this._left+e)%1,c.left=this._left+"px"),f&&(this._top=(this._top+f)%1,c.top=this._top+"px"))},c.prototype.clear=function(){c.eve("raphael.clear",this);for(var a=this.canvas;a.firstChild;)a.removeChild(a.firstChild);this.bottom=this.top=null,(this.desc=q("desc")).appendChild(c._g.doc.createTextNode("Created with Raphaël "+c.version)),a.appendChild(this.desc),a.appendChild(this.defs=q("defs"))},c.prototype.remove=function(){k("raphael.remove",this),this.canvas.parentNode&&this.canvas.parentNode.removeChild(this.canvas);for(var a in this)this[a]="function"==typeof this[a]?c._removedFactory(a):null};var B=c.st;for(var C in A)A[a](C)&&!B[a](C)&&(B[C]=function(a){return function(){var b=arguments;return this.forEach(function(c){c[a].apply(c,b)})}}(C))}}(),function(){if(c.vml){var a="hasOwnProperty",b=String,d=parseFloat,e=Math,f=e.round,g=e.max,h=e.min,i=e.abs,j="fill",k=/[, ]+/,l=c.eve,m=" progid:DXImageTransform.Microsoft",n=" ",o="",p={M:"m",L:"l",C:"c",Z:"x",m:"t",l:"r",c:"v",z:"x"},q=/([clmz]),?([^clmz]*)/gi,r=/ progid:\S+Blur\([^\)]+\)/g,s=/-?[^,\s-]+/g,t="position:absolute;left:0;top:0;width:1px;height:1px",u=21600,v={path:1,rect:1,image:1},w={circle:1,ellipse:1},x=function(a){var d=/[ahqstv]/gi,e=c._pathToAbsolute;if(b(a).match(d)&&(e=c._path2curve),d=/[clmz]/g,e==c._pathToAbsolute&&!b(a).match(d)){var g=b(a).replace(q,function(a,b,c){var d=[],e="m"==b.toLowerCase(),g=p[b];return c.replace(s,function(a){e&&2==d.length&&(g+=d+p["m"==b?"l":"L"],d=[]),d.push(f(a*u))}),g+d});return g}var h,i,j=e(a);g=[];for(var k=0,l=j.length;l>k;k++){h=j[k],i=j[k][0].toLowerCase(),"z"==i&&(i="x");for(var m=1,r=h.length;r>m;m++)i+=f(h[m]*u)+(m!=r-1?",":o);g.push(i)}return g.join(n)},y=function(a,b,d){var e=c.matrix();return e.rotate(-a,.5,.5),{dx:e.x(b,d),dy:e.y(b,d)}},z=function(a,b,c,d,e,f){var g=a._,h=a.matrix,k=g.fillpos,l=a.node,m=l.style,o=1,p="",q=u/b,r=u/c;if(m.visibility="hidden",b&&c){if(l.coordsize=i(q)+n+i(r),m.rotation=f*(0>b*c?-1:1),f){var s=y(f,d,e);d=s.dx,e=s.dy}if(0>b&&(p+="x"),0>c&&(p+=" y")&&(o=-1),m.flip=p,l.coordorigin=d*-q+n+e*-r,k||g.fillsize){var t=l.getElementsByTagName(j);t=t&&t[0],l.removeChild(t),k&&(s=y(f,h.x(k[0],k[1]),h.y(k[0],k[1])),t.position=s.dx*o+n+s.dy*o),g.fillsize&&(t.size=g.fillsize[0]*i(b)+n+g.fillsize[1]*i(c)),l.appendChild(t)}m.visibility="visible"}};c.toString=function(){return"Your browser doesn’t support SVG. Falling down to VML.\nYou are running Raphaël "+this.version};var A=function(a,c,d){for(var e=b(c).toLowerCase().split("-"),f=d?"end":"start",g=e.length,h="classic",i="medium",j="medium";g--;)switch(e[g]){case"block":case"classic":case"oval":case"diamond":case"open":case"none":h=e[g];break;case"wide":case"narrow":j=e[g];break;case"long":case"short":i=e[g]}var k=a.node.getElementsByTagName("stroke")[0];k[f+"arrow"]=h,k[f+"arrowlength"]=i,k[f+"arrowwidth"]=j},B=function(e,i){e.attrs=e.attrs||{};var l=e.node,m=e.attrs,p=l.style,q=v[e.type]&&(i.x!=m.x||i.y!=m.y||i.width!=m.width||i.height!=m.height||i.cx!=m.cx||i.cy!=m.cy||i.rx!=m.rx||i.ry!=m.ry||i.r!=m.r),r=w[e.type]&&(m.cx!=i.cx||m.cy!=i.cy||m.r!=i.r||m.rx!=i.rx||m.ry!=i.ry),s=e;for(var t in i)i[a](t)&&(m[t]=i[t]);if(q&&(m.path=c._getPath[e.type](e),e._.dirty=1),i.href&&(l.href=i.href),i.title&&(l.title=i.title),i.target&&(l.target=i.target),i.cursor&&(p.cursor=i.cursor),"blur"in i&&e.blur(i.blur),(i.path&&"path"==e.type||q)&&(l.path=x(~b(m.path).toLowerCase().indexOf("r")?c._pathToAbsolute(m.path):m.path),"image"==e.type&&(e._.fillpos=[m.x,m.y],e._.fillsize=[m.width,m.height],z(e,1,1,0,0,0))),"transform"in i&&e.transform(i.transform),r){var y=+m.cx,B=+m.cy,D=+m.rx||+m.r||0,E=+m.ry||+m.r||0;l.path=c.format("ar{0},{1},{2},{3},{4},{1},{4},{1}x",f((y-D)*u),f((B-E)*u),f((y+D)*u),f((B+E)*u),f(y*u)),e._.dirty=1}if("clip-rect"in i){var G=b(i["clip-rect"]).split(k);if(4==G.length){G[2]=+G[2]+ +G[0],G[3]=+G[3]+ +G[1];var H=l.clipRect||c._g.doc.createElement("div"),I=H.style;I.clip=c.format("rect({1}px {2}px {3}px {0}px)",G),l.clipRect||(I.position="absolute",I.top=0,I.left=0,I.width=e.paper.width+"px",I.height=e.paper.height+"px",l.parentNode.insertBefore(H,l),H.appendChild(l),l.clipRect=H)}i["clip-rect"]||l.clipRect&&(l.clipRect.style.clip="auto")}if(e.textpath){var J=e.textpath.style;i.font&&(J.font=i.font),i["font-family"]&&(J.fontFamily='"'+i["font-family"].split(",")[0].replace(/^['"]+|['"]+$/g,o)+'"'),i["font-size"]&&(J.fontSize=i["font-size"]),i["font-weight"]&&(J.fontWeight=i["font-weight"]),i["font-style"]&&(J.fontStyle=i["font-style"])}if("arrow-start"in i&&A(s,i["arrow-start"]),"arrow-end"in i&&A(s,i["arrow-end"],1),null!=i.opacity||null!=i["stroke-width"]||null!=i.fill||null!=i.src||null!=i.stroke||null!=i["stroke-width"]||null!=i["stroke-opacity"]||null!=i["fill-opacity"]||null!=i["stroke-dasharray"]||null!=i["stroke-miterlimit"]||null!=i["stroke-linejoin"]||null!=i["stroke-linecap"]){var K=l.getElementsByTagName(j),L=!1;if(K=K&&K[0],!K&&(L=K=F(j)),"image"==e.type&&i.src&&(K.src=i.src),i.fill&&(K.on=!0),(null==K.on||"none"==i.fill||null===i.fill)&&(K.on=!1),K.on&&i.fill){var M=b(i.fill).match(c._ISURL);if(M){K.parentNode==l&&l.removeChild(K),K.rotate=!0,K.src=M[1],K.type="tile";var N=e.getBBox(1);K.position=N.x+n+N.y,e._.fillpos=[N.x,N.y],c._preload(M[1],function(){e._.fillsize=[this.offsetWidth,this.offsetHeight]})}else K.color=c.getRGB(i.fill).hex,K.src=o,K.type="solid",c.getRGB(i.fill).error&&(s.type in{circle:1,ellipse:1}||"r"!=b(i.fill).charAt())&&C(s,i.fill,K)&&(m.fill="none",m.gradient=i.fill,K.rotate=!1)}if("fill-opacity"in i||"opacity"in i){var O=((+m["fill-opacity"]+1||2)-1)*((+m.opacity+1||2)-1)*((+c.getRGB(i.fill).o+1||2)-1);O=h(g(O,0),1),K.opacity=O,K.src&&(K.color="none")}l.appendChild(K);var P=l.getElementsByTagName("stroke")&&l.getElementsByTagName("stroke")[0],Q=!1;!P&&(Q=P=F("stroke")),(i.stroke&&"none"!=i.stroke||i["stroke-width"]||null!=i["stroke-opacity"]||i["stroke-dasharray"]||i["stroke-miterlimit"]||i["stroke-linejoin"]||i["stroke-linecap"])&&(P.on=!0),("none"==i.stroke||null===i.stroke||null==P.on||0==i.stroke||0==i["stroke-width"])&&(P.on=!1);var R=c.getRGB(i.stroke);P.on&&i.stroke&&(P.color=R.hex),O=((+m["stroke-opacity"]+1||2)-1)*((+m.opacity+1||2)-1)*((+R.o+1||2)-1);var S=.75*(d(i["stroke-width"])||1);if(O=h(g(O,0),1),null==i["stroke-width"]&&(S=m["stroke-width"]),i["stroke-width"]&&(P.weight=S),S&&1>S&&(O*=S)&&(P.weight=1),P.opacity=O,i["stroke-linejoin"]&&(P.joinstyle=i["stroke-linejoin"]||"miter"),P.miterlimit=i["stroke-miterlimit"]||8,i["stroke-linecap"]&&(P.endcap="butt"==i["stroke-linecap"]?"flat":"square"==i["stroke-linecap"]?"square":"round"),i["stroke-dasharray"]){var T={"-":"shortdash",".":"shortdot","-.":"shortdashdot","-..":"shortdashdotdot",". ":"dot","- ":"dash","--":"longdash","- .":"dashdot","--.":"longdashdot","--..":"longdashdotdot"};P.dashstyle=T[a](i["stroke-dasharray"])?T[i["stroke-dasharray"]]:o}Q&&l.appendChild(P)}if("text"==s.type){s.paper.canvas.style.display=o;var U=s.paper.span,V=100,W=m.font&&m.font.match(/\d+(?:\.\d*)?(?=px)/);p=U.style,m.font&&(p.font=m.font),m["font-family"]&&(p.fontFamily=m["font-family"]),m["font-weight"]&&(p.fontWeight=m["font-weight"]),m["font-style"]&&(p.fontStyle=m["font-style"]),W=d(m["font-size"]||W&&W[0])||10,p.fontSize=W*V+"px",s.textpath.string&&(U.innerHTML=b(s.textpath.string).replace(/</g,"&#60;").replace(/&/g,"&#38;").replace(/\n/g,"<br>"));var X=U.getBoundingClientRect();s.W=m.w=(X.right-X.left)/V,s.H=m.h=(X.bottom-X.top)/V,s.X=m.x,s.Y=m.y+s.H/2,("x"in i||"y"in i)&&(s.path.v=c.format("m{0},{1}l{2},{1}",f(m.x*u),f(m.y*u),f(m.x*u)+1));for(var Y=["x","y","text","font","font-family","font-weight","font-style","font-size"],Z=0,$=Y.length;$>Z;Z++)if(Y[Z]in i){s._.dirty=1;break}switch(m["text-anchor"]){case"start":s.textpath.style["v-text-align"]="left",s.bbx=s.W/2;break;case"end":s.textpath.style["v-text-align"]="right",s.bbx=-s.W/2;break;default:s.textpath.style["v-text-align"]="center",s.bbx=0}s.textpath.style["v-text-kern"]=!0}},C=function(a,f,g){a.attrs=a.attrs||{};var h=(a.attrs,Math.pow),i="linear",j=".5 .5";if(a.attrs.gradient=f,f=b(f).replace(c._radial_gradient,function(a,b,c){return i="radial",b&&c&&(b=d(b),c=d(c),h(b-.5,2)+h(c-.5,2)>.25&&(c=e.sqrt(.25-h(b-.5,2))*(2*(c>.5)-1)+.5),j=b+n+c),o}),f=f.split(/\s*\-\s*/),"linear"==i){var k=f.shift();if(k=-d(k),isNaN(k))return null}var l=c._parseDots(f);if(!l)return null;if(a=a.shape||a.node,l.length){a.removeChild(g),g.on=!0,g.method="none",g.color=l[0].color,g.color2=l[l.length-1].color;for(var m=[],p=0,q=l.length;q>p;p++)l[p].offset&&m.push(l[p].offset+n+l[p].color);g.colors=m.length?m.join():"0% "+g.color,"radial"==i?(g.type="gradientTitle",g.focus="100%",g.focussize="0 0",g.focusposition=j,g.angle=0):(g.type="gradient",g.angle=(270-k)%360),a.appendChild(g)}return 1},D=function(a,b){this[0]=this.node=a,a.raphael=!0,this.id=c._oid++,a.raphaelid=this.id,this.X=0,this.Y=0,this.attrs={},this.paper=b,this.matrix=c.matrix(),this._={transform:[],sx:1,sy:1,dx:0,dy:0,deg:0,dirty:1,dirtyT:1},!b.bottom&&(b.bottom=this),this.prev=b.top,b.top&&(b.top.next=this),b.top=this,this.next=null},E=c.el;D.prototype=E,E.constructor=D,E.transform=function(a){if(null==a)return this._.transform;var d,e=this.paper._viewBoxShift,f=e?"s"+[e.scale,e.scale]+"-1-1t"+[e.dx,e.dy]:o;e&&(d=a=b(a).replace(/\.{3}|\u2026/g,this._.transform||o)),c._extractTransform(this,f+a);var g,h=this.matrix.clone(),i=this.skew,j=this.node,k=~b(this.attrs.fill).indexOf("-"),l=!b(this.attrs.fill).indexOf("url(");if(h.translate(1,1),l||k||"image"==this.type)if(i.matrix="1 0 0 1",i.offset="0 0",g=h.split(),k&&g.noRotation||!g.isSimple){j.style.filter=h.toFilter();var m=this.getBBox(),p=this.getBBox(1),q=m.x-p.x,r=m.y-p.y;j.coordorigin=q*-u+n+r*-u,z(this,1,1,q,r,0)}else j.style.filter=o,z(this,g.scalex,g.scaley,g.dx,g.dy,g.rotate);else j.style.filter=o,i.matrix=b(h),i.offset=h.offset();return d&&(this._.transform=d),this},E.rotate=function(a,c,e){if(this.removed)return this;if(null!=a){if(a=b(a).split(k),a.length-1&&(c=d(a[1]),e=d(a[2])),a=d(a[0]),null==e&&(c=e),null==c||null==e){var f=this.getBBox(1);c=f.x+f.width/2,e=f.y+f.height/2}return this._.dirtyT=1,this.transform(this._.transform.concat([["r",a,c,e]])),this}},E.translate=function(a,c){return this.removed?this:(a=b(a).split(k),a.length-1&&(c=d(a[1])),a=d(a[0])||0,c=+c||0,this._.bbox&&(this._.bbox.x+=a,this._.bbox.y+=c),this.transform(this._.transform.concat([["t",a,c]])),this)},E.scale=function(a,c,e,f){if(this.removed)return this;if(a=b(a).split(k),a.length-1&&(c=d(a[1]),e=d(a[2]),f=d(a[3]),isNaN(e)&&(e=null),isNaN(f)&&(f=null)),a=d(a[0]),null==c&&(c=a),null==f&&(e=f),null==e||null==f)var g=this.getBBox(1);return e=null==e?g.x+g.width/2:e,f=null==f?g.y+g.height/2:f,this.transform(this._.transform.concat([["s",a,c,e,f]])),this._.dirtyT=1,this},E.hide=function(){return!this.removed&&(this.node.style.display="none"),this},E.show=function(){return!this.removed&&(this.node.style.display=o),this},E._getBBox=function(){return this.removed?{}:{x:this.X+(this.bbx||0)-this.W/2,y:this.Y-this.H,width:this.W,height:this.H}},E.remove=function(){if(!this.removed&&this.node.parentNode){this.paper.__set__&&this.paper.__set__.exclude(this),c.eve.unbind("raphael.*.*."+this.id),c._tear(this,this.paper),this.node.parentNode.removeChild(this.node),this.shape&&this.shape.parentNode.removeChild(this.shape);for(var a in this)this[a]="function"==typeof this[a]?c._removedFactory(a):null;this.removed=!0}},E.attr=function(b,d){if(this.removed)return this;if(null==b){var e={};for(var f in this.attrs)this.attrs[a](f)&&(e[f]=this.attrs[f]);return e.gradient&&"none"==e.fill&&(e.fill=e.gradient)&&delete e.gradient,e.transform=this._.transform,e}if(null==d&&c.is(b,"string")){if(b==j&&"none"==this.attrs.fill&&this.attrs.gradient)return this.attrs.gradient;for(var g=b.split(k),h={},i=0,m=g.length;m>i;i++)b=g[i],h[b]=b in this.attrs?this.attrs[b]:c.is(this.paper.customAttributes[b],"function")?this.paper.customAttributes[b].def:c._availableAttrs[b];return m-1?h:h[g[0]]}if(this.attrs&&null==d&&c.is(b,"array")){for(h={},i=0,m=b.length;m>i;i++)h[b[i]]=this.attr(b[i]);return h}var n;null!=d&&(n={},n[b]=d),null==d&&c.is(b,"object")&&(n=b);for(var o in n)l("raphael.attr."+o+"."+this.id,this,n[o]);if(n){for(o in this.paper.customAttributes)if(this.paper.customAttributes[a](o)&&n[a](o)&&c.is(this.paper.customAttributes[o],"function")){var p=this.paper.customAttributes[o].apply(this,[].concat(n[o]));this.attrs[o]=n[o];for(var q in p)p[a](q)&&(n[q]=p[q])}n.text&&"text"==this.type&&(this.textpath.string=n.text),B(this,n)}return this},E.toFront=function(){return!this.removed&&this.node.parentNode.appendChild(this.node),this.paper&&this.paper.top!=this&&c._tofront(this,this.paper),this},E.toBack=function(){return this.removed?this:(this.node.parentNode.firstChild!=this.node&&(this.node.parentNode.insertBefore(this.node,this.node.parentNode.firstChild),c._toback(this,this.paper)),this)},E.insertAfter=function(a){return this.removed?this:(a.constructor==c.st.constructor&&(a=a[a.length-1]),a.node.nextSibling?a.node.parentNode.insertBefore(this.node,a.node.nextSibling):a.node.parentNode.appendChild(this.node),c._insertafter(this,a,this.paper),this)},E.insertBefore=function(a){return this.removed?this:(a.constructor==c.st.constructor&&(a=a[0]),a.node.parentNode.insertBefore(this.node,a.node),c._insertbefore(this,a,this.paper),this)},E.blur=function(a){var b=this.node.runtimeStyle,d=b.filter;return d=d.replace(r,o),0!==+a?(this.attrs.blur=a,b.filter=d+n+m+".Blur(pixelradius="+(+a||1.5)+")",b.margin=c.format("-{0}px 0 0 -{0}px",f(+a||1.5))):(b.filter=d,b.margin=0,delete this.attrs.blur),this},c._engine.path=function(a,b){var c=F("shape");c.style.cssText=t,c.coordsize=u+n+u,c.coordorigin=b.coordorigin;var d=new D(c,b),e={fill:"none",stroke:"#000"};a&&(e.path=a),d.type="path",d.path=[],d.Path=o,B(d,e),b.canvas.appendChild(c);var f=F("skew");return f.on=!0,c.appendChild(f),d.skew=f,d.transform(o),d},c._engine.rect=function(a,b,d,e,f,g){var h=c._rectPath(b,d,e,f,g),i=a.path(h),j=i.attrs;return i.X=j.x=b,i.Y=j.y=d,i.W=j.width=e,i.H=j.height=f,j.r=g,j.path=h,i.type="rect",i},c._engine.ellipse=function(a,b,c,d,e){var f=a.path();return f.attrs,f.X=b-d,f.Y=c-e,f.W=2*d,f.H=2*e,f.type="ellipse",B(f,{cx:b,cy:c,rx:d,ry:e}),f},c._engine.circle=function(a,b,c,d){var e=a.path();return e.attrs,e.X=b-d,e.Y=c-d,e.W=e.H=2*d,e.type="circle",B(e,{cx:b,cy:c,r:d}),e},c._engine.image=function(a,b,d,e,f,g){var h=c._rectPath(d,e,f,g),i=a.path(h).attr({stroke:"none"}),k=i.attrs,l=i.node,m=l.getElementsByTagName(j)[0];return k.src=b,i.X=k.x=d,i.Y=k.y=e,i.W=k.width=f,i.H=k.height=g,k.path=h,i.type="image",m.parentNode==l&&l.removeChild(m),m.rotate=!0,m.src=b,m.type="tile",i._.fillpos=[d,e],i._.fillsize=[f,g],l.appendChild(m),z(i,1,1,0,0,0),i},c._engine.text=function(a,d,e,g){var h=F("shape"),i=F("path"),j=F("textpath");d=d||0,e=e||0,g=g||"",i.v=c.format("m{0},{1}l{2},{1}",f(d*u),f(e*u),f(d*u)+1),i.textpathok=!0,j.string=b(g),j.on=!0,h.style.cssText=t,h.coordsize=u+n+u,h.coordorigin="0 0";var k=new D(h,a),l={fill:"#000",stroke:"none",font:c._availableAttrs.font,text:g};k.shape=h,k.path=i,k.textpath=j,k.type="text",k.attrs.text=b(g),k.attrs.x=d,k.attrs.y=e,k.attrs.w=1,k.attrs.h=1,B(k,l),h.appendChild(j),h.appendChild(i),a.canvas.appendChild(h);var m=F("skew");return m.on=!0,h.appendChild(m),k.skew=m,k.transform(o),k},c._engine.setSize=function(a,b){var d=this.canvas.style;return this.width=a,this.height=b,a==+a&&(a+="px"),b==+b&&(b+="px"),d.width=a,d.height=b,d.clip="rect(0 "+a+" "+b+" 0)",this._viewBox&&c._engine.setViewBox.apply(this,this._viewBox),this},c._engine.setViewBox=function(a,b,d,e,f){c.eve("raphael.setViewBox",this,this._viewBox,[a,b,d,e,f]);var h,i,j=this.width,k=this.height,l=1/g(d/j,e/k);return f&&(h=k/e,i=j/d,j>d*h&&(a-=(j-d*h)/2/h),k>e*i&&(b-=(k-e*i)/2/i)),this._viewBox=[a,b,d,e,!!f],this._viewBoxShift={dx:-a,dy:-b,scale:l},this.forEach(function(a){a.transform("...")}),this};var F;c._engine.initWin=function(a){var b=a.document;b.createStyleSheet().addRule(".rvml","behavior:url(#default#VML)");try{!b.namespaces.rvml&&b.namespaces.add("rvml","urn:schemas-microsoft-com:vml"),F=function(a){return b.createElement("<rvml:"+a+' class="rvml">')}}catch(c){F=function(a){return b.createElement("<"+a+' xmlns="urn:schemas-microsoft.com:vml" class="rvml">')}}},c._engine.initWin(c._g.win),c._engine.create=function(){var a=c._getContainer.apply(0,arguments),b=a.container,d=a.height,e=a.width,f=a.x,g=a.y;if(!b)throw new Error("VML container not found.");var h=new c._Paper,i=h.canvas=c._g.doc.createElement("div"),j=i.style;return f=f||0,g=g||0,e=e||512,d=d||342,h.width=e,h.height=d,e==+e&&(e+="px"),d==+d&&(d+="px"),h.coordsize=1e3*u+n+1e3*u,h.coordorigin="0 0",h.span=c._g.doc.createElement("span"),h.span.style.cssText="position:absolute;left:-9999em;top:-9999em;padding:0;margin:0;line-height:1;",i.appendChild(h.span),j.cssText=c.format("top:0;left:0;width:{0};height:{1};display:inline-block;position:relative;clip:rect(0 {0} {1} 0);overflow:hidden",e,d),1==b?(c._g.doc.body.appendChild(i),j.left=f+"px",j.top=g+"px",j.position="absolute"):b.firstChild?b.insertBefore(i,b.firstChild):b.appendChild(i),h.renderfix=function(){},h},c.prototype.clear=function(){c.eve("raphael.clear",this),this.canvas.innerHTML=o,this.span=c._g.doc.createElement("span"),this.span.style.cssText="position:absolute;left:-9999em;top:-9999em;padding:0;margin:0;line-height:1;display:inline;",this.canvas.appendChild(this.span),this.bottom=this.top=null},c.prototype.remove=function(){c.eve("raphael.remove",this),this.canvas.parentNode.removeChild(this.canvas);for(var a in this)this[a]="function"==typeof this[a]?c._removedFactory(a):null;return!0};var G=c.st;for(var H in E)E[a](H)&&!G[a](H)&&(G[H]=function(a){return function(){var b=arguments;return this.forEach(function(c){c[a].apply(c,b)})}}(H))}}(),B.was?A.win.Raphael=c:Raphael=c,c});
\ No newline at end of file

unit/build.xml 64(+49 -15)

diff --git a/unit/build.xml b/unit/build.xml
index b35b866..f0fa718 100644
--- a/unit/build.xml
+++ b/unit/build.xml
@@ -7,9 +7,8 @@
 	<property name="dist.packages.dir" value="${base.dir}/dist/unit/packages" />
 	
 	<property name="java.src.dir" value="${base.dir}/unit/java" />
-	<property name="job.conf.dir" value="${base.dir}/unit/executions/exectest1" />
-	<property name="job.conf.dir2" value="${base.dir}/unit/executions/exectest2" />
-	
+	<property name="job.conf.dir" value="${base.dir}/unit/executions" />
+		
 	<property environment="env" />
 
 	<path id="main.classpath">
@@ -23,7 +22,7 @@
 	<!-- set the build number based on environment variable, otherwise blank -->
 	<property environment="env" description="System environment variables (including those set by Hudson)" />
 
-	<target name="all" depends="clean, jars" description="Builds all jars" />
+	<target name="all" depends="clean, package" description="Builds and packages" />
 
 	<target name="clean" description="Delete generated files.">
 		<echo message="Deleting generated files in dist" />
@@ -53,27 +52,62 @@
 		</jar>
 	</target>
 	
-	<target name="package-testjob" depends="jars" description="Creates a test zip">
-		<delete dir="${dist.packages.dir}" />
-		<mkdir dir="${dist.packages.dir}" />
-		
+	<target name="package-exectest1" depends="jars" description="Creates a test zip">
 		<!-- Tarball it -->
-		<zip destfile="${dist.packages.dir}/testjob.zip">
+		<zip destfile="${dist.packages.dir}/exectest1.zip">
 			<zipfileset dir="${dist.jar.dir}" />
-			<zipfileset dir="${job.conf.dir}" />
+      <zipfileset dir="${job.conf.dir}/exectest1" />
 		</zip>
 	</target>
 
 	<target name="package-exectest2" depends="jars" description="Creates a test zip">
-		<delete dir="${dist.packages.dir}" />
-		<mkdir dir="${dist.packages.dir}" />
-		
 		<!-- Tarball it -->
 		<zip destfile="${dist.packages.dir}/exectest2.zip">
 			<zipfileset dir="${dist.jar.dir}" />
-			<zipfileset dir="${job.conf.dir2}" />
+      <zipfileset dir="${job.conf.dir}/exectest2" />
+		</zip>
+	</target>
+	
+	<target name="package-animal" depends="jars" description="Creates a test zip">
+		<!-- Tarball it -->
+		<zip destfile="${dist.packages.dir}/animal.zip">
+			<zipfileset dir="${dist.jar.dir}" />
+      <zipfileset dir="${job.conf.dir}/animal" />
 		</zip>
 	</target>
-
 	
+	<target name="package-embedded" depends="jars" description="Creates a test zip">
+		<!-- Tarball it -->
+		<zip destfile="${dist.packages.dir}/embedded.zip">
+			<zipfileset dir="${dist.jar.dir}" />
+      <zipfileset dir="${job.conf.dir}/embedded" />
+		</zip>
+	</target>
+
+	<target name="package-embedded2" depends="jars" description="Creates a test zip">
+		<!-- Tarball it -->
+		<zip destfile="${dist.packages.dir}/embedded2.zip">
+			<zipfileset dir="${dist.jar.dir}" />
+      <zipfileset dir="${job.conf.dir}/embedded2" />
+		</zip>
+  </target>	
+  
+  <target name="package-embedded3" depends="jars" description="Creates a test zip">
+		<!-- Tarball it -->
+		<zip destfile="${dist.packages.dir}/embedded3.zip">
+			<zipfileset dir="${dist.jar.dir}" />
+      <zipfileset dir="${job.conf.dir}/embedded3" />
+		</zip>
+  </target>	
+  
+  <target name="package-embeddedBad" depends="jars" description="Creates a test zip">
+		<!-- Tarball it -->
+		<zip destfile="${dist.packages.dir}/embeddedBad.zip">
+			<zipfileset dir="${dist.jar.dir}" />
+      <zipfileset dir="${job.conf.dir}/embeddedBad" />
+		</zip>
+  </target>	
+  
+  <target name="package" depends="package-exectest1, package-exectest2, package-animal, package-embedded, package-embedded2, package-embedded3, package-embeddedBad" description="Creates all packages">
+  </target>
 </project>
diff --git a/unit/executions/animal/albatross.job b/unit/executions/animal/albatross.job
new file mode 100644
index 0000000..4f9cf95
--- /dev/null
+++ b/unit/executions/animal/albatross.job
@@ -0,0 +1,4 @@
+type=java
+job.class=azkaban.test.executor.SleepJavaJob
+seconds=21
+fail=false
\ No newline at end of file
diff --git a/unit/executions/animal/animals.job b/unit/executions/animal/animals.job
new file mode 100644
index 0000000..8d2e4a4
--- /dev/null
+++ b/unit/executions/animal/animals.job
@@ -0,0 +1,5 @@
+type=java
+job.class=azkaban.test.executor.SleepJavaJob
+seconds=11
+fail=false
+dependencies=humpback-whale
\ No newline at end of file
diff --git a/unit/executions/animal/baboon.job b/unit/executions/animal/baboon.job
new file mode 100644
index 0000000..0a4e652
--- /dev/null
+++ b/unit/executions/animal/baboon.job
@@ -0,0 +1,5 @@
+type=java
+job.class=azkaban.test.executor.SleepJavaJob
+seconds=7
+fail=false
+dependencies=albatross
\ No newline at end of file
diff --git a/unit/executions/animal/caiman.job b/unit/executions/animal/caiman.job
new file mode 100644
index 0000000..2e258ac
--- /dev/null
+++ b/unit/executions/animal/caiman.job
@@ -0,0 +1,5 @@
+type=java
+job.class=azkaban.test.executor.SleepJavaJob
+seconds=13
+fail=false
+dependencies=baboon
\ No newline at end of file
diff --git a/unit/executions/animal/camel.job b/unit/executions/animal/camel.job
new file mode 100644
index 0000000..a854a97
--- /dev/null
+++ b/unit/executions/animal/camel.job
@@ -0,0 +1,5 @@
+type=java
+job.class=azkaban.test.executor.SleepJavaJob
+seconds=18
+fail=false
+dependencies=baboon
\ No newline at end of file
diff --git a/unit/executions/animal/capybara.job b/unit/executions/animal/capybara.job
new file mode 100644
index 0000000..02db6f5
--- /dev/null
+++ b/unit/executions/animal/capybara.job
@@ -0,0 +1,5 @@
+type=java
+job.class=azkaban.test.executor.SleepJavaJob
+seconds=16
+fail=false
+dependencies=baboon
\ No newline at end of file
diff --git a/unit/executions/animal/cat.job b/unit/executions/animal/cat.job
new file mode 100644
index 0000000..f7de0a7
--- /dev/null
+++ b/unit/executions/animal/cat.job
@@ -0,0 +1,5 @@
+type=java
+job.class=azkaban.test.executor.SleepJavaJob
+seconds=23
+fail=false
+dependencies=baboon
\ No newline at end of file
diff --git a/unit/executions/animal/caterpillar.job b/unit/executions/animal/caterpillar.job
new file mode 100644
index 0000000..6b4347c
--- /dev/null
+++ b/unit/executions/animal/caterpillar.job
@@ -0,0 +1,5 @@
+type=java
+job.class=azkaban.test.executor.SleepJavaJob
+seconds=19
+fail=false
+dependencies=baboon
\ No newline at end of file
diff --git a/unit/executions/animal/catfish.job b/unit/executions/animal/catfish.job
new file mode 100644
index 0000000..538d93a
--- /dev/null
+++ b/unit/executions/animal/catfish.job
@@ -0,0 +1,5 @@
+type=java
+job.class=azkaban.test.executor.SleepJavaJob
+seconds=14
+fail=false
+dependencies=baboon
\ No newline at end of file
diff --git a/unit/executions/animal/centipede.job b/unit/executions/animal/centipede.job
new file mode 100644
index 0000000..2e258ac
--- /dev/null
+++ b/unit/executions/animal/centipede.job
@@ -0,0 +1,5 @@
+type=java
+job.class=azkaban.test.executor.SleepJavaJob
+seconds=13
+fail=false
+dependencies=baboon
\ No newline at end of file
diff --git a/unit/executions/animal/chameleon.job b/unit/executions/animal/chameleon.job
new file mode 100644
index 0000000..7393129
--- /dev/null
+++ b/unit/executions/animal/chameleon.job
@@ -0,0 +1,5 @@
+type=java
+job.class=azkaban.test.executor.SleepJavaJob
+seconds=25
+fail=false
+dependencies=baboon
\ No newline at end of file
diff --git a/unit/executions/animal/cheetah.job b/unit/executions/animal/cheetah.job
new file mode 100644
index 0000000..5003eb3
--- /dev/null
+++ b/unit/executions/animal/cheetah.job
@@ -0,0 +1,5 @@
+type=java
+job.class=azkaban.test.executor.SleepJavaJob
+seconds=7
+fail=false
+dependencies=baboon
\ No newline at end of file
diff --git a/unit/executions/animal/chicken.job b/unit/executions/animal/chicken.job
new file mode 100644
index 0000000..dbbbc72
--- /dev/null
+++ b/unit/executions/animal/chicken.job
@@ -0,0 +1,5 @@
+type=java
+job.class=azkaban.test.executor.SleepJavaJob
+seconds=29
+fail=false
+dependencies=baboon
\ No newline at end of file
diff --git a/unit/executions/animal/chihuahua.job b/unit/executions/animal/chihuahua.job
new file mode 100644
index 0000000..a854a97
--- /dev/null
+++ b/unit/executions/animal/chihuahua.job
@@ -0,0 +1,5 @@
+type=java
+job.class=azkaban.test.executor.SleepJavaJob
+seconds=18
+fail=false
+dependencies=baboon
\ No newline at end of file
diff --git a/unit/executions/animal/clown-fish.job b/unit/executions/animal/clown-fish.job
new file mode 100644
index 0000000..538d93a
--- /dev/null
+++ b/unit/executions/animal/clown-fish.job
@@ -0,0 +1,5 @@
+type=java
+job.class=azkaban.test.executor.SleepJavaJob
+seconds=14
+fail=false
+dependencies=baboon
\ No newline at end of file
diff --git a/unit/executions/animal/cockroach.job b/unit/executions/animal/cockroach.job
new file mode 100644
index 0000000..7449537
--- /dev/null
+++ b/unit/executions/animal/cockroach.job
@@ -0,0 +1,5 @@
+type=java
+job.class=azkaban.test.executor.SleepJavaJob
+seconds=9
+fail=false
+dependencies=baboon
\ No newline at end of file
diff --git a/unit/executions/animal/cougar.job b/unit/executions/animal/cougar.job
new file mode 100644
index 0000000..d29ca57
--- /dev/null
+++ b/unit/executions/animal/cougar.job
@@ -0,0 +1,5 @@
+type=java
+job.class=azkaban.test.executor.SleepJavaJob
+seconds=30
+fail=false
+dependencies=baboon
\ No newline at end of file
diff --git a/unit/executions/animal/cuttlefish.job b/unit/executions/animal/cuttlefish.job
new file mode 100644
index 0000000..dbbbc72
--- /dev/null
+++ b/unit/executions/animal/cuttlefish.job
@@ -0,0 +1,5 @@
+type=java
+job.class=azkaban.test.executor.SleepJavaJob
+seconds=29
+fail=false
+dependencies=baboon
\ No newline at end of file
diff --git a/unit/executions/animal/dolphin.job b/unit/executions/animal/dolphin.job
new file mode 100644
index 0000000..a1b6a0c
--- /dev/null
+++ b/unit/executions/animal/dolphin.job
@@ -0,0 +1,5 @@
+type=java
+job.class=azkaban.test.executor.SleepJavaJob
+seconds=14
+fail=false
+dependencies=caterpillar,chameleon,cougar,camel,cuttlefish,centipede,cheetah
\ No newline at end of file
diff --git a/unit/executions/animal/elephant.job b/unit/executions/animal/elephant.job
new file mode 100644
index 0000000..d2b6853
--- /dev/null
+++ b/unit/executions/animal/elephant.job
@@ -0,0 +1,5 @@
+type=java
+job.class=azkaban.test.executor.SleepJavaJob
+seconds=8
+fail=false
+dependencies=camel,caiman,capybara,cat,catfish,chicken,chihuahua,clown-fish,cockroach,dolphin
\ No newline at end of file
diff --git a/unit/executions/animal/flamingo.job b/unit/executions/animal/flamingo.job
new file mode 100644
index 0000000..07b68a9
--- /dev/null
+++ b/unit/executions/animal/flamingo.job
@@ -0,0 +1,5 @@
+type=java
+job.class=azkaban.test.executor.SleepJavaJob
+seconds=7
+fail=false
+dependencies=camel,elephant
\ No newline at end of file
diff --git a/unit/executions/animal/gorilla.job b/unit/executions/animal/gorilla.job
new file mode 100644
index 0000000..f5b16e7
--- /dev/null
+++ b/unit/executions/animal/gorilla.job
@@ -0,0 +1,5 @@
+type=java
+job.class=azkaban.test.executor.SleepJavaJob
+seconds=27
+fail=false
+dependencies=flamingo
\ No newline at end of file
diff --git a/unit/executions/animal/humpback-whale.job b/unit/executions/animal/humpback-whale.job
new file mode 100644
index 0000000..4981525
--- /dev/null
+++ b/unit/executions/animal/humpback-whale.job
@@ -0,0 +1,5 @@
+type=java
+job.class=azkaban.test.executor.SleepJavaJob
+seconds=19
+fail=false
+dependencies=gorilla
\ No newline at end of file
diff --git a/unit/executions/embedded/innerFlow.job b/unit/executions/embedded/innerFlow.job
new file mode 100644
index 0000000..e9b3b89
--- /dev/null
+++ b/unit/executions/embedded/innerFlow.job
@@ -0,0 +1,4 @@
+type=javaprocess
+seconds=1
+fail=false
+dependencies=innerJobB,innerJobC
\ No newline at end of file
diff --git a/unit/executions/embedded/innerJobA.job b/unit/executions/embedded/innerJobA.job
new file mode 100644
index 0000000..665b38d
--- /dev/null
+++ b/unit/executions/embedded/innerJobA.job
@@ -0,0 +1,4 @@
+type=javaprocess
+java.class=azkaban.test.executor.SleepJavaJob
+seconds=1
+fail=false
diff --git a/unit/executions/embedded/innerJobB.job b/unit/executions/embedded/innerJobB.job
new file mode 100644
index 0000000..178bbef
--- /dev/null
+++ b/unit/executions/embedded/innerJobB.job
@@ -0,0 +1,5 @@
+type=javaprocess
+java.class=azkaban.test.executor.SleepJavaJob
+seconds=1
+fail=false
+dependencies=innerJobA
diff --git a/unit/executions/embedded/innerJobC.job b/unit/executions/embedded/innerJobC.job
new file mode 100644
index 0000000..178bbef
--- /dev/null
+++ b/unit/executions/embedded/innerJobC.job
@@ -0,0 +1,5 @@
+type=javaprocess
+java.class=azkaban.test.executor.SleepJavaJob
+seconds=1
+fail=false
+dependencies=innerJobA
diff --git a/unit/executions/embedded/joba.job b/unit/executions/embedded/joba.job
new file mode 100644
index 0000000..665b38d
--- /dev/null
+++ b/unit/executions/embedded/joba.job
@@ -0,0 +1,4 @@
+type=javaprocess
+java.class=azkaban.test.executor.SleepJavaJob
+seconds=1
+fail=false
diff --git a/unit/executions/embedded/jobb.job b/unit/executions/embedded/jobb.job
new file mode 100644
index 0000000..8844d8c
--- /dev/null
+++ b/unit/executions/embedded/jobb.job
@@ -0,0 +1,3 @@
+type=flow
+flow.name=innerFlow
+dependencies=joba
diff --git a/unit/executions/embedded/jobc.job b/unit/executions/embedded/jobc.job
new file mode 100644
index 0000000..8844d8c
--- /dev/null
+++ b/unit/executions/embedded/jobc.job
@@ -0,0 +1,3 @@
+type=flow
+flow.name=innerFlow
+dependencies=joba
diff --git a/unit/executions/embedded/jobd.job b/unit/executions/embedded/jobd.job
new file mode 100644
index 0000000..8844d8c
--- /dev/null
+++ b/unit/executions/embedded/jobd.job
@@ -0,0 +1,3 @@
+type=flow
+flow.name=innerFlow
+dependencies=joba
diff --git a/unit/executions/embedded/jobe.job b/unit/executions/embedded/jobe.job
new file mode 100644
index 0000000..fe986d5
--- /dev/null
+++ b/unit/executions/embedded/jobe.job
@@ -0,0 +1,5 @@
+type=javaprocess
+java.class=azkaban.test.executor.SleepJavaJob
+seconds=1
+fail=false
+dependencies=jobb,jobc,jobd
diff --git a/unit/executions/embedded2/innerFlow.job b/unit/executions/embedded2/innerFlow.job
new file mode 100644
index 0000000..dfa0e9d
--- /dev/null
+++ b/unit/executions/embedded2/innerFlow.job
@@ -0,0 +1,2 @@
+type=test
+dependencies=innerJobB,innerJobC
\ No newline at end of file
diff --git a/unit/executions/embedded2/innerFlow2.job b/unit/executions/embedded2/innerFlow2.job
new file mode 100644
index 0000000..35cbccb
--- /dev/null
+++ b/unit/executions/embedded2/innerFlow2.job
@@ -0,0 +1,2 @@
+type=test
+dependencies=innerJobA
\ No newline at end of file
diff --git a/unit/executions/embedded2/innerJobA.job b/unit/executions/embedded2/innerJobA.job
new file mode 100644
index 0000000..35ebd72
--- /dev/null
+++ b/unit/executions/embedded2/innerJobA.job
@@ -0,0 +1,2 @@
+type=test
+
diff --git a/unit/executions/embedded2/innerJobB.job b/unit/executions/embedded2/innerJobB.job
new file mode 100644
index 0000000..dca1223
--- /dev/null
+++ b/unit/executions/embedded2/innerJobB.job
@@ -0,0 +1,2 @@
+type=test
+dependencies=innerJobA
diff --git a/unit/executions/embedded2/innerJobC.job b/unit/executions/embedded2/innerJobC.job
new file mode 100644
index 0000000..dca1223
--- /dev/null
+++ b/unit/executions/embedded2/innerJobC.job
@@ -0,0 +1,2 @@
+type=test
+dependencies=innerJobA
diff --git a/unit/executions/embedded2/joba.job b/unit/executions/embedded2/joba.job
new file mode 100644
index 0000000..80ad69e
--- /dev/null
+++ b/unit/executions/embedded2/joba.job
@@ -0,0 +1,2 @@
+type=test
+param1=joba.1
\ No newline at end of file
diff --git a/unit/executions/embedded2/joba1.job b/unit/executions/embedded2/joba1.job
new file mode 100644
index 0000000..98fd5f5
--- /dev/null
+++ b/unit/executions/embedded2/joba1.job
@@ -0,0 +1 @@
+type=test
diff --git a/unit/executions/embedded2/jobb.job b/unit/executions/embedded2/jobb.job
new file mode 100644
index 0000000..4531028
--- /dev/null
+++ b/unit/executions/embedded2/jobb.job
@@ -0,0 +1,5 @@
+type=flow
+flow.name=innerFlow
+dependencies=joba
+testprops=moo
+output.override=jobb
\ No newline at end of file
diff --git a/unit/executions/embedded2/jobc.job b/unit/executions/embedded2/jobc.job
new file mode 100644
index 0000000..2bfc5ff
--- /dev/null
+++ b/unit/executions/embedded2/jobc.job
@@ -0,0 +1,2 @@
+type=test
+dependencies=joba
diff --git a/unit/executions/embedded2/jobd.job b/unit/executions/embedded2/jobd.job
new file mode 100644
index 0000000..e80f82b
--- /dev/null
+++ b/unit/executions/embedded2/jobd.job
@@ -0,0 +1,4 @@
+type=flow
+flow.name=innerFlow2
+dependencies=joba
+jobdprop=poop
diff --git a/unit/executions/embedded2/jobe.job b/unit/executions/embedded2/jobe.job
new file mode 100644
index 0000000..331a81e
--- /dev/null
+++ b/unit/executions/embedded2/jobe.job
@@ -0,0 +1,2 @@
+type=test
+dependencies=jobb,jobc,jobd
diff --git a/unit/executions/embedded2/jobf.job b/unit/executions/embedded2/jobf.job
new file mode 100644
index 0000000..b1b00ce
--- /dev/null
+++ b/unit/executions/embedded2/jobf.job
@@ -0,0 +1,2 @@
+type=test
+dependencies=jobe,joba1
diff --git a/unit/executions/embedded2/jobg.job b/unit/executions/embedded2/jobg.job
new file mode 100644
index 0000000..b1b00ce
--- /dev/null
+++ b/unit/executions/embedded2/jobg.job
@@ -0,0 +1,2 @@
+type=test
+dependencies=jobe,joba1
diff --git a/unit/executions/embedded2/pipeline1.job b/unit/executions/embedded2/pipeline1.job
new file mode 100644
index 0000000..4afbfdc
--- /dev/null
+++ b/unit/executions/embedded2/pipeline1.job
@@ -0,0 +1 @@
+type=test
\ No newline at end of file
diff --git a/unit/executions/embedded2/pipeline1_1.job b/unit/executions/embedded2/pipeline1_1.job
new file mode 100644
index 0000000..cfe35cc
--- /dev/null
+++ b/unit/executions/embedded2/pipeline1_1.job
@@ -0,0 +1,4 @@
+type=flow
+flow.name=innerFlow2
+testprops=moo
+output.override=jobb
\ No newline at end of file
diff --git a/unit/executions/embedded2/pipeline1_2.job b/unit/executions/embedded2/pipeline1_2.job
new file mode 100644
index 0000000..711d823
--- /dev/null
+++ b/unit/executions/embedded2/pipeline1_2.job
@@ -0,0 +1,5 @@
+type=flow
+flow.name=innerFlow2
+dependencies=pipeline1_1
+testprops=moo
+output.override=jobb
\ No newline at end of file
diff --git a/unit/executions/embedded2/pipeline2.job b/unit/executions/embedded2/pipeline2.job
new file mode 100644
index 0000000..84f6498
--- /dev/null
+++ b/unit/executions/embedded2/pipeline2.job
@@ -0,0 +1,2 @@
+type=test
+dependencies=pipeline1
\ No newline at end of file
diff --git a/unit/executions/embedded2/pipeline4.job b/unit/executions/embedded2/pipeline4.job
new file mode 100644
index 0000000..b24c4ba
--- /dev/null
+++ b/unit/executions/embedded2/pipeline4.job
@@ -0,0 +1,2 @@
+type=test
+dependencies=pipelineEmbeddedFlow3
\ No newline at end of file
diff --git a/unit/executions/embedded2/pipelineEmbeddedFlow3.job b/unit/executions/embedded2/pipelineEmbeddedFlow3.job
new file mode 100644
index 0000000..0a1ae46
--- /dev/null
+++ b/unit/executions/embedded2/pipelineEmbeddedFlow3.job
@@ -0,0 +1,5 @@
+type=flow
+flow.name=innerFlow
+dependencies=pipeline2
+testprops=moo
+output.override=jobb
\ No newline at end of file
diff --git a/unit/executions/embedded2/pipelineFlow.job b/unit/executions/embedded2/pipelineFlow.job
new file mode 100644
index 0000000..e50329c
--- /dev/null
+++ b/unit/executions/embedded2/pipelineFlow.job
@@ -0,0 +1,2 @@
+type=test
+dependencies=pipeline4
\ No newline at end of file
diff --git a/unit/executions/embedded2/test1.properties b/unit/executions/embedded2/test1.properties
new file mode 100644
index 0000000..120fc25
--- /dev/null
+++ b/unit/executions/embedded2/test1.properties
@@ -0,0 +1,4 @@
+param1=test1.1
+param2=test1.2
+param3=test1.3
+param4=test1.4
\ No newline at end of file
diff --git a/unit/executions/embedded2/test2.properties b/unit/executions/embedded2/test2.properties
new file mode 100644
index 0000000..7df7744
--- /dev/null
+++ b/unit/executions/embedded2/test2.properties
@@ -0,0 +1,4 @@
+param5=test2.5
+param6=test2.6
+param7=test2.7
+param8=test2.8
\ No newline at end of file
diff --git a/unit/executions/embedded3/innerFlow.job b/unit/executions/embedded3/innerFlow.job
new file mode 100644
index 0000000..e9b3b89
--- /dev/null
+++ b/unit/executions/embedded3/innerFlow.job
@@ -0,0 +1,4 @@
+type=javaprocess
+seconds=1
+fail=false
+dependencies=innerJobB,innerJobC
\ No newline at end of file
diff --git a/unit/executions/embedded3/innerFlow2.job b/unit/executions/embedded3/innerFlow2.job
new file mode 100644
index 0000000..2346982
--- /dev/null
+++ b/unit/executions/embedded3/innerFlow2.job
@@ -0,0 +1,4 @@
+type=javaprocess
+seconds=1
+fail=false
+dependencies=innerJobA
\ No newline at end of file
diff --git a/unit/executions/embedded3/innerJobA.job b/unit/executions/embedded3/innerJobA.job
new file mode 100644
index 0000000..665b38d
--- /dev/null
+++ b/unit/executions/embedded3/innerJobA.job
@@ -0,0 +1,4 @@
+type=javaprocess
+java.class=azkaban.test.executor.SleepJavaJob
+seconds=1
+fail=false
diff --git a/unit/executions/embedded3/innerJobB.job b/unit/executions/embedded3/innerJobB.job
new file mode 100644
index 0000000..24a2e04
--- /dev/null
+++ b/unit/executions/embedded3/innerJobB.job
@@ -0,0 +1,3 @@
+type=flow
+flow.name=innerFlow2
+dependencies=innerJobA
diff --git a/unit/executions/embedded3/innerJobC.job b/unit/executions/embedded3/innerJobC.job
new file mode 100644
index 0000000..178bbef
--- /dev/null
+++ b/unit/executions/embedded3/innerJobC.job
@@ -0,0 +1,5 @@
+type=javaprocess
+java.class=azkaban.test.executor.SleepJavaJob
+seconds=1
+fail=false
+dependencies=innerJobA
diff --git a/unit/executions/embedded3/joba.job b/unit/executions/embedded3/joba.job
new file mode 100644
index 0000000..665b38d
--- /dev/null
+++ b/unit/executions/embedded3/joba.job
@@ -0,0 +1,4 @@
+type=javaprocess
+java.class=azkaban.test.executor.SleepJavaJob
+seconds=1
+fail=false
diff --git a/unit/executions/embedded3/jobb.job b/unit/executions/embedded3/jobb.job
new file mode 100644
index 0000000..8844d8c
--- /dev/null
+++ b/unit/executions/embedded3/jobb.job
@@ -0,0 +1,3 @@
+type=flow
+flow.name=innerFlow
+dependencies=joba
diff --git a/unit/executions/embedded3/jobc.job b/unit/executions/embedded3/jobc.job
new file mode 100644
index 0000000..8844d8c
--- /dev/null
+++ b/unit/executions/embedded3/jobc.job
@@ -0,0 +1,3 @@
+type=flow
+flow.name=innerFlow
+dependencies=joba
diff --git a/unit/executions/embedded3/jobd.job b/unit/executions/embedded3/jobd.job
new file mode 100644
index 0000000..8844d8c
--- /dev/null
+++ b/unit/executions/embedded3/jobd.job
@@ -0,0 +1,3 @@
+type=flow
+flow.name=innerFlow
+dependencies=joba
diff --git a/unit/executions/embedded3/jobe.job b/unit/executions/embedded3/jobe.job
new file mode 100644
index 0000000..fe986d5
--- /dev/null
+++ b/unit/executions/embedded3/jobe.job
@@ -0,0 +1,5 @@
+type=javaprocess
+java.class=azkaban.test.executor.SleepJavaJob
+seconds=1
+fail=false
+dependencies=jobb,jobc,jobd
diff --git a/unit/executions/embeddedBad/innerFlow.job b/unit/executions/embeddedBad/innerFlow.job
new file mode 100644
index 0000000..da71d64
--- /dev/null
+++ b/unit/executions/embeddedBad/innerFlow.job
@@ -0,0 +1,5 @@
+type=javaprocess
+java.class=azkaban.test.executor.SleepJavaJob
+seconds=1
+fail=false
+dependencies=innerJobB,innerJobC
\ No newline at end of file
diff --git a/unit/executions/embeddedBad/innerJobA.job b/unit/executions/embeddedBad/innerJobA.job
new file mode 100644
index 0000000..665b38d
--- /dev/null
+++ b/unit/executions/embeddedBad/innerJobA.job
@@ -0,0 +1,4 @@
+type=javaprocess
+java.class=azkaban.test.executor.SleepJavaJob
+seconds=1
+fail=false
diff --git a/unit/executions/embeddedBad/innerJobB.job b/unit/executions/embeddedBad/innerJobB.job
new file mode 100644
index 0000000..dc29b4a
--- /dev/null
+++ b/unit/executions/embeddedBad/innerJobB.job
@@ -0,0 +1,3 @@
+type=flow
+flow.name=jobe
+dependencies=innerJobA
diff --git a/unit/executions/embeddedBad/innerJobC.job b/unit/executions/embeddedBad/innerJobC.job
new file mode 100644
index 0000000..178bbef
--- /dev/null
+++ b/unit/executions/embeddedBad/innerJobC.job
@@ -0,0 +1,5 @@
+type=javaprocess
+java.class=azkaban.test.executor.SleepJavaJob
+seconds=1
+fail=false
+dependencies=innerJobA
diff --git a/unit/executions/embeddedBad/joba.job b/unit/executions/embeddedBad/joba.job
new file mode 100644
index 0000000..665b38d
--- /dev/null
+++ b/unit/executions/embeddedBad/joba.job
@@ -0,0 +1,4 @@
+type=javaprocess
+java.class=azkaban.test.executor.SleepJavaJob
+seconds=1
+fail=false
diff --git a/unit/executions/embeddedBad/jobb.job b/unit/executions/embeddedBad/jobb.job
new file mode 100644
index 0000000..8844d8c
--- /dev/null
+++ b/unit/executions/embeddedBad/jobb.job
@@ -0,0 +1,3 @@
+type=flow
+flow.name=innerFlow
+dependencies=joba
diff --git a/unit/executions/embeddedBad/jobc.job b/unit/executions/embeddedBad/jobc.job
new file mode 100644
index 0000000..8844d8c
--- /dev/null
+++ b/unit/executions/embeddedBad/jobc.job
@@ -0,0 +1,3 @@
+type=flow
+flow.name=innerFlow
+dependencies=joba
diff --git a/unit/executions/embeddedBad/jobd.job b/unit/executions/embeddedBad/jobd.job
new file mode 100644
index 0000000..8844d8c
--- /dev/null
+++ b/unit/executions/embeddedBad/jobd.job
@@ -0,0 +1,3 @@
+type=flow
+flow.name=innerFlow
+dependencies=joba
diff --git a/unit/executions/embeddedBad/jobe.job b/unit/executions/embeddedBad/jobe.job
new file mode 100644
index 0000000..fe986d5
--- /dev/null
+++ b/unit/executions/embeddedBad/jobe.job
@@ -0,0 +1,5 @@
+type=javaprocess
+java.class=azkaban.test.executor.SleepJavaJob
+seconds=1
+fail=false
+dependencies=jobb,jobc,jobd
diff --git a/unit/executions/embeddedBad/selfreference.job b/unit/executions/embeddedBad/selfreference.job
new file mode 100644
index 0000000..708f351
--- /dev/null
+++ b/unit/executions/embeddedBad/selfreference.job
@@ -0,0 +1,2 @@
+type=flow
+flow.name=selfreference
\ No newline at end of file
diff --git a/unit/java/azkaban/Scrubber.java b/unit/java/azkaban/Scrubber.java
new file mode 100644
index 0000000..17c7d65
--- /dev/null
+++ b/unit/java/azkaban/Scrubber.java
@@ -0,0 +1,20 @@
+package azkaban;
+
+import java.io.File;
+
+import org.apache.log4j.Logger;
+
+import azkaban.utils.DirectoryFlowLoader;
+
+public class Scrubber {
+	private static Logger logger = Logger.getLogger(Scrubber.class);
+	
+	public static void main(String[] args) {
+		DirectoryFlowLoader loader = new DirectoryFlowLoader(logger);
+		
+		File baseDir = new File(args[0]);
+		loader.loadProjectFlow(baseDir);
+	
+		loader.getFlowMap();
+	}
+}
\ No newline at end of file
diff --git a/unit/java/azkaban/test/execapp/event/LocalFlowWatcherTest.java b/unit/java/azkaban/test/execapp/event/LocalFlowWatcherTest.java
index 2bdfc7a..c87f2c6 100644
--- a/unit/java/azkaban/test/execapp/event/LocalFlowWatcherTest.java
+++ b/unit/java/azkaban/test/execapp/event/LocalFlowWatcherTest.java
@@ -20,6 +20,7 @@ import azkaban.executor.ExecutorLoader;
 import azkaban.executor.Status;
 import azkaban.flow.Flow;
 import azkaban.jobtype.JobTypeManager;
+import azkaban.project.Project;
 import azkaban.project.ProjectLoader;
 import azkaban.test.execapp.EventCollectorListener;
 import azkaban.test.execapp.MockExecutorLoader;
@@ -136,15 +137,15 @@ public class LocalFlowWatcherTest {
 			Assert.assertEquals(node.getStatus(), Status.SUCCEEDED);
 			
 			// check it's start time is after the first's children.
-			ExecutableNode watchedNode = first.getExecutableNode(node.getJobId());
+			ExecutableNode watchedNode = first.getExecutableNode(node.getId());
 			if (watchedNode == null) {
 				continue;
 			}
 			Assert.assertEquals(watchedNode.getStatus(), Status.SUCCEEDED);
 			
-			System.out.println("Node " + node.getJobId() + 
+			System.out.println("Node " + node.getId() + 
 					" start: " + node.getStartTime() + 
-					" dependent on " + watchedNode.getJobId() + 
+					" dependent on " + watchedNode.getId() + 
 					" " + watchedNode.getEndTime() + 
 					" diff: " + (node.getStartTime() - watchedNode.getEndTime()));
 
@@ -170,7 +171,7 @@ public class LocalFlowWatcherTest {
 			Assert.assertEquals(node.getStatus(), Status.SUCCEEDED);
 			
 			// check it's start time is after the first's children.
-			ExecutableNode watchedNode = first.getExecutableNode(node.getJobId());
+			ExecutableNode watchedNode = first.getExecutableNode(node.getId());
 			if (watchedNode == null) {
 				continue;
 			}
@@ -185,10 +186,11 @@ public class LocalFlowWatcherTest {
 				Assert.assertEquals(child.getStatus(), Status.SUCCEEDED);
 				long diff = node.getStartTime() - child.getEndTime();
 				minDiff = Math.min(minDiff, diff);
-				System.out.println("Node " + node.getJobId() + 
+				System.out.println("Node " + node.getId() + 
 						" start: " + node.getStartTime() + 
 						" dependent on " + watchedChild + " " + child.getEndTime() +
 						" diff: " + diff);
+				
 				Assert.assertTrue(node.getStartTime() >= child.getEndTime());
 			}
 			
@@ -228,8 +230,9 @@ public class LocalFlowWatcherTest {
 		@SuppressWarnings("unchecked")
 		HashMap<String, Object> flowObj = (HashMap<String, Object>) JSONUtils.parseJSONFromFile(jsonFlowFile);
 		
+		Project project = new Project(1, "test");
 		Flow flow = Flow.flowFromObject(flowObj);
-		ExecutableFlow execFlow = new ExecutableFlow(flow);
+		ExecutableFlow execFlow = new ExecutableFlow(project, flow);
 		execFlow.setExecutionId(execId);
 		execFlow.setExecutionPath(workingDir.getPath());
 		return execFlow;
diff --git a/unit/java/azkaban/test/execapp/event/RemoteFlowWatcherTest.java b/unit/java/azkaban/test/execapp/event/RemoteFlowWatcherTest.java
index edf520c..45c3a85 100644
--- a/unit/java/azkaban/test/execapp/event/RemoteFlowWatcherTest.java
+++ b/unit/java/azkaban/test/execapp/event/RemoteFlowWatcherTest.java
@@ -14,12 +14,14 @@ import azkaban.execapp.FlowRunner;
 import azkaban.execapp.event.FlowWatcher;
 import azkaban.execapp.event.RemoteFlowWatcher;
 import azkaban.executor.ExecutableFlow;
+import azkaban.executor.ExecutableFlowBase;
 import azkaban.executor.ExecutableNode;
 import azkaban.executor.ExecutionOptions;
 import azkaban.executor.ExecutorLoader;
 import azkaban.executor.Status;
 import azkaban.flow.Flow;
 import azkaban.jobtype.JobTypeManager;
+import azkaban.project.Project;
 import azkaban.project.ProjectLoader;
 import azkaban.test.execapp.EventCollectorListener;
 import azkaban.test.execapp.MockExecutorLoader;
@@ -71,8 +73,10 @@ public class RemoteFlowWatcherTest {
 		FlowRunner runner2 = createFlowRunner(workingDir2, loader, eventCollector, "exec1", 2, watcher, 2);
 		Thread runner2Thread = new Thread(runner2);
 		
+		printCurrentState("runner1 ", runner1.getExecutableFlow());
 		runner1Thread.start();
 		runner2Thread.start();
+		
 		runner2Thread.join();
 		
 		FileUtils.deleteDirectory(workingDir1);
@@ -137,15 +141,15 @@ public class RemoteFlowWatcherTest {
 			Assert.assertEquals(node.getStatus(), Status.SUCCEEDED);
 			
 			// check it's start time is after the first's children.
-			ExecutableNode watchedNode = first.getExecutableNode(node.getJobId());
+			ExecutableNode watchedNode = first.getExecutableNode(node.getId());
 			if (watchedNode == null) {
 				continue;
 			}
 			Assert.assertEquals(watchedNode.getStatus(), Status.SUCCEEDED);
 			
-			System.out.println("Node " + node.getJobId() + 
+			System.out.println("Node " + node.getId() + 
 					" start: " + node.getStartTime() + 
-					" dependent on " + watchedNode.getJobId() + 
+					" dependent on " + watchedNode.getId() + 
 					" " + watchedNode.getEndTime() + 
 					" diff: " + (node.getStartTime() - watchedNode.getEndTime()));
 
@@ -170,7 +174,7 @@ public class RemoteFlowWatcherTest {
 			Assert.assertEquals(node.getStatus(), Status.SUCCEEDED);
 			
 			// check it's start time is after the first's children.
-			ExecutableNode watchedNode = first.getExecutableNode(node.getJobId());
+			ExecutableNode watchedNode = first.getExecutableNode(node.getId());
 			if (watchedNode == null) {
 				continue;
 			}
@@ -185,7 +189,7 @@ public class RemoteFlowWatcherTest {
 				Assert.assertEquals(child.getStatus(), Status.SUCCEEDED);
 				long diff = node.getStartTime() - child.getEndTime();
 				minDiff = Math.min(minDiff, diff);
-				System.out.println("Node " + node.getJobId() + 
+				System.out.println("Node " + node.getId() + 
 						" start: " + node.getStartTime() + 
 						" dependent on " + watchedChild + " " + child.getEndTime() +
 						" diff: " + diff);
@@ -221,6 +225,16 @@ public class RemoteFlowWatcherTest {
 		return runner;
 	}
 	
+	private void printCurrentState(String prefix, ExecutableFlowBase flow) {
+		for(ExecutableNode node: flow.getExecutableNodes()) {
+
+			System.err.println(prefix + node.getNestedId() + "->" + node.getStatus().name());
+			if (node instanceof ExecutableFlowBase) {
+				printCurrentState(prefix, (ExecutableFlowBase)node);
+			}
+		}
+	}
+	
 	private ExecutableFlow prepareExecDir(File workingDir, File execDir, String flowName, int execId) throws IOException {
 		FileUtils.copyDirectory(execDir, workingDir);
 		
@@ -228,8 +242,9 @@ public class RemoteFlowWatcherTest {
 		@SuppressWarnings("unchecked")
 		HashMap<String, Object> flowObj = (HashMap<String, Object>) JSONUtils.parseJSONFromFile(jsonFlowFile);
 		
+		Project project = new Project(1, "test");
 		Flow flow = Flow.flowFromObject(flowObj);
-		ExecutableFlow execFlow = new ExecutableFlow(flow);
+		ExecutableFlow execFlow = new ExecutableFlow(project, flow);
 		execFlow.setExecutionId(execId);
 		execFlow.setExecutionPath(workingDir.getPath());
 		return execFlow;
diff --git a/unit/java/azkaban/test/execapp/FlowRunnerPipelineTest.java b/unit/java/azkaban/test/execapp/FlowRunnerPipelineTest.java
new file mode 100644
index 0000000..9df97ac
--- /dev/null
+++ b/unit/java/azkaban/test/execapp/FlowRunnerPipelineTest.java
@@ -0,0 +1,620 @@
+package azkaban.test.execapp;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import junit.framework.Assert;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.log4j.Logger;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import azkaban.execapp.FlowRunner;
+import azkaban.execapp.event.FlowWatcher;
+import azkaban.execapp.event.LocalFlowWatcher;
+import azkaban.executor.ExecutableFlow;
+import azkaban.executor.ExecutableFlowBase;
+import azkaban.executor.ExecutableNode;
+import azkaban.executor.ExecutionOptions;
+import azkaban.executor.ExecutorLoader;
+import azkaban.executor.Status;
+import azkaban.flow.Flow;
+import azkaban.jobtype.JobTypeManager;
+import azkaban.project.Project;
+import azkaban.project.ProjectLoader;
+import azkaban.project.ProjectManagerException;
+import azkaban.test.executor.InteractiveTestJob;
+import azkaban.test.executor.JavaJob;
+import azkaban.utils.DirectoryFlowLoader;
+
+/**
+ * Flows in this test:
+ * joba 
+ * jobb
+ * joba1
+ * jobc->joba
+ * jobd->joba
+ * jobe->jobb,jobc,jobd
+ * jobf->jobe,joba1
+ * 
+ * jobb = innerFlow
+ * innerJobA
+ * innerJobB->innerJobA
+ * innerJobC->innerJobB
+ * innerFlow->innerJobB,innerJobC
+ * 
+ * jobd=innerFlow2
+ * innerFlow2->innerJobA
+ * @author rpark
+ *
+ */
+public class FlowRunnerPipelineTest {
+	private File workingDir;
+	private JobTypeManager jobtypeManager;
+	private ProjectLoader fakeProjectLoader;
+	private ExecutorLoader fakeExecutorLoader;
+	private Logger logger = Logger.getLogger(FlowRunnerTest2.class);
+	private Project project;
+	private Map<String, Flow> flowMap;
+	private static int id=101;
+	
+	public FlowRunnerPipelineTest() {
+	}
+	
+	@Before
+	public void setUp() throws Exception {
+		System.out.println("Create temp dir");
+		workingDir = new File("_AzkabanTestDir_" + System.currentTimeMillis());
+		if (workingDir.exists()) {
+			FileUtils.deleteDirectory(workingDir);
+		}
+		workingDir.mkdirs();
+		jobtypeManager = new JobTypeManager(null, this.getClass().getClassLoader());
+		jobtypeManager.registerJobType("java", JavaJob.class);
+		jobtypeManager.registerJobType("test", InteractiveTestJob.class);
+		fakeProjectLoader = new MockProjectLoader(workingDir);
+		fakeExecutorLoader = new MockExecutorLoader();
+		project = new Project(1, "testProject");
+		
+		File dir = new File("unit/executions/embedded2");
+		prepareProject(dir);
+		
+		InteractiveTestJob.clearTestJobs();
+	}
+	
+	@After
+	public void tearDown() throws IOException {
+		System.out.println("Teardown temp dir");
+		if (workingDir != null) {
+			FileUtils.deleteDirectory(workingDir);
+			workingDir = null;
+		}
+	}
+	
+	@Test
+	public void testBasicPipelineLevel1Run() throws Exception {
+		EventCollectorListener eventCollector = new EventCollectorListener();
+		FlowRunner previousRunner = createFlowRunner(eventCollector, "jobf", "prev");
+		
+		ExecutionOptions options = new ExecutionOptions();
+		options.setPipelineExecutionId(previousRunner.getExecutableFlow().getExecutionId());
+		options.setPipelineLevel(1);
+		FlowWatcher watcher = new LocalFlowWatcher(previousRunner);
+		FlowRunner pipelineRunner = createFlowRunner(eventCollector, "jobf", "pipe", options);
+		pipelineRunner.setFlowWatcher(watcher);
+		
+		Map<String, Status> previousExpectedStateMap = new HashMap<String, Status>();
+		Map<String, Status> pipelineExpectedStateMap = new HashMap<String, Status>();
+		Map<String, ExecutableNode> previousNodeMap = new HashMap<String, ExecutableNode>();
+		Map<String, ExecutableNode> pipelineNodeMap = new HashMap<String, ExecutableNode>();
+		
+		// 1. START FLOW
+		ExecutableFlow pipelineFlow = pipelineRunner.getExecutableFlow();
+		ExecutableFlow previousFlow = previousRunner.getExecutableFlow();
+		createExpectedStateMap(previousFlow, previousExpectedStateMap, previousNodeMap);
+		createExpectedStateMap(pipelineFlow, pipelineExpectedStateMap, pipelineNodeMap);
+		
+		Thread thread1 = runFlowRunnerInThread(previousRunner);
+		pause(250);
+		Thread thread2 = runFlowRunnerInThread(pipelineRunner);
+		pause(500);
+		
+		previousExpectedStateMap.put("joba", Status.RUNNING);
+		previousExpectedStateMap.put("joba1", Status.RUNNING);
+		pipelineExpectedStateMap.put("joba", Status.QUEUED);
+		pipelineExpectedStateMap.put("joba1", Status.QUEUED);
+		compareStates(previousExpectedStateMap, previousNodeMap);
+		compareStates(pipelineExpectedStateMap, pipelineNodeMap);
+		
+		InteractiveTestJob.getTestJob("prev:joba").succeedJob();
+		pause(250);
+		previousExpectedStateMap.put("joba", Status.SUCCEEDED);
+		previousExpectedStateMap.put("jobb", Status.RUNNING);
+		previousExpectedStateMap.put("jobb:innerJobA", Status.RUNNING);
+		previousExpectedStateMap.put("jobd", Status.RUNNING);
+		previousExpectedStateMap.put("jobc", Status.RUNNING);
+		previousExpectedStateMap.put("jobd:innerJobA", Status.RUNNING);
+		pipelineExpectedStateMap.put("joba", Status.RUNNING);
+		compareStates(previousExpectedStateMap, previousNodeMap);
+		compareStates(pipelineExpectedStateMap, pipelineNodeMap);
+
+		InteractiveTestJob.getTestJob("prev:jobb:innerJobA").succeedJob();
+		pause(250);
+		previousExpectedStateMap.put("jobb:innerJobA", Status.SUCCEEDED);
+		previousExpectedStateMap.put("jobb:innerJobB", Status.RUNNING);
+		previousExpectedStateMap.put("jobb:innerJobC", Status.RUNNING);
+		compareStates(previousExpectedStateMap, previousNodeMap);
+		compareStates(pipelineExpectedStateMap, pipelineNodeMap);
+		
+		InteractiveTestJob.getTestJob("pipe:joba").succeedJob();
+		pause(250);
+		pipelineExpectedStateMap.put("joba", Status.SUCCEEDED);
+		pipelineExpectedStateMap.put("jobb", Status.RUNNING);
+		pipelineExpectedStateMap.put("jobd", Status.RUNNING);
+		pipelineExpectedStateMap.put("jobc", Status.QUEUED);
+		pipelineExpectedStateMap.put("jobd:innerJobA", Status.QUEUED);
+		pipelineExpectedStateMap.put("jobb:innerJobA", Status.RUNNING);
+		compareStates(previousExpectedStateMap, previousNodeMap);
+		compareStates(pipelineExpectedStateMap, pipelineNodeMap);
+		
+		InteractiveTestJob.getTestJob("prev:jobd:innerJobA").succeedJob();
+		pause(250);
+		previousExpectedStateMap.put("jobd:innerJobA", Status.SUCCEEDED);
+		previousExpectedStateMap.put("jobd:innerFlow2", Status.RUNNING);
+		pipelineExpectedStateMap.put("jobd:innerJobA", Status.RUNNING);
+		compareStates(previousExpectedStateMap, previousNodeMap);
+		compareStates(pipelineExpectedStateMap, pipelineNodeMap);
+		
+		// Finish the previous d side
+		InteractiveTestJob.getTestJob("prev:jobd:innerFlow2").succeedJob();
+		pause(250);
+		previousExpectedStateMap.put("jobd:innerFlow2", Status.SUCCEEDED);
+		previousExpectedStateMap.put("jobd", Status.SUCCEEDED);
+		compareStates(previousExpectedStateMap, previousNodeMap);
+		
+		InteractiveTestJob.getTestJob("prev:jobb:innerJobB").succeedJob();
+		InteractiveTestJob.getTestJob("prev:jobb:innerJobC").succeedJob();
+		InteractiveTestJob.getTestJob("prev:jobc").succeedJob();
+		pause(250);
+		InteractiveTestJob.getTestJob("pipe:jobb:innerJobA").succeedJob();
+		pause(250);
+		previousExpectedStateMap.put("jobb:innerJobB", Status.SUCCEEDED);
+		previousExpectedStateMap.put("jobb:innerJobC", Status.SUCCEEDED);
+		previousExpectedStateMap.put("jobb:innerFlow", Status.RUNNING);
+		previousExpectedStateMap.put("jobc", Status.SUCCEEDED);
+		pipelineExpectedStateMap.put("jobb:innerJobA", Status.SUCCEEDED);
+		pipelineExpectedStateMap.put("jobc", Status.RUNNING);
+		pipelineExpectedStateMap.put("jobb:innerJobB", Status.RUNNING);
+		pipelineExpectedStateMap.put("jobb:innerJobC", Status.RUNNING);
+		compareStates(previousExpectedStateMap, previousNodeMap);
+		compareStates(pipelineExpectedStateMap, pipelineNodeMap);
+		
+		InteractiveTestJob.getTestJob("prev:jobb:innerFlow").succeedJob();
+		InteractiveTestJob.getTestJob("pipe:jobc").succeedJob();
+		pause(250);
+		previousExpectedStateMap.put("jobb:innerFlow", Status.SUCCEEDED);
+		previousExpectedStateMap.put("jobb", Status.SUCCEEDED);
+		previousExpectedStateMap.put("jobe", Status.RUNNING);
+		pipelineExpectedStateMap.put("jobc", Status.SUCCEEDED);
+		
+		compareStates(previousExpectedStateMap, previousNodeMap);
+		compareStates(pipelineExpectedStateMap, pipelineNodeMap);
+		
+		InteractiveTestJob.getTestJob("pipe:jobb:innerJobB").succeedJob();
+		InteractiveTestJob.getTestJob("pipe:jobb:innerJobC").succeedJob();
+		InteractiveTestJob.getTestJob("prev:jobe").succeedJob();
+		pause(250);
+		previousExpectedStateMap.put("jobe", Status.SUCCEEDED);
+		pipelineExpectedStateMap.put("jobb:innerJobB", Status.SUCCEEDED);
+		pipelineExpectedStateMap.put("jobb:innerJobC", Status.SUCCEEDED);
+		pipelineExpectedStateMap.put("jobb:innerFlow", Status.RUNNING);
+		
+		compareStates(previousExpectedStateMap, previousNodeMap);
+		compareStates(pipelineExpectedStateMap, pipelineNodeMap);
+
+		InteractiveTestJob.getTestJob("pipe:jobd:innerJobA").succeedJob();
+		InteractiveTestJob.getTestJob("pipe:jobb:innerFlow").succeedJob();
+		pause(250);
+		pipelineExpectedStateMap.put("jobb", Status.SUCCEEDED);
+		pipelineExpectedStateMap.put("jobd:innerJobA", Status.SUCCEEDED);
+		pipelineExpectedStateMap.put("jobb:innerFlow", Status.SUCCEEDED);
+		pipelineExpectedStateMap.put("jobd:innerFlow2", Status.RUNNING);
+		compareStates(pipelineExpectedStateMap, pipelineNodeMap);
+		
+		InteractiveTestJob.getTestJob("pipe:jobd:innerFlow2").succeedJob();
+		InteractiveTestJob.getTestJob("prev:joba1").succeedJob();
+		pause(250);
+		pipelineExpectedStateMap.put("jobd:innerFlow2", Status.SUCCEEDED);
+		pipelineExpectedStateMap.put("jobd", Status.SUCCEEDED);
+		previousExpectedStateMap.put("jobf", Status.RUNNING);
+		previousExpectedStateMap.put("joba1", Status.SUCCEEDED);
+		pipelineExpectedStateMap.put("joba1", Status.RUNNING);
+		pipelineExpectedStateMap.put("jobe", Status.RUNNING);
+		compareStates(pipelineExpectedStateMap, pipelineNodeMap);
+		compareStates(previousExpectedStateMap, previousNodeMap);
+		
+		InteractiveTestJob.getTestJob("pipe:jobe").succeedJob();
+		InteractiveTestJob.getTestJob("prev:jobf").succeedJob();
+		pause(250);
+		pipelineExpectedStateMap.put("jobe", Status.SUCCEEDED);
+		previousExpectedStateMap.put("jobf", Status.SUCCEEDED);
+		Assert.assertEquals(Status.SUCCEEDED, previousFlow.getStatus());
+		compareStates(previousExpectedStateMap, previousNodeMap);
+		compareStates(pipelineExpectedStateMap, pipelineNodeMap);
+		
+		InteractiveTestJob.getTestJob("pipe:joba1").succeedJob();
+		pause(250);
+		pipelineExpectedStateMap.put("joba1", Status.SUCCEEDED);
+		pipelineExpectedStateMap.put("jobf", Status.RUNNING);
+		compareStates(pipelineExpectedStateMap, pipelineNodeMap);
+		
+		InteractiveTestJob.getTestJob("pipe:jobf").succeedJob();
+		pause(250);
+		Assert.assertEquals(Status.SUCCEEDED, pipelineFlow.getStatus());
+		Assert.assertFalse(thread1.isAlive());
+		Assert.assertFalse(thread2.isAlive());
+	}
+	
+	@Test
+	public void testBasicPipelineLevel2Run() throws Exception {
+		EventCollectorListener eventCollector = new EventCollectorListener();
+		FlowRunner previousRunner = createFlowRunner(eventCollector, "pipelineFlow", "prev");
+		
+		ExecutionOptions options = new ExecutionOptions();
+		options.setPipelineExecutionId(previousRunner.getExecutableFlow().getExecutionId());
+		options.setPipelineLevel(2);
+		FlowWatcher watcher = new LocalFlowWatcher(previousRunner);
+		FlowRunner pipelineRunner = createFlowRunner(eventCollector, "pipelineFlow", "pipe", options);
+		pipelineRunner.setFlowWatcher(watcher);
+		
+		Map<String, Status> previousExpectedStateMap = new HashMap<String, Status>();
+		Map<String, Status> pipelineExpectedStateMap = new HashMap<String, Status>();
+		Map<String, ExecutableNode> previousNodeMap = new HashMap<String, ExecutableNode>();
+		Map<String, ExecutableNode> pipelineNodeMap = new HashMap<String, ExecutableNode>();
+		
+		// 1. START FLOW
+		ExecutableFlow pipelineFlow = pipelineRunner.getExecutableFlow();
+		ExecutableFlow previousFlow = previousRunner.getExecutableFlow();
+		createExpectedStateMap(previousFlow, previousExpectedStateMap, previousNodeMap);
+		createExpectedStateMap(pipelineFlow, pipelineExpectedStateMap, pipelineNodeMap);
+		
+		Thread thread1 = runFlowRunnerInThread(previousRunner);
+		pause(250);
+		Thread thread2 = runFlowRunnerInThread(pipelineRunner);
+		pause(250);
+		
+		previousExpectedStateMap.put("pipeline1", Status.RUNNING);
+		pipelineExpectedStateMap.put("pipeline1", Status.QUEUED);
+		compareStates(previousExpectedStateMap, previousNodeMap);
+		compareStates(pipelineExpectedStateMap, pipelineNodeMap);
+		
+		InteractiveTestJob.getTestJob("prev:pipeline1").succeedJob();
+		pause(250);
+		previousExpectedStateMap.put("pipeline1", Status.SUCCEEDED);
+		previousExpectedStateMap.put("pipeline2", Status.RUNNING);
+		compareStates(previousExpectedStateMap, previousNodeMap);
+		compareStates(pipelineExpectedStateMap, pipelineNodeMap);
+		
+		InteractiveTestJob.getTestJob("prev:pipeline2").succeedJob();
+		pause(250);
+		previousExpectedStateMap.put("pipeline2", Status.SUCCEEDED);
+		previousExpectedStateMap.put("pipelineEmbeddedFlow3", Status.RUNNING);
+		previousExpectedStateMap.put("pipelineEmbeddedFlow3:innerJobA", Status.RUNNING);
+		pipelineExpectedStateMap.put("pipeline1", Status.RUNNING);
+		compareStates(previousExpectedStateMap, previousNodeMap);
+		compareStates(pipelineExpectedStateMap, pipelineNodeMap);
+		
+		InteractiveTestJob.getTestJob("pipe:pipeline1").succeedJob();
+		pause(250);
+		pipelineExpectedStateMap.put("pipeline1", Status.SUCCEEDED);
+		pipelineExpectedStateMap.put("pipeline2", Status.QUEUED);
+		compareStates(previousExpectedStateMap, previousNodeMap);
+		compareStates(pipelineExpectedStateMap, pipelineNodeMap);
+		
+		InteractiveTestJob.getTestJob("prev:pipelineEmbeddedFlow3:innerJobA").succeedJob();
+		pause(250);
+		previousExpectedStateMap.put("pipelineEmbeddedFlow3:innerJobA", Status.SUCCEEDED);
+		previousExpectedStateMap.put("pipelineEmbeddedFlow3:innerJobB", Status.RUNNING);
+		previousExpectedStateMap.put("pipelineEmbeddedFlow3:innerJobC", Status.RUNNING);
+		pipelineExpectedStateMap.put("pipeline2", Status.RUNNING);
+		compareStates(previousExpectedStateMap, previousNodeMap);
+		compareStates(pipelineExpectedStateMap, pipelineNodeMap);
+
+		InteractiveTestJob.getTestJob("pipe:pipeline2").succeedJob();
+		pause(250);
+		pipelineExpectedStateMap.put("pipeline2", Status.SUCCEEDED);
+		pipelineExpectedStateMap.put("pipelineEmbeddedFlow3", Status.RUNNING);
+		pipelineExpectedStateMap.put("pipelineEmbeddedFlow3:innerJobA", Status.QUEUED);
+		compareStates(previousExpectedStateMap, previousNodeMap);
+		compareStates(pipelineExpectedStateMap, pipelineNodeMap);
+		
+		InteractiveTestJob.getTestJob("prev:pipelineEmbeddedFlow3:innerJobB").succeedJob();
+		pause(250);
+		previousExpectedStateMap.put("pipelineEmbeddedFlow3:innerJobB", Status.SUCCEEDED);
+		compareStates(previousExpectedStateMap, previousNodeMap);
+		compareStates(pipelineExpectedStateMap, pipelineNodeMap);
+		
+		InteractiveTestJob.getTestJob("prev:pipelineEmbeddedFlow3:innerJobC").succeedJob();
+		pause(250);
+		previousExpectedStateMap.put("pipelineEmbeddedFlow3:innerFlow", Status.RUNNING);
+		previousExpectedStateMap.put("pipelineEmbeddedFlow3:innerJobC", Status.SUCCEEDED);
+		pipelineExpectedStateMap.put("pipelineEmbeddedFlow3:innerJobA", Status.RUNNING);
+		compareStates(previousExpectedStateMap, previousNodeMap);
+		compareStates(pipelineExpectedStateMap, pipelineNodeMap);
+
+		InteractiveTestJob.getTestJob("pipe:pipelineEmbeddedFlow3:innerJobA").succeedJob();
+		pause(250);
+		pipelineExpectedStateMap.put("pipelineEmbeddedFlow3:innerJobA", Status.SUCCEEDED);
+		pipelineExpectedStateMap.put("pipelineEmbeddedFlow3:innerJobC", Status.QUEUED);
+		pipelineExpectedStateMap.put("pipelineEmbeddedFlow3:innerJobB", Status.QUEUED);
+		previousExpectedStateMap.put("pipelineEmbeddedFlow3:innerFlow", Status.RUNNING);
+		compareStates(previousExpectedStateMap, previousNodeMap);
+		compareStates(pipelineExpectedStateMap, pipelineNodeMap);
+		
+		InteractiveTestJob.getTestJob("prev:pipelineEmbeddedFlow3:innerFlow").succeedJob();
+		pause(250);
+		previousExpectedStateMap.put("pipelineEmbeddedFlow3:innerFlow", Status.SUCCEEDED);
+		previousExpectedStateMap.put("pipelineEmbeddedFlow3", Status.SUCCEEDED);
+		previousExpectedStateMap.put("pipeline4", Status.RUNNING);
+		pipelineExpectedStateMap.put("pipelineEmbeddedFlow3:innerJobC", Status.RUNNING);
+		pipelineExpectedStateMap.put("pipelineEmbeddedFlow3:innerJobB", Status.RUNNING);
+		compareStates(previousExpectedStateMap, previousNodeMap);
+		compareStates(pipelineExpectedStateMap, pipelineNodeMap);
+		
+		InteractiveTestJob.getTestJob("pipe:pipelineEmbeddedFlow3:innerJobB").succeedJob();
+		InteractiveTestJob.getTestJob("pipe:pipelineEmbeddedFlow3:innerJobC").succeedJob();
+		pause(250);
+		pipelineExpectedStateMap.put("pipelineEmbeddedFlow3:innerJobC", Status.SUCCEEDED);
+		pipelineExpectedStateMap.put("pipelineEmbeddedFlow3:innerJobB", Status.SUCCEEDED);
+		pipelineExpectedStateMap.put("pipelineEmbeddedFlow3:innerFlow", Status.QUEUED);
+		compareStates(previousExpectedStateMap, previousNodeMap);
+		compareStates(pipelineExpectedStateMap, pipelineNodeMap);
+		
+		InteractiveTestJob.getTestJob("prev:pipeline4").succeedJob();
+		pause(250);
+		previousExpectedStateMap.put("pipeline4", Status.SUCCEEDED);
+		previousExpectedStateMap.put("pipelineFlow", Status.RUNNING);
+		pipelineExpectedStateMap.put("pipelineEmbeddedFlow3:innerFlow", Status.RUNNING);
+		compareStates(previousExpectedStateMap, previousNodeMap);
+		compareStates(pipelineExpectedStateMap, pipelineNodeMap);
+		
+		InteractiveTestJob.getTestJob("prev:pipelineFlow").succeedJob();
+		pause(250);
+		previousExpectedStateMap.put("pipelineFlow", Status.SUCCEEDED);
+		Assert.assertEquals(Status.SUCCEEDED, previousFlow.getStatus());
+		Assert.assertFalse(thread1.isAlive());
+		compareStates(previousExpectedStateMap, previousNodeMap);
+		compareStates(pipelineExpectedStateMap, pipelineNodeMap);
+		
+		InteractiveTestJob.getTestJob("pipe:pipelineEmbeddedFlow3:innerFlow").succeedJob();
+		pause(250);
+		pipelineExpectedStateMap.put("pipelineEmbeddedFlow3:innerFlow", Status.SUCCEEDED);
+		pipelineExpectedStateMap.put("pipelineEmbeddedFlow3", Status.SUCCEEDED);
+		pipelineExpectedStateMap.put("pipeline4", Status.RUNNING);
+		compareStates(previousExpectedStateMap, previousNodeMap);
+		compareStates(pipelineExpectedStateMap, pipelineNodeMap);
+		
+		InteractiveTestJob.getTestJob("pipe:pipeline4").succeedJob();
+		pause(250);
+		pipelineExpectedStateMap.put("pipeline4", Status.SUCCEEDED);
+		pipelineExpectedStateMap.put("pipelineFlow", Status.RUNNING);
+		compareStates(previousExpectedStateMap, previousNodeMap);
+		compareStates(pipelineExpectedStateMap, pipelineNodeMap);
+		
+		InteractiveTestJob.getTestJob("pipe:pipelineFlow").succeedJob();
+		pause(250);
+		pipelineExpectedStateMap.put("pipelineFlow", Status.SUCCEEDED);
+		compareStates(previousExpectedStateMap, previousNodeMap);
+		compareStates(pipelineExpectedStateMap, pipelineNodeMap);
+		Assert.assertEquals(Status.SUCCEEDED, pipelineFlow.getStatus());
+		Assert.assertFalse(thread2.isAlive());
+	}
+	
+	@Test
+	public void testBasicPipelineLevel2Run2() throws Exception {
+		EventCollectorListener eventCollector = new EventCollectorListener();
+		FlowRunner previousRunner = createFlowRunner(eventCollector, "pipeline1_2", "prev");
+		
+		ExecutionOptions options = new ExecutionOptions();
+		options.setPipelineExecutionId(previousRunner.getExecutableFlow().getExecutionId());
+		options.setPipelineLevel(2);
+		FlowWatcher watcher = new LocalFlowWatcher(previousRunner);
+		FlowRunner pipelineRunner = createFlowRunner(eventCollector, "pipeline1_2", "pipe", options);
+		pipelineRunner.setFlowWatcher(watcher);
+		
+		Map<String, Status> previousExpectedStateMap = new HashMap<String, Status>();
+		Map<String, Status> pipelineExpectedStateMap = new HashMap<String, Status>();
+		Map<String, ExecutableNode> previousNodeMap = new HashMap<String, ExecutableNode>();
+		Map<String, ExecutableNode> pipelineNodeMap = new HashMap<String, ExecutableNode>();
+		
+		// 1. START FLOW
+		ExecutableFlow pipelineFlow = pipelineRunner.getExecutableFlow();
+		ExecutableFlow previousFlow = previousRunner.getExecutableFlow();
+		createExpectedStateMap(previousFlow, previousExpectedStateMap, previousNodeMap);
+		createExpectedStateMap(pipelineFlow, pipelineExpectedStateMap, pipelineNodeMap);
+		
+		Thread thread1 = runFlowRunnerInThread(previousRunner);
+		pause(250);
+		Thread thread2 = runFlowRunnerInThread(pipelineRunner);
+		pause(250);
+		
+		previousExpectedStateMap.put("pipeline1_1", Status.RUNNING);
+		previousExpectedStateMap.put("pipeline1_1:innerJobA", Status.RUNNING);
+		pipelineExpectedStateMap.put("pipeline1_1", Status.RUNNING);
+		pipelineExpectedStateMap.put("pipeline1_1:innerJobA", Status.QUEUED);
+		compareStates(previousExpectedStateMap, previousNodeMap);
+		compareStates(pipelineExpectedStateMap, pipelineNodeMap);
+		
+		InteractiveTestJob.getTestJob("prev:pipeline1_1:innerJobA").succeedJob();
+		pause(250);
+		previousExpectedStateMap.put("pipeline1_1:innerJobA", Status.SUCCEEDED);
+		previousExpectedStateMap.put("pipeline1_1:innerFlow2", Status.RUNNING);
+		compareStates(previousExpectedStateMap, previousNodeMap);
+		compareStates(pipelineExpectedStateMap, pipelineNodeMap);
+		
+		InteractiveTestJob.getTestJob("prev:pipeline1_1:innerFlow2").succeedJob();
+		pause(250);
+		previousExpectedStateMap.put("pipeline1_1", Status.SUCCEEDED);
+		previousExpectedStateMap.put("pipeline1_1:innerFlow2", Status.SUCCEEDED);
+		previousExpectedStateMap.put("pipeline1_2", Status.RUNNING);
+		previousExpectedStateMap.put("pipeline1_2:innerJobA", Status.RUNNING);
+		pipelineExpectedStateMap.put("pipeline1_1:innerJobA", Status.RUNNING);
+		compareStates(previousExpectedStateMap, previousNodeMap);
+		compareStates(pipelineExpectedStateMap, pipelineNodeMap);
+		
+		InteractiveTestJob.getTestJob("pipe:pipeline1_1:innerJobA").succeedJob();
+		pause(250);
+		pipelineExpectedStateMap.put("pipeline1_1:innerJobA", Status.SUCCEEDED);
+		pipelineExpectedStateMap.put("pipeline1_1:innerFlow2", Status.QUEUED);
+		compareStates(previousExpectedStateMap, previousNodeMap);
+		compareStates(pipelineExpectedStateMap, pipelineNodeMap);
+		
+		InteractiveTestJob.getTestJob("prev:pipeline1_2:innerJobA").succeedJob();
+		pause(250);
+		previousExpectedStateMap.put("pipeline1_2:innerJobA", Status.SUCCEEDED);
+		previousExpectedStateMap.put("pipeline1_2:innerFlow2", Status.RUNNING);
+		pipelineExpectedStateMap.put("pipeline1_1:innerFlow2", Status.RUNNING);
+		compareStates(previousExpectedStateMap, previousNodeMap);
+		compareStates(pipelineExpectedStateMap, pipelineNodeMap);
+		
+		InteractiveTestJob.getTestJob("pipe:pipeline1_1:innerFlow2").succeedJob();
+		pause(250);
+		pipelineExpectedStateMap.put("pipeline1_1", Status.SUCCEEDED);
+		pipelineExpectedStateMap.put("pipeline1_1:innerFlow2", Status.SUCCEEDED);
+		pipelineExpectedStateMap.put("pipeline1_2", Status.RUNNING);
+		pipelineExpectedStateMap.put("pipeline1_2:innerJobA", Status.QUEUED);
+		compareStates(previousExpectedStateMap, previousNodeMap);
+		compareStates(pipelineExpectedStateMap, pipelineNodeMap);
+		
+		InteractiveTestJob.getTestJob("pipe:pipeline1_1:innerFlow2").succeedJob();
+		pause(250);
+		pipelineExpectedStateMap.put("pipeline1_1", Status.SUCCEEDED);
+		pipelineExpectedStateMap.put("pipeline1_1:innerFlow2", Status.SUCCEEDED);
+		pipelineExpectedStateMap.put("pipeline1_2", Status.RUNNING);
+		pipelineExpectedStateMap.put("pipeline1_2:innerJobA", Status.QUEUED);
+		compareStates(previousExpectedStateMap, previousNodeMap);
+		compareStates(pipelineExpectedStateMap, pipelineNodeMap);
+		
+		InteractiveTestJob.getTestJob("prev:pipeline1_2:innerFlow2").succeedJob();
+		pause(250);
+		previousExpectedStateMap.put("pipeline1_2:innerFlow2", Status.SUCCEEDED);
+		previousExpectedStateMap.put("pipeline1_2", Status.SUCCEEDED);
+		Assert.assertEquals(Status.SUCCEEDED, previousFlow.getStatus());
+		Assert.assertFalse(thread1.isAlive());
+		pipelineExpectedStateMap.put("pipeline1_2:innerJobA", Status.RUNNING);
+		compareStates(previousExpectedStateMap, previousNodeMap);
+		compareStates(pipelineExpectedStateMap, pipelineNodeMap);
+		
+		InteractiveTestJob.getTestJob("pipe:pipeline1_2:innerJobA").succeedJob();
+		pause(250);
+		pipelineExpectedStateMap.put("pipeline1_2:innerJobA", Status.SUCCEEDED);
+		pipelineExpectedStateMap.put("pipeline1_2:innerFlow2", Status.RUNNING);
+		compareStates(previousExpectedStateMap, previousNodeMap);
+		compareStates(pipelineExpectedStateMap, pipelineNodeMap);
+		
+		InteractiveTestJob.getTestJob("pipe:pipeline1_2:innerFlow2").succeedJob();
+		pause(250);
+		pipelineExpectedStateMap.put("pipeline1_2", Status.SUCCEEDED);
+		pipelineExpectedStateMap.put("pipeline1_2:innerFlow2", Status.SUCCEEDED);
+		compareStates(previousExpectedStateMap, previousNodeMap);
+		compareStates(pipelineExpectedStateMap, pipelineNodeMap);
+		Assert.assertEquals(Status.SUCCEEDED, pipelineFlow.getStatus());
+		Assert.assertFalse(thread2.isAlive());
+	}
+	
+	private Thread runFlowRunnerInThread(FlowRunner runner) {
+		Thread thread = new Thread(runner);
+		thread.start();
+		return thread;
+	}
+	
+	private void pause(long millisec) {
+		synchronized(this) {
+			try {
+				wait(millisec);
+			}
+			catch (InterruptedException e) {
+			}
+		}
+	}
+	
+	private void createExpectedStateMap(ExecutableFlowBase flow, Map<String, Status> expectedStateMap, Map<String, ExecutableNode> nodeMap) {
+		for (ExecutableNode node: flow.getExecutableNodes()) {
+			expectedStateMap.put(node.getNestedId(), node.getStatus());
+			nodeMap.put(node.getNestedId(), node);
+			
+			if (node instanceof ExecutableFlowBase) {
+				createExpectedStateMap((ExecutableFlowBase)node, expectedStateMap, nodeMap);
+			}
+		}
+	}
+	
+	private void compareStates(Map<String, Status> expectedStateMap, Map<String, ExecutableNode> nodeMap) {
+		for (String printedId: expectedStateMap.keySet()) {
+			Status expectedStatus = expectedStateMap.get(printedId);
+			ExecutableNode node = nodeMap.get(printedId);
+			if (node == null) {
+				System.out.println("id node: " + printedId + " doesn't exist.");
+			}
+			if (expectedStatus != node.getStatus()) {
+				Assert.fail("Expected values do not match for " + printedId + ". Expected " + expectedStatus + ", instead received " + node.getStatus());
+			}
+		}
+	}
+	
+	private void prepareProject(File directory) throws ProjectManagerException, IOException {
+		DirectoryFlowLoader loader = new DirectoryFlowLoader(logger);
+		loader.loadProjectFlow(directory);
+		if (!loader.getErrors().isEmpty()) {
+			for (String error: loader.getErrors()) {
+				System.out.println(error);
+			}
+			
+			throw new RuntimeException("Errors found in setup");
+		}
+		
+		flowMap = loader.getFlowMap();
+		project.setFlows(flowMap);
+		FileUtils.copyDirectory(directory, workingDir);
+	}
+	
+//	private void printCurrentState(String prefix, ExecutableFlowBase flow) {
+//		for(ExecutableNode node: flow.getExecutableNodes()) {
+//
+//			System.err.println(prefix + node.getNestedId() + "->" + node.getStatus().name());
+//			if (node instanceof ExecutableFlowBase) {
+//				printCurrentState(prefix, (ExecutableFlowBase)node);
+//			}
+//		}
+//	}
+//	
+	private FlowRunner createFlowRunner(EventCollectorListener eventCollector, String flowName, String groupName) throws Exception {
+		return createFlowRunner(eventCollector, flowName, groupName, new ExecutionOptions());
+	}
+	
+	private FlowRunner createFlowRunner(EventCollectorListener eventCollector, String flowName, String groupName, ExecutionOptions options) throws Exception {
+		Flow flow = flowMap.get(flowName);
+
+		int exId = id++;
+		ExecutableFlow exFlow = new ExecutableFlow(project, flow);
+		exFlow.setExecutionPath(workingDir.getPath());
+		exFlow.setExecutionId(exId);
+
+		Map<String, String> flowParam = new HashMap<String, String>();
+		flowParam.put("group", groupName);
+		options.addAllFlowParameters(flowParam);
+		exFlow.setExecutionOptions(options);
+		fakeExecutorLoader.uploadExecutableFlow(exFlow);
+	
+		FlowRunner runner = new FlowRunner(fakeExecutorLoader.fetchExecutableFlow(exId), fakeExecutorLoader, fakeProjectLoader, jobtypeManager);
+
+		runner.addListener(eventCollector);
+		
+		return runner;
+	}
+
+}
diff --git a/unit/java/azkaban/test/execapp/FlowRunnerTest.java b/unit/java/azkaban/test/execapp/FlowRunnerTest.java
index 7628c13..42026fc 100644
--- a/unit/java/azkaban/test/execapp/FlowRunnerTest.java
+++ b/unit/java/azkaban/test/execapp/FlowRunnerTest.java
@@ -21,7 +21,9 @@ import azkaban.executor.Status;
 
 import azkaban.flow.Flow;
 import azkaban.jobtype.JobTypeManager;
+import azkaban.project.Project;
 import azkaban.project.ProjectLoader;
+import azkaban.test.executor.InteractiveTestJob;
 import azkaban.test.executor.JavaJob;
 import azkaban.utils.JSONUtils;
 
@@ -36,22 +38,29 @@ public class FlowRunnerTest {
 	@Before
 	public void setUp() throws Exception {
 		System.out.println("Create temp dir");
-		workingDir = new File("_AzkabanTestDir_" + System.currentTimeMillis());
-		if (workingDir.exists()) {
-			FileUtils.deleteDirectory(workingDir);
+		synchronized ( this) {
+			workingDir = new File("_AzkabanTestDir_" + System.currentTimeMillis());
+			if (workingDir.exists()) {
+				FileUtils.deleteDirectory(workingDir);
+			}
+			workingDir.mkdirs();
 		}
-		workingDir.mkdirs();
 		jobtypeManager = new JobTypeManager(null, this.getClass().getClassLoader());
 		jobtypeManager.registerJobType("java", JavaJob.class);
+		jobtypeManager.registerJobType("test", InteractiveTestJob.class);
 		fakeProjectLoader = new MockProjectLoader(workingDir);
+		
+		InteractiveTestJob.clearTestJobs();
 	}
 	
 	@After
 	public void tearDown() throws IOException {
 		System.out.println("Teardown temp dir");
-		if (workingDir != null) {
-			FileUtils.deleteDirectory(workingDir);
-			workingDir = null;
+		synchronized ( this) {
+			if (workingDir != null) {
+				FileUtils.deleteDirectory(workingDir);
+				workingDir = null;
+			}
 		}
 	}
 	
@@ -190,6 +199,14 @@ public class FlowRunnerTest {
 		
 		Assert.assertTrue("Expected flow " + Status.FAILED + " instead " + exFlow.getStatus(), exFlow.getStatus() == Status.FAILED);
 		
+		synchronized(this) {
+			try {
+				wait(500);
+			} catch(InterruptedException e) {
+				
+			}
+		}
+
 		testStatus(exFlow, "job1", Status.SUCCEEDED);
 		testStatus(exFlow, "job2d", Status.FAILED);
 		testStatus(exFlow, "job3", Status.KILLED);
@@ -225,6 +242,14 @@ public class FlowRunnerTest {
 		ExecutableFlow exFlow = runner.getExecutableFlow();
 		Assert.assertTrue("Expected flow " + Status.FAILED + " instead " + exFlow.getStatus(), exFlow.getStatus() == Status.FAILED);
 		
+		synchronized(this) {
+			try {
+				wait(500);
+			} catch(InterruptedException e) {
+				
+			}
+		}
+		
 		testStatus(exFlow, "job1", Status.SUCCEEDED);
 		testStatus(exFlow, "job2d", Status.FAILED);
 		testStatus(exFlow, "job3", Status.SUCCEEDED);
@@ -259,7 +284,7 @@ public class FlowRunnerTest {
 
 		synchronized(this) {
 			try {
-				wait(4500);
+				wait(5000);
 			} catch (InterruptedException e) {
 				// TODO Auto-generated catch block
 				e.printStackTrace();
@@ -273,7 +298,7 @@ public class FlowRunnerTest {
 		synchronized(this) {
 			// Wait for cleanup.
 			try {
-				wait(1000);
+				wait(2000);
 			} catch (InterruptedException e) {
 				// TODO Auto-generated catch block
 				e.printStackTrace();
@@ -326,7 +351,7 @@ public class FlowRunnerTest {
 		ExecutableNode node = flow.getExecutableNode(name);
 		
 		if (node.getStatus() != status) {
-			Assert.fail("Status of job " + node.getJobId() + " is " + node.getStatus() + " not " + status + " as expected.");
+			Assert.fail("Status of job " + node.getId() + " is " + node.getStatus() + " not " + status + " as expected.");
 		}
 	}
 	
@@ -339,14 +364,19 @@ public class FlowRunnerTest {
 	}
 	
 	private ExecutableFlow prepareExecDir(File execDir, String flowName, int execId) throws IOException {
-		FileUtils.copyDirectory(execDir, workingDir);
+		synchronized ( this) {
+			FileUtils.copyDirectory(execDir, workingDir);
+		}
 		
 		File jsonFlowFile = new File(workingDir, flowName + ".flow");
 		@SuppressWarnings("unchecked")
 		HashMap<String, Object> flowObj = (HashMap<String, Object>) JSONUtils.parseJSONFromFile(jsonFlowFile);
 		
+		Project project = new Project(1, "myproject");
+		project.setVersion(2);
+
 		Flow flow = Flow.flowFromObject(flowObj);
-		ExecutableFlow execFlow = new ExecutableFlow(flow);
+		ExecutableFlow execFlow = new ExecutableFlow(project, flow);
 		execFlow.setExecutionId(execId);
 		execFlow.setExecutionPath(workingDir.getPath());
 		return execFlow;
@@ -372,7 +402,7 @@ public class FlowRunnerTest {
 		
 		//System.out.println("Node " + node.getJobId() + " start:" + startTime + " end:" + endTime + " previous:" + previousEndTime);
 		Assert.assertTrue("Checking start and end times", startTime > 0 && endTime >= startTime);
-		Assert.assertTrue("Start time for " + node.getJobId() + " is " + startTime +" and less than " + previousEndTime, startTime >= previousEndTime);
+		Assert.assertTrue("Start time for " + node.getId() + " is " + startTime +" and less than " + previousEndTime, startTime >= previousEndTime);
 		
 		for (String outNode : node.getOutNodes()) {
 			ExecutableNode childNode = flow.getExecutableNode(outNode);
diff --git a/unit/java/azkaban/test/execapp/FlowRunnerTest2.java b/unit/java/azkaban/test/execapp/FlowRunnerTest2.java
new file mode 100644
index 0000000..0ac1fb6
--- /dev/null
+++ b/unit/java/azkaban/test/execapp/FlowRunnerTest2.java
@@ -0,0 +1,896 @@
+package azkaban.test.execapp;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import junit.framework.Assert;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.log4j.Logger;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import azkaban.execapp.FlowRunner;
+import azkaban.executor.ExecutableFlow;
+import azkaban.executor.ExecutableFlowBase;
+import azkaban.executor.ExecutableNode;
+import azkaban.executor.ExecutionOptions.FailureAction;
+import azkaban.executor.ExecutorLoader;
+import azkaban.executor.Status;
+import azkaban.flow.Flow;
+import azkaban.jobtype.JobTypeManager;
+import azkaban.project.Project;
+import azkaban.project.ProjectLoader;
+import azkaban.project.ProjectManagerException;
+import azkaban.test.executor.InteractiveTestJob;
+import azkaban.test.executor.JavaJob;
+import azkaban.utils.DirectoryFlowLoader;
+import azkaban.utils.Props;
+
+public class FlowRunnerTest2 {
+	private File workingDir;
+	private JobTypeManager jobtypeManager;
+	private ProjectLoader fakeProjectLoader;
+	private ExecutorLoader fakeExecutorLoader;
+	private Logger logger = Logger.getLogger(FlowRunnerTest2.class);
+	private Project project;
+	private Map<String, Flow> flowMap;
+	private static int id=101;
+	
+	public FlowRunnerTest2() {
+	}
+	
+	@Before
+	public void setUp() throws Exception {
+		System.out.println("Create temp dir");
+		workingDir = new File("_AzkabanTestDir_" + System.currentTimeMillis());
+		if (workingDir.exists()) {
+			FileUtils.deleteDirectory(workingDir);
+		}
+		workingDir.mkdirs();
+		jobtypeManager = new JobTypeManager(null, this.getClass().getClassLoader());
+		jobtypeManager.registerJobType("java", JavaJob.class);
+		jobtypeManager.registerJobType("test", InteractiveTestJob.class);
+		fakeProjectLoader = new MockProjectLoader(workingDir);
+		fakeExecutorLoader = new MockExecutorLoader();
+		project = new Project(1, "testProject");
+		
+		File dir = new File("unit/executions/embedded2");
+		prepareProject(dir);
+		
+		InteractiveTestJob.clearTestJobs();
+	}
+	
+	@After
+	public void tearDown() throws IOException {
+		System.out.println("Teardown temp dir");
+		if (workingDir != null) {
+			FileUtils.deleteDirectory(workingDir);
+			workingDir = null;
+		}
+	}
+	
+	@Test
+	public void testBasicRun() throws Exception {
+		EventCollectorListener eventCollector = new EventCollectorListener();
+		FlowRunner runner = createFlowRunner(eventCollector, "jobf");
+		
+		Map<String, Status> expectedStateMap = new HashMap<String, Status>();
+		Map<String, ExecutableNode> nodeMap = new HashMap<String, ExecutableNode>();
+		
+		// 1. START FLOW
+		ExecutableFlow flow = runner.getExecutableFlow();
+		createExpectedStateMap(flow, expectedStateMap, nodeMap);
+		Thread thread = runFlowRunnerInThread(runner);
+		pause(250);
+		
+		// After it starts up, only joba should be running
+		expectedStateMap.put("joba", Status.RUNNING);
+		expectedStateMap.put("joba1", Status.RUNNING);
+		
+		compareStates(expectedStateMap, nodeMap);
+		Props joba = nodeMap.get("joba").getInputProps();
+		Assert.assertEquals("joba.1", joba.get("param1"));
+		Assert.assertEquals("test1.2", joba.get("param2"));
+		Assert.assertEquals("test1.3", joba.get("param3"));
+		Assert.assertEquals("override.4", joba.get("param4"));
+		Assert.assertEquals("test2.5", joba.get("param5"));
+		Assert.assertEquals("test2.6", joba.get("param6"));
+		Assert.assertEquals("test2.7", joba.get("param7"));
+		Assert.assertEquals("test2.8", joba.get("param8"));
+		
+		Props joba1 = nodeMap.get("joba1").getInputProps();
+		Assert.assertEquals("test1.1", joba1.get("param1"));
+		Assert.assertEquals("test1.2", joba1.get("param2"));
+		Assert.assertEquals("test1.3", joba1.get("param3"));
+		Assert.assertEquals("override.4", joba1.get("param4"));
+		Assert.assertEquals("test2.5", joba1.get("param5"));
+		Assert.assertEquals("test2.6", joba1.get("param6"));
+		Assert.assertEquals("test2.7", joba1.get("param7"));
+		Assert.assertEquals("test2.8", joba1.get("param8"));
+		
+		// 2. JOB A COMPLETES SUCCESSFULLY 
+		InteractiveTestJob.getTestJob("joba").succeedJob(Props.of("output.joba", "joba", "output.override", "joba"));
+		pause(250);
+		expectedStateMap.put("joba", Status.SUCCEEDED);
+		expectedStateMap.put("joba1", Status.RUNNING);
+		expectedStateMap.put("jobb", Status.RUNNING);
+		expectedStateMap.put("jobc", Status.RUNNING);
+		expectedStateMap.put("jobd", Status.RUNNING);
+		expectedStateMap.put("jobd:innerJobA", Status.RUNNING);
+		expectedStateMap.put("jobb:innerJobA", Status.RUNNING);
+		compareStates(expectedStateMap, nodeMap);
+		
+		Props jobb = nodeMap.get("jobb").getInputProps();
+		Assert.assertEquals("test1.1", jobb.get("param1"));
+		Assert.assertEquals("test1.1", jobb.get("param1"));
+		Assert.assertEquals("test1.2", jobb.get("param2"));
+		Assert.assertEquals("test1.3", jobb.get("param3"));
+		Assert.assertEquals("override.4", jobb.get("param4"));
+		Assert.assertEquals("test2.5", jobb.get("param5"));
+		Assert.assertEquals("test2.6", jobb.get("param6"));
+		Assert.assertEquals("test2.7", jobb.get("param7"));
+		Assert.assertEquals("test2.8", jobb.get("param8"));
+		Assert.assertEquals("test2.8", jobb.get("param8"));
+		// Test that jobb properties overwrites the output properties
+		Assert.assertEquals("moo", jobb.get("testprops"));
+		Assert.assertEquals("jobb", jobb.get("output.override"));
+		Assert.assertEquals("joba", jobb.get("output.joba"));
+		
+		Props jobbInnerJobA = nodeMap.get("jobb:innerJobA").getInputProps();
+		Assert.assertEquals("test1.1", jobbInnerJobA.get("param1"));
+		Assert.assertEquals("test1.2", jobbInnerJobA.get("param2"));
+		Assert.assertEquals("test1.3", jobbInnerJobA.get("param3"));
+		Assert.assertEquals("override.4", jobbInnerJobA.get("param4"));
+		Assert.assertEquals("test2.5", jobbInnerJobA.get("param5"));
+		Assert.assertEquals("test2.6", jobbInnerJobA.get("param6"));
+		Assert.assertEquals("test2.7", jobbInnerJobA.get("param7"));
+		Assert.assertEquals("test2.8", jobbInnerJobA.get("param8"));
+		Assert.assertEquals("joba", jobbInnerJobA.get("output.joba"));
+		
+		// 3. jobb:Inner completes
+		/// innerJobA completes
+		InteractiveTestJob.getTestJob("jobb:innerJobA").succeedJob(Props.of("output.jobb.innerJobA", "jobb.innerJobA"));
+		pause(250);
+		expectedStateMap.put("jobb:innerJobA", Status.SUCCEEDED);
+		expectedStateMap.put("jobb:innerJobB", Status.RUNNING);
+		expectedStateMap.put("jobb:innerJobC", Status.RUNNING);
+		compareStates(expectedStateMap, nodeMap);
+		Props jobbInnerJobB = nodeMap.get("jobb:innerJobB").getInputProps();
+		Assert.assertEquals("test1.1", jobbInnerJobB.get("param1"));
+		Assert.assertEquals("override.4", jobbInnerJobB.get("param4"));
+		Assert.assertEquals("jobb.innerJobA", jobbInnerJobB.get("output.jobb.innerJobA"));
+		Assert.assertEquals("moo", jobbInnerJobB.get("testprops"));
+		/// innerJobB, C completes
+		InteractiveTestJob.getTestJob("jobb:innerJobB").succeedJob(Props.of("output.jobb.innerJobB", "jobb.innerJobB"));
+		InteractiveTestJob.getTestJob("jobb:innerJobC").succeedJob(Props.of("output.jobb.innerJobC", "jobb.innerJobC"));
+		pause(250);
+		expectedStateMap.put("jobb:innerJobB", Status.SUCCEEDED);
+		expectedStateMap.put("jobb:innerJobC", Status.SUCCEEDED);
+		expectedStateMap.put("jobb:innerFlow", Status.RUNNING);
+		compareStates(expectedStateMap, nodeMap);
+		
+		Props jobbInnerJobD = nodeMap.get("jobb:innerFlow").getInputProps();
+		Assert.assertEquals("test1.1", jobbInnerJobD.get("param1"));
+		Assert.assertEquals("override.4", jobbInnerJobD.get("param4"));
+		Assert.assertEquals("jobb.innerJobB", jobbInnerJobD.get("output.jobb.innerJobB"));
+		Assert.assertEquals("jobb.innerJobC", jobbInnerJobD.get("output.jobb.innerJobC"));
+		
+		// 4. Finish up on inner flow for jobb
+		InteractiveTestJob.getTestJob("jobb:innerFlow").succeedJob(Props.of("output1.jobb", "test1", "output2.jobb", "test2"));
+		pause(250);
+		expectedStateMap.put("jobb:innerFlow", Status.SUCCEEDED);
+		expectedStateMap.put("jobb", Status.SUCCEEDED);
+		compareStates(expectedStateMap, nodeMap);
+		Props jobbOutput = nodeMap.get("jobb").getOutputProps();
+		Assert.assertEquals("test1", jobbOutput.get("output1.jobb"));
+		Assert.assertEquals("test2", jobbOutput.get("output2.jobb"));
+		
+		// 5. Finish jobc, jobd
+		InteractiveTestJob.getTestJob("jobc").succeedJob(Props.of("output.jobc", "jobc"));
+		pause(250);
+		expectedStateMap.put("jobc", Status.SUCCEEDED);
+		compareStates(expectedStateMap, nodeMap);
+		InteractiveTestJob.getTestJob("jobd:innerJobA").succeedJob();
+		pause(250);
+		InteractiveTestJob.getTestJob("jobd:innerFlow2").succeedJob();
+		pause(250);
+		expectedStateMap.put("jobd:innerJobA", Status.SUCCEEDED);
+		expectedStateMap.put("jobd:innerFlow2", Status.SUCCEEDED);
+		expectedStateMap.put("jobd", Status.SUCCEEDED);
+		expectedStateMap.put("jobe", Status.RUNNING);
+		compareStates(expectedStateMap, nodeMap);
+		
+		Props jobd = nodeMap.get("jobe").getInputProps();
+		Assert.assertEquals("test1", jobd.get("output1.jobb"));
+		Assert.assertEquals("jobc", jobd.get("output.jobc"));
+		
+		// 6. Finish off flow
+		InteractiveTestJob.getTestJob("joba1").succeedJob();
+		pause(250);
+		InteractiveTestJob.getTestJob("jobe").succeedJob();
+		pause(250);
+		expectedStateMap.put("joba1", Status.SUCCEEDED);
+		expectedStateMap.put("jobe", Status.SUCCEEDED);
+		expectedStateMap.put("jobf", Status.RUNNING);
+		compareStates(expectedStateMap, nodeMap);
+		
+		InteractiveTestJob.getTestJob("jobf").succeedJob();
+		pause(250);
+		expectedStateMap.put("jobf", Status.SUCCEEDED);
+		compareStates(expectedStateMap, nodeMap);
+		Assert.assertEquals(Status.SUCCEEDED, flow.getStatus());
+		
+		Assert.assertFalse(thread.isAlive());
+	}
+	
+	@Test
+	public void testDisabledNormal() throws Exception {
+		EventCollectorListener eventCollector = new EventCollectorListener();
+		FlowRunner runner = createFlowRunner(eventCollector, "jobf");
+		ExecutableFlow flow = runner.getExecutableFlow();
+		Map<String, Status> expectedStateMap = new HashMap<String, Status>();
+		Map<String, ExecutableNode> nodeMap = new HashMap<String, ExecutableNode>();
+		flow.getExecutableNode("jobb").setStatus(Status.DISABLED);
+		((ExecutableFlowBase)flow.getExecutableNode("jobd")).getExecutableNode("innerJobA").setStatus(Status.DISABLED);
+		
+		// 1. START FLOW
+		createExpectedStateMap(flow, expectedStateMap, nodeMap);
+		Thread thread = runFlowRunnerInThread(runner);
+		pause(250);
+		
+		// After it starts up, only joba should be running
+		expectedStateMap.put("joba", Status.RUNNING);
+		expectedStateMap.put("joba1", Status.RUNNING);
+		compareStates(expectedStateMap, nodeMap);
+
+		// 2. JOB A COMPLETES SUCCESSFULLY, others should be skipped
+		InteractiveTestJob.getTestJob("joba").succeedJob();
+		pause(250);
+		expectedStateMap.put("joba", Status.SUCCEEDED);
+		expectedStateMap.put("joba1", Status.RUNNING);
+		expectedStateMap.put("jobb", Status.SKIPPED);
+		expectedStateMap.put("jobc", Status.RUNNING);
+		expectedStateMap.put("jobd", Status.RUNNING);
+		expectedStateMap.put("jobd:innerJobA", Status.SKIPPED);
+		expectedStateMap.put("jobd:innerFlow2", Status.RUNNING);
+		expectedStateMap.put("jobb:innerJobA", Status.READY);
+		expectedStateMap.put("jobb:innerJobB", Status.READY);
+		expectedStateMap.put("jobb:innerJobC", Status.READY);
+		expectedStateMap.put("jobb:innerFlow", Status.READY);
+		compareStates(expectedStateMap, nodeMap);
+
+		// 3. jobb:Inner completes
+		/// innerJobA completes
+		InteractiveTestJob.getTestJob("jobc").succeedJob();
+		InteractiveTestJob.getTestJob("jobd:innerFlow2").succeedJob();
+		pause(250);
+		expectedStateMap.put("jobd:innerFlow2", Status.SUCCEEDED);
+		expectedStateMap.put("jobd", Status.SUCCEEDED);
+		expectedStateMap.put("jobc", Status.SUCCEEDED);
+		expectedStateMap.put("jobe", Status.RUNNING);
+		compareStates(expectedStateMap, nodeMap);
+
+		InteractiveTestJob.getTestJob("jobe").succeedJob();
+		InteractiveTestJob.getTestJob("joba1").succeedJob();
+		pause(250);
+		expectedStateMap.put("jobe", Status.SUCCEEDED);
+		expectedStateMap.put("joba1", Status.SUCCEEDED);
+		expectedStateMap.put("jobf", Status.RUNNING);
+		compareStates(expectedStateMap, nodeMap);
+		
+		// 4. Finish up on inner flow for jobb
+		InteractiveTestJob.getTestJob("jobf").succeedJob();
+		pause(250);
+		expectedStateMap.put("jobf", Status.SUCCEEDED);
+		compareStates(expectedStateMap, nodeMap);
+		
+		Assert.assertEquals(Status.SUCCEEDED, flow.getStatus());
+		Assert.assertFalse(thread.isAlive());
+	}
+	
+	@Test
+	public void testNormalFailure1() throws Exception {
+		// Test propagation of KILLED status to embedded flows.
+		EventCollectorListener eventCollector = new EventCollectorListener();
+		FlowRunner runner = createFlowRunner(eventCollector, "jobf");
+		ExecutableFlow flow = runner.getExecutableFlow();
+		Map<String, Status> expectedStateMap = new HashMap<String, Status>();
+		Map<String, ExecutableNode> nodeMap = new HashMap<String, ExecutableNode>();
+		
+		// 1. START FLOW
+		createExpectedStateMap(flow, expectedStateMap, nodeMap);
+		Thread thread = runFlowRunnerInThread(runner);
+		pause(250);
+		
+		// After it starts up, only joba should be running
+		expectedStateMap.put("joba", Status.RUNNING);
+		expectedStateMap.put("joba1", Status.RUNNING);
+		compareStates(expectedStateMap, nodeMap);
+
+		// 2. JOB A COMPLETES SUCCESSFULLY, others should be skipped
+		InteractiveTestJob.getTestJob("joba").failJob();
+		pause(250);
+		Assert.assertEquals(Status.FAILED_FINISHING, flow.getStatus());
+		expectedStateMap.put("joba", Status.FAILED);
+		expectedStateMap.put("joba1", Status.RUNNING);
+		expectedStateMap.put("jobb", Status.KILLED);
+		expectedStateMap.put("jobc", Status.KILLED);
+		expectedStateMap.put("jobd", Status.KILLED);
+		expectedStateMap.put("jobd:innerJobA", Status.READY);
+		expectedStateMap.put("jobd:innerFlow2", Status.READY);
+		expectedStateMap.put("jobb:innerJobA", Status.READY);
+		expectedStateMap.put("jobb:innerFlow", Status.READY);
+		expectedStateMap.put("jobe", Status.KILLED);
+		compareStates(expectedStateMap, nodeMap);
+
+		// 3. jobb:Inner completes
+		/// innerJobA completes
+		InteractiveTestJob.getTestJob("joba1").succeedJob();
+		pause(250);
+		expectedStateMap.put("jobf", Status.KILLED);
+		Assert.assertEquals(Status.FAILED, flow.getStatus());
+		Assert.assertFalse(thread.isAlive());
+	}
+	
+	@Test
+	public void testNormalFailure2() throws Exception {
+		// Test propagation of KILLED status to embedded flows different branch
+		EventCollectorListener eventCollector = new EventCollectorListener();
+		FlowRunner runner = createFlowRunner(eventCollector, "jobf");
+		ExecutableFlow flow = runner.getExecutableFlow();
+		Map<String, Status> expectedStateMap = new HashMap<String, Status>();
+		Map<String, ExecutableNode> nodeMap = new HashMap<String, ExecutableNode>();
+		
+		// 1. START FLOW
+		createExpectedStateMap(flow, expectedStateMap, nodeMap);
+		Thread thread = runFlowRunnerInThread(runner);
+		pause(250);
+		
+		// After it starts up, only joba should be running
+		expectedStateMap.put("joba", Status.RUNNING);
+		expectedStateMap.put("joba1", Status.RUNNING);
+		compareStates(expectedStateMap, nodeMap);
+
+		// 2. JOB A COMPLETES SUCCESSFULLY, others should be skipped
+		InteractiveTestJob.getTestJob("joba").succeedJob();
+		pause(250);
+		expectedStateMap.put("joba", Status.SUCCEEDED);
+		expectedStateMap.put("jobb", Status.RUNNING);
+		expectedStateMap.put("jobc", Status.RUNNING);
+		expectedStateMap.put("jobd", Status.RUNNING);
+		expectedStateMap.put("jobb:innerJobA", Status.RUNNING);
+		expectedStateMap.put("jobc", Status.RUNNING);
+		expectedStateMap.put("jobd:innerJobA", Status.RUNNING);
+		
+		InteractiveTestJob.getTestJob("joba1").failJob();
+		pause(250);
+		expectedStateMap.put("joba1", Status.FAILED);
+		compareStates(expectedStateMap, nodeMap);
+
+		// 3. joba completes, everything is killed
+		InteractiveTestJob.getTestJob("jobb:innerJobA").succeedJob();
+		InteractiveTestJob.getTestJob("jobd:innerJobA").succeedJob();
+		pause(250);
+		expectedStateMap.put("jobb:innerJobA", Status.SUCCEEDED);
+		expectedStateMap.put("jobd:innerJobA", Status.SUCCEEDED);
+		expectedStateMap.put("jobb:innerJobB", Status.KILLED);
+		expectedStateMap.put("jobb:innerJobC", Status.KILLED);
+		expectedStateMap.put("jobb:innerFlow", Status.KILLED);
+		expectedStateMap.put("jobd:innerFlow2", Status.KILLED);
+		expectedStateMap.put("jobb", Status.KILLED);
+		expectedStateMap.put("jobd", Status.KILLED);
+		compareStates(expectedStateMap, nodeMap);
+		Assert.assertEquals(Status.FAILED_FINISHING, flow.getStatus());
+		
+		InteractiveTestJob.getTestJob("jobc").succeedJob();
+		pause(250);
+		expectedStateMap.put("jobc", Status.SUCCEEDED);
+		expectedStateMap.put("jobe", Status.KILLED);
+		expectedStateMap.put("jobf", Status.KILLED);
+		compareStates(expectedStateMap, nodeMap);
+		Assert.assertEquals(Status.FAILED, flow.getStatus());
+		
+		Assert.assertFalse(thread.isAlive());
+	}
+	
+	@Test
+	public void testNormalFailure3() throws Exception {
+		// Test propagation of KILLED status to embedded flows different branch
+		EventCollectorListener eventCollector = new EventCollectorListener();
+		FlowRunner runner = createFlowRunner(eventCollector, "jobf");
+		ExecutableFlow flow = runner.getExecutableFlow();
+		Map<String, Status> expectedStateMap = new HashMap<String, Status>();
+		Map<String, ExecutableNode> nodeMap = new HashMap<String, ExecutableNode>();
+		
+		// 1. START FLOW
+		createExpectedStateMap(flow, expectedStateMap, nodeMap);
+		Thread thread = runFlowRunnerInThread(runner);
+		pause(250);
+		
+		// After it starts up, only joba should be running
+		expectedStateMap.put("joba", Status.RUNNING);
+		expectedStateMap.put("joba1", Status.RUNNING);
+		compareStates(expectedStateMap, nodeMap);
+
+		// 2. JOB in subflow FAILS
+		InteractiveTestJob.getTestJob("joba").succeedJob();
+		pause(250);
+		expectedStateMap.put("joba", Status.SUCCEEDED);
+		expectedStateMap.put("jobb", Status.RUNNING);
+		expectedStateMap.put("jobc", Status.RUNNING);
+		expectedStateMap.put("jobd", Status.RUNNING);
+		expectedStateMap.put("jobb:innerJobA", Status.RUNNING);
+		expectedStateMap.put("jobd:innerJobA", Status.RUNNING);
+		compareStates(expectedStateMap, nodeMap);
+		
+		InteractiveTestJob.getTestJob("joba1").succeedJob();
+		InteractiveTestJob.getTestJob("jobb:innerJobA").succeedJob();
+		pause(250);
+		expectedStateMap.put("jobb:innerJobA", Status.SUCCEEDED);
+		expectedStateMap.put("joba1", Status.SUCCEEDED);
+		expectedStateMap.put("jobb:innerJobB", Status.RUNNING);
+		expectedStateMap.put("jobb:innerJobC", Status.RUNNING);
+		compareStates(expectedStateMap, nodeMap);
+		
+		InteractiveTestJob.getTestJob("jobb:innerJobB").failJob();
+		pause(250);
+		expectedStateMap.put("jobb", Status.FAILED_FINISHING);
+		expectedStateMap.put("jobb:innerJobB", Status.FAILED);
+		Assert.assertEquals(Status.FAILED_FINISHING, flow.getStatus());
+		compareStates(expectedStateMap, nodeMap);
+
+		InteractiveTestJob.getTestJob("jobb:innerJobC").succeedJob();
+		InteractiveTestJob.getTestJob("jobd:innerJobA").succeedJob();
+		pause(250);
+		expectedStateMap.put("jobd:innerJobA", Status.SUCCEEDED);
+		expectedStateMap.put("jobd:innerFlow2", Status.KILLED);
+		expectedStateMap.put("jobd", Status.KILLED);
+		expectedStateMap.put("jobb:innerJobC", Status.SUCCEEDED);
+		expectedStateMap.put("jobb:innerFlow", Status.KILLED);
+		expectedStateMap.put("jobb", Status.FAILED);
+		compareStates(expectedStateMap, nodeMap);
+
+		// 3. jobc completes, everything is killed
+		InteractiveTestJob.getTestJob("jobc").succeedJob();
+		pause(250);
+		expectedStateMap.put("jobc", Status.SUCCEEDED);
+		expectedStateMap.put("jobe", Status.KILLED);
+		expectedStateMap.put("jobf", Status.KILLED);
+		compareStates(expectedStateMap, nodeMap);
+		Assert.assertEquals(Status.FAILED, flow.getStatus());
+		
+		Assert.assertFalse(thread.isAlive());
+	}
+	
+	@Test
+	public void testFailedFinishingFailure3() throws Exception {
+		// Test propagation of KILLED status to embedded flows different branch
+		EventCollectorListener eventCollector = new EventCollectorListener();
+		FlowRunner runner = createFlowRunner(eventCollector, "jobf", FailureAction.FINISH_ALL_POSSIBLE);
+		ExecutableFlow flow = runner.getExecutableFlow();
+		Map<String, Status> expectedStateMap = new HashMap<String, Status>();
+		Map<String, ExecutableNode> nodeMap = new HashMap<String, ExecutableNode>();
+		
+		// 1. START FLOW
+		createExpectedStateMap(flow, expectedStateMap, nodeMap);
+		Thread thread = runFlowRunnerInThread(runner);
+		pause(250);
+		
+		// After it starts up, only joba should be running
+		expectedStateMap.put("joba", Status.RUNNING);
+		expectedStateMap.put("joba1", Status.RUNNING);
+		compareStates(expectedStateMap, nodeMap);
+
+		// 2. JOB in subflow FAILS
+		InteractiveTestJob.getTestJob("joba").succeedJob();
+		pause(250);
+		expectedStateMap.put("joba", Status.SUCCEEDED);
+		expectedStateMap.put("jobb", Status.RUNNING);
+		expectedStateMap.put("jobc", Status.RUNNING);
+		expectedStateMap.put("jobd", Status.RUNNING);
+		expectedStateMap.put("jobb:innerJobA", Status.RUNNING);
+		expectedStateMap.put("jobd:innerJobA", Status.RUNNING);
+		compareStates(expectedStateMap, nodeMap);
+		
+		InteractiveTestJob.getTestJob("joba1").succeedJob();
+		InteractiveTestJob.getTestJob("jobb:innerJobA").succeedJob();
+		pause(250);
+		expectedStateMap.put("jobb:innerJobA", Status.SUCCEEDED);
+		expectedStateMap.put("joba1", Status.SUCCEEDED);
+		expectedStateMap.put("jobb:innerJobB", Status.RUNNING);
+		expectedStateMap.put("jobb:innerJobC", Status.RUNNING);
+		compareStates(expectedStateMap, nodeMap);
+		
+		InteractiveTestJob.getTestJob("jobb:innerJobB").failJob();
+		pause(250);
+		expectedStateMap.put("jobb", Status.FAILED_FINISHING);
+		expectedStateMap.put("jobb:innerJobB", Status.FAILED);
+		Assert.assertEquals(Status.FAILED_FINISHING, flow.getStatus());
+		compareStates(expectedStateMap, nodeMap);
+
+		InteractiveTestJob.getTestJob("jobb:innerJobC").succeedJob();
+		InteractiveTestJob.getTestJob("jobd:innerJobA").succeedJob();
+		pause(250);
+		expectedStateMap.put("jobb", Status.FAILED);
+		expectedStateMap.put("jobd:innerJobA", Status.SUCCEEDED);
+		expectedStateMap.put("jobd:innerFlow2", Status.RUNNING);
+		expectedStateMap.put("jobb:innerJobC", Status.SUCCEEDED);
+		expectedStateMap.put("jobb:innerFlow", Status.KILLED);
+		compareStates(expectedStateMap, nodeMap);
+
+		InteractiveTestJob.getTestJob("jobd:innerFlow2").succeedJob();
+		pause(250);
+		expectedStateMap.put("jobd:innerFlow2", Status.SUCCEEDED);
+		expectedStateMap.put("jobd", Status.SUCCEEDED);
+		
+		// 3. jobc completes, everything is killed
+		InteractiveTestJob.getTestJob("jobc").succeedJob();
+		pause(250);
+		expectedStateMap.put("jobc", Status.SUCCEEDED);
+		expectedStateMap.put("jobe", Status.KILLED);
+		expectedStateMap.put("jobf", Status.KILLED);
+		compareStates(expectedStateMap, nodeMap);
+		Assert.assertEquals(Status.FAILED, flow.getStatus());
+		Assert.assertFalse(thread.isAlive());
+	}
+	
+	@Test
+	public void testCancelOnFailure() throws Exception {
+		// Test propagation of KILLED status to embedded flows different branch
+		EventCollectorListener eventCollector = new EventCollectorListener();
+		FlowRunner runner = createFlowRunner(eventCollector, "jobf", FailureAction.CANCEL_ALL);
+		ExecutableFlow flow = runner.getExecutableFlow();
+		Map<String, Status> expectedStateMap = new HashMap<String, Status>();
+		Map<String, ExecutableNode> nodeMap = new HashMap<String, ExecutableNode>();
+		
+		// 1. START FLOW
+		createExpectedStateMap(flow, expectedStateMap, nodeMap);
+		Thread thread = runFlowRunnerInThread(runner);
+		pause(250);
+		
+		// After it starts up, only joba should be running
+		expectedStateMap.put("joba", Status.RUNNING);
+		expectedStateMap.put("joba1", Status.RUNNING);
+		compareStates(expectedStateMap, nodeMap);
+
+		// 2. JOB in subflow FAILS
+		InteractiveTestJob.getTestJob("joba").succeedJob();
+		pause(250);
+		expectedStateMap.put("joba", Status.SUCCEEDED);
+		expectedStateMap.put("jobb", Status.RUNNING);
+		expectedStateMap.put("jobc", Status.RUNNING);
+		expectedStateMap.put("jobd", Status.RUNNING);
+		expectedStateMap.put("jobb:innerJobA", Status.RUNNING);
+		expectedStateMap.put("jobd:innerJobA", Status.RUNNING);
+		compareStates(expectedStateMap, nodeMap);
+		
+		InteractiveTestJob.getTestJob("joba1").succeedJob();
+		InteractiveTestJob.getTestJob("jobb:innerJobA").succeedJob();
+		pause(250);
+		expectedStateMap.put("jobb:innerJobA", Status.SUCCEEDED);
+		expectedStateMap.put("joba1", Status.SUCCEEDED);
+		expectedStateMap.put("jobb:innerJobB", Status.RUNNING);
+		expectedStateMap.put("jobb:innerJobC", Status.RUNNING);
+		compareStates(expectedStateMap, nodeMap);
+		
+		InteractiveTestJob.getTestJob("jobb:innerJobB").failJob();
+		pause(250);
+		expectedStateMap.put("jobb", Status.FAILED);
+		expectedStateMap.put("jobb:innerJobB", Status.FAILED);
+		expectedStateMap.put("jobb:innerJobC", Status.FAILED);
+		expectedStateMap.put("jobb:innerFlow", Status.KILLED);
+		expectedStateMap.put("jobc", Status.FAILED);
+		expectedStateMap.put("jobd", Status.KILLED);
+		expectedStateMap.put("jobd:innerJobA", Status.FAILED);
+		expectedStateMap.put("jobd:innerFlow2", Status.KILLED);
+		expectedStateMap.put("jobe", Status.KILLED);
+		expectedStateMap.put("jobf", Status.KILLED);
+		compareStates(expectedStateMap, nodeMap);
+		
+		Assert.assertFalse(thread.isAlive());
+		Assert.assertEquals(Status.FAILED, flow.getStatus());
+
+	}
+	
+	@Test
+	public void testRetryOnFailure() throws Exception {
+		// Test propagation of KILLED status to embedded flows different branch
+		EventCollectorListener eventCollector = new EventCollectorListener();
+		FlowRunner runner = createFlowRunner(eventCollector, "jobf");
+		ExecutableFlow flow = runner.getExecutableFlow();
+		Map<String, Status> expectedStateMap = new HashMap<String, Status>();
+		Map<String, ExecutableNode> nodeMap = new HashMap<String, ExecutableNode>();
+		flow.getExecutableNode("joba").setStatus(Status.DISABLED);
+		((ExecutableFlowBase)flow.getExecutableNode("jobb")).getExecutableNode("innerFlow").setStatus(Status.DISABLED);
+		
+		// 1. START FLOW
+		createExpectedStateMap(flow, expectedStateMap, nodeMap);
+		Thread thread = runFlowRunnerInThread(runner);
+		pause(250);
+		
+		// After it starts up, only joba should be running
+		expectedStateMap.put("joba", Status.SKIPPED);
+		expectedStateMap.put("joba1", Status.RUNNING);
+		expectedStateMap.put("jobb", Status.RUNNING);
+		expectedStateMap.put("jobc", Status.RUNNING);
+		expectedStateMap.put("jobd", Status.RUNNING);
+		expectedStateMap.put("jobb:innerJobA", Status.RUNNING);
+		expectedStateMap.put("jobd:innerJobA", Status.RUNNING);
+		compareStates(expectedStateMap, nodeMap);
+
+		InteractiveTestJob.getTestJob("jobb:innerJobA").succeedJob();
+		pause(250);
+		expectedStateMap.put("jobb:innerJobA", Status.SUCCEEDED);
+		expectedStateMap.put("jobb:innerJobB", Status.RUNNING);
+		expectedStateMap.put("jobb:innerJobC", Status.RUNNING);
+		compareStates(expectedStateMap, nodeMap);
+		
+		InteractiveTestJob.getTestJob("jobb:innerJobB").failJob();
+		InteractiveTestJob.getTestJob("jobb:innerJobC").failJob();
+		pause(250);
+		InteractiveTestJob.getTestJob("jobd:innerJobA").succeedJob();
+		pause(250);
+		expectedStateMap.put("jobb", Status.FAILED);
+		expectedStateMap.put("jobb:innerJobB", Status.FAILED);
+		expectedStateMap.put("jobb:innerJobC", Status.FAILED);
+		expectedStateMap.put("jobb:innerFlow", Status.SKIPPED);
+		expectedStateMap.put("jobd:innerFlow2", Status.KILLED);
+		expectedStateMap.put("jobd:innerJobA", Status.SUCCEEDED);
+		expectedStateMap.put("jobd", Status.KILLED);
+		Assert.assertEquals(Status.FAILED_FINISHING, flow.getStatus());
+		compareStates(expectedStateMap, nodeMap);
+		
+		runner.retryFailures("me");
+		pause(500);
+		expectedStateMap.put("jobb:innerJobB", Status.RUNNING);
+		expectedStateMap.put("jobb:innerJobC", Status.RUNNING);
+		expectedStateMap.put("jobb", Status.RUNNING);
+		expectedStateMap.put("jobd", Status.RUNNING);
+		expectedStateMap.put("jobb:innerFlow", Status.DISABLED);
+		expectedStateMap.put("jobd:innerFlow2", Status.RUNNING);
+		Assert.assertEquals(Status.RUNNING, flow.getStatus());
+		compareStates(expectedStateMap, nodeMap);
+		Assert.assertTrue(thread.isAlive());
+		
+		InteractiveTestJob.getTestJob("jobb:innerJobB").succeedJob();
+		InteractiveTestJob.getTestJob("jobb:innerJobC").succeedJob();
+		InteractiveTestJob.getTestJob("jobd:innerFlow2").succeedJob();
+		InteractiveTestJob.getTestJob("jobc").succeedJob();
+		pause(250);
+		expectedStateMap.put("jobb:innerFlow", Status.SKIPPED);
+		expectedStateMap.put("jobb", Status.SUCCEEDED);
+		expectedStateMap.put("jobb:innerJobB", Status.SUCCEEDED);
+		expectedStateMap.put("jobb:innerJobC", Status.SUCCEEDED);
+		expectedStateMap.put("jobc", Status.SUCCEEDED);
+		expectedStateMap.put("jobd", Status.SUCCEEDED);
+		expectedStateMap.put("jobd:innerFlow2", Status.SUCCEEDED);
+		expectedStateMap.put("jobe", Status.RUNNING);
+		compareStates(expectedStateMap, nodeMap);
+		
+		InteractiveTestJob.getTestJob("jobe").succeedJob();
+		pause(250);
+		expectedStateMap.put("jobe", Status.SUCCEEDED);
+		compareStates(expectedStateMap, nodeMap);
+		
+		InteractiveTestJob.getTestJob("joba1").succeedJob();
+		pause(250);
+		expectedStateMap.put("joba1", Status.SUCCEEDED);
+		expectedStateMap.put("jobf", Status.RUNNING);
+		compareStates(expectedStateMap, nodeMap);
+		
+		InteractiveTestJob.getTestJob("jobf").succeedJob();
+		pause(250);
+		expectedStateMap.put("jobf", Status.SUCCEEDED);
+		compareStates(expectedStateMap, nodeMap);
+		Assert.assertEquals(Status.SUCCEEDED, flow.getStatus());
+		Assert.assertFalse(thread.isAlive());
+	}
+	
+	@Test
+	public void testCancel() throws Exception {
+		// Test propagation of KILLED status to embedded flows different branch
+		EventCollectorListener eventCollector = new EventCollectorListener();
+		FlowRunner runner = createFlowRunner(eventCollector, "jobf", FailureAction.CANCEL_ALL);
+		ExecutableFlow flow = runner.getExecutableFlow();
+		Map<String, Status> expectedStateMap = new HashMap<String, Status>();
+		Map<String, ExecutableNode> nodeMap = new HashMap<String, ExecutableNode>();
+		
+		// 1. START FLOW
+		createExpectedStateMap(flow, expectedStateMap, nodeMap);
+		Thread thread = runFlowRunnerInThread(runner);
+		pause(1000);
+		
+		// After it starts up, only joba should be running
+		expectedStateMap.put("joba", Status.RUNNING);
+		expectedStateMap.put("joba1", Status.RUNNING);
+		compareStates(expectedStateMap, nodeMap);
+
+		// 2. JOB in subflow FAILS
+		InteractiveTestJob.getTestJob("joba").succeedJob();
+		pause(250);
+		expectedStateMap.put("joba", Status.SUCCEEDED);
+		expectedStateMap.put("jobb", Status.RUNNING);
+		expectedStateMap.put("jobc", Status.RUNNING);
+		expectedStateMap.put("jobd", Status.RUNNING);
+		expectedStateMap.put("jobb:innerJobA", Status.RUNNING);
+		expectedStateMap.put("jobd:innerJobA", Status.RUNNING);
+		compareStates(expectedStateMap, nodeMap);
+		
+		InteractiveTestJob.getTestJob("joba1").succeedJob();
+		InteractiveTestJob.getTestJob("jobb:innerJobA").succeedJob();
+		pause(250);
+		expectedStateMap.put("jobb:innerJobA", Status.SUCCEEDED);
+		expectedStateMap.put("joba1", Status.SUCCEEDED);
+		expectedStateMap.put("jobb:innerJobB", Status.RUNNING);
+		expectedStateMap.put("jobb:innerJobC", Status.RUNNING);
+		compareStates(expectedStateMap, nodeMap);
+		
+		runner.cancel("me");
+		pause(250);
+		
+		expectedStateMap.put("jobb", Status.KILLED);
+		expectedStateMap.put("jobb:innerJobB", Status.FAILED);
+		expectedStateMap.put("jobb:innerJobC", Status.FAILED);
+		expectedStateMap.put("jobb:innerFlow", Status.KILLED);
+		expectedStateMap.put("jobc", Status.FAILED);
+		expectedStateMap.put("jobd", Status.KILLED);
+		expectedStateMap.put("jobd:innerJobA", Status.FAILED);
+		expectedStateMap.put("jobd:innerFlow2", Status.KILLED);
+		expectedStateMap.put("jobe", Status.KILLED);
+		expectedStateMap.put("jobf", Status.KILLED);
+		
+		Assert.assertEquals(Status.KILLED, flow.getStatus());
+		compareStates(expectedStateMap, nodeMap);
+		Assert.assertFalse(thread.isAlive());
+	}
+	
+	@Test
+	public void testManualCancelOnFailure() throws Exception {
+		// Test propagation of KILLED status to embedded flows different branch
+		EventCollectorListener eventCollector = new EventCollectorListener();
+		FlowRunner runner = createFlowRunner(eventCollector, "jobf");
+		ExecutableFlow flow = runner.getExecutableFlow();
+		Map<String, Status> expectedStateMap = new HashMap<String, Status>();
+		Map<String, ExecutableNode> nodeMap = new HashMap<String, ExecutableNode>();
+		
+		// 1. START FLOW
+		createExpectedStateMap(flow, expectedStateMap, nodeMap);
+		Thread thread = runFlowRunnerInThread(runner);
+		pause(250);
+		
+		// After it starts up, only joba should be running
+		expectedStateMap.put("joba", Status.RUNNING);
+		expectedStateMap.put("joba1", Status.RUNNING);
+		compareStates(expectedStateMap, nodeMap);
+
+		// 2. JOB in subflow FAILS
+		InteractiveTestJob.getTestJob("joba").succeedJob();
+		pause(250);
+		expectedStateMap.put("joba", Status.SUCCEEDED);
+		expectedStateMap.put("jobb", Status.RUNNING);
+		expectedStateMap.put("jobc", Status.RUNNING);
+		expectedStateMap.put("jobd", Status.RUNNING);
+		expectedStateMap.put("jobb:innerJobA", Status.RUNNING);
+		expectedStateMap.put("jobd:innerJobA", Status.RUNNING);
+		compareStates(expectedStateMap, nodeMap);
+		
+		InteractiveTestJob.getTestJob("joba1").succeedJob();
+		InteractiveTestJob.getTestJob("jobb:innerJobA").succeedJob();
+		pause(250);
+		expectedStateMap.put("jobb:innerJobA", Status.SUCCEEDED);
+		expectedStateMap.put("joba1", Status.SUCCEEDED);
+		expectedStateMap.put("jobb:innerJobB", Status.RUNNING);
+		expectedStateMap.put("jobb:innerJobC", Status.RUNNING);
+		compareStates(expectedStateMap, nodeMap);
+		
+		InteractiveTestJob.getTestJob("jobb:innerJobB").failJob();
+		pause(250);
+		expectedStateMap.put("jobb:innerJobB", Status.FAILED);
+		expectedStateMap.put("jobb", Status.FAILED_FINISHING);
+		Assert.assertEquals(Status.FAILED_FINISHING, flow.getStatus());
+		compareStates(expectedStateMap, nodeMap);
+		
+		runner.cancel("me");
+		pause(1000);
+		
+		expectedStateMap.put("jobb", Status.FAILED);
+		expectedStateMap.put("jobb:innerJobC", Status.FAILED);
+		expectedStateMap.put("jobb:innerFlow", Status.KILLED);
+		expectedStateMap.put("jobc", Status.FAILED);
+		expectedStateMap.put("jobd", Status.KILLED);
+		expectedStateMap.put("jobd:innerJobA", Status.FAILED);
+		expectedStateMap.put("jobd:innerFlow2", Status.KILLED);
+		expectedStateMap.put("jobe", Status.KILLED);
+		expectedStateMap.put("jobf", Status.KILLED);
+		
+		Assert.assertEquals(Status.FAILED, flow.getStatus());
+		compareStates(expectedStateMap, nodeMap);
+		Assert.assertFalse(thread.isAlive());
+	}
+	
+	private Thread runFlowRunnerInThread(FlowRunner runner) {
+		Thread thread = new Thread(runner);
+		thread.start();
+		return thread;
+	}
+	
+	private void pause(long millisec) {
+		synchronized(this) {
+			try {
+				wait(millisec);
+			}
+			catch (InterruptedException e) {
+			}
+		}
+	}
+	
+	private void createExpectedStateMap(ExecutableFlowBase flow, Map<String, Status> expectedStateMap, Map<String, ExecutableNode> nodeMap) {
+		for (ExecutableNode node: flow.getExecutableNodes()) {
+			expectedStateMap.put(node.getNestedId(), node.getStatus());
+			nodeMap.put(node.getNestedId(), node);
+			
+			if (node instanceof ExecutableFlowBase) {
+				createExpectedStateMap((ExecutableFlowBase)node, expectedStateMap, nodeMap);
+			}
+		}
+	}
+	
+	private void compareStates(Map<String, Status> expectedStateMap, Map<String, ExecutableNode> nodeMap) {
+		for (String printedId: expectedStateMap.keySet()) {
+			Status expectedStatus = expectedStateMap.get(printedId);
+			ExecutableNode node = nodeMap.get(printedId);
+			
+			if (expectedStatus != node.getStatus()) {
+				Assert.fail("Expected values do not match for " + printedId + ". Expected " + expectedStatus + ", instead received " + node.getStatus());
+			}
+		}
+	}
+	
+	private void prepareProject(File directory) throws ProjectManagerException, IOException {
+		DirectoryFlowLoader loader = new DirectoryFlowLoader(logger);
+		loader.loadProjectFlow(directory);
+		if (!loader.getErrors().isEmpty()) {
+			for (String error: loader.getErrors()) {
+				System.out.println(error);
+			}
+			
+			throw new RuntimeException("Errors found in setup");
+		}
+		
+		flowMap = loader.getFlowMap();
+		project.setFlows(flowMap);
+		FileUtils.copyDirectory(directory, workingDir);
+	}
+	
+	private FlowRunner createFlowRunner(EventCollectorListener eventCollector, String flowName) throws Exception {
+		return createFlowRunner(eventCollector, flowName, FailureAction.FINISH_CURRENTLY_RUNNING);
+	}
+	
+	private FlowRunner createFlowRunner(EventCollectorListener eventCollector, String flowName, FailureAction action) throws Exception {
+		Flow flow = flowMap.get(flowName);
+
+		int exId = id++;
+		ExecutableFlow exFlow = new ExecutableFlow(project, flow);
+		exFlow.setExecutionPath(workingDir.getPath());
+		exFlow.setExecutionId(exId);
+
+		Map<String, String> flowParam = new HashMap<String, String>();
+		flowParam.put("param4", "override.4");
+		flowParam.put("param10", "override.10");
+		flowParam.put("param11", "override.11");
+		exFlow.getExecutionOptions().addAllFlowParameters(flowParam);
+		exFlow.getExecutionOptions().setFailureAction(action);
+		fakeExecutorLoader.uploadExecutableFlow(exFlow);
+	
+		FlowRunner runner = new FlowRunner(fakeExecutorLoader.fetchExecutableFlow(exId), fakeExecutorLoader, fakeProjectLoader, jobtypeManager);
+
+		runner.addListener(eventCollector);
+		
+		return runner;
+	}
+
+}
diff --git a/unit/java/azkaban/test/execapp/JobRunnerTest.java b/unit/java/azkaban/test/execapp/JobRunnerTest.java
index 02f35ae..3201767 100644
--- a/unit/java/azkaban/test/execapp/JobRunnerTest.java
+++ b/unit/java/azkaban/test/execapp/JobRunnerTest.java
@@ -73,11 +73,11 @@ public class JobRunnerTest {
 		Assert.assertTrue( node.getEndTime() - node.getStartTime() > 1000);
 		
 		File logFile = new File(runner.getLogFilePath());
-		Props outputProps = runner.getOutputProps();
+		Props outputProps = runner.getNode().getOutputProps();
 		Assert.assertTrue(outputProps != null);
 		Assert.assertTrue(logFile.exists());
 		
-		Assert.assertTrue(loader.getNodeUpdateCount(node.getJobId()) == 3);
+		Assert.assertTrue(loader.getNodeUpdateCount(node.getId()) == 3);
 		
 		Assert.assertTrue(eventCollector.checkOrdering());
 		try {
@@ -104,12 +104,12 @@ public class JobRunnerTest {
 		Assert.assertTrue(node.getEndTime() - node.getStartTime() > 1000);
 		
 		File logFile = new File(runner.getLogFilePath());
-		Props outputProps = runner.getOutputProps();
+		Props outputProps = runner.getNode().getOutputProps();
 		Assert.assertTrue(outputProps == null);
 		Assert.assertTrue(logFile.exists());
 		Assert.assertTrue(eventCollector.checkOrdering());
 		Assert.assertTrue(!runner.isCancelled());
-		Assert.assertTrue(loader.getNodeUpdateCount(node.getJobId()) == 3);
+		Assert.assertTrue(loader.getNodeUpdateCount(node.getId()) == 3);
 		
 		try {
 			eventCollector.checkEventExists(new Type[] {Type.JOB_STARTED, Type.JOB_STATUS_CHANGED, Type.JOB_FINISHED});
@@ -139,12 +139,12 @@ public class JobRunnerTest {
 		Assert.assertTrue( node.getEndTime() - node.getStartTime() < 10);
 		
 		// Log file and output files should not exist.
-		Props outputProps = runner.getOutputProps();
+		Props outputProps = runner.getNode().getOutputProps();
 		Assert.assertTrue(outputProps == null);
 		Assert.assertTrue(runner.getLogFilePath() == null);
 		Assert.assertTrue(eventCollector.checkOrdering());
 		
-		Assert.assertTrue(loader.getNodeUpdateCount(node.getJobId()) == null);
+		Assert.assertTrue(loader.getNodeUpdateCount(node.getId()) == null);
 		
 		try {
 			eventCollector.checkEventExists(new Type[] {Type.JOB_STARTED, Type.JOB_FINISHED});
@@ -174,10 +174,10 @@ public class JobRunnerTest {
 		// Give it 10 ms to fail.
 		Assert.assertTrue( node.getEndTime() - node.getStartTime() < 10);
 		
-		Assert.assertTrue(loader.getNodeUpdateCount(node.getJobId()) == null);
+		Assert.assertTrue(loader.getNodeUpdateCount(node.getId()) == null);
 		
 		// Log file and output files should not exist.
-		Props outputProps = runner.getOutputProps();
+		Props outputProps = runner.getNode().getOutputProps();
 		Assert.assertTrue(outputProps == null);
 		Assert.assertTrue(runner.getLogFilePath() == null);
 		Assert.assertTrue(!runner.isCancelled());
@@ -222,11 +222,11 @@ public class JobRunnerTest {
 		Assert.assertTrue(node.getStartTime() > 0 && node.getEndTime() > 0);
 		// Give it 10 ms to fail.
 		Assert.assertTrue(node.getEndTime() - node.getStartTime() < 3000);
-		Assert.assertTrue(loader.getNodeUpdateCount(node.getJobId()) == 3);
+		Assert.assertTrue(loader.getNodeUpdateCount(node.getId()) == 3);
 		
 		// Log file and output files should not exist.
 		File logFile = new File(runner.getLogFilePath());
-		Props outputProps = runner.getOutputProps();
+		Props outputProps = runner.getNode().getOutputProps();
 		Assert.assertTrue(outputProps == null);
 		Assert.assertTrue(logFile.exists());
 		Assert.assertTrue(eventCollector.checkOrdering());
@@ -263,11 +263,11 @@ public class JobRunnerTest {
 		Assert.assertTrue(node.getStartTime() - startTime >= 5000);
 		
 		File logFile = new File(runner.getLogFilePath());
-		Props outputProps = runner.getOutputProps();
+		Props outputProps = runner.getNode().getOutputProps();
 		Assert.assertTrue(outputProps != null);
 		Assert.assertTrue(logFile.exists());
 		Assert.assertFalse(runner.isCancelled());
-		Assert.assertTrue(loader.getNodeUpdateCount(node.getJobId()) == 3);
+		Assert.assertTrue(loader.getNodeUpdateCount(node.getId()) == 3);
 		
 		Assert.assertTrue(eventCollector.checkOrdering());
 		try {
@@ -320,7 +320,7 @@ public class JobRunnerTest {
 		Assert.assertTrue(runner.isCancelled());
 		
 		File logFile = new File(runner.getLogFilePath());
-		Props outputProps = runner.getOutputProps();
+		Props outputProps = runner.getNode().getOutputProps();
 		Assert.assertTrue(outputProps == null);
 		Assert.assertTrue(logFile.exists());
 		
@@ -349,13 +349,14 @@ public class JobRunnerTest {
 		ExecutableFlow flow = new ExecutableFlow();
 		flow.setExecutionId(execId);
 		ExecutableNode node = new ExecutableNode();
-		node.setJobId(name);
-		node.setExecutableFlow(flow);
+		node.setId(name);
+		node.setParentFlow(flow);
 		
 		Props props = createProps(time, fail);
+		node.setInputProps(props);
 		HashSet<String> proxyUsers = new HashSet<String>();
 		proxyUsers.add(flow.getSubmitUser());
-		JobRunner runner = new JobRunner(node, props, workingDir, loader, jobtypeManager);
+		JobRunner runner = new JobRunner(node, workingDir, loader, jobtypeManager);
 		runner.setLogSettings(logger, "5MB", 4);
 
 		runner.addListener(listener);
diff --git a/unit/java/azkaban/test/execapp/MockExecutorLoader.java b/unit/java/azkaban/test/execapp/MockExecutorLoader.java
index 2b81f82..05641ec 100644
--- a/unit/java/azkaban/test/execapp/MockExecutorLoader.java
+++ b/unit/java/azkaban/test/execapp/MockExecutorLoader.java
@@ -65,7 +65,6 @@ public class MockExecutorLoader implements ExecutorLoader {
 
 	}
 
-	@SuppressWarnings("unchecked")
 	@Override
 	public void updateExecutableFlow(ExecutableFlow flow) throws ExecutorManagerException {
 		ExecutableFlow toUpdate = flows.get(flow.getExecutionId());
@@ -76,24 +75,27 @@ public class MockExecutorLoader implements ExecutorLoader {
 
 	@Override
 	public void uploadExecutableNode(ExecutableNode node, Props inputParams) throws ExecutorManagerException {
-		nodes.put(node.getJobId(), ExecutableNode.createNodeFromObject(node.toObject(), null));
-		jobUpdateCount.put(node.getJobId(), 1);
+		ExecutableNode exNode = new ExecutableNode();
+		exNode.fillExecutableFromMapObject(node.toObject());
+		
+		nodes.put(node.getId(), exNode);
+		jobUpdateCount.put(node.getId(), 1);
 	}
 
 	@Override
 	public void updateExecutableNode(ExecutableNode node) throws ExecutorManagerException {
-		ExecutableNode foundNode = nodes.get(node.getJobId());
+		ExecutableNode foundNode = nodes.get(node.getId());
 		foundNode.setEndTime(node.getEndTime());
 		foundNode.setStartTime(node.getStartTime());
 		foundNode.setStatus(node.getStatus());
 		foundNode.setUpdateTime(node.getUpdateTime());
 		
-		Integer value = jobUpdateCount.get(node.getJobId());
+		Integer value = jobUpdateCount.get(node.getId());
 		if (value == null) {
 			throw new ExecutorManagerException("The node has not been uploaded");
 		}
 		else {
-			jobUpdateCount.put(node.getJobId(), ++value);
+			jobUpdateCount.put(node.getId(), ++value);
 		}
 		
 		flowUpdateCount++;
diff --git a/unit/java/azkaban/test/executor/ExecutableFlowTest.java b/unit/java/azkaban/test/executor/ExecutableFlowTest.java
new file mode 100644
index 0000000..9f2a3c2
--- /dev/null
+++ b/unit/java/azkaban/test/executor/ExecutableFlowTest.java
@@ -0,0 +1,384 @@
+package azkaban.test.executor;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import junit.framework.Assert;
+
+import org.apache.log4j.Logger;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import azkaban.executor.ExecutableFlow;
+import azkaban.executor.ExecutableFlowBase;
+import azkaban.executor.ExecutableNode;
+import azkaban.executor.ExecutionOptions;
+import azkaban.executor.ExecutionOptions.FailureAction;
+import azkaban.executor.Status;
+import azkaban.flow.Flow;
+import azkaban.project.Project;
+import azkaban.utils.DirectoryFlowLoader;
+import azkaban.utils.JSONUtils;
+
+public class ExecutableFlowTest {
+	private Project project;
+	
+    @Before
+    public void setUp() throws Exception {
+		Logger logger = Logger.getLogger(this.getClass());
+    	DirectoryFlowLoader loader = new DirectoryFlowLoader(logger);
+    	loader.loadProjectFlow(new File("unit/executions/embedded"));
+    	Assert.assertEquals(0, loader.getErrors().size());
+    	
+    	project = new Project(11, "myTestProject");
+    	project.setFlows(loader.getFlowMap());
+    	project.setVersion(123);
+    }
+    
+    @After
+    public void tearDown() throws Exception {
+    }
+	
+	@Test
+	public void testExecutorFlowCreation() throws Exception {
+		Flow flow = project.getFlow("jobe");
+		Assert.assertNotNull(flow);
+		
+		ExecutableFlow exFlow = new ExecutableFlow(project, flow);
+		Assert.assertNotNull(exFlow.getExecutableNode("joba"));
+		Assert.assertNotNull(exFlow.getExecutableNode("jobb"));
+		Assert.assertNotNull(exFlow.getExecutableNode("jobc"));
+		Assert.assertNotNull(exFlow.getExecutableNode("jobd"));
+		Assert.assertNotNull(exFlow.getExecutableNode("jobe"));
+		
+		Assert.assertFalse(exFlow.getExecutableNode("joba") instanceof ExecutableFlowBase);
+		Assert.assertTrue(exFlow.getExecutableNode("jobb") instanceof ExecutableFlowBase);
+		Assert.assertTrue(exFlow.getExecutableNode("jobc") instanceof ExecutableFlowBase);
+		Assert.assertTrue(exFlow.getExecutableNode("jobd") instanceof ExecutableFlowBase);
+		Assert.assertFalse(exFlow.getExecutableNode("jobe") instanceof ExecutableFlowBase);
+		
+		ExecutableFlowBase jobbFlow = (ExecutableFlowBase)exFlow.getExecutableNode("jobb");
+		ExecutableFlowBase jobcFlow = (ExecutableFlowBase)exFlow.getExecutableNode("jobc");
+		ExecutableFlowBase jobdFlow = (ExecutableFlowBase)exFlow.getExecutableNode("jobd");
+
+		Assert.assertEquals("innerFlow", jobbFlow.getFlowId());
+		Assert.assertEquals("jobb", jobbFlow.getId());
+		Assert.assertEquals(4, jobbFlow.getExecutableNodes().size());
+		
+		Assert.assertEquals("innerFlow", jobcFlow.getFlowId());
+		Assert.assertEquals("jobc", jobcFlow.getId());
+		Assert.assertEquals(4, jobcFlow.getExecutableNodes().size());
+		
+		Assert.assertEquals("innerFlow", jobdFlow.getFlowId());
+		Assert.assertEquals("jobd", jobdFlow.getId());
+		Assert.assertEquals(4, jobdFlow.getExecutableNodes().size());
+	}
+	
+	@Test
+	public void testExecutorFlowJson() throws Exception {
+		Flow flow = project.getFlow("jobe");
+		Assert.assertNotNull(flow);
+		
+		ExecutableFlow exFlow = new ExecutableFlow(project, flow);
+		
+		Object obj = exFlow.toObject();
+		String exFlowJSON = JSONUtils.toJSON(obj);
+		@SuppressWarnings("unchecked")
+		Map<String,Object> flowObjMap = (Map<String,Object>)JSONUtils.parseJSONFromString(exFlowJSON);
+		
+		ExecutableFlow parsedExFlow = ExecutableFlow.createExecutableFlowFromObject(flowObjMap);
+		testEquals(exFlow, parsedExFlow);
+	}
+	
+	@Test
+	public void testExecutorFlowJson2() throws Exception {
+		Flow flow = project.getFlow("jobe");
+		Assert.assertNotNull(flow);
+		
+		ExecutableFlow exFlow = new ExecutableFlow(project, flow);
+		exFlow.setExecutionId(101);
+		exFlow.setAttempt(2);
+		exFlow.setDelayedExecution(1000);
+		
+		ExecutionOptions options = new ExecutionOptions();
+		options.setConcurrentOption("blah");
+		options.setDisabledJobs(Arrays.asList(new Object[] {"bee", null, "boo"}));
+		options.setFailureAction(FailureAction.CANCEL_ALL);
+		options.setFailureEmails(Arrays.asList(new String[] {"doo", null, "daa"}));
+		options.setSuccessEmails(Arrays.asList(new String[] {"dee", null, "dae"}));
+		options.setPipelineLevel(2);
+		options.setPipelineExecutionId(3);
+		options.setNotifyOnFirstFailure(true);
+		options.setNotifyOnLastFailure(true);
+		
+		HashMap<String, String> flowProps = new HashMap<String,String>();
+		flowProps.put("la", "fa");
+		options.addAllFlowParameters(flowProps);
+		exFlow.setExecutionOptions(options);
+		
+		Object obj = exFlow.toObject();
+		String exFlowJSON = JSONUtils.toJSON(obj);
+		@SuppressWarnings("unchecked")
+		Map<String,Object> flowObjMap = (Map<String,Object>)JSONUtils.parseJSONFromString(exFlowJSON);
+		
+		ExecutableFlow parsedExFlow = ExecutableFlow.createExecutableFlowFromObject(flowObjMap);
+		testEquals(exFlow, parsedExFlow);
+	}
+	
+	@SuppressWarnings("rawtypes")
+	@Test
+	public void testExecutorFlowUpdates() throws Exception {
+		Flow flow = project.getFlow("jobe");
+		ExecutableFlow exFlow = new ExecutableFlow(project, flow);
+		exFlow.setExecutionId(101);
+		
+		// Create copy of flow
+		Object obj = exFlow.toObject();
+		String exFlowJSON = JSONUtils.toJSON(obj);
+		@SuppressWarnings("unchecked")
+		Map<String,Object> flowObjMap = (Map<String,Object>)JSONUtils.parseJSONFromString(exFlowJSON);
+		ExecutableFlow copyFlow = ExecutableFlow.createExecutableFlowFromObject(flowObjMap);
+		
+		testEquals(exFlow, copyFlow);
+		
+		ExecutableNode joba = exFlow.getExecutableNode("joba");
+		ExecutableFlowBase jobb = (ExecutableFlowBase)(exFlow.getExecutableNode("jobb"));
+		ExecutableFlowBase jobc = (ExecutableFlowBase)(exFlow.getExecutableNode("jobc"));
+		ExecutableFlowBase jobd = (ExecutableFlowBase)(exFlow.getExecutableNode("jobd"));
+		ExecutableNode jobe = exFlow.getExecutableNode("jobe");
+		assertNotNull(joba, jobb, jobc, jobd, jobe);
+		
+		ExecutableNode jobbInnerFlowA = jobb.getExecutableNode("innerJobA");
+		ExecutableNode jobbInnerFlowB = jobb.getExecutableNode("innerJobB");
+		ExecutableNode jobbInnerFlowC = jobb.getExecutableNode("innerJobC");
+		ExecutableNode jobbInnerFlow = jobb.getExecutableNode("innerFlow");
+		assertNotNull(jobbInnerFlowA, jobbInnerFlowB, jobbInnerFlowC, jobbInnerFlow);
+		
+		ExecutableNode jobcInnerFlowA = jobc.getExecutableNode("innerJobA");
+		ExecutableNode jobcInnerFlowB = jobc.getExecutableNode("innerJobB");
+		ExecutableNode jobcInnerFlowC = jobc.getExecutableNode("innerJobC");
+		ExecutableNode jobcInnerFlow = jobc.getExecutableNode("innerFlow");
+		assertNotNull(jobcInnerFlowA, jobcInnerFlowB, jobcInnerFlowC, jobcInnerFlow);
+		
+		ExecutableNode jobdInnerFlowA = jobd.getExecutableNode("innerJobA");
+		ExecutableNode jobdInnerFlowB = jobd.getExecutableNode("innerJobB");
+		ExecutableNode jobdInnerFlowC = jobd.getExecutableNode("innerJobC");
+		ExecutableNode jobdInnerFlow = jobd.getExecutableNode("innerFlow");
+		assertNotNull(jobdInnerFlowA, jobdInnerFlowB, jobdInnerFlowC, jobdInnerFlow);
+		
+		exFlow.setEndTime(1000);
+		exFlow.setStartTime(500);
+		exFlow.setStatus(Status.RUNNING);
+		exFlow.setUpdateTime(133);
+		
+		// Change one job and see if it updates
+		long time = System.currentTimeMillis();
+		jobe.setEndTime(time);
+		jobe.setUpdateTime(time);
+		jobe.setStatus(Status.DISABLED);
+		jobe.setStartTime(time - 1);
+		// Should be one node that was changed
+		Map<String,Object> updateObject = exFlow.toUpdateObject(0);
+		Assert.assertEquals(1, ((List)(updateObject.get("nodes"))).size());
+		// Reapplying should give equal results.
+		copyFlow.applyUpdateObject(updateObject);
+		testEquals(exFlow, copyFlow);
+		
+		// This update shouldn't provide any results
+		updateObject = exFlow.toUpdateObject(System.currentTimeMillis());
+		Assert.assertNull(updateObject.get("nodes"));
+		
+		// Change inner flow
+		long currentTime = time + 1 ;
+		jobbInnerFlowA.setEndTime(currentTime);
+		jobbInnerFlowA.setUpdateTime(currentTime);
+		jobbInnerFlowA.setStatus(Status.DISABLED);
+		jobbInnerFlowA.setStartTime(currentTime - 100);
+		// We should get 2 updates if we do a toUpdateObject using 0 as the start time
+		updateObject = exFlow.toUpdateObject(0);
+		Assert.assertEquals(2, ((List)(updateObject.get("nodes"))).size());
+
+		// This should provide 1 update. That we can apply
+		updateObject = exFlow.toUpdateObject(jobe.getUpdateTime());
+		Assert.assertNotNull(updateObject.get("nodes"));
+		Assert.assertEquals(1, ((List)(updateObject.get("nodes"))).size());
+		copyFlow.applyUpdateObject(updateObject);
+		testEquals(exFlow, copyFlow);
+		
+		// This shouldn't give any results anymore
+		updateObject = exFlow.toUpdateObject(jobbInnerFlowA.getUpdateTime());
+		Assert.assertNull(updateObject.get("nodes"));
+	}
+	
+	private void assertNotNull(ExecutableNode ... nodes) {
+		for (ExecutableNode node: nodes) {
+			Assert.assertNotNull(node);
+		}
+	}
+	
+	public static void testEquals(ExecutableNode a, ExecutableNode b) {
+		if (a instanceof ExecutableFlow) {
+			if (b instanceof ExecutableFlow) {
+				ExecutableFlow exA = (ExecutableFlow)a;
+				ExecutableFlow exB = (ExecutableFlow)b;
+				
+				Assert.assertEquals(exA.getScheduleId(), exB.getScheduleId());
+				Assert.assertEquals(exA.getProjectId(), exB.getProjectId());
+				Assert.assertEquals(exA.getVersion(), exB.getVersion());
+				Assert.assertEquals(exA.getSubmitTime(), exB.getSubmitTime());
+				Assert.assertEquals(exA.getSubmitUser(), exB.getSubmitUser());
+				Assert.assertEquals(exA.getExecutionPath(), exB.getExecutionPath());
+				
+				testEquals(exA.getExecutionOptions(), exB.getExecutionOptions());
+			}
+			else {
+				Assert.fail("A is ExecutableFlow, but B is not");
+			}
+		}
+
+		if (a instanceof ExecutableFlowBase) {
+			if (b instanceof ExecutableFlowBase) {
+				ExecutableFlowBase exA = (ExecutableFlowBase)a;
+				ExecutableFlowBase exB = (ExecutableFlowBase)b;
+				
+				Assert.assertEquals(exA.getFlowId(), exB.getFlowId());
+				Assert.assertEquals(exA.getExecutableNodes().size(), exB.getExecutableNodes().size());
+				
+				for(ExecutableNode nodeA : exA.getExecutableNodes()) {
+					ExecutableNode nodeB = exB.getExecutableNode(nodeA.getId());
+					Assert.assertNotNull(nodeB);
+					Assert.assertEquals(a, nodeA.getParentFlow());
+					Assert.assertEquals(b, nodeB.getParentFlow());
+					
+					testEquals(nodeA, nodeB);
+				}
+			}
+			else {
+				Assert.fail("A is ExecutableFlowBase, but B is not");
+			}
+		}
+		
+		Assert.assertEquals(a.getId(), b.getId());
+		Assert.assertEquals(a.getStatus(), b.getStatus());
+		Assert.assertEquals(a.getStartTime(), b.getStartTime());
+		Assert.assertEquals(a.getEndTime(), b.getEndTime());
+		Assert.assertEquals(a.getUpdateTime(), b.getUpdateTime());
+		Assert.assertEquals(a.getAttempt(), b.getAttempt());
+
+		Assert.assertEquals(a.getJobSource(), b.getJobSource());
+		Assert.assertEquals(a.getPropsSource(), b.getPropsSource());
+		Assert.assertEquals(a.getInNodes(), a.getInNodes());
+		Assert.assertEquals(a.getOutNodes(), a.getOutNodes());
+	}
+	
+	public static void testEquals(ExecutionOptions optionsA, ExecutionOptions optionsB) {
+		Assert.assertEquals(optionsA.getConcurrentOption(), optionsB.getConcurrentOption());
+		Assert.assertEquals(optionsA.getNotifyOnFirstFailure(), optionsB.getNotifyOnFirstFailure());
+		Assert.assertEquals(optionsA.getNotifyOnLastFailure(), optionsB.getNotifyOnLastFailure());
+		Assert.assertEquals(optionsA.getFailureAction(), optionsB.getFailureAction());
+		Assert.assertEquals(optionsA.getPipelineExecutionId(), optionsB.getPipelineExecutionId());
+		Assert.assertEquals(optionsA.getPipelineLevel(), optionsB.getPipelineLevel());
+		Assert.assertEquals(optionsA.isFailureEmailsOverridden(), optionsB.isFailureEmailsOverridden());
+		Assert.assertEquals(optionsA.isSuccessEmailsOverridden(), optionsB.isSuccessEmailsOverridden());
+		
+		testDisabledEquals(optionsA.getDisabledJobs(), optionsB.getDisabledJobs());
+		testEquals(optionsA.getSuccessEmails(), optionsB.getSuccessEmails());
+		testEquals(optionsA.getFailureEmails(), optionsB.getFailureEmails());
+		testEquals(optionsA.getFlowParameters(), optionsB.getFlowParameters());
+	}
+	
+	public static void testEquals(Set<String> a, Set<String> b) {
+		if (a == b) {
+			return;
+		}
+		
+		if (a == null || b == null) {
+			Assert.fail();
+		}
+		
+		Assert.assertEquals(a.size(), b.size());
+		
+		Iterator<String> iterA = a.iterator();
+		
+		while(iterA.hasNext()) {
+			String aStr = iterA.next();
+			Assert.assertTrue(b.contains(aStr));
+		}
+	}
+	
+	public static void testEquals(List<String> a, List<String> b) {
+		if (a == b) {
+			return;
+		}
+		
+		if (a == null || b == null) {
+			Assert.fail();
+		}
+		
+		Assert.assertEquals(a.size(), b.size());
+		
+		Iterator<String> iterA = a.iterator();
+		Iterator<String> iterB = b.iterator();
+		
+		while(iterA.hasNext()) {
+			String aStr = iterA.next();
+			String bStr = iterB.next();
+			Assert.assertEquals(aStr, bStr);
+		}
+	}
+	
+	@SuppressWarnings("unchecked")
+	public static void testDisabledEquals(List<Object> a, List<Object> b) {
+		if (a == b) {
+			return;
+		}
+		
+		if (a == null || b == null) {
+			Assert.fail();
+		}
+		
+		Assert.assertEquals(a.size(), b.size());
+		
+		Iterator<Object> iterA = a.iterator();
+		Iterator<Object> iterB = b.iterator();
+		
+		while(iterA.hasNext()) {
+			Object aStr = iterA.next();
+			Object bStr = iterB.next();
+			
+			if (aStr instanceof Map && bStr instanceof Map) {
+				Map<String, Object> aMap = (Map<String, Object>)aStr;
+				Map<String, Object> bMap = (Map<String, Object>)bStr;
+				
+				Assert.assertEquals((String)aMap.get("id"), (String)bMap.get("id"));
+				testDisabledEquals((List<Object>)aMap.get("children"), (List<Object>)bMap.get("children"));
+			}
+			else {
+				Assert.assertEquals(aStr, bStr);
+			}
+		}
+	}
+	
+	
+	public static void testEquals(Map<String, String> a, Map<String, String> b) {
+		if (a == b) {
+			return;
+		}
+		
+		if (a == null || b == null) {
+			Assert.fail();
+		}
+		
+		Assert.assertEquals(a.size(), b.size());
+		
+		for (String key: a.keySet()) {
+			Assert.assertEquals(a.get(key), b.get(key));
+		}
+	}
+}
diff --git a/unit/java/azkaban/test/executor/InteractiveTestJob.java b/unit/java/azkaban/test/executor/InteractiveTestJob.java
new file mode 100644
index 0000000..3c385de
--- /dev/null
+++ b/unit/java/azkaban/test/executor/InteractiveTestJob.java
@@ -0,0 +1,93 @@
+package azkaban.test.executor;
+
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.log4j.Logger;
+
+import azkaban.flow.CommonJobProperties;
+import azkaban.jobExecutor.AbstractProcessJob;
+import azkaban.utils.Props;
+
+public class InteractiveTestJob extends AbstractProcessJob {
+	private static ConcurrentHashMap<String, InteractiveTestJob> testJobs = new ConcurrentHashMap<String, InteractiveTestJob>();
+	private Props generatedProperties = new Props();
+	private boolean isWaiting = true;
+	private boolean succeed = true;
+
+	public static InteractiveTestJob getTestJob(String name) {
+		return testJobs.get(name);
+	}
+	
+	public static void clearTestJobs() {
+		testJobs.clear();
+	}
+	
+	public InteractiveTestJob(String jobId, Props sysProps, Props jobProps, Logger log) {
+		super(jobId, sysProps, jobProps, log);
+	}
+
+	@Override
+	public void run() throws Exception {
+		String nestedFlowPath = this.getJobProps().get(CommonJobProperties.NESTED_FLOW_PATH);
+		String groupName = this.getJobProps().getString("group", null);
+		String id = nestedFlowPath == null ? this.getId() : nestedFlowPath;
+		if (groupName != null) {
+			id = groupName + ":" + id;
+		}
+		testJobs.put(id, this);
+		
+		while(isWaiting) {
+			synchronized(this) {
+				try {
+					wait(30000);
+				} catch (InterruptedException e) {
+				}
+				
+				if (!isWaiting) {
+					if (!succeed) {
+						throw new RuntimeException("Forced failure of " + getId());
+					}
+					else {
+						info("Job " + getId() + " succeeded.");
+					}
+				}
+			}
+		}
+	}
+	
+	public void failJob() {
+		synchronized(this) {
+			succeed = false;
+			isWaiting = false;
+			this.notify();
+		}
+	}
+	
+	public void succeedJob() {
+		synchronized(this) {
+			succeed = true;
+			isWaiting = false;
+			this.notify();
+		}
+	}
+	
+	public void succeedJob(Props generatedProperties) {
+		synchronized(this) {
+			this.generatedProperties = generatedProperties;
+			succeed = true;
+			isWaiting = false;
+			this.notify();
+		}
+	}
+	
+	@Override
+	public Props getJobGeneratedProperties() {
+		return generatedProperties;
+	}
+
+	@Override
+	public void cancel() throws InterruptedException {
+		info("Killing job");
+		failJob();
+	}
+}
diff --git a/unit/java/azkaban/test/executor/JavaJobRunnerMain.java b/unit/java/azkaban/test/executor/JavaJobRunnerMain.java
index 9add12a..d1766e6 100644
--- a/unit/java/azkaban/test/executor/JavaJobRunnerMain.java
+++ b/unit/java/azkaban/test/executor/JavaJobRunnerMain.java
@@ -185,7 +185,9 @@ public class JavaJobRunnerMain {
 			new RuntimeException("Unable to store output properties to: " + outputFileStr);
 		} finally {
 			try {
-				writer.close();
+				if (writer != null) {
+					writer.close();
+				}
 			} catch (IOException e) {
 			}
 		}
diff --git a/unit/java/azkaban/test/executor/JdbcExecutorLoaderTest.java b/unit/java/azkaban/test/executor/JdbcExecutorLoaderTest.java
index 302eaac..6a2e399 100644
--- a/unit/java/azkaban/test/executor/JdbcExecutorLoaderTest.java
+++ b/unit/java/azkaban/test/executor/JdbcExecutorLoaderTest.java
@@ -31,6 +31,7 @@ import azkaban.executor.Status;
 import azkaban.flow.Flow;
 
 import azkaban.database.DataSourceUtils;
+import azkaban.project.Project;
 import azkaban.utils.FileIOUtils.LogData;
 import azkaban.utils.JSONUtils;
 import azkaban.utils.Pair;
@@ -244,7 +245,7 @@ public class JdbcExecutorLoaderTest {
 		Assert.assertEquals(flow.getProjectId(), info.getProjectId());
 		Assert.assertEquals(flow.getVersion(), info.getVersion());
 		Assert.assertEquals(flow.getFlowId(), info.getFlowId());
-		Assert.assertEquals(oldNode.getJobId(), info.getJobId());
+		Assert.assertEquals(oldNode.getId(), info.getJobId());
 		Assert.assertEquals(oldNode.getStatus(), info.getStatus());
 		Assert.assertEquals(oldNode.getStartTime(), info.getStartTime());
 		Assert.assertEquals("endTime = " + oldNode.getEndTime() + " info endTime = " + info.getEndTime(), oldNode.getEndTime(), info.getEndTime());
@@ -408,7 +409,12 @@ public class JdbcExecutorLoaderTest {
 		HashMap<String, Object> flowObj = (HashMap<String, Object>) JSONUtils.parseJSONFromFile(jsonFlowFile);
 		
 		Flow flow = Flow.flowFromObject(flowObj);
-		ExecutableFlow execFlow = new ExecutableFlow(executionId, flow);
+		Project project = new Project(1, "flow");
+		HashMap<String, Flow> flowMap = new HashMap<String, Flow>();
+		flowMap.put(flow.getId(), flow);
+		project.setFlows(flowMap);
+		ExecutableFlow execFlow = new ExecutableFlow(project, flow);
+		execFlow.setExecutionId(executionId);
 
 		return execFlow;
 	}
@@ -419,7 +425,11 @@ public class JdbcExecutorLoaderTest {
 		HashMap<String, Object> flowObj = (HashMap<String, Object>) JSONUtils.parseJSONFromFile(jsonFlowFile);
 		
 		Flow flow = Flow.flowFromObject(flowObj);
-		ExecutableFlow execFlow = new ExecutableFlow(flow);
+		Project project = new Project(1, "flow");
+		HashMap<String, Flow> flowMap = new HashMap<String, Flow>();
+		flowMap.put(flow.getId(), flow);
+		project.setFlows(flowMap);
+		ExecutableFlow execFlow = new ExecutableFlow(project, flow);
 
 		return execFlow;
 	}
diff --git a/unit/java/azkaban/test/executor/SleepJavaJob.java b/unit/java/azkaban/test/executor/SleepJavaJob.java
index ac831fc..535151a 100644
--- a/unit/java/azkaban/test/executor/SleepJavaJob.java
+++ b/unit/java/azkaban/test/executor/SleepJavaJob.java
@@ -1,6 +1,9 @@
 package azkaban.test.executor;
 
+import java.io.BufferedReader;
+import java.io.FileReader;
 import java.util.Map;
+import java.util.Properties;
 
 public class SleepJavaJob {
 	private boolean fail;
@@ -8,8 +11,19 @@ public class SleepJavaJob {
 	private int attempts;
 	private int currentAttempt;
 
+	public SleepJavaJob(String id, Properties props) {
+		setup(props);
+	}
+	
 	public SleepJavaJob(String id, Map<String, String> parameters) {
-		String failStr = parameters.get("fail");
+		Properties properties = new Properties();
+		properties.putAll(parameters);
+		
+		setup(properties);
+	}
+	
+	private void setup(Properties props) {
+		String failStr = (String)props.get("fail");
 		
 		if (failStr == null || failStr.equals("false")) {
 			fail = false;
@@ -18,15 +32,15 @@ public class SleepJavaJob {
 			fail = true;
 		}
 	
-		currentAttempt = parameters.containsKey("azkaban.job.attempt") ? Integer.parseInt(parameters.get("azkaban.job.attempt")) : 0;
-		String attemptString = parameters.get("passRetry");
+		currentAttempt = props.containsKey("azkaban.job.attempt") ? Integer.parseInt((String)props.get("azkaban.job.attempt")) : 0;
+		String attemptString = (String)props.get("passRetry");
 		if (attemptString == null) {
 			attempts = -1;
 		}
 		else {
 			attempts = Integer.valueOf(attemptString);
 		}
-		seconds = parameters.get("seconds");
+		seconds = (String)props.get("seconds");
 
 		if (fail) {
 			System.out.println("Planning to fail after " + seconds + " seconds. Attempts left " + currentAttempt + " of " + attempts);
@@ -36,6 +50,17 @@ public class SleepJavaJob {
 		}
 	}
 	
+	public static void main(String[] args) throws Exception {
+		String propsFile = System.getenv("JOB_PROP_FILE");
+		Properties prop = new Properties();
+		prop.load(new BufferedReader(new FileReader(propsFile)));
+		
+		String jobName = System.getenv("JOB_NAME");
+		SleepJavaJob job = new SleepJavaJob(jobName, prop);
+		
+		job.run();
+	}
+	
 	public void run() throws Exception {
 		if (seconds == null) {
 			throw new RuntimeException("Seconds not set");
diff --git a/unit/java/azkaban/test/trigger/ExecuteFlowActionTest.java b/unit/java/azkaban/test/trigger/ExecuteFlowActionTest.java
index 905ca3d..4c49dab 100644
--- a/unit/java/azkaban/test/trigger/ExecuteFlowActionTest.java
+++ b/unit/java/azkaban/test/trigger/ExecuteFlowActionTest.java
@@ -21,7 +21,7 @@ public class ExecuteFlowActionTest {
 		loader.init(new Props());
 		
 		ExecutionOptions options = new ExecutionOptions();
-		List<String> disabledJobs = new ArrayList<String>();
+		List<Object> disabledJobs = new ArrayList<Object>();
 		options.setDisabledJobs(disabledJobs);
 		
 		ExecuteFlowAction executeFlowAction = new ExecuteFlowAction("ExecuteFlowAction", 1, "testproject", "testflow", "azkaban", options, null);
diff --git a/unit/java/azkaban/test/utils/DirectoryFlowLoaderTest.java b/unit/java/azkaban/test/utils/DirectoryFlowLoaderTest.java
index 4316c5c..d671e6f 100644
--- a/unit/java/azkaban/test/utils/DirectoryFlowLoaderTest.java
+++ b/unit/java/azkaban/test/utils/DirectoryFlowLoaderTest.java
@@ -2,6 +2,8 @@ package azkaban.test.utils;
 
 import java.io.File;
 
+import junit.framework.Assert;
+
 import org.apache.log4j.Logger;
 import org.junit.Test;
 
@@ -18,4 +20,26 @@ public class DirectoryFlowLoaderTest {
 		logger.info(loader.getFlowMap().size());
 	}
 	
+	@Test
+	public void testLoadEmbeddedFlow() {
+		Logger logger = Logger.getLogger(this.getClass());
+		DirectoryFlowLoader loader = new DirectoryFlowLoader(logger);
+		
+		loader.loadProjectFlow(new File("unit/executions/embedded"));
+		Assert.assertEquals(0, loader.getErrors().size());
+	}
+	
+	@Test
+	public void testRecursiveLoadEmbeddedFlow() {
+		Logger logger = Logger.getLogger(this.getClass());
+		DirectoryFlowLoader loader = new DirectoryFlowLoader(logger);
+		
+		loader.loadProjectFlow(new File("unit/executions/embeddedBad"));
+		for (String error: loader.getErrors()) {
+			System.out.println(error);
+		}
+		
+		// Should be 3 errors: jobe->innerFlow, innerFlow->jobe, innerFlow
+		Assert.assertEquals(3, loader.getErrors().size());
+	}
 }
diff --git a/unit/java/azkaban/test/utils/PropsUtilsTest.java b/unit/java/azkaban/test/utils/PropsUtilsTest.java
index 1623005..09edb89 100644
--- a/unit/java/azkaban/test/utils/PropsUtilsTest.java
+++ b/unit/java/azkaban/test/utils/PropsUtilsTest.java
@@ -87,6 +87,7 @@ public class PropsUtilsTest {
 	
 	private void failIfNotException(Props props) {
 		try {
+			PropsUtils.resolveProps(props);
 			Assert.fail();
 		}
 		catch (UndefinedPropertyException e) {