azkaban-developers
Changes
src/java/azkaban/execapp/FlowRunner.java 600(+308 -292)
src/java/azkaban/execapp/JobRunner.java 409(+265 -144)
src/java/azkaban/executor/ExecutableFlow.java 456(+95 -361)
src/java/azkaban/executor/ExecutableFlowBase.java 409(+409 -0)
src/java/azkaban/executor/ExecutableNode.java 537(+296 -241)
src/java/azkaban/executor/ExecutionOptions.java 110(+57 -53)
src/java/azkaban/executor/ExecutorManager.java 35(+22 -13)
src/java/azkaban/executor/Status.java 15(+13 -2)
src/java/azkaban/flow/Node.java 18(+17 -1)
src/java/azkaban/user/User.java 7(+7 -0)
src/java/azkaban/utils/Props.java 7(+6 -1)
src/java/azkaban/utils/PropsUtils.java 36(+18 -18)
src/java/azkaban/utils/TypedMapWrapper.java 141(+141 -0)
src/java/azkaban/webapp/servlet/ExecutorServlet.java 203(+150 -53)
src/web/css/azkaban-graph.css 371(+371 -0)
src/web/images/graph-icon.png 0(+0 -0)
src/web/js/azkaban.flow.extended.view.js 62(+62 -0)
src/web/js/azkaban.flow.job.view.js 5(+4 -1)
src/web/js/azkaban.layout.js 144(+96 -48)
src/web/js/azkaban.svg.exflow.helper.js 200(+200 -0)
src/web/js/azkaban.svg.flow.helper.js 165(+165 -0)
src/web/js/azkaban.svg.flow.loader.js 201(+201 -0)
src/web/js/azkaban.svg.graph.view.js 651(+460 -191)
src/web/js/svgNavigate.js 15(+13 -2)
src/web/js/svgutils.js 70(+70 -0)
unit/build.xml 22(+22 -0)
unit/executions/embedded/joba.job 4(+4 -0)
unit/executions/embedded/jobb.job 3(+3 -0)
unit/executions/embedded/jobc.job 3(+3 -0)
unit/executions/embedded/jobd.job 3(+3 -0)
unit/executions/embedded/jobe.job 5(+5 -0)
unit/executions/embedded2/joba.job 2(+2 -0)
unit/executions/embedded2/joba1.job 1(+1 -0)
unit/executions/embedded2/jobb.job 5(+5 -0)
unit/executions/embedded2/jobc.job 2(+2 -0)
unit/executions/embedded2/jobd.job 4(+4 -0)
unit/executions/embedded2/jobe.job 2(+2 -0)
unit/executions/embedded2/jobf.job 2(+2 -0)
unit/executions/embedded2/jobg.job 2(+2 -0)
unit/executions/embedded3/joba.job 4(+4 -0)
unit/executions/embedded3/jobb.job 3(+3 -0)
unit/executions/embedded3/jobc.job 3(+3 -0)
unit/executions/embedded3/jobd.job 3(+3 -0)
unit/executions/embedded3/jobe.job 5(+5 -0)
unit/executions/embeddedBad/joba.job 4(+4 -0)
unit/executions/embeddedBad/jobb.job 3(+3 -0)
unit/executions/embeddedBad/jobc.job 3(+3 -0)
unit/executions/embeddedBad/jobd.job 3(+3 -0)
unit/executions/embeddedBad/jobe.job 5(+5 -0)
unit/java/azkaban/test/execapp/FlowRunnerTest2.java 896(+896 -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);
}
src/java/azkaban/execapp/FlowRunner.java 600(+308 -292)
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) {
src/java/azkaban/execapp/JobRunner.java 409(+265 -144)
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";
src/java/azkaban/executor/ExecutableFlow.java 456(+95 -361)
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
src/java/azkaban/executor/ExecutableFlowBase.java 409(+409 -0)
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
src/java/azkaban/executor/ExecutableNode.java 537(+296 -241)
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
src/java/azkaban/executor/ExecutionOptions.java 110(+57 -53)
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;
}
src/java/azkaban/executor/ExecutorManager.java 35(+22 -13)
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);
}
}
src/java/azkaban/executor/Status.java 15(+13 -2)
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.
*/
src/java/azkaban/flow/Node.java 18(+17 -1)
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;
}
-
-
}
src/java/azkaban/user/User.java 7(+7 -0)
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());
src/java/azkaban/utils/Props.java 7(+6 -1)
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;
src/java/azkaban/utils/PropsUtils.java 36(+18 -18)
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 = '\'';
src/java/azkaban/utils/TypedMapWrapper.java 141(+141 -0)
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);
+ }
+}
src/java/azkaban/webapp/servlet/ExecutorServlet.java 203(+150 -53)
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
src/web/css/azkaban-graph.css 371(+371 -0)
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%;
+}
+*/
src/web/images/graph-icon.png 0(+0 -0)
diff --git a/src/web/images/graph-icon.png b/src/web/images/graph-icon.png
new file mode 100644
index 0000000..d315d97
Binary files /dev/null and b/src/web/images/graph-icon.png differ
diff --git a/src/web/js/azkaban.flow.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;
src/web/js/azkaban.flow.extended.view.js 62(+62 -0)
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
src/web/js/azkaban.flow.job.view.js 5(+4 -1)
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')});
}
});
src/web/js/azkaban.layout.js 144(+96 -48)
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;
}
}
}
src/web/js/azkaban.svg.exflow.helper.js 200(+200 -0)
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);
+}
src/web/js/azkaban.svg.flow.helper.js 165(+165 -0)
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);
+}
src/web/js/azkaban.svg.flow.loader.js 201(+201 -0)
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);
+}
+
src/web/js/azkaban.svg.graph.view.js 651(+460 -191)
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);
}
});
src/web/js/svgNavigate.js 15(+13 -2)
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;
src/web/js/svgutils.js 70(+70 -0)
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
unit/executions/embedded/joba.job 4(+4 -0)
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
unit/executions/embedded/jobb.job 3(+3 -0)
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
unit/executions/embedded/jobc.job 3(+3 -0)
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
unit/executions/embedded/jobd.job 3(+3 -0)
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
unit/executions/embedded/jobe.job 5(+5 -0)
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
unit/executions/embedded2/joba.job 2(+2 -0)
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
unit/executions/embedded2/joba1.job 1(+1 -0)
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
unit/executions/embedded2/jobb.job 5(+5 -0)
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
unit/executions/embedded2/jobc.job 2(+2 -0)
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
unit/executions/embedded2/jobd.job 4(+4 -0)
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
unit/executions/embedded2/jobe.job 2(+2 -0)
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
unit/executions/embedded2/jobf.job 2(+2 -0)
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
unit/executions/embedded2/jobg.job 2(+2 -0)
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
unit/executions/embedded3/joba.job 4(+4 -0)
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
unit/executions/embedded3/jobb.job 3(+3 -0)
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
unit/executions/embedded3/jobc.job 3(+3 -0)
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
unit/executions/embedded3/jobd.job 3(+3 -0)
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
unit/executions/embedded3/jobe.job 5(+5 -0)
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
unit/executions/embeddedBad/joba.job 4(+4 -0)
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
unit/executions/embeddedBad/jobb.job 3(+3 -0)
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
unit/executions/embeddedBad/jobc.job 3(+3 -0)
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
unit/executions/embeddedBad/jobd.job 3(+3 -0)
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
unit/executions/embeddedBad/jobe.job 5(+5 -0)
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);
unit/java/azkaban/test/execapp/FlowRunnerTest2.java 896(+896 -0)
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) {