azkaban-developers

Changes

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

unit/build.xml 22(+22 -0)

Details

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..95ca583 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;
@@ -231,7 +231,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 +259,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 e194687..7021c01 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,92 +341,71 @@ 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:
-			case FAILED_SUCCEEDED:
-				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;
+		}
+
+		long currentTime = System.currentTimeMillis();
+		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(currentTime);
+				fireEventListeners(Event.create(this, Type.JOB_FINISHED, node));
+			}
+			else if (nextStatus == Status.DISABLED) {
+				logger.info("Skipping disabled job " + node.getId() + ".");
+				node.skipNode(currentTime);
+				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) {
+				succeeded = false;
+			}
+			
+			Props output = node.getOutputProps();
 			if (output != null) {
 				output = Props.clone(output);
 				output.setParent(previousOutput);
@@ -465,58 +413,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;
+		}
+		
+		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;
+		}
 		
-		JobRunner jobRunner = new JobRunner(node, prop, path.getParentFile(), executorLoader, jobtypeManager);
+		// 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);
 		}
@@ -593,80 +678,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);
@@ -684,62 +744,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 FAILED_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:
-			case FAILED_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();
@@ -750,54 +759,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;
 	}
@@ -812,7 +828,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);
@@ -826,7 +842,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..97f10fe 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) {
diff --git a/src/java/azkaban/execapp/JobRunner.java b/src/java/azkaban/execapp/JobRunner.java
index de9bfe3..12805c3 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.jobId, node.getAttempt(), files);
+			} catch (ExecutorManagerException e) {
+				flowLogger.error("Error writing out logs for job " + this.jobId, 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);
+				changeStatus(Status.FAILED);
 				logError("Job run failed preparing the job.");
 			}
-			
-			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");
-			}
 		}
-		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,13 +462,13 @@ 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;
 				}
 			}
 			
 			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");
@@ -383,25 +486,47 @@ public class JobRunner extends EventHandler implements Runnable {
 			e.printStackTrace();
 			
 			if (props.getBoolean("job.succeed.on.failure", false)) {
-				node.setStatus(Status.FAILED_SUCCEEDED);
+				changeStatus(Status.FAILED_SUCCEEDED);
 				logError("Job run failed, but will treat it like success.");
 				logError(e.getMessage() + e.getCause());
 			}
 			else {
-				node.setStatus(Status.FAILED);
+				changeStatus(Status.FAILED);
 				logError("Job run failed!");
 				logError(e.getMessage() + e.getCause());
 			}
-			return;
 		}
-
-		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.");
@@ -439,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..d5ecc7f 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,6 +18,7 @@ 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;
@@ -25,6 +26,7 @@ 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 +36,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;
@@ -58,7 +75,7 @@ public class ExecutionOptions {
 	
 	private Set<String> initiallyDisabledJobs = new HashSet<String>();
 	
-	public void setFlowParameters(Map<String,String> flowParam) {
+	public void addAllFlowParameters(Map<String,String> flowParam) {
 		flowParameters.putAll(flowParam);
 	}
 	
@@ -168,21 +185,21 @@ public class ExecutionOptions {
 	
 	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 +210,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 = new HashSet<String>(wrapper.<String>getCollection(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 bf87720..be910d3 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
@@ -410,6 +410,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);
 
@@ -418,29 +420,43 @@ 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);
+					String[] splits = disabledId.split(":");
+					ExecutableNode node = exflow;
+					
+					for (String split: splits) {
+						if (node instanceof ExecutableFlowBase) {
+							node = ((ExecutableFlowBase)node).getExecutableNode(split);
+						}
+						else {
+							message = "Cannot disable job " + disabledId + " since flow " + split + " cannot be found. \n";
+						}
+					}
+
+					if (node == null) {
+						throw new ExecutorManagerException("Cannot disable job " + disabledId + ". Cannot find corresponding node.");
+					}
 					node.setStatus(Status.DISABLED);
 				}
 			}
 			
-			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";
 				}
 			}
 			
@@ -637,9 +653,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();
@@ -921,7 +934,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();
@@ -1141,8 +1154,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 5373b7d..b0eab38 100644
--- a/src/java/azkaban/executor/JdbcExecutorLoader.java
+++ b/src/java/azkaban/executor/JdbcExecutorLoader.java
@@ -390,7 +390,14 @@ public class JdbcExecutorLoader extends AbstractJdbcLoader implements ExecutorLo
 			}
 		}
 		
-		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(
@@ -398,8 +405,8 @@ 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(),
@@ -407,13 +414,13 @@ public class JdbcExecutorLoader extends AbstractJdbcLoader implements ExecutorLo
 					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=?";
+		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();
@@ -434,11 +441,12 @@ 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);
 		}
 	}
 	
diff --git a/src/java/azkaban/executor/Status.java b/src/java/azkaban/executor/Status.java
index c62287d..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
@@ -72,4 +72,15 @@ public enum Status {
 			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 ef59fbd..7324205 100644
--- a/src/java/azkaban/flow/CommonJobProperties.java
+++ b/src/java/azkaban/flow/CommonJobProperties.java
@@ -82,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.
 	 */
diff --git a/src/java/azkaban/flow/Node.java b/src/java/azkaban/flow/Node.java
index 7903fe1..1a62b03 100644
--- a/src/java/azkaban/flow/Node.java
+++ b/src/java/azkaban/flow/Node.java
@@ -32,6 +32,8 @@ public class Node {
 	private int expectedRunTimeSec = 1;
 	private String type;
 	
+	private String embeddedFlowId;
+	
 	public Node(String id) {
 		this.id = id;
 	}
@@ -102,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;
@@ -111,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) {
@@ -153,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/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 df9e1d1..491db7f 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;
 	}
 
 	/**
@@ -187,7 +187,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/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..e61c2d7 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 {
@@ -687,7 +689,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..a5afa44 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
@@ -70,17 +58,12 @@ public class ScheduleManager implements TriggerAgent {
 	 * 
 	 * @param loader
 	 */
+<<<<<<< HEAD
 	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 +161,6 @@ public class ScheduleManager implements TriggerAgent {
 		return scheduleIDMap.get(scheduleId);
 	}
 
-
 	/**
 	 * Removes the flow from the schedule if it exists.
 	 * 
@@ -199,6 +181,7 @@ public class ScheduleManager implements TriggerAgent {
 			removeSchedule(sched);
 		}
 	}
+
 	/**
 	 * Removes the flow from the schedule if it exists.
 	 * 
@@ -219,15 +202,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 +320,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 +334,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 +589,4 @@ public class ScheduleManager implements TriggerAgent {
 	public String getTriggerSource() {
 		return triggerSource;
 	}
-	
-
 }
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 81d4c5b..2bcd069 100644
--- a/src/java/azkaban/utils/DirectoryFlowLoader.java
+++ b/src/java/azkaban/utils/DirectoryFlowLoader.java
@@ -87,7 +87,8 @@ public class DirectoryFlowLoader {
 		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);
 		
@@ -179,9 +180,12 @@ public class DirectoryFlowLoader {
 	}
 	
 	private void resolveEmbeddedFlow(String flowId, Set<String> visited) {
-		visited.add(flowId);
-		
 		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);
@@ -321,6 +325,7 @@ public class DirectoryFlowLoader {
 				flowDependencies.put(flow.getId(), embeddedFlows);
 			}
 
+			node.setEmbeddedFlowId(embeddedFlow);
 			embeddedFlows.add(embeddedFlow);
 		}
 		Map<String, Edge> dependencies = nodeDependencies.get(node.getId());
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 2281ccf..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;
@@ -208,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/StringUtils.java b/src/java/azkaban/utils/StringUtils.java
index 77c5941..0ea393c 100644
--- a/src/java/azkaban/utils/StringUtils.java
+++ b/src/java/azkaban/utils/StringUtils.java
@@ -17,7 +17,6 @@
 package azkaban.utils;
 
 import java.util.Collection;
-import java.util.List;
 
 public class StringUtils {
 	public static final char SINGLE_QUOTE = '\'';
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/servlet/ExecutorServlet.java b/src/java/azkaban/webapp/servlet/ExecutorServlet.java
index 6cc091b..fdae097 100644
--- a/src/java/azkaban/webapp/servlet/ExecutorServlet.java
+++ b/src/java/azkaban/webapp/servlet/ExecutorServlet.java
@@ -21,6 +21,7 @@ import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 
 import javax.servlet.ServletConfig;
 import javax.servlet.ServletException;
@@ -28,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;
@@ -303,7 +306,7 @@ public class ExecutorServlet extends LoginAbstractAzkabanServlet {
 		else {
 			return project;
 		}
-		
+
 		return null;
 	}	
 
@@ -368,7 +371,7 @@ public class ExecutorServlet extends LoginAbstractAzkabanServlet {
 			ret.put("error", e.getMessage());
 		}
 	}
-	
+
 	/**
 	 * Gets the logs through plain text stream to reduce memory overhead.
 	 * 
@@ -574,10 +577,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());
@@ -588,7 +591,7 @@ public class ExecutorServlet extends LoginAbstractAzkabanServlet {
 		if (project == null) {
 			return;
 		}
-		
+
 		try {
 			executorManager.cancelFlow(exFlow, user.getUserId());
 		} catch (ExecutorManagerException e) {
@@ -641,66 +644,89 @@ 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;
-		}
-
+	private long fillUpdateExecutableFlowInfo(ExecutableFlowBase flow, long lastUpdateTime, HashMap<String, Object> ret) {
 		// 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) {
+		HashMap<String, Map<String,Object>> nodeMap = new HashMap<String, Map<String,Object>>();
+		
+		long updateTime = flow.getUpdateTime();
+		for (ExecutableNode node : flow.getExecutableNodes()) {
+			HashMap<String, Object> nodeObj = null;
+			if (node instanceof ExecutableFlowBase) {
+				nodeObj = new HashMap<String, Object>();
+				long subUpdateTime = fillUpdateExecutableFlowInfo((ExecutableFlowBase)node, lastUpdateTime, nodeObj);
+				updateTime = Math.max(updateTime, subUpdateTime);
+				if (updateTime <= lastUpdateTime) {
+					continue;
+				}
+			}
+			else if (node.getUpdateTime() <= lastUpdateTime){
 				continue;
 			}
-			
-			HashMap<String, Object> nodeObj = new HashMap<String,Object>();
-			nodeObj.put("id", node.getJobId());
-			nodeObj.put("status", node.getStatus());
-			nodeObj.put("startTime", node.getStartTime());
-			nodeObj.put("endTime", node.getEndTime());
-			nodeObj.put("attempt", node.getAttempt());
-			
-			if (node.getAttempt() > 0) {
-				nodeObj.put("pastAttempts", node.getAttemptObjects());
+			else {
+				nodeObj = new HashMap<String, Object>();
+				updateTime = Math.max(updateTime, node.getUpdateTime());
+	
+				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("attempt", node.getAttempt());
+	
+				if (node.getAttempt() > 0) {
+					nodeObj.put("pastAttempts", node.getAttemptObjects());
+				}
 			}
 			
+			nodeMap.put(node.getId(), nodeObj);
 			nodeList.add(nodeObj);
 		}
 
 		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());
+		ret.put("status", flow.getStatus().toString());
+		ret.put("startTime", flow.getStartTime());
+		ret.put("endTime", flow.getEndTime());
+		ret.put("updateTime", updateTime);
+		return updateTime;
 	}
 	
-	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);
+		Project project = getProjectAjaxByPermission(ret,
+				exFlow.getProjectId(), user, Type.READ);
 		if (project == null) {
 			return;
 		}
 		
+		fillUpdateExecutableFlowInfo(exFlow, lastUpdateTime, ret);
+	}
+
+	private long fillExecutableFlowInfo(ExecutableFlowBase flow, HashMap<String, Object> ret) {
+		long updateTime = flow.getUpdateTime();
+		
 		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());
+		ArrayList<Map<String, Object>> edgeList = new ArrayList<Map<String, Object>>();
+		
+		ArrayList<String> executorQueue = new ArrayList<String>();
+		executorQueue.addAll(flow.getStartNodes());
+
+		for (ExecutableNode node : flow.getExecutableNodes()) {
+			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("type", node.getType());
 			
 			// Add past attempts
 			if (node.getPastAttemptList() != null) {
 				ArrayList<Object> pastAttempts = new ArrayList<Object>();
-				for (ExecutableNode.Attempt attempt: node.getPastAttemptList()) {
+				for (ExecutionAttempt attempt : node.getPastAttemptList()) {
 					pastAttempts.add(attempt.toObject());
 				}
 				nodeObj.put("pastAttempts", pastAttempts);
@@ -709,24 +735,95 @@ public class ExecutorServlet extends LoginAbstractAzkabanServlet {
 			nodeList.add(nodeObj);
 			
 			// Add edges
-			for (String out: node.getOutNodes()) {
-				HashMap<String, Object> edgeObj = new HashMap<String,Object>();
-				edgeObj.put("from", node.getJobId());
+			for (String out : node.getOutNodes()) {
+				HashMap<String, Object> edgeObj = new HashMap<String, Object>();
+				edgeObj.put("from", node.getId());
 				edgeObj.put("target", out);
 				edgeList.add(edgeObj);
 			}
+			
+			// If it's an embedded flow, add the embedded flow info
+			if (node instanceof ExecutableFlowBase) {
+				long subUpdateTime = fillExecutableFlowInfo((ExecutableFlowBase)node, nodeObj);
+				updateTime = Math.max(updateTime, subUpdateTime);
+			}
+			else {
+				nodeObj.put("updateTime", updateTime);
+			}
 		}
-
+		
 		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("status", flow.getStatus().toString());
+		ret.put("startTime", flow.getStartTime());
+		ret.put("endTime", flow.getEndTime());
+		ret.put("updateTime", updateTime);
+		return updateTime;
+	}
+	
+	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;
+		}
+
+		fillExecutableFlowInfo(exFlow, ret);
 		ret.put("submitTime", exFlow.getSubmitTime());
 		ret.put("submitUser", exFlow.getSubmitUser());
+//		
+//		
+//		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.getId());
+//			nodeObj.put("status", node.getStatus());
+//			nodeObj.put("startTime", node.getStartTime());
+//			nodeObj.put("endTime", node.getEndTime());
+//			nodeObj.put("type", node.getType());
+//			
+//			// Add past attempts
+//			if (node.getPastAttemptList() != null) {
+//				ArrayList<Object> pastAttempts = new ArrayList<Object>();
+//				for (ExecutionAttempt 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.getId());
+//				edgeObj.put("target", out);
+//				edgeList.add(edgeObj);
+//			}
+//			
+//			// If it's an embedded flow, add the embedded flow info
+//			if (node instanceof ExecutableFlowBase) {
+//				
+//			}
+//		}
+//
+//		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());
 	}
-	
-	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");
 		
@@ -735,7 +832,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) {
@@ -755,7 +852,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) {
@@ -776,7 +873,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..264d3d4 100644
--- a/src/java/azkaban/webapp/servlet/HttpRequestUtils.java
+++ b/src/java/azkaban/webapp/servlet/HttpRequestUtils.java
@@ -96,7 +96,7 @@ 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");
diff --git a/src/java/azkaban/webapp/servlet/ProjectManagerServlet.java b/src/java/azkaban/webapp/servlet/ProjectManagerServlet.java
index 662bad0..8a271f5 100644
--- a/src/java/azkaban/webapp/servlet/ProjectManagerServlet.java
+++ b/src/java/azkaban/webapp/servlet/ProjectManagerServlet.java
@@ -28,6 +28,7 @@ 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;
@@ -66,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;
@@ -208,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);
@@ -455,7 +462,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");
 		
@@ -547,6 +553,50 @@ public class ProjectManagerServlet extends LoginAbstractAzkabanServlet {
 	
 	private void ajaxFetchFlowGraph(Project project, HashMap<String, Object> ret, HttpServletRequest req) throws ServletException {
 		String flowId = getParam(req, "flow");
+		
+		fillFlowInfo2(project, flowId, ret);
+	}
+	
+	private void fillFlowInfo2(Project project, String flowId, HashMap<String, Object> ret) {
+		Flow flow = project.getFlow(flowId);
+		
+		ArrayList<Map<String, Object>> nodeList = new ArrayList<Map<String, Object>>();
+		for (Node node: flow.getNodes()) {
+			HashMap<String, Object> nodeObj = new HashMap<String, Object>();
+			nodeObj.put("id", node.getId());
+			nodeObj.put("type", node.getType());
+			if (node.getEmbeddedFlowId() != null) {
+				nodeObj.put("flowId", node.getEmbeddedFlowId());
+				HashMap<String, Object> embeddedNodeObj = new HashMap<String, Object>();
+				fillFlowInfo2(project, node.getEmbeddedFlowId(), embeddedNodeObj);
+				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);
+			}
+		}
+		
+		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"));
+			}
+		});
+
+		ret.put("flow", flowId);
+		ret.put("nodes", nodeList);
+	}
+	
+	private void fillFlowInfo(Project project, String flowId, HashMap<String, Object> ret) {
 		Flow flow = project.getFlow(flowId);
 		
 		//Collections.sort(flowNodes, NODE_LEVEL_COMPARATOR);
@@ -555,7 +605,11 @@ public class ProjectManagerServlet extends LoginAbstractAzkabanServlet {
 			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());
+			}
+			
 			nodeList.add(nodeObj);
 		}
 		
@@ -577,6 +631,47 @@ public class ProjectManagerServlet extends LoginAbstractAzkabanServlet {
 		ret.put("edges", edgeList);
 	}
 	
+	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("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) {
+				HashMap<String, Object> flowMap = new HashMap<String, Object>();
+				fillFlowInfo2(project, node.getEmbeddedFlowId(), flowMap);
+				ret.put("flowData", flowMap);
+			}
+		}
+	}
+	
 	private void ajaxFetchFlow(Project project, HashMap<String, Object> ret, HttpServletRequest req, HttpServletResponse resp) throws ServletException {
 		String flowId = getParam(req, "flow");
 		Flow flow = project.getFlow(flowId);
diff --git a/src/java/azkaban/webapp/servlet/velocity/executingflowpage.vm b/src/java/azkaban/webapp/servlet/velocity/executingflowpage.vm
index 693a707..b290f48 100644
--- a/src/java/azkaban/webapp/servlet/velocity/executingflowpage.vm
+++ b/src/java/azkaban/webapp/servlet/velocity/executingflowpage.vm
@@ -29,9 +29,9 @@
 		<script type="text/javascript" src="${context}/js/azkaban.ajax.utils.js"></script>
 		<script type="text/javascript" src="${context}/js/azkaban.job.status.utils.js"></script>
 		<script type="text/javascript" src="${context}/js/azkaban.layout.js"></script>
-		<script type="text/javascript" src="${context}/js/azkaban.exflow.view.js"></script>
 		<script type="text/javascript" src="${context}/js/azkaban.flow.job.view.js"></script>
 		<script type="text/javascript" src="${context}/js/azkaban.svg.graph.view.js"></script>
+		<script type="text/javascript" src="${context}/js/azkaban.exflow.view.js"></script>
 		<script type="text/javascript" src="${context}/js/svgNavigate.js"></script>
 		<script type="text/javascript">
 			var contextURL = "${context}";
@@ -44,8 +44,10 @@
 			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" />
+		<link rel="stylesheet" type="text/css" href="${context}/css/azkaban-graph.css" /> 
 	</head>
 	<body>
 
diff --git a/src/java/azkaban/webapp/servlet/velocity/flowexecutionpanel.vm b/src/java/azkaban/webapp/servlet/velocity/flowexecutionpanel.vm
index cda497a..281bdf7 100644
--- a/src/java/azkaban/webapp/servlet/velocity/flowexecutionpanel.vm
+++ b/src/java/azkaban/webapp/servlet/velocity/flowexecutionpanel.vm
@@ -220,4 +220,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/flowpage.vm b/src/java/azkaban/webapp/servlet/velocity/flowpage.vm
index 83dfff0..a3672b3 100644
--- a/src/java/azkaban/webapp/servlet/velocity/flowpage.vm
+++ b/src/java/azkaban/webapp/servlet/velocity/flowpage.vm
@@ -47,8 +47,8 @@
 			var flowId = "${flowid}";
 			var execId = null;
 		</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/azkaban-graph.css" /> 
 	</head>
 	<body>
 
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/web/css/azkaban-graph.css b/src/web/css/azkaban-graph.css
new file mode 100644
index 0000000..c78a508
--- /dev/null
+++ b/src/web/css/azkaban-graph.css
@@ -0,0 +1,371 @@
+.nodebox text {
+	pointer-events: none;
+}
+
+.nodebox image {
+	pointer-events: none;
+}
+
+/* Nodes */
+.node:hover {
+	cursor: pointer;
+}
+
+.nodebox > .border:hover {
+	fill-opacity: 0.7;
+}
+
+.nodebox > .flowborder:hover {
+	stroke-opacity: 0.7;
+}
+
+.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;
+}
+
+.FAILED_FINISHING > g > rect {
+	fill: #ed9c28;
+	stroke: #ed9c28;
+}
+
+.FAILED_FINISHING > g > text {
+	fill: #FFF;
+}
+
+.DISABLED > g > rect {
+	fill: #800000;
+	stroke: #800000;
+}
+
+.DISABLED > g > rect {
+	fill: #800000;
+	stroke: #800000;
+}
+
+.SKIPPED > g > rect {
+	fill: #CCC;
+	stroke: #CCC;
+}
+
+.SKIPPED > g > rect {
+	fill: #CCC;
+	stroke: #CCC;
+}
+
+.DISABLED {
+	opacity: 0.25;
+}
+
+.SKIPPED {
+	opacity: 0.25;
+}
+
+/* Edges */
+.edge {
+	stroke: #CCC;
+	stroke-width: 1.5;
+}
+
+.edge:hover {
+	stroke: #009FC9;
+	stroke-width: 1.5;
+}
+
+/*
+svg text1 {
+	pointer-events: none;
+}
+
+svg g.nodebox1 {
+	pointer-events: none;
+}
+
+svg .edge {
+	stroke: #BBB;
+	stroke-width: 2;
+}
+
+svg .edge:hover {
+	stroke: #009FC9;
+	stroke-width: 4;
+}
+
+svg .node.disabled {
+	opacity: 0.3;
+}
+
+svg .node .backboard {
+	fill: #FFF;
+	opacity: 0.05;
+}
+
+svg .node:hover {
+	cursor: pointer;
+}
+
+svg .node:hover .backboard {
+	opacity: 0.7;
+}
+
+svg .selected .backboard {
+	opacity: 0.4;
+}
+
+svg .node circle {
+	fill: #888;
+	stroke: #777;
+	stroke-width: 2;
+}
+
+svg .node:hover circle {
+	stroke: #009FC9;
+}
+
+svg .node:hover text {
+	fill: #009FC9;
+}
+
+svg .selected text {
+	fill: #338AB0;
+}
+
+svg .selected circle {
+	stroke: #009FC9;
+	stroke-width: 4;
+}
+
+svg .READY circle {
+	fill: #CCC;
+}
+
+svg .RUNNING circle {
+	fill: #009FC9;
+}
+
+svg .QUEUED circle {
+	opacity: 0.5;
+	fill: #009FC9;
+}
+
+svg .FAILED circle {
+	fill: #CC0000;
+}
+
+svg .KILLED circle {
+	fill: #CC0000;
+}
+
+svg .SUCCEEDED circle {
+	fill: #00CC33;
+}
+
+svg .DISABLED circle {
+	opacity: 0.3;
+}
+
+svg .SKIPPED circle {
+	opacity: 0.3;
+}
+
+svg .selected circle {
+	stroke: #009FC9;
+	stroke-width: 4;
+}
+
+svg .selected .nodebox rect {
+	stroke: #009FC9;
+	stroke-width: 3;
+}
+
+svg .node rect {
+	fill: #CCC;
+	stroke: #CCC;
+	stroke-width: 2;
+}
+
+svg .node .nodebox text {
+	fill: #FFF;
+}
+
+svg .READY .nodebox text {
+	fill: #000;
+}
+
+svg .node:hover rect {
+	stroke: #009FC9;
+}
+
+svg .READY rect {
+	fill: #CCC;
+}
+
+svg .RUNNING rect {
+	fill: #009FC9;
+}
+
+svg .QUEUED rect {
+	opacity: 0.5;
+	fill: #009FC9;
+}
+
+svg .FAILED rect {
+	fill: #CC0000;
+}
+
+svg .KILLED rect {
+	fill: #CC0000;
+}
+
+svg .SUCCEEDED rect {
+	fill: #30ad23;
+}
+
+svg .DISABLED rect {
+	opacity: 0.3;
+}
+
+svg .SKIPPED rect {
+	opacity: 0.3;
+}
+
+svg .nodebox text {
+	fill: #fff;
+}
+
+
+#Used for charts
+svg circle.READY {
+	stroke: #CCC;
+	stroke-width: 2px;
+	fill: #FFF;
+}
+
+svg circle.RUNNING {
+	stroke: #009FC9;
+	stroke-width: 2px;
+	fill: #FFF;
+}
+
+svg circle.FAILED {
+	stroke: #CC0000;
+	stroke-width: 2px;
+	fill: #FFF;
+}
+
+svg circle.KILLED {
+	stroke: #CC0000;
+	stroke-width: 2px;
+	fill: #FFF;
+}
+
+svg circle.SUCCEEDED {
+	stroke: #00CC33;
+	stroke-width: 2px;
+	fill: #FFF;
+}
+
+svg circle.DISABLED {
+	stroke: #CCC;
+	opacity: 0.3;
+	stroke-width: 2px;
+	fill: #FFF;
+}
+
+svg circle.SKIPPED {
+	stroke: #CCC;
+	opacity: 0.3;
+	stroke-width: 2px;
+	fill: #FFF;
+}
+
+.flowExtendedView {
+	position: absolute;
+	background-color: rgba(255, 255, 255, 0.95);
+	-moz-box-shadow: 1px 1px 5px rgba(0, 0, 0, 0.5);
+	-webkit-box-shadow: 1px 1px 5px rgba(0, 0, 0, 0.5);
+	box-shadow:         1px 1px 5px rgba(0, 0, 0, 0.5);
+
+	min-width: 200px;
+	min-height: 150px;
+}
+
+.dataJobProperties {
+
+}
+
+.flowInfoTitle {
+	padding-top: 8px;
+	padding-left: 8px;
+	padding-bottom: 5px;
+	cursor: pointer;
+}
+
+.flowInfoTitle:hover {
+	background-color: #CCC;
+}
+
+.nodeId {
+	font-size: 16px;
+	font-weight: bold;
+	margin: 20px 20px;
+}
+
+.nodeType {
+	font-style: italic;
+}
+
+.dataContent {
+	margin: 5px;
+}
+
+.dataFlow {
+	width: 100%;
+}
+
+.svgTiny {
+	width: 100%;
+	height: 100%;
+}
+*/
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.execute.view.js b/src/web/js/azkaban.flow.execute.view.js
index 2fc3b4b..f309163 100644
--- a/src/web/js/azkaban.flow.execute.view.js
+++ b/src/web/js/azkaban.flow.execute.view.js
@@ -552,7 +552,7 @@ var touchDescendents = function(jobid, disable) {
 	executableGraphModel.trigger("change:disabled");
 }
 
-var nodeClickCallback = function(event) {
+var exNodeClickCallback = function(event) {
 	console.log("Node clicked callback");
 	var jobId = event.currentTarget.jobid;
 	var flowId = executableGraphModel.get("flowId");
@@ -580,11 +580,11 @@ var nodeClickCallback = function(event) {
 	contextMenuView.show(event, menu);
 }
 
-var edgeClickCallback = function(event) {
+var exEdgeClickCallback = function(event) {
 	console.log("Edge clicked callback");
 }
 
-var graphClickCallback = function(event) {
+var exGraphClickCallback = function(event) {
 	console.log("Graph clicked callback");
 	var flowId = executableGraphModel.get("flowId");
 	var requestURL = contextURL + "/manager?project=" + projectName + "&flow=" + flowId;
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.flow.job.view.js b/src/web/js/azkaban.flow.job.view.js
index fe8cf6d..cb28674 100644
--- a/src/web/js/azkaban.flow.job.view.js
+++ b/src/web/js/azkaban.flow.job.view.js
@@ -20,6 +20,7 @@ azkaban.JobListView = Backbone.View.extend({
 		"click .job": "handleJobClick",
 		"click .resetPanZoomBtn": "handleResetPanZoom",
 		"contextmenu li": "handleContextMenuClick"
+		"change .autoPanZoom" : "handleAutoPanZoom",
 	},
 	
 	initialize: function(settings) {
@@ -112,7 +113,6 @@ azkaban.JobListView = Backbone.View.extend({
 	render: function(self) {
 		var data = this.model.get("data");
 		var nodes = data.nodes;
-		var edges = data.edges;
 		
 		this.listNodes = {}; 
 		if (nodes.length == 0) {
@@ -211,5 +211,8 @@ azkaban.JobListView = Backbone.View.extend({
 	
 	handleResetPanZoom: function(evt) {
 		this.model.trigger("resetPanZoom");
+	},
+	handleAutoPanZoom: function(evt) {
+		this.model.set({"autoPanZoom": $(evt.currentTarget).is(':checked')});
 	}
 });
diff --git a/src/web/js/azkaban.layout.js b/src/web/js/azkaban.layout.js
index 6f6c2c5..96cdd21 100644
--- a/src/web/js/azkaban.layout.js
+++ b/src/web/js/azkaban.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.svg.exflow.helper.js b/src/web/js/azkaban.svg.exflow.helper.js
new file mode 100644
index 0000000..6604020
--- /dev/null
+++ b/src/web/js/azkaban.svg.exflow.helper.js
@@ -0,0 +1,200 @@
+var extendedViewPanels = {};
+var extendedDataModels = {};
+var openJobDisplayCallback = function(nodeId, flowId, evt) {
+	console.log("Open up data");
+	
+	var nodeInfoPanelID = flowId + ":" + nodeId + "-info";
+	if ($("#" + nodeInfoPanelID).length) {
+		$("#flowInfoBase").before(cloneStuff);
+		extendedViewPanels[nodeInfoPanelID].showExtendedView(evt);
+		return;
+	}
+	
+	var cloneStuff = $("#flowInfoBase").clone();
+	$(cloneStuff).attr("id", nodeInfoPanelID);
+	
+	
+	
+	/*
+	$("#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 extendedDataModels = {};
+var createModelFromAjaxCall = function(data, model) {
+	  var nodes = {};
+  	  for (var i=0; i < data.nodes.length; ++i) {
+  		var graphModel = new azkaban.GraphModel();
+  	  	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;
+  	  }
+  
+      var nodeQueue = new Array();
+      for (var key in nodes) {
+          if (!nodes[key].inNodes) {
+             nodeQueue.push(nodes[key]);
+          }
+      }
+  	  
+      // 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;
+          }
+          
+          if (node.outNodes) {
+             for (var key in node.outNodes) {
+                 nodeQueue.push(node.outNodes[key]);
+             }
+          }
+          index++;
+      }
+      
+      for (var key in nodes) {
+    	  var node = nodes[key];
+    	  
+    	  if (node.type == "flow") {
+    		  var graphModel = new azkaban.GraphModel();
+    		  createModelFromAjaxCall(node, graphModel);
+    		  extendedDataModels["test"] = graphModel;
+    	  }
+      }
+
+      console.log("data fetched");
+      model.set({data: data});
+      model.set({nodes: nodes});
+      model.set({disabled: {}});
+}
+
+var nodeClickCallback = function(event, model, type) {
+	console.log("Node clicked callback");
+	var jobId = event.currentTarget.jobid;
+	var flowId = model.get("flowId");
+	var requestURL = contextURL + "/manager?project=" + projectName + "&flow=" + flowId + "&job=" + jobId;
+
+	if (event.currentTarget.jobtype == "flow") {
+		var flowRequestURL = contextURL + "/manager?project=" + projectName + "&flow=" + event.currentTarget.flowId;
+		menu = [
+				{title: "View Flow...", 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", jobId)}}
+		];
+	}
+	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() {model.trigger("centerNode", jobId)}}
+		];
+	}
+	contextMenuView.show(event, menu);
+}
+
+var jobClickCallback = function(event, model) {
+	console.log("Node clicked callback");
+	var jobId = event.currentTarget.jobid;
+	var requestURL = contextURL + "/manager?project=" + projectName + "&flow=" + flowId + "&job=" + jobId;
+
+	var menu;
+	if (event.currentTarget.jobtype == "flow") {
+		var flowRequestURL = contextURL + "/manager?project=" + projectName + "&flow=" + event.currentTarget.flowId;
+		menu = [
+				{title: "View Flow...", 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", jobId)}}
+		];
+	}
+	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", jobId)}}
+		];
+	}
+	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.svg.flow.helper.js b/src/web/js/azkaban.svg.flow.helper.js
new file mode 100644
index 0000000..96e56a4
--- /dev/null
+++ b/src/web/js/azkaban.svg.flow.helper.js
@@ -0,0 +1,165 @@
+var extendedViewPanels = {};
+var extendedDataModels = {};
+var openJobDisplayCallback = function(nodeId, flowId, evt) {
+	console.log("Open up data");
+	
+	var nodeInfoPanelID = flowId + ":" + nodeId + "-info";
+	if ($("#" + nodeInfoPanelID).length) {
+		$("#flowInfoBase").before(cloneStuff);
+		extendedViewPanels[nodeInfoPanelID].showExtendedView(evt);
+		return;
+	}
+	
+	var cloneStuff = $("#flowInfoBase").clone();
+	$(cloneStuff).attr("id", nodeInfoPanelID);
+	
+	$("#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 closeAllSubDisplays = function() {
+	for (var key in extendedViewPanels) {
+		var panel = extendedViewPanels[key];
+		panel.handleClosePanel();
+	}
+}
+
+var createModelFromAjaxCall = function(data, model) {
+	var nodes = {};
+	for (var i=0; i < data.nodes.length; ++i) {
+		var node = data.nodes[i];
+		nodes[node.id] = node;
+	}
+	
+	for (var nodeKey in nodes) {
+		var node = nodes[nodeKey];
+		if (node.)
+	}
+	
+	for (var nodeKey in nodes) {
+		var node = nodes[nodeKey];
+		if (node.in) {
+			node.inNodes = {};
+			for (var i=0; i < node.in.length; ++i) {
+				node.inNodes[node.in[i]] = nodes[node.in[i]];
+			}
+		}
+		if (node.out) {
+			node.outNodes = {};
+			for (var i=0; i < node.out.length; ++i) {
+				node.outNodes[node.out[i]] = nodes[node.out[i]];
+			}
+		}
+	}
+
+	console.log("data fetched");
+	model.set({data: data});
+	model.set({nodes: nodes});
+	model.set({disabled: {}});
+}
+
+var nodeClickCallback = function(event, model, type) {
+	console.log("Node clicked callback");
+	var jobId = event.currentTarget.jobid;
+	var flowId = model.get("flowId");
+	var requestURL = contextURL + "/manager?project=" + projectName + "&flow=" + flowId + "&job=" + jobId;
+
+	if (event.currentTarget.jobtype == "flow") {
+		var flowRequestURL = contextURL + "/manager?project=" + projectName + "&flow=" + event.currentTarget.flowId;
+		menu = [
+				{title: "View Flow...", 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", jobId)}}
+		];
+	}
+	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() {model.trigger("centerNode", jobId)}}
+		];
+	}
+	contextMenuView.show(event, menu);
+}
+
+var jobClickCallback = function(event, model) {
+	console.log("Node clicked callback");
+	var jobId = event.currentTarget.jobid;
+	var requestURL = contextURL + "/manager?project=" + projectName + "&flow=" + flowId + "&job=" + jobId;
+
+	var menu;
+	if (event.currentTarget.jobtype == "flow") {
+		var flowRequestURL = contextURL + "/manager?project=" + projectName + "&flow=" + event.currentTarget.flowId;
+		menu = [
+				{title: "View Flow...", 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", jobId)}}
+		];
+	}
+	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", jobId)}}
+		];
+	}
+	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.svg.flow.loader.js b/src/web/js/azkaban.svg.flow.loader.js
new file mode 100644
index 0000000..88e46cf
--- /dev/null
+++ b/src/web/js/azkaban.svg.flow.loader.js
@@ -0,0 +1,201 @@
+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;
+	}
+
+	// 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" && node.flowData) {
+			processFlowData(node.flowData);
+			// Weird cycle. Evaluate whether we can instead unwrap these things.
+			node.flowData.node = node;
+		}
+	}
+	
+	// Assign the node map and the edge list
+	console.log("data fetched");
+	data.nodeMap = nodes;
+	data.edges = edges;
+	data.disabled = {};
+}
+
+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.flowId;
+	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=" + event.currentTarget.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) {
+	console.log("Node clicked callback");
+	var jobId = event.currentTarget.jobid;
+	var node = target.data;
+	var requestURL = contextURL + "/manager?project=" + projectName + "&flow=" + flowId + "&job=" + jobId;
+
+	var menu;
+	if (event.currentTarget.jobtype == "flow") {
+		var flowRequestURL = contextURL + "/manager?project=" + projectName + "&flow=" + event.currentTarget.flowId;
+		menu = [
+				{title: "View Flow...", 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.svg.graph.view.js b/src/web/js/azkaban.svg.graph.view.js
index 67d2530..7f90247 100644
--- a/src/web/js/azkaban.svg.graph.view.js
+++ b/src/web/js/azkaban.svg.graph.view.js
@@ -13,115 +13,106 @@
  * 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"
+
+	},
+	test: function() {
+		console.log("test");
 	},
 	
   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];
 		this.svgGraph = svg;
-		
+		$(this.svgGraph).svg();
+		this.svg = $(svg).svg('get');
+
+		// 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();
+		
+		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;
+			});
+		}
+
+		if (settings.render) {
+			this.render();
+		}
 	},
-	
-  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() {
+		console.log("graph render");
+		
+		this.graphBounds = this.renderGraph(this.model.get("data"), this.mainG);
+		this.resetPanZoom(0);
 	},
-	
-  render: function(self) {
+	renderGraph: function(data, g) {
 		console.log("graph render");
 
-		// Clean everything
-		while (this.mainG.lastChild) {
-			this.mainG.removeChild(this.mainG.lastChild);
-		}
-
-		var data = this.model.get("data");
+		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]);
+		for (var i = 0; i < nodes.length; ++i) {
+			this.drawNode(this, nodes[i], g);
 		}
+
+		// 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,10 +120,56 @@ 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 (data.disabled && data.disabled.length > 0) {
+			this.handleDisabledChange(self);
+		}
+
+/*
+		if (this.rightClick) {
+			var callbacks = this.rightClick;
+			var currentTarget = self.currentTarget;
+			if (callbacks.node && currentTarget.jobid) {
+				callbacks.node(self, this.model, currentTarget.nodeobj);
+			}
+			else if (callbacks.edge && (currentTarget.nodeName == "polyline" || currentTarget.nodeName == "line")) {
+				callbacks.edge(self, this.model);
+			}
+			else if (callbacks.graph) {
+				callbacks.graph(self, this.model);
+			}
+			return false;
+		}
+	
+*/
+
+		var self = this;
+		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;
+				});
+			}
+
+		};
+
+		return bounds;
 	},
 	
   handleDisabledChange: function(evt) {
@@ -150,13 +187,14 @@ azkaban.SvgGraphView = Backbone.View.extend({
 			}
 		}
 	},
-	
-  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);
+			$(g).attr("title", initialStatus);
 		}
 	},
 	
@@ -177,17 +215,15 @@ azkaban.SvgGraphView = Backbone.View.extend({
 			
 			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")) {
+				var offset = 150;
+				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});
+			}
 		}
 	},
 	
@@ -201,6 +237,7 @@ azkaban.SvgGraphView = Backbone.View.extend({
 				this.handleRemoveAllStatus(g);
 				
 				addClass(g, updateNode.status);
+				$(g).attr("title", updateNode.status);
 			}
 		}
 	},
@@ -224,138 +261,370 @@ azkaban.SvgGraphView = Backbone.View.extend({
 			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);
+		}
+// 		
+// 		var boundingBox = node.gNode.getBBox();
+// 		node.width = boundingBox.width;
+// 		node.height = boundingBox.height;
+// 		node.centerX = node.width/2;
+// 		node.centerY = node.height/2;
+	},
+	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 flowData = node.flowData;
+		
+		var bbox;
+		if (!innerG.expandedFlow) {
+			var topmargin= 30, bottommargin=5;
+			var hmargin = 10;
+		
+			var expandedFlow = svg.group(innerG, "", {class: "expandedGraph"});
+			this.renderGraph(flowData, 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;
+		var flowData = node.flowData;
+
+		removeClass(innerG, "expanded");
+		addClass(innerG, "collapsed");
+
+		node.height = node.collapsedHeight;
+		node.width = node.collapsedWidth;
+
+		$(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 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); 
+		var parent = node.parent;
+		if (parent) {
+			layoutGraph(parent.nodes, parent.edges, 10);
+			if (parent.node) {
+				this.relayoutFlow(parent.node);
+			}
+		}
+		
+		// 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;
+				
+			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]);
 				
-		this.addBounds(bounds, {
-			minX: node.x - xOffset, 
-			minY: node.y - yOffset, 
-			maxX: node.x + xOffset, 
-			maxY: node.y + yOffset
-		});
-		
-		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;
+				animatePolylineEdge(svg, edge, pointArray, 250);
+				edge.oldpoints = pointArray;
+			}
+			else {
+				$(line).animate({svgX1: startNode.x, svgY1: startPointY, svgX2: endNode.x, svgY2: endPointY});
+			}
+		}
+
 	},
-	
-  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;
+	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;
+		
+		return bounds;
+	},
+	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;
+	},
+	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.flowData.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.node) {
+		
+			var parentPos = this.globalNodePosition(node.parent.node);
+			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/svgNavigate.js b/src/web/js/svgNavigate.js
index b42f8c1..6fcb7cd 100644
--- a/src/web/js/svgNavigate.js
+++ b/src/web/js/svgNavigate.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/svgutils.js b/src/web/js/svgutils.js
new file mode 100644
index 0000000..33dea78
--- /dev/null
+++ b/src/web/js/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

unit/build.xml 22(+22 -0)

diff --git a/unit/build.xml b/unit/build.xml
index cb07807..c9f3f20 100644
--- a/unit/build.xml
+++ b/unit/build.xml
@@ -87,4 +87,26 @@
 		</zip>
 	</target>
 	
+	<target name="package-embedded" depends="jars" description="Creates a test zip">
+		<delete dir="${dist.packages.dir}" />
+		<mkdir dir="${dist.packages.dir}" />
+		
+		<!-- Tarball it -->
+		<zip destfile="${dist.packages.dir}/embedded.zip">
+			<zipfileset dir="${dist.jar.dir}" />
+			<zipfileset dir="${base.dir}/unit/executions/embedded" />
+		</zip>
+	</target>
+
+	<target name="package-embedded3" depends="jars" description="Creates a test zip">
+		<delete dir="${dist.packages.dir}" />
+		<mkdir dir="${dist.packages.dir}" />
+
+                <!-- Tarball it -->
+                <zip destfile="${dist.packages.dir}/embedded.zip">
+                	<zipfileset dir="${dist.jar.dir}" />
+                	<zipfileset dir="${base.dir}/unit/executions/embedded3" />
+                </zip>
+
+	</target>	
 </project>
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/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..9031518
--- /dev/null
+++ b/unit/java/azkaban/test/executor/ExecutableFlowTest.java
@@ -0,0 +1,352 @@
+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 String[] {"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());
+		
+		testEquals(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);
+		}
+	}
+	
+	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/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) {