You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@oozie.apache.org by an...@apache.org on 2018/06/19 09:46:51 UTC

[01/16] oozie git commit: OOZIE-2339 [fluent-job] Minimum Viable Fluent Job API (daniel.becker, andras.piros via rkanter, gezapeti, pbacsko)

Repository: oozie
Updated Branches:
  refs/heads/master f8cbce617 -> 8a0a6487d


http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-client/src/test/java/org/apache/oozie/jobs/client/minitest/TestShellAction.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-client/src/test/java/org/apache/oozie/jobs/client/minitest/TestShellAction.java b/fluent-job/fluent-job-client/src/test/java/org/apache/oozie/jobs/client/minitest/TestShellAction.java
new file mode 100644
index 0000000..552933b
--- /dev/null
+++ b/fluent-job/fluent-job-client/src/test/java/org/apache/oozie/jobs/client/minitest/TestShellAction.java
@@ -0,0 +1,90 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.jobs.client.minitest;
+
+import org.apache.oozie.client.OozieClientException;
+import org.apache.oozie.fluentjob.api.GraphVisualization;
+import org.apache.oozie.fluentjob.api.action.Prepare;
+import org.apache.oozie.fluentjob.api.action.PrepareBuilder;
+import org.apache.oozie.fluentjob.api.action.ShellAction;
+import org.apache.oozie.fluentjob.api.action.ShellActionBuilder;
+import org.apache.oozie.fluentjob.api.dag.Graph;
+import org.apache.oozie.fluentjob.api.serialization.WorkflowMarshaller;
+import org.apache.oozie.fluentjob.api.workflow.Workflow;
+import org.apache.oozie.fluentjob.api.workflow.WorkflowBuilder;
+import org.apache.oozie.test.WorkflowTestCase;
+
+import javax.xml.bind.JAXBException;
+import java.io.IOException;
+
+public class TestShellAction extends WorkflowTestCase {
+    public void testForkedShellActions() throws IOException, JAXBException, OozieClientException {
+        final Prepare prepare = new PrepareBuilder()
+                .withDelete("hdfs://localhost:8020/user/${wf:user()}/examples/output")
+                .build();
+
+        final ShellAction parent = ShellActionBuilder.create()
+                .withResourceManager(getJobTrackerUri())
+                .withNameNode(getNameNodeUri())
+                .withPrepare(prepare)
+                .withConfigProperty("mapred.job.queue.name", "default")
+                .withArgument("arg1")
+                .withExecutable("python")
+                .withEnvironmentVariable("PATH=$PATH:/opt/python27/bin")
+                .withCaptureOutput(true)
+                .build();
+
+        //  We are reusing the definition of parent and only modifying and adding what is different.
+        final ShellAction leftChild = ShellActionBuilder.createFromExistingAction(parent)
+                .withParent(parent)
+                .withoutArgument("arg1")
+                .withArgument("arg2")
+                .withExecutable("python3")
+                .withoutEnvironmentVariable("PATH=$PATH:/opt/python27/bin")
+                .withEnvironmentVariable("PATH=$PATH:/opt/python36/bin")
+                .withCaptureOutput(false)
+                .build();
+
+        ShellActionBuilder.createFromExistingAction(leftChild)
+                .withoutArgument("arg2")
+                .withArgument("arg3")
+                .withExecutable("python4")
+                .withoutEnvironmentVariable("PATH=$PATH:/opt/python36/bin")
+                .withEnvironmentVariable("PATH=$PATH:/opt/python42/bin")
+                .build();
+
+        final Workflow workflow = new WorkflowBuilder()
+                .withName("simple-shell-example")
+                .withDagContainingNode(parent).build();
+
+        final String xml = WorkflowMarshaller.marshal(workflow);
+
+        System.out.println(xml);
+
+        GraphVisualization.workflowToPng(workflow, "simple-shell-example-workflow.png");
+
+        final Graph intermediateGraph = new Graph(workflow);
+
+        GraphVisualization.graphToPng(intermediateGraph, "simple-shell-example-graph.png");
+
+        log.debug("Workflow XML is:\n{0}", xml);
+
+        validate(xml);
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-client/src/test/java/org/apache/oozie/jobs/client/minitest/TestSparkAction.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-client/src/test/java/org/apache/oozie/jobs/client/minitest/TestSparkAction.java b/fluent-job/fluent-job-client/src/test/java/org/apache/oozie/jobs/client/minitest/TestSparkAction.java
new file mode 100644
index 0000000..3a1ce3b
--- /dev/null
+++ b/fluent-job/fluent-job-client/src/test/java/org/apache/oozie/jobs/client/minitest/TestSparkAction.java
@@ -0,0 +1,87 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.jobs.client.minitest;
+
+import org.apache.oozie.client.OozieClientException;
+import org.apache.oozie.fluentjob.api.GraphVisualization;
+import org.apache.oozie.fluentjob.api.action.Prepare;
+import org.apache.oozie.fluentjob.api.action.PrepareBuilder;
+import org.apache.oozie.fluentjob.api.action.SparkAction;
+import org.apache.oozie.fluentjob.api.action.SparkActionBuilder;
+import org.apache.oozie.fluentjob.api.dag.Graph;
+import org.apache.oozie.fluentjob.api.serialization.WorkflowMarshaller;
+import org.apache.oozie.fluentjob.api.workflow.Workflow;
+import org.apache.oozie.fluentjob.api.workflow.WorkflowBuilder;
+import org.apache.oozie.test.WorkflowTestCase;
+
+import javax.xml.bind.JAXBException;
+import java.io.IOException;
+
+public class TestSparkAction extends WorkflowTestCase {
+    public void testForkedSparkActions() throws IOException, JAXBException, OozieClientException {
+        final Prepare prepare = new PrepareBuilder()
+                .withDelete("hdfs://localhost:8020/user/${wf:user()}/examples/output")
+                .build();
+
+        final SparkAction parent = SparkActionBuilder.create()
+                .withResourceManager(getJobTrackerUri())
+                .withNameNode(getNameNodeUri())
+                .withPrepare(prepare)
+                .withConfigProperty("mapred.job.queue.name", "default")
+                .withArg("inputpath=hdfs://localhost/input/file.txt")
+                .withArg("value=1")
+                .withMaster("yarn")
+                .withMode("cluster")
+                .withActionName("Spark Example")
+                .withActionClass("org.apache.spark.examples.mllib.JavaALS")
+                .withJar("/lib/spark-examples_2.10-1.1.0.jar")
+                .withSparkOpts("--executor-memory 20G --num-executors 50")
+                .build();
+
+        //  We are reusing the definition of parent and only modifying and adding what is different.
+        final SparkAction leftChild = SparkActionBuilder.createFromExistingAction(parent)
+                .withParent(parent)
+                .withoutArg("value=1")
+                .withArg("value=3")
+                .build();
+
+        SparkActionBuilder.createFromExistingAction(leftChild)
+                .withoutArg("value=2")
+                .withArg("value=3")
+                .build();
+
+        final Workflow workflow = new WorkflowBuilder()
+                .withName("simple-spark-example")
+                .withDagContainingNode(parent).build();
+
+        final String xml = WorkflowMarshaller.marshal(workflow);
+
+        System.out.println(xml);
+
+        GraphVisualization.workflowToPng(workflow, "simple-spark-example-workflow.png");
+
+        final Graph intermediateGraph = new Graph(workflow);
+
+        GraphVisualization.graphToPng(intermediateGraph, "simple-spark-example-graph.png");
+
+        log.debug("Workflow XML is:\n{0}", xml);
+
+        validate(xml);
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-client/src/test/java/org/apache/oozie/jobs/client/minitest/TestSqoopAction.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-client/src/test/java/org/apache/oozie/jobs/client/minitest/TestSqoopAction.java b/fluent-job/fluent-job-client/src/test/java/org/apache/oozie/jobs/client/minitest/TestSqoopAction.java
new file mode 100644
index 0000000..3d6fec9
--- /dev/null
+++ b/fluent-job/fluent-job-client/src/test/java/org/apache/oozie/jobs/client/minitest/TestSqoopAction.java
@@ -0,0 +1,80 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.jobs.client.minitest;
+
+import org.apache.oozie.client.OozieClientException;
+import org.apache.oozie.fluentjob.api.GraphVisualization;
+import org.apache.oozie.fluentjob.api.action.Prepare;
+import org.apache.oozie.fluentjob.api.action.PrepareBuilder;
+import org.apache.oozie.fluentjob.api.action.SqoopAction;
+import org.apache.oozie.fluentjob.api.action.SqoopActionBuilder;
+import org.apache.oozie.fluentjob.api.dag.Graph;
+import org.apache.oozie.fluentjob.api.serialization.WorkflowMarshaller;
+import org.apache.oozie.fluentjob.api.workflow.Workflow;
+import org.apache.oozie.fluentjob.api.workflow.WorkflowBuilder;
+import org.apache.oozie.test.WorkflowTestCase;
+
+import javax.xml.bind.JAXBException;
+import java.io.IOException;
+
+public class TestSqoopAction extends WorkflowTestCase {
+    public void testForkedSqoopActions() throws IOException, JAXBException, OozieClientException {
+        final Prepare prepare = new PrepareBuilder()
+                .withDelete("hdfs://localhost:8020/user/${wf:user()}/examples/output")
+                .build();
+
+        final SqoopAction parent = SqoopActionBuilder.create()
+                .withResourceManager(getJobTrackerUri())
+                .withNameNode(getNameNodeUri())
+                .withPrepare(prepare)
+                .withConfigProperty("mapred.job.queue.name", "default")
+                .withCommand("python")
+                .build();
+
+        //  We are reusing the definition of parent and only modifying and adding what is different.
+        final SqoopAction leftChild = SqoopActionBuilder.createFromExistingAction(parent)
+                .withParent(parent)
+                .withCommand("python3")
+                .build();
+
+        SqoopActionBuilder.createFromExistingAction(leftChild)
+                .withoutArgument("arg2")
+                .withArgument("arg3")
+                .withCommand(null)
+                .build();
+
+        final Workflow workflow = new WorkflowBuilder()
+                .withName("simple-sqoop-example")
+                .withDagContainingNode(parent).build();
+
+        final String xml = WorkflowMarshaller.marshal(workflow);
+
+        System.out.println(xml);
+
+        GraphVisualization.workflowToPng(workflow, "simple-sqoop-example-workflow.png");
+
+        final Graph intermediateGraph = new Graph(workflow);
+
+        GraphVisualization.graphToPng(intermediateGraph, "simple-sqoop-example-graph.png");
+
+        log.debug("Workflow XML is:\n{0}", xml);
+
+        validate(xml);
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-client/src/test/java/org/apache/oozie/jobs/client/minitest/TestSshAction.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-client/src/test/java/org/apache/oozie/jobs/client/minitest/TestSshAction.java b/fluent-job/fluent-job-client/src/test/java/org/apache/oozie/jobs/client/minitest/TestSshAction.java
new file mode 100644
index 0000000..20ffdbf
--- /dev/null
+++ b/fluent-job/fluent-job-client/src/test/java/org/apache/oozie/jobs/client/minitest/TestSshAction.java
@@ -0,0 +1,74 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.jobs.client.minitest;
+
+import org.apache.oozie.client.OozieClientException;
+import org.apache.oozie.fluentjob.api.GraphVisualization;
+import org.apache.oozie.fluentjob.api.action.SshAction;
+import org.apache.oozie.fluentjob.api.action.SshActionBuilder;
+import org.apache.oozie.fluentjob.api.dag.Graph;
+import org.apache.oozie.fluentjob.api.serialization.WorkflowMarshaller;
+import org.apache.oozie.fluentjob.api.workflow.Workflow;
+import org.apache.oozie.fluentjob.api.workflow.WorkflowBuilder;
+import org.apache.oozie.test.WorkflowTestCase;
+
+import javax.xml.bind.JAXBException;
+import java.io.IOException;
+
+public class TestSshAction extends WorkflowTestCase {
+    public void testForkedSshActions() throws IOException, JAXBException, OozieClientException {
+        final SshAction parent = SshActionBuilder.create()
+                .withArg("\"Hello Oozie!\"")
+                .withHost("localhost")
+                .withCommand("echo")
+                .withCaptureOutput(true)
+                .build();
+
+        //  We are reusing the definition of parent and only modifying and adding what is different.
+        final SshAction leftChild = SshActionBuilder.createFromExistingAction(parent)
+                .withParent(parent)
+                .withoutArg("\"Hello Oozie!\"")
+                .withArg("\"Hello Oozie!!\"")
+                .withCaptureOutput(false)
+                .build();
+
+        SshActionBuilder.createFromExistingAction(leftChild)
+                .withoutArg("\"Hello Oozie!!\"")
+                .withArg("\"Hello Oozie!!!\"")
+                .build();
+
+        final Workflow workflow = new WorkflowBuilder()
+                .withName("simple-ssh-example")
+                .withDagContainingNode(parent).build();
+
+        final String xml = WorkflowMarshaller.marshal(workflow);
+
+        System.out.println(xml);
+
+        GraphVisualization.workflowToPng(workflow, "simple-ssh-example-workflow.png");
+
+        final Graph intermediateGraph = new Graph(workflow);
+
+        GraphVisualization.graphToPng(intermediateGraph, "simple-ssh-example-graph.png");
+
+        log.debug("Workflow XML is:\n{0}", xml);
+
+        validate(xml);
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-client/src/test/resources/workflow-all-actions.xml
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-client/src/test/resources/workflow-all-actions.xml b/fluent-job/fluent-job-client/src/test/resources/workflow-all-actions.xml
new file mode 100644
index 0000000..df66ac8
--- /dev/null
+++ b/fluent-job/fluent-job-client/src/test/resources/workflow-all-actions.xml
@@ -0,0 +1,234 @@
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you under the Apache License, Version 2.0 (the
+  "License"); you may not use this file except in compliance
+  with the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+
+<workflow-app xmlns="uri:oozie:workflow:1.0"
+              xmlns:sla="uri:oozie:sla:0.2"
+              name="jaxb-example-wf">
+
+    <start to="mr-node"/>
+
+    <action name="mr-node">
+        <map-reduce>
+            <job-tracker>${jobTracker}</job-tracker>
+            <name-node>${nameNode}</name-node>
+            <prepare>
+                <delete path="${nameNode}/user/${wf:user()}/${examplesRoot}/output-data/${outputDir}"/>
+            </prepare>
+            <configuration>
+                <property>
+                    <name>mapred.job.queue.name</name>
+                    <value>${queueName}</value>
+                </property>
+                <property>
+                    <name>mapred.mapper.class</name>
+                    <value>org.apache.oozie.example.SampleMapper</value>
+                </property>
+                <property>
+                    <name>mapred.reducer.class</name>
+                    <value>org.apache.oozie.example.SampleReducer</value>
+                </property>
+                <property>
+                    <name>mapred.map.tasks</name>
+                    <value>1</value>
+                </property>
+                <property>
+                    <name>mapred.input.dir</name>
+                    <value>/user/${wf:user()}/${examplesRoot}/input-data/text</value>
+                </property>
+                <property>
+                    <name>mapred.output.dir</name>
+                    <value>/user/${wf:user()}/${examplesRoot}/output-data/${outputDir}</value>
+                </property>
+            </configuration>
+        </map-reduce>
+        <ok to="distcp"/>
+        <error to="fail"/>
+        <!--<sla:info>-->
+            <!--<sla:nominal-time>${nominal_time}</sla:nominal-time>-->
+            <!--<sla:should-start>${5 * MINUTES}</sla:should-start>-->
+            <!--<sla:should-end>${15 * MINUTES}</sla:should-end>-->
+            <!--<sla:max-duration>${15 * MINUTES}</sla:max-duration>-->
+            <!--<sla:alert-events>start_miss,end_met,end_miss</sla:alert-events>-->
+            <!--<sla:alert-contact>joe@example.com</sla:alert-contact>-->
+        <!--</sla:info>-->
+    </action>
+
+    <action name="distcp">
+        <distcp xmlns="uri:oozie:distcp-action:1.0">
+            <job-tracker>${jobTracker}</job-tracker>
+            <name-node>${nameNode}</name-node>
+            <prepare>
+                <delete path="${nameNode}/user/${wf:user()}/${examplesRoot}/output-data/${outputDir}"/>
+                <mkdir path="${nameNode}/user/${wf:user()}/${examplesRoot}/output-data/${outputDir}"/>
+            </prepare>
+            <configuration>
+                <property>
+                    <name>a</name>
+                    <value>A</value>
+                </property>
+                <property>
+                    <name>b</name>
+                    <value>B</value>
+                </property>
+            </configuration>
+            <arg>${nameNode}/user/${wf:user()}/${examplesRoot}/input-data/${inputDir}/data.txt</arg>
+            <arg>${nameNode}/user/${wf:user()}/${examplesRoot}/output-data/${outputDir}/data.txt</arg>
+        </distcp>
+        <ok to="email"/>
+        <error to="fail"/>
+    </action>
+
+    <action name="email">
+        <email xmlns="uri:oozie:email-action:0.2">
+            <to>foo@bar.com</to>
+            <subject>foo</subject>
+            <body>bar</body>
+        </email>
+        <ok to="end"/>
+        <error to="fail"/>
+    </action>
+
+    <action name="hive2">
+        <hive2 xmlns="uri:oozie:hive2-action:1.0">
+            <job-tracker>foo:8021</job-tracker>
+            <name-node>bar:8020</name-node>
+            <prepare>
+                <delete path="${jobOutput}"/>
+            </prepare>
+            <configuration>
+                <property>
+                    <name>mapred.compress.map.output</name>
+                    <value>true</value>
+                </property>
+            </configuration>
+            <jdbc-url>jdbc:hive2://localhost:10000/default</jdbc-url>
+            <password>foo</password>
+            <script>myscript.q</script>
+            <param>InputDir=/home/rkanter/input-data</param>
+            <param>OutputDir=${jobOutput}</param>
+        </hive2>
+        <ok to="hive"/>
+        <error to="fail"/>
+    </action>
+
+    <action name="hive">
+        <hive xmlns="uri:oozie:hive-action:1.0">
+            <job-tracker>foo:8021</job-tracker>
+            <name-node>bar:8020</name-node>
+            <prepare>
+                <delete path="${jobOutput}"/>
+            </prepare>
+            <configuration>
+                <property>
+                    <name>mapred.compress.map.output</name>
+                    <value>true</value>
+                </property>
+            </configuration>
+            <script>myscript.q</script>
+            <param>InputDir=/home/tucu/input-data</param>
+            <param>OutputDir=${jobOutput}</param>
+        </hive>
+        <ok to="shell"/>
+        <error to="fail"/>
+    </action>
+
+    <action name="shell">
+        <shell xmlns="uri:oozie:shell-action:1.0">
+            <exec>echo</exec>
+            <argument>foo</argument>
+            <capture-output/>
+        </shell>
+        <ok to="spark"/>
+        <error to="fail"/>
+    </action>
+
+    <action name="spark">
+        <spark xmlns="uri:oozie:spark-action:1.0">
+            <job-tracker>foo:8021</job-tracker>
+            <name-node>bar:8020</name-node>
+            <prepare>
+                <delete path="${jobOutput}"/>
+            </prepare>
+            <configuration>
+                <property>
+                    <name>mapred.compress.map.output</name>
+                    <value>true</value>
+                </property>
+            </configuration>
+            <master>local[*]</master>
+            <mode>client</mode>
+            <name>Spark Example</name>
+            <class>org.apache.spark.examples.mllib.JavaALS</class>
+            <jar>/lib/spark-examples_2.10-1.1.0.jar</jar>
+            <spark-opts>--executor-memory 20G --num-executors 50
+                --conf spark.executor.extraJavaOptions="-XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=/tmp"</spark-opts>
+            <arg>inputpath=hdfs://localhost/input/file.txt</arg>
+            <arg>value=2</arg>
+        </spark>
+        <ok to="sqoop"/>
+        <error to="fail"/>
+    </action>
+
+    <action name="sqoop">
+        <sqoop xmlns="uri:oozie:sqoop-action:1.0">
+            <job-tracker>foo:8021</job-tracker>
+            <name-node>bar:8020</name-node>
+            <prepare>
+                <delete path="${jobOutput}"/>
+            </prepare>
+            <configuration>
+                <property>
+                    <name>mapred.compress.map.output</name>
+                    <value>true</value>
+                </property>
+            </configuration>
+            <command>
+                import --connect jdbc:hsqldb:file:db.hsqldb --table TT --target-dir hdfs://localhost:8020/user/tucu/foo -m 1
+            </command>
+        </sqoop>
+        <ok to="ssh"/>
+        <error to="fail"/>
+    </action>
+
+    <action name="ssh">
+        <ssh xmlns="uri:oozie:ssh-action:0.2">
+            <host>foo@bar.com</host>
+            <command>uploaddata</command>
+            <args>jdbc:derby://bar.com:1527/myDB</args>
+            <args>hdfs://foobar.com:8020/usr/tucu/myData</args>
+        </ssh>
+        <ok to="end"/>
+        <error to="fail"/>
+    </action>
+
+    <kill name = "fail">
+        <message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
+    </kill>
+
+    <end name="end"/>
+
+    <sla:info>
+        <sla:nominal-time>${nominal_time}</sla:nominal-time>
+        <sla:should-start>${10 * MINUTES}</sla:should-start>
+        <sla:should-end>${30 * MINUTES}</sla:should-end>
+        <sla:max-duration>${30 * MINUTES}</sla:max-duration>
+        <sla:alert-events>start_miss,end_met,end_miss</sla:alert-events>
+        <sla:alert-contact>joe@example.com</sla:alert-contact>
+    </sla:info>
+
+</workflow-app>

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-client/src/test/resources/workflow-mapreduce-action.xml
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-client/src/test/resources/workflow-mapreduce-action.xml b/fluent-job/fluent-job-client/src/test/resources/workflow-mapreduce-action.xml
new file mode 100644
index 0000000..21c03b1
--- /dev/null
+++ b/fluent-job/fluent-job-client/src/test/resources/workflow-mapreduce-action.xml
@@ -0,0 +1,63 @@
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you under the Apache License, Version 2.0 (the
+  "License"); you may not use this file except in compliance
+  with the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+
+<workflow-app xmlns="uri:oozie:workflow:1.0" name="jaxb-example-wf">
+    <start to="mr-node"/>
+    <action name="mr-node">
+        <map-reduce>
+            <resource-manager>${resourceManager}</resource-manager>
+            <name-node>${nameNode}</name-node>
+            <prepare>
+                <delete path="${nameNode}/user/${wf:user()}/${examplesRoot}/output-data/${outputDir}"/>
+            </prepare>
+            <configuration>
+                <property>
+                    <name>mapred.job.queue.name</name>
+                    <value>${queueName}</value>
+                </property>
+                <property>
+                    <name>mapred.mapper.class</name>
+                    <value>org.apache.oozie.example.SampleMapper</value>
+                </property>
+                <property>
+                    <name>mapred.reducer.class</name>
+                    <value>org.apache.oozie.example.SampleReducer</value>
+                </property>
+                <property>
+                    <name>mapred.map.tasks</name>
+                    <value>1</value>
+                </property>
+                <property>
+                    <name>mapred.input.dir</name>
+                    <value>/user/${wf:user()}/${examplesRoot}/input-data/text</value>
+                </property>
+                <property>
+                    <name>mapred.output.dir</name>
+                    <value>/user/${wf:user()}/${examplesRoot}/output-data/${outputDir}</value>
+                </property>
+            </configuration>
+        </map-reduce>
+        <ok to="end"/>
+        <error to="fail"/>
+    </action>
+
+    <kill name = "fail">
+        <message>Map/Reduce failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
+    </kill>
+    <end name="end"/>
+</workflow-app>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/pom.xml
----------------------------------------------------------------------
diff --git a/fluent-job/pom.xml b/fluent-job/pom.xml
new file mode 100644
index 0000000..5b24c91
--- /dev/null
+++ b/fluent-job/pom.xml
@@ -0,0 +1,62 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you under the Apache License, Version 2.0 (the
+  "License"); you may not use this file except in compliance
+  with the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <parent>
+        <artifactId>oozie-main</artifactId>
+        <groupId>org.apache.oozie</groupId>
+        <version>5.1.0-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+
+    <artifactId>oozie-fluent-job</artifactId>
+    <description>Apache Oozie Fluent Job</description>
+    <name>Apache Oozie Fluent Job</name>
+    <packaging>pom</packaging>
+
+    <modules>
+        <module>fluent-job-api</module>
+        <module>fluent-job-client</module>
+    </modules>
+
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-deploy-plugin</artifactId>
+                <configuration>
+                    <skip>true</skip>
+                </configuration>
+            </plugin>
+            <plugin>
+                <groupId>org.apache.rat</groupId>
+                <artifactId>apache-rat-plugin</artifactId>
+                <configuration>
+                    <excludeSubProjects>false</excludeSubProjects>
+                    <excludes>
+                        <!-- excluding all as the root POM does the full check -->
+                        <exclude>**</exclude>
+                    </excludes>
+                </configuration>
+            </plugin>
+        </plugins>
+    </build>
+
+</project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/minitest/pom.xml
----------------------------------------------------------------------
diff --git a/minitest/pom.xml b/minitest/pom.xml
index 1847838..56afddb 100644
--- a/minitest/pom.xml
+++ b/minitest/pom.xml
@@ -17,7 +17,7 @@
   limitations under the License.
 -->
 <project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-    xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
     <modelVersion>4.0.0</modelVersion>
 
     <parent>
@@ -80,4 +80,20 @@
             <artifactId>hadoop-minicluster</artifactId>
         </dependency>
     </dependencies>
+
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-jar-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <goals>
+                            <goal>test-jar</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+        </plugins>
+    </build>
 </project>

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/minitest/src/test/java/org/apache/oozie/test/TestWorkflow.java
----------------------------------------------------------------------
diff --git a/minitest/src/test/java/org/apache/oozie/test/TestWorkflow.java b/minitest/src/test/java/org/apache/oozie/test/TestWorkflow.java
index 4257b60..3c20bf2 100644
--- a/minitest/src/test/java/org/apache/oozie/test/TestWorkflow.java
+++ b/minitest/src/test/java/org/apache/oozie/test/TestWorkflow.java
@@ -20,20 +20,8 @@ package org.apache.oozie.test;
 
 import com.google.common.base.Strings;
 import org.apache.oozie.action.hadoop.JavaActionExecutor;
-import org.apache.oozie.client.OozieClientException;
-import org.apache.oozie.service.XLogService;
-import org.apache.oozie.client.OozieClient;
 import org.apache.oozie.client.WorkflowJob;
-import org.apache.oozie.local.LocalOozie;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
 
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.InputStreamReader;
-import java.io.Reader;
-import java.io.Writer;
-import java.io.OutputStreamWriter;
 import java.util.Date;
 import java.util.Properties;
 
@@ -42,58 +30,13 @@ import static org.junit.Assume.assumeFalse;
 /**
  * {@code MiniOozie} integration test for different workflow kinds.
  */
-public class TestWorkflow extends MiniOozieTestCase {
-
-    @Override
-    protected void setUp() throws Exception {
-        System.setProperty(XLogService.LOG4J_FILE, "oozie-log4j.properties");
-        super.setUp();
-    }
-
-    @Override
-    protected void tearDown() throws Exception {
-        super.tearDown();
-    }
+public class TestWorkflow extends WorkflowTestCase {
 
     public void testWorkflowWithStartAndEndCompletesSuccessfully() throws Exception {
-        final String wfApp = "<workflow-app xmlns='uri:oozie:workflow:0.1' name='test-wf'>" + "    <start to='end'/>"
+        final String workflowXml = "<workflow-app xmlns='uri:oozie:workflow:0.1' name='test-wf'>" + "    <start to='end'/>"
                 + "    <end name='end'/>" + "</workflow-app>";
 
-        final FileSystem fs = getFileSystem();
-        final Path appPath = new Path(getFsTestCaseDir(), "app");
-        fs.mkdirs(appPath);
-        fs.mkdirs(new Path(appPath, "lib"));
-
-        final Writer writer = new OutputStreamWriter(fs.create(new Path(appPath, "workflow.xml")));
-        writer.write(wfApp);
-        writer.close();
-
-        final OozieClient wc = LocalOozie.getClient();
-
-        final Properties conf = wc.createConfiguration();
-        conf.setProperty(OozieClient.APP_PATH, new Path(appPath, "workflow.xml").toString());
-        conf.setProperty(OozieClient.USER_NAME, getTestUser());
-
-
-        final String jobId = wc.submit(conf);
-        assertNotNull(jobId);
-
-        WorkflowJob wf = wc.getJobInfo(jobId);
-        assertNotNull(wf);
-        assertEquals(WorkflowJob.Status.PREP, wf.getStatus());
-
-        wc.start(jobId);
-
-        waitFor(1000, new Predicate() {
-            public boolean evaluate() throws Exception {
-                final WorkflowJob wf = wc.getJobInfo(jobId);
-                return wf.getStatus() == WorkflowJob.Status.SUCCEEDED;
-            }
-        });
-
-        wf = wc.getJobInfo(jobId);
-        assertNotNull(wf);
-        assertEquals(WorkflowJob.Status.SUCCEEDED, wf.getStatus());
+        submitAndAssert(workflowXml, WorkflowJob.Status.SUCCEEDED);
     }
 
     public void testFsDecisionWorkflowCompletesSuccessfully() throws Exception {
@@ -116,100 +59,4 @@ public class TestWorkflow extends MiniOozieTestCase {
 
         runWorkflowFromFile(workflowFileName, additionalWorkflowProperties);
     }
-
-    private void runWorkflowFromFile(final String workflowFileName, final Properties additionalWorkflowProperties)
-            throws IOException, OozieClientException {
-        final FileSystem fs = getFileSystem();
-        final Path appPath = new Path(getFsTestCaseDir(), "app");
-        fs.mkdirs(appPath);
-        fs.mkdirs(new Path(appPath, "lib"));
-
-        final Reader reader = getResourceAsReader(workflowFileName, -1);
-        final Writer writer = new OutputStreamWriter(fs.create(new Path(appPath, "workflow.xml")));
-        copyCharStream(reader, writer);
-        writer.close();
-        reader.close();
-
-        final Path path = getFsTestCaseDir();
-
-        final OozieClient oozieClient = LocalOozie.getClient();
-
-        final Properties conf = oozieClient.createConfiguration();
-        conf.setProperty(OozieClient.APP_PATH, new Path(appPath, "workflow.xml").toString());
-        conf.setProperty(OozieClient.USER_NAME, getTestUser());
-        conf.setProperty("nameNodeBasePath", path.toString());
-        conf.setProperty("base", path.toUri().getPath());
-        conf.setProperty("nameNode", getNameNodeUri());
-        conf.setProperty("jobTracker", getJobTrackerUri());
-
-        for (final String additionalKey : additionalWorkflowProperties.stringPropertyNames()) {
-            conf.setProperty(additionalKey, additionalWorkflowProperties.getProperty(additionalKey));
-        }
-
-        final String jobId = oozieClient.submit(conf);
-        assertNotNull(jobId);
-
-        WorkflowJob wf = oozieClient.getJobInfo(jobId);
-        assertNotNull(wf);
-        assertEquals(WorkflowJob.Status.PREP, wf.getStatus());
-
-        oozieClient.start(jobId);
-
-        waitFor(15 * 1000, new Predicate() {
-            public boolean evaluate() throws Exception {
-                final WorkflowJob wf = oozieClient.getJobInfo(jobId);
-                return wf.getStatus() == WorkflowJob.Status.SUCCEEDED;
-            }
-        });
-
-        wf = oozieClient.getJobInfo(jobId);
-        assertNotNull(wf);
-        assertEquals(WorkflowJob.Status.SUCCEEDED, wf.getStatus());
-    }
-
-    /**
-     * Return a classpath resource as a stream.
-     * <p/>
-     *
-     * @param path classpath for the resource.
-     * @param maxLen max content length allowed.
-     * @return the stream for the resource.
-     * @throws IOException thrown if the resource could not be read.
-     */
-    private InputStream getResourceAsStream(final String path, final int maxLen) throws IOException {
-        final InputStream is = Thread.currentThread().getContextClassLoader().getResourceAsStream(path);
-        if (is == null) {
-            throw new IllegalArgumentException("resource " + path + " not found");
-        }
-        return is;
-    }
-
-    /**
-     * Return a classpath resource as a reader.
-     * <p/>
-     * It is assumed that the resource is a text resource.
-     *
-     * @param path classpath for the resource.
-     * @param maxLen max content length allowed.
-     * @return the reader for the resource.
-     * @throws IOException thrown if the resource could not be read.
-     */
-    private Reader getResourceAsReader(final String path, final int maxLen) throws IOException {
-        return new InputStreamReader(getResourceAsStream(path, maxLen));
-    }
-
-    /**
-     * Copies an char input stream into an char output stream.
-     *
-     * @param reader reader to copy from.
-     * @param writer writer to copy to.
-     * @throws IOException thrown if the copy failed.
-     */
-    private void copyCharStream(final Reader reader, final Writer writer) throws IOException {
-        final char[] buffer = new char[4096];
-        int read;
-        while ((read = reader.read(buffer)) > -1) {
-            writer.write(buffer, 0, read);
-        }
-    }
 }

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/minitest/src/test/java/org/apache/oozie/test/WorkflowTestCase.java
----------------------------------------------------------------------
diff --git a/minitest/src/test/java/org/apache/oozie/test/WorkflowTestCase.java b/minitest/src/test/java/org/apache/oozie/test/WorkflowTestCase.java
new file mode 100644
index 0000000..3a47972
--- /dev/null
+++ b/minitest/src/test/java/org/apache/oozie/test/WorkflowTestCase.java
@@ -0,0 +1,259 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.test;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.oozie.client.OozieClient;
+import org.apache.oozie.client.OozieClientException;
+import org.apache.oozie.client.WorkflowJob;
+import org.apache.oozie.local.LocalOozie;
+import org.apache.oozie.service.XLogService;
+import org.apache.oozie.servlet.V2ValidateServlet;
+
+import java.io.*;
+import java.util.Properties;
+
+public abstract class WorkflowTestCase extends MiniOozieTestCase {
+
+    @Override
+    protected void setUp() throws Exception {
+        System.setProperty(XLogService.LOG4J_FILE, "oozie-log4j.properties");
+        super.setUp();
+    }
+
+    @Override
+    protected void tearDown() throws Exception {
+        super.tearDown();
+    }
+
+    protected void submitAndAssert(final String workflowXml, final WorkflowJob.Status terminalStatus)
+            throws OozieClientException, IOException {
+        final WorkflowJob finishedWorkflowJob = new WorkflowJobBuilder()
+                .submit(workflowXml)
+                .start()
+                .waitForSucceeded()
+                .build();
+
+        assertNotNull(finishedWorkflowJob);
+        assertEquals(terminalStatus, finishedWorkflowJob.getStatus());
+    }
+
+    protected void validate(final String workflowXml) throws IOException, OozieClientException {
+        new WorkflowJobBuilder()
+                .validate(workflowXml);
+    }
+
+    protected void runWorkflowFromFile(final String workflowFileName, final Properties additionalWorkflowProperties)
+            throws IOException, OozieClientException {
+        final FileSystem fs = getFileSystem();
+        final Path appPath = new Path(getFsTestCaseDir(), "app");
+        fs.mkdirs(appPath);
+        fs.mkdirs(new Path(appPath, "lib"));
+
+        final Reader reader = getResourceAsReader(workflowFileName, -1);
+        final Writer writer = new OutputStreamWriter(fs.create(new Path(appPath, "workflow.xml")));
+        copyCharStream(reader, writer);
+        writer.close();
+        reader.close();
+
+        final Path path = getFsTestCaseDir();
+
+        final OozieClient oozieClient = LocalOozie.getClient();
+
+        final Properties conf = oozieClient.createConfiguration();
+        conf.setProperty(OozieClient.APP_PATH, new Path(appPath, "workflow.xml").toString());
+        conf.setProperty(OozieClient.USER_NAME, getTestUser());
+        conf.setProperty("nameNodeBasePath", path.toString());
+        conf.setProperty("base", path.toUri().getPath());
+        conf.setProperty("nameNode", getNameNodeUri());
+        conf.setProperty("jobTracker", getJobTrackerUri());
+
+        for (final String additionalKey : additionalWorkflowProperties.stringPropertyNames()) {
+            conf.setProperty(additionalKey, additionalWorkflowProperties.getProperty(additionalKey));
+        }
+
+        final String jobId = oozieClient.submit(conf);
+        assertNotNull(jobId);
+
+        WorkflowJob wf = oozieClient.getJobInfo(jobId);
+        assertNotNull(wf);
+        assertEquals(WorkflowJob.Status.PREP, wf.getStatus());
+
+        oozieClient.start(jobId);
+
+        waitFor(15_000, new Predicate() {
+            public boolean evaluate() throws Exception {
+                final WorkflowJob wf = oozieClient.getJobInfo(jobId);
+                return wf.getStatus() == WorkflowJob.Status.SUCCEEDED;
+            }
+        });
+
+        wf = oozieClient.getJobInfo(jobId);
+        assertNotNull(wf);
+        assertEquals(WorkflowJob.Status.SUCCEEDED, wf.getStatus());
+    }
+
+    /**
+     * Return a classpath resource as a stream.
+     * <p/>
+     *
+     * @param path classpath for the resource.
+     * @param maxLen max content length allowed.
+     * @return the stream for the resource.
+     * @throws IOException thrown if the resource could not be read.
+     */
+    private InputStream getResourceAsStream(final String path, final int maxLen) throws IOException {
+        final InputStream is = Thread.currentThread().getContextClassLoader().getResourceAsStream(path);
+        if (is == null) {
+            throw new IllegalArgumentException("resource " + path + " not found");
+        }
+        return is;
+    }
+
+    /**
+     * Return a classpath resource as a reader.
+     * <p/>
+     * It is assumed that the resource is a text resource.
+     *
+     * @param path classpath for the resource.
+     * @param maxLen max content length allowed.
+     * @return the reader for the resource.
+     * @throws IOException thrown if the resource could not be read.
+     */
+    private Reader getResourceAsReader(final String path, final int maxLen) throws IOException {
+        return new InputStreamReader(getResourceAsStream(path, maxLen));
+    }
+
+    /**
+     * Copies an char input stream into an char output stream.
+     *
+     * @param reader reader to copy from.
+     * @param writer writer to copy to.
+     * @throws IOException thrown if the copy failed.
+     */
+    private void copyCharStream(final Reader reader, final Writer writer) throws IOException {
+        final char[] buffer = new char[4096];
+        int read;
+        while ((read = reader.read(buffer)) > -1) {
+            writer.write(buffer, 0, read);
+        }
+    }
+
+    private class WorkflowJobBuilder {
+        private final FileSystem dfs;
+        private final Path appPath;
+        private final OozieClient oozieClient = LocalOozie.getClient();
+        private String workflowJobId;
+        private WorkflowJob workflowJob;
+        private final Path localPath;
+
+        private WorkflowJobBuilder() throws IOException {
+            this.dfs = getFileSystem();
+            this.appPath = new Path(getFsTestCaseDir(), "app");
+            this.localPath = new Path(File.createTempFile(appPath.getName(), "workflow.xml").toString());
+
+            dfs.mkdirs(appPath);
+            dfs.mkdirs(new Path(appPath, "lib"));
+        }
+
+        private WorkflowJobBuilder submit(final String workflowXml) throws IOException, OozieClientException {
+            writeToDFS(workflowXml);
+
+            final Properties conf = createAndResolveConfiguration();
+
+            workflowJobId = oozieClient.submit(conf);
+
+            assertNotNull(workflowJobId);
+
+            return this;
+        }
+
+        private WorkflowJobBuilder validate(final String workflowXml) throws IOException, OozieClientException {
+            final String result = oozieClient.validateXML(workflowXml);
+
+            assertEquals("not a valid workflow xml", V2ValidateServlet.VALID_WORKFLOW_APP, result);
+
+            return this;
+        }
+
+        private void writeToDFS(final String workflowXml) throws IOException {
+            try (final Writer writer = new OutputStreamWriter(dfs.create(getDFSWorkflowPath()))) {
+                writer.write(workflowXml);
+                writer.flush();
+            }
+        }
+
+        private Properties createAndResolveConfiguration() {
+            final OozieClient wc = LocalOozie.getClient();
+
+            final Properties conf = wc.createConfiguration();
+            conf.setProperty(OozieClient.APP_PATH, getDFSWorkflowPath().toString());
+            conf.setProperty(OozieClient.USER_NAME, getTestUser());
+            conf.setProperty("nameNodeBasePath", getFsTestCaseDir().toString());
+            conf.setProperty("base", getFsTestCaseDir().toUri().getPath());
+            conf.setProperty("nameNode", getNameNodeUri());
+            conf.setProperty("jobTracker", getJobTrackerUri());
+            return conf;
+        }
+
+        private void writeToLocalFile(final String workflowXml) throws IOException {
+            try (final Writer writer = new FileWriter(localPath.toString())) {
+                writer.write(workflowXml);
+                writer.flush();
+            }
+        }
+
+        private Path getDFSWorkflowPath() {
+            return new Path(appPath, "workflow.xml");
+        }
+
+        private WorkflowJobBuilder start() throws OozieClientException {
+            workflowJob = oozieClient.getJobInfo(workflowJobId);
+
+            assertNotNull(workflowJob);
+            assertEquals(WorkflowJob.Status.PREP, workflowJob.getStatus());
+
+            oozieClient.start(workflowJobId);
+
+            workflowJob = oozieClient.getJobInfo(workflowJobId);
+
+            assertEquals(WorkflowJob.Status.RUNNING, workflowJob.getStatus());
+
+            return this;
+        }
+
+        private WorkflowJobBuilder waitForSucceeded() throws OozieClientException {
+            waitFor(15_000, new Predicate() {
+                public boolean evaluate() throws Exception {
+                    final WorkflowJob wf = oozieClient.getJobInfo(workflowJobId);
+                    return wf.getStatus() == WorkflowJob.Status.SUCCEEDED;
+                }
+            });
+
+            workflowJob = oozieClient.getJobInfo(workflowJobId);
+
+            return this;
+        }
+
+        private WorkflowJob build() {
+            return workflowJob;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 8c34cf4..7f03e6d 100644
--- a/pom.xml
+++ b/pom.xml
@@ -128,6 +128,7 @@
         <module>docs</module>
         <module>tools</module>
         <module>minitest</module>
+        <module>fluent-job</module>
         <module>server</module>
         <module>distro</module>
         <module>zookeeper-security-tests</module>
@@ -190,6 +191,12 @@
             </dependency>
             <dependency>
                 <groupId>org.apache.oozie</groupId>
+                <artifactId>oozie-core</artifactId>
+                <type>test-jar</type>
+                <version>${project.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.oozie</groupId>
                 <artifactId>oozie-examples</artifactId>
                 <version>${project.version}</version>
             </dependency>
@@ -255,6 +262,28 @@
                 <version>${project.version}</version>
                 <type>war</type>
             </dependency>
+            <dependency>
+                <groupId>org.apache.oozie</groupId>
+                <artifactId>oozie-fluent-job-api</artifactId>
+                <version>${project.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.oozie</groupId>
+                <artifactId>oozie-fluent-job-api</artifactId>
+                <version>${project.version}</version>
+                <classifier>tests</classifier>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.oozie.test</groupId>
+                <artifactId>oozie-mini</artifactId>
+                <version>${project.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.oozie.test</groupId>
+                <artifactId>oozie-mini</artifactId>
+                <version>${project.version}</version>
+                <classifier>tests</classifier>
+            </dependency>
 
             <dependency>
                 <groupId>org.apache.hadoop</groupId>
@@ -840,11 +869,19 @@
                 <artifactId>hadoop-yarn-server-web-proxy</artifactId>
                 <version>${hadoop.version}</version>
             </dependency>
+
             <dependency>
                 <groupId>org.apache.hadoop</groupId>
                 <artifactId>hadoop-mapreduce-client-core</artifactId>
                 <version>${hadoop.version}</version>
             </dependency>
+
+            <dependency>
+                <groupId>org.apache.hadoop</groupId>
+                <artifactId>hadoop-annotations</artifactId>
+                <version>${hadoop.version}</version>
+            </dependency>
+
             <dependency>
                 <groupId>org.apache.pig</groupId>
                 <artifactId>pig</artifactId>
@@ -1513,13 +1550,41 @@
                 <version>${dropwizard.metrics.version}</version>
             </dependency>
 
-            <!-- For drawing runtime DAG -->
+            <!-- Draw runtime DAG -->
             <dependency>
                 <groupId>guru.nidi</groupId>
                 <artifactId>graphviz-java</artifactId>
                 <version>0.2.2</version>
             </dependency>
 
+            <!-- Fluent Job API conversion to XML -->
+            <dependency>
+                <groupId>net.sf.dozer</groupId>
+                <artifactId>dozer</artifactId>
+                <version>5.5.1</version>
+            </dependency>
+            <dependency>
+                <groupId>org.jvnet.jaxb2_commons</groupId>
+                <artifactId>jaxb2-basics</artifactId>
+                <version>1.11.1</version>
+            </dependency>
+            <dependency>
+                <groupId>org.jvnet.jaxb2_commons</groupId>
+                <artifactId>jaxb2-namespace-prefix</artifactId>
+                <version>1.3</version>
+            </dependency>
+            <dependency>
+                <groupId>org.codehaus.mojo</groupId>
+                <artifactId>jaxb2-maven-plugin</artifactId>
+                <version>2.2</version>
+            </dependency>
+
+            <dependency>
+                <groupId>org.xmlunit</groupId>
+                <artifactId>xmlunit-core</artifactId>
+                <version>2.3.0</version>
+            </dependency>
+
             <dependency>
                 <groupId>org.eclipse.jgit</groupId>
                 <artifactId>org.eclipse.jgit</artifactId>
@@ -1710,6 +1775,11 @@
                     <artifactId>maven-clean-plugin</artifactId>
                     <version>3.0.0</version>
                 </plugin>
+                <plugin>
+                    <groupId>org.codehaus.mojo</groupId>
+                    <artifactId>jaxb2-maven-plugin</artifactId>
+                    <version>2.3.1</version>
+                </plugin>
             </plugins>
         </pluginManagement>
 

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/release-log.txt
----------------------------------------------------------------------
diff --git a/release-log.txt b/release-log.txt
index c53d4d1..cfb558c 100644
--- a/release-log.txt
+++ b/release-log.txt
@@ -1,5 +1,6 @@
 -- Oozie 5.1.0 release (trunk - unreleased)
 
+OOZIE-2339 [fluent-job] Minimum Viable Fluent Job API (daniel.becker, andras.piros via rkanter, gezapeti, pbacsko)
 OOZIE-3224 Upgrade Jetty to 9.3 (kmarton via andras.piros)
 OOZIE-3284 Upgrade maven-javadoc-plugin to 3.0.1 (kmarton via pbacsko, andras.piros)
 OOZIE-3278 Oozie fails to start with Hadoop 2.6.0 (kmarton via andras.piros)


[11/16] oozie git commit: OOZIE-2339 [fluent-job] Minimum Viable Fluent Job API (daniel.becker, andras.piros via rkanter, gezapeti, pbacsko)

Posted by an...@apache.org.
http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/Streaming.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/Streaming.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/Streaming.java
new file mode 100644
index 0000000..cfd8314
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/Streaming.java
@@ -0,0 +1,98 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+import java.util.List;
+
+/**
+ * A class representing the streaming information within a {@link MapReduceAction}.
+ *
+ * Instances of this class should be built using the builder {@link StreamingBuilder}.
+ *
+ * The properties of the builder can only be set once, an attempt to set them a second time will trigger
+ * an {@link IllegalStateException}.
+ *
+ * Builder instances can be used to build several elements, although properties already set cannot be changed after
+ * a call to {@link StreamingBuilder#build} either.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class Streaming {
+    private final String mapper;
+    private final String reducer;
+    private final String recordReader;
+    private final ImmutableList<String> recordReaderMappings;
+    private final ImmutableList<String> envs;
+
+    Streaming(final String mapper,
+              final String reducer,
+              final String recordReader,
+              final ImmutableList<String> recordReaderMappings,
+              final ImmutableList<String> envs) {
+        this.mapper = mapper;
+        this.reducer = reducer;
+        this.recordReader = recordReader;
+        this.recordReaderMappings = recordReaderMappings;
+        this.envs = envs;
+    }
+
+    /**
+     * Returns the mapper of this {@link Streaming} object.
+     * @return The mapper of this {@link Streaming} object.
+     */
+    public String getMapper() {
+        return mapper;
+    }
+
+    /**
+     * Returns the reducer of this {@link Streaming} object.
+     * @return The reducer of this {@link Streaming} object.
+     */
+    public String getReducer() {
+        return reducer;
+    }
+
+    /**
+     * Returns the record reader of this {@link Streaming} object.
+     * @return The record reader of this {@link Streaming} object.
+     */
+    public String getRecordReader() {
+        return recordReader;
+    }
+
+    /**
+     * Returns the record reader mappings of this {@link Streaming} object as a list.
+     * @return The record reader mappings of this {@link Streaming} object as a list.
+     */
+    public List<String> getRecordReaderMappings() {
+        return recordReaderMappings;
+    }
+
+    /**
+     * Returns the environment variables of this {@link Streaming} object as a list.
+     * @return The environment variables of this {@link Streaming} object as a list.
+     */
+    public List<String> getEnvs() {
+        return envs;
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/StreamingBuilder.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/StreamingBuilder.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/StreamingBuilder.java
new file mode 100644
index 0000000..ab52969
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/StreamingBuilder.java
@@ -0,0 +1,117 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.oozie.fluentjob.api.ModifyOnce;
+
+/**
+ * A builder class for {@link Streaming}.
+ *
+ * The properties of the builder can only be set once, an attempt to set them a second time will trigger
+ * an {@link IllegalStateException}. The properties that are lists are an exception to this rule, of course multiple
+ * elements can be added / removed.
+ *
+ * Builder instances can be used to build several elements, although properties already set cannot be changed after
+ * a call to {@link StreamingBuilder#build} either.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class StreamingBuilder implements Builder<Streaming> {
+    private final ModifyOnce<String> mapper;
+    private final ModifyOnce<String> reducer;
+    private final ModifyOnce<String> recordReader;
+    private final ImmutableList.Builder<String> recordReaderMappings;
+    private final ImmutableList.Builder<String> envs;
+
+    /**
+     * Creates a new {@link StreamingBuilder}.
+     */
+    public StreamingBuilder() {
+        mapper = new ModifyOnce<>();
+        reducer = new ModifyOnce<>();
+        recordReader = new ModifyOnce<>();
+
+        recordReaderMappings = new ImmutableList.Builder<>();
+        envs = new ImmutableList.Builder<>();
+    }
+
+    /**
+     * Registers a mapper with this builder.
+     * @param mapper The mapper to register with this builder.
+     * @return This builder.
+     */
+    public StreamingBuilder withMapper(final String mapper) {
+        this.mapper.set(mapper);
+        return this;
+    }
+
+    /**
+     * Registers a reducer with this builder.
+     * @param reducer The reducer to register with this builder.
+     * @return This builder.
+     */
+    public StreamingBuilder withReducer(final String reducer) {
+        this.reducer.set(reducer);
+        return this;
+    }
+
+    /**
+     * Registers a record reader with this builder.
+     * @param recordReader The record reader to register with this builder.
+     * @return This builder.
+     */
+    public StreamingBuilder withRecordReader(final String recordReader) {
+        this.recordReader.set(recordReader);
+        return this;
+    }
+
+    /**
+     * Registers a record reader mapping with this builder.
+     * @param recordReaderMapping The record reader mapping to register with this builder.
+     * @return This builder.
+     */
+    public StreamingBuilder withRecordReaderMapping(final String recordReaderMapping) {
+        this.recordReaderMappings.add(recordReaderMapping);
+        return this;
+    }
+
+    /**
+     * Registers an environment variable with this builder.
+     * @param env The environment variable to register with this builder.
+     * @return This builder.
+     */
+    public StreamingBuilder withEnv(final String env) {
+        this.envs.add(env);
+        return this;
+    }
+
+    /**
+     * Creates a new {@link Streaming} object with the properties stores in this builder.
+     * The new {@link Streaming} object is independent of this builder and the builder can be used to build
+     * new instances.
+     * @return A new {@link Streaming} object with the properties stored in this builder.
+     */
+    @Override
+    public Streaming build() {
+        return new Streaming(mapper.get(), reducer.get(), recordReader.get(), recordReaderMappings.build(), envs.build());
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/SubWorkflowAction.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/SubWorkflowAction.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/SubWorkflowAction.java
new file mode 100644
index 0000000..238bc13
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/SubWorkflowAction.java
@@ -0,0 +1,88 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import com.google.common.collect.ImmutableMap;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+import java.util.Map;
+
+/**
+ * A class representing the Oozie subworkflow action.
+ * Instances of this class should be built using the builder {@link SubWorkflowActionBuilder}.
+ *
+ * The properties of the builder can only be set once, an attempt to set them a second time will trigger
+ * an {@link IllegalStateException}.
+ *
+ * Builder instances can be used to build several elements, although properties already set cannot be changed after
+ * a call to {@link SubWorkflowActionBuilder#build} either.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class SubWorkflowAction extends Node {
+    private final String appPath;
+    private final boolean propagateConfiguration;
+    private final ImmutableMap<String, String> configuration;
+
+    SubWorkflowAction(final Node.ConstructionData constructionData,
+                      final String appPath,
+                      final boolean propagateConfiguration,
+                      final ImmutableMap<String, String> configuration) {
+        super(constructionData);
+
+        this.appPath = appPath;
+        this.propagateConfiguration = propagateConfiguration;
+        this.configuration = configuration;
+    }
+
+    /**
+     * Returns the path to the application definition (usually workflow.xml) of the subworkflow.
+     * @return The path to the application definition (usually workflow.xml) of the subworkflow.
+     */
+    public String getAppPath() {
+        return appPath;
+    }
+
+    /**
+     * Returns whether the configuration of the main workflow should propagate to the subworkflow.
+     * @return {@code true} if the configuration of the main workflow should propagate to the subworkflow;
+     *         {@code false} otherwise.
+     */
+    public boolean isPropagatingConfiguration() {
+        return propagateConfiguration;
+    }
+
+    /**
+     * Returns the value associated with the provided configuration property name.
+     * @param property The name of the configuration property for which the value will be returned.
+     * @return The value associated with the provided configuration property name.
+     */
+    public String getConfigProperty(final String property) {
+        return configuration.get(property);
+    }
+
+    /**
+     * Returns an immutable map of the configuration key-value pairs stored in this {@link MapReduceAction} object.
+     * @return An immutable map of the configuration key-value pairs stored in this {@link MapReduceAction} object.
+     */
+    public Map<String, String> getConfiguration() {
+        return configuration;
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/SubWorkflowActionBuilder.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/SubWorkflowActionBuilder.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/SubWorkflowActionBuilder.java
new file mode 100644
index 0000000..07761d4
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/SubWorkflowActionBuilder.java
@@ -0,0 +1,159 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.oozie.fluentjob.api.ModifyOnce;
+
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+/**
+ * A builder class for {@link SubWorkflowAction}.
+ *
+ * The properties of the builder can only be set once, an attempt to set them a second time will trigger
+ * an {@link IllegalStateException}. The properties that are lists are an exception to this rule, of course multiple
+ * elements can be added / removed.
+ *
+ * Builder instances can be used to build several elements, although properties already set cannot be changed after
+ * a call to {@link SubWorkflowActionBuilder#build} either.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class SubWorkflowActionBuilder
+        extends NodeBuilderBaseImpl<SubWorkflowActionBuilder> implements Builder<SubWorkflowAction> {
+    private final ModifyOnce<String> appPath;
+    private final ModifyOnce<Boolean> propagateConfiguration;
+    private final Map<String, ModifyOnce<String>> configuration;
+
+    /**
+     * Creates and returns an empty builder.
+     * @return An empty builder.
+     */
+    public static SubWorkflowActionBuilder create() {
+        final ModifyOnce<String> appPath = new ModifyOnce<>();
+        final ModifyOnce<Boolean> propagateConfiguration = new ModifyOnce<>(false);
+        final Map<String, ModifyOnce<String>> configuration = new LinkedHashMap<>();
+
+        return new SubWorkflowActionBuilder(null, appPath, propagateConfiguration, configuration);
+    }
+
+    /**
+     * Create and return a new {@link SubWorkflowActionBuilder} that is based on an already built
+     * {@link SubWorkflowAction} object. The properties of the builder will initially be the same as those of the
+     * provided {@link SubWorkflowAction} object, but it is possible to modify them once.
+     * @param action The {@link SubWorkflowAction} object on which this {@link SubWorkflowActionBuilder} will be based.
+     * @return A new {@link SubWorkflowActionBuilder} that is based on a previously built {@link SubWorkflowAction} object.
+     */
+    public static SubWorkflowActionBuilder createFromExistingAction(final SubWorkflowAction action) {
+        final ModifyOnce<String> appPath = new ModifyOnce<>(action.getAppPath());
+        final ModifyOnce<Boolean> propagateConfiguration = new ModifyOnce<>(action.isPropagatingConfiguration());
+        final Map<String, ModifyOnce<String>> configuration =
+                ActionAttributesBuilder.convertToModifyOnceMap(action.getConfiguration());
+
+        return new SubWorkflowActionBuilder(action, appPath, propagateConfiguration, configuration);
+    }
+
+    SubWorkflowActionBuilder(final SubWorkflowAction action,
+                             final ModifyOnce<String> appPath,
+                             final ModifyOnce<Boolean> propagateConfiguration,
+                             final Map<String, ModifyOnce<String>> configuration) {
+        super(action);
+
+        this.appPath = appPath;
+        this.propagateConfiguration = propagateConfiguration;
+        this.configuration = configuration;
+    }
+
+    /**
+     * Registers the path to the application definition (usually workflow.xml) of the subworkflow.
+     * @param appPath HDFS application path
+     * @return This builder.
+     */
+    public SubWorkflowActionBuilder withAppPath(final String appPath) {
+        this.appPath.set(appPath);
+        return this;
+    }
+
+    /**
+     * Registers that the configuration of the main workflow should propagate to the subworkflow.
+     * @return This builder.
+     */
+    public SubWorkflowActionBuilder withPropagatingConfiguration() {
+        this.propagateConfiguration.set(true);
+        return this;
+    }
+
+    /**
+     * Registers that the configuration of the main workflow should NOT propagate to the subworkflow.
+     * @return This builder.
+     */
+    public SubWorkflowActionBuilder withoutPropagatingConfiguration() {
+        this.propagateConfiguration.set(false);
+        return this;
+    }
+
+    /**
+     * Registers a configuration property (a key-value pair) with this builder. If the provided key has already been
+     * set on this builder, an exception is thrown. Setting a key to null means deleting it.
+     * @param key The name of the property to set.
+     * @param value The value of the property to set.
+     * @return this
+     * @throws IllegalStateException if the provided key has already been set on this builder.
+     */
+    public SubWorkflowActionBuilder withConfigProperty(final String key, final String value) {
+        ModifyOnce<String> mappedValue = this.configuration.get(key);
+
+        if (mappedValue == null) {
+            mappedValue = new ModifyOnce<>(value);
+            this.configuration.put(key, mappedValue);
+        }
+
+        mappedValue.set(value);
+
+        return this;
+    }
+
+    /**
+     * Creates a new {@link SubWorkflowAction} object with the properties stores in this builder.
+     * The new {@link MapReduceAction} object is independent of this builder and the builder can be used to build
+     * new instances.
+     * @return A new {@link MapReduceAction} object with the properties stored in this builder.
+     */
+    @Override
+    public SubWorkflowAction build() {
+        final Node.ConstructionData constructionData = getConstructionData();
+
+        final SubWorkflowAction instance = new SubWorkflowAction(
+                constructionData,
+                appPath.get(),
+                propagateConfiguration.get(),
+                ActionAttributesBuilder.convertToConfigurationMap(configuration));
+
+        addAsChildToAllParents(instance);
+
+        return instance;
+    }
+
+    @Override
+    protected SubWorkflowActionBuilder getRuntimeSelfReference() {
+        return this;
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/Touchz.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/Touchz.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/Touchz.java
new file mode 100644
index 0000000..11c655a
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/Touchz.java
@@ -0,0 +1,47 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * A class representing the touchz command of {@link FSAction}.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class Touchz {
+    private final String path;
+
+    /**
+     * Creates a new {@link Touchz} object.
+     * @param path The path of the file that the touch operation will be executed on.
+     */
+    public Touchz(final String path) {
+        this.path = path;
+    }
+
+    /**
+     * Returns the path of the file that the touch operation will be executed on.
+     * @return The path of the file that the touch operation will be executed on.
+     */
+    public String getPath() {
+        return path;
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/dag/DagNodeWithCondition.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/dag/DagNodeWithCondition.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/dag/DagNodeWithCondition.java
new file mode 100644
index 0000000..a0ef83a
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/dag/DagNodeWithCondition.java
@@ -0,0 +1,108 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.dag;
+
+import org.apache.oozie.fluentjob.api.Condition;
+
+import java.util.Collection;
+
+/**
+ * This is a class bundling together a {@link NodeBase} object and a {@link Condition} object. There are no restrictions
+ * as to whether the node should be a parent with an outgoing conditional path or a child with an incoming conditional path.
+ */
+public class DagNodeWithCondition {
+    private final NodeBase node;
+    private final Condition condition;
+
+    /**
+     * Removes the first {@link DagNodeWithCondition} object from a collection that has the provided node as its node.
+     * If there is no such object in the collection, this method returns false;
+     * @param collection The collection from which to remove an element
+     * @param node The node to remove together with its condition.
+     * @return {@code true} if an element was removed; {@code false} if no matching element was contained in the collection.
+     */
+    public static boolean removeFromCollection(final Collection<DagNodeWithCondition> collection, final NodeBase node) {
+        DagNodeWithCondition element = null;
+        for (final DagNodeWithCondition nodeWithCondition : collection) {
+            if (node.equals(nodeWithCondition.getNode())) {
+                element = nodeWithCondition;
+            }
+        }
+
+        if (element != null) {
+            collection.remove(element);
+        }
+
+        return element != null;
+    }
+
+    /**
+     * Creates a new {@link DagNodeWithCondition} object.
+     * @param node A {@link NodeBase} object.
+     * @param condition A {@link Condition} object.
+     */
+    public DagNodeWithCondition(final NodeBase node,
+                                final Condition condition) {
+        this.node = node;
+        this.condition = condition;
+    }
+
+    /**
+     * Returns the {@link NodeBase} object of this {@link DagNodeWithCondition}.
+     * @return The {@link NodeBase} object of this {@link DagNodeWithCondition}.
+     */
+    public NodeBase getNode() {
+        return node;
+    }
+
+    /**
+     * Returns the {@link Condition} object of this {@link DagNodeWithCondition}.
+     * @return The {@link Condition} object of this {@link DagNodeWithCondition}.
+     */
+    public Condition getCondition() {
+        return condition;
+    }
+
+    @Override
+    public boolean equals(final Object o) {
+        if (this == o) {
+            return true;
+        }
+
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+
+        final DagNodeWithCondition that = (DagNodeWithCondition) o;
+
+        if (node != null ? !node.equals(that.node) : that.node != null) {
+            return false;
+        }
+
+        return condition != null ? condition.equals(that.condition) : that.condition == null;
+    }
+
+    @Override
+    public int hashCode() {
+        int result = node != null ? node.hashCode() : 0;
+        result = 31 * result + (condition != null ? condition.hashCode() : 0);
+
+        return result;
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/dag/Decision.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/dag/Decision.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/dag/Decision.java
new file mode 100644
index 0000000..1b1a742
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/dag/Decision.java
@@ -0,0 +1,190 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.dag;
+
+import com.google.common.base.Preconditions;
+import org.apache.oozie.fluentjob.api.Condition;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * A class representing decision nodes in an Oozie workflow definition DAG. These nodes are generated automatically,
+ * the end user should not need to use this class directly.
+ */
+public class Decision extends NodeBase {
+    private NodeBase parent;
+    private final List<DagNodeWithCondition> childrenWithConditions;
+    private NodeBase defaultChild;
+
+    /**
+     * Create a new decision node with the given name.
+     * @param name The name of the new decision node.
+     */
+    public Decision(final String name) {
+        super(name);
+        this.parent = null;
+        this.childrenWithConditions = new ArrayList<>();
+    }
+
+    /**
+     * Returns the parent of this node.
+     * @return The parent of this node.
+     */
+    public NodeBase getParent() {
+        return parent;
+    }
+
+    /**
+     * Adds the provided node as a parent of this node.
+     * @param parent The new parent of this node.
+     * @throws IllegalStateException if this node already has a parent.
+     */
+    @Override
+    public void addParent(final NodeBase parent) {
+        Preconditions.checkState(this.parent == null, "Decision nodes cannot have multiple parents.");
+
+        this.parent = parent;
+        this.parent.addChild(this);
+    }
+
+    /**
+     * Adds the provided node as a conditional parent of this node.
+     * @param parent The new conditional parent of this node.
+     * @param condition The condition which must be true in addition the parent completing successfully for this node
+     *                  to be executed.
+     * @throws IllegalStateException if this node already has a parent.
+     */
+    @Override
+    public void addParentWithCondition(final Decision parent, final Condition condition) {
+        Preconditions.checkState(this.parent == null, "Decision nodes cannot have multiple parents.");
+
+        this.parent = parent;
+        parent.addChildWithCondition(this, condition);
+    }
+
+    /**
+     * Adds the provided node as the default conditional parent of this node.
+     * @param parent The new conditional parent of this node.
+     * @throws IllegalStateException if this node already has a parent.
+     */
+    @Override
+    public void addParentDefaultConditional(final Decision parent) {
+        Preconditions.checkState(this.parent == null, "Decision nodes cannot have multiple parents.");
+
+        this.parent = parent;
+        parent.addDefaultChild(this);
+    }
+
+    @Override
+    public void removeParent(final NodeBase parent) {
+        Preconditions.checkArgument(this.parent == parent, "Trying to remove a nonexistent parent.");
+
+        if (this.parent != null) {
+            this.parent.removeChild(this);
+        }
+
+        this.parent = null;
+    }
+
+    @Override
+    public void clearParents() {
+        removeParent(parent);
+    }
+
+    @Override
+    public List<NodeBase> getChildren() {
+        final List<NodeBase> results = new ArrayList<>();
+
+        for (final DagNodeWithCondition nodeWithCondition : getChildrenWithConditions()) {
+            results.add(nodeWithCondition.getNode());
+        }
+
+        return Collections.unmodifiableList(results);
+    }
+
+    /**
+     * Returns the children of this {@link Decision} node together with their conditions (all children are conditional),
+     * including the default child.
+     * @return The conditional children of this {@link Decision} node together with their conditions,
+     *         including the default child.
+     */
+    public List<DagNodeWithCondition> getChildrenWithConditions() {
+        final List<DagNodeWithCondition> results = new ArrayList<>(childrenWithConditions);
+
+        if (defaultChild != null) {
+            results.add(new DagNodeWithCondition(defaultChild, Condition.defaultCondition()));
+        }
+
+        return Collections.unmodifiableList(results);
+    }
+
+    /**
+     * Returns the default child of this {@code Decision} node.
+     * @return The default child of this {@code Decision} node.
+     */
+    public NodeBase getDefaultChild() {
+        return defaultChild;
+    }
+
+    @Override
+    protected void addChild(final NodeBase child) {
+        throw new IllegalStateException("Decision nodes cannot have normal children.");
+    }
+
+    void addChildWithCondition(final NodeBase child, final Condition condition) {
+        if (condition.isDefault()) {
+            addDefaultChild(child);
+        }
+        else {
+            this.childrenWithConditions.add(new DagNodeWithCondition(child, condition));
+        }
+    }
+
+    void addDefaultChild(final NodeBase child) {
+        Preconditions.checkState(defaultChild == null, "Trying to add a default child to a Decision node that already has one.");
+
+        defaultChild = child;
+    }
+
+    @Override
+    protected void removeChild(final NodeBase child) {
+        if (defaultChild == child) {
+            defaultChild = null;
+        }
+        else {
+            final int index = indexOfNodeBaseInChildrenWithConditions(child);
+
+            Preconditions.checkArgument(index >= 0, "Trying to remove a nonexistent child.");
+
+            this.childrenWithConditions.remove(index);
+        }
+    }
+
+    private int indexOfNodeBaseInChildrenWithConditions(final NodeBase child) {
+        for (int i = 0; i < this.childrenWithConditions.size(); ++i) {
+            if (child == this.childrenWithConditions.get(i).getNode()) {
+                return i;
+            }
+        }
+
+        return -1;
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/dag/DecisionJoin.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/dag/DecisionJoin.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/dag/DecisionJoin.java
new file mode 100644
index 0000000..c96753e
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/dag/DecisionJoin.java
@@ -0,0 +1,51 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.dag;
+
+/**
+ * This class represents a joining point where two or more (but not necessarily all) conditional branches originating
+ * from the same decision node meet.
+ * This class will NOT be mapped to JAXB classes and XML as decision nodes don't need to be joined is Oozie, this class
+ * only exists to make implementing the algorithms easier.
+ */
+public class DecisionJoin extends JoiningNodeBase<Decision> {
+
+    /**
+     * Creates a new {@link DecisionJoin} object.
+     * @param name The name of the new decision object.
+     * @param decision The {@link Decision} node that this {@link DecisionJoin} node closes.
+     */
+    public DecisionJoin(final String name, final Decision decision) {
+        super(name, decision);
+    }
+
+    public NodeBase getFirstNonDecisionJoinDescendant() {
+        NodeBase descendant = getChild();
+
+        while (descendant != null) {
+            if (!(descendant instanceof DecisionJoin)) {
+                return descendant;
+            }
+
+            descendant = ((DecisionJoin) descendant).getChild();
+        }
+
+        return null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/dag/End.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/dag/End.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/dag/End.java
new file mode 100644
index 0000000..0a28eb3
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/dag/End.java
@@ -0,0 +1,121 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.dag;
+
+import com.google.common.base.Preconditions;
+import org.apache.oozie.fluentjob.api.Condition;
+
+import java.util.Arrays;
+import java.util.List;
+
+/**
+ * A class representing end nodes in an Oozie workflow definition DAG. These nodes are generated automatically,
+ * the end user should not need to use this class directly.
+ */
+public class End extends NodeBase {
+    private NodeBase parent;
+
+    /**
+     * Create a new end node with the given name.
+     * @param name The name of the new end node.
+     */
+    public End(final String name) {
+        super(name);
+    }
+
+    /**
+     * Returns the parent of this node.
+     * @return The parent of this node.
+     */
+    public NodeBase getParent() {
+        return parent;
+    }
+
+    /**
+     * Adds the provided node as a parent of this node.
+     * @param parent The new parent of this node.
+     * @throws IllegalStateException if this node already has a parent.
+     */
+    @Override
+    public void addParent(final NodeBase parent) {
+        Preconditions.checkState(this.parent == null, "End nodes cannot have multiple parents.");
+
+        this.parent = parent;
+        parent.addChild(this);
+    }
+
+    /**
+     * Adds the provided node as a conditional parent of this node.
+     * @param parent The new conditional parent of this node.
+     * @param condition The condition which must be true in addition the parent completing successfully for this node
+     *                  to be executed.
+     * @throws IllegalStateException if this node already has a parent.
+     */
+    @Override
+    public void addParentWithCondition(final Decision parent, final Condition condition) {
+        Preconditions.checkState(this.parent == null, "End nodes cannot have multiple parents.");
+
+        this.parent = parent;
+        parent.addChildWithCondition(this, condition);
+    }
+
+    /**
+     * Adds the provided node as the default conditional parent of this node.
+     * @param parent The new conditional parent of this node.
+     * @throws IllegalStateException if this node already has a parent.
+     */
+    @Override
+    public void addParentDefaultConditional(Decision parent) {
+        Preconditions.checkState(this.parent == null, "End nodes cannot have multiple parents.");
+
+        this.parent = parent;
+        parent.addDefaultChild(this);
+    }
+
+    @Override
+    public void removeParent(final NodeBase parent) {
+        Preconditions.checkArgument(this.parent == parent, "Trying to remove a nonexistent parent.");
+
+        if (this.parent != null) {
+            this.parent.removeChild(this);
+        }
+
+        this.parent = null;
+    }
+
+    @Override
+    public void clearParents() {
+        removeParent(parent);
+    }
+
+    @Override
+    public List<NodeBase> getChildren() {
+        return Arrays.asList();
+    }
+
+    @Override
+    protected void addChild(final NodeBase child) {
+        throw new IllegalStateException("End nodes cannot have children.");
+    }
+
+    @Override
+    protected void removeChild(final NodeBase child) {
+        throw new IllegalStateException("End nodes cannot have children.");
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/dag/ExplicitNode.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/dag/ExplicitNode.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/dag/ExplicitNode.java
new file mode 100644
index 0000000..ae16a53
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/dag/ExplicitNode.java
@@ -0,0 +1,150 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.dag;
+
+import com.google.common.base.Preconditions;
+import org.apache.oozie.fluentjob.api.action.Node;
+import org.apache.oozie.fluentjob.api.Condition;
+
+import java.util.Arrays;
+import java.util.List;
+
+/**
+ * A class representing action nodes in an Oozie workflow definition DAG. These are the nodes in the intermediate graph
+ * representation that correspond to the nodes that are explicitly defined by the user.
+ */
+public class ExplicitNode extends NodeBase {
+    private NodeBase parent;
+    private NodeBase child;
+    private final Node realNode;
+
+    /**
+     * Create a new explicit node with the given name.
+     * @param name The name of the new explicit node.
+     * @param realNode The API level {@link Node} object defined by the end user.
+     */
+    public ExplicitNode(final String name, final Node realNode) {
+        super(name);
+        this.realNode = realNode;
+    }
+
+    /**
+     * Returns the API level {@link Node} object defined by the end user.
+     * @return The API level {@link Node} object defined by the end user.
+     */
+    public Node getRealNode() {
+        return realNode;
+    }
+
+    /**
+     * Returns the parent of this node.
+     * @return The parent of this node.
+     */
+    public NodeBase getParent() {
+        return parent;
+    }
+
+    /**
+     * Returns the child of this node.
+     * @return The child of this node.
+     */
+    public NodeBase getChild() {
+        return child;
+    }
+
+    /**
+     * Adds the provided node as a parent of this node.
+     * @param parent The new parent of this node.
+     * @throws IllegalStateException if this node already has a parent.
+     */
+    @Override
+    public void addParent(final NodeBase parent) {
+        Preconditions.checkState(this.parent == null, "An explicit node cannot have multiple parents.");
+
+        this.parent = parent;
+        parent.addChild(this);
+    }
+
+    /**
+     * Adds the provided node as a conditional parent of this node.
+     * @param parent The new conditional parent of this node.
+     * @param condition The condition which must be true in addition the parent completing successfully for this node
+     *                  to be executed.
+     * @throws IllegalStateException if this node already has a parent.
+     */
+    @Override
+    public void addParentWithCondition(final Decision parent, final Condition condition) {
+        Preconditions.checkState(this.parent == null, "An explicit node cannot have multiple parents.");
+
+        this.parent = parent;
+        parent.addChildWithCondition(this, condition);
+    }
+
+    /**
+     * Adds the provided node as the default conditional parent of this node.
+     * @param parent The new conditional parent of this node.
+     * @throws IllegalStateException if this node already has a parent.
+     */
+    @Override
+    public void addParentDefaultConditional(Decision parent) {
+        Preconditions.checkState(this.parent == null, "An explicit node cannot have multiple parents.");
+
+        this.parent = parent;
+        parent.addDefaultChild(this);
+    }
+
+    @Override
+    public void removeParent(final NodeBase parent) {
+        Preconditions.checkArgument(this.parent == parent, "Trying to remove a nonexistent parent.");
+
+        if (this.parent != null) {
+            this.parent.removeChild(this);
+        }
+
+        this.parent = null;
+    }
+
+    @Override
+    public void clearParents() {
+        removeParent(parent);
+    }
+
+    @Override
+    public List<NodeBase> getChildren() {
+        if (child == null) {
+            return Arrays.asList();
+        }
+
+        return Arrays.asList(child);
+    }
+
+    @Override
+    protected void addChild(final NodeBase child) {
+        Preconditions.checkState(this.child == null, "Normal nodes cannot have multiple children.");
+
+        this.child = child;
+    }
+
+    @Override
+    protected void removeChild(final NodeBase child) {
+        Preconditions.checkArgument(this.child == child, "Trying to remove a nonexistent child.");
+
+        this.child = null;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/dag/Fork.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/dag/Fork.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/dag/Fork.java
new file mode 100644
index 0000000..2657e63
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/dag/Fork.java
@@ -0,0 +1,142 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.dag;
+
+import com.google.common.base.Preconditions;
+import org.apache.oozie.fluentjob.api.Condition;
+import org.apache.oozie.fluentjob.api.ModifyOnce;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * A class representing end nodes in an Oozie workflow definition DAG. These nodes are generated automatically,
+ * the end user should not need to use this class directly.
+ */
+public class Fork extends NodeBase {
+    private NodeBase parent;
+    private final List<NodeBase> children;
+
+    private final ModifyOnce<Join> closingJoin;
+
+    /**
+     * Create a new fork node with the given name.
+     * @param name The name of the new fork node.
+     */
+    public Fork(final String name) {
+        super(name);
+
+        this.parent = null;
+        this.children = new ArrayList<>();
+        this.closingJoin = new ModifyOnce<>();
+    }
+
+    /**
+     * Returns the parent of this node.
+     * @return The parent of this node.
+     */
+    public NodeBase getParent() {
+        return parent;
+    }
+
+    /**
+     * Adds the provided node as a parent of this node.
+     * @param parent The new parent of this node.
+     * @throws IllegalStateException if this node already has a parent.
+     */
+    @Override
+    public void addParent(final NodeBase parent) {
+        Preconditions.checkState(this.parent == null, "Fork nodes cannot have multiple parents.");
+
+        this.parent = parent;
+        parent.addChild(this);
+    }
+
+    /**
+     * Adds the provided node as a conditional parent of this node.
+     * @param parent The new conditional parent of this node.
+     * @param condition The condition which must be true in addition the parent completing successfully for this node
+     *                  to be executed.
+     * @throws IllegalStateException if this node already has a parent.
+     */
+    @Override
+    public void addParentWithCondition(final Decision parent, final Condition condition) {
+        Preconditions.checkState(this.parent == null, "Fork nodes cannot have multiple parents.");
+
+        this.parent = parent;
+        parent.addChildWithCondition(this, condition);
+    }
+
+    /**
+     * Adds the provided node as the default conditional parent of this node.
+     * @param parent The new conditional parent of this node.
+     * @throws IllegalStateException if this node already has a parent.
+     */
+    @Override
+    public void addParentDefaultConditional(Decision parent) {
+        Preconditions.checkState(this.parent == null, "Fork nodes cannot have multiple parents.");
+
+        this.parent = parent;
+        parent.addDefaultChild(this);
+    }
+
+    @Override
+    public void removeParent(final NodeBase parent) {
+        Preconditions.checkArgument(this.parent == parent, "Trying to remove a nonexistent parent.");
+
+        if (this.parent != null) {
+            this.parent.removeChild(this);
+        }
+
+        this.parent = null;
+    }
+
+    @Override
+    public void clearParents() {
+        removeParent(parent);
+    }
+
+    @Override
+    public List<NodeBase> getChildren() {
+        return Collections.unmodifiableList(new ArrayList<>(children));
+    }
+
+    Join getClosingJoin() {
+        return closingJoin.get();
+    }
+
+    boolean isClosed() {
+        return getClosingJoin() != null;
+    }
+
+    void close(final Join join) {
+        closingJoin.set(join);
+    }
+
+    @Override
+    protected void addChild(final NodeBase child) {
+        children.add(child);
+    }
+
+    @Override
+    protected void removeChild(final NodeBase child) {
+        Preconditions.checkArgument(this.children.remove(child),"Trying to remove a nonexistent child.");
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/dag/Graph.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/dag/Graph.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/dag/Graph.java
new file mode 100644
index 0000000..08cb896
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/dag/Graph.java
@@ -0,0 +1,872 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.dag;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import org.apache.oozie.fluentjob.api.action.Node;
+import org.apache.oozie.fluentjob.api.workflow.Credentials;
+import org.apache.oozie.fluentjob.api.workflow.Global;
+import org.apache.oozie.fluentjob.api.workflow.Parameters;
+import org.apache.oozie.fluentjob.api.workflow.Workflow;
+import org.apache.oozie.fluentjob.api.Condition;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * The class holding the intermediate representation of the workflow. This is where the API level {@link Workflow}
+ * object is transformed to an intermediate graph (an object of this class), and all control nodes are generated.
+ * This graph is later further transformed to JAXB objects and to xml.
+ *
+ * The conversion from the API level {@link Workflow} object to the intermediate graph is as follows:
+ * We take the nodes in topological order, meaning every node is processed after all of its dependencies
+ * have been processed. There are two main possibilities when processing a node:
+ *      - the node has zero or one parent
+ *      - the node has at least two parents.
+ *
+ * In the first case, we simply add the converted node as a child to its parent (or the start node if there are none),
+ * possibly inserting a fork if the parent already has children, or using a pre-existing fork if a parent already
+ * has one.
+ *
+ * In the second case, we have to insert a join. We first check if we can join all incoming paths in a single join
+ * node or if we have to split them up because they come from multiple embedded forks. It is also possible that some
+ * incoming paths originate from the same fork but that fork has other outgoing paths as well. In that case we split
+ * the fork up into multiple embedded forks.
+ *
+ * After this, we examine all paths that we are going to join and look for side branches that lead out of the
+ * fork / join block, violating Oozie's constraints. If these are non-conditional branches, we simply cut them down
+ * from their original parents and put them under the new join (and possibly under a fork), and make them siblings
+ * of whatever nodes originally come after the join. This way all original dependencies are preserved, as the original
+ * parents will still be ancestors (though indirectly) to the relocated nodes, but new dependencies are introduced.
+ * This preserves the correctness of the workflow but decreases its parallelism. This is unfortunate but Oozie's graph
+ * format is more restrictive than a general DAG, so we have to accept it.
+ *
+ * If the side branches are conditional, we cut above the decision node and insert a join there. We reinsert the
+ * decision node under the new join. This is very similar to the handling of non-conditional paths, but it
+ * decreases parallelism even more (we cut one level higher).
+ * A problem occurs if two or more decision nodes come right after the fork that we want to close. If we cut above
+ * the decision nodes as usual we gain nothing, because we insert a join and a fork and arrive at the same situation
+ * as before - multiple decision nodes under a fork. Currently, we are not able to handle this situation and we throw
+ * an exception.
+ */
+public class Graph {
+    private final String name;
+    private final Start start = new Start("start");
+    private final End end = new End("end");
+    private final Parameters parameters;
+    private final Global global;
+    private final Credentials credentials;
+    private final Map<String, NodeBase> nodesByName = new LinkedHashMap<>();
+    private final Map<Fork, Integer> forkNumbers = new HashMap<>();
+    private int forkCounter = 1;
+
+    private final Map<NodeBase, Decision> originalParentToCorrespondingDecision = new HashMap<>();
+    private final Map<Decision, Integer> closedPathsOfDecisionNodes = new HashMap<>();
+    private int decisionCounter = 1;
+    private int decisionJoinCounter = 1;
+
+    /**
+     * Nodes that have a join downstream to them are closed, they should never get new children.
+     */
+    private final Map<NodeBase, Join> closingJoins = new HashMap<>();
+
+    /**
+     * Creates a new {@link Graph} object transforming the graph of the provided {@link Workflow} object
+     * into an intermediate level graph.
+     * @param workflow The {@link Workflow} object to transform.
+     */
+    public Graph(final Workflow workflow) {
+        this.name = workflow.getName();
+        this.parameters = workflow.getParameters();
+        this.global = workflow.getGlobal();
+        this.credentials = workflow.getCredentials();
+
+        final List<Node> nodesFromRootsToLeaves = getNodesFromRootsToLeaves(workflow);
+
+        storeNode(start);
+        storeNode(end);
+
+        convert(nodesFromRootsToLeaves);
+    }
+
+    /**
+     * Returns the name of this graph.
+     * @return The name of this graph.
+     */
+    public String getName() {
+        return name;
+    }
+
+    public Parameters getParameters() {
+        return parameters;
+    }
+
+    public Global getGlobal() {
+        return global;
+    }
+
+    /**
+     * Returns the start node of this graph.
+     * @return The start node of this graph.
+     */
+    public Start getStart() {
+        return start;
+    }
+
+    /**
+     * Returns the end node of this graph.
+     * @return The end node of this graph.
+     */
+    public End getEnd() {
+        return end;
+    }
+
+    /**
+     * Returns the node with the given name in this graph if it exists, {@code null} otherwise.
+     * @param name The name of the node that will be returned.
+     * @return The node with the given name in this graph if it exists, {@code null} otherwise.
+     */
+    public NodeBase getNodeByName(final String name) {
+        return nodesByName.get(name);
+    }
+
+    /**
+     * Returns a collection of the nodes in this graph.
+     * @return A collection of the nodes in this graph.
+     */
+    public Collection<NodeBase> getNodes() {
+        return nodesByName.values();
+    }
+
+    private void convert(final List<Node> nodesInTopologicalOrder) {
+        final Map<Node, NodeBase> nodeToNodeBase = new HashMap<>();
+
+        for (final Node originalNode : nodesInTopologicalOrder) {
+            final ExplicitNode convertedNode = new ExplicitNode(originalNode.getName(), originalNode);
+            nodeToNodeBase.put(originalNode, convertedNode);
+            storeNode(convertedNode);
+
+            checkAndInsertDecisionNode(originalNode, convertedNode);
+
+            final List<DagNodeWithCondition> mappedParentsWithConditions = findMappedParents(originalNode, nodeToNodeBase);
+
+            handleNodeWithParents(convertedNode, mappedParentsWithConditions);
+        }
+
+        final List<DagNodeWithCondition> finalNodes = findFinalNodes();
+
+        handleNodeWithParents(end, finalNodes);
+    }
+
+    private void checkAndInsertDecisionNode(Node originalNode, ExplicitNode convertedNode) {
+        if (!originalNode.getChildrenWithConditions().isEmpty()) {
+            // We insert a decision node below the current convertedNode.
+            final Decision decision = newDecision();
+            decision.addParent(convertedNode);
+            originalParentToCorrespondingDecision.put(convertedNode, decision);
+        }
+    }
+
+    private List<DagNodeWithCondition> findMappedParents(Node originalNode, Map<Node, NodeBase> nodeToNodeBase) {
+        final List<DagNodeWithCondition> mappedParentsWithConditions = new ArrayList<>();
+
+        for (final Node.NodeWithCondition parentNodeWithCondition : originalNode.getParentsWithConditions()) {
+            final NodeBase mappedParentNode = nodeToNodeBase.get(parentNodeWithCondition.getNode());
+            final Condition condition = parentNodeWithCondition.getCondition();
+            final DagNodeWithCondition parentNodeBaseWithCondition = new DagNodeWithCondition(mappedParentNode, condition);
+            mappedParentsWithConditions.add(parentNodeBaseWithCondition);
+        }
+
+        for (final Node parent : originalNode.getParentsWithoutConditions()) {
+            mappedParentsWithConditions.add(new DagNodeWithCondition(nodeToNodeBase.get(parent), null));
+        }
+
+        return mappedParentsWithConditions;
+    }
+
+    private List<DagNodeWithCondition> findFinalNodes() {
+        final List<DagNodeWithCondition> finalNodes = new ArrayList<>();
+
+        for (final NodeBase maybeFinalNode : nodesByName.values()) {
+            final boolean hasNoChildren = maybeFinalNode.getChildren().isEmpty();
+            final boolean isNotEnd = maybeFinalNode != end;
+            if (hasNoChildren && isNotEnd) {
+                finalNodes.add(new DagNodeWithCondition(maybeFinalNode, null));
+            }
+        }
+
+        return finalNodes;
+    }
+
+    private void storeNode(final NodeBase node) {
+        final String name = node.getName();
+
+        final boolean isPresent = nodesByName.containsKey(name);
+        if (isPresent) {
+            final String errorMessage = String.format("Duplicate name '%s' found in graph '%s'", node.getName(), this.getName());
+            throw new IllegalArgumentException(errorMessage);
+        }
+
+        nodesByName.put(node.getName(), node);
+    }
+
+    private NodeBase getNewParent(final NodeBase originalParent) {
+        NodeBase newParent = originalParent;
+
+        if (originalParentToCorrespondingDecision.containsKey(newParent)) {
+            newParent = originalParentToCorrespondingDecision.get(newParent);
+        }
+
+        newParent = getNearestNonClosedDescendant(newParent);
+
+        return newParent;
+    }
+
+    private void handleNodeWithParents(final NodeBase node, final List<DagNodeWithCondition> parentsWithConditions) {
+        // Avoiding adding children to nodes that are inside a closed fork / join pair and to original parents of decision nodes.
+        final List<DagNodeWithCondition> newParentsWithConditions = new ArrayList<>();
+        for (final DagNodeWithCondition parentWithCondition : parentsWithConditions) {
+            final NodeBase parent = parentWithCondition.getNode();
+            final Condition condition = parentWithCondition.getCondition();
+
+            final NodeBase newParent = getNewParent(parent);
+            final DagNodeWithCondition newParentWithCondition = new DagNodeWithCondition(newParent, condition);
+
+            if (!newParentsWithConditions.contains(newParentWithCondition)) {
+                newParentsWithConditions.add(newParentWithCondition);
+            }
+        }
+
+        if (newParentsWithConditions.isEmpty()) {
+            handleSingleParentNode(new DagNodeWithCondition(start, null), node);
+        }
+        else if (newParentsWithConditions.size() == 1) {
+            handleSingleParentNode(newParentsWithConditions.get(0), node);
+        }
+        else {
+            handleMultiParentNodeWithParents(node, newParentsWithConditions);
+        }
+    }
+
+    private void handleSingleParentNode(final DagNodeWithCondition parentWithCondition, final NodeBase node) {
+        addParentWithForkIfNeeded(node, parentWithCondition);
+    }
+
+    private void handleMultiParentNodeWithParents(final NodeBase node, final List<DagNodeWithCondition> parentsWithConditions) {
+        final List<PathInformation> paths = new ArrayList<>();
+        for (final DagNodeWithCondition parentWithCondition : parentsWithConditions) {
+            final NodeBase parent = parentWithCondition.getNode();
+            paths.add(getPathInfo(parent));
+        }
+
+        final BranchingToClose toClose = chooseBranchingToClose(paths);
+
+        // Eliminating redundant parents.
+        if (toClose.isRedundantParent()) {
+            final List<DagNodeWithCondition> parentsWithoutRedundant = new ArrayList<>(parentsWithConditions);
+            DagNodeWithCondition.removeFromCollection(parentsWithoutRedundant, toClose.getRedundantParent());
+
+            handleNodeWithParents(node, parentsWithoutRedundant);
+        }
+        else if (toClose.isDecision()) {
+            insertDecisionJoin(node, parentsWithConditions, toClose);
+        }
+        else {
+            insertJoin(parentsWithConditions, node, toClose);
+        }
+    }
+
+    private void insertDecisionJoin(final NodeBase node,
+                                    final List<DagNodeWithCondition> parentsWithConditions,
+                                    final BranchingToClose branchingToClose) {
+        final Decision decision = branchingToClose.getDecision();
+        final DecisionJoin decisionJoin = newDecisionJoin(decision, branchingToClose.getPaths().size());
+
+        for (final DagNodeWithCondition parentWithCondition : parentsWithConditions) {
+            addParentWithForkIfNeeded(decisionJoin, parentWithCondition);
+        }
+
+        addParentWithForkIfNeeded(node, new DagNodeWithCondition(decisionJoin, null));
+    }
+
+    private void insertJoin(final List<DagNodeWithCondition> parentsWithConditions,
+                            final NodeBase node,
+                            final BranchingToClose branchingToClose) {
+        if (branchingToClose.isSplittingJoinNeeded()) {
+            // We have to close a subset of the paths.
+            final List<DagNodeWithCondition> newParentsWithConditions = new ArrayList<>(parentsWithConditions);
+//            final List<NodeBase> parentsInToClose = new ArrayList<>();
+
+            for (final PathInformation path : branchingToClose.getPaths()) {
+//                parentsInToClose.add(path.getBottom());
+                DagNodeWithCondition.removeFromCollection(newParentsWithConditions, path.getBottom());
+            }
+
+            final Join newJoin = joinPaths(branchingToClose.getFork(), branchingToClose.getPaths());
+
+            newParentsWithConditions.add(new DagNodeWithCondition(newJoin, null));
+
+            handleMultiParentNodeWithParents(node, newParentsWithConditions);
+        }
+        else {
+            // There are no intermediary fork / join pairs to insert, we have to join all paths in a single join.
+            final Join newJoin = joinPaths(branchingToClose.getFork(), branchingToClose.getPaths());
+
+            if (newJoin != null) {
+                addParentWithForkIfNeeded(node, new DagNodeWithCondition(newJoin, null));
+            }
+            else {
+                // Null means a part of the paths was relocated because of a decision node.
+                handleNodeWithParents(node, parentsWithConditions);
+            }
+        }
+    }
+
+    // Returning null means we have relocated a part of the paths because of decision nodes, so the caller should try
+    // adding the node again.
+    private Join joinPaths(final Fork fork, final List<PathInformation> pathsToJoin) {
+        final Map<PathInformation, Decision> highestDecisionNodes = new LinkedHashMap<>();
+        for (final PathInformation path : pathsToJoin) {
+            for (int ixNodeOnPath = 0; ixNodeOnPath < path.getNodes().size(); ++ixNodeOnPath) {
+                final NodeBase nodeOnPath = path.getNodes().get(ixNodeOnPath);
+
+                if (nodeOnPath instanceof Decision) {
+                    // Excluding decision nodes where no branch goes out of this fork / join pair.
+                    if (!isDecisionClosed((Decision) nodeOnPath)) {
+                        highestDecisionNodes.put(path, (Decision) nodeOnPath);
+                    }
+                }
+                else if (nodeOnPath == fork) {
+                    break;
+                }
+            }
+        }
+
+        if (highestDecisionNodes.isEmpty()) {
+            return joinPathsWithoutDecisions(fork, pathsToJoin);
+        }
+        else {
+            return joinPathsWithDecisions(fork, pathsToJoin, highestDecisionNodes);
+        }
+    }
+
+    private Join joinPathsWithoutDecisions(final Fork fork, final List<PathInformation> pathsToJoin) {
+        final Set<NodeBase> mainBranchNodes = new LinkedHashSet<>();
+        for (final PathInformation pathInformation : pathsToJoin) {
+            mainBranchNodes.addAll(pathInformation.getNodes());
+        }
+
+        // Taking care of side branches.
+        final Set<NodeBase> closedNodes = new HashSet<>();
+        final List<NodeBase> sideBranches = new ArrayList<>();
+        for (final PathInformation path : pathsToJoin) {
+            for (int ixNodeOnPath = 0; ixNodeOnPath < path.getNodes().size(); ++ixNodeOnPath) {
+                final NodeBase nodeOnPath = path.getNodes().get(ixNodeOnPath);
+
+                if (nodeOnPath == fork) {
+                    break;
+                }
+
+                if (nodeOnPath instanceof Decision && isDecisionClosed((Decision) nodeOnPath)) {
+                    break;
+                }
+
+                sideBranches.addAll(cutSideBranches(nodeOnPath, mainBranchNodes));
+                closedNodes.add(nodeOnPath);
+            }
+        }
+
+        final Join newJoin;
+
+        // Check if we have to divide the fork.
+        final boolean hasMoreForkedChildren = pathsToJoin.size() < fork.getChildren().size();
+        if (hasMoreForkedChildren) {
+            // Dividing the fork.
+            newJoin = divideForkAndCloseSubFork(fork, pathsToJoin);
+        } else {
+            // We don't divide the fork.
+            newJoin = newJoin(fork);
+
+            for (final PathInformation path : pathsToJoin) {
+                addParentWithForkIfNeeded(newJoin, new DagNodeWithCondition(path.getBottom(), null));
+            }
+        }
+
+        // Inserting the side branches under the new join node.
+        for (final NodeBase sideBranch : sideBranches) {
+            addParentWithForkIfNeeded(sideBranch, new DagNodeWithCondition(newJoin, null));
+        }
+
+        // Marking the nodes as closed.
+        for (final NodeBase closedNode : closedNodes) {
+            markAsClosed(closedNode, newJoin);
+        }
+
+        return newJoin;
+    }
+
+    private Join joinPathsWithDecisions(final Fork fork,
+                                        final List<PathInformation> pathsToJoin,
+                                        final Map<PathInformation, Decision> highestDecisionNodes) {
+        final Set<Decision> decisions = new HashSet<>(highestDecisionNodes.values());
+
+        final List<PathInformation> newPaths = new ArrayList<>();
+        boolean shouldCloseJoinAndAddOtherDecisionsUnderIt = false;
+        for (final Decision decision : decisions) {
+            final NodeBase parentOfDecision = decision.getParent();
+
+            if (parentOfDecision == fork) {
+                shouldCloseJoinAndAddOtherDecisionsUnderIt = true;
+                break;
+            }
+
+            newPaths.add(getPathInfo(parentOfDecision));
+            removeParentWithForkIfNeeded(decision, decision.getParent());
+        }
+
+        if (shouldCloseJoinAndAddOtherDecisionsUnderIt) {
+            closeJoinAndAddOtherDecisionsUnderIt(fork, decisions);
+        }
+        else {
+            for (final PathInformation path : pathsToJoin) {
+                if (!highestDecisionNodes.containsKey(path)) {
+                    newPaths.add(path);
+                }
+            }
+
+            final Join newJoin = joinPaths(fork, newPaths);
+
+            for (final Decision decision : decisions) {
+                addParentWithForkIfNeeded(decision, new DagNodeWithCondition(newJoin, null));
+            }
+        }
+
+        return null;
+    }
+
+    private void closeJoinAndAddOtherDecisionsUnderIt(final Fork fork, final Set<Decision> decisions) {
+        // TODO: Either implement it correctly or give a more informative error message.
+        throw new IllegalStateException("Conditional paths originating ultimately from the same parallel branching (fork) " +
+                "do not converge to the same join.");
+    }
+
+    private void markAsClosed(final NodeBase node, final Join join) {
+        closingJoins.put(node, join);
+    }
+
+    private List<NodeBase> cutSideBranches(final NodeBase node,
+                                           final Set<NodeBase> mainBranchNodes) {
+        final List<NodeBase> sideBranches = new ArrayList<>();
+
+        // Closed forks cannot have side branches.
+        final boolean isClosedFork = node instanceof Fork && ((Fork) node).isClosed();
+        if (!isClosedFork) {
+            for (final NodeBase childOfForkOrParent : node.getChildren()) {
+                if (!mainBranchNodes.contains(childOfForkOrParent)) {
+                    removeParentWithForkIfNeeded(childOfForkOrParent, node);
+                    sideBranches.add(childOfForkOrParent);
+                }
+            }
+        }
+
+        return sideBranches;
+    }
+
+    private Join divideForkAndCloseSubFork(final Fork correspondingFork,
+                                           final List<PathInformation> paths) {
+        final Fork newFork = newFork();
+        for (final PathInformation path : paths) {
+            final int indexOfFork = path.getNodes().indexOf(correspondingFork);
+            final NodeBase childOfOriginalFork = path.getNodes().get(indexOfFork - 1);
+
+            childOfOriginalFork.removeParent(correspondingFork);
+            childOfOriginalFork.addParent(newFork);
+        }
+
+        newFork.addParent(correspondingFork);
+
+        final Join newJoin = newJoin(newFork);
+
+        for (final PathInformation path : paths) {
+            newJoin.addParent(path.getBottom());
+        }
+
+        return newJoin;
+    }
+
+    private BranchingToClose chooseBranchingToClose(final List<PathInformation> paths) {
+        int maxPathLength = 0;
+        for (final PathInformation pathInformation : paths) {
+            if (maxPathLength < pathInformation.getNodes().size()) {
+                maxPathLength = pathInformation.getNodes().size();
+            }
+        }
+
+        for (int ixLevel = 0; ixLevel < maxPathLength; ++ixLevel) {
+            final BranchingToClose foundAtThisLevel = chooseBranchingToClose(paths, ixLevel);
+
+            if (foundAtThisLevel != null) {
+                return foundAtThisLevel;
+            }
+        }
+
+        throw new IllegalStateException("We should never reach here.");
+    }
+
+    private BranchingToClose chooseBranchingToClose(final List<PathInformation> paths, final int ixLevel) {
+        for (final PathInformation path : paths) {
+            if (ixLevel < path.getNodes().size()) {
+                final NodeBase branching = path.getNodes().get(ixLevel);
+
+                final List<PathInformation> pathsMeetingAtCurrentFork = getPathsContainingNode(branching, paths);
+
+                if (pathsMeetingAtCurrentFork.size() > 1) {
+                    final boolean needToSplitJoin = pathsMeetingAtCurrentFork.size() < paths.size();
+
+                    // If branching is not a Fork or a Decision, then it is a redundant parent.
+                    if (branching instanceof Fork) {
+                        return BranchingToClose.withFork((Fork) branching, pathsMeetingAtCurrentFork, needToSplitJoin);
+                    } else if (branching instanceof Decision) {
+                        return BranchingToClose.withDecision((Decision) branching, pathsMeetingAtCurrentFork, needToSplitJoin);
+                    }
+                    else {
+                        return BranchingToClose.withRedundantParent(branching, pathsMeetingAtCurrentFork, needToSplitJoin);
+                    }
+                }
+            }
+        }
+
+        return null;
+    }
+
+    private List<PathInformation> getPathsContainingNode(final NodeBase node, final List<PathInformation> paths) {
+        final List<PathInformation> pathsContainingNode = new ArrayList<>();
+
+        for (final PathInformation pathInformationMaybeContaining : paths) {
+            if (pathInformationMaybeContaining.getNodes().contains(node)) {
+                pathsContainingNode.add(pathInformationMaybeContaining);
+            }
+        }
+
+        return pathsContainingNode;
+    }
+
+    private PathInformation getPathInfo(final NodeBase node) {
+        NodeBase current = node;
+
+        final List<NodeBase> nodes = new ArrayList<>();
+
+        while (current != start) {
+            nodes.add(current);
+
+            if (current instanceof Join) {
+                // Get the fork pair of this join and go towards that
+                final Fork forkPair = ((Join) current).getBranchingPair();
+                current = forkPair;
+            }
+            else if (current instanceof DecisionJoin) {
+                final Decision decisionPair = ((DecisionJoin) current).getBranchingPair();
+                current = decisionPair;
+            }
+            else {
+                current = getSingleParent(current);
+            }
+        }
+
+        return new PathInformation(nodes);
+    }
+
+    private boolean isDecisionClosed(final Decision decision) {
+        final Integer closedPathsOfDecisionNode = closedPathsOfDecisionNodes.get(decision);
+        return closedPathsOfDecisionNode != null && decision.getChildren().size() == closedPathsOfDecisionNode;
+    }
+
+    private NodeBase getSingleParent(final NodeBase node) {
+        if (node instanceof End) {
+            return ((End) node).getParent();
+        }
+        else if (node instanceof Fork) {
+            return ((Fork) node).getParent();
+        }
+        else if (node instanceof Decision) {
+            return ((Decision) node).getParent();
+        }
+        else if (node instanceof ExplicitNode) {
+            return ((ExplicitNode) node).getParent();
+        }
+        else if (node instanceof Start) {
+            throw new IllegalStateException("Start nodes have no parent.");
+        }
+        else if (node instanceof Join) {
+            final Join join = (Join) node;
+            final int numberOfParents = join.getParents().size();
+            if (numberOfParents != 1) {
+                throw new IllegalStateException("The join node called '" + node.getName()
+                        + "' has " + numberOfParents + " parents instead of 1.");
+            }
+
+            return join.getParents().get(0);
+        }
+        else if (node == null) {
+            throw new IllegalArgumentException("Null node found.");
+        }
+
+        throw new IllegalArgumentException("Unknown node type.");
+    }
+
+    // Returns the first descendant that is not inside a closed fork / join pair.
+    private NodeBase getNearestNonClosedDescendant(final NodeBase node) {
+        NodeBase current = node;
+
+        while (closingJoins.containsKey(current)) {
+            current = closingJoins.get(current);
+        }
+
+        return current;
+    }
+
+    private void addParentWithForkIfNeeded(final NodeBase node, final DagNodeWithCondition parentWithCondition) {
+        final NodeBase parent = parentWithCondition.getNode();
+        final Condition condition = parentWithCondition.getCondition();
+        if (parent.getChildren().isEmpty() || parent instanceof Fork || parent instanceof Decision) {
+            if (condition != null) {
+                if (!(parent instanceof Decision)) {
+                    throw new IllegalStateException("Trying to add a conditional parent that is not a decision.");
+                }
+
+                node.addParentWithCondition((Decision) parent, condition);
+            }
+            else {
+                node.addParent(parent);
+            }
+        }
+        else {
+            // If there is no child, we never get to this point.
+            // There is only one child, otherwise it is a fork and we don't get here.
+            final NodeBase child = parent.getChildren().get(0);
+
+            if (child instanceof Fork) {
+                node.addParent(child);
+            }
+            else if (child instanceof Join) {
+                addParentWithForkIfNeeded(node, new DagNodeWithCondition(child, null));
+            }
+            else {
+                final Fork newFork = newFork();
+
+                child.removeParent(parent);
+                child.addParent(newFork);
+                node.addParent(newFork);
+                newFork.addParent(parent);
+            }
+        }
+    }
+
+    private void removeParentWithForkIfNeeded(final NodeBase node, final NodeBase parent) {
+        node.removeParent(parent);
+
+        final boolean isParentForkAndHasOneChild = parent instanceof Fork && parent.getChildren().size() == 1;
+        if (isParentForkAndHasOneChild) {
+            final NodeBase grandparent = ((Fork) parent).getParent();
+            final NodeBase child = parent.getChildren().get(0);
+
+            removeParentWithForkIfNeeded(parent, grandparent);
+            child.removeParent(parent);
+            child.addParent(grandparent);
+            nodesByName.remove(parent.getName());
+        }
+    }
+
+    private Fork newFork() {
+        final Fork fork = new Fork("fork" + forkCounter);
+
+        forkNumbers.put(fork, forkCounter);
+        forkCounter++;
+
+        storeNode(fork);
+
+        return fork;
+    }
+
+    private Join newJoin(final Fork correspondingFork) {
+        final Join join = new Join("join" + forkNumbers.get(correspondingFork), correspondingFork);
+
+        storeNode(join);
+
+        return join;
+    }
+
+    private Decision newDecision() {
+        final Decision decision = new Decision("decision" + decisionCounter);
+
+        decisionCounter++;
+
+        storeNode(decision);
+
+        return decision;
+    }
+
+    private DecisionJoin newDecisionJoin(final Decision correspondingDecision, final int numberOfPathsClosed) {
+        final DecisionJoin decisionJoin = new DecisionJoin("decisionJoin" + decisionJoinCounter, correspondingDecision);
+
+        final Integer numberOfAlreadyClosedChildren = closedPathsOfDecisionNodes.get(correspondingDecision);
+        final int newNumber = numberOfPathsClosed + (numberOfAlreadyClosedChildren == null ? 0 : numberOfAlreadyClosedChildren);
+
+        closedPathsOfDecisionNodes.put(correspondingDecision, newNumber);
+
+        decisionJoinCounter++;
+
+        storeNode(decisionJoin);
+
+        return decisionJoin;
+    }
+
+    private static List<Node> getNodesFromRootsToLeaves(final Workflow workflow) {
+        final List<Node> nodes = new ArrayList<>(workflow.getRoots());
+
+        for (int i = 0; i < nodes.size(); ++i) {
+            final Node current  = nodes.get(i);
+
+            for (final Node child : current.getAllChildren()) {
+                // Checking if every dependency has been processed, if not, we do not add the start to the list.
+                final List<Node> dependencies = child.getAllParents();
+                if (nodes.containsAll(dependencies) && !nodes.contains(child)) {
+                    nodes.add(child);
+                }
+            }
+        }
+
+        return nodes;
+    }
+
+    public Credentials getCredentials() {
+        return credentials;
+    }
+
+    private static class PathInformation {
+        private final ImmutableList<NodeBase> nodes;
+
+        PathInformation(final List<NodeBase> nodes) {
+            this.nodes = new ImmutableList.Builder<NodeBase>().addAll(nodes).build();
+        }
+
+        NodeBase getBottom() {
+            return nodes.get(0);
+        }
+
+        public List<NodeBase> getNodes() {
+            return nodes;
+        }
+
+    }
+
+    private static class BranchingToClose {
+        private final Fork fork;
+        private final Decision decision;
+        private final NodeBase redundantParent;
+        private final ImmutableList<PathInformation> paths;
+        private final boolean needToSplitJoin;
+
+        static BranchingToClose withFork(final Fork fork,
+                                         final List<PathInformation> paths,
+                                         final boolean needToSplitJoin) {
+            return new BranchingToClose(fork, null, null, paths, needToSplitJoin);
+        }
+
+        static BranchingToClose withDecision(final Decision decision,
+                                             final List<PathInformation> paths,
+                                             final boolean needToSplitJoin) {
+            return new BranchingToClose(null, decision, null, paths, needToSplitJoin);
+        }
+
+        static BranchingToClose withRedundantParent(final NodeBase redundantParent,
+                                                    final List<PathInformation> paths,
+                                                    final boolean needToSplitJoin) {
+            return new BranchingToClose(null, null, redundantParent, paths, needToSplitJoin);
+        }
+
+        private BranchingToClose(final Fork fork,
+                                 final Decision decision,
+                                 final NodeBase redundantParent,
+                                 final List<PathInformation> paths,
+                                 final boolean needToSplitJoin) {
+            checkOnlyOneIsNotNull(fork, decision, redundantParent);
+
+            this.fork = fork;
+            this.decision = decision;
+            this.redundantParent = redundantParent;
+            this.paths = ImmutableList.copyOf(paths);
+            this.needToSplitJoin = needToSplitJoin;
+        }
+
+        public Fork getFork() {
+            return fork;
+        }
+
+        public Decision getDecision() {
+            return decision;
+        }
+
+        NodeBase getRedundantParent() {
+            return redundantParent;
+        }
+
+        List<PathInformation> getPaths() {
+            return paths;
+        }
+
+        boolean isDecision() {
+            return decision != null;
+        }
+
+        boolean isRedundantParent() {
+            return redundantParent != null;
+        }
+
+        boolean isSplittingJoinNeeded() {
+            return needToSplitJoin;
+        }
+
+        private void checkOnlyOneIsNotNull(final Fork fork, final Decision decision, final NodeBase redundantParent) {
+            int counter = 0;
+
+            if (fork != null) {
+                ++counter;
+            }
+
+            if (decision != null) {
+                ++counter;
+            }
+
+            if (redundantParent != null) {
+                ++counter;
+            }
+
+            Preconditions.checkArgument(counter == 1, "Exactly one of 'fork' and 'redundantParent' must be non-null.");
+        }
+    }
+}


[06/16] oozie git commit: OOZIE-2339 [fluent-job] Minimum Viable Fluent Job API (daniel.becker, andras.piros via rkanter, gezapeti, pbacsko)

Posted by an...@apache.org.
http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestJavaActionBuilder.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestJavaActionBuilder.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestJavaActionBuilder.java
new file mode 100644
index 0000000..c068550
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestJavaActionBuilder.java
@@ -0,0 +1,225 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestJavaActionBuilder extends TestNodeBuilderBaseImpl<JavaAction, JavaActionBuilder> {
+    private static final String NAME = "ssh-name";
+    private static final String NAME_NODE = "${nameNode}";
+    private static final String EXAMPLE_DIR = "/path/to/directory";
+    private static final String[] ARGS = {"arg1", "arg2", "arg3"};
+    private static final String MAPRED_JOB_QUEUE_NAME = "mapred.job.queue.name";
+    private static final String DEFAULT = "default";
+    private static final String RESOURCE_MANAGER = "${resourceManager}";
+    private static final String PATH_TO_DELETE = "/path/to/delete";
+    private static final String PATH_TO_MKDIR = "/path/to/mkdir";
+
+    @Override
+    protected JavaActionBuilder getBuilderInstance() {
+        return JavaActionBuilder.create();
+    }
+
+    @Override
+    protected JavaActionBuilder getBuilderInstance(final JavaAction action) {
+        return JavaActionBuilder.createFromExistingAction(action);
+    }
+
+    @Test
+    public void testResourceManagerAdded() {
+        final JavaActionBuilder builder = getBuilderInstance();
+        builder.withResourceManager(RESOURCE_MANAGER);
+
+        final JavaAction action = builder.build();
+        assertEquals(RESOURCE_MANAGER, action.getResourceManager());
+    }
+
+    @Test
+    public void testNameNodeAdded() {
+        final JavaActionBuilder builder = getBuilderInstance();
+        builder.withNameNode(NAME_NODE);
+
+        final JavaAction action = builder.build();
+        assertEquals(NAME_NODE, action.getNameNode());
+    }
+
+    @Test
+    public void testPrepareAdded() {
+        final JavaActionBuilder builder = getBuilderInstance();
+        builder.withPrepare(new PrepareBuilder().withDelete(EXAMPLE_DIR).build());
+
+        final JavaAction action = builder.build();
+        assertEquals(EXAMPLE_DIR, action.getPrepare().getDeletes().get(0).getPath());
+    }
+
+    @Test
+    public void testSameConfigPropertyAddedTwiceThrows() {
+        final JavaActionBuilder builder = getBuilderInstance();
+        builder.withConfigProperty(MAPRED_JOB_QUEUE_NAME, DEFAULT);
+
+        expectedException.expect(IllegalStateException.class);
+        builder.withConfigProperty(MAPRED_JOB_QUEUE_NAME, DEFAULT);
+    }
+
+    @Test
+    public void testSeveralArgsAdded() {
+        final JavaActionBuilder builder = getBuilderInstance();
+
+        for (final String arg : ARGS) {
+            builder.withArg(arg);
+        }
+
+        final JavaAction action = builder.build();
+
+        final List<String> argList = action.getArgs();
+        assertEquals(ARGS.length, argList.size());
+
+        for (int i = 0; i < ARGS.length; ++i) {
+            assertEquals(ARGS[i], argList.get(i));
+        }
+    }
+
+    @Test
+    public void testRemoveArgs() {
+        final JavaActionBuilder builder = getBuilderInstance();
+
+        for (final String file : ARGS) {
+            builder.withArg(file);
+        }
+
+        builder.withoutArg(ARGS[0]);
+
+        final JavaAction action = builder.build();
+
+        final List<String> argList = action.getArgs();
+        final String[] remainingArgs = Arrays.copyOfRange(ARGS, 1, ARGS.length);
+        assertEquals(remainingArgs.length, argList.size());
+
+        for (int i = 0; i < remainingArgs.length; ++i) {
+            assertEquals(remainingArgs[i], argList.get(i));
+        }
+    }
+
+    @Test
+    public void testClearArgs() {
+        final JavaActionBuilder builder = getBuilderInstance();
+
+        for (final String file : ARGS) {
+            builder.withArg(file);
+        }
+
+        builder.clearArgs();
+
+        final JavaAction action = builder.build();
+
+        final List<String> argList = action.getArgs();
+        assertEquals(0, argList.size());
+    }
+
+    @Test
+    public void testFromExistingJavaAction() {
+        final JavaActionBuilder builder = getBuilderInstance();
+
+        builder.withName(NAME)
+                .withResourceManager(RESOURCE_MANAGER)
+                .withNameNode(NAME_NODE)
+                .withConfigProperty(MAPRED_JOB_QUEUE_NAME, DEFAULT)
+                .withPrepare(new PrepareBuilder()
+                        .withDelete(PATH_TO_DELETE)
+                        .withMkdir(PATH_TO_MKDIR)
+                        .build())
+                .withLauncher(new LauncherBuilder()
+                        .withMemoryMb(1024L)
+                        .withVCores(2L)
+                        .withQueue(DEFAULT)
+                        .withSharelib(DEFAULT)
+                        .withViewAcl(DEFAULT)
+                        .withModifyAcl(DEFAULT)
+                        .build())
+                .withMainClass(DEFAULT)
+                .withJavaOptsString(DEFAULT)
+                .withJavaOpt(DEFAULT)
+                .withArg(ARGS[0])
+                .withArg(ARGS[1])
+                .withArchive(DEFAULT)
+                .withFile(DEFAULT)
+                .withCaptureOutput(true);
+
+        final JavaAction action = builder.build();
+
+        final JavaActionBuilder fromExistingBuilder = getBuilderInstance(action);
+
+        final String newName = "fromExisting_" + NAME;
+        fromExistingBuilder.withName(newName)
+                .withoutArg(ARGS[1])
+                .withArg(ARGS[2]);
+
+        final JavaAction modifiedAction = fromExistingBuilder.build();
+
+        assertEquals(newName, modifiedAction.getName());
+        assertEquals(action.getNameNode(), modifiedAction.getNameNode());
+
+        final Map<String, String> expectedConfiguration = new LinkedHashMap<>();
+        expectedConfiguration.put(MAPRED_JOB_QUEUE_NAME, DEFAULT);
+        assertEquals(expectedConfiguration, modifiedAction.getConfiguration());
+
+        assertEquals(Arrays.asList(ARGS[0], ARGS[2]), modifiedAction.getArgs());
+
+        assertEquals(PATH_TO_DELETE, modifiedAction.getPrepare().getDeletes().get(0).getPath());
+        assertEquals(PATH_TO_MKDIR, modifiedAction.getPrepare().getMkdirs().get(0).getPath());
+
+        assertEquals(1024L, modifiedAction.getLauncher().getMemoryMb());
+        assertEquals(2L, modifiedAction.getLauncher().getVCores());
+        assertEquals(DEFAULT, modifiedAction.getLauncher().getQueue());
+        assertEquals(DEFAULT, modifiedAction.getLauncher().getSharelib());
+        assertEquals(DEFAULT, modifiedAction.getLauncher().getViewAcl());
+        assertEquals(DEFAULT, modifiedAction.getLauncher().getModifyAcl());
+
+        assertEquals(action.getMainClass(), modifiedAction.getMainClass());
+        assertEquals(action.getJavaOptsString(), modifiedAction.getJavaOptsString());
+        assertEquals(action.getJavaOpts().get(0), modifiedAction.getJavaOpts().get(0));
+        assertEquals(action.isCaptureOutput(), modifiedAction.isCaptureOutput());
+    }
+
+    @Test
+    public void testFromOtherAction() {
+        final ShellAction parent = ShellActionBuilder.create()
+                .withName("parent")
+                .build();
+
+        final ShellAction otherAction = ShellActionBuilder.createFromExistingAction(parent)
+                .withName("shell")
+                .withParent(parent)
+                .build();
+
+        final JavaAction fromOtherAction = JavaActionBuilder.createFromExistingAction(otherAction)
+                .withName("java")
+                .build();
+
+        assertEquals("java", fromOtherAction.getName());
+        assertEquals(parent, fromOtherAction.getParentsWithoutConditions().get(0));
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestLauncherBuilder.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestLauncherBuilder.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestLauncherBuilder.java
new file mode 100644
index 0000000..acf8920
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestLauncherBuilder.java
@@ -0,0 +1,44 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestLauncherBuilder {
+    @Test
+    public void testAttributesSetOnce() {
+        final Launcher launcher = new LauncherBuilder()
+                .withMemoryMb(1024)
+                .withVCores(2)
+                .withQueue("default")
+                .withSharelib("default")
+                .withViewAcl("default")
+                .withModifyAcl("default")
+                .build();
+
+        assertEquals(1024, launcher.getMemoryMb());
+        assertEquals(2, launcher.getVCores());
+        assertEquals("default", launcher.getQueue());
+        assertEquals("default", launcher.getSharelib());
+        assertEquals("default", launcher.getViewAcl());
+        assertEquals("default", launcher.getModifyAcl());
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestMapReduceActionBuilder.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestMapReduceActionBuilder.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestMapReduceActionBuilder.java
new file mode 100644
index 0000000..e6419ff
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestMapReduceActionBuilder.java
@@ -0,0 +1,392 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestMapReduceActionBuilder extends TestNodeBuilderBaseImpl<MapReduceAction, MapReduceActionBuilder> {
+    private static final String NAME = "map-reduce-name";
+    private static final String NAME_NODE = "${nameNode}";
+    private static final String EXAMPLE_DIR = "/path/to/directory";
+    private static final String CONFIG_CLASS = "AnyConfigClass.class";
+    private static final String[] JOB_XMLS = {"jobXml1.xml", "jobXml2.xml", "jobXml3.xml", "jobXml4.xml"};
+    private static final String[] FILES = {"file1.xml", "file2.xml", "file3.xml", "file4.xml"};
+    private static final String[] ARCHIVES = {"archive1.jar", "archive2.jar", "archive3.jar", "archive4.jar"};
+    private static final String RESOURCE_MANAGER = "${resourceManager}";
+
+    private static final String MAPRED_JOB_QUEUE_NAME = "mapred.job.queue.name";
+    private static final String DEFAULT = "default";
+
+    @Override
+    protected MapReduceActionBuilder getBuilderInstance() {
+        return MapReduceActionBuilder.create();
+    }
+
+    @Override
+    protected MapReduceActionBuilder getBuilderInstance(final MapReduceAction action) {
+        return MapReduceActionBuilder.createFromExistingAction(action);
+    }
+
+    @Test
+    public void testResourceManagerAdded() {
+        final MapReduceActionBuilder builder = getBuilderInstance();
+        builder.withResourceManager(RESOURCE_MANAGER);
+
+        final MapReduceAction action = builder.build();
+        assertEquals(RESOURCE_MANAGER, action.getResourceManager());
+    }
+
+    @Test
+    public void testNameNodeAdded() {
+        final MapReduceActionBuilder builder = getBuilderInstance();
+        builder.withNameNode(NAME_NODE);
+
+        final MapReduceAction mrAction = builder.build();
+        assertEquals(NAME_NODE, mrAction.getNameNode());
+    }
+
+    @Test
+    public void testNameNodeAddedTwiceThrows() {
+        final MapReduceActionBuilder builder = getBuilderInstance();
+        builder.withNameNode(NAME_NODE);
+
+        expectedException.expect(IllegalStateException.class);
+        builder.withNameNode("any_string");
+    }
+
+    @Test
+    public void testPrepareAdded() {
+        final MapReduceActionBuilder builder = getBuilderInstance();
+        builder.withPrepare(new PrepareBuilder().withDelete(EXAMPLE_DIR).build());
+
+        final MapReduceAction mrAction = builder.build();
+        assertEquals(EXAMPLE_DIR, mrAction.getPrepare().getDeletes().get(0).getPath());
+    }
+
+    @Test
+    public void testPrepareAddedTwiceThrows() {
+        final MapReduceActionBuilder builder = getBuilderInstance();
+        builder.withPrepare(new PrepareBuilder().withDelete(EXAMPLE_DIR).build());
+
+        expectedException.expect(IllegalStateException.class);
+        builder.withPrepare(new PrepareBuilder().withDelete("any_directory").build());
+    }
+
+    @Test
+    public void testStreamingAdded() {
+        final Streaming streaming = new StreamingBuilder().withMapper("mapper.sh").withReducer("reducer.sh").build();
+
+        final MapReduceActionBuilder builder = getBuilderInstance();
+        builder.withStreaming(streaming);
+
+        final MapReduceAction mrAction = builder.build();
+        assertEquals(streaming, mrAction.getStreaming());
+    }
+
+    @Test
+    public void testStreamingAddedTwiceThrows() {
+        final Streaming streaming1= new StreamingBuilder().withMapper("mapper1.sh").withReducer("reducer1.sh").build();
+        final Streaming streaming2 = new StreamingBuilder().withMapper("mapper2.sh").withReducer("reducer2.sh").build();
+
+        final MapReduceActionBuilder builder = getBuilderInstance();
+        builder.withStreaming(streaming1);
+
+        expectedException.expect(IllegalStateException.class);
+        builder.withStreaming(streaming2);
+    }
+
+    @Test
+    public void testPipesAdded() {
+        final Pipes pipes = new PipesBuilder().withMap("map").withReduce("reduce").build();
+
+        final MapReduceActionBuilder builder = getBuilderInstance();
+        builder.withPipes(pipes);
+
+        final MapReduceAction mrAction = builder.build();
+        assertEquals(pipes, mrAction.getPipes());
+    }
+
+    @Test
+    public void testPipesAddedTwiceThrows() {
+        final Pipes pipes1 = new PipesBuilder().withMap("map1").withReduce("reduce1").build();
+        final Pipes pipes2 = new PipesBuilder().withMap("map2").withReduce("reduce2").build();
+
+        final MapReduceActionBuilder builder = getBuilderInstance();
+        builder.withPipes(pipes1);
+
+        expectedException.expect(IllegalStateException.class);
+        builder.withPipes(pipes2);
+    }
+
+    @Test
+    public void testConfigClassAdded() {
+        final MapReduceActionBuilder builder = getBuilderInstance();
+        builder.withConfigClass(CONFIG_CLASS);
+
+        final MapReduceAction mrAction = builder.build();
+        assertEquals(CONFIG_CLASS, mrAction.getConfigClass());
+    }
+
+    @Test
+    public void testConfigClassAddedTwiceThrows() {
+        final MapReduceActionBuilder builder = getBuilderInstance();
+        builder.withConfigClass(CONFIG_CLASS);
+
+        expectedException.expect(IllegalStateException.class);
+        builder.withConfigClass("AnyClass");
+    }
+
+    @Test
+    public void testSeveralJobXmlsAdded() {
+        final MapReduceActionBuilder builder = getBuilderInstance();
+
+        for (final String jobXml : JOB_XMLS) {
+            builder.withJobXml(jobXml);
+        }
+
+        final MapReduceAction mrAction = builder.build();
+
+        final List<String> jobXmlsList = mrAction.getJobXmls();
+        assertEquals(JOB_XMLS.length, jobXmlsList.size());
+
+        for (int i = 0; i < JOB_XMLS.length; ++i) {
+            assertEquals(JOB_XMLS[i], jobXmlsList.get(i));
+        }
+    }
+
+    @Test
+    public void testWithoutJobXmls() {
+        final MapReduceActionBuilder builder = getBuilderInstance();
+
+        for (final String jobXml : JOB_XMLS) {
+            builder.withJobXml(jobXml);
+        }
+
+        builder.withoutJobXml(JOB_XMLS[0]);
+
+        final MapReduceAction mrAction = builder.build();
+
+        final List<String> jobXmlsList = mrAction.getJobXmls();
+        final String[] remainingJobXmls = Arrays.copyOfRange(JOB_XMLS, 1, JOB_XMLS.length);
+        assertEquals(remainingJobXmls.length, jobXmlsList.size());
+
+        for (int i = 0; i < remainingJobXmls.length; ++i) {
+            assertEquals(remainingJobXmls[i], jobXmlsList.get(i));
+        }
+    }
+
+    @Test
+    public void testClearJobXmls() {
+        final MapReduceActionBuilder builder = getBuilderInstance();
+
+        for (final String jobXml : JOB_XMLS) {
+            builder.withJobXml(jobXml);
+        }
+
+        builder.clearJobXmls();
+
+        final MapReduceAction mrAction = builder.build();
+
+        final List<String> jobXmlsList = mrAction.getJobXmls();
+        assertEquals(0, jobXmlsList.size());
+    }
+
+    @Test
+    public void testSameConfigPropertyAddedTwiceThrows() {
+        final MapReduceActionBuilder builder = getBuilderInstance();
+        builder.withConfigProperty(MAPRED_JOB_QUEUE_NAME, DEFAULT);
+
+        expectedException.expect(IllegalStateException.class);
+        builder.withConfigProperty(MAPRED_JOB_QUEUE_NAME, DEFAULT);
+    }
+
+    @Test
+    public void testSeveralFilesAdded() {
+        final MapReduceActionBuilder builder = getBuilderInstance();
+
+        for (final String file : FILES) {
+            builder.withFile(file);
+        }
+
+        final MapReduceAction mrAction = builder.build();
+
+        final List<String> filesList = mrAction.getFiles();
+        assertEquals(FILES.length, filesList.size());
+
+        for (int i = 0; i < FILES.length; ++i) {
+            assertEquals(FILES[i], filesList.get(i));
+        }
+    }
+
+    @Test
+    public void testRemoveFiles() {
+        final MapReduceActionBuilder builder = getBuilderInstance();
+
+        for (final String file : FILES) {
+            builder.withFile(file);
+        }
+
+        builder.withoutFile(FILES[0]);
+
+        final MapReduceAction mrAction = builder.build();
+
+        final List<String> filesList = mrAction.getFiles();
+        final String[] remainingFiles = Arrays.copyOfRange(FILES, 1, FILES.length);
+        assertEquals(remainingFiles.length, filesList.size());
+
+        for (int i = 0; i < remainingFiles.length; ++i) {
+            assertEquals(remainingFiles[i], filesList.get(i));
+        }
+    }
+
+    @Test
+    public void testClearFiles() {
+        final MapReduceActionBuilder builder = getBuilderInstance();
+
+        for (final String file : FILES) {
+            builder.withFile(file);
+        }
+
+        builder.clearFiles();
+
+        final MapReduceAction mrAction = builder.build();
+
+        final List<String> filesList = mrAction.getFiles();
+        assertEquals(0, filesList.size());
+    }
+
+    @Test
+    public void testSeveralArchivesAdded() {
+        final MapReduceActionBuilder builder = getBuilderInstance();
+
+        for (final String archive : ARCHIVES) {
+            builder.withArchive(archive);
+        }
+
+        final MapReduceAction mrAction = builder.build();
+
+        final List<String> filesList = mrAction.getArchives();
+        assertEquals(ARCHIVES.length, filesList.size());
+
+        for (int i = 0; i < ARCHIVES.length; ++i) {
+            assertEquals(ARCHIVES[i], filesList.get(i));
+        }
+    }
+
+    @Test
+    public void testRemoveArchives() {
+        final MapReduceActionBuilder builder = getBuilderInstance();
+
+        for (final String archive : ARCHIVES) {
+            builder.withArchive(archive);
+        }
+
+        builder.withoutArchive(ARCHIVES[0]);
+
+        final MapReduceAction mrAction = builder.build();
+
+        final List<String> archivesList = mrAction.getArchives();
+        final String[] remainingArchives = Arrays.copyOfRange(ARCHIVES, 1, ARCHIVES.length);
+        assertEquals(remainingArchives.length, archivesList.size());
+
+        for (int i = 0; i < remainingArchives.length; ++i) {
+            assertEquals(remainingArchives[i], archivesList.get(i));
+        }
+    }
+
+    @Test
+    public void testClearArchives() {
+        final MapReduceActionBuilder builder = getBuilderInstance();
+
+        for (final String archive : ARCHIVES) {
+            builder.withArchive(archive);
+        }
+
+        builder.clearArchives();
+
+        final MapReduceAction mrAction = builder.build();
+
+        final List<String> archivesList = mrAction.getArchives();
+        assertEquals(0, archivesList.size());
+    }
+
+    @Test
+    public void testFromExistingActionMapReduceSpecific() {
+        final Streaming streaming = new StreamingBuilder().withMapper("mapper.sh").withReducer("reducer.sh").build();
+        final Pipes pipes = new PipesBuilder().withMap("map").withReduce("reduce").build();
+
+        final MapReduceActionBuilder builder = getBuilderInstance();
+
+        builder.withName(NAME)
+                .withNameNode(NAME_NODE)
+                .withStreaming(streaming)
+                .withPipes(pipes)
+                .withConfigProperty(MAPRED_JOB_QUEUE_NAME, DEFAULT)
+                .withFile(FILES[0])
+                .withFile(FILES[1]);
+
+        final MapReduceAction mrAction = builder.build();
+
+        final MapReduceActionBuilder fromExistingBuilder = getBuilderInstance(mrAction);
+
+        final String newName = "fromExisting_" + NAME;
+        fromExistingBuilder.withName(newName)
+                .withoutFile(FILES[1])
+                .withFile(FILES[2]);
+
+        final MapReduceAction modifiedMrAction = fromExistingBuilder.build();
+
+        assertEquals(newName, modifiedMrAction.getName());
+        assertEquals(mrAction.getNameNode(), modifiedMrAction.getNameNode());
+        assertEquals(streaming, modifiedMrAction.getStreaming());
+        assertEquals(pipes, modifiedMrAction.getPipes());
+
+        final Map<String, String> expectedConfiguration = new LinkedHashMap<>();
+        expectedConfiguration.put(MAPRED_JOB_QUEUE_NAME, DEFAULT);
+        assertEquals(expectedConfiguration, modifiedMrAction.getConfiguration());
+
+        assertEquals(Arrays.asList(FILES[0], FILES[2]), modifiedMrAction.getFiles());
+    }
+
+    @Test
+    public void testFromOtherAction() {
+        final ShellAction parent = ShellActionBuilder.create()
+                .withName("parent")
+                .build();
+
+        final ShellAction otherAction = ShellActionBuilder.createFromExistingAction(parent)
+                .withName("shell")
+                .withParent(parent)
+                .build();
+
+        final MapReduceAction fromOtherAction = MapReduceActionBuilder.createFromExistingAction(otherAction)
+                .withName("map-reduce")
+                .build();
+
+        assertEquals("map-reduce", fromOtherAction.getName());
+        assertEquals(parent, fromOtherAction.getParentsWithoutConditions().get(0));
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestMove.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestMove.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestMove.java
new file mode 100644
index 0000000..629ec96
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestMove.java
@@ -0,0 +1,36 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestMove {
+    @Test
+    public void testSourceAndTargetAreCorrect() {
+        final String source = "/path/to/source";
+        final String target = "/path/to/target";
+
+        final Move move = new Move(source, target);
+
+        assertEquals(source, move.getSource());
+        assertEquals(target, move.getTarget());
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestNodeBuilderBaseImpl.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestNodeBuilderBaseImpl.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestNodeBuilderBaseImpl.java
new file mode 100644
index 0000000..18bfa9e
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestNodeBuilderBaseImpl.java
@@ -0,0 +1,523 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.oozie.fluentjob.api.Condition;
+import org.apache.oozie.fluentjob.api.workflow.Credential;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.mockito.Mockito;
+
+import java.util.Arrays;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+
+public abstract class TestNodeBuilderBaseImpl <N extends Node,
+        B extends NodeBuilderBaseImpl<B> & Builder<N>> {
+    static final String NAME = "map-reduce-name";
+
+    @Rule
+    public final ExpectedException expectedException = ExpectedException.none();
+
+    protected abstract B getBuilderInstance();
+    protected abstract B getBuilderInstance(N action);
+
+    @Test
+    public final void testIncorrectSubclassingThrows() {
+        class WrongBuilder extends NodeBuilderBaseImpl<MapReduceActionBuilder> implements Builder<MapReduceAction> {
+
+            private WrongBuilder() {
+                super();
+            }
+
+            public MapReduceActionBuilder getRuntimeSelfReference() {
+                return MapReduceActionBuilder.create();
+            }
+
+            @Override
+            public MapReduceAction build() {
+                return null;
+            }
+        }
+
+        expectedException.expect(IllegalStateException.class);
+
+        new WrongBuilder().withName("obsolete");
+    }
+
+    @Test
+    public void testErrorHandlerAdded() {
+        final ErrorHandler errorHandler = ErrorHandler.buildAsErrorHandler(
+                MapReduceActionBuilder.create().withName("error-handler"));
+
+        final B builder = getBuilderInstance();
+        builder.withErrorHandler(errorHandler);
+
+        final N node = builder.build();
+
+        assertEquals(errorHandler, node.getErrorHandler());
+    }
+
+    @Test
+    public void testErrorHandlerAddedTwiceThrows() {
+        final ErrorHandler errorHandler1 = ErrorHandler.buildAsErrorHandler(
+                MapReduceActionBuilder.create().withName("error-handler1"));
+        final ErrorHandler errorHandler2 = ErrorHandler.buildAsErrorHandler(
+                MapReduceActionBuilder.create().withName("error-handler2"));
+
+        final B builder = getBuilderInstance();
+        builder.withErrorHandler(errorHandler1);
+
+        expectedException.expect(IllegalStateException.class);
+        builder.withErrorHandler(errorHandler2);
+    }
+
+    @Test
+    public void testWithoutErrorHandler() {
+        final ErrorHandler errorHandler = ErrorHandler.buildAsErrorHandler(
+                MapReduceActionBuilder.create().withName("error-handler"));
+
+        final B builder = getBuilderInstance();
+        builder.withErrorHandler(errorHandler);
+
+        final N node = builder.build();
+
+        final B fromExistingBuilder = getBuilderInstance(node);
+
+        fromExistingBuilder.withoutErrorHandler();
+
+        final N modifiedNode = fromExistingBuilder.build();
+
+        assertEquals(null, modifiedNode.getErrorHandler());
+    }
+
+    @Test
+    public void testRemovingErrorHandlerAfterAddingItThrows() {
+        final ErrorHandler errorHandler = ErrorHandler.buildAsErrorHandler(
+                MapReduceActionBuilder.create().withName("error-handler"));
+
+        final B builder = getBuilderInstance();
+        builder.withErrorHandler(errorHandler);
+
+        expectedException.expect(IllegalStateException.class);
+        builder.withoutErrorHandler();
+    }
+
+    @Test
+    public void testAddParents() {
+        final N parent1 = Mockito.spy(getBuilderInstance().build());
+        final N parent2 = Mockito.spy(getBuilderInstance().build());
+
+        final B builder = getBuilderInstance();
+        builder.withParent(parent1)
+                .withParent(parent2);
+
+        final N child = builder.build();
+
+        assertEquals(Arrays.asList(parent1, parent2), child.getAllParents());
+
+        Mockito.verify(parent1).addChild(child);
+        Mockito.verify(parent2).addChild(child);
+
+        Mockito.verifyNoMoreInteractions(parent1);
+        Mockito.verifyNoMoreInteractions(parent2);
+    }
+
+    @Test
+    public void testWithConditionalParents() {
+        final String condition1 = "condition1";
+        final String condition2 = "condition2";
+
+        final N parent1 = Mockito.spy(getBuilderInstance().build());
+        final N parent2 = Mockito.spy(getBuilderInstance().build());
+
+        final B builder = getBuilderInstance();
+        builder.withParentWithCondition(parent1, condition1)
+                .withParentWithCondition(parent2, condition2);
+
+        final N child = builder.build();
+
+        final List<Node.NodeWithCondition> nodesWithConditions = child.getParentsWithConditions();
+
+        assertEquals(parent1, nodesWithConditions.get(0).getNode());
+        assertEquals(Condition.actualCondition(condition1), nodesWithConditions.get(0).getCondition());
+
+        assertEquals(parent2, nodesWithConditions.get(1).getNode());
+        assertEquals(Condition.actualCondition(condition2), nodesWithConditions.get(1).getCondition());
+
+        Mockito.verify(parent1).addChildWithCondition(child, condition1);
+        Mockito.verify(parent2).addChildWithCondition(child, condition2);
+
+        Mockito.verifyNoMoreInteractions(parent1);
+        Mockito.verifyNoMoreInteractions(parent2);
+    }
+
+    @Test
+    public void testAddingDuplicateParentBothTimesWithoutConditionThrows() {
+        final N parent = getBuilderInstance().build();
+
+        final B builder = getBuilderInstance();
+
+        builder.withParent(parent);
+
+        expectedException.expect(IllegalArgumentException.class);
+        builder.withParent(parent);
+    }
+
+    @Test
+    public void testAddingDuplicateParentBothTimesWithConditionThrows() {
+        final N parent = getBuilderInstance().build();
+
+        final B builder = getBuilderInstance();
+
+        builder.withParentWithCondition(parent, "condition1");
+
+        expectedException.expect(IllegalArgumentException.class);
+        builder.withParentWithCondition(parent, "condition2");
+    }
+
+    @Test
+    public void testAddingDuplicateParentFirstWithoutConditionThenWithConditionThrows() {
+        final N parent = getBuilderInstance().build();
+
+        final B builder = getBuilderInstance();
+
+        builder.withParent(parent);
+
+        expectedException.expect(IllegalArgumentException.class);
+        builder.withParentWithCondition(parent, "any_condition");
+    }
+
+    @Test
+    public void testAddingDuplicateParentFirstWithConditionThenWithoutConditionThrows() {
+        final N parent = getBuilderInstance().build();
+
+        final B builder = getBuilderInstance();
+
+        builder.withParentWithCondition(parent, "condition");
+
+        expectedException.expect(IllegalArgumentException.class);
+        builder.withParent(parent);
+    }
+
+    @Test
+    public void testWithoutParent() {
+        final N parent1 = Mockito.spy(getBuilderInstance().build());
+        final N parent2 = Mockito.spy(getBuilderInstance().build());
+
+        final B builder = getBuilderInstance();
+        builder.withParent(parent1)
+                .withParent(parent2);
+
+        builder.withoutParent(parent2);
+
+        final N child = builder.build();
+
+        assertEquals(Arrays.asList(parent1), child.getAllParents());
+
+        Mockito.verify(parent1).addChild(child);
+
+        Mockito.verifyNoMoreInteractions(parent1);
+        Mockito.verifyNoMoreInteractions(parent2);
+    }
+
+    @Test
+    public void testAddParentWithAndWithoutCondition() {
+        final Node parent1 = getBuilderInstance().build();
+        final Node parent2 = getBuilderInstance().build();
+
+        final String condition = "condition";
+
+        final Node child = getBuilderInstance()
+                .withParent(parent1)
+                .withParentWithCondition(parent2, condition)
+                .build();
+
+        assertEquals(Arrays.asList(parent1, parent2), child.getAllParents());
+        assertEquals(Arrays.asList(parent1), child.getParentsWithoutConditions());
+
+        final List<Node.NodeWithCondition> parentsWithConditions = child.getParentsWithConditions();
+        assertEquals(parent2, parentsWithConditions.get(0).getNode());
+        assertEquals(condition, parentsWithConditions.get(0).getCondition().getCondition());
+    }
+
+    @Test
+    public void testAddDuplicateDefaultParentTwiceAsDefaultThrows() {
+        final Node parent = getBuilderInstance().build();
+
+        final NodeBuilderBaseImpl<B> builder = getBuilderInstance()
+                .withParentDefaultConditional(parent);
+
+        expectedException.expect(IllegalArgumentException.class);
+        builder.withParentDefaultConditional(parent);
+    }
+
+    @Test
+    public void testAddDuplicateDefaultParentFirstAsNormalConditionalThrows() {
+        final Node parent = getBuilderInstance().build();
+
+        final NodeBuilderBaseImpl<B> builder = getBuilderInstance()
+                .withParentWithCondition(parent, "any_condition");
+
+        expectedException.expect(IllegalArgumentException.class);
+        builder.withParentDefaultConditional(parent);
+    }
+
+    @Test
+    public void testAddMultipleDefaultConditionalChildrenThrows() {
+        final N parent = getBuilderInstance().withName("parent").build();
+
+        getBuilderInstance().withName("defaultChild1").withParentDefaultConditional(parent).build();
+
+        final B defaultChild2Builder = getBuilderInstance().withName("defaultChild2").withParentDefaultConditional(parent);
+
+        expectedException.expect(IllegalStateException.class);
+        defaultChild2Builder.build();
+    }
+
+    @Test
+    public void testWithoutParentWhenConditionExists() {
+        final Node parent1 = getBuilderInstance().build();
+        final Node parent2 = getBuilderInstance().build();
+        final Node parent3 = getBuilderInstance().build();
+        final Node parent4 = getBuilderInstance().build();
+
+        final String condition1 = "condition1";
+        final String condition2 = "condition2";
+
+        final B builder = getBuilderInstance()
+                .withParentWithCondition(parent1, condition1)
+                .withParentWithCondition(parent2, condition2)
+                .withParent(parent3)
+                .withParent(parent4);
+
+        builder.withoutParent(parent2);
+
+        final N child = builder.build();
+
+        assertEquals(Arrays.asList(parent3, parent4, parent1), child.getAllParents());
+        assertEquals(Arrays.asList(parent3, parent4), child.getParentsWithoutConditions());
+
+        final List<Node.NodeWithCondition> parentsWithConditions = child.getParentsWithConditions();
+        assertEquals(parent1, parentsWithConditions.get(0).getNode());
+        assertEquals(condition1, parentsWithConditions.get(0).getCondition().getCondition());
+    }
+
+    @Test
+    public void testAddedAsParentWithCondition() {
+        final N parent = getBuilderInstance().withName("parent").build();
+
+        final String condition1 = "condition1";
+        final String condition2 = "condition2";
+
+        final N child1 = getBuilderInstance().withName("child1").withParentWithCondition(parent, condition1).build();
+        final N child2 = getBuilderInstance().withName("child2").withParentWithCondition(parent, condition2).build();
+        final N defaultChild = getBuilderInstance().withName("defaultChild").withParentDefaultConditional(parent).build();
+
+
+        final List<Node.NodeWithCondition> childrenWithConditions = parent.getChildrenWithConditions();
+
+        assertEquals(3, childrenWithConditions.size());
+
+        assertEquals(child1, childrenWithConditions.get(0).getNode());
+        assertEquals(condition1, childrenWithConditions.get(0).getCondition().getCondition());
+
+        assertEquals(child2, childrenWithConditions.get(1).getNode());
+        assertEquals(condition2, childrenWithConditions.get(1).getCondition().getCondition());
+
+        assertEquals(defaultChild, childrenWithConditions.get(2).getNode());
+        assertTrue(childrenWithConditions.get(2).getCondition().isDefault());
+
+        assertEquals(defaultChild, parent.getDefaultConditionalChild());
+
+        assertEquals(Arrays.asList(child1, child2, defaultChild), parent.getAllChildren());
+    }
+
+    @Test
+    public void testAddedAsParentWithoutCondition() {
+        final N parent = getBuilderInstance().withName("parent").build();
+
+        final N child1 = getBuilderInstance().withName("child1").withParent(parent).build();
+        final N child2 = getBuilderInstance().withName("child2").withParent(parent).build();
+
+
+        final List<Node> childrenWithoutConditions = parent.getChildrenWithoutConditions();
+
+        assertEquals(Arrays.asList(child1, child2), childrenWithoutConditions);
+    }
+
+    @Test
+    public void testAddedAsParentWithConditionWhenChildWithoutConditionExistsThrows() {
+        final N parent = getBuilderInstance().build();
+
+        getBuilderInstance()
+                .withParent(parent)
+                .build();
+        final B child2builder = getBuilderInstance()
+                .withParentWithCondition(parent, "any_condition");
+
+        expectedException.expect(IllegalStateException.class);
+        child2builder.build();
+    }
+
+    @Test
+    public void testAddedAsParentWithoutConditionWhenChildWithConditionExistsThrows() {
+        final N parent = getBuilderInstance().build();
+
+        getBuilderInstance()
+                .withParentWithCondition(parent, "any_condition")
+                .build();
+        final B child2builder = getBuilderInstance()
+                .withParent(parent);
+
+        expectedException.expect(IllegalStateException.class);
+        child2builder.build();
+    }
+
+    @Test
+    public void testClearParents() {
+        final N parent1 = Mockito.spy(getBuilderInstance().build());
+        final N parent2 = Mockito.spy(getBuilderInstance().build());
+        final N parent3 = Mockito.spy(getBuilderInstance().build());
+
+        final B builder = getBuilderInstance();
+        builder.withParent(parent1)
+                .withParent(parent2)
+                .withParentWithCondition(parent3, "any_condition");
+
+        builder.clearParents();
+
+        final N child = builder.build();
+
+        assertEquals(0, child.getAllParents().size());
+
+        Mockito.verifyNoMoreInteractions(parent1);
+        Mockito.verifyNoMoreInteractions(parent2);
+        Mockito.verifyNoMoreInteractions(parent3);
+    }
+
+    @Test
+    public void testNameAdded() {
+        final B builder = getBuilderInstance();
+        builder.withName(NAME);
+
+        final N action = builder.build();
+        assertEquals(NAME, action.getName());
+    }
+
+    @Test
+    public void testNameAddedTwiceThrows() {
+        final B builder = getBuilderInstance();
+        builder.withName(NAME);
+
+        expectedException.expect(IllegalStateException.class);
+        builder.withName("any_name");
+    }
+
+    @Test
+    public void testCredentialAddedAndRemoved() {
+        final Credential first = new Credential("first name", "first type", ImmutableList.of());
+        final Credential second = new Credential("second name", "second type", ImmutableList.of());
+
+        final B builder = getBuilderInstance();
+        builder.withCredential(first);
+        builder.withCredential(second);
+
+        assertEquals(2, builder.build().getCredentials().size());
+        assertEquals(first, builder.build().getCredentials().get(0));
+        assertEquals(second, builder.build().getCredentials().get(1));
+
+        builder.withoutCredential(first);
+
+        assertEquals(1, builder.build().getCredentials().size());
+        assertEquals(second, builder.build().getCredentials().get(0));
+
+        builder.clearCredentials();
+        assertEquals(0, builder.build().getCredentials().size());
+    }
+
+    @Test
+    public void testRetryAttributesAddedTwiceThrows() {
+        final B builder = getBuilderInstance();
+
+        builder.withRetryInterval(1);
+        builder.withRetryMax(3);
+        builder.withRetryPolicy("retry-policy");
+
+        assertEquals(Integer.valueOf(1), builder.build().getRetryInterval());
+        assertEquals(Integer.valueOf(3), builder.build().getRetryMax());
+        assertEquals("retry-policy", builder.build().getRetryPolicy());
+
+        expectedException.expect(IllegalStateException.class);
+        builder.withRetryInterval(null);
+
+        expectedException.expect(IllegalStateException.class);
+        builder.withRetryMax(null);
+
+        expectedException.expect(IllegalStateException.class);
+        builder.withRetryPolicy(null);
+    }
+
+    @Test
+    public void testFromExistingNode() {
+        final ErrorHandler errorHandler = ErrorHandler.buildAsErrorHandler(
+                MapReduceActionBuilder.create().withName("error-handler"));
+
+        final Node parent1 = MapReduceActionBuilder.create().withName("parent1").build();
+        final Node parent2 = MapReduceActionBuilder.create().withName("parent2").build();
+        final Node parent3 = MapReduceActionBuilder.create().withName("parent3").build();
+        final Node parent4 = MapReduceActionBuilder.create().withName("parent4").build();
+
+        final String condition = "condition";
+
+        final B builder = getBuilderInstance();
+
+        builder.withName(NAME)
+                .withParent(parent1)
+                .withParent(parent2)
+                .withParentWithCondition(parent4, condition)
+                .withErrorHandler(errorHandler);
+
+        final N node = builder.build();
+
+        final B fromExistingBuilder = getBuilderInstance(node);
+
+        final String newName = "fromExisting_" + NAME;
+        fromExistingBuilder.withName(newName)
+                .withoutParent(parent2)
+                .withParent(parent3);
+
+        final Node modifiedNode = fromExistingBuilder.build();
+
+        assertEquals(newName, modifiedNode.getName());
+        assertEquals(Arrays.asList(parent1, parent3, parent4), modifiedNode.getAllParents());
+        assertEquals(Arrays.asList(parent1, parent3), modifiedNode.getParentsWithoutConditions());
+        assertEquals(errorHandler, modifiedNode.getErrorHandler());
+
+        final List<Node.NodeWithCondition> parentsWithConditions = modifiedNode.getParentsWithConditions();
+        assertEquals(1, parentsWithConditions.size());
+        assertEquals(parent4, parentsWithConditions.get(0).getNode());
+        assertEquals(condition, parentsWithConditions.get(0).getCondition().getCondition());
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestPigActionBuilder.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestPigActionBuilder.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestPigActionBuilder.java
new file mode 100644
index 0000000..d8f0918
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestPigActionBuilder.java
@@ -0,0 +1,218 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestPigActionBuilder extends TestNodeBuilderBaseImpl<PigAction, PigActionBuilder> {
+    private static final String NAME = "pig-name";
+    private static final String NAME_NODE = "${nameNode}";
+    private static final String EXAMPLE_DIR = "/path/to/directory";
+    private static final String[] ARGS = {"arg1", "arg2", "arg3"};
+    private static final String MAPRED_JOB_QUEUE_NAME = "mapred.job.queue.name";
+    private static final String DEFAULT = "default";
+    private static final String RESOURCE_MANAGER = "${resourceManager}";
+    private static final String PATH_TO_DELETE = "/path/to/delete";
+    private static final String PATH_TO_MKDIR = "/path/to/mkdir";
+
+    @Override
+    protected PigActionBuilder getBuilderInstance() {
+        return PigActionBuilder.create();
+    }
+
+    @Override
+    protected PigActionBuilder getBuilderInstance(final PigAction action) {
+        return PigActionBuilder.createFromExistingAction(action);
+    }
+
+    @Test
+    public void testResourceManagerAdded() {
+        final PigActionBuilder builder = getBuilderInstance();
+        builder.withResourceManager(RESOURCE_MANAGER);
+
+        final PigAction action = builder.build();
+        assertEquals(RESOURCE_MANAGER, action.getResourceManager());
+    }
+
+    @Test
+    public void testNameNodeAdded() {
+        final PigActionBuilder builder = getBuilderInstance();
+        builder.withNameNode(NAME_NODE);
+
+        final PigAction action = builder.build();
+        assertEquals(NAME_NODE, action.getNameNode());
+    }
+
+    @Test
+    public void testPrepareAdded() {
+        final PigActionBuilder builder = getBuilderInstance();
+        builder.withPrepare(new PrepareBuilder().withDelete(EXAMPLE_DIR).build());
+
+        final PigAction action = builder.build();
+        assertEquals(EXAMPLE_DIR, action.getPrepare().getDeletes().get(0).getPath());
+    }
+
+    @Test
+    public void testSameConfigPropertyAddedTwiceThrows() {
+        final PigActionBuilder builder = getBuilderInstance();
+        builder.withConfigProperty(MAPRED_JOB_QUEUE_NAME, DEFAULT);
+
+        expectedException.expect(IllegalStateException.class);
+        builder.withConfigProperty(MAPRED_JOB_QUEUE_NAME, DEFAULT);
+    }
+
+    @Test
+    public void testSeveralArgsAdded() {
+        final PigActionBuilder builder = getBuilderInstance();
+
+        for (final String arg : ARGS) {
+            builder.withArg(arg);
+        }
+
+        final PigAction action = builder.build();
+
+        final List<String> argList = action.getArgs();
+        assertEquals(ARGS.length, argList.size());
+
+        for (int i = 0; i < ARGS.length; ++i) {
+            assertEquals(ARGS[i], argList.get(i));
+        }
+    }
+
+    @Test
+    public void testRemoveArgs() {
+        final PigActionBuilder builder = getBuilderInstance();
+
+        for (final String file : ARGS) {
+            builder.withArg(file);
+        }
+
+        builder.withoutArg(ARGS[0]);
+
+        final PigAction action = builder.build();
+
+        final List<String> argList = action.getArgs();
+        final String[] remainingArgs = Arrays.copyOfRange(ARGS, 1, ARGS.length);
+        assertEquals(remainingArgs.length, argList.size());
+
+        for (int i = 0; i < remainingArgs.length; ++i) {
+            assertEquals(remainingArgs[i], argList.get(i));
+        }
+    }
+
+    @Test
+    public void testClearArgs() {
+        final PigActionBuilder builder = getBuilderInstance();
+
+        for (final String file : ARGS) {
+            builder.withArg(file);
+        }
+
+        builder.clearArgs();
+
+        final PigAction action = builder.build();
+
+        final List<String> argList = action.getArgs();
+        assertEquals(0, argList.size());
+    }
+
+    @Test
+    public void testFromExistingPigAction() {
+        final PigActionBuilder builder = getBuilderInstance();
+
+        builder.withName(NAME)
+                .withResourceManager(RESOURCE_MANAGER)
+                .withNameNode(NAME_NODE)
+                .withConfigProperty(MAPRED_JOB_QUEUE_NAME, DEFAULT)
+                .withPrepare(new PrepareBuilder()
+                        .withDelete(PATH_TO_DELETE)
+                        .withMkdir(PATH_TO_MKDIR)
+                        .build())
+                .withLauncher(new LauncherBuilder()
+                        .withMemoryMb(1024L)
+                        .withVCores(2L)
+                        .withQueue(DEFAULT)
+                        .withSharelib(DEFAULT)
+                        .withViewAcl(DEFAULT)
+                        .withModifyAcl(DEFAULT)
+                        .build())
+                .withArg(ARGS[0])
+                .withArg(ARGS[1])
+                .withArchive(DEFAULT)
+                .withFile(DEFAULT);
+
+        final PigAction action = builder.build();
+
+        final PigActionBuilder fromExistingBuilder = getBuilderInstance(action);
+
+        final String newName = "fromExisting_" + NAME;
+        fromExistingBuilder.withName(newName)
+                .withoutArg(ARGS[1])
+                .withArg(ARGS[2]);
+
+        final PigAction modifiedAction = fromExistingBuilder.build();
+
+        assertEquals(newName, modifiedAction.getName());
+        assertEquals(action.getNameNode(), modifiedAction.getNameNode());
+
+        final Map<String, String> expectedConfiguration = new LinkedHashMap<>();
+        expectedConfiguration.put(MAPRED_JOB_QUEUE_NAME, DEFAULT);
+        assertEquals(expectedConfiguration, modifiedAction.getConfiguration());
+
+        assertEquals(Arrays.asList(ARGS[0], ARGS[2]), modifiedAction.getArgs());
+
+        assertEquals(PATH_TO_DELETE, modifiedAction.getPrepare().getDeletes().get(0).getPath());
+        assertEquals(PATH_TO_MKDIR, modifiedAction.getPrepare().getMkdirs().get(0).getPath());
+
+        assertEquals(1024L, modifiedAction.getLauncher().getMemoryMb());
+        assertEquals(2L, modifiedAction.getLauncher().getVCores());
+        assertEquals(DEFAULT, modifiedAction.getLauncher().getQueue());
+        assertEquals(DEFAULT, modifiedAction.getLauncher().getSharelib());
+        assertEquals(DEFAULT, modifiedAction.getLauncher().getViewAcl());
+        assertEquals(DEFAULT, modifiedAction.getLauncher().getModifyAcl());
+
+        assertEquals(action.getScript(), modifiedAction.getScript());
+    }
+
+    @Test
+    public void testFromOtherAction() {
+        final ShellAction parent = ShellActionBuilder.create()
+                .withName("parent")
+                .build();
+
+        final ShellAction otherAction = ShellActionBuilder.createFromExistingAction(parent)
+                .withName("shell")
+                .withParent(parent)
+                .build();
+
+        final PigAction fromOtherAction = PigActionBuilder.createFromExistingAction(otherAction)
+                .withName("pig")
+                .build();
+
+        assertEquals("pig", fromOtherAction.getName());
+        assertEquals(parent, fromOtherAction.getParentsWithoutConditions().get(0));
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestPipesBuilder.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestPipesBuilder.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestPipesBuilder.java
new file mode 100644
index 0000000..7b0ce07
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestPipesBuilder.java
@@ -0,0 +1,168 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestPipesBuilder {
+    @Rule
+    public final ExpectedException expectedException = ExpectedException.none();
+
+    @Test
+    public void testWithMap() {
+        final String map = "map";
+
+        final PipesBuilder builder = new PipesBuilder();
+        builder.withMap(map);
+
+        final Pipes pipes = builder.build();
+        assertEquals(map, pipes.getMap());
+    }
+
+    @Test
+    public void testMapCalledTwiceThrows() {
+        final String map1 = "map1";
+        final String map2 = "map2";
+
+        final PipesBuilder builder = new PipesBuilder();
+        builder.withMap(map1);
+
+        expectedException.expect(IllegalStateException.class);
+        builder.withMap(map2);
+    }
+
+    @Test
+    public void testWithReduce() {
+        final String reduce = "reduce";
+
+        final PipesBuilder builder = new PipesBuilder();
+        builder.withReduce(reduce);
+
+        final Pipes pipes = builder.build();
+        assertEquals(reduce, pipes.getReduce());
+    }
+
+    @Test
+    public void testWithReduceCalledTwiceThrows() {
+        final String reduce1 = "reduce1";
+        final String reduce2= "reduce2";
+
+        final PipesBuilder builder = new PipesBuilder();
+        builder.withReduce(reduce1);
+
+        expectedException.expect(IllegalStateException.class);
+        builder.withReduce(reduce2);
+    }
+
+    @Test
+    public void testWithInputformat() {
+        final String inputformat = "inputformat";
+
+        final PipesBuilder builder = new PipesBuilder();
+        builder.withInputformat(inputformat);
+
+        final Pipes pipes = builder.build();
+        assertEquals(inputformat, pipes.getInputformat());
+    }
+
+    @Test
+    public void testWithInputformatCalledTwiceThrows() {
+        final String inputformat1 = "inputformat1";
+        final String inputformat2 = "inputformat2";
+
+        final PipesBuilder builder = new PipesBuilder();
+        builder.withInputformat(inputformat1);
+
+        expectedException.expect(IllegalStateException.class);
+        builder.withInputformat(inputformat2);
+    }
+
+    @Test
+    public void testWithPartitioner() {
+        final String partitioner = "partitioner";
+
+        final PipesBuilder builder = new PipesBuilder();
+        builder.withPartitioner(partitioner);
+
+        final Pipes pipes = builder.build();
+        assertEquals(partitioner, pipes.getPartitioner());
+    }
+
+    @Test
+    public void testWithPartitionerCalledTwiceThrows() {
+        final String partitioner1 = "partitioner1";
+        final String partitioner2 = "partitioner2";
+
+        final PipesBuilder builder = new PipesBuilder();
+        builder.withPartitioner(partitioner1);
+
+        expectedException.expect(IllegalStateException.class);
+        builder.withPartitioner(partitioner2);
+    }
+
+    @Test
+    public void testWithWriter() {
+        final String writer = "writer";
+
+        final PipesBuilder builder = new PipesBuilder();
+        builder.withWriter(writer);
+
+        final Pipes pipes = builder.build();
+        assertEquals(writer, pipes.getWriter());
+    }
+
+    @Test
+    public void testWithWriterCalledTwiceThrows() {
+        final String writer1 = "writer1";
+        final String writer2 = "writer2";
+
+        final PipesBuilder builder = new PipesBuilder();
+        builder.withWriter(writer1);
+
+        expectedException.expect(IllegalStateException.class);
+        builder.withWriter(writer2);
+    }
+
+    @Test
+    public void testWithProgram() {
+        final String program = "program";
+
+        final PipesBuilder builder = new PipesBuilder();
+        builder.withProgram(program);
+
+        final Pipes pipes = builder.build();
+        assertEquals(program, pipes.getProgram());
+    }
+
+    @Test
+    public void testWithProgramCalledTwiceThrows() {
+        final String program1 = "program1";
+        final String program2 = "program2";
+
+        final PipesBuilder builder = new PipesBuilder();
+        builder.withProgram(program1);
+
+        expectedException.expect(IllegalStateException.class);
+        builder.withProgram(program2);
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestPrepareBuilder.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestPrepareBuilder.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestPrepareBuilder.java
new file mode 100644
index 0000000..37a6dbd
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestPrepareBuilder.java
@@ -0,0 +1,105 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestPrepareBuilder {
+    private static final String[] TEST_FOLDER_NAMES = {
+            "/user/testpath/testdir1",
+            "/user/testpath/testdir2",
+            "/user/testpath/testdir3"
+    };
+
+    private PrepareBuilder pb;
+
+    @Before
+    public void setUp() {
+        pb = new PrepareBuilder();
+    }
+
+    @Test
+    public void testOneDeleteIsAddedWithSkipTrashTrue() {
+        pb.withDelete(TEST_FOLDER_NAMES[0], true);
+
+        final Prepare prepare = pb.build();
+
+        assertEquals(1, prepare.getDeletes().size());
+
+        final Delete delete = prepare.getDeletes().get(0);
+        assertEquals(TEST_FOLDER_NAMES[0], delete.getPath());
+        assertEquals(true, delete.getSkipTrash());
+
+        assertEquals(0, prepare.getMkdirs().size());
+    }
+
+    @Test
+    public void testSeveralDeletesAreAddedWithSkipTrashNotSpecified() {
+        for (final String testDir : TEST_FOLDER_NAMES) {
+            pb.withDelete(testDir);
+        }
+
+        final Prepare prepare = pb.build();
+
+        assertEquals(TEST_FOLDER_NAMES.length, prepare.getDeletes().size());
+
+        for (int i = 0; i < TEST_FOLDER_NAMES.length; ++i) {
+            final Delete delete = prepare.getDeletes().get(i);
+            assertEquals(TEST_FOLDER_NAMES[i], delete.getPath());
+            assertEquals(null, delete.getSkipTrash());
+        }
+
+        assertEquals(0, prepare.getMkdirs().size());
+    }
+
+    @Test
+    public void testOneMkdirIsAdded() {
+        pb.withMkdir(TEST_FOLDER_NAMES[0]);
+
+        final Prepare prepare = pb.build();
+
+        assertEquals(1, prepare.getMkdirs().size());
+
+        final Mkdir mkdir = prepare.getMkdirs().get(0);
+        assertEquals(TEST_FOLDER_NAMES[0], mkdir.getPath());
+
+        assertEquals(0, prepare.getDeletes().size());
+    }
+
+    @Test
+    public void testSeveralMkdirsAreAdded() {
+        for (final String testDir : TEST_FOLDER_NAMES) {
+            pb.withMkdir(testDir);
+        }
+
+        final Prepare prepare = pb.build();
+
+        assertEquals(TEST_FOLDER_NAMES.length, prepare.getMkdirs().size());
+
+        for (int i = 0; i < TEST_FOLDER_NAMES.length; ++i) {
+            final Mkdir mkdir = prepare.getMkdirs().get(i);
+            assertEquals(TEST_FOLDER_NAMES[i], mkdir.getPath());
+        }
+
+        assertEquals(0, prepare.getDeletes().size());
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestShellActionBuilder.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestShellActionBuilder.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestShellActionBuilder.java
new file mode 100644
index 0000000..7ffec51
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestShellActionBuilder.java
@@ -0,0 +1,223 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestShellActionBuilder extends TestNodeBuilderBaseImpl<ShellAction, ShellActionBuilder> {
+    private static final String NAME = "shell-name";
+    private static final String NAME_NODE = "${nameNode}";
+    private static final String EXAMPLE_DIR = "/path/to/directory";
+    private static final String[] ARGS = {"arg1", "arg2", "arg3"};
+    private static final String MAPRED_JOB_QUEUE_NAME = "mapred.job.queue.name";
+    private static final String DEFAULT = "default";
+    private static final String RESOURCE_MANAGER = "${resourceManager}";
+    private static final String PATH_TO_DELETE = "/path/to/delete";
+    private static final String PATH_TO_MKDIR = "/path/to/mkdir";
+
+    @Override
+    protected ShellActionBuilder getBuilderInstance() {
+        return ShellActionBuilder.create();
+    }
+
+    @Override
+    protected ShellActionBuilder getBuilderInstance(final ShellAction action) {
+        return ShellActionBuilder.createFromExistingAction(action);
+    }
+
+    @Test
+    public void testResourceManagerAdded() {
+        final ShellActionBuilder builder = getBuilderInstance();
+        builder.withResourceManager(RESOURCE_MANAGER);
+
+        final ShellAction action = builder.build();
+        assertEquals(RESOURCE_MANAGER, action.getResourceManager());
+    }
+
+    @Test
+    public void testNameNodeAdded() {
+        final ShellActionBuilder builder = getBuilderInstance();
+        builder.withNameNode(NAME_NODE);
+
+        final ShellAction action = builder.build();
+        assertEquals(NAME_NODE, action.getNameNode());
+    }
+
+    @Test
+    public void testPrepareAdded() {
+        final ShellActionBuilder builder = getBuilderInstance();
+        builder.withPrepare(new PrepareBuilder().withDelete(EXAMPLE_DIR).build());
+
+        final ShellAction action = builder.build();
+        assertEquals(EXAMPLE_DIR, action.getPrepare().getDeletes().get(0).getPath());
+    }
+
+    @Test
+    public void testSameConfigPropertyAddedTwiceThrows() {
+        final ShellActionBuilder builder = getBuilderInstance();
+        builder.withConfigProperty(MAPRED_JOB_QUEUE_NAME, DEFAULT);
+
+        expectedException.expect(IllegalStateException.class);
+        builder.withConfigProperty(MAPRED_JOB_QUEUE_NAME, DEFAULT);
+    }
+
+    @Test
+    public void testSeveralArgumentsAdded() {
+        final ShellActionBuilder builder = getBuilderInstance();
+
+        for (final String arg : ARGS) {
+            builder.withArgument(arg);
+        }
+
+        final ShellAction action = builder.build();
+
+        final List<String> argList = action.getArguments();
+        assertEquals(ARGS.length, argList.size());
+
+        for (int i = 0; i < ARGS.length; ++i) {
+            assertEquals(ARGS[i], argList.get(i));
+        }
+    }
+
+    @Test
+    public void testRemoveArguments() {
+        final ShellActionBuilder builder = getBuilderInstance();
+
+        for (final String file : ARGS) {
+            builder.withArgument(file);
+        }
+
+        builder.withoutArgument(ARGS[0]);
+
+        final ShellAction action = builder.build();
+
+        final List<String> argList = action.getArguments();
+        final String[] remainingArgs = Arrays.copyOfRange(ARGS, 1, ARGS.length);
+        assertEquals(remainingArgs.length, argList.size());
+
+        for (int i = 0; i < remainingArgs.length; ++i) {
+            assertEquals(remainingArgs[i], argList.get(i));
+        }
+    }
+
+    @Test
+    public void testClearArguments() {
+        final ShellActionBuilder builder = getBuilderInstance();
+
+        for (final String file : ARGS) {
+            builder.withArgument(file);
+        }
+
+        builder.clearArguments();
+
+        final ShellAction action = builder.build();
+
+        final List<String> argList = action.getArguments();
+        assertEquals(0, argList.size());
+    }
+
+    @Test
+    public void testFromExistingShellAction() {
+        final ShellActionBuilder builder = getBuilderInstance();
+
+        builder.withName(NAME)
+                .withResourceManager(RESOURCE_MANAGER)
+                .withNameNode(NAME_NODE)
+                .withConfigProperty(MAPRED_JOB_QUEUE_NAME, DEFAULT)
+                .withPrepare(new PrepareBuilder()
+                        .withDelete(PATH_TO_DELETE)
+                        .withMkdir(PATH_TO_MKDIR)
+                        .build())
+                .withLauncher(new LauncherBuilder()
+                        .withMemoryMb(1024L)
+                        .withVCores(2L)
+                        .withQueue(DEFAULT)
+                        .withSharelib(DEFAULT)
+                        .withViewAcl(DEFAULT)
+                        .withModifyAcl(DEFAULT)
+                        .build())
+                .withExecutable(DEFAULT)
+                .withEnvironmentVariable(DEFAULT)
+                .withArgument(ARGS[0])
+                .withArgument(ARGS[1])
+                .withArchive(DEFAULT)
+                .withFile(DEFAULT)
+                .withCaptureOutput(true);
+
+        final ShellAction action = builder.build();
+
+        final ShellActionBuilder fromExistingBuilder = getBuilderInstance(action);
+
+        final String newName = "fromExisting_" + NAME;
+        fromExistingBuilder.withName(newName)
+                .withoutArgument(ARGS[1])
+                .withArgument(ARGS[2]);
+
+        final ShellAction modifiedAction = fromExistingBuilder.build();
+
+        assertEquals(newName, modifiedAction.getName());
+        assertEquals(action.getNameNode(), modifiedAction.getNameNode());
+
+        final Map<String, String> expectedConfiguration = new LinkedHashMap<>();
+        expectedConfiguration.put(MAPRED_JOB_QUEUE_NAME, DEFAULT);
+        assertEquals(expectedConfiguration, modifiedAction.getConfiguration());
+
+        assertEquals(Arrays.asList(ARGS[0], ARGS[2]), modifiedAction.getArguments());
+
+        assertEquals(PATH_TO_DELETE, modifiedAction.getPrepare().getDeletes().get(0).getPath());
+        assertEquals(PATH_TO_MKDIR, modifiedAction.getPrepare().getMkdirs().get(0).getPath());
+
+        assertEquals(1024L, modifiedAction.getLauncher().getMemoryMb());
+        assertEquals(2L, modifiedAction.getLauncher().getVCores());
+        assertEquals(DEFAULT, modifiedAction.getLauncher().getQueue());
+        assertEquals(DEFAULT, modifiedAction.getLauncher().getSharelib());
+        assertEquals(DEFAULT, modifiedAction.getLauncher().getViewAcl());
+        assertEquals(DEFAULT, modifiedAction.getLauncher().getModifyAcl());
+
+        assertEquals(action.getExecutable(), modifiedAction.getExecutable());
+        assertEquals(action.getEnvironmentVariables().get(0), modifiedAction.getEnvironmentVariables().get(0));
+        assertEquals(action.isCaptureOutput(), modifiedAction.isCaptureOutput());
+    }
+
+    @Test
+    public void testFromOtherAction() {
+        final PigAction parent = PigActionBuilder.create()
+                .withName("parent")
+                .build();
+
+        final PigAction otherAction = PigActionBuilder.createFromExistingAction(parent)
+                .withName("pig")
+                .withParent(parent)
+                .build();
+
+        final ShellAction fromOtherAction = ShellActionBuilder.createFromExistingAction(otherAction)
+                .withName("shell")
+                .build();
+
+        assertEquals("shell", fromOtherAction.getName());
+        assertEquals(parent, fromOtherAction.getParentsWithoutConditions().get(0));
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestSparkActionBuilder.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestSparkActionBuilder.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestSparkActionBuilder.java
new file mode 100644
index 0000000..0572e94
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestSparkActionBuilder.java
@@ -0,0 +1,229 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestSparkActionBuilder extends TestNodeBuilderBaseImpl<SparkAction, SparkActionBuilder> {
+    private static final String NAME = "spark-name";
+    private static final String NAME_NODE = "${nameNode}";
+    private static final String EXAMPLE_DIR = "/path/to/directory";
+    private static final String[] ARGS = {"arg1", "arg2", "arg3"};
+    private static final String MAPRED_JOB_QUEUE_NAME = "mapred.job.queue.name";
+    private static final String DEFAULT = "default";
+    private static final String RESOURCE_MANAGER = "${resourceManager}";
+    private static final String PATH_TO_DELETE = "/path/to/delete";
+    private static final String PATH_TO_MKDIR = "/path/to/mkdir";
+
+    @Override
+    protected SparkActionBuilder getBuilderInstance() {
+        return SparkActionBuilder.create();
+    }
+
+    @Override
+    protected SparkActionBuilder getBuilderInstance(final SparkAction action) {
+        return SparkActionBuilder.createFromExistingAction(action);
+    }
+
+    @Test
+    public void testResourceManagerAdded() {
+        final SparkActionBuilder builder = getBuilderInstance();
+        builder.withResourceManager(RESOURCE_MANAGER);
+
+        final SparkAction action = builder.build();
+        assertEquals(RESOURCE_MANAGER, action.getResourceManager());
+    }
+
+    @Test
+    public void testNameNodeAdded() {
+        final SparkActionBuilder builder = getBuilderInstance();
+        builder.withNameNode(NAME_NODE);
+
+        final SparkAction action = builder.build();
+        assertEquals(NAME_NODE, action.getNameNode());
+    }
+
+    @Test
+    public void testPrepareAdded() {
+        final SparkActionBuilder builder = getBuilderInstance();
+        builder.withPrepare(new PrepareBuilder().withDelete(EXAMPLE_DIR).build());
+
+        final SparkAction action = builder.build();
+        assertEquals(EXAMPLE_DIR, action.getPrepare().getDeletes().get(0).getPath());
+    }
+
+    @Test
+    public void testSameConfigPropertyAddedTwiceThrows() {
+        final SparkActionBuilder builder = getBuilderInstance();
+        builder.withConfigProperty(MAPRED_JOB_QUEUE_NAME, DEFAULT);
+
+        expectedException.expect(IllegalStateException.class);
+        builder.withConfigProperty(MAPRED_JOB_QUEUE_NAME, DEFAULT);
+    }
+
+    @Test
+    public void testSeveralArgsAdded() {
+        final SparkActionBuilder builder = getBuilderInstance();
+
+        for (final String arg : ARGS) {
+            builder.withArg(arg);
+        }
+
+        final SparkAction action = builder.build();
+
+        final List<String> argList = action.getArgs();
+        assertEquals(ARGS.length, argList.size());
+
+        for (int i = 0; i < ARGS.length; ++i) {
+            assertEquals(ARGS[i], argList.get(i));
+        }
+    }
+
+    @Test
+    public void testRemoveArgs() {
+        final SparkActionBuilder builder = getBuilderInstance();
+
+        for (final String file : ARGS) {
+            builder.withArg(file);
+        }
+
+        builder.withoutArg(ARGS[0]);
+
+        final SparkAction action = builder.build();
+
+        final List<String> argList = action.getArgs();
+        final String[] remainingArgs = Arrays.copyOfRange(ARGS, 1, ARGS.length);
+        assertEquals(remainingArgs.length, argList.size());
+
+        for (int i = 0; i < remainingArgs.length; ++i) {
+            assertEquals(remainingArgs[i], argList.get(i));
+        }
+    }
+
+    @Test
+    public void testClearArgs() {
+        final SparkActionBuilder builder = getBuilderInstance();
+
+        for (final String file : ARGS) {
+            builder.withArg(file);
+        }
+
+        builder.clearArgs();
+
+        final SparkAction action = builder.build();
+
+        final List<String> argList = action.getArgs();
+        assertEquals(0, argList.size());
+    }
+
+    @Test
+    public void testFromExistingSparkAction() {
+        final SparkActionBuilder builder = getBuilderInstance();
+
+        builder.withName(NAME)
+                .withResourceManager(RESOURCE_MANAGER)
+                .withNameNode(NAME_NODE)
+                .withConfigProperty(MAPRED_JOB_QUEUE_NAME, DEFAULT)
+                .withPrepare(new PrepareBuilder()
+                        .withDelete(PATH_TO_DELETE)
+                        .withMkdir(PATH_TO_MKDIR)
+                        .build())
+                .withLauncher(new LauncherBuilder()
+                        .withMemoryMb(1024L)
+                        .withVCores(2L)
+                        .withQueue(DEFAULT)
+                        .withSharelib(DEFAULT)
+                        .withViewAcl(DEFAULT)
+                        .withModifyAcl(DEFAULT)
+                        .build())
+                .withArg(ARGS[0])
+                .withArg(ARGS[1])
+                .withMaster(DEFAULT)
+                .withMode(DEFAULT)
+                .withActionName(DEFAULT)
+                .withActionClass(DEFAULT)
+                .withJar(DEFAULT)
+                .withSparkOpts(DEFAULT)
+                .withArchive(DEFAULT)
+                .withFile(DEFAULT);
+
+        final SparkAction action = builder.build();
+
+        final SparkActionBuilder fromExistingBuilder = getBuilderInstance(action);
+
+        final String newName = "fromExisting_" + NAME;
+        fromExistingBuilder.withName(newName)
+                .withoutArg(ARGS[1])
+                .withArg(ARGS[2]);
+
+        final SparkAction modifiedAction = fromExistingBuilder.build();
+
+        assertEquals(newName, modifiedAction.getName());
+        assertEquals(action.getNameNode(), modifiedAction.getNameNode());
+
+        final Map<String, String> expectedConfiguration = new LinkedHashMap<>();
+        expectedConfiguration.put(MAPRED_JOB_QUEUE_NAME, DEFAULT);
+        assertEquals(expectedConfiguration, modifiedAction.getConfiguration());
+
+        assertEquals(Arrays.asList(ARGS[0], ARGS[2]), modifiedAction.getArgs());
+
+        assertEquals(PATH_TO_DELETE, modifiedAction.getPrepare().getDeletes().get(0).getPath());
+        assertEquals(PATH_TO_MKDIR, modifiedAction.getPrepare().getMkdirs().get(0).getPath());
+
+        assertEquals(1024L, modifiedAction.getLauncher().getMemoryMb());
+        assertEquals(2L, modifiedAction.getLauncher().getVCores());
+        assertEquals(DEFAULT, modifiedAction.getLauncher().getQueue());
+        assertEquals(DEFAULT, modifiedAction.getLauncher().getSharelib());
+        assertEquals(DEFAULT, modifiedAction.getLauncher().getViewAcl());
+        assertEquals(DEFAULT, modifiedAction.getLauncher().getModifyAcl());
+
+        assertEquals(action.getMaster(), modifiedAction.getMaster());
+        assertEquals(action.getMode(), modifiedAction.getMode());
+        assertEquals(action.getActionName(), modifiedAction.getActionName());
+        assertEquals(action.getActionClass(), modifiedAction.getActionClass());
+        assertEquals(action.getJar(), modifiedAction.getJar());
+        assertEquals(action.getSparkOpts(), modifiedAction.getSparkOpts());
+    }
+
+    @Test
+    public void testFromOtherAction() {
+        final ShellAction parent = ShellActionBuilder.create()
+                .withName("parent")
+                .build();
+
+        final ShellAction otherAction = ShellActionBuilder.createFromExistingAction(parent)
+                .withName("shell")
+                .withParent(parent)
+                .build();
+
+        final SparkAction fromOtherAction = SparkActionBuilder.createFromExistingAction(otherAction)
+                .withName("spark")
+                .build();
+
+        assertEquals("spark", fromOtherAction.getName());
+        assertEquals(parent, fromOtherAction.getParentsWithoutConditions().get(0));
+    }
+}
\ No newline at end of file


[04/16] oozie git commit: OOZIE-2339 [fluent-job] Minimum Viable Fluent Job API (daniel.becker, andras.piros via rkanter, gezapeti, pbacsko)

Posted by an...@apache.org.
http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/dag/TestGraph.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/dag/TestGraph.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/dag/TestGraph.java
new file mode 100644
index 0000000..ff5df72
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/dag/TestGraph.java
@@ -0,0 +1,941 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.dag;
+
+import org.apache.oozie.fluentjob.api.Condition;
+import org.apache.oozie.fluentjob.api.NodesToPng;
+import org.apache.oozie.fluentjob.api.action.MapReduceActionBuilder;
+import org.apache.oozie.fluentjob.api.action.Node;
+import org.apache.oozie.fluentjob.api.workflow.Workflow;
+import org.apache.oozie.fluentjob.api.workflow.WorkflowBuilder;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+public class TestGraph {
+    @Rule
+    public final ExpectedException expectedException = ExpectedException.none();
+
+    @Rule
+    public final NodesToPng nodesToPng = new NodesToPng();
+
+    @Test
+    public void testNameIsCorrect() {
+        final Node a = MapReduceActionBuilder.create().withName("A").build();
+
+        MapReduceActionBuilder.create().withName("B").withParent(a).build();
+        MapReduceActionBuilder.create().withName("C").withParent(a).build();
+
+        final String name = "workflow-name";
+        final Workflow workflow = new WorkflowBuilder().withName(name).withDagContainingNode(a).build();
+
+        final Graph graph = new Graph(workflow);
+        assertEquals(name, graph.getName());
+
+//        nodesToPng.withWorkflow(workflow);
+//        nodesToPng.withGraph(graph);
+    }
+
+    @Test
+    public void testDuplicateNamesThrow() {
+        final Node a = MapReduceActionBuilder.create().withName("A").build();
+        MapReduceActionBuilder.create().withName("A").withParent(a).build();
+
+        // The exception will be thrown by the Workflow object,
+        // but if it breaks there, we want to catch duplicates here, too.
+        expectedException.expect(IllegalArgumentException.class);
+        final Workflow workflow = new WorkflowBuilder().withDagContainingNode(a).build();
+
+        new Graph(workflow);
+    }
+
+    @Test
+    public void testWorkflowWithoutJoin() {
+        final Node a = MapReduceActionBuilder.create().withName("A").build();
+
+        MapReduceActionBuilder.create().withName("B").withParent(a).build();
+        MapReduceActionBuilder.create().withName("C").withParent(a).build();
+
+        final Workflow workflow = new WorkflowBuilder().withName("without-join").withDagContainingNode(a).build();
+        final Graph graph = new Graph(workflow);
+
+        checkDependencies(workflow.getNodes(), graph);
+
+//        nodesToPng.withWorkflow(workflow);
+//        nodesToPng.withGraph(graph);
+    }
+
+    @Test
+    public void testWorkflowWithTrivialJoin() {
+        final Node a = MapReduceActionBuilder.create().withName("A").build();
+
+        final Node b = MapReduceActionBuilder.create().withName("B").withParent(a).build();
+        final Node c = MapReduceActionBuilder.create().withName("C").withParent(a).build();
+        MapReduceActionBuilder.create().withName("D").withParent(b).withParent(c).build();
+
+        final Workflow workflow = new WorkflowBuilder().withName("trivial-join").withDagContainingNode(a).build();
+        final Graph graph = new Graph(workflow);
+
+        checkDependencies(workflow.getNodes(), graph);
+
+//        nodesToPng.withWorkflow(workflow);
+//        nodesToPng.withGraph(graph);
+    }
+
+    @Test
+    public void testWorkflowNewDependenciesNeeded() {
+        final Node a = MapReduceActionBuilder.create().withName("A").build();
+
+        final Node b = MapReduceActionBuilder.create().withName("B").withParent(a).build();
+        final Node c = MapReduceActionBuilder.create().withName("C").withParent(a).build();
+
+        final Node d = MapReduceActionBuilder.create().withName("D").withParent(b).withParent(c).build();
+        final Node e = MapReduceActionBuilder.create().withName("E").withParent(c).build();
+
+        MapReduceActionBuilder.create().withName("F").withParent(d).withParent(e).build();
+
+        final Workflow workflow = new WorkflowBuilder().withName("new-dependencies-needed").withDagContainingNode(a).build();
+        final Graph graph = new Graph(workflow);
+
+        checkDependencies(workflow.getNodes(), graph);
+
+        final NodeBase A = new ExplicitNode("A", null);
+        final NodeBase B = new ExplicitNode("B", null);
+        final NodeBase C = new ExplicitNode("C", null);
+        final NodeBase D = new ExplicitNode("D", null);
+        final NodeBase E = new ExplicitNode("E", null);
+        final NodeBase F = new ExplicitNode("F", null);
+
+        final Start start = new Start("start");
+        final End end = new End("end");
+        final Fork fork1 = new Fork("fork1");
+        final Fork fork2 = new Fork("fork2");
+        final Join join1 = new Join("join1", fork1);
+        final Join join2 = new Join("join2", fork2);
+
+        end.addParent(F);
+        F.addParent(join2);
+        join2.addParent(D);
+        join2.addParent(E);
+        D.addParent(fork2);
+        E.addParent(fork2);
+        fork2.addParent(join1);
+        join1.addParent(B);
+        join1.addParent(C);
+        B.addParent(fork1);
+        C.addParent(fork1);
+        fork1.addParent(A);
+        A.addParent(start);
+
+        final List<NodeBase> nodes = Arrays.asList(start, end, fork1, fork2, join1, join2, A, B, C, D, E, F);
+
+        checkEqualStructureByNames(nodes, graph);
+
+//        nodesToPng.withWorkflow(workflow);
+//        nodesToPng.withGraph(graph);
+    }
+
+    @Test
+    public void testCrossingDependencyLines() {
+        final Node a = MapReduceActionBuilder.create().withName("A").build();
+
+        final Node b = MapReduceActionBuilder.create().withName("B").build();
+        MapReduceActionBuilder.create().withName("C").withParent(a).withParent(b).build();
+
+        MapReduceActionBuilder.create().withName("D").withParent(a).withParent(b).build();
+
+        final Workflow workflow = new WorkflowBuilder().withName("crossing-dependencies").withDagContainingNode(a).build();
+        final Graph graph = new Graph(workflow);
+
+        checkDependencies(workflow.getNodes(), graph);
+
+        final NodeBase A = new ExplicitNode("A", null);
+        final NodeBase B = new ExplicitNode("B", null);
+        final NodeBase C = new ExplicitNode("C", null);
+        final NodeBase D = new ExplicitNode("D", null);
+
+        final Start start = new Start("start");
+        final End end = new End("end");
+        final Fork fork1 = new Fork("fork1");
+        final Fork fork2 = new Fork("fork2");
+        final Join join1 = new Join("join1", fork1);
+        final Join join2 = new Join("join2", fork2);
+
+        end.addParent(join2);
+        join2.addParent(C);
+        join2.addParent(D);
+        C.addParent(fork2);
+        D.addParent(fork2);
+        fork2.addParent(join1);
+        join1.addParent(A);
+        join1.addParent(B);
+        A.addParent(fork1);
+        B.addParent(fork1);
+        fork1.addParent(start);
+
+        final List<NodeBase> nodes = Arrays.asList(start, end, fork1, fork2, join1, join2, A, B, C, D);
+        checkEqualStructureByNames(nodes, graph);
+
+//        nodesToPng.withWorkflow(workflow);
+//        nodesToPng.withGraph(graph);
+    }
+
+    @Test
+    public void testSplittingJoins() {
+        final Node a = MapReduceActionBuilder.create().withName("A").build();
+
+        final Node b = MapReduceActionBuilder.create().withName("B").withParent(a).build();
+        final Node c = MapReduceActionBuilder.create().withName("C").withParent(b).build();
+
+        final Node d = MapReduceActionBuilder.create().withName("D").withParent(b).build();
+        final Node e = MapReduceActionBuilder.create().withName("E").withParent(a).build();
+
+        MapReduceActionBuilder.create().withName("F").withParent(c).withParent(d).withParent(e).build();
+
+        final Workflow workflow = new WorkflowBuilder().withName("splitting-joins").withDagContainingNode(a).build();
+        final Graph graph = new Graph(workflow);
+
+        checkDependencies(workflow.getNodes(), graph);
+
+        final NodeBase A = new ExplicitNode("A", null);
+        final NodeBase B = new ExplicitNode("B", null);
+        final NodeBase C = new ExplicitNode("C", null);
+        final NodeBase D = new ExplicitNode("D", null);
+        final NodeBase E = new ExplicitNode("E", null);
+        final NodeBase F = new ExplicitNode("F", null);
+
+        final Start start = new Start("start");
+        final End end = new End("end");
+        final Fork fork1 = new Fork("fork1");
+        final Fork fork2 = new Fork("fork2");
+        final Join join1 = new Join("join1", fork1);
+        final Join join2 = new Join("join2", fork2);
+
+        end.addParent(F);
+        F.addParent(join1);
+        join1.addParent(join2);
+        join1.addParent(E);
+        join2.addParent(C);
+        join2.addParent(D);
+        C.addParent(fork2);
+        D.addParent(fork2);
+        fork2.addParent(B);
+        B.addParent(fork1);
+        E.addParent(fork1);
+        fork1.addParent(A);
+        A.addParent(start);
+
+        final List<NodeBase> nodes = Arrays.asList(start, end, fork1, fork2, join1, join2, A, B, C, D, E, F);
+
+        checkEqualStructureByNames(nodes, graph);
+
+//        nodesToPng.withWorkflow(workflow);
+//        nodesToPng.withGraph(graph);
+    }
+
+    @Test
+    public void testSplittingForks() {
+        final Node a = MapReduceActionBuilder.create().withName("A").build();
+
+        final Node b = MapReduceActionBuilder.create().withName("B").withParent(a).build();
+        final Node c = MapReduceActionBuilder.create().withName("C").withParent(a).build();
+
+        final Node d = MapReduceActionBuilder.create().withName("D").withParent(a).build();
+        final Node e = MapReduceActionBuilder.create().withName("E").withParent(b).withParent(c).build();
+
+        MapReduceActionBuilder.create().withName("F").withParent(e).withParent(d).build();
+
+        final Workflow workflow = new WorkflowBuilder().withName("splitting-forks").withDagContainingNode(a).build();
+        final Graph graph = new Graph(workflow);
+
+        checkDependencies(workflow.getNodes(), graph);
+
+        final NodeBase A = new ExplicitNode("A", null);
+        final NodeBase B = new ExplicitNode("B", null);
+        final NodeBase C = new ExplicitNode("C", null);
+        final NodeBase D = new ExplicitNode("D", null);
+        final NodeBase E = new ExplicitNode("E", null);
+        final NodeBase F = new ExplicitNode("F", null);
+
+        final Start start = new Start("start");
+        final End end = new End("end");
+        final Fork fork1 = new Fork("fork1");
+        final Fork fork2 = new Fork("fork2");
+        final Join join1 = new Join("join1", fork1);
+        final Join join2 = new Join("join2", fork2);
+
+        end.addParent(F);
+        F.addParent(join1);
+        join1.addParent(E);
+        join1.addParent(D);
+        E.addParent(join2);
+        join2.addParent(B);
+        join2.addParent(C);
+        B.addParent(fork2);
+        C.addParent(fork2);
+        fork2.addParent(fork1);
+        D.addParent(fork1);
+        fork1.addParent(A);
+        A.addParent(start);
+
+        final List<NodeBase> nodes = Arrays.asList(start, end, fork1, fork2, join1, join2, A, B, C, D, E, F);
+
+        checkEqualStructureByNames(nodes, graph);
+
+//        nodesToPng.withWorkflow(workflow);
+//        nodesToPng.withGraph(graph);
+    }
+
+    @Test
+    public void testBranchingUncles() {
+        final Node a = MapReduceActionBuilder.create().withName("A").build();
+
+        final Node b = MapReduceActionBuilder.create().withName("B").withParent(a).build();
+        final Node c = MapReduceActionBuilder.create().withName("C").withParent(a).build();
+
+        final Node d = MapReduceActionBuilder.create().withName("D").withParent(b).build();
+        final Node e = MapReduceActionBuilder.create().withName("E").withParent(c).build();
+
+        final Node f = MapReduceActionBuilder.create().withName("F").withParent(d).withParent(e).build();
+        final Node g = MapReduceActionBuilder.create().withName("G").withParent(c).build();
+        MapReduceActionBuilder.create().withName("H").withParent(f).withParent(g).build();
+
+        final Workflow workflow = new WorkflowBuilder().withName("branching-uncles").withDagContainingNode(a).build();
+        final Graph graph = new Graph(workflow);
+
+        checkDependencies(workflow.getNodes(), graph);
+
+        final NodeBase A = new ExplicitNode("A", null);
+        final NodeBase B = new ExplicitNode("B", null);
+        final NodeBase C = new ExplicitNode("C", null);
+        final NodeBase D = new ExplicitNode("D", null);
+        final NodeBase E = new ExplicitNode("E", null);
+        final NodeBase F = new ExplicitNode("F", null);
+        final NodeBase G = new ExplicitNode("G", null);
+        final NodeBase H = new ExplicitNode("H", null);
+
+        final Start start = new Start("start");
+        final End end = new End("end");
+        final Fork fork1 = new Fork("fork1");
+        final Fork fork2 = new Fork("fork3");
+        final Join join1 = new Join("join1", fork1);
+        final Join join2 = new Join("join3", fork2);
+
+        end.addParent(H);
+        H.addParent(join2);
+        join2.addParent(F);
+        join2.addParent(G);
+        F.addParent(fork2);
+        G.addParent(fork2);
+        fork2.addParent(join1);
+        join1.addParent(D);
+        join1.addParent(E);
+        D.addParent(B);
+        E.addParent(C);
+        B.addParent(fork1);
+        C.addParent(fork1);
+        fork1.addParent(A);
+        A.addParent(start);
+
+        final List<NodeBase> nodes = Arrays.asList(start, end, fork1, fork2, join1, join2, A, B, C, D, E, F, G, H);
+
+        checkEqualStructureByNames(nodes, graph);
+
+//        nodesToPng.withWorkflow(workflow);
+//        nodesToPng.withGraph(graph);
+    }
+
+    @Test
+    public void testTrivialRedundantEdge() {
+        final Node a = MapReduceActionBuilder.create().withName("A").build();
+
+        final Node b = MapReduceActionBuilder.create().withName("B").withParent(a).build();
+        MapReduceActionBuilder.create().withName("C").withParent(a).withParent(b).build();
+
+        final Workflow workflow = new WorkflowBuilder().withName("trivial-redundant-edge").withDagContainingNode(a).build();
+        final Graph graph = new Graph(workflow);
+
+        checkDependencies(workflow.getNodes(), graph);
+
+        final NodeBase A = new ExplicitNode("A", null);
+        final NodeBase B = new ExplicitNode("B", null);
+        final NodeBase C = new ExplicitNode("C", null);
+
+        final Start start = new Start("start");
+        final End end = new End("end");
+
+        end.addParent(C);
+        C.addParent(B);
+        B.addParent(A);
+        A.addParent(start);
+
+        final List<NodeBase> nodes = Arrays.asList(start, end, A, B, C);
+
+        checkEqualStructureByNames(nodes, graph);
+//
+//        nodesToPng.withWorkflow(workflow);
+//        nodesToPng.withGraph(graph);
+    }
+
+    @Test
+    public void testRedundantEdge() {
+        final Node a = MapReduceActionBuilder.create().withName("A").build();
+
+        final Node b = MapReduceActionBuilder.create().withName("B").withParent(a).build();
+        final Node c = MapReduceActionBuilder.create().withName("C").withParent(a).build();
+
+        final Node d = MapReduceActionBuilder.create().withName("D").withParent(b).withParent(c).build();
+        final Node e = MapReduceActionBuilder.create().withName("E").withParent(c).build();
+
+        MapReduceActionBuilder.create().withName("F").withParent(d).withParent(e).withParent(a).build();
+
+        final Workflow workflow = new WorkflowBuilder().withName("redundant-edge").withDagContainingNode(a).build();
+        final Graph graph = new Graph(workflow);
+
+        checkDependencies(workflow.getNodes(), graph);
+
+        final NodeBase A = new ExplicitNode("A", null);
+        final NodeBase B = new ExplicitNode("B", null);
+        final NodeBase C = new ExplicitNode("C", null);
+        final NodeBase D = new ExplicitNode("D", null);
+        final NodeBase E = new ExplicitNode("E", null);
+        final NodeBase F = new ExplicitNode("F", null);
+
+        final Start start = new Start("start");
+        final End end = new End("end");
+        final Fork fork1 = new Fork("fork1");
+        final Fork fork2 = new Fork("fork2");
+        final Join join1 = new Join("join1", fork1);
+        final Join join2 = new Join("join2", fork2);
+
+        end.addParent(F);
+        F.addParent(join2);
+        join2.addParent(D);
+        join2.addParent(E);
+        D.addParent(fork2);
+        E.addParent(fork2);
+        fork2.addParent(join1);
+        join1.addParent(B);
+        join1.addParent(C);
+        B.addParent(fork1);
+        C.addParent(fork1);
+        fork1.addParent(A);
+        A.addParent(start);
+
+        final List<NodeBase> nodes = Arrays.asList(start, end, fork1, fork2, join1, join2, A, B, C, D, E, F);
+
+        checkEqualStructureByNames(nodes, graph);
+
+//        nodesToPng.withWorkflow(workflow);
+//        nodesToPng.withGraph(graph);
+    }
+
+    @Test
+    public void testLateUncle() {
+        final Node a = MapReduceActionBuilder.create().withName("A").build();
+
+        final Node b = MapReduceActionBuilder.create().withName("B").withParent(a).build();
+        final Node c = MapReduceActionBuilder.create().withName("C").withParent(a).build();
+
+        final Node d = MapReduceActionBuilder.create().withName("D").withParent(b).build();
+        final Node e = MapReduceActionBuilder.create().withName("E").withParent(b).build();
+
+        final Node f = MapReduceActionBuilder.create().withName("F").withParent(c).build();
+
+        final Node g = MapReduceActionBuilder.create().withName("G").withParent(e).build();
+        final Node h = MapReduceActionBuilder.create().withName("H").withParent(f).build();
+        final Node i = MapReduceActionBuilder.create().withName("I").withParent(d).withParent(g).build();
+        final Node j = MapReduceActionBuilder.create().withName("J").withParent(e).withParent(h).build();
+        MapReduceActionBuilder.create().withName("K").withParent(i).withParent(j).build();
+
+        final Workflow workflow = new WorkflowBuilder().withName("late-uncle").withDagContainingNode(a).build();
+        final Graph graph = new Graph(workflow);
+
+        checkDependencies(workflow.getNodes(), graph);
+
+        final NodeBase A = new ExplicitNode("A", null);
+        final NodeBase B = new ExplicitNode("B", null);
+        final NodeBase C = new ExplicitNode("C", null);
+        final NodeBase D = new ExplicitNode("D", null);
+        final NodeBase E = new ExplicitNode("E", null);
+        final NodeBase F = new ExplicitNode("F", null);
+        final NodeBase G = new ExplicitNode("G", null);
+        final NodeBase H = new ExplicitNode("H", null);
+        final NodeBase I = new ExplicitNode("I", null);
+        final NodeBase J = new ExplicitNode("J", null);
+        final NodeBase K = new ExplicitNode("K", null);
+
+        final Start start = new Start("start");
+        final End end = new End("end");
+        final Fork fork1 = new Fork("fork1");
+        final Fork fork2 = new Fork("fork2");
+        final Fork fork3 = new Fork("fork3");
+        final Join join1 = new Join("join1", fork1);
+        final Join join2 = new Join("join2", fork2);
+        final Join join3 = new Join("join3", fork3);
+
+        end.addParent(K);
+        K.addParent(join3);
+        join3.addParent(I);
+        join3.addParent(J);
+        I.addParent(fork3);
+        J.addParent(fork3);
+        fork3.addParent(join1);
+        join1.addParent(join2);
+        join1.addParent(H);
+        join2.addParent(D);
+        join2.addParent(G);
+        G.addParent(E);
+        D.addParent(fork2);
+        E.addParent(fork2);
+        fork2.addParent(B);
+        B.addParent(fork1);
+        H.addParent(F);
+        F.addParent(C);
+        C.addParent(fork1);
+        fork1.addParent(A);
+        A.addParent(start);
+
+        final List<NodeBase> nodes = Arrays.asList(start, end, fork1, fork2, fork3, join1, join2, join3,
+                                             A, B, C, D, E, F, G, H, I, J, K);
+
+        checkEqualStructureByNames(nodes, graph);
+
+//        nodesToPng.withWorkflow(workflow);
+//        nodesToPng.withGraph(graph);
+    }
+
+    @Test
+    public void testMultipleRoots() {
+        final Node a = MapReduceActionBuilder.create().withName("A").build();
+        final Node g = MapReduceActionBuilder.create().withName("G").build();
+
+        final Node b = MapReduceActionBuilder.create().withName("B").withParent(a).withParent(g).build();
+        final Node c = MapReduceActionBuilder.create().withName("C").withParent(a).build();
+
+        final Node d = MapReduceActionBuilder.create().withName("D").withParent(b).withParent(c).build();
+        final Node e = MapReduceActionBuilder.create().withName("E").withParent(c).build();
+
+        MapReduceActionBuilder.create().withName("F").withParent(d).withParent(e).build();
+
+        final Workflow workflow = new WorkflowBuilder().withName("multiple-roots").withDagContainingNode(a).build();
+        final Graph graph = new Graph(workflow);
+
+        checkDependencies(workflow.getNodes(), graph);
+
+        final NodeBase A = new ExplicitNode("A", null);
+        final NodeBase B = new ExplicitNode("B", null);
+        final NodeBase C = new ExplicitNode("C", null);
+        final NodeBase D = new ExplicitNode("D", null);
+        final NodeBase E = new ExplicitNode("E", null);
+        final NodeBase F = new ExplicitNode("F", null);
+        final NodeBase G = new ExplicitNode("G", null);
+
+        final Start start = new Start("start");
+        final End end = new End("end");
+        final Fork fork1 = new Fork("fork1");
+        final Fork fork2 = new Fork("fork2");
+        final Fork fork3 = new Fork("fork3");
+        final Join join1 = new Join("join1", fork1);
+        final Join join2 = new Join("join2", fork2);
+        final Join join3 = new Join("join3", fork3);
+
+        end.addParent(F);
+        F.addParent(join3);
+        join3.addParent(D);
+        join3.addParent(E);
+        D.addParent(fork3);
+        E.addParent(fork3);
+        fork3.addParent(join2);
+        join2.addParent(B);
+        join2.addParent(C);
+        B.addParent(fork2);
+        C.addParent(fork2);
+        fork2.addParent(join1);
+        join1.addParent(G);
+        join1.addParent(A);
+        G.addParent(fork1);
+        A.addParent(fork1);
+        fork1.addParent(start);
+
+        final List<NodeBase> nodes = Arrays.asList(
+                start, end, fork1, fork2, fork3, join1, join2, join3, A, B, C, D, E, F, G);
+
+        checkEqualStructureByNames(nodes, graph);
+
+//        nodesToPng.withWorkflow(workflow);
+//        nodesToPng.withGraph(graph);
+    }
+
+    @Test
+    public void testTrivialDecision() {
+        final String conditionGotoB = "condition_goto_B";
+
+        final Node a = MapReduceActionBuilder.create().withName("A").build();
+        MapReduceActionBuilder.create().withName("B").withParentWithCondition(a, conditionGotoB).build();
+        MapReduceActionBuilder.create().withName("C").withParentDefaultConditional(a).build();
+
+        final Workflow workflow = new WorkflowBuilder()
+                .withName("Workflow_to_map")
+                .withDagContainingNode(a)
+                .build();
+        final Graph graph = new Graph(workflow);
+
+        final Start start = new Start("start");
+        final End end = new End("end");
+        final Decision decision = new Decision("decision1");
+        final DecisionJoin decisionJoin = new DecisionJoin("decisionJoin1", decision);
+
+        final NodeBase A = new ExplicitNode("A", null);
+        final NodeBase B = new ExplicitNode("B", null);
+        final NodeBase C = new ExplicitNode("C", null);
+
+        end.addParent(decisionJoin);
+        decisionJoin.addParent(B);
+        decisionJoin.addParent(C);
+        B.addParentWithCondition(decision, Condition.actualCondition(conditionGotoB));
+        C.addParentDefaultConditional(decision);
+        decision.addParent(A);
+        A.addParent(start);
+
+        final List<NodeBase> nodes = Arrays.asList(
+                start, end, decision, decisionJoin, A, B, C);
+
+        checkEqualStructureByNames(nodes, graph);
+    }
+
+    @Test
+    public void testTrivialDiamondDecision() {
+        final String conditionGotoB = "condition_goto_B";
+        final String conditionGotoC = "condition_goto_C";
+
+        final Node a = MapReduceActionBuilder.create().withName("A").build();
+        final Node b = MapReduceActionBuilder.create().withName("B").withParentWithCondition(a, conditionGotoB).build();
+        final Node c = MapReduceActionBuilder.create().withName("C").withParentWithCondition(a, conditionGotoC).build();
+        MapReduceActionBuilder.create().withName("D").withParent(b).withParent(c).build();
+
+        final Workflow workflow = new WorkflowBuilder().withName("trivial-decision").withDagContainingNode(a).build();
+        final Graph graph = new Graph(workflow);
+
+        final NodeBase A = new ExplicitNode("A", null);
+        final NodeBase B = new ExplicitNode("B", null);
+        final NodeBase C = new ExplicitNode("C", null);
+        final NodeBase D = new ExplicitNode("D", null);
+
+        final Start start = new Start("start");
+        final End end = new End("end");
+        final Decision decision = new Decision("decision1");
+        final DecisionJoin decisionJoin = new DecisionJoin("decisionJoin1", decision);
+
+        end.addParent(D);
+        D.addParent(decisionJoin);
+        decisionJoin.addParent(B);
+        decisionJoin.addParent(C);
+        B.addParentWithCondition(decision, Condition.actualCondition(conditionGotoB));
+        C.addParentWithCondition(decision, Condition.actualCondition(conditionGotoC));
+        decision.addParent(A);
+        A.addParent(start);
+
+        final List<NodeBase> nodes = Arrays.asList(
+                start, end, decision, decisionJoin, A, B, C, D);
+
+        checkEqualStructureByNames(nodes, graph);
+
+//        nodesToPng.withWorkflow(workflow);
+//        nodesToPng.withGraph(graph);
+    }
+
+    @Test
+    public void testDecisionAndJoin() {
+        final String conditionGotoD = "condition_goto_D";
+        final String conditionGotoE = "condition_goto_E";
+
+        final Node a = MapReduceActionBuilder.create().withName("A").build();
+        final Node b = MapReduceActionBuilder.create().withName("B").withParent(a).build();
+        final Node c = MapReduceActionBuilder.create().withName("C").withParent(a).build();
+        final Node d = MapReduceActionBuilder.create().withName("D").withParent(b)
+                .withParentWithCondition(c, conditionGotoD).build();
+        final Node e = MapReduceActionBuilder.create().withName("E").withParentWithCondition(c, conditionGotoE).build();
+        MapReduceActionBuilder.create().withName("F").withParent(d).build();
+        MapReduceActionBuilder.create().withName("G").withParent(e).build();
+
+        final Workflow workflow = new WorkflowBuilder().withName("decision-and-join").withDagContainingNode(a).build();
+        final Graph graph = new Graph(workflow);
+
+        final NodeBase A = new ExplicitNode("A", null);
+        final NodeBase B = new ExplicitNode("B", null);
+        final NodeBase C = new ExplicitNode("C", null);
+        final NodeBase D = new ExplicitNode("D", null);
+        final NodeBase E = new ExplicitNode("E", null);
+        final NodeBase F = new ExplicitNode("F", null);
+        final NodeBase G = new ExplicitNode("G", null);
+
+        final Start start = new Start("start");
+        final End end = new End("end");
+        final Fork fork = new Fork("fork1");
+        final Join join = new Join("join1", fork);
+        final Decision decision = new Decision("decision1");
+        final DecisionJoin decisionJoin = new DecisionJoin("decisionJoin1", decision);
+
+        end.addParent(decisionJoin);
+        decisionJoin.addParent(F);
+        decisionJoin.addParent(G);
+        F.addParent(D);
+        D.addParentWithCondition(decision, Condition.actualCondition(conditionGotoD));
+        G.addParent(E);
+        E.addParentWithCondition(decision, Condition.actualCondition(conditionGotoE));
+        decision.addParent(join);
+        join.addParent(B);
+        join.addParent(C);
+        B.addParent(fork);
+        C.addParent(fork);
+        fork.addParent(A);
+        A.addParent(start);
+
+        final List<NodeBase> nodes = Arrays.asList(
+                start, end, fork, join, decision, decisionJoin, A, B, C, D, E, F, G);
+
+//        nodesToPng.withWorkflow(workflow);
+//        nodesToPng.withGraph(graph);
+
+        checkEqualStructureByNames(nodes, graph);
+    }
+
+    @Test
+    public void testDecisionAtUncleOfJoin() {
+        final String conditionGotoD = "condition_goto_D";
+        final String conditionGotoF = "condition_goto_F";
+
+        final Node a = MapReduceActionBuilder.create().withName("A").build();
+        final Node b = MapReduceActionBuilder.create().withName("B").withParent(a).build();
+        final Node c = MapReduceActionBuilder.create().withName("C").withParent(a).build();
+        final Node d = MapReduceActionBuilder.create().withName("D").withParentWithCondition(c, conditionGotoD).build();
+        final Node e = MapReduceActionBuilder.create().withName("E").withParent(b).withParent(d).build();
+        final Node f = MapReduceActionBuilder.create().withName("F").withParentWithCondition(c, conditionGotoF).build();
+        MapReduceActionBuilder.create().withName("G").withParent(e).build();
+        MapReduceActionBuilder.create().withName("H").withParent(f).build();
+
+        final Workflow workflow = new WorkflowBuilder().withName("decision-at-uncle-of-join").withDagContainingNode(a).build();
+        final Graph graph = new Graph(workflow);
+
+        final NodeBase A = new ExplicitNode("A", null);
+        final NodeBase B = new ExplicitNode("B", null);
+        final NodeBase C = new ExplicitNode("C", null);
+        final NodeBase D = new ExplicitNode("D", null);
+        final NodeBase E = new ExplicitNode("E", null);
+        final NodeBase F = new ExplicitNode("F", null);
+        final NodeBase G = new ExplicitNode("G", null);
+        final NodeBase H = new ExplicitNode("H", null);
+
+        final Start start = new Start("start");
+        final End end = new End("end");
+        final Fork fork = new Fork("fork1");
+        final Join join = new Join("join1", fork);
+        final Decision decision = new Decision("decision1");
+        final DecisionJoin decisionJoin = new DecisionJoin("decisionJoin1", decision);
+
+        end.addParent(decisionJoin);
+        decisionJoin.addParent(G);
+        decisionJoin.addParent(H);
+        G.addParent(E);
+        H.addParent(F);
+        E.addParent(D);
+        D.addParentWithCondition(decision, Condition.actualCondition(conditionGotoD));
+        F.addParentWithCondition(decision, Condition.actualCondition(conditionGotoF));
+        decision.addParent(join);
+        join.addParent(B);
+        join.addParent(C);
+        B.addParent(fork);
+        C.addParent(fork);
+        fork.addParent(A);
+        A.addParent(start);
+
+        final List<NodeBase> nodes = Arrays.asList(
+                start, end, fork, join, decision, decisionJoin, A, B, C, D, E, F, G, H);
+
+//        nodesToPng.withWorkflow(workflow);
+//        nodesToPng.withGraph(graph);
+
+        checkEqualStructureByNames(nodes, graph);
+    }
+
+    @Test
+    public void testAlreadyClosedDecisionBranching() {
+        final String conditionGotoD = "condition_goto_D";
+        final String conditionGotoE = "condition_goto_E";
+
+        final Node a = MapReduceActionBuilder.create().withName("A").build();
+        final Node b = MapReduceActionBuilder.create().withName("B").withParent(a).build();
+        final Node c = MapReduceActionBuilder.create().withName("C").withParent(a).build();
+        final Node d = MapReduceActionBuilder.create().withName("D").withParentWithCondition(b, conditionGotoD).build();
+        final Node e = MapReduceActionBuilder.create().withName("E").withParentWithCondition(b, conditionGotoE).build();
+
+        final Node f = MapReduceActionBuilder.create().withName("F").withParent(d).withParent(e).build();
+        MapReduceActionBuilder.create().withName("G").withParent(f).withParent(c).build();
+
+        final Workflow workflow = new WorkflowBuilder()
+                .withName("already-closed-decision-branching")
+                .withDagContainingNode(a)
+                .build();
+        final Graph graph = new Graph(workflow);
+
+        final NodeBase A = new ExplicitNode("A", null);
+        final NodeBase B = new ExplicitNode("B", null);
+        final NodeBase C = new ExplicitNode("C", null);
+        final NodeBase D = new ExplicitNode("D", null);
+        final NodeBase E = new ExplicitNode("E", null);
+        final NodeBase F = new ExplicitNode("F", null);
+        final NodeBase G = new ExplicitNode("G", null);
+
+        final Start start = new Start("start");
+        final End end = new End("end");
+        final Fork fork = new Fork("fork1");
+        final Join join = new Join("join1", fork);
+        final Decision decision = new Decision("decision1");
+        final DecisionJoin decisionJoin = new DecisionJoin("decisionJoin1", decision);
+
+        end.addParent(G);
+        G.addParent(join);
+        join.addParent(F);
+        join.addParent(C);
+        F.addParent(decisionJoin);
+        decisionJoin.addParent(D);
+        decisionJoin.addParent(E);
+        D.addParentWithCondition(decision, Condition.actualCondition(conditionGotoD));
+        E.addParentWithCondition(decision, Condition.actualCondition(conditionGotoE));
+        decision.addParent(B);
+        B.addParent(fork);
+        C.addParent(fork);
+        fork.addParent(A);
+        A.addParent(start);
+
+        final List<NodeBase> nodes = Arrays.asList(
+                start, end, fork, join, decision, decisionJoin, A, B, C, D, E, F, G);
+
+//        nodesToPng.withWorkflow(workflow);
+//        nodesToPng.withGraph(graph);
+
+        checkEqualStructureByNames(nodes, graph);
+
+    }
+
+    @Test
+    public void testIncomingConditionalBranchesFromDifferentDecisionsThrows() {
+        final Node a = MapReduceActionBuilder.create().withName("A").build();
+
+        final Node b = MapReduceActionBuilder.create().withName("B").withParent(a).build();
+        final Node c = MapReduceActionBuilder.create().withName("C").withParent(a).build();
+        final Node d = MapReduceActionBuilder.create().withName("D").withParent(a).build();
+
+        MapReduceActionBuilder.create().withName("E").withParentWithCondition(c, "condition_goto_E").build();
+        final Node f = MapReduceActionBuilder.create().withName("F").withParentDefaultConditional(c).build();
+
+        final Node g = MapReduceActionBuilder.create().withName("G").withParentWithCondition(d, "condition_goto_G").build();
+        final Node h = MapReduceActionBuilder.create().withName("H").withParentDefaultConditional(d).build();
+
+        MapReduceActionBuilder.create().withName("I").withParent(b).withParent(f).withParent(g).build();
+        MapReduceActionBuilder.create().withName("J").withParent(h).build();
+
+        final Workflow workflow = new WorkflowBuilder()
+                .withName("incoming-conditional-branches-from-different-decisions")
+                .withDagContainingNode(a)
+                .build();
+
+//        nodesToPng.withWorkflow(workflow);
+
+        // TODO: We might choose to implement it later without an exception.
+        expectedException.expect(IllegalStateException.class);
+        new Graph(workflow);
+    }
+
+    private void checkEqualStructureByNames(final Collection<NodeBase> expectedNodes, final Graph graph2) {
+        assertEquals(expectedNodes.size(), graph2.getNodes().size());
+
+        for (final NodeBase expectedNode : expectedNodes) {
+            final NodeBase nodeInOtherGraph = graph2.getNodeByName(expectedNode.getName());
+
+            assertNotNull(nodeInOtherGraph);
+
+            final List<NodeBase> expectedChildren = expectedNode.getChildren();
+            final List<NodeBase> actualChildren = nodeInOtherGraph.getChildren();
+
+            final List<String> expectedChildrenNames = new ArrayList<>();
+            for (final NodeBase child : expectedChildren) {
+                expectedChildrenNames.add(child.getName());
+            }
+
+            final List<String> actualChildrenNames = new ArrayList<>();
+            for (final NodeBase child : actualChildren) {
+                actualChildrenNames.add(child.getName());
+            }
+
+            if (expectedNode instanceof Fork) {
+                // The order of the children of fork nodes is not important.
+                Collections.sort(expectedChildrenNames);
+                Collections.sort(actualChildrenNames);
+            }
+
+            assertEquals(expectedChildrenNames.size(), actualChildrenNames.size());
+
+            for (int i = 0; i < expectedChildren.size(); ++i) {
+                final String expectedName = expectedChildrenNames.get(i);
+                final String actualName = actualChildrenNames.get(i);
+
+                if (graph2.getNodeByName(actualName) instanceof ExplicitNode) {
+                    assertEquals(expectedName, actualName);
+                }
+            }
+        }
+    }
+
+    private void checkDependencies(final Set<Node> originalNodes, final Graph graph) {
+        for (final Node originalNode : originalNodes) {
+            for (final Node originalParent : originalNode.getAllParents()) {
+                final NodeBase node = graph.getNodeByName(originalNode.getName());
+                final NodeBase parent = graph.getNodeByName(originalParent.getName());
+
+                assertTrue(verifyDependency(parent, node));
+            }
+        }
+    }
+
+    private boolean verifyDependency(final NodeBase dependency, final NodeBase dependent) {
+        final List<NodeBase> children = dependency.getChildren();
+
+        for (final NodeBase child : children) {
+            if (child == dependent || verifyDependency(child, dependent)) {
+                return true;
+            }
+        }
+
+        return false;
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/dag/TestJoin.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/dag/TestJoin.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/dag/TestJoin.java
new file mode 100644
index 0000000..f915bce
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/dag/TestJoin.java
@@ -0,0 +1,52 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.dag;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class TestJoin extends TestJoiningNodeBase<Fork, Join> {
+    @Override
+    protected Join getInstance(final String name) {
+        return getJoiningInstance(name);
+    }
+
+    @Override
+    protected Fork getBranchingInstance(String name) {
+        return new Fork(name);
+    }
+
+    @Override
+    protected Join getJoiningInstance(String name, Fork branchingPair) {
+        return new Join(name, branchingPair);
+    }
+
+    @Test
+    public void testCorrespondingForkIsCorrect() {
+        Fork fork = new Fork("fork");
+        Join join = new Join("join", fork);
+
+        assertEquals(fork, join.getBranchingPair());
+
+        assertEquals(join, fork.getClosingJoin());
+        assertTrue(fork.isClosed());
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/dag/TestJoiningNodeBase.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/dag/TestJoiningNodeBase.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/dag/TestJoiningNodeBase.java
new file mode 100644
index 0000000..d498019
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/dag/TestJoiningNodeBase.java
@@ -0,0 +1,157 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.dag;
+
+import org.junit.Test;
+
+import java.util.Arrays;
+
+import static org.junit.Assert.assertEquals;
+
+public abstract class TestJoiningNodeBase<B, J extends JoiningNodeBase<B>> extends TestNodeBase<JoiningNodeBase<B>> {
+    protected abstract B getBranchingInstance(final String name);
+    protected abstract J getJoiningInstance(final String name, final B branchingPair);
+
+    protected  J getJoiningInstance(final String name) {
+        return getJoiningInstance(name, getBranchingInstance("branching"));
+    }
+
+    @Test
+    public void testCorrespondingBranchingIsCorrect() {
+        B branching = getBranchingInstance("branching");
+        J joining = getJoiningInstance("joining", branching);
+
+        assertEquals(branching, joining.getBranchingPair());
+    }
+
+    @Test
+    public void testAddParentWhenNoneAlreadyExists() {
+        final ExplicitNode parent = new ExplicitNode("parent", null);
+        final J instance = getJoiningInstance("instance");
+
+        instance.addParent(parent);
+        assertEquals(Arrays.asList(parent), instance.getParents());
+        assertEquals(instance, parent.getChild());
+    }
+
+    @Test
+    public void testAddParentWhenSomeAlreadyExist() {
+        final NodeBase parent1 = new ExplicitNode("parent1", null);
+        final NodeBase parent2 = new ExplicitNode("parent2", null);
+
+        final J instance = getJoiningInstance("instance");
+
+        instance.addParent(parent1);
+        instance.addParent(parent2);
+
+        assertEquals(Arrays.asList(parent1, parent2), instance.getParents());
+    }
+
+    @Test
+    public void testRemoveExistingParent() {
+        final ExplicitNode parent1 = new ExplicitNode("parent1", null);
+        final ExplicitNode parent2 = new ExplicitNode("parent2", null);
+
+        final J instance = getJoiningInstance("instance");
+
+        instance.addParent(parent1);
+        instance.addParent(parent2);
+
+        instance.removeParent(parent2);
+        assertEquals(Arrays.asList(parent1), instance.getParents());
+        assertEquals(null, parent2.getChild());
+    }
+
+    @Test
+    public void testRemoveNonexistentParentThrows() {
+        final ExplicitNode parent = new ExplicitNode("parent", null);
+        final J instance = getJoiningInstance("instance");
+
+        expectedException.expect(IllegalArgumentException.class);
+        instance.removeParent(parent);
+    }
+
+    @Test
+    public void testClearExistingParent() {
+        final ExplicitNode parent1 = new ExplicitNode("parent1", null);
+        final ExplicitNode parent2 = new ExplicitNode("parent2", null);
+
+        final J instance = getJoiningInstance("instance");
+
+        instance.addParent(parent1);
+        instance.addParent(parent2);
+
+        instance.clearParents();
+        assertEquals(0, instance.getParents().size());
+        assertEquals(null, parent1.getChild());
+        assertEquals(null, parent2.getChild());
+    }
+
+    @Test
+    public void testClearNonExistentParent() {
+        final J instance = getJoiningInstance("instance");
+
+        instance.clearParents();
+        assertEquals(0, instance.getParents().size());
+    }
+
+    @Test
+    public void testJoinAddedAsParentWhenItHasNoChild() {
+        final J instance = getJoiningInstance("instance");
+        final NodeBase child = new ExplicitNode("child", null);
+
+        child.addParent(instance);
+
+        assertEquals(child, instance.getChild());
+    }
+
+    @Test
+    public void testJoinAddedAsParentWhenItAlreadyHasAChildThrows() {
+        final J instance = getJoiningInstance("instance");
+        final NodeBase child1 = new ExplicitNode("child1", null);
+        final NodeBase child2 = new ExplicitNode("child2", null);
+
+        child1.addParent(instance);
+
+        expectedException.expect(IllegalStateException.class);
+        child2.addParent(instance);
+    }
+
+    @Test
+    public void testJoinRemovedAsParent() {
+        final J instance = getJoiningInstance("instance");
+        final NodeBase child = new ExplicitNode("child", null);
+
+        child.addParent(instance);
+
+        child.removeParent(instance);
+
+        assertEquals(null, instance.getChild());
+    }
+
+    @Test
+    public void testGetChildren() {
+        final J instance = getJoiningInstance("instance");
+        final NodeBase child = new ExplicitNode("child", null);
+
+        child.addParent(instance);
+
+        assertEquals(Arrays.asList(child), instance.getChildren());
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/dag/TestNodeBase.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/dag/TestNodeBase.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/dag/TestNodeBase.java
new file mode 100644
index 0000000..5f14c0b
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/dag/TestNodeBase.java
@@ -0,0 +1,40 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.dag;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import static org.junit.Assert.assertEquals;
+
+public abstract class TestNodeBase<T extends NodeBase> {
+    @Rule
+    public final ExpectedException expectedException = ExpectedException.none();
+
+    static final String NAME = "node name";
+
+    protected abstract T getInstance(final String name);
+
+    @Test
+    public void testNameIsCorrect() {
+        final T instance = getInstance(NAME);
+        assertEquals(NAME, instance.getName());
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/dag/TestStart.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/dag/TestStart.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/dag/TestStart.java
new file mode 100644
index 0000000..c131db0
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/dag/TestStart.java
@@ -0,0 +1,113 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.dag;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import java.util.Arrays;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestStart extends TestNodeBase<Start> {
+    @Rule
+    public final ExpectedException expectedException = ExpectedException.none();
+
+    @Override
+    protected Start getInstance(final String name) {
+        return new Start(name);
+    }
+
+    @Test
+    public void testAddParent() {
+        final ExplicitNode parent = new ExplicitNode("parent", null);
+        final Start start = getInstance("start");
+
+        expectedException.expect(IllegalStateException.class);
+        start.addParent(parent);
+    }
+
+    @Test
+    public void testRemoveParent() {
+        final Start start = getInstance("start");
+
+        expectedException.expect(IllegalStateException.class);
+        start.removeParent(null);
+    }
+
+    @Test
+    public void testClearExistingParent() {
+        new Start("parent");
+        final Start instance = getInstance("instance");
+
+        instance.clearParents();
+    }
+
+    @Test
+    public void testClearNonExistentParent() {
+        new Start("parent");
+        final Start instance = getInstance("instance");
+
+        instance.clearParents();
+    }
+
+    @Test
+    public void testStartAddedAsParentWhenItHasNoChild() {
+        final Start start = getInstance("start");
+        final NodeBase child = new ExplicitNode("child", null);
+
+        child.addParent(start);
+
+        assertEquals(child, start.getChild());
+    }
+
+    @Test
+    public void testStartAddedAsParentWhenItAlreadyHasAChildThrows() {
+        final Start start = getInstance("start");
+        final NodeBase child1 = new ExplicitNode("child1", null);
+        final NodeBase child2 = new ExplicitNode("child2", null);
+
+        child1.addParent(start);
+
+        expectedException.expect(IllegalStateException.class);
+        child2.addParent(start);
+    }
+
+    @Test
+    public void testStartRemovedAsParent() {
+        final Start instance = getInstance("instance");
+        final NodeBase child = new ExplicitNode("child", null);
+
+        child.addParent(instance);
+        child.removeParent(instance);
+
+        assertEquals(null, instance.getChild());
+    }
+
+    @Test
+    public void testGetChildren() {
+        final Start start = getInstance("start");
+        final NodeBase child = new ExplicitNode("child", null);
+
+        child.addParent(start);
+
+        assertEquals(Arrays.asList(child), start.getChildren());
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/factory/SimpleWorkflowFactory.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/factory/SimpleWorkflowFactory.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/factory/SimpleWorkflowFactory.java
new file mode 100644
index 0000000..b1a7ffe
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/factory/SimpleWorkflowFactory.java
@@ -0,0 +1,65 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.factory;
+
+import org.apache.oozie.fluentjob.api.workflow.Workflow;
+import org.apache.oozie.fluentjob.api.workflow.WorkflowBuilder;
+import org.apache.oozie.fluentjob.api.action.ShellAction;
+import org.apache.oozie.fluentjob.api.action.ShellActionBuilder;
+
+/**
+ * An easily understandable {@link WorkflowFactory} that creates a {@link Workflow} instance consisting of
+ * three {@link ShellAction}s, the two latter depending conditionally on the output of the former.
+ * <p>
+ * It demonstrates how the Jobs API can be used to create dynamic {@code Workflow} artifacts, as well as
+ * serves as an input for {@code TestOozieCLI} methods that check, submit or run Jobs API {@code .jar} files.
+ */
+public class SimpleWorkflowFactory implements WorkflowFactory {
+
+    @Override
+    public Workflow create() {
+        final ShellAction parent = ShellActionBuilder.create()
+                .withName("parent")
+                .withResourceManager("${resourceManager}")
+                .withNameNode("${nameNode}")
+                .withConfigProperty("mapred.job.queue.name", "${queueName}")
+                .withArgument("my_output=Hello Oozie")
+                .withExecutable("echo")
+                .withCaptureOutput(true)
+                .build();
+
+        ShellActionBuilder.createFromExistingAction(parent)
+                .withName("happy-path")
+                .withParentWithCondition(parent, "${wf:actionData('parent')['my_output'] eq 'Hello Oozie'}")
+                .withoutArgument("my_output=Hello Oozie")
+                .withArgument("Happy path")
+                .withCaptureOutput(null)
+                .build();
+
+        ShellActionBuilder.createFromExistingAction(parent)
+                .withName("sad-path")
+                .withParentDefaultConditional(parent)
+                .withArgument("Sad path")
+                .build();
+
+        return new WorkflowBuilder()
+                .withName("shell-example")
+                .withDagContainingNode(parent).build();
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/SourceDataFactory.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/SourceDataFactory.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/SourceDataFactory.java
new file mode 100644
index 0000000..4def062
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/SourceDataFactory.java
@@ -0,0 +1,48 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.mapping;
+
+import com.google.common.collect.Lists;
+import org.apache.oozie.fluentjob.api.generated.workflow.CREDENTIALS;
+import org.apache.oozie.fluentjob.api.workflow.ConfigurationEntry;
+import org.apache.oozie.fluentjob.api.workflow.Credentials;
+import org.apache.oozie.fluentjob.api.workflow.CredentialsBuilder;
+
+import static org.junit.Assert.assertEquals;
+
+class SourceDataFactory {
+
+    Credentials createCredentials() {
+        return CredentialsBuilder.create()
+                .withCredential("hbase", "hbase")
+                .withCredential("hive2", "hive2",
+                        Lists.newArrayList(new ConfigurationEntry("jdbcUrl", "jdbc://localhost/hive2")))
+                .build();
+    }
+
+    void assertCredentials(final CREDENTIALS destination) {
+        assertEquals("hbase", destination.getCredential().get(0).getName());
+        assertEquals("hbase", destination.getCredential().get(0).getType());
+        assertEquals(0, destination.getCredential().get(0).getProperty().size());
+        assertEquals("hive2", destination.getCredential().get(1).getName());
+        assertEquals("hive2", destination.getCredential().get(1).getType());
+        assertEquals("jdbcUrl", destination.getCredential().get(1).getProperty().get(0).getName());
+        assertEquals("jdbc://localhost/hive2", destination.getCredential().get(1).getProperty().get(0).getValue());
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestActionAttributesMapping.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestActionAttributesMapping.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestActionAttributesMapping.java
new file mode 100644
index 0000000..3f9aab5
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestActionAttributesMapping.java
@@ -0,0 +1,100 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.mapping;
+
+import com.google.common.base.Strings;
+import org.apache.oozie.fluentjob.api.action.MapReduceAction;
+import org.apache.oozie.fluentjob.api.action.MapReduceActionBuilder;
+import org.apache.oozie.fluentjob.api.dag.ExplicitNode;
+import org.apache.oozie.fluentjob.api.generated.workflow.ACTION;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+public class TestActionAttributesMapping {
+
+    private final SourceDataFactory factory = new SourceDataFactory();
+
+    @Test
+    public void testMappingNoCredentialsToAction() {
+        final MapReduceAction source = MapReduceActionBuilder
+                .create()
+                .build();
+
+        final ACTION target = DozerBeanMapperSingleton.instance().map(new ExplicitNode("explicitNode", source), ACTION.class);
+
+        assertTrue(Strings.isNullOrEmpty(target.getCred()));
+    }
+
+    @Test
+    public void testMappingOneCredentialToAction() {
+        final MapReduceAction source = MapReduceActionBuilder
+                .create()
+                .withCredential(factory.createCredentials().getCredentials().get(0))
+                .build();
+
+        final ACTION target = DozerBeanMapperSingleton.instance().map(new ExplicitNode("explicitNode", source), ACTION.class);
+
+        assertEquals("hbase", target.getCred());
+    }
+
+    @Test
+    public void testMappingTwoCredentialsToSameAction() {
+        final MapReduceAction source = MapReduceActionBuilder
+                .create()
+                .withCredential(factory.createCredentials().getCredentials().get(0))
+                .withCredential(factory.createCredentials().getCredentials().get(1))
+                .build();
+
+        final ACTION target = DozerBeanMapperSingleton.instance().map(new ExplicitNode("explicitNode", source), ACTION.class);
+
+        assertEquals("hbase,hive2", target.getCred());
+    }
+
+    @Test
+    public void testMappingNoRetryAttributesToAction() {
+        final MapReduceAction source = MapReduceActionBuilder
+                .create()
+                .build();
+
+        final ACTION target = DozerBeanMapperSingleton.instance().map(new ExplicitNode("explicitNode", source), ACTION.class);
+
+        assertNull(target.getRetryInterval());
+        assertNull(target.getRetryMax());
+        assertNull(target.getRetryPolicy());
+    }
+
+    @Test
+    public void testMappingRetryAttributesToAction() {
+        final MapReduceAction source = MapReduceActionBuilder
+                .create()
+                .withRetryInterval(1)
+                .withRetryMax(3)
+                .withRetryPolicy("retry-policy")
+                .build();
+
+        final ACTION target = DozerBeanMapperSingleton.instance().map(new ExplicitNode("explicitNode", source), ACTION.class);
+
+        assertEquals("1", target.getRetryInterval());
+        assertEquals("3", target.getRetryMax());
+        assertEquals("retry-policy", target.getRetryPolicy());
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestConfigurationMapping.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestConfigurationMapping.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestConfigurationMapping.java
new file mode 100644
index 0000000..fac4c03
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestConfigurationMapping.java
@@ -0,0 +1,45 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.mapping;
+
+import com.google.common.collect.ImmutableMap;
+import org.apache.oozie.fluentjob.api.generated.workflow.CONFIGURATION;
+import org.junit.Test;
+
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestConfigurationMapping {
+    @Test
+    public void testMappingMapToConfiguration() {
+        final String key = "key";
+        final String value = "value";
+        final ImmutableMap<String, String> map = new ImmutableMap.Builder<String, String>().put(key, value).build();
+
+        final CONFIGURATION configuration
+                = DozerBeanMapperSingleton.instance().map(map, CONFIGURATION.class);
+
+        final List<CONFIGURATION.Property> properties = configuration.getProperty();
+        final CONFIGURATION.Property property = properties.get(0);
+
+        assertEquals(key, property.getName());
+        assertEquals(value, property.getValue());
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestControlNodeMappingBase.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestControlNodeMappingBase.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestControlNodeMappingBase.java
new file mode 100644
index 0000000..c65943c
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestControlNodeMappingBase.java
@@ -0,0 +1,27 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.mapping;
+
+import org.junit.Rule;
+import org.junit.rules.ExpectedException;
+
+public abstract class TestControlNodeMappingBase {
+    @Rule
+    public final ExpectedException expectedException = ExpectedException.none();
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestCredentialsMapping.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestCredentialsMapping.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestCredentialsMapping.java
new file mode 100644
index 0000000..8b08868
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestCredentialsMapping.java
@@ -0,0 +1,37 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.mapping;
+
+import org.apache.oozie.fluentjob.api.generated.workflow.CREDENTIALS;
+import org.apache.oozie.fluentjob.api.workflow.Credentials;
+import org.junit.Test;
+
+public class TestCredentialsMapping {
+
+    private final SourceDataFactory factory = new SourceDataFactory();
+
+    @Test
+    public void testMappingCredentials() {
+        final Credentials source = factory.createCredentials();
+
+        final CREDENTIALS destination = DozerBeanMapperSingleton.instance().map(source, CREDENTIALS.class);
+
+        factory.assertCredentials(destination);
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestDecisionMapping.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestDecisionMapping.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestDecisionMapping.java
new file mode 100644
index 0000000..10a7ad9
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestDecisionMapping.java
@@ -0,0 +1,113 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.mapping;
+
+import org.apache.oozie.fluentjob.api.generated.workflow.DECISION;
+import org.apache.oozie.fluentjob.api.generated.workflow.DEFAULT;
+import org.apache.oozie.fluentjob.api.Condition;
+import org.apache.oozie.fluentjob.api.generated.workflow.CASE;
+import org.apache.oozie.fluentjob.api.generated.workflow.SWITCH;
+import org.apache.oozie.fluentjob.api.dag.Decision;
+import org.apache.oozie.fluentjob.api.dag.DecisionJoin;
+import org.apache.oozie.fluentjob.api.dag.ExplicitNode;
+import org.apache.oozie.fluentjob.api.dag.NodeBase;
+import org.junit.Test;
+
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestDecisionMapping extends TestControlNodeMappingBase {
+    @Test
+    public void testMappingDecision() {
+        final String name = "decision";
+        final Decision decision = new Decision(name);
+
+        final NodeBase child1 = new ExplicitNode("child1", null);
+        final NodeBase child2 = new ExplicitNode("child2", null);
+        final NodeBase defaultChild = new ExplicitNode("defaultChild", null);
+
+        final String condition1String = "condition1";
+        final String condition2String = "condition2";
+
+        child1.addParentWithCondition(decision, Condition.actualCondition(condition1String));
+        child2.addParentWithCondition(decision, Condition.actualCondition(condition2String));
+        defaultChild.addParentDefaultConditional(decision);
+
+        final DECISION mappedDecision = DozerBeanMapperSingleton.instance().map(decision, DECISION.class);
+
+        assertEquals(name, mappedDecision.getName());
+
+        final SWITCH decisionSwitch = mappedDecision.getSwitch();
+        final List<CASE> cases = decisionSwitch.getCase();
+
+        assertEquals(2, cases.size());
+
+        assertEquals(child1.getName(), cases.get(0).getTo());
+        assertEquals(condition1String, cases.get(0).getValue());
+
+        assertEquals(child2.getName(), cases.get(1).getTo());
+        assertEquals(condition2String, cases.get(1).getValue());
+
+        final DEFAULT decisionDefault = decisionSwitch.getDefault();
+        assertEquals(defaultChild.getName(), decisionDefault.getTo());
+    }
+
+    @Test
+    public void testMappingDecisionWithoutDefaultThrows() {
+        final String name = "decision";
+        final Decision decision = new Decision(name);
+
+        final NodeBase child1 = new ExplicitNode("child1", null);
+        final NodeBase child2 = new ExplicitNode("child2", null);
+
+        final Condition condition1 = Condition.actualCondition("condition1");
+        final Condition condition2 = Condition.actualCondition("condition2");
+
+        child1.addParentWithCondition(decision, condition1);
+        child2.addParentWithCondition(decision, condition2);
+
+        expectedException.expect(IllegalStateException.class);
+        DozerBeanMapperSingleton.instance().map(decision, DECISION.class);
+    }
+
+    @Test
+    public void testMappingDecisionWithDecisionJoin() {
+        final String child1Name = "child1";
+        final String child2Name = "child2";
+        final Decision decision = new Decision("decision");
+
+        final NodeBase decisionJoin1 = new DecisionJoin("decisionJoin1", new Decision("decision"));
+        decisionJoin1.addParentWithCondition(decision, Condition.actualCondition("condition"));
+
+        final NodeBase decisionJoin2 = new DecisionJoin("decisionJoin2", new Decision("decision2"));
+        decisionJoin2.addParentDefaultConditional(decision);
+
+        final NodeBase child1 = new ExplicitNode(child1Name, null);
+        child1.addParent(decisionJoin1);
+
+        final NodeBase child2 = new ExplicitNode(child2Name, null);
+        child2.addParent(decisionJoin2);
+
+        final DECISION mappedDecision = DozerBeanMapperSingleton.instance().map(decision, DECISION.class);
+
+        assertEquals(child1Name, mappedDecision.getSwitch().getCase().get(0).getTo());
+        assertEquals(child2Name, mappedDecision.getSwitch().getDefault().getTo());
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestDeleteMapping.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestDeleteMapping.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestDeleteMapping.java
new file mode 100644
index 0000000..7869237
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestDeleteMapping.java
@@ -0,0 +1,38 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.mapping;
+
+import org.apache.oozie.fluentjob.api.action.Delete;
+import org.apache.oozie.fluentjob.api.generated.workflow.DELETE;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestDeleteMapping {
+    @Test
+    public void testMappingDelete() {
+        final String path = "path/to/location";
+        final Boolean skipTrash = true;
+        final Delete delete = new Delete(path, skipTrash);
+
+        final DELETE deleteJAXB = DozerBeanMapperSingleton.instance().map(delete, DELETE.class);
+
+        assertEquals(path, deleteJAXB.getPath());
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestDistcpActionMapping.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestDistcpActionMapping.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestDistcpActionMapping.java
new file mode 100644
index 0000000..55d66c2
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestDistcpActionMapping.java
@@ -0,0 +1,66 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.mapping;
+
+import org.apache.oozie.fluentjob.api.action.DistcpAction;
+import org.apache.oozie.fluentjob.api.action.DistcpActionBuilder;
+import org.apache.oozie.fluentjob.api.action.PrepareBuilder;
+import org.apache.oozie.fluentjob.api.generated.action.distcp.ACTION;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+public class TestDistcpActionMapping {
+    @Test
+    public void testMappingDistcpAction() {
+        final String resourceManager = "${resourceManager}";
+        final String nameNode = "${nameNode}";
+        final String javaOpts = "-Dopt1 -Dopt2";
+        final List<String> args = Arrays.asList("arg1", "arg2");
+
+        final DistcpActionBuilder builder = DistcpActionBuilder.create();
+
+        builder.withResourceManager(resourceManager)
+                .withNameNode(nameNode)
+                .withPrepare(new PrepareBuilder().build())
+                .withJavaOpts(javaOpts);
+
+        for (final String arg : args) {
+            builder.withArg(arg);
+        }
+
+        builder.withConfigProperty("propertyName1", "propertyValue1")
+                .withConfigProperty("propertyName2", "propertyValue2");
+
+        final DistcpAction action = builder.build();
+
+        final ACTION distcp = DozerBeanMapperSingleton.instance().map(action, ACTION.class);
+
+        assertEquals(resourceManager, distcp.getResourceManager());
+        assertEquals(nameNode, distcp.getNameNode());
+        assertNotNull(distcp.getPrepare());
+        assertNotNull(distcp.getConfiguration());
+        assertEquals(javaOpts, distcp.getJavaOpts());
+        assertEquals(args, distcp.getArg());
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestEmailActionMapping.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestEmailActionMapping.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestEmailActionMapping.java
new file mode 100644
index 0000000..0bea733
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestEmailActionMapping.java
@@ -0,0 +1,60 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.mapping;
+
+import org.apache.oozie.fluentjob.api.action.EmailAction;
+import org.apache.oozie.fluentjob.api.action.EmailActionBuilder;
+import org.apache.oozie.fluentjob.api.generated.action.email.ACTION;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestEmailActionMapping {
+    @Test
+    public void testMappingEmailAction() {
+        final String to = "recipient@something.com";
+        final String cc = "cc@something.com";
+        final String bcc = "bcc@something.com";
+        final String subject = "Subject";
+        final String body = "Email body.";
+        final String contentType = "content_type";
+        final String attachment = "attachment";
+
+        final EmailAction action = EmailActionBuilder.create()
+                .withName("email-action")
+                .withRecipient(to)
+                .withCc(cc)
+                .withBcc(bcc)
+                .withSubject(subject)
+                .withBody(body)
+                .withContentType(contentType)
+                .withAttachment(attachment)
+                .build();
+
+        final ACTION emailAction = DozerBeanMapperSingleton.instance().map(action, ACTION.class);
+
+        assertEquals(to, emailAction.getTo());
+        assertEquals(cc, emailAction.getCc());
+        assertEquals(bcc, emailAction.getBcc());
+        assertEquals(subject, emailAction.getSubject());
+        assertEquals(body, emailAction.getBody());
+        assertEquals(contentType, emailAction.getContentType());
+        assertEquals(attachment, emailAction.getAttachment());
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestEndMapping.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestEndMapping.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestEndMapping.java
new file mode 100644
index 0000000..079b8fa
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestEndMapping.java
@@ -0,0 +1,37 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.mapping;
+
+import org.apache.oozie.fluentjob.api.generated.workflow.END;
+import org.apache.oozie.fluentjob.api.dag.End;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestEndMapping extends TestControlNodeMappingBase {
+    @Test
+    public void testMappingEnd() {
+        final String name = "end";
+        final End end = new End(name);
+
+        final END mappedEnd = DozerBeanMapperSingleton.instance().map(end, END.class);
+
+        assertEquals(name, mappedEnd.getName());
+    }
+}


[12/16] oozie git commit: OOZIE-2339 [fluent-job] Minimum Viable Fluent Job API (daniel.becker, andras.piros via rkanter, gezapeti, pbacsko)

Posted by an...@apache.org.
http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/NodeBuilderBaseImpl.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/NodeBuilderBaseImpl.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/NodeBuilderBaseImpl.java
new file mode 100644
index 0000000..a4d37bb
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/NodeBuilderBaseImpl.java
@@ -0,0 +1,340 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
+import com.google.common.collect.ImmutableList;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.oozie.fluentjob.api.Condition;
+import org.apache.oozie.fluentjob.api.ModifyOnce;
+import org.apache.oozie.fluentjob.api.workflow.Credential;
+
+import java.security.SecureRandom;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Locale;
+
+/**
+ * An abstract base class for builders that build concrete instances of subclasses of {@link Node}. This class doesn't
+ * implement the {@link Builder} interface as no type information as to the concrete node to build. The concrete node
+ * builder classes of course should implement {@link Builder}.
+ *
+ * The concrete builders should provide a fluent API, and to facilitate this, the methods in this base class have to
+ * return the concrete builder. Therefore it is templated on the type of the concrete builder class. Although it cannot
+ * be enforced that the provided generic parameter is the same as the class deriving from this class, it definitely
+ * should be, and the constraint on the type parameter tries to minimize the chance that the class is subclassed
+ * incorrectly.
+ *
+ * The properties of the builder can only be set once, an attempt to set them a second time will trigger
+ * an {@link IllegalStateException}. The properties that are lists are an exception to this rule, of course multiple
+ * elements can be added / removed.
+ *
+ * @param <B> The type of the concrete builder class deriving from this class.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public abstract class NodeBuilderBaseImpl <B extends NodeBuilderBaseImpl<B>> {
+    private final ModifyOnce<String> name;
+    private final List<Credential> credentials;
+    private final ModifyOnce<Integer> retryMax;
+    private final ModifyOnce<Integer> retryInterval;
+    private final ModifyOnce<String> retryPolicy;
+    private final List<Node> parents;
+    private final List<Node.NodeWithCondition> parentsWithConditions;
+
+    private final ModifyOnce<ErrorHandler> errorHandler;
+
+    NodeBuilderBaseImpl() {
+        this(null);
+    }
+
+    NodeBuilderBaseImpl(final Node node) {
+        if (node == null) {
+            name = new ModifyOnce<>();
+            credentials = new ArrayList<>();
+            retryMax = new ModifyOnce<>();
+            retryInterval = new ModifyOnce<>();
+            retryPolicy = new ModifyOnce<>();
+            parents = new ArrayList<>();
+            parentsWithConditions = new ArrayList<>();
+            errorHandler = new ModifyOnce<>();
+        }
+        else {
+            // Names won't be copied as we need unique names within a workflow
+            name = new ModifyOnce<>();
+            credentials = new ArrayList<>(node.getCredentials());
+            retryMax = new ModifyOnce<>(node.getRetryMax());
+            retryInterval = new ModifyOnce<>(node.getRetryInterval());
+            retryPolicy = new ModifyOnce<>(node.getRetryPolicy());
+            parents = new ArrayList<>(node.getParentsWithoutConditions());
+            parentsWithConditions = new ArrayList<>(node.getParentsWithConditions());
+            errorHandler = new ModifyOnce<>(node.getErrorHandler());
+        }
+    }
+
+    /**
+     * Registers an error handler with this builder.
+     * @param errorHandler The error handler to register.
+     * @return This builder.
+     */
+    public B withErrorHandler(final ErrorHandler errorHandler) {
+        this.errorHandler.set(errorHandler);
+        return ensureRuntimeSelfReference();
+    }
+
+    /**
+     * Removes the currently registered error handler if any.
+     * @return This builder.
+     */
+    public B withoutErrorHandler() {
+        errorHandler.set(null);
+        return ensureRuntimeSelfReference();
+    }
+
+    /**
+     * Registers a name that will be the name of the action built by this builder.
+     * @param name The name of the action that will be built.
+     * @return This builder.
+     */
+    public B withName(final String name) {
+        this.name.set(name);
+        return ensureRuntimeSelfReference();
+    }
+
+    /**
+     * Registers a {@link Credential} that will be the {@code cred} of the action built by this builder.
+     * @param credential The {@link Credential} of the action that will be built.
+     * @return This builder.
+     */
+    public B withCredential(final Credential credential) {
+        this.credentials.add(credential);
+        return ensureRuntimeSelfReference();
+    }
+
+    /**
+     * Removes a {@link Credential} registered with this builder. If the {@code credential} is not registered with this builder,
+     * this method does nothing.
+     * @param credential The {@link Credential} to remove.
+     * @return This builder.
+     */
+    public B withoutCredential(final Credential credential) {
+        this.credentials.remove(credential);
+        return ensureRuntimeSelfReference();
+    }
+
+    /**
+     * Removes all {@link Credential}s registered with this builder.
+     * @return This builder.
+     */
+    public B clearCredentials() {
+        this.credentials.clear();
+        return ensureRuntimeSelfReference();
+    }
+
+    /**
+     * Registers an {@link Integer} that will be the {@code retry-max} of the action built by this builder.
+     * @param retryMax The {@code retry-max} of the action that will be built.
+     * @return This builder.
+     */
+    public B withRetryMax(final Integer retryMax) {
+        this.retryMax.set(retryMax);
+        return ensureRuntimeSelfReference();
+    }
+
+    /**
+     * Registers an {@link Integer} that will be the {@code retry-interval} of the action built by this builder.
+     * @param retryInterval The {@code retry-interval} of the action that will be built.
+     * @return This builder.
+     */
+    public B withRetryInterval(final Integer retryInterval) {
+        this.retryInterval.set(retryInterval);
+        return ensureRuntimeSelfReference();
+    }
+
+    /**
+     * Registers a {@link String} that will be the {@code retry-policy} of the action built by this builder.
+     * @param retryPolicy The {@code retry-policy} of the action that will be built.
+     * @return This builder.
+     */
+    public B withRetryPolicy(final String retryPolicy) {
+        this.retryPolicy.set(retryPolicy);
+        return ensureRuntimeSelfReference();
+    }
+
+    /**
+     * Registers an unconditional parent with this builder. If the parent is already registered with this builder,
+     * {@link IllegalArgumentException} is thrown.
+     * @param parent The node that will be the parent of the built action.
+     * @return This builder.
+     *
+     * @throws IllegalArgumentException if the provided node is already registered as a parent.
+     */
+    public B withParent(final Node parent) {
+        checkNoDuplicateParent(parent);
+
+        parents.add(parent);
+        return ensureRuntimeSelfReference();
+    }
+
+    /**
+     * Registers a conditional parent with this builder. If the parent is already registered with this builder,
+     * {@link IllegalArgumentException} is thrown.
+     * @param parent The node that will be the parent of the built action.
+     * @param condition The condition of the parent.
+     * @return This builder.
+     *
+     * @throws IllegalArgumentException if the provided node is already registered as a parent.
+     */
+    public B withParentWithCondition(final Node parent, final String condition) {
+        checkNoDuplicateParent(parent);
+
+        parentsWithConditions.add(new Node.NodeWithCondition(parent, Condition.actualCondition(condition)));
+        return ensureRuntimeSelfReference();
+    }
+
+    /**
+     * Registers a conditional parent for which this node is the default transition. If the parent is already registered
+     * with this builder, {@link IllegalArgumentException} is thrown.
+     * {@link IllegalArgumentException} is thrown.
+     * @param parent The node that will be the parent of the built action.
+     * @return This builder.
+     *
+     * @throws IllegalArgumentException if the provided node is already registered as a parent.
+     */
+    public B withParentDefaultConditional(final Node parent) {
+        checkNoDuplicateParent(parent);
+
+        parentsWithConditions.add(new Node.NodeWithCondition(parent, Condition.defaultCondition()));
+        return ensureRuntimeSelfReference();
+    }
+
+    /**
+     * Removes a parent registered with this builder. If the parent is not registered with this builder, this method
+     * does nothing.
+     * @param parent The parent to remove.
+     * @return This builder.
+     */
+    public B withoutParent(final Node parent) {
+        if (parents.contains(parent)) {
+            parents.remove(parent);
+        } else {
+            int index = indexOfParentAmongParentsWithConditions(parent);
+            if (index >= 0) {
+                parentsWithConditions.remove(index);
+            }
+        }
+
+        return ensureRuntimeSelfReference();
+    }
+
+    /**
+     * Removes all parents registered with this builder.
+     * @return This builder.
+     */
+    public B clearParents() {
+        parents.clear();
+        parentsWithConditions.clear();
+        return ensureRuntimeSelfReference();
+    }
+
+    final B ensureRuntimeSelfReference() {
+        final B runtimeSelfReference = getRuntimeSelfReference();
+
+        Preconditions.checkState(runtimeSelfReference == this, "The builder type B doesn't extend NodeBuilderBaseImpl<B>.");
+
+        return runtimeSelfReference;
+    }
+
+    private void checkNoDuplicateParent(final Node parent) {
+        boolean parentsContains = parents.contains(parent);
+        boolean parentsWithConditionsContains = indexOfParentAmongParentsWithConditions(parent) != -1;
+
+        Preconditions.checkArgument(!parentsContains && !parentsWithConditionsContains,
+                "Trying to add a parent that is already a parent of this node.");
+    }
+
+    private int indexOfParentAmongParentsWithConditions(final Node parent) {
+        for (int i = 0; i < parentsWithConditions.size(); ++i) {
+            if (parent == parentsWithConditions.get(i).getNode()) {
+                return i;
+            }
+        }
+
+        return -1;
+    }
+
+    protected void addAsChildToAllParents(final Node child) {
+        final List<Node> parentsList = child.getParentsWithoutConditions();
+        if (parentsList != null) {
+            for (final Node parent : parentsList) {
+                parent.addChild(child);
+            }
+        }
+
+        final List<Node.NodeWithCondition> parentsWithConditionsList = child.getParentsWithConditions();
+        if (parentsWithConditionsList != null) {
+            for (final Node.NodeWithCondition parentWithCondition : parentsWithConditionsList) {
+                final Node parent = parentWithCondition.getNode();
+                final Condition condition = parentWithCondition.getCondition();
+
+                if (condition.isDefault()) {
+                    parent.addChildAsDefaultConditional(child);
+                }
+                else {
+                    parent.addChildWithCondition(child, condition.getCondition());
+                }
+            }
+        }
+    }
+
+    Node.ConstructionData getConstructionData() {
+        final String nameStr = ensureName();
+
+        final ImmutableList<Node> parentsList = new ImmutableList.Builder<Node>().addAll(parents).build();
+        final ImmutableList<Node.NodeWithCondition> parentsWithConditionsList
+                = new ImmutableList.Builder<Node.NodeWithCondition>().addAll(parentsWithConditions).build();
+
+        return new Node.ConstructionData(
+                new Node.Attributes(nameStr,
+                        credentials,
+                        retryMax.get(),
+                        retryInterval.get(),
+                        retryPolicy.get()),
+                parentsList,
+                parentsWithConditionsList,
+                errorHandler.get()
+        );
+    }
+
+    private String ensureName() {
+        if (Strings.isNullOrEmpty(this.name.get())) {
+            final String type = getRuntimeSelfReference().getClass().getSimpleName()
+                    .toLowerCase(Locale.getDefault()).replaceAll("actionbuilder", "");
+            final int randomSuffix = new SecureRandom().nextInt(1_000_000_000);
+
+            this.name.set(String.format("%s-%d", type, randomSuffix));
+        }
+
+        return this.name.get();
+    }
+
+    protected abstract B getRuntimeSelfReference();
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/PigAction.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/PigAction.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/PigAction.java
new file mode 100644
index 0000000..d77c539
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/PigAction.java
@@ -0,0 +1,107 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+import java.util.List;
+import java.util.Map;
+
+/**
+ * A class representing the Oozie Pig action.
+ * Instances of this class should be built using the builder {@link PigActionBuilder}.
+ *
+ * The properties of the builder can only be set once, an attempt to set them a second time will trigger
+ * an {@link IllegalStateException}.
+ *
+ * Builder instances can be used to build several elements, although properties already set cannot be changed after
+ * a call to {@link PigActionBuilder#build} either.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class PigAction extends Node implements HasAttributes {
+    private final ActionAttributes attributes;
+    private final String script;
+    private final ImmutableList<String> params;
+
+    PigAction(final ConstructionData constructionData,
+              final ActionAttributes attributes,
+              final String script,
+              final ImmutableList<String> params) {
+        super(constructionData);
+
+        this.attributes = attributes;
+        this.script = script;
+        this.params = params;
+    }
+
+    public String getResourceManager() {
+        return attributes.getResourceManager();
+    }
+
+    public String getNameNode() {
+        return attributes.getNameNode();
+    }
+
+    public Prepare getPrepare() {
+        return attributes.getPrepare();
+    }
+
+    public Launcher getLauncher() {
+        return attributes.getLauncher();
+    }
+
+    public List<String> getJobXmls() {
+        return attributes.getJobXmls();
+    }
+
+    public String getConfigProperty(final String property) {
+        return attributes.getConfiguration().get(property);
+    }
+
+    public Map<String, String> getConfiguration() {
+        return attributes.getConfiguration();
+    }
+
+    public String getScript() {
+        return script;
+    }
+
+    public List<String> getParams() {
+        return params;
+    }
+
+    public List<String> getArgs() {
+        return attributes.getArgs();
+    }
+
+    public List<String> getFiles() {
+        return attributes.getFiles();
+    }
+
+    public List<String> getArchives() {
+        return attributes.getArchives();
+    }
+
+    public ActionAttributes getAttributes() {
+        return attributes;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/PigActionBuilder.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/PigActionBuilder.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/PigActionBuilder.java
new file mode 100644
index 0000000..aa1ff16
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/PigActionBuilder.java
@@ -0,0 +1,215 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.oozie.fluentjob.api.ModifyOnce;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * A builder class for {@link PigAction}.
+ *
+ * The properties of the builder can only be set once, an attempt to set them a second time will trigger
+ * an {@link IllegalStateException}. The properties that are lists are an exception to this rule, of course multiple
+ * elements can be added / removed.
+ *
+ * Builder instances can be used to build several elements, although properties already set cannot be changed after
+ * a call to {@link PigActionBuilder#build} either.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class PigActionBuilder extends NodeBuilderBaseImpl<PigActionBuilder> implements Builder<PigAction> {
+    protected final ActionAttributesBuilder attributesBuilder;
+    protected final ModifyOnce<String> script;
+    protected final List<String> params;
+
+    public static PigActionBuilder create() {
+        final ActionAttributesBuilder builder = ActionAttributesBuilder.create();
+        final ModifyOnce<String> script = new ModifyOnce<>();
+        final List<String> params = new ArrayList<>();
+
+        return new PigActionBuilder(
+                null,
+                builder,
+                script,
+                params);
+    }
+
+    public static PigActionBuilder createFromExistingAction(final PigAction action) {
+        final ActionAttributesBuilder builder = ActionAttributesBuilder.createFromExisting(action.getAttributes());
+        final ModifyOnce<String> script = new ModifyOnce<>(action.getScript());
+        final List<String> params = new ArrayList<>(action.getParams());
+
+        return new PigActionBuilder(action,
+                builder,
+                script,
+                params);
+    }
+
+    public static PigActionBuilder createFromExistingAction(final Node action) {
+        final ActionAttributesBuilder builder = ActionAttributesBuilder.createFromAction(action);
+        final ModifyOnce<String> script = new ModifyOnce<>();
+        final List<String> params = new ArrayList<>();
+
+        return new PigActionBuilder(action,
+                builder,
+                script,
+                params);
+    }
+
+    PigActionBuilder(final Node action,
+                     final ActionAttributesBuilder attributesBuilder,
+                     final ModifyOnce<String> script,
+                     final List<String> params) {
+        super(action);
+
+        this.attributesBuilder = attributesBuilder;
+        this.script = script;
+        this.params = params;
+    }
+
+    public PigActionBuilder withResourceManager(final String resourceManager) {
+        this.attributesBuilder.withResourceManager(resourceManager);
+        return this;
+    }
+
+    public PigActionBuilder withNameNode(final String nameNode) {
+        this.attributesBuilder.withNameNode(nameNode);
+        return this;
+    }
+
+    public PigActionBuilder withPrepare(final Prepare prepare) {
+        this.attributesBuilder.withPrepare(prepare);
+        return this;
+    }
+
+    public PigActionBuilder withLauncher(final Launcher launcher) {
+        this.attributesBuilder.withLauncher(launcher);
+        return this;
+    }
+
+    public PigActionBuilder withJobXml(final String jobXml) {
+        this.attributesBuilder.withJobXml(jobXml);
+        return this;
+    }
+
+    public PigActionBuilder withoutJobXml(final String jobXml) {
+        this.attributesBuilder.withoutJobXml(jobXml);
+        return this;
+    }
+
+    public PigActionBuilder clearJobXmls() {
+        this.attributesBuilder.clearJobXmls();
+        return this;
+    }
+
+    public PigActionBuilder withConfigProperty(final String key, final String value) {
+        this.attributesBuilder.withConfigProperty(key, value);
+        return this;
+    }
+
+    public PigActionBuilder withScript(final String script) {
+        this.script.set(script);
+        return this;
+    }
+
+    public PigActionBuilder withParam(final String param) {
+        this.params.add(param);
+        return this;
+    }
+
+    public PigActionBuilder withoutParam(final String param) {
+        this.params.remove(param);
+        return this;
+    }
+
+    public PigActionBuilder clearParams() {
+        this.params.clear();
+        return this;
+    }
+
+    public PigActionBuilder withArg(final String arg) {
+        this.attributesBuilder.withArg(arg);
+        return this;
+    }
+
+    public PigActionBuilder withoutArg(final String arg) {
+        this.attributesBuilder.withoutArg(arg);
+        return this;
+    }
+
+    public PigActionBuilder clearArgs() {
+        this.attributesBuilder.clearArgs();
+        return this;
+    }
+
+    public PigActionBuilder withFile(final String file) {
+        this.attributesBuilder.withFile(file);
+        return this;
+    }
+
+    public PigActionBuilder withoutFile(final String file) {
+        this.attributesBuilder.withoutFile(file);
+        return this;
+    }
+
+    public PigActionBuilder clearFiles() {
+        this.attributesBuilder.clearFiles();
+        return this;
+    }
+
+    public PigActionBuilder withArchive(final String archive) {
+        this.attributesBuilder.withArchive(archive);
+        return this;
+    }
+
+    public PigActionBuilder withoutArchive(final String archive) {
+        this.attributesBuilder.withoutArchive(archive);
+        return this;
+    }
+
+    public PigActionBuilder clearArchives() {
+        this.attributesBuilder.clearArchives();
+        return this;
+    }
+
+    @Override
+    public PigAction build() {
+        final Node.ConstructionData constructionData = getConstructionData();
+
+        final PigAction instance = new PigAction(
+                constructionData,
+                attributesBuilder.build(),
+                script.get(),
+                ImmutableList.copyOf(params));
+
+        addAsChildToAllParents(instance);
+
+        return instance;
+    }
+
+    @Override
+    protected PigActionBuilder getRuntimeSelfReference() {
+        return this;
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/Pipes.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/Pipes.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/Pipes.java
new file mode 100644
index 0000000..526a202
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/Pipes.java
@@ -0,0 +1,106 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * A class representing the piping information within a {@link MapReduceAction}.
+ *
+ * Instances of this class should be built using the builder {@link PipesBuilder}.
+ *
+ * The properties of the builder can only be set once, an attempt to set them a second time will trigger
+ * an {@link IllegalStateException}.
+ *
+ * Builder instances can be used to build several elements, although properties already set cannot be changed after
+ * a call to {@link PipesBuilder#build} either.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class Pipes {
+    private final String map;
+    private final String reduce;
+    private final String inputformat;
+    private final String partitioner;
+    private final String writer;
+    private final String program;
+
+    Pipes(final String map,
+          final String reduce,
+          final String inputformat,
+          final String partitioner,
+          final String writer,
+          final String program) {
+        this.map = map;
+        this.reduce = reduce;
+        this.inputformat = inputformat;
+        this.partitioner = partitioner;
+        this.writer = writer;
+        this.program = program;
+    }
+
+    /**
+     * Returns the mapper of this {@link Pipes} object.
+     * @return The mapper of this {@link Pipes} object.
+     */
+    public String getMap() {
+        return map;
+    }
+
+    /**
+     * Returns the reducer of this {@link Pipes} object.
+     * @return The reducer of this {@link Pipes} object.
+     */
+    public String getReduce() {
+        return reduce;
+    }
+
+    /**
+     * Returns the input format of this {@link Pipes} object.
+     * @return The input format of this {@link Pipes} object.
+     */
+    public String getInputformat() {
+        return inputformat;
+    }
+
+    /**
+     * Returns the partitioner of this {@link Pipes} object.
+     * @return The partitioner of this {@link Pipes} object.
+     */
+    public String getPartitioner() {
+        return partitioner;
+    }
+
+    /**
+     * Returns the writer of this {@link Pipes} object.
+     * @return The writer of this {@link Pipes} object.
+     */
+    public String getWriter() {
+        return writer;
+    }
+
+    /**
+     * Returns the program of this {@link Pipes} object.
+     * @return The program of this {@link Pipes} object.
+     */
+    public String getProgram() {
+        return program;
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/PipesBuilder.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/PipesBuilder.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/PipesBuilder.java
new file mode 100644
index 0000000..0699dba
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/PipesBuilder.java
@@ -0,0 +1,127 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.oozie.fluentjob.api.ModifyOnce;
+
+/**
+ * A builder class for {@link Pipes}.
+ *
+ * The properties of the builder can only be set once, an attempt to set them a second time will trigger
+ * an {@link IllegalStateException}. The properties that are lists are an exception to this rule, of course multiple
+ * elements can be added / removed.
+ *
+ * Builder instances can be used to build several elements, although properties already set cannot be changed after
+ * a call to {@link PipesBuilder#build} either.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class PipesBuilder implements Builder<Pipes> {
+    private final ModifyOnce<String> map;
+    private final ModifyOnce<String> reduce;
+    private final ModifyOnce<String> inputformat;
+    private final ModifyOnce<String> partitioner;
+    private final ModifyOnce<String> writer;
+    private final ModifyOnce<String> program;
+
+    /**
+     * Creates a new {@link PipesBuilder}.
+     */
+    public PipesBuilder() {
+        map = new ModifyOnce<>();
+        reduce = new ModifyOnce<>();
+        inputformat = new ModifyOnce<>();
+        partitioner = new ModifyOnce<>();
+        writer = new ModifyOnce<>();
+        program = new ModifyOnce<>();
+    }
+
+    /**
+     * Registers a mapper with this builder.
+     * @param map The mapper to register with this builder.
+     * @return This builder.
+     */
+    public PipesBuilder withMap(final String map) {
+        this.map.set(map);
+        return this;
+    }
+
+    /**
+     * Registers a reducer with this builder.
+     * @param reduce The reducer to register with this builder.
+     * @return This builder.
+     */
+    public PipesBuilder withReduce(final String reduce) {
+        this.reduce.set(reduce);
+        return this;
+    }
+
+    /**
+     * Registers an input format with this builder.
+     * @param inputformat The input format to register with this builder.
+     * @return This builder.
+     */
+    public PipesBuilder withInputformat(final String inputformat) {
+        this.inputformat.set(inputformat);
+        return this;
+    }
+
+    /**
+     * Registers a partitioner with this builder.
+     * @param partitioner The partitioner to register with this builder.
+     * @return This builder.
+     */
+    public PipesBuilder withPartitioner(final String partitioner) {
+        this.partitioner.set(partitioner);
+        return this;
+    }
+
+    /**
+     * Registers a writer with this builder.
+     * @param writer The writer to register with this builder.
+     * @return This builder.
+     */
+    public PipesBuilder withWriter(final String writer) {
+        this.writer.set(writer);
+        return this;
+    }
+
+    /**
+     * Registers an executable program with this builder.
+     * @param program The executable program to register with this builder.
+     * @return This builder.
+     */
+    public PipesBuilder withProgram(final String program) {
+        this.program.set(program);
+        return this;
+    }
+
+    /**
+     * Creates a new {@link Pipes} object with the properties stores in this builder.
+     * The new {@link Pipes} object is independent of this builder and the builder can be used to build
+     * new instances.
+     * @return A new {@link Pipes} object with the properties stored in this builder.
+     */
+    @Override
+    public Pipes build() {
+        return new Pipes(map.get(), reduce.get(), inputformat.get(), partitioner.get(), writer.get(), program.get());
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/Prepare.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/Prepare.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/Prepare.java
new file mode 100644
index 0000000..2d121a2
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/Prepare.java
@@ -0,0 +1,58 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+import java.util.List;
+
+/**
+ * A class representing the prepare section of various actions.
+ *
+ * Instances of this class should be built using the builder {@link PrepareBuilder}.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class Prepare {
+    private final ImmutableList<Delete> deletes;
+    private final ImmutableList<Mkdir> mkdirs;
+
+    Prepare(final ImmutableList<Delete> deletes, final ImmutableList<Mkdir> mkdirs) {
+        this.deletes = deletes;
+        this.mkdirs = mkdirs;
+    }
+
+    /**
+     * Returns the {@link Delete} objects that specify which directories or files will be deleted.
+     * @return The {@link Delete} objects that specify which directories or files will be deleted.
+     */
+    public List<Delete> getDeletes() {
+        return deletes;
+    }
+
+    /**
+     * Returns the {@link Mkdir} objects that specify which directories will be created.
+     * @return The {@link Mkdir} objects that specify which directories will be created.
+     */
+    public List<Mkdir> getMkdirs() {
+        return mkdirs;
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/PrepareBuilder.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/PrepareBuilder.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/PrepareBuilder.java
new file mode 100644
index 0000000..22042e9
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/PrepareBuilder.java
@@ -0,0 +1,85 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * A builder class for {@link Prepare}.
+ * <p>
+ * Builder instances can be used to build several elements, although properties already set cannot be changed after
+ * a call to {@link FSActionBuilder#build} either.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class PrepareBuilder implements Builder<Prepare> {
+    private final ImmutableList.Builder<Delete> deletes;
+    private final ImmutableList.Builder<Mkdir> mkdirs;
+
+    public PrepareBuilder() {
+        deletes = new ImmutableList.Builder<>();
+        mkdirs = new ImmutableList.Builder<>();
+    }
+
+    /**
+     * Registers a {@link Delete} object with this builder. The {@link Delete} object will have the provided path as
+     * its target and the default value (true) for skip-trash.
+     * @param path The target of the {@link Delete} object.
+     * @return this
+     */
+    public PrepareBuilder withDelete(final String path) {
+        return withDelete(path, null);
+    }
+
+    /**
+     * Registers a {@link Delete} object with this builder. The {@link Delete} object will have the provided path as
+     * its target and the given boolean value for skip-trash.
+     * @param path The target of the {@link Delete} object.
+     * @param skipTrash Whether to skip trash when deleting the items.
+     * @return this
+     */
+    public PrepareBuilder withDelete(final String path, final Boolean skipTrash) {
+        deletes.add(new Delete(path, skipTrash));
+        return this;
+    }
+
+    /**
+     * Registers a {@link Mkdir} object with this builder The {@link Mkdir} object will have the provided path as
+     * its target.
+     * @param path The target of the {@link Mkdir}.
+     * @return this
+     */
+    public PrepareBuilder withMkdir(final String path) {
+        mkdirs.add(new Mkdir(path));
+        return this;
+    }
+
+    /**
+     * Creates a new {@link Prepare} object with the properties stores in this builder.
+     * The new {@link Prepare} object is independent of this builder and the builder can be used to build
+     * new instances.
+     * @return A new {@link Prepare} object with the properties stored in this builder.
+     */
+    @Override
+    public Prepare build() {
+        return new Prepare(deletes.build(), mkdirs.build());
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/ShellAction.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/ShellAction.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/ShellAction.java
new file mode 100644
index 0000000..c1b79aa
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/ShellAction.java
@@ -0,0 +1,111 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+import java.util.List;
+import java.util.Map;
+
+/**
+ * A class representing the Oozie shell action.
+ * Instances of this class should be built using the builder {@link EmailActionBuilder}.
+ *
+ * The properties of the builder can only be set once, an attempt to set them a second time will trigger
+ * an {@link IllegalStateException}.
+ *
+ * Builder instances can be used to build several elements, although properties already set cannot be changed after
+ * a call to {@link ShellActionBuilder#build} either.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class ShellAction extends Node implements HasAttributes {
+    private final ActionAttributes attributes;
+    private final String executable;
+    private final ImmutableList<String> environmentVariables;
+
+    ShellAction(final ConstructionData constructionData,
+                final ActionAttributes attributes,
+                final String executable,
+                final ImmutableList<String> environmentVariables) {
+        super(constructionData);
+
+        this.attributes = attributes;
+        this.executable = executable;
+        this.environmentVariables = environmentVariables;
+    }
+
+    public String getResourceManager() {
+        return attributes.getResourceManager();
+    }
+
+    public String getNameNode() {
+        return attributes.getNameNode();
+    }
+
+    public Prepare getPrepare() {
+        return attributes.getPrepare();
+    }
+
+    public Launcher getLauncher() {
+        return attributes.getLauncher();
+    }
+
+    public List<String> getJobXmls() {
+        return attributes.getJobXmls();
+    }
+
+    public String getConfigProperty(final String property) {
+        return attributes.getConfiguration().get(property);
+    }
+
+    public Map<String, String> getConfiguration() {
+        return attributes.getConfiguration();
+    }
+
+    public String getExecutable() {
+        return executable;
+    }
+
+    public List<String> getArguments() {
+        return attributes.getArgs();
+    }
+
+    public List<String> getEnvironmentVariables() {
+        return environmentVariables;
+    }
+
+    public List<String> getFiles() {
+        return attributes.getFiles();
+    }
+
+    public List<String> getArchives() {
+        return attributes.getArchives();
+    }
+
+    public boolean isCaptureOutput() {
+        return attributes.isCaptureOutput();
+    }
+
+    public ActionAttributes getAttributes() {
+        return attributes;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/ShellActionBuilder.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/ShellActionBuilder.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/ShellActionBuilder.java
new file mode 100644
index 0000000..21b4192
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/ShellActionBuilder.java
@@ -0,0 +1,220 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.oozie.fluentjob.api.ModifyOnce;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * A builder class for {@link ShellAction}.
+ *
+ * The properties of the builder can only be set once, an attempt to set them a second time will trigger
+ * an {@link IllegalStateException}. The properties that are lists are an exception to this rule, of course multiple
+ * elements can be added / removed.
+ *
+ * Builder instances can be used to build several elements, although properties already set cannot be changed after
+ * a call to {@link ShellActionBuilder#build} either.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class ShellActionBuilder extends NodeBuilderBaseImpl<ShellActionBuilder> implements Builder<ShellAction> {
+    private final ActionAttributesBuilder attributesBuilder;
+    private final ModifyOnce<String> executable;
+    private final List<String> environmentVariables;
+
+    public static ShellActionBuilder create() {
+        final ActionAttributesBuilder builder = ActionAttributesBuilder.create();
+        final ModifyOnce<String> executable = new ModifyOnce<>();
+        final List<String> environmentVariables = new ArrayList<>();
+
+        return new ShellActionBuilder(
+                null,
+                builder,
+                executable,
+                environmentVariables);
+    }
+
+    public static ShellActionBuilder createFromExistingAction(final ShellAction action) {
+        final ActionAttributesBuilder builder = ActionAttributesBuilder.createFromExisting(action.getAttributes());
+        final ModifyOnce<String> executable = new ModifyOnce<>(action.getExecutable());
+        final List<String> environmentVariables = new ArrayList<>(action.getEnvironmentVariables());
+
+        return new ShellActionBuilder(action,
+                builder,
+                executable,
+                environmentVariables);
+    }
+
+    public static ShellActionBuilder createFromExistingAction(final Node action) {
+        final ActionAttributesBuilder builder = ActionAttributesBuilder.createFromAction(action);
+        final ModifyOnce<String> executable = new ModifyOnce<>();
+        final List<String> environmentVariables = new ArrayList<>();
+
+        return new ShellActionBuilder(action,
+                builder,
+                executable,
+                environmentVariables);
+    }
+
+    private ShellActionBuilder(final Node action,
+                               final ActionAttributesBuilder attributesBuilder,
+                               final ModifyOnce<String> executable,
+                               final List<String> environmentVariables) {
+        super(action);
+
+        this.attributesBuilder = attributesBuilder;
+        this.executable = executable;
+        this.environmentVariables = environmentVariables;
+    }
+
+    public ShellActionBuilder withResourceManager(final String resourceManager) {
+        this.attributesBuilder.withResourceManager(resourceManager);
+        return this;
+    }
+
+    public ShellActionBuilder withNameNode(final String nameNode) {
+        this.attributesBuilder.withNameNode(nameNode);
+        return this;
+    }
+
+    public ShellActionBuilder withPrepare(final Prepare prepare) {
+        this.attributesBuilder.withPrepare(prepare);
+        return this;
+    }
+
+    public ShellActionBuilder withLauncher(final Launcher launcher) {
+        this.attributesBuilder.withLauncher(launcher);
+        return this;
+    }
+
+    public ShellActionBuilder withJobXml(final String jobXml) {
+        this.attributesBuilder.withJobXml(jobXml);
+        return this;
+    }
+
+    public ShellActionBuilder withoutJobXml(final String jobXml) {
+        this.attributesBuilder.withoutJobXml(jobXml);
+        return this;
+    }
+
+    public ShellActionBuilder clearJobXmls() {
+        this.attributesBuilder.clearJobXmls();
+        return this;
+    }
+
+    public ShellActionBuilder withConfigProperty(final String key, final String value) {
+        this.attributesBuilder.withConfigProperty(key, value);
+        return this;
+    }
+
+    public ShellActionBuilder withExecutable(final String executable) {
+        this.executable.set(executable);
+        return this;
+    }
+
+    public ShellActionBuilder withArgument(final String argument) {
+        this.attributesBuilder.withArg(argument);
+        return this;
+    }
+
+    public ShellActionBuilder withoutArgument(final String argument) {
+        this.attributesBuilder.withoutArg(argument);
+        return this;
+    }
+
+    public ShellActionBuilder clearArguments() {
+        this.attributesBuilder.clearArgs();
+        return this;
+    }
+
+    public ShellActionBuilder withEnvironmentVariable(final String environmentVariable) {
+        this.environmentVariables.add(environmentVariable);
+        return this;
+    }
+
+    public ShellActionBuilder withoutEnvironmentVariable(final String environmentVariable) {
+        this.environmentVariables.remove(environmentVariable);
+        return this;
+    }
+
+    public ShellActionBuilder clearEnvironmentVariables() {
+        this.environmentVariables.clear();
+        return this;
+    }
+
+    public ShellActionBuilder withFile(final String file) {
+        this.attributesBuilder.withFile(file);
+        return this;
+    }
+
+    public ShellActionBuilder withoutFile(final String file) {
+        this.attributesBuilder.withoutFile(file);
+        return this;
+    }
+
+    public ShellActionBuilder clearFiles() {
+        this.attributesBuilder.clearFiles();
+        return this;
+    }
+
+    public ShellActionBuilder withArchive(final String archive) {
+        this.attributesBuilder.withArchive(archive);
+        return this;
+    }
+
+    public ShellActionBuilder withoutArchive(final String archive) {
+        this.attributesBuilder.withoutArchive(archive);
+        return this;
+    }
+
+    public ShellActionBuilder clearArchives() {
+        this.attributesBuilder.clearArchives();
+        return this;
+    }
+
+    public ShellActionBuilder withCaptureOutput(final Boolean captureOutput) {
+        this.attributesBuilder.withCaptureOutput(captureOutput);
+        return this;
+    }
+
+    @Override
+    public ShellAction build() {
+        final Node.ConstructionData constructionData = getConstructionData();
+
+        final ShellAction instance = new ShellAction(
+                constructionData,
+                attributesBuilder.build(),
+                executable.get(),
+                ImmutableList.copyOf(environmentVariables));
+
+        addAsChildToAllParents(instance);
+
+        return instance;
+    }
+
+    @Override
+    protected ShellActionBuilder getRuntimeSelfReference() {
+        return this;
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/SparkAction.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/SparkAction.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/SparkAction.java
new file mode 100644
index 0000000..9db041d
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/SparkAction.java
@@ -0,0 +1,134 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+import java.util.List;
+import java.util.Map;
+
+/**
+ * A class representing the Oozie Spark action.
+ * Instances of this class should be built using the builder {@link SparkActionBuilder}.
+ *
+ * The properties of the builder can only be set once, an attempt to set them a second time will trigger
+ * an {@link IllegalStateException}.
+ *
+ * Builder instances can be used to build several elements, although properties already set cannot be changed after
+ * a call to {@link SparkActionBuilder#build} either.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class SparkAction extends Node implements HasAttributes {
+    private final ActionAttributes attributes;
+    private final String master;
+    private final String mode;
+    private final String actionName;
+    private final String actionClass;
+    private final String jar;
+    private final String sparkOpts;
+
+    public SparkAction(final ConstructionData constructionData,
+                       final ActionAttributes attributes,
+                       final String master,
+                       final String mode,
+                       final String actionName,
+                       final String actionClass,
+                       final String jar,
+                       final String sparkOpts) {
+        super(constructionData);
+
+        this.attributes = attributes;
+        this.master = master;
+        this.mode = mode;
+        this.actionName = actionName;
+        this.actionClass = actionClass;
+        this.jar = jar;
+        this.sparkOpts = sparkOpts;
+    }
+
+    public String getResourceManager() {
+        return attributes.getResourceManager();
+    }
+
+    public String getNameNode() {
+        return attributes.getNameNode();
+    }
+
+    public Prepare getPrepare() {
+        return attributes.getPrepare();
+    }
+
+    public Launcher getLauncher() {
+        return attributes.getLauncher();
+    }
+
+    public List<String> getJobXmls() {
+        return attributes.getJobXmls();
+    }
+
+    public String getConfigProperty(final String property) {
+        return attributes.getConfiguration().get(property);
+    }
+
+    public Map<String, String> getConfiguration() {
+        return attributes.getConfiguration();
+    }
+
+    public String getMaster() {
+        return master;
+    }
+
+    public String getMode() {
+        return mode;
+    }
+
+    public String getActionName() {
+        return actionName;
+    }
+
+    public String getActionClass() {
+        return actionClass;
+    }
+
+    public String getJar() {
+        return jar;
+    }
+
+    public String getSparkOpts() {
+        return sparkOpts;
+    }
+
+    public List<String> getArgs() {
+        return attributes.getArgs();
+    }
+
+    public List<String> getFiles() {
+        return attributes.getFiles();
+    }
+
+    public List<String> getArchives() {
+        return attributes.getArchives();
+    }
+
+    public ActionAttributes getAttributes() {
+        return attributes;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/SparkActionBuilder.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/SparkActionBuilder.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/SparkActionBuilder.java
new file mode 100644
index 0000000..abc1122
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/SparkActionBuilder.java
@@ -0,0 +1,261 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.oozie.fluentjob.api.ModifyOnce;
+
+/**
+ * A builder class for {@link SparkAction}.
+ *
+ * The properties of the builder can only be set once, an attempt to set them a second time will trigger
+ * an {@link IllegalStateException}. The properties that are lists are an exception to this rule, of course multiple
+ * elements can be added / removed.
+ *
+ * Builder instances can be used to build several elements, although properties already set cannot be changed after
+ * a call to {@link SparkActionBuilder#build} either.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class SparkActionBuilder extends NodeBuilderBaseImpl<SparkActionBuilder> implements Builder<SparkAction> {
+    private final ActionAttributesBuilder attributesBuilder;
+    private final ModifyOnce<String> master;
+    private final ModifyOnce<String> mode;
+    private final ModifyOnce<String> actionName;
+    private final ModifyOnce<String> actionClass;
+    private final ModifyOnce<String> jar;
+    private final ModifyOnce<String> sparkOpts;
+
+    public static SparkActionBuilder create() {
+        final ActionAttributesBuilder builder = ActionAttributesBuilder.create();
+        final ModifyOnce<String> master = new ModifyOnce<>();
+        final ModifyOnce<String> mode = new ModifyOnce<>();
+        final ModifyOnce<String> actionName = new ModifyOnce<>();
+        final ModifyOnce<String> actionClass = new ModifyOnce<>();
+        final ModifyOnce<String> jar = new ModifyOnce<>();
+        final ModifyOnce<String> sparkOpts = new ModifyOnce<>();
+
+        return new SparkActionBuilder(
+                null,
+                builder,
+                master,
+                mode,
+                actionName,
+                actionClass,
+                jar,
+                sparkOpts);
+    }
+
+    public static SparkActionBuilder createFromExistingAction(final SparkAction action) {
+        final ActionAttributesBuilder builder = ActionAttributesBuilder.createFromExisting(action.getAttributes());
+        final ModifyOnce<String> master = new ModifyOnce<>(action.getMaster());
+        final ModifyOnce<String> mode = new ModifyOnce<>(action.getMode());
+        final ModifyOnce<String> actionName = new ModifyOnce<>(action.getActionName());
+        final ModifyOnce<String> actionClass = new ModifyOnce<>(action.getActionClass());
+        final ModifyOnce<String> jar = new ModifyOnce<>(action.getJar());
+        final ModifyOnce<String> sparkOpts = new ModifyOnce<>(action.getSparkOpts());
+
+        return new SparkActionBuilder(action,
+                builder,
+                master,
+                mode,
+                actionName,
+                actionClass,
+                jar,
+                sparkOpts);
+    }
+
+    public static SparkActionBuilder createFromExistingAction(final Node action) {
+        final ActionAttributesBuilder builder = ActionAttributesBuilder.createFromAction(action);
+        final ModifyOnce<String> master = new ModifyOnce<>();
+        final ModifyOnce<String> mode = new ModifyOnce<>();
+        final ModifyOnce<String> actionName = new ModifyOnce<>();
+        final ModifyOnce<String> actionClass = new ModifyOnce<>();
+        final ModifyOnce<String> jar = new ModifyOnce<>();
+        final ModifyOnce<String> sparkOpts = new ModifyOnce<>();
+
+        return new SparkActionBuilder(action,
+                builder,
+                master,
+                mode,
+                actionName,
+                actionClass,
+                jar,
+                sparkOpts);
+    }
+
+    SparkActionBuilder(final Node action,
+                       final ActionAttributesBuilder attributesBuilder,
+                       final ModifyOnce<String> master,
+                       final ModifyOnce<String> mode,
+                       final ModifyOnce<String> actionName,
+                       final ModifyOnce<String> actionClass,
+                       final ModifyOnce<String> jar,
+                       final ModifyOnce<String> sparkOpts) {
+        super(action);
+
+        this.attributesBuilder = attributesBuilder;
+        this.master = master;
+        this.mode = mode;
+        this.actionName = actionName;
+        this.actionClass = actionClass;
+        this.jar = jar;
+        this.sparkOpts = sparkOpts;
+    }
+
+    public SparkActionBuilder withResourceManager(final String resourceManager) {
+        this.attributesBuilder.withResourceManager(resourceManager);
+        return this;
+    }
+
+    public SparkActionBuilder withNameNode(final String nameNode) {
+        this.attributesBuilder.withNameNode(nameNode);
+        return this;
+    }
+
+    public SparkActionBuilder withPrepare(final Prepare prepare) {
+        this.attributesBuilder.withPrepare(prepare);
+        return this;
+    }
+
+    public SparkActionBuilder withLauncher(final Launcher launcher) {
+        this.attributesBuilder.withLauncher(launcher);
+        return this;
+    }
+
+    public SparkActionBuilder withJobXml(final String jobXml) {
+        this.attributesBuilder.withJobXml(jobXml);
+        return this;
+    }
+
+    public SparkActionBuilder withoutJobXml(final String jobXml) {
+        this.attributesBuilder.withoutJobXml(jobXml);
+        return this;
+    }
+
+    public SparkActionBuilder clearJobXmls() {
+        this.attributesBuilder.clearJobXmls();
+        return this;
+    }
+
+    public SparkActionBuilder withConfigProperty(final String key, final String value) {
+        this.attributesBuilder.withConfigProperty(key, value);
+        return this;
+    }
+
+    public SparkActionBuilder withMaster(final String master) {
+        this.master.set(master);
+        return this;
+    }
+
+    public SparkActionBuilder withMode(final String mode) {
+        this.mode.set(mode);
+        return this;
+    }
+
+    public SparkActionBuilder withActionName(final String actionName) {
+        this.actionName.set(actionName);
+        return this;
+    }
+
+    public SparkActionBuilder withActionClass(final String actionClass) {
+        this.actionClass.set(actionClass);
+        return this;
+    }
+
+    public SparkActionBuilder withJar(final String jar) {
+        this.jar.set(jar);
+        return this;
+    }
+
+    public SparkActionBuilder withSparkOpts(final String sparkOpts) {
+        this.sparkOpts.set(sparkOpts);
+        return this;
+    }
+
+    public SparkActionBuilder withArg(final String arg) {
+        this.attributesBuilder.withArg(arg);
+        return this;
+    }
+
+    public SparkActionBuilder withoutArg(final String arg) {
+        this.attributesBuilder.withoutArg(arg);
+        return this;
+    }
+
+    public SparkActionBuilder clearArgs() {
+        this.attributesBuilder.clearArgs();
+        return this;
+    }
+
+    public SparkActionBuilder withFile(final String file) {
+        this.attributesBuilder.withFile(file);
+        return this;
+    }
+
+    public SparkActionBuilder withoutFile(final String file) {
+        this.attributesBuilder.withoutFile(file);
+        return this;
+    }
+
+    public SparkActionBuilder clearFiles() {
+        this.attributesBuilder.clearFiles();
+        return this;
+    }
+
+    public SparkActionBuilder withArchive(final String archive) {
+        this.attributesBuilder.withArchive(archive);
+        return this;
+    }
+
+    public SparkActionBuilder withoutArchive(final String archive) {
+        this.attributesBuilder.withoutArchive(archive);
+        return this;
+    }
+
+    public SparkActionBuilder clearArchive() {
+        this.attributesBuilder.clearArchives();
+        return this;
+    }
+
+    @Override
+    public SparkAction build() {
+        final Node.ConstructionData constructionData = getConstructionData();
+
+        final SparkAction instance = new SparkAction(
+                constructionData,
+                attributesBuilder.build(),
+                master.get(),
+                mode.get(),
+                actionName.get(),
+                actionClass.get(),
+                jar.get(),
+                sparkOpts.get());
+
+        addAsChildToAllParents(instance);
+
+        return instance;
+    }
+
+    @Override
+    protected SparkActionBuilder getRuntimeSelfReference() {
+        return this;
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/SqoopAction.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/SqoopAction.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/SqoopAction.java
new file mode 100644
index 0000000..3fbf08a
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/SqoopAction.java
@@ -0,0 +1,99 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+import java.util.List;
+import java.util.Map;
+
+/**
+ * A class representing the Oozie Sqoop action.
+ * Instances of this class should be built using the builder {@link SqoopActionBuilder}.
+ *
+ * The properties of the builder can only be set once, an attempt to set them a second time will trigger
+ * an {@link IllegalStateException}.
+ *
+ * Builder instances can be used to build several elements, although properties already set cannot be changed after
+ * a call to {@link SqoopActionBuilder#build} either.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class SqoopAction extends Node implements HasAttributes {
+    private final ActionAttributes attributes;
+    private final String command;
+
+    SqoopAction(final ConstructionData constructionData,
+                final ActionAttributes attributes,
+                final String command) {
+        super(constructionData);
+
+        this.attributes = attributes;
+        this.command = command;
+    }
+
+    public String getResourceManager() {
+        return attributes.getResourceManager();
+    }
+
+    public String getNameNode() {
+        return attributes.getNameNode();
+    }
+
+    public Prepare getPrepare() {
+        return attributes.getPrepare();
+    }
+
+    public Launcher getLauncher() {
+        return attributes.getLauncher();
+    }
+
+    public List<String> getJobXmls() {
+        return attributes.getJobXmls();
+    }
+
+    public String getConfigProperty(final String property) {
+        return attributes.getConfiguration().get(property);
+    }
+
+    public Map<String, String> getConfiguration() {
+        return attributes.getConfiguration();
+    }
+
+    public String getCommand() {
+        return command;
+    }
+
+    public List<String> getArguments() {
+        return attributes.getArgs();
+    }
+
+    public List<String> getFiles() {
+        return attributes.getFiles();
+    }
+
+    public List<String> getArchives() {
+        return attributes.getArchives();
+    }
+
+    public ActionAttributes getAttributes() {
+        return attributes;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/SqoopActionBuilder.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/SqoopActionBuilder.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/SqoopActionBuilder.java
new file mode 100644
index 0000000..8b7e7e5
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/SqoopActionBuilder.java
@@ -0,0 +1,186 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.oozie.fluentjob.api.ModifyOnce;
+
+/**
+ * A builder class for {@link SqoopAction}.
+ *
+ * The properties of the builder can only be set once, an attempt to set them a second time will trigger
+ * an {@link IllegalStateException}. The properties that are lists are an exception to this rule, of course multiple
+ * elements can be added / removed.
+ *
+ * Builder instances can be used to build several elements, although properties already set cannot be changed after
+ * a call to {@link SqoopActionBuilder#build} either.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class SqoopActionBuilder extends NodeBuilderBaseImpl<SqoopActionBuilder> implements Builder<SqoopAction> {
+    private final ActionAttributesBuilder attributesBuilder;
+    private final ModifyOnce<String> command;
+
+    public static SqoopActionBuilder create() {
+        final ActionAttributesBuilder builder = ActionAttributesBuilder.create();
+        final ModifyOnce<String> command = new ModifyOnce<>();
+
+        return new SqoopActionBuilder(
+                null,
+                builder,
+                command);
+    }
+
+    public static SqoopActionBuilder createFromExistingAction(final SqoopAction action) {
+        final ActionAttributesBuilder builder = ActionAttributesBuilder.createFromExisting(action.getAttributes());
+        final ModifyOnce<String> command = new ModifyOnce<>(action.getCommand());
+
+        return new SqoopActionBuilder(action,
+                builder,
+                command);
+    }
+
+    public static SqoopActionBuilder createFromExistingAction(final Node action) {
+        final ActionAttributesBuilder builder = ActionAttributesBuilder.createFromAction(action);
+        final ModifyOnce<String> command = new ModifyOnce<>();
+
+        return new SqoopActionBuilder(action,
+                builder,
+                command);
+    }
+
+    private SqoopActionBuilder(final Node action,
+                               final ActionAttributesBuilder attributesBuilder,
+                               final ModifyOnce<String> command) {
+        super(action);
+
+        this.attributesBuilder = attributesBuilder;
+        this.command = command;
+    }
+
+    public SqoopActionBuilder withResourceManager(final String resourceManager) {
+        this.attributesBuilder.withResourceManager(resourceManager);
+        return this;
+    }
+
+    public SqoopActionBuilder withNameNode(final String nameNode) {
+        this.attributesBuilder.withNameNode(nameNode);
+        return this;
+    }
+
+    public SqoopActionBuilder withPrepare(final Prepare prepare) {
+        this.attributesBuilder.withPrepare(prepare);
+        return this;
+    }
+
+    public SqoopActionBuilder withLauncher(final Launcher launcher) {
+        this.attributesBuilder.withLauncher(launcher);
+        return this;
+    }
+
+    public SqoopActionBuilder withJobXml(final String jobXml) {
+        this.attributesBuilder.withJobXml(jobXml);
+        return this;
+    }
+
+    public SqoopActionBuilder withoutJobXml(final String jobXml) {
+        this.attributesBuilder.withoutJobXml(jobXml);
+        return this;
+    }
+
+    public SqoopActionBuilder clearJobXmls() {
+        this.attributesBuilder.clearJobXmls();
+        return this;
+    }
+
+    public SqoopActionBuilder withConfigProperty(final String key, final String value) {
+        this.attributesBuilder.withConfigProperty(key, value);
+        return this;
+    }
+
+    public SqoopActionBuilder withCommand(final String command) {
+        this.command.set(command);
+        return this;
+    }
+
+    public SqoopActionBuilder withArgument(final String argument) {
+        this.attributesBuilder.withArg(argument);
+        return this;
+    }
+
+    public SqoopActionBuilder withoutArgument(final String argument) {
+        this.attributesBuilder.withoutArg(argument);
+        return this;
+    }
+
+    public SqoopActionBuilder clearArguments() {
+        this.attributesBuilder.clearArgs();
+        return this;
+    }
+
+    public SqoopActionBuilder withFile(final String file) {
+        this.attributesBuilder.withFile(file);
+        return this;
+    }
+
+    public SqoopActionBuilder withoutFile(final String file) {
+        this.attributesBuilder.withoutFile(file);
+        return this;
+    }
+
+    public SqoopActionBuilder clearFiles() {
+        this.attributesBuilder.clearFiles();
+        return this;
+    }
+
+    public SqoopActionBuilder withArchive(final String archive) {
+        this.attributesBuilder.withArchive(archive);
+        return this;
+    }
+
+    public SqoopActionBuilder withoutArchive(final String archive) {
+        this.attributesBuilder.withoutArchive(archive);
+        return this;
+    }
+
+    public SqoopActionBuilder clearArchives() {
+        this.attributesBuilder.clearArchives();
+        return this;
+    }
+
+    @Override
+    public SqoopAction build() {
+        final Node.ConstructionData constructionData = getConstructionData();
+
+        final SqoopAction instance = new SqoopAction(
+                constructionData,
+                attributesBuilder.build(),
+                command.get());
+
+        addAsChildToAllParents(instance);
+
+        return instance;
+    }
+
+    @Override
+    protected SqoopActionBuilder getRuntimeSelfReference() {
+        return this;
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/SshAction.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/SshAction.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/SshAction.java
new file mode 100644
index 0000000..8e73ce6
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/SshAction.java
@@ -0,0 +1,74 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+import java.util.List;
+
+/**
+ * A class representing the Oozie ssh action.
+ * Instances of this class should be built using the builder {@link SshActionBuilder}.
+ *
+ * The properties of the builder can only be set once, an attempt to set them a second time will trigger
+ * an {@link IllegalStateException}.
+ *
+ * Builder instances can be used to build several elements, although properties already set cannot be changed after
+ * a call to {@link SshActionBuilder#build} either.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class SshAction extends Node implements HasAttributes {
+    private final ActionAttributes attributes;
+    private final String host;
+    private final String command;
+
+
+    SshAction(final ConstructionData constructionData,
+              final ActionAttributes attributes,
+              final String host,
+              final String command) {
+        super(constructionData);
+
+        this.attributes = attributes;
+        this.host = host;
+        this.command = command;
+    }
+
+    public String getHost() {
+        return host;
+    }
+
+    public String getCommand() {
+        return command;
+    }
+
+    public List<String> getArgs() {
+        return attributes.getArgs();
+    }
+
+    public boolean isCaptureOutput() {
+        return attributes.isCaptureOutput();
+    }
+
+    public ActionAttributes getAttributes() {
+        return attributes;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/SshActionBuilder.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/SshActionBuilder.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/SshActionBuilder.java
new file mode 100644
index 0000000..1da74eb
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/SshActionBuilder.java
@@ -0,0 +1,136 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.oozie.fluentjob.api.ModifyOnce;
+
+/**
+ * A builder class for {@link EmailAction}.
+ *
+ * The properties of the builder can only be set once, an attempt to set them a second time will trigger
+ * an {@link IllegalStateException}. The properties that are lists are an exception to this rule, of course multiple
+ * elements can be added / removed.
+ *
+ * Builder instances can be used to build several elements, although properties already set cannot be changed after
+ * a call to {@link SshActionBuilder#build} either.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class SshActionBuilder extends NodeBuilderBaseImpl<SshActionBuilder> implements Builder<SshAction> {
+    private final ActionAttributesBuilder attributesBuilder;
+    private final ModifyOnce<String> host;
+    private final ModifyOnce<String> command;
+
+    public static SshActionBuilder create() {
+        final ActionAttributesBuilder builder = ActionAttributesBuilder.create();
+        final ModifyOnce<String> host = new ModifyOnce<>();
+        final ModifyOnce<String> command = new ModifyOnce<>();
+
+        return new SshActionBuilder(
+                null,
+                builder,
+                host,
+                command);
+    }
+
+    public static SshActionBuilder createFromExistingAction(final SshAction action) {
+        final ActionAttributesBuilder builder = ActionAttributesBuilder.createFromExisting(action.getAttributes());
+        final ModifyOnce<String> host = new ModifyOnce<>(action.getHost());
+        final ModifyOnce<String> command = new ModifyOnce<>(action.getCommand());
+
+        return new SshActionBuilder(action,
+                builder,
+                host,
+                command);
+    }
+
+    public static SshActionBuilder createFromExistingAction(final Node action) {
+        final ActionAttributesBuilder builder = ActionAttributesBuilder.createFromAction(action);
+        final ModifyOnce<String> host = new ModifyOnce<>();
+        final ModifyOnce<String> command = new ModifyOnce<>();
+
+        return new SshActionBuilder(action,
+                builder,
+                host,
+                command);
+    }
+
+    private SshActionBuilder(final Node action,
+                             final ActionAttributesBuilder attributesBuilder,
+                             final ModifyOnce<String> host,
+                             final ModifyOnce<String> command) {
+        super(action);
+
+        this.attributesBuilder = attributesBuilder;
+        this.host = host;
+        this.command = command;
+    }
+
+    public SshActionBuilder withHost(final String host) {
+        this.host.set(host);
+        return this;
+    }
+
+    public SshActionBuilder withCommand(final String command) {
+        this.command.set(command);
+        return this;
+    }
+
+    public SshActionBuilder withArg(final String arg) {
+        this.attributesBuilder.withArg(arg);
+        return this;
+    }
+
+    public SshActionBuilder withoutArg(final String arg) {
+        this.attributesBuilder.withoutArg(arg);
+        return this;
+    }
+
+    public SshActionBuilder clearArgs() {
+        this.attributesBuilder.clearArgs();
+        return this;
+    }
+
+    public SshActionBuilder withCaptureOutput(final Boolean captureOutput) {
+        this.attributesBuilder.withCaptureOutput(captureOutput);
+        return this;
+    }
+
+    @Override
+    public SshAction build() {
+        final Node.ConstructionData constructionData = getConstructionData();
+
+        final SshAction instance = new SshAction(
+                constructionData,
+                attributesBuilder.build(),
+                host.get(),
+                command.get());
+
+        addAsChildToAllParents(instance);
+
+        return instance;
+    }
+
+    @Override
+    protected SshActionBuilder getRuntimeSelfReference() {
+        return this;
+    }
+}


[13/16] oozie git commit: OOZIE-2339 [fluent-job] Minimum Viable Fluent Job API (daniel.becker, andras.piros via rkanter, gezapeti, pbacsko)

Posted by an...@apache.org.
http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/HasAttributes.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/HasAttributes.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/HasAttributes.java
new file mode 100644
index 0000000..9aec459
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/HasAttributes.java
@@ -0,0 +1,31 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Designates whether the implementing POJO has {@link ActionAttributes}.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public interface HasAttributes {
+    ActionAttributes getAttributes();
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/Hive2Action.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/Hive2Action.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/Hive2Action.java
new file mode 100644
index 0000000..fbaf55d
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/Hive2Action.java
@@ -0,0 +1,65 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * A class representing the Oozie Hive2 action.
+ * Instances of this class should be built using the builder {@link Hive2ActionBuilder}.
+ *
+ * The properties of the builder can only be set once, an attempt to set them a second time will trigger
+ * an {@link IllegalStateException}.
+ *
+ * Builder instances can be used to build several elements, although properties already set cannot be changed after
+ * a call to {@link Hive2ActionBuilder#build} either.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class Hive2Action extends HiveAction {
+    private final String jdbcUrl;
+    private final String password;
+
+    public Hive2Action(final ConstructionData constructionData,
+                       final ActionAttributes attributes,
+                       final String jdbcUrl,
+                       final String password,
+                       final String script,
+                       final String query,
+                       final ImmutableList<String> params) {
+        super(constructionData,
+              attributes,
+              script,
+              query,
+              params);
+
+        this.jdbcUrl = jdbcUrl;
+        this.password = password;
+    }
+
+    public String getJdbcUrl() {
+        return jdbcUrl;
+    }
+
+    public String getPassword() {
+        return password;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/Hive2ActionBuilder.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/Hive2ActionBuilder.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/Hive2ActionBuilder.java
new file mode 100644
index 0000000..63b8f47
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/Hive2ActionBuilder.java
@@ -0,0 +1,259 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.oozie.fluentjob.api.ModifyOnce;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * A builder class for {@link Hive2Action}.
+ *
+ * The properties of the builder can only be set once, an attempt to set them a second time will trigger
+ * an {@link IllegalStateException}. The properties that are lists are an exception to this rule, of course multiple
+ * elements can be added / removed.
+ *
+ * Builder instances can be used to build several elements, although properties already set cannot be changed after
+ * a call to {@link Hive2ActionBuilder#build} either.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class Hive2ActionBuilder extends NodeBuilderBaseImpl<Hive2ActionBuilder> implements Builder<Hive2Action> {
+    private final HiveActionBuilder delegate;
+    private final ModifyOnce<String> jdbcUrl;
+    private final ModifyOnce<String> password;
+
+    public static Hive2ActionBuilder create() {
+        final ActionAttributesBuilder builder = ActionAttributesBuilder.create();
+        final ModifyOnce<String> jdbcUrl = new ModifyOnce<>();
+        final ModifyOnce<String> password = new ModifyOnce<>();
+        final ModifyOnce<String> script = new ModifyOnce<>();
+        final ModifyOnce<String> query = new ModifyOnce<>();
+        final List<String> params = new ArrayList<>();
+
+        return new Hive2ActionBuilder(
+                null,
+                builder,
+                jdbcUrl,
+                password,
+                script,
+                query,
+                params);
+    }
+
+    public static Hive2ActionBuilder createFromExistingAction(final Hive2Action action) {
+        final ActionAttributesBuilder builder = ActionAttributesBuilder.createFromExisting(action.getAttributes());
+        final ModifyOnce<String> jdbcUrl = new ModifyOnce<>(action.getJdbcUrl());
+        final ModifyOnce<String> password = new ModifyOnce<>(action.getPassword());
+        final ModifyOnce<String> script = new ModifyOnce<>(action.getScript());
+        final ModifyOnce<String> query = new ModifyOnce<>(action.getQuery());
+        final List<String> params = new ArrayList<>(action.getParams());
+
+        return new Hive2ActionBuilder(action,
+                builder,
+                jdbcUrl,
+                password,
+                script,
+                query,
+                params);
+    }
+
+    public static Hive2ActionBuilder createFromExistingAction(final Node action) {
+        final ActionAttributesBuilder builder = ActionAttributesBuilder.createFromAction(action);
+        final ModifyOnce<String> jdbcUrl = new ModifyOnce<>();
+        final ModifyOnce<String> password = new ModifyOnce<>();
+        final ModifyOnce<String> script = new ModifyOnce<>();
+        final ModifyOnce<String> query = new ModifyOnce<>();
+        final List<String> params = new ArrayList<>();
+
+        return new Hive2ActionBuilder(action,
+                builder,
+                jdbcUrl,
+                password,
+                script,
+                query,
+                params);
+    }
+
+    Hive2ActionBuilder(final Node action,
+                       final ActionAttributesBuilder attributesBuilder,
+                       final ModifyOnce<String> jdbcUrl,
+                       final ModifyOnce<String> password,
+                       final ModifyOnce<String> script,
+                       final ModifyOnce<String> query,
+                       final List<String> params) {
+        super(action);
+
+        this.delegate = new HiveActionBuilder(action,
+                attributesBuilder,
+                script,
+                query,
+                params);
+
+        this.jdbcUrl = jdbcUrl;
+        this.password = password;
+    }
+
+    public Hive2ActionBuilder withResourceManager(final String resourceManager) {
+        delegate.withResourceManager(resourceManager);
+        return this;
+    }
+
+    public Hive2ActionBuilder withNameNode(final String nameNode) {
+        delegate.withNameNode(nameNode);
+        return this;
+    }
+
+    public Hive2ActionBuilder withPrepare(final Prepare prepare) {
+        delegate.withPrepare(prepare);
+        return this;
+    }
+
+    public Hive2ActionBuilder withLauncher(final Launcher launcher) {
+        delegate.withLauncher(launcher);
+        return this;
+    }
+
+    public Hive2ActionBuilder withJobXml(final String jobXml) {
+        delegate.withJobXml(jobXml);
+        return this;
+    }
+
+    public Hive2ActionBuilder withoutJobXml(final String jobXml) {
+        delegate.withoutJobXml(jobXml);
+        return this;
+    }
+
+    public Hive2ActionBuilder clearJobXmls() {
+        delegate.clearJobXmls();
+        return this;
+    }
+
+    public Hive2ActionBuilder withConfigProperty(final String key, final String value) {
+        delegate.withConfigProperty(key, value);
+        return this;
+    }
+
+    public Hive2ActionBuilder withScript(final String script) {
+        delegate.withScript(script);
+        return this;
+    }
+
+    public Hive2ActionBuilder withQuery(final String query) {
+        delegate.withQuery(query);
+        return this;
+    }
+
+    public Hive2ActionBuilder withParam(final String param) {
+        delegate.withParam(param);
+        return this;
+    }
+
+    public Hive2ActionBuilder withoutParam(final String param) {
+        delegate.withoutParam(param);
+        return this;
+    }
+
+    public Hive2ActionBuilder clearParams() {
+        delegate.clearParams();
+        return this;
+    }
+
+    public Hive2ActionBuilder withArg(final String arg) {
+        delegate.withArg(arg);
+        return this;
+    }
+
+    public Hive2ActionBuilder withoutArg(final String arg) {
+        delegate.withoutArg(arg);
+        return this;
+    }
+
+    public Hive2ActionBuilder clearArgs() {
+        delegate.clearArgs();
+        return this;
+    }
+
+    public Hive2ActionBuilder withFile(final String file) {
+        delegate.withFile(file);
+        return this;
+    }
+
+    public Hive2ActionBuilder withoutFile(final String file) {
+        delegate.withoutFile(file);
+        return this;
+    }
+
+    public Hive2ActionBuilder clearFiles() {
+        delegate.clearFiles();
+        return this;
+    }
+
+    public Hive2ActionBuilder withArchive(final String archive) {
+        delegate.withArchive(archive);
+        return this;
+    }
+
+    public Hive2ActionBuilder withoutArchive(final String archive) {
+        delegate.withoutArchive(archive);
+        return this;
+    }
+
+    public Hive2ActionBuilder clearArchives() {
+        delegate.clearArchives();
+        return this;
+    }
+
+    public Hive2ActionBuilder withJdbcUrl(final String jdbcUrl) {
+        this.jdbcUrl.set(jdbcUrl);
+        return this;
+    }
+
+    public Hive2ActionBuilder withPassword(final String password) {
+        this.password.set(password);
+        return this;
+    }
+
+    @Override
+    public Hive2Action build() {
+        final Node.ConstructionData constructionData = getConstructionData();
+
+        final Hive2Action instance = new Hive2Action(
+                constructionData,
+                delegate.getAttributesBuilder().build(),
+                jdbcUrl.get(),
+                password.get(),
+                delegate.getScript().get(),
+                delegate.query.get(),
+                ImmutableList.copyOf(delegate.getParams()));
+
+        addAsChildToAllParents(instance);
+
+        return instance;
+    }
+
+    @Override
+    protected Hive2ActionBuilder getRuntimeSelfReference() {
+        return this;
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/HiveAction.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/HiveAction.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/HiveAction.java
new file mode 100644
index 0000000..7b12624
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/HiveAction.java
@@ -0,0 +1,56 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * A class representing the Oozie Hive action.
+ * Instances of this class should be built using the builder {@link HiveActionBuilder}.
+ *
+ * The properties of the builder can only be set once, an attempt to set them a second time will trigger
+ * an {@link IllegalStateException}.
+ *
+ * Builder instances can be used to build several elements, although properties already set cannot be changed after
+ * a call to {@link HiveActionBuilder#build} either.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class HiveAction extends PigAction {
+    private final String query;
+
+    HiveAction(final ConstructionData constructionData,
+               final ActionAttributes attributes,
+               final String script,
+               final String query,
+               final ImmutableList<String> params) {
+        super(constructionData,
+              attributes,
+              script,
+              params);
+
+        this.query = query;
+    }
+
+    public String getQuery() {
+        return query;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/HiveActionBuilder.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/HiveActionBuilder.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/HiveActionBuilder.java
new file mode 100644
index 0000000..86bb398
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/HiveActionBuilder.java
@@ -0,0 +1,242 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.oozie.fluentjob.api.ModifyOnce;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * A builder class for {@link HiveAction}.
+ *
+ * The properties of the builder can only be set once, an attempt to set them a second time will trigger
+ * an {@link IllegalStateException}. The properties that are lists are an exception to this rule, of course multiple
+ * elements can be added / removed.
+ *
+ * Builder instances can be used to build several elements, although properties already set cannot be changed after
+ * a call to {@link HiveActionBuilder#build} either.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class HiveActionBuilder extends NodeBuilderBaseImpl<HiveActionBuilder> implements Builder<HiveAction> {
+    private final PigActionBuilder delegate;
+    protected final ModifyOnce<String> query;
+
+    public static HiveActionBuilder create() {
+        final ActionAttributesBuilder builder = ActionAttributesBuilder.create();
+        final ModifyOnce<String> script = new ModifyOnce<>();
+        final ModifyOnce<String> query = new ModifyOnce<>();
+        final List<String> params = new ArrayList<>();
+
+        return new HiveActionBuilder(
+                null,
+                builder,
+                script,
+                query,
+                params);
+    }
+
+    public static HiveActionBuilder createFromExistingAction(final HiveAction action) {
+        final ActionAttributesBuilder builder = ActionAttributesBuilder.createFromExisting(action.getAttributes());
+        final ModifyOnce<String> script = new ModifyOnce<>(action.getScript());
+        final ModifyOnce<String> query = new ModifyOnce<>(action.getQuery());
+        final List<String> params = new ArrayList<>(action.getParams());
+
+        return new HiveActionBuilder(action,
+                builder,
+                script,
+                query,
+                params);
+    }
+
+    public static HiveActionBuilder createFromExistingAction(final Node action) {
+        final ActionAttributesBuilder builder = ActionAttributesBuilder.createFromAction(action);
+        final ModifyOnce<String> script = new ModifyOnce<>();
+        final ModifyOnce<String> query = new ModifyOnce<>();
+        final List<String> params = new ArrayList<>();
+
+        return new HiveActionBuilder(action,
+                builder,
+                script,
+                query,
+                params);
+    }
+
+    HiveActionBuilder(final Node action,
+                      final ActionAttributesBuilder attributesBuilder,
+                      final ModifyOnce<String> script,
+                      final ModifyOnce<String> query,
+                      final List<String> params) {
+        super(action);
+
+        this.delegate = new PigActionBuilder(action,
+                                             attributesBuilder,
+                                             script,
+                                             params);
+
+        this.query = query;
+    }
+
+    public HiveActionBuilder withResourceManager(final String resourceManager) {
+        this.delegate.withResourceManager(resourceManager);
+        return this;
+    }
+
+    public HiveActionBuilder withNameNode(final String nameNode) {
+        this.delegate.withNameNode(nameNode);
+        return this;
+    }
+
+    public HiveActionBuilder withPrepare(final Prepare prepare) {
+        this.delegate.withPrepare(prepare);
+        return this;
+    }
+
+    public HiveActionBuilder withLauncher(final Launcher launcher) {
+        this.delegate.withLauncher(launcher);
+        return this;
+    }
+
+    public HiveActionBuilder withJobXml(final String jobXml) {
+        this.delegate.withJobXml(jobXml);
+        return this;
+    }
+
+    public HiveActionBuilder withoutJobXml(final String jobXml) {
+        this.delegate.withoutJobXml(jobXml);
+        return this;
+    }
+
+    public HiveActionBuilder clearJobXmls() {
+        this.delegate.clearJobXmls();
+        return this;
+    }
+
+    public HiveActionBuilder withConfigProperty(final String key, final String value) {
+        this.delegate.withConfigProperty(key, value);
+        return this;
+    }
+
+    public HiveActionBuilder withScript(final String script) {
+        this.delegate.withScript(script);
+        return this;
+    }
+
+    public HiveActionBuilder withQuery(final String query) {
+        this.query.set(query);
+        return this;
+    }
+
+    public HiveActionBuilder withParam(final String param) {
+        this.delegate.withParam(param);
+        return this;
+    }
+
+    public HiveActionBuilder withoutParam(final String param) {
+        this.delegate.withoutParam(param);
+        return this;
+    }
+
+    public HiveActionBuilder clearParams() {
+        this.delegate.clearParams();
+        return this;
+    }
+
+    public HiveActionBuilder withArg(final String arg) {
+        this.delegate.withArg(arg);
+        return this;
+    }
+
+    public HiveActionBuilder withoutArg(final String arg) {
+        this.delegate.withoutArg(arg);
+        return this;
+    }
+
+    public HiveActionBuilder clearArgs() {
+        this.delegate.clearArgs();
+        return this;
+    }
+
+    public HiveActionBuilder withFile(final String file) {
+        this.delegate.withFile(file);
+        return this;
+    }
+
+    public HiveActionBuilder withoutFile(final String file) {
+        this.delegate.withoutFile(file);
+        return this;
+    }
+
+    public HiveActionBuilder clearFiles() {
+        this.delegate.clearFiles();
+        return this;
+    }
+
+    public HiveActionBuilder withArchive(final String archive) {
+        this.delegate.withArchive(archive);
+        return this;
+    }
+
+    public HiveActionBuilder withoutArchive(final String archive) {
+        this.delegate.withoutArchive(archive);
+        return this;
+    }
+
+    public HiveActionBuilder clearArchives() {
+        this.delegate.clearArchives();
+        return this;
+    }
+
+    ActionAttributesBuilder getAttributesBuilder() {
+        return delegate.attributesBuilder;
+    }
+
+    ModifyOnce<String> getScript() {
+        return delegate.script;
+    }
+
+    List<String> getParams() {
+        return delegate.params;
+    }
+
+    @Override
+    public HiveAction build() {
+        final Node.ConstructionData constructionData = getConstructionData();
+
+        final HiveAction instance = new HiveAction(
+                constructionData,
+                getAttributesBuilder().build(),
+                getScript().get(),
+                query.get(),
+                ImmutableList.copyOf(getParams()));
+
+        addAsChildToAllParents(instance);
+
+        return instance;
+    }
+
+    @Override
+    protected HiveActionBuilder getRuntimeSelfReference() {
+        return this;
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/JavaAction.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/JavaAction.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/JavaAction.java
new file mode 100644
index 0000000..cadc825
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/JavaAction.java
@@ -0,0 +1,119 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+import java.util.List;
+import java.util.Map;
+
+/**
+ * A class representing the Oozie Java action.
+ * Instances of this class should be built using the builder {@link JavaActionBuilder}.
+ *
+ * The properties of the builder can only be set once, an attempt to set them a second time will trigger
+ * an {@link IllegalStateException}.
+ *
+ * Builder instances can be used to build several elements, although properties already set cannot be changed after
+ * a call to {@link JavaActionBuilder#build} either.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class JavaAction extends Node {
+    private final ActionAttributes attributes;
+    private final String mainClass;
+    private final String javaOptsString;
+    private final ImmutableList<String> javaOpts;
+
+
+    JavaAction(final ConstructionData constructionData,
+               final ActionAttributes attributes,
+               final String mainClass,
+               final String javaOptsString,
+               final ImmutableList<String> javaOpts) {
+        super(constructionData);
+
+        this.attributes = attributes;
+        this.mainClass = mainClass;
+        this.javaOptsString = javaOptsString;
+        this.javaOpts = javaOpts;
+    }
+
+    public String getResourceManager() {
+        return attributes.getResourceManager();
+    }
+
+    public String getNameNode() {
+        return attributes.getNameNode();
+    }
+
+    public Prepare getPrepare() {
+        return attributes.getPrepare();
+    }
+
+    public Launcher getLauncher() {
+        return attributes.getLauncher();
+    }
+
+    public List<String> getJobXmls() {
+        return attributes.getJobXmls();
+    }
+
+    public String getConfigProperty(final String property) {
+        return attributes.getConfiguration().get(property);
+    }
+
+    public Map<String, String> getConfiguration() {
+        return attributes.getConfiguration();
+    }
+
+    public String getMainClass() {
+        return mainClass;
+    }
+
+    public String getJavaOptsString() {
+        return javaOptsString;
+    }
+
+    public List<String> getJavaOpts() {
+        return javaOpts;
+    }
+
+    public List<String> getArgs() {
+        return attributes.getArgs();
+    }
+
+    public List<String> getFiles() {
+        return attributes.getFiles();
+    }
+
+    public List<String> getArchives() {
+        return attributes.getArchives();
+    }
+
+    public boolean isCaptureOutput() {
+        return attributes.isCaptureOutput();
+    }
+
+    ActionAttributes getAttributes() {
+        return attributes;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/JavaActionBuilder.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/JavaActionBuilder.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/JavaActionBuilder.java
new file mode 100644
index 0000000..af991cd
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/JavaActionBuilder.java
@@ -0,0 +1,235 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.oozie.fluentjob.api.ModifyOnce;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * A builder class for {@link JavaAction}.
+ *
+ * The properties of the builder can only be set once, an attempt to set them a second time will trigger
+ * an {@link IllegalStateException}. The properties that are lists are an exception to this rule, of course multiple
+ * elements can be added / removed.
+ *
+ * Builder instances can be used to build several elements, although properties already set cannot be changed after
+ * a call to {@link JavaActionBuilder#build} either.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class JavaActionBuilder extends NodeBuilderBaseImpl<JavaActionBuilder> implements Builder<JavaAction> {
+    private final ActionAttributesBuilder attributesBuilder;
+    private final ModifyOnce<String> mainClass;
+    private final ModifyOnce<String> javaOptsString;
+    private final List<String> javaOpts;
+
+    public static JavaActionBuilder create() {
+        final ActionAttributesBuilder builder = ActionAttributesBuilder.create();
+        final ModifyOnce<String> mainClass = new ModifyOnce<>();
+        final ModifyOnce<String> javaOptsString = new ModifyOnce<>();
+        final List<String> javaOpts = new ArrayList<>();
+
+        return new JavaActionBuilder(
+                null,
+                builder,
+                mainClass,
+                javaOptsString,
+                javaOpts);
+    }
+
+    public static JavaActionBuilder createFromExistingAction(final JavaAction action) {
+        final ActionAttributesBuilder builder = ActionAttributesBuilder.createFromExisting(action.getAttributes());
+        final ModifyOnce<String> mainClass = new ModifyOnce<>(action.getMainClass());
+        final ModifyOnce<String> javaOptsString = new ModifyOnce<>(action.getJavaOptsString());
+        final List<String> javaOpts = new ArrayList<>(action.getJavaOpts());
+
+        return new JavaActionBuilder(action,
+                builder,
+                mainClass,
+                javaOptsString,
+                javaOpts);
+    }
+
+    public static JavaActionBuilder createFromExistingAction(final Node action) {
+        final ActionAttributesBuilder builder = ActionAttributesBuilder.createFromAction(action);
+        final ModifyOnce<String> mainClass = new ModifyOnce<>();
+        final ModifyOnce<String> javaOptsString = new ModifyOnce<>();
+        final List<String> javaOpts = new ArrayList<>();
+
+        return new JavaActionBuilder(action,
+                builder,
+                mainClass,
+                javaOptsString,
+                javaOpts);
+    }
+
+    private JavaActionBuilder(final Node action,
+                              final ActionAttributesBuilder attributesBuilder,
+                              final ModifyOnce<String> mainClass,
+                              final ModifyOnce<String> javaOptsString,
+                              final List<String> javaOpts) {
+        super(action);
+
+        this.attributesBuilder = attributesBuilder;
+        this.mainClass = mainClass;
+        this.javaOptsString = javaOptsString;
+        this.javaOpts = javaOpts;
+    }
+
+    public JavaActionBuilder withResourceManager(final String resourceManager) {
+        this.attributesBuilder.withResourceManager(resourceManager);
+        return this;
+    }
+
+    public JavaActionBuilder withNameNode(final String nameNode) {
+        this.attributesBuilder.withNameNode(nameNode);
+        return this;
+    }
+
+    public JavaActionBuilder withPrepare(final Prepare prepare) {
+        this.attributesBuilder.withPrepare(prepare);
+        return this;
+    }
+
+    public JavaActionBuilder withLauncher(final Launcher launcher) {
+        this.attributesBuilder.withLauncher(launcher);
+        return this;
+    }
+
+    public JavaActionBuilder withJobXml(final String jobXml) {
+        this.attributesBuilder.withJobXml(jobXml);
+        return this;
+    }
+
+    public JavaActionBuilder withoutJobXml(final String jobXml) {
+        this.attributesBuilder.withoutJobXml(jobXml);
+        return this;
+    }
+
+    public JavaActionBuilder clearJobXmls() {
+        this.attributesBuilder.clearJobXmls();
+        return this;
+    }
+
+    public JavaActionBuilder withConfigProperty(final String key, final String value) {
+        this.attributesBuilder.withConfigProperty(key, value);
+        return this;
+    }
+
+    public JavaActionBuilder withMainClass(final String mainClass) {
+        this.mainClass.set(mainClass);
+        return this;
+    }
+
+    public JavaActionBuilder withJavaOptsString(final String javaOptsString) {
+        this.javaOptsString.set(javaOptsString);
+        return this;
+    }
+
+    public JavaActionBuilder withJavaOpt(final String javaOpt) {
+        this.javaOpts.add(javaOpt);
+        return this;
+    }
+
+    public JavaActionBuilder withoutJavaOpt(final String javaOpt) {
+        this.javaOpts.remove(javaOpt);
+        return this;
+    }
+
+    public JavaActionBuilder clearJavaOpts() {
+        this.javaOpts.clear();
+        return this;
+    }
+
+    public JavaActionBuilder withArg(final String arg) {
+        this.attributesBuilder.withArg(arg);
+        return this;
+    }
+
+    public JavaActionBuilder withoutArg(final String arg) {
+        this.attributesBuilder.withoutArg(arg);
+        return this;
+    }
+
+    public JavaActionBuilder clearArgs() {
+        this.attributesBuilder.clearArgs();
+        return this;
+    }
+
+    public JavaActionBuilder withFile(final String file) {
+        this.attributesBuilder.withFile(file);
+        return this;
+    }
+
+    public JavaActionBuilder withoutFile(final String file) {
+        this.attributesBuilder.withoutFile(file);
+        return this;
+    }
+
+    public JavaActionBuilder clearFiles() {
+        this.attributesBuilder.clearFiles();
+        return this;
+    }
+
+    public JavaActionBuilder withArchive(final String archive) {
+        this.attributesBuilder.withArchive(archive);
+        return this;
+    }
+
+    public JavaActionBuilder withoutArchive(final String archive) {
+        this.attributesBuilder.withoutArchive(archive);
+        return this;
+    }
+
+    public JavaActionBuilder clearArchives() {
+        this.attributesBuilder.clearArchives();
+        return this;
+    }
+
+    public JavaActionBuilder withCaptureOutput(final Boolean captureOutput) {
+        this.attributesBuilder.withCaptureOutput(captureOutput);
+        return this;
+    }
+
+    @Override
+    public JavaAction build() {
+        final Node.ConstructionData constructionData = getConstructionData();
+
+        final JavaAction instance = new JavaAction(
+                constructionData,
+                attributesBuilder.build(),
+                mainClass.get(),
+                javaOptsString.get(),
+                ImmutableList.copyOf(javaOpts));
+
+        addAsChildToAllParents(instance);
+
+        return instance;
+    }
+
+    @Override
+    protected JavaActionBuilder getRuntimeSelfReference() {
+        return this;
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/Launcher.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/Launcher.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/Launcher.java
new file mode 100644
index 0000000..aa41564
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/Launcher.java
@@ -0,0 +1,81 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.oozie.fluentjob.api.workflow.Workflow;
+
+/**
+ * Represents the {@code <launcher>} element and its siblings inside workflow XML / XSD.
+ * <p>
+ * By assigning non-{@code null} field values, the resulting parent {@code <workflow>} will have its
+ * optional {@code <launcher>} element and its siblings filled.
+ * <p>
+ * This class is used only as part of a {@link Workflow}, isn't
+ * to be used alone with Jobs API.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class Launcher {
+    private final long memoryMb;
+    private final long vCores;
+    private final String queue;
+    private final String sharelib;
+    private final String viewAcl;
+    private final String modifyAcl;
+
+    Launcher(final long memoryMb,
+                    final long vCores,
+                    final String queue,
+                    final String sharelib,
+                    final String viewAcl,
+                    final String modifyAcl) {
+        this.memoryMb = memoryMb;
+        this.vCores = vCores;
+        this.queue = queue;
+        this.sharelib = sharelib;
+        this.viewAcl = viewAcl;
+        this.modifyAcl = modifyAcl;
+    }
+
+    public long getMemoryMb() {
+        return memoryMb;
+    }
+
+    public long getVCores() {
+        return vCores;
+    }
+
+    public String getQueue() {
+        return queue;
+    }
+
+    public String getSharelib() {
+        return sharelib;
+    }
+
+    public String getViewAcl() {
+        return viewAcl;
+    }
+
+    public String getModifyAcl() {
+        return modifyAcl;
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/LauncherBuilder.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/LauncherBuilder.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/LauncherBuilder.java
new file mode 100644
index 0000000..3e4ea1d
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/LauncherBuilder.java
@@ -0,0 +1,92 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.oozie.fluentjob.api.ModifyOnce;
+
+/**
+ * A builder class for {@link Launcher}.
+ *
+ * The properties of the builder can only be set once, an attempt to set them a second time will trigger
+ * an {@link IllegalStateException}.
+ *
+ * Builder instances can be used to build several elements, although properties already set cannot be changed after
+ * a call to {@link LauncherBuilder#build} either.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class LauncherBuilder implements Builder<Launcher> {
+    private final ModifyOnce<Long> memoryMb;
+    private final ModifyOnce<Long> vCores;
+    private final ModifyOnce<String> queue;
+    private final ModifyOnce<String> sharelib;
+    private final ModifyOnce<String> viewAcl;
+    private final ModifyOnce<String> modifyAcl;
+
+    public LauncherBuilder() {
+        this.memoryMb = new ModifyOnce<>();
+        this.vCores = new ModifyOnce<>();
+        this.queue = new ModifyOnce<>();
+        this.sharelib = new ModifyOnce<>();
+        this.viewAcl = new ModifyOnce<>();
+        this.modifyAcl = new ModifyOnce<>();
+    }
+
+    @Override
+    public Launcher build() {
+        return new Launcher(memoryMb.get(),
+                vCores.get(),
+                queue.get(),
+                sharelib.get(),
+                viewAcl.get(),
+                modifyAcl.get());
+    }
+
+    public LauncherBuilder withMemoryMb(final long memoryMb) {
+        this.memoryMb.set(memoryMb);
+        return this;
+    }
+
+    public LauncherBuilder withVCores(final long vCores) {
+        this.vCores.set(vCores);
+        return this;
+    }
+
+    public LauncherBuilder withQueue(final String queue) {
+        this.queue.set(queue);
+        return this;
+    }
+
+    public LauncherBuilder withSharelib(final String sharelib) {
+        this.sharelib.set(sharelib);
+        return this;
+    }
+
+    public LauncherBuilder withViewAcl(final String viewAcl) {
+        this.viewAcl.set(viewAcl);
+        return this;
+    }
+
+    public LauncherBuilder withModifyAcl(final String modifyAcl) {
+        this.modifyAcl.set(modifyAcl);
+        return this;
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/MapReduceAction.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/MapReduceAction.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/MapReduceAction.java
new file mode 100644
index 0000000..a90cc3f
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/MapReduceAction.java
@@ -0,0 +1,138 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+import java.util.List;
+import java.util.Map;
+
+/**
+ * A class representing the Oozie map-reduce action.
+ * Instances of this class should be built using the builder {@link MapReduceActionBuilder}.
+ *
+ * The properties of the builder can only be set once, an attempt to set them a second time will trigger
+ * an {@link IllegalStateException}.
+ *
+ * Builder instances can be used to build several elements, although properties already set cannot be changed after
+ * a call to {@link MapReduceActionBuilder#build} either.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class MapReduceAction extends Node implements HasAttributes {
+    private final ActionAttributes attributes;
+
+    MapReduceAction(final ConstructionData constructionData,
+                    final ActionAttributes attributes) {
+        super(constructionData);
+
+        this.attributes = attributes;
+    }
+
+    public String getResourceManager() {
+        return attributes.getResourceManager();
+    }
+
+    /**
+     * Returns the name node stored in this {@link MapReduceAction} object.
+     * @return The name node stored in this {@link MapReduceAction} object.
+     */
+    public String getNameNode() {
+        return attributes.getNameNode();
+    }
+
+    /**
+     * Returns the {@link Prepare} object stored in this {@link MapReduceAction} object.
+     * @return The {@link Prepare} object stored in this {@link MapReduceAction} object.
+     */
+    public Prepare getPrepare() {
+        return attributes.getPrepare();
+    }
+
+    /**
+     * Returns the {@link Streaming} object stored in this {@link MapReduceAction} object.
+     * @return The {@link Streaming} object stored in this {@link MapReduceAction} object.
+     */
+    public Streaming getStreaming() {
+        return attributes.getStreaming();
+    }
+
+    /**
+     * Returns the {@link Pipes} object stored in this {@link MapReduceAction} object.
+     * @return The {@link Pipes} object stored in this {@link MapReduceAction} object.
+     */
+    public Pipes getPipes() {
+        return attributes.getPipes();
+    }
+
+    /**
+     * Returns the list of job XMLs stored in this {@link MapReduceAction} object.
+     * @return The list of job XMLs stored in this {@link MapReduceAction} object.
+     */
+    public List<String> getJobXmls() {
+        return attributes.getJobXmls();
+    }
+
+
+    /**
+     * Returns the value associated with the provided configuration property name.
+     * @param property The name of the configuration property for which the value will be returned.
+     * @return The value associated with the provided configuration property name.
+     */
+    public String getConfigProperty(final String property) {
+        return attributes.getConfiguration().get(property);
+    }
+
+    /**
+     * Returns an immutable map of the configuration key-value pairs stored in this {@link MapReduceAction} object.
+     * @return An immutable map of the configuration key-value pairs stored in this {@link MapReduceAction} object.
+     */
+    public Map<String, String> getConfiguration() {
+        return attributes.getConfiguration();
+    }
+
+    /**
+     * Returns the configuration class property of this {@link MapReduceAction} object.
+     * @return The configuration class property of this {@link MapReduceAction} object.
+     */
+    public String getConfigClass() {
+        return attributes.getConfigClass();
+    }
+
+    /**
+     * Returns an immutable list of the names of the files associated with this {@link MapReduceAction} object.
+     * @return An immutable list of the names of the files associated with this {@link MapReduceAction} object.
+     */
+    public List<String> getFiles() {
+        return attributes.getFiles();
+    }
+
+    /**
+     * Returns an immutable list of the names of the archives associated with this {@link MapReduceAction} object.
+     * @return An immutable list of the names of the archives associated with this {@link MapReduceAction} object.
+     */
+    public List<String> getArchives() {
+        return attributes.getArchives();
+    }
+
+    public ActionAttributes getAttributes() {
+        return attributes;
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/MapReduceActionBuilder.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/MapReduceActionBuilder.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/MapReduceActionBuilder.java
new file mode 100644
index 0000000..c8b3991
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/MapReduceActionBuilder.java
@@ -0,0 +1,256 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * A builder class for {@link MapReduceAction}.
+ * <p>
+ * The properties of the builder can only be set once, an attempt to set them a second time will trigger
+ * an {@link IllegalStateException}. The properties that are lists are an exception to this rule, of course multiple
+ * elements can be added / removed.
+ * <p>
+ * Builder instances can be used to build several elements, although properties already set cannot be changed after
+ * a call to {@link MapReduceActionBuilder#build} either.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class MapReduceActionBuilder extends NodeBuilderBaseImpl<MapReduceActionBuilder> implements Builder<MapReduceAction> {
+    private final ActionAttributesBuilder attributesBuilder;
+
+    /**
+     * Creates and returns an empty builder.
+     * @return An empty builder.
+     */
+    public static MapReduceActionBuilder create() {
+        final ActionAttributesBuilder builder = ActionAttributesBuilder.create();
+
+        return new MapReduceActionBuilder(
+                null,
+                builder);
+    }
+
+    /**
+     * Create and return a new {@link MapReduceActionBuilder} that is based on an already built
+     * {@link MapReduceAction} object. The properties of the builder will initially be the same as those of the
+     * provided {@link MapReduceAction} object, but it is possible to modify them once.
+     * @param action The {@link MapReduceAction} object on which this {@link MapReduceActionBuilder} will be based.
+     * @return A new {@link MapReduceActionBuilder} that is based on a previously built {@link MapReduceAction} object.
+     */
+    public static MapReduceActionBuilder createFromExistingAction(final Node action) {
+        final ActionAttributesBuilder builder = ActionAttributesBuilder.createFromAction(action);
+
+        return new MapReduceActionBuilder(
+                action,
+                builder);
+    }
+
+    MapReduceActionBuilder(final Node action,
+                           final ActionAttributesBuilder attributesBuilder) {
+        super(action);
+
+        this.attributesBuilder = attributesBuilder;
+    }
+
+    public MapReduceActionBuilder withResourceManager(final String resourceManager) {
+        attributesBuilder.withResourceManager(resourceManager);
+        return this;
+    }
+
+    /**
+     * Registers a name node.
+     * @param nameNode The string representing the name node.
+     * @return this
+     * @throws IllegalStateException if a name node has already been set on this builder.
+     */
+    public MapReduceActionBuilder withNameNode(final String nameNode) {
+        attributesBuilder.withNameNode(nameNode);
+        return this;
+    }
+
+    /**
+     * Registers a {@link Prepare} object.
+     * @param prepare The {@link Prepare} object to register.
+     * @return this
+     * @throws IllegalStateException if a {@link Prepare} object has already been set on this builder.
+     */
+    public MapReduceActionBuilder withPrepare(final Prepare prepare) {
+        attributesBuilder.withPrepare(prepare);
+        return this;
+    }
+
+    /**
+     * Registers a {@link Streaming} object.
+     * @param streaming The {@link Streaming} object to register.
+     * @return this
+     * @throws IllegalStateException if a {@link Streaming} object has already been set on this builder.
+     */
+    public MapReduceActionBuilder withStreaming(final Streaming streaming) {
+        attributesBuilder.withStreaming(streaming);
+        return this;
+    }
+
+    /**
+     * Registers a {@link Pipes} object.
+     * @param pipes The {@link Pipes} object to register.
+     * @return this
+     * @throws IllegalStateException if a {@link Pipes} object has already been set on this builder.
+     */
+    public MapReduceActionBuilder withPipes(final Pipes pipes) {
+        attributesBuilder.withPipes(pipes);
+        return this;
+    }
+
+    /**
+     * Registers a job XML with this builder.
+     * @param jobXml The job XML to register.
+     * @return this
+     */
+    public MapReduceActionBuilder withJobXml(final String jobXml) {
+        attributesBuilder.withJobXml(jobXml);
+        return this;
+    }
+
+    /**
+     * Removes a job XML if it is registered with this builder, otherwise does nothing.
+     * @param jobXml The job XML to remove.
+     * @return this
+     */
+    public MapReduceActionBuilder withoutJobXml(final String jobXml) {
+        attributesBuilder.withoutJobXml(jobXml);
+        return this;
+    }
+
+    /**
+     * Removes all job XMLs that are registered with this builder.
+     * @return this
+     */
+    public MapReduceActionBuilder clearJobXmls() {
+        attributesBuilder.clearJobXmls();
+        return this;
+    }
+
+    /**
+     * Registers a configuration property (a key-value pair) with this builder. If the provided key has already been
+     * set on this builder, an exception is thrown. Setting a key to null means deleting it.
+     * @param key The name of the property to set.
+     * @param value The value of the property to set.
+     * @return this
+     * @throws IllegalStateException if the provided key has already been set on this builder.
+     */
+    public MapReduceActionBuilder withConfigProperty(final String key, final String value) {
+        attributesBuilder.withConfigProperty(key, value);
+        return this;
+    }
+
+    /**
+     * Registers a configuration class with this builder.
+     * @param configClass The string representing the configuration class.
+     * @return this
+     * @throws IllegalStateException if a configuration class has already been set on this builder.
+     */
+    public MapReduceActionBuilder withConfigClass(final String configClass) {
+        attributesBuilder.withConfigClass(configClass);
+        return this;
+    }
+
+    /**
+     * Registers a file with this builder.
+     * @param file The file to register.
+     * @return this
+     */
+    public MapReduceActionBuilder withFile(final String file) {
+        attributesBuilder.withFile(file);
+        return this;
+    }
+
+    /**
+     * Removes a file if it is registered with this builder, otherwise does nothing.
+     * @param file The file to remove.
+     * @return this
+     */
+    public MapReduceActionBuilder withoutFile(final String file) {
+        attributesBuilder.withoutFile(file);
+        return this;
+    }
+
+    /**
+     * Removes all files that are registered with this builder.
+     * @return this
+     */
+    public MapReduceActionBuilder clearFiles() {
+        attributesBuilder.clearFiles();
+        return this;
+    }
+
+    /**
+     * Registers an archive with this builder.
+     * @param archive The archive to register.
+     * @return this
+     */
+    public MapReduceActionBuilder withArchive(final String archive) {
+        attributesBuilder.withArchive(archive);
+        return this;
+    }
+
+    /**
+     * Removes an archive if it is registered with this builder, otherwise does nothing.
+     * @param archive The archive to remove.
+     * @return this
+     */
+    public MapReduceActionBuilder withoutArchive(final String archive) {
+        attributesBuilder.withoutArchive(archive);
+        return this;
+    }
+
+    /**
+     * Removes all archives that are registered with this builder.
+     * @return this
+     */
+    public MapReduceActionBuilder clearArchives() {
+        attributesBuilder.clearArchives();
+        return this;
+    }
+
+    /**
+     * Creates a new {@link MapReduceAction} object with the properties stores in this builder.
+     * The new {@link MapReduceAction} object is independent of this builder and the builder can be used to build
+     * new instances.
+     * @return A new {@link MapReduceAction} object with the properties stored in this builder.
+     */
+    @Override
+    public MapReduceAction build() {
+        final Node.ConstructionData constructionData = getConstructionData();
+
+        final MapReduceAction instance = new MapReduceAction(
+                constructionData,
+                attributesBuilder.build());
+
+        addAsChildToAllParents(instance);
+
+        return instance;
+    }
+
+    @Override
+    protected MapReduceActionBuilder getRuntimeSelfReference() {
+        return this;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/Mkdir.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/Mkdir.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/Mkdir.java
new file mode 100644
index 0000000..f98cd72
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/Mkdir.java
@@ -0,0 +1,48 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * A class representing the mkdir command of {@link FSAction} and the prepare section of other actions.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class Mkdir {
+    private final String path;
+
+    /**
+     * Creates a new {@link Mkdir} object with the provided path.
+     * @param path The path of the directory that will be created when this operation is run.
+     */
+    public Mkdir(final String path) {
+        this.path = path;
+    }
+
+    /**
+     * Returns the path of the directory that will be created.
+     * @return The path of the directory that will be created.
+     */
+    public String getPath() {
+        return path;
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/Move.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/Move.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/Move.java
new file mode 100644
index 0000000..489a8f9
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/Move.java
@@ -0,0 +1,58 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * A class representing the move command of {@link FSAction}.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class Move {
+    private final String source;
+    private final String target;
+
+    /**
+     * Creates a new {@link Move} object with the provided path.
+     * @param source HDFS path
+     * @param target HDFS path
+     */
+    public Move(final String source, final String target) {
+        this.source = source;
+        this.target = target;
+    }
+
+    /**
+     * Returns the source path of this {@link Move} operation.
+     * @return The source path of this {@link Move} operation.
+     */
+    public String getSource() {
+        return source;
+    }
+
+    /**
+     * Returns the target path of this {@link Move} operation.
+     * @return The target path of this {@link Move} operation.
+     */
+    public String getTarget() {
+        return target;
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/Node.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/Node.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/Node.java
new file mode 100644
index 0000000..2abdba9
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/Node.java
@@ -0,0 +1,282 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.oozie.fluentjob.api.Condition;
+import org.apache.oozie.fluentjob.api.workflow.Credential;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * An abstract base class for API level action nodes. Concrete instances of the actions should be created using their
+ * respective builders that inherit from {@link NodeBuilderBaseImpl} and implement the {@link Builder} interface.
+ *
+ * The public interface of {@link Node} objects is immutable. This way we can ensure at compile time that there are no
+ * cycles in the graph: once a node is built, it cannot get new parents. On the other hand, {@link Node} objects still
+ * keep track of their children, which are necessarily added after the parent node is built, so  these objects are not
+ * really immutable and should not be used in a multithreaded environment without external synchronization.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public abstract class Node {
+    private final Attributes attributes;
+    private final ImmutableList<Node> parentsWithoutConditions;
+    private final ImmutableList<Node.NodeWithCondition> parentsWithConditions;
+    private final ErrorHandler errorHandler;
+
+    private final List<Node> childrenWithoutConditions; // MUTABLE!
+    private final List<NodeWithCondition> childrenWithConditions; // MUTABLE!
+    private Node defaultConditionalChild; // MUTABLE!
+
+    Node(final ConstructionData constructionData) {
+        this(constructionData.attributes,
+             constructionData.parents,
+             constructionData.parentsWithConditions,
+             constructionData.errorHandler);
+    }
+
+    Node(final Attributes attributes,
+         final ImmutableList<Node> parentsWithoutConditions,
+         final ImmutableList<Node.NodeWithCondition> parentsWithConditions,
+         final ErrorHandler errorHandler)
+    {
+        this.attributes = attributes;
+        this.parentsWithoutConditions = parentsWithoutConditions;
+        this.parentsWithConditions = parentsWithConditions;
+        this.errorHandler = errorHandler;
+
+        this.childrenWithoutConditions = new ArrayList<>();
+        this.childrenWithConditions = new ArrayList<>();
+        this.defaultConditionalChild = null;
+    }
+
+    /**
+     * Returns the name of the node.
+     * @return The name of the node.
+     */
+    public String getName() {
+        return attributes.name;
+    }
+
+    /**
+     * Get the {@link Credential} associated with this {@link Node}.
+     * @return the {@link Credential} associated with this {@link Node}
+     */
+    public List<Credential> getCredentials() {
+        return attributes.credentials;
+    }
+
+    /**
+     * Get the {@code retry-max} attribute of this {@link Node}.
+     * @return the {@code retry-max}
+     */
+    public Integer getRetryMax() {
+        return attributes.retryMax;
+    }
+
+    /**
+     * Get the {@code retry-interval} attribute of this {@link Node}.
+     * @return the {@code retry-interval}
+     */
+    public Integer getRetryInterval() {
+        return attributes.retryInterval;
+    }
+
+    /**
+     * Get the {@code retry-policy} attribute of this {@link Node}.
+     * @return the {@code retry-policy}
+     */
+    public String getRetryPolicy() {
+        return attributes.retryPolicy;
+    }
+
+    /**
+     * Returns a list of all the parents of this node, including unconditional and conditional parents.
+     * @return A list of all the parents of this node object.
+     */
+    public List<Node> getAllParents() {
+        final List<Node> allParents = new ArrayList<>(parentsWithoutConditions);
+
+        for (final NodeWithCondition parentWithCondition : parentsWithConditions) {
+            allParents.add(parentWithCondition.getNode());
+        }
+
+        return Collections.unmodifiableList(allParents);
+    }
+
+    /**
+     * Returns a list of the unconditional parents of this node.
+     * @return A list of the unconditional parents of this node.
+     */
+    public List<Node> getParentsWithoutConditions() {
+        return parentsWithoutConditions;
+    }
+
+    /**
+     * Returns a list of the conditional parents of this node together with their conditions.
+     * @return A list of the conditional parents of this node together with their conditions.
+     */
+    public List<Node.NodeWithCondition> getParentsWithConditions() {
+        return parentsWithConditions;
+    }
+
+    /**
+     * Returns the error handler of this node.
+     * @return The error handler of this node.
+     */
+    public ErrorHandler getErrorHandler() {
+        return errorHandler;
+    }
+
+    void addChild(final Node child) {
+        Preconditions.checkState(childrenWithConditions.isEmpty(),
+                "Trying to add a child without condition to a node that already has at least one child with a condition.");
+
+        this.childrenWithoutConditions.add(child);
+    }
+
+    void addChildWithCondition(final Node child, final String condition) {
+        Preconditions.checkState(childrenWithoutConditions.isEmpty(),
+                "Trying to add a child with condition to a node that already has at least one child without a condition.");
+
+        this.childrenWithConditions.add(new NodeWithCondition(child, Condition.actualCondition(condition)));
+    }
+
+    void addChildAsDefaultConditional(final Node child) {
+        Preconditions.checkState(childrenWithoutConditions.isEmpty(),
+                "Trying to add a default conditional child to a node that already has at least one child without a condition.");
+
+        Preconditions.checkState(defaultConditionalChild == null,
+                "Trying to add a default conditional child to a node that already has one.");
+
+        this.defaultConditionalChild = child;
+    }
+
+    /**
+     * Returns an unmodifiable view of list of all the children of this {@link Node}.
+     * @return An unmodifiable view of list of all the children of this {@link Node}.
+     */
+    public List<Node> getAllChildren() {
+        final List<Node> allChildren = new ArrayList<>(childrenWithoutConditions);
+
+        for (final NodeWithCondition nodeWithCondition : getChildrenWithConditions()) {
+            allChildren.add(nodeWithCondition.getNode());
+        }
+
+        return Collections.unmodifiableList(allChildren);
+    }
+
+    /**
+     * Returns an unmodifiable view of list of the children without condition of this {@link Node}.
+     * @return An unmodifiable view of list of the children without condition of this {@link Node}.
+     */
+    public List<Node> getChildrenWithoutConditions() {
+        return Collections.unmodifiableList(childrenWithoutConditions);
+    }
+
+    /**
+     * Returns an unmodifiable view of list of the children with condition (including the default) of this {@link Node}.
+     * @return An unmodifiable view of list of the children with condition (including the default) of this {@link Node}.
+     */
+    public List<NodeWithCondition> getChildrenWithConditions() {
+        if (defaultConditionalChild == null) {
+            return Collections.unmodifiableList(childrenWithConditions);
+        }
+
+        final List<NodeWithCondition> results = new ArrayList<>(childrenWithConditions);
+        results.add(new NodeWithCondition(defaultConditionalChild, Condition.defaultCondition()));
+
+        return Collections.unmodifiableList(results);
+    }
+
+    /**
+     * Returns the default conditional child of this {@link Node}.
+     * @return The default conditional child of this {@link Node}.
+     */
+    public Node getDefaultConditionalChild() {
+        return defaultConditionalChild;
+    }
+
+    public static class NodeWithCondition {
+        private final Node node;
+        private final Condition condition;
+
+        public NodeWithCondition(final Node node,
+                                 final Condition condition) {
+            this.node = node;
+            this.condition = condition;
+        }
+
+        public Node getNode() {
+            return node;
+        }
+
+        public Condition getCondition() {
+            return condition;
+        }
+    }
+
+    static class ConstructionData {
+
+        ConstructionData(final Attributes attributes,
+                         final ImmutableList<Node> parents,
+                         final ImmutableList<NodeWithCondition> parentsWithConditions,
+                         final ErrorHandler errorHandler) {
+            this.attributes = attributes;
+            this.parents = parents;
+            this.parentsWithConditions = parentsWithConditions;
+            this.errorHandler = errorHandler;
+        }
+
+        private final Attributes attributes;
+        private final ImmutableList<Node> parents;
+        private final ImmutableList<NodeWithCondition> parentsWithConditions;
+        private final ErrorHandler errorHandler;
+    }
+
+    static class Attributes {
+        private final String name;
+        private final ImmutableList<Credential> credentials;
+        private final Integer retryMax;
+        private final Integer retryInterval;
+        private final String retryPolicy;
+
+        Attributes(final String name) {
+            this(name, ImmutableList.of(), null, null, null);
+        }
+
+        Attributes(final String name,
+                   final List<Credential> credentials,
+                   final Integer retryMax,
+                   final Integer retryInterval,
+                   final String retryPolicy) {
+            this.name = name;
+            this.credentials = ImmutableList.copyOf(credentials);
+            this.retryMax = retryMax;
+            this.retryInterval = retryInterval;
+            this.retryPolicy = retryPolicy;
+        }
+    }
+}


[14/16] oozie git commit: OOZIE-2339 [fluent-job] Minimum Viable Fluent Job API (daniel.becker, andras.piros via rkanter, gezapeti, pbacsko)

Posted by an...@apache.org.
http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/ActionAttributesBuilder.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/ActionAttributesBuilder.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/ActionAttributesBuilder.java
new file mode 100644
index 0000000..d52b9c9
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/ActionAttributesBuilder.java
@@ -0,0 +1,567 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.oozie.fluentjob.api.ModifyOnce;
+
+import java.util.ArrayList;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * A builder class for {@link ActionAttributes}.
+ *
+ * The properties of the builder can only be set once, an attempt to set them a second time will trigger
+ * an {@link IllegalStateException}. The properties that are lists are an exception to this rule, of course multiple
+ * elements can be added / removed.
+ *
+ * Builder instances can be used to build several elements, although properties already set cannot be changed after
+ * a call to {@link ActionAttributesBuilder#build} either.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class ActionAttributesBuilder implements Builder<ActionAttributes> {
+    private final ModifyOnce<String> resourceManager;
+    private final ModifyOnce<String> nameNode;
+    private final ModifyOnce<Prepare> prepare;
+    private final ModifyOnce<Streaming> streaming;
+    private final ModifyOnce<Pipes> pipes;
+    private final List<String> jobXmls;
+    private final Map<String, ModifyOnce<String>> configuration;
+    private final ModifyOnce<String> configClass;
+    private final List<String> files;
+    private final List<String> archives;
+    private final List<Delete> deletes;
+    private final List<Mkdir> mkdirs;
+    private final List<Move> moves;
+    private final List<Chmod> chmods;
+    private final List<Touchz> touchzs;
+    private final List<Chgrp> chgrps;
+    private final ModifyOnce<String> javaOpts;
+    private final List<String> args;
+    private final ModifyOnce<Launcher> launcher;
+    private final ModifyOnce<Boolean> captureOutput;
+
+    /**
+     * Creates and returns an empty builder.
+     * @return An empty builder.
+     */
+    public static ActionAttributesBuilder create() {
+        final ModifyOnce<String> resourceManager = new ModifyOnce<>();
+        final ModifyOnce<String> nameNode = new ModifyOnce<>();
+        final ModifyOnce<Prepare> prepare = new ModifyOnce<>();
+        final ModifyOnce<Streaming> streaming = new ModifyOnce<>();
+        final ModifyOnce<Pipes> pipes = new ModifyOnce<>();
+        final List<String> jobXmls = new ArrayList<>();
+        final Map<String, ModifyOnce<String>> configuration = new LinkedHashMap<>();
+        final ModifyOnce<String> configClass = new ModifyOnce<>();
+        final List<String> files = new ArrayList<>();
+        final List<String> archives = new ArrayList<>();
+        final List<Delete> deletes = new ArrayList<>();
+        final List<Mkdir> mkdirs = new ArrayList<>();
+        final List<Move> moves = new ArrayList<>();
+        final List<Chmod> chmods = new ArrayList<>();
+        final List<Touchz> touchzs = new ArrayList<>();
+        final List<Chgrp> chgrps = new ArrayList<>();
+        final ModifyOnce<String> javaOpts = new ModifyOnce<>();
+        final List<String> args = new ArrayList<>();
+        final ModifyOnce<Launcher> launcher = new ModifyOnce<>();
+        final ModifyOnce<Boolean> captureOutput = new ModifyOnce<>();
+
+        return new ActionAttributesBuilder(
+                resourceManager,
+                prepare,
+                streaming,
+                pipes,
+                jobXmls,
+                configuration,
+                configClass,
+                files,
+                archives,
+                deletes,
+                mkdirs,
+                moves,
+                chmods,
+                touchzs,
+                chgrps,
+                javaOpts,
+                args,
+                nameNode,
+                launcher,
+                captureOutput);
+    }
+
+    /**
+     * Create and return a new {@link ActionAttributesBuilder} that is based on an already built
+     * {@link ActionAttributes} object. The properties of the builder will initially be the same as those of the
+     * provided {@link ActionAttributes} object, but it is possible to modify them once.
+     * @param attributes The {@link ActionAttributes} object on which this {@link ActionAttributesBuilder} will be based.
+     * @return A new {@link ActionAttributesBuilder} that is based on a previously built
+     *         {@link ActionAttributes} object.
+     */
+    public static ActionAttributesBuilder createFromExisting(final ActionAttributes attributes) {
+        final ModifyOnce<String> resourceManager = new ModifyOnce<>(attributes.getResourceManager());
+        final ModifyOnce<String> nameNode = new ModifyOnce<>(attributes.getNameNode());
+        final ModifyOnce<Prepare> prepare = new ModifyOnce<>(attributes.getPrepare());
+        final ModifyOnce<Streaming> streaming = new ModifyOnce<>(attributes.getStreaming());
+        final ModifyOnce<Pipes> pipes = new ModifyOnce<>(attributes.getPipes());
+        final List<String> jobXmls = new ArrayList<>(attributes.getJobXmls());
+        final Map<String, ModifyOnce<String>> configuration = convertToModifyOnceMap(attributes.getConfiguration());
+        final ModifyOnce<String> configClass = new ModifyOnce<>(attributes.getConfigClass());
+        final List<String> files = new ArrayList<>(attributes.getFiles());
+        final List<String> archives = new ArrayList<>(attributes.getArchives());
+        final List<Delete> deletes = new ArrayList<>(attributes.getDeletes());
+        final List<Mkdir> mkdirs = new ArrayList<>(attributes.getMkdirs());
+        final List<Move> moves = new ArrayList<>(attributes.getMoves());
+        final List<Chmod> chmods = new ArrayList<>(attributes.getChmods());
+        final List<Touchz> touchzs = new ArrayList<>(attributes.getTouchzs());
+        final List<Chgrp> chgrps = new ArrayList<>(attributes.getChgrps());
+        final ModifyOnce<String> javaOpts = new ModifyOnce<>(attributes.getJavaOpts());
+        final List<String> args = new ArrayList<>(attributes.getArgs());
+        final ModifyOnce<Launcher> launcher = new ModifyOnce<>(attributes.getLauncher());
+        final ModifyOnce<Boolean> captureOutput = new ModifyOnce<>(attributes.isCaptureOutput());
+
+        return new ActionAttributesBuilder(
+                resourceManager,
+                prepare,
+                streaming,
+                pipes,
+                jobXmls,
+                configuration,
+                configClass,
+                files,
+                archives,
+                deletes,
+                mkdirs,
+                moves,
+                chmods,
+                touchzs,
+                chgrps,
+                javaOpts,
+                args,
+                nameNode,
+                launcher,
+                captureOutput);
+    }
+
+    public static ActionAttributesBuilder createFromAction(final Node action) {
+        if (HasAttributes.class.isAssignableFrom(action.getClass()) && action instanceof HasAttributes) {
+            return ActionAttributesBuilder.createFromExisting(((HasAttributes) action).getAttributes());
+        }
+
+        return ActionAttributesBuilder.create();
+    }
+
+    private ActionAttributesBuilder(final ModifyOnce<String> resourceManager,
+                                    final ModifyOnce<Prepare> prepare,
+                                    final ModifyOnce<Streaming> streaming,
+                                    final ModifyOnce<Pipes> pipes,
+                                    final List<String> jobXmls,
+                                    final Map<String, ModifyOnce<String>> configuration,
+                                    final ModifyOnce<String> configClass,
+                                    final List<String> files,
+                                    final List<String> archives,
+                                    final List<Delete> deletes,
+                                    final List<Mkdir> mkdirs,
+                                    final List<Move> moves,
+                                    final List<Chmod> chmods,
+                                    final List<Touchz> touchzs,
+                                    final List<Chgrp> chgrps,
+                                    final ModifyOnce<String> javaOpts,
+                                    final List<String> args,
+                                    final ModifyOnce<String> nameNode,
+                                    final ModifyOnce<Launcher> launcher,
+                                    final ModifyOnce<Boolean> captureOutput) {
+        this.nameNode = nameNode;
+        this.prepare = prepare;
+        this.streaming = streaming;
+        this.pipes = pipes;
+        this.jobXmls = jobXmls;
+        this.configuration = configuration;
+        this.configClass = configClass;
+        this.files = files;
+        this.archives = archives;
+        this.deletes = deletes;
+        this.mkdirs = mkdirs;
+        this.moves = moves;
+        this.chmods = chmods;
+        this.touchzs = touchzs;
+        this.chgrps = chgrps;
+        this.javaOpts = javaOpts;
+        this.args = args;
+        this.resourceManager = resourceManager;
+        this.launcher = launcher;
+        this.captureOutput = captureOutput;
+    }
+
+    public void withResourceManager(final String resourceManager) {
+        this.resourceManager.set(resourceManager);
+    }
+
+    /**
+     * Registers a name node.
+     * @param nameNode The string representing the name node.
+     * @throws IllegalStateException if a name node has already been set on this builder.
+     */
+    public void withNameNode(final String nameNode) {
+        this.nameNode.set(nameNode);
+    }
+
+    /**
+     * Registers a {@link Prepare} object.
+     * @param prepare The {@link Prepare} object to register.
+     * @throws IllegalStateException if a {@link Prepare} object has already been set on this builder.
+     */
+    void withPrepare(final Prepare prepare) {
+        this.prepare.set(prepare);
+    }
+
+    /**
+     * Registers a {@link Streaming} object.
+     * @param streaming The {@link Streaming} object to register.
+     * @throws IllegalStateException if a {@link Streaming} object has already been set on this builder.
+     */
+    void withStreaming(final Streaming streaming) {
+        this.streaming.set(streaming);
+    }
+
+    /**
+     * Registers a {@link Pipes} object.
+     * @param pipes The {@link Pipes} object to register.
+     * @throws IllegalStateException if a {@link Pipes} object has already been set on this builder.
+     */
+    void withPipes(final Pipes pipes) {
+        this.pipes.set(pipes);
+    }
+
+    /**
+     * Registers a job XML with this builder.
+     * @param jobXml The job XML to register.
+     */
+    public void withJobXml(final String jobXml) {
+        this.jobXmls.add(jobXml);
+    }
+
+    /**
+     * Removes a job XML if it is registered with this builder, otherwise does nothing.
+     * @param jobXml The job XML to remove.
+     */
+    public void withoutJobXml(final String jobXml) {
+        jobXmls.remove(jobXml);
+    }
+
+    /**
+     * Removes all job XMLs that are registered with this builder.
+     */
+    public void clearJobXmls() {
+        jobXmls.clear();
+    }
+
+    /**
+     * Registers a configuration property (a key-value pair) with this builder. If the provided key has already been
+     * set on this builder, an exception is thrown. Setting a key to null means deleting it.
+     * @param key The name of the property to set.
+     * @param value The value of the property to set.
+     * @throws IllegalStateException if the provided key has already been set on this builder.
+     */
+    public void withConfigProperty(final String key, final String value) {
+        ModifyOnce<String> mappedValue = this.configuration.get(key);
+
+        if (mappedValue == null) {
+            mappedValue = new ModifyOnce<>(value);
+            this.configuration.put(key, mappedValue);
+        }
+
+        mappedValue.set(value);
+    }
+
+    /**
+     * Registers a configuration class with this builder.
+     * @param configClass The string representing the configuration class.
+     * @throws IllegalStateException if a configuration class has already been set on this builder.
+     */
+    void withConfigClass(final String configClass) {
+        this.configClass.set(configClass);
+    }
+
+    /**
+     * Registers a file with this builder.
+     * @param file The file to register.
+     */
+    void withFile(final String file) {
+        this.files.add(file);
+    }
+
+    /**
+     * Removes a file if it is registered with this builder, otherwise does nothing.
+     * @param file The file to remove.
+     */
+    void withoutFile(final String file) {
+        files.remove(file);
+    }
+
+    /**
+     * Removes all files that are registered with this builder.
+     */
+    void clearFiles() {
+        files.clear();
+    }
+
+    /**
+     * Registers an archive with this builder.
+     * @param archive The archive to register.
+     */
+    void withArchive(final String archive) {
+        this.archives.add(archive);
+    }
+
+    /**
+     * Removes an archive if it is registered with this builder, otherwise does nothing.
+     * @param archive The archive to remove.
+     */
+    void withoutArchive(final String archive) {
+        archives.remove(archive);
+    }
+
+    /**
+     * Removes all archives that are registered with this builder.
+     */
+    void clearArchives() {
+        archives.clear();
+    }
+
+    /**
+     * Registers a {@link Delete} object with this builder.
+     * @param delete The {@link Delete} object to register.
+     */
+    void withDelete(final Delete delete) {
+        this.deletes.add(delete);
+    }
+
+    /**
+     * Removes a {@link Delete} object if it is registered with this builder, otherwise does nothing.
+     * @param delete The {@link Delete} object to remove.
+     */
+    void withoutDelete(final Delete delete) {
+        deletes.remove(delete);
+    }
+
+    /**
+     * Removes all {@link Delete} objects that are registered with this builder.
+     */
+    void clearDeletes() {
+        deletes.clear();
+    }
+
+    /**
+     * Registers a {@link Mkdir} object with this builder.
+     * @param mkdir The {@link Mkdir} object to register.
+     */
+    void withMkdir(final Mkdir mkdir) {
+        this.mkdirs.add(mkdir);
+    }
+
+    /**
+     * Removes a {@link Mkdir} object if it is registered with this builder, otherwise does nothing.
+     * @param mkdir The {@link Mkdir} object to remove.
+     */
+    void withoutMkdir(final Mkdir mkdir) {
+        mkdirs.remove(mkdir);
+    }
+
+    /**
+     * Removes all {@link Mkdir} objects that are registered with this builder.
+     */
+    void clearMkdirs() {
+        mkdirs.clear();
+    }
+
+    /**
+     * Registers a {@link Move} object with this builder.
+     * @param move The {@link Move} object to register.
+     */
+    void withMove(final Move move) {
+        this.moves.add(move);
+    }
+
+    /**
+     * Removes a {@link Move} object if it is registered with this builder, otherwise does nothing.
+     * @param move The {@link Move} object to remove.
+     */
+    void withoutMove(final Move move) {
+        moves.remove(move);
+    }
+
+    /**
+     * Removes all {@link Move} objects that are registered with this builder.
+     */
+    void clearMoves() {
+        moves.clear();
+    }
+
+    /**
+     * Registers a {@link Chmod} object with this builder.
+     * @param chmod The {@link Chmod} object to register.
+     */
+    void withChmod(final Chmod chmod) {
+        this.chmods.add(chmod);
+    }
+
+    /**
+     * Removes a {@link Chmod} object if it is registered with this builder, otherwise does nothing.
+     * @param chmod The {@link Chmod} object to remove.
+     */
+    void withoutChmod(final Chmod chmod) {
+        chmods.remove(chmod);
+    }
+
+    /**
+     * Removes all {@link Chmod} objects that are registered with this builder.
+     */
+    void clearChmods() {
+        chmods.clear();
+    }
+
+    /**
+     * Registers a {@link Touchz} object with this builder.
+     * @param touchz The {@link Touchz} object to register.
+     */
+    void withTouchz(final Touchz touchz) {
+        this.touchzs.add(touchz);
+    }
+
+    /**
+     * Removes a {@link Touchz} object if it is registered with this builder, otherwise does nothing.
+     * @param touchz The {@link Touchz} object to remove.
+     */
+    void withoutTouchz(final Touchz touchz) {
+        touchzs.remove(touchz);
+    }
+
+    /**
+     * Removes all {@link Touchz} objects that are registered with this builder.
+     */
+    void clearTouchzs() {
+        touchzs.clear();
+    }
+
+    /**
+     * Registers a {@link Chgrp} object with this builder.
+     * @param chgrp The {@link Chgrp} object to register.
+     */
+    void withChgrp(final Chgrp chgrp) {
+        this.chgrps.add(chgrp);
+    }
+
+    /**
+     * Removes a {@link Chgrp} object if it is registered with this builder, otherwise does nothing.
+     * @param chgrp The {@link Chgrp} object to remove.
+     */
+    void withoutChgrp(final Chgrp chgrp) {
+        chgrps.remove(chgrp);
+    }
+
+    /**
+     * Removes all {@link Chgrp} objects that are registered with this builder.
+     */
+    void clearChgrps() {
+        chgrps.clear();
+    }
+
+    void withJavaOpts(final String javaOpts) {
+        this.javaOpts.set(javaOpts);
+    }
+
+    void withArg(final String arg) {
+        this.args.add(arg);
+    }
+
+    void withoutArg(final String arg) {
+        this.args.remove(arg);
+    }
+
+    void clearArgs() {
+        args.clear();
+    }
+
+    public void withLauncher(final Launcher launcher) {
+        this.launcher.set(launcher);
+    }
+
+    void withCaptureOutput(final Boolean captureOutput) {
+        this.captureOutput.set(captureOutput);
+    }
+
+    /**
+     * Creates a new {@link ActionAttributes} object with the properties stores in this builder.
+     * The new {@link ActionAttributes} object is independent of this builder and the builder can be used to build
+     * new instances.
+     * @return A new {@link ActionAttributes} object with the propertied stores in this builder.
+     */
+    public ActionAttributes build() {
+        return new ActionAttributes(
+                resourceManager.get(),
+                nameNode.get(),
+                prepare.get(),
+                streaming.get(),
+                pipes.get(),
+                ImmutableList.copyOf(jobXmls),
+                convertToConfigurationMap(configuration),
+                configClass.get(),
+                ImmutableList.copyOf(files),
+                ImmutableList.copyOf(archives),
+                ImmutableList.copyOf(deletes),
+                ImmutableList.copyOf(mkdirs),
+                ImmutableList.copyOf(moves),
+                ImmutableList.copyOf(chmods),
+                ImmutableList.copyOf(touchzs),
+                ImmutableList.copyOf(chgrps),
+                javaOpts.get(),
+                ImmutableList.copyOf(args),
+                launcher.get(),
+                captureOutput.get());
+    }
+
+    static Map<String, ModifyOnce<String>> convertToModifyOnceMap(final Map<String, String> configurationMap) {
+        final Map<String, ModifyOnce<String>> modifyOnceEntries = new LinkedHashMap<>();
+
+        for (final Map.Entry<String, String> keyAndValue : configurationMap.entrySet()) {
+            modifyOnceEntries.put(keyAndValue.getKey(), new ModifyOnce<>(keyAndValue.getValue()));
+        }
+
+        return modifyOnceEntries;
+    }
+
+    static ImmutableMap<String, String> convertToConfigurationMap(final Map<String, ModifyOnce<String>> map) {
+        final Map<String, String> mutableConfiguration = new LinkedHashMap<>();
+
+        for (final Map.Entry<String, ModifyOnce<String>> modifyOnceEntry : map.entrySet()) {
+            if (modifyOnceEntry.getValue().get() != null) {
+                mutableConfiguration.put(modifyOnceEntry.getKey(), modifyOnceEntry.getValue().get());
+            }
+        }
+
+        return ImmutableMap.copyOf(mutableConfiguration);
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/Builder.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/Builder.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/Builder.java
new file mode 100644
index 0000000..c71a04b
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/Builder.java
@@ -0,0 +1,31 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+/**
+ * A common interface for builders.
+ * @param <T> The type of the object that is build using this builder.
+ */
+public interface Builder<T> {
+    /**
+     * Builds and returns an object.
+     * @return The built object.
+     */
+    T build();
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/ChFSBase.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/ChFSBase.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/ChFSBase.java
new file mode 100644
index 0000000..2fc1ff6
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/ChFSBase.java
@@ -0,0 +1,81 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * A base class for {@link Chgrp} and {@link Chmod}.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class ChFSBase {
+    private final boolean recursive;
+    private final String path;
+    private final String dirFiles;
+
+    ChFSBase(final ConstructionData constructionData) {
+        this.recursive = constructionData.recursive;
+        this.path = constructionData.path;
+        this.dirFiles = constructionData.dirFiles;
+    }
+
+    /**
+     * Returns whether this file system operation is recursive.
+     * @return {@code true} if this file system operation is recursive; {@code false} otherwise.
+     */
+    public boolean isRecursive() {
+        return recursive;
+    }
+
+    /**
+     * Returns the path of the target of this file system operation.
+     * @return The path of the target of this file system operation.
+     */
+    public String getPath() {
+        return path;
+    }
+
+    /**
+     * Returns whether this file system operation should be applied to all files in the given directory.
+     * @return "true" if this file system operation should be applied to all files in the given directory;
+     *         "false" otherwise.
+     */
+    public String getDirFiles() {
+        return dirFiles;
+    }
+
+    /**
+     * Helper class that is used by the subclasses of this class and their builders.
+     */
+    public static class ConstructionData {
+        private final boolean recursive;
+        private final String path;
+        private final String dirFiles;
+
+        public ConstructionData(final boolean recursive,
+                                final String path,
+                                final String dirFiles) {
+            this.recursive = recursive;
+            this.path = path;
+            this.dirFiles = dirFiles;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/ChFSBaseBuilder.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/ChFSBaseBuilder.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/ChFSBaseBuilder.java
new file mode 100644
index 0000000..d1303ac
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/ChFSBaseBuilder.java
@@ -0,0 +1,95 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.oozie.fluentjob.api.ModifyOnce;
+
+/**
+ * A base class for {@link ChgrpBuilder} and {@link ChmodBuilder}.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public abstract class ChFSBaseBuilder <B extends ChFSBaseBuilder<B>> {
+    private final ModifyOnce<Boolean> recursive;
+    private final ModifyOnce<String> path;
+    private final ModifyOnce<String> dirFiles;
+
+    public ChFSBaseBuilder() {
+        recursive = new ModifyOnce<>(false);
+        path = new ModifyOnce<>();
+        dirFiles = new ModifyOnce<>("true");
+    }
+
+    /**
+     * Sets this file system operation to be recursive.
+     * @return This builder.
+     */
+    public B setRecursive() {
+        this.recursive.set(true);
+        return ensureRuntimeSelfReference();
+    }
+
+    /**
+     * Sets this file system operation to be non-recursive.
+     * @return This builder.
+     */
+    public B setNonRecursive() {
+        this.recursive.set(false);
+        return ensureRuntimeSelfReference();
+    }
+
+    /**
+     * Sets the path of the target of this file system operation.
+     * @param path the path of the target
+     * @return This builder.
+     */
+    public B withPath(final String path) {
+        this.path.set(path);
+        return ensureRuntimeSelfReference();
+    }
+
+    /**
+     * Sets whether this file system operation should be applied to all files in the given directory.
+     * @param dirFiles {@code true} if the operation should be applied to all files in the given directory;
+     *                 {@code false} if it shouldn't.
+     * @return This builder.
+     */
+    public B setDirFiles(final boolean dirFiles) {
+        this.dirFiles.set(Boolean.toString(dirFiles));
+        return ensureRuntimeSelfReference();
+    }
+
+    final B ensureRuntimeSelfReference() {
+        final B concrete = getRuntimeSelfReference();
+        if (concrete != this) {
+            throw new IllegalStateException(
+                    "The builder type B doesn't extend ChFSBaseBuilder<B>.");
+        }
+
+        return concrete;
+    }
+
+    protected ChFSBase.ConstructionData getConstructionData() {
+        return new ChFSBase.ConstructionData(recursive.get(), path.get(), dirFiles.get());
+    }
+
+    protected abstract B getRuntimeSelfReference();
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/Chgrp.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/Chgrp.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/Chgrp.java
new file mode 100644
index 0000000..1fa77eb
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/Chgrp.java
@@ -0,0 +1,53 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * A class representing the chgrp command of {@link FSAction}.
+ * Instances of this class should be built using the builder {@link ChgrpBuilder}.
+ *
+ * The properties of the builder can only be set once, an attempt to set them a second time will trigger
+ * an {@link IllegalStateException}.
+ *
+ * Builder instances can be used to build several elements, although properties already set cannot be changed after
+ * a call to {@link ChgrpBuilder#build} either.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class Chgrp extends ChFSBase {
+    private String group;
+
+    Chgrp(final ChFSBase.ConstructionData constructionData,
+          final String group) {
+        super(constructionData);
+        this.group = group;
+    }
+
+
+    /**
+     * Returns the new group that will be set by the command.
+     * @return The new group that will be set by the command.
+     */
+    public String getGroup() {
+        return group;
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/ChgrpBuilder.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/ChgrpBuilder.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/ChgrpBuilder.java
new file mode 100644
index 0000000..05cbe9b
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/ChgrpBuilder.java
@@ -0,0 +1,68 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.oozie.fluentjob.api.ModifyOnce;
+
+/**
+ * A builder class for {@link Chgrp}.
+ *
+ * The properties of the builder can only be set once, an attempt to set them a second time will trigger
+ * an {@link IllegalStateException}. The properties that are lists are an exception to this rule, of course multiple
+ * elements can be added / removed.
+ *
+ * Builder instances can be used to build several elements, although properties already set cannot be changed after
+ * a call to {@link ChgrpBuilder#build} either.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class ChgrpBuilder extends ChFSBaseBuilder<ChgrpBuilder> implements Builder<Chgrp> {
+    private final ModifyOnce<String> group;
+
+    public ChgrpBuilder() {
+        this.group = new ModifyOnce<>();
+    }
+
+    /**
+     * Sets the new group that will be set by the operation.
+     * @param group The group that will be set by the operation.
+     * @return This builder.
+     */
+    public ChgrpBuilder withGroup(final String group) {
+        this.group.set(group);
+        return this;
+    }
+
+    /**
+     * Builds and returns a new {@link Chgrp} object with the properties set in this builder.
+     * The new {@link Chgrp} object is independent of this builder and the builder can be used to build new instances.
+     * @return The newly built {@link Chgrp} object.
+     */
+    @Override
+    public Chgrp build() {
+        return new Chgrp(getConstructionData(), group.get());
+    }
+
+    @Override
+    protected ChgrpBuilder getRuntimeSelfReference() {
+        return this;
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/Chmod.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/Chmod.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/Chmod.java
new file mode 100644
index 0000000..b0e998b
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/Chmod.java
@@ -0,0 +1,52 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * A class representing the chmod command of {@link FSAction}.
+ * Instances of this class should be built using the builder {@link ChmodBuilder}.
+ *
+ * The properties of the builder can only be set once, an attempt to set them a second time will trigger
+ * an {@link IllegalStateException}.
+ *
+ * Builder instances can be used to build several elements, although properties already set cannot be changed after
+ * a call to {@link ChmodBuilder#build} either.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class Chmod extends ChFSBase {
+    private final String permissions;
+
+    Chmod(final ChFSBase.ConstructionData constructionData,
+          final String permissions) {
+        super(constructionData);
+        this.permissions = permissions;
+    }
+
+    /**
+     * Returns the new permissions that will be set by the command.
+     * @return The new permissions that will be set by the command.
+     */
+    public String getPermissions() {
+        return permissions;
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/ChmodBuilder.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/ChmodBuilder.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/ChmodBuilder.java
new file mode 100644
index 0000000..a6b439a
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/ChmodBuilder.java
@@ -0,0 +1,68 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.oozie.fluentjob.api.ModifyOnce;
+
+/**
+ * A builder class for {@link Chmod}.
+ *
+ * The properties of the builder can only be set once, an attempt to set them a second time will trigger
+ * an {@link IllegalStateException}. The properties that are lists are an exception to this rule, of course multiple
+ * elements can be added / removed.
+ *
+ * Builder instances can be used to build several elements, although properties already set cannot be changed after
+ * a call to {@link ChmodBuilder#build} either.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class ChmodBuilder extends ChFSBaseBuilder<ChmodBuilder> implements Builder<Chmod> {
+    private final ModifyOnce<String> permissions;
+
+    public ChmodBuilder() {
+        super();
+        permissions = new ModifyOnce<>();
+    }
+
+    /**
+     * Sets the new permissions that will be set by the operation.
+     * @param permissions The new permissions that will be set by the operation.
+     * @return This builder.
+     */
+    public ChmodBuilder withPermissions(final String permissions) {
+        this.permissions.set(permissions);
+        return this;
+    }
+
+    /**
+     * Builds and returns a new {@link Chmod} object with the properties set in this builder.
+     * The new {@link Chmod} object is independent of this builder and the builder can be used to build new instances.
+     * @return The newly built {@link Chmod} object.
+     */
+    public Chmod build() {
+        return new Chmod(getConstructionData(), permissions.get());
+    }
+
+    @Override
+    protected ChmodBuilder getRuntimeSelfReference() {
+        return this;
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/Delete.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/Delete.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/Delete.java
new file mode 100644
index 0000000..17aab7f
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/Delete.java
@@ -0,0 +1,60 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * A class representing the delete operation of {@link FSAction} and the prepare section of other actions.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class Delete {
+    private final String path;
+    private final Boolean skipTrash;
+
+    /**
+     * Creates a new {@link Delete} object.
+     * @param path The path of the file or directory to be deleted.
+     * @param skipTrash {@code true} if the deleted items should NOT be moved to the trash but deleted completely;
+     *                  {@code false} if the items should be moved to trash instead of deleting them conpletely.
+     */
+    public Delete(final String path, final Boolean skipTrash) {
+        this.path = path;
+        this.skipTrash = skipTrash;
+    }
+
+    /**
+     * Returns the path of the item to be deleted.
+     * @return The path of the item to be deleted.
+     */
+    public String getPath() {
+        return path;
+    }
+
+    /**
+     * Returns whether the trash should be skipped when deleting the items.
+     * @return {@code true} if the deleted items should NOT be moved to the trash but deleted completely;
+     *         {@code false} if the items should be moved to trash instead of deleting them conpletely.
+     */
+    public Boolean getSkipTrash() {
+        return skipTrash;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/DistcpAction.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/DistcpAction.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/DistcpAction.java
new file mode 100644
index 0000000..3886af6
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/DistcpAction.java
@@ -0,0 +1,80 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Represents the Oozie DistCp action.
+ * Instances of this class should be built using the builder {@link DistcpActionBuilder}.
+ *
+ * The properties of the builder can only be set once, an attempt to set them a second time will trigger
+ * an {@link IllegalStateException}.
+ *
+ * Builder instances can be used to build several elements, although properties already set cannot be changed after
+ * a call to {@link DistcpActionBuilder#build} either.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class DistcpAction extends Node implements HasAttributes {
+    private final ActionAttributes attributes;
+
+    DistcpAction(final ConstructionData constructionData,
+                 final ActionAttributes attributes) {
+        super(constructionData);
+
+        this.attributes = attributes;
+    }
+
+    public String getResourceManager() {
+        return attributes.getResourceManager();
+    }
+
+    public String getNameNode() {
+        return attributes.getNameNode();
+    }
+
+    public Prepare getPrepare() {
+        return attributes.getPrepare();
+    }
+
+    public String getConfigProperty(final String property) {
+        return attributes.getConfiguration().get(property);
+    }
+
+    public Map<String, String> getConfiguration() {
+        return attributes.getConfiguration();
+    }
+
+    public String getJavaOpts() {
+        return attributes.getJavaOpts();
+    }
+
+    public List<String> getArgs() {
+        return attributes.getArgs();
+    }
+
+    public ActionAttributes getAttributes() {
+        return attributes;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/DistcpActionBuilder.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/DistcpActionBuilder.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/DistcpActionBuilder.java
new file mode 100644
index 0000000..42731ef
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/DistcpActionBuilder.java
@@ -0,0 +1,118 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * A builder class for {@link DistcpAction}.
+ *
+ * The properties of the builder can only be set once, an attempt to set them a second time will trigger
+ * an {@link IllegalStateException}. The properties that are lists are an exception to this rule, of course multiple
+ * elements can be added / removed.
+ *
+ * Builder instances can be used to build several elements, although properties already set cannot be changed after
+ * a call to {@link DistcpActionBuilder#build} either.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class DistcpActionBuilder extends NodeBuilderBaseImpl<DistcpActionBuilder> implements Builder<DistcpAction> {
+    private final ActionAttributesBuilder attributesBuilder;
+
+    public static DistcpActionBuilder create() {
+        final ActionAttributesBuilder builder = ActionAttributesBuilder.create();
+
+        return new DistcpActionBuilder(
+                null,
+                builder);
+    }
+
+    public static DistcpActionBuilder createFromExistingAction(final Node action) {
+        final ActionAttributesBuilder builder = ActionAttributesBuilder.createFromAction(action);
+
+        return new DistcpActionBuilder(action,
+                builder);
+    }
+
+    DistcpActionBuilder(final Node action,
+                        final ActionAttributesBuilder attributesBuilder) {
+        super(action);
+
+        this.attributesBuilder = attributesBuilder;
+    }
+
+    public DistcpActionBuilder withResourceManager(final String resourceManager) {
+        attributesBuilder.withResourceManager(resourceManager);
+        return this;
+    }
+
+    public DistcpActionBuilder withNameNode(final String nameNode) {
+        attributesBuilder.withNameNode(nameNode);
+        return this;
+    }
+
+    public DistcpActionBuilder withPrepare(final Prepare prepare) {
+        attributesBuilder.withPrepare(prepare);
+        return this;
+    }
+
+    public DistcpActionBuilder withConfigProperty(final String key, final String value) {
+        attributesBuilder.withConfigProperty(key, value);
+        return this;
+    }
+
+    public DistcpActionBuilder withJavaOpts(final String javaOpts) {
+        attributesBuilder.withJavaOpts(javaOpts);
+        return this;
+    }
+
+    public DistcpActionBuilder withArg(final String arg) {
+        attributesBuilder.withArg(arg);
+        return this;
+    }
+
+    public DistcpActionBuilder withoutArg(final String arg) {
+        attributesBuilder.withoutArg(arg);
+        return this;
+    }
+
+    public DistcpActionBuilder clearArgs() {
+        attributesBuilder.clearArgs();
+        return this;
+    }
+
+    @Override
+    public DistcpAction build() {
+        final Node.ConstructionData constructionData = getConstructionData();
+
+        final DistcpAction instance = new DistcpAction(
+                constructionData,
+                attributesBuilder.build());
+
+        addAsChildToAllParents(instance);
+
+        return instance;
+    }
+
+    @Override
+    protected DistcpActionBuilder getRuntimeSelfReference() {
+        return this;
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/EmailAction.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/EmailAction.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/EmailAction.java
new file mode 100644
index 0000000..bbaecbc
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/EmailAction.java
@@ -0,0 +1,118 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * A class representing the Oozie email action.
+ * Instances of this class should be built using the builder {@link EmailActionBuilder}.
+ *
+ * The properties of the builder can only be set once, an attempt to set them a second time will trigger
+ * an {@link IllegalStateException}.
+ *
+ * Builder instances can be used to build several elements, although properties already set cannot be changed after
+ * a call to {@link EmailActionBuilder#build} either.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class EmailAction extends Node {
+    private final String to;
+    private final String cc;
+    private final String bcc;
+    private final String subject;
+    private final String body;
+    private final String contentType;
+    private final String attachment;
+
+    EmailAction(final Node.ConstructionData constructionData,
+                final String to,
+                final String cc,
+                final String bcc,
+                final String subject,
+                final String body,
+                final String contentType,
+                final String attachment) {
+        super(constructionData);
+        this.to = to;
+        this.cc = cc;
+        this.bcc = bcc;
+        this.subject = subject;
+        this.body = body;
+        this.contentType = contentType;
+        this.attachment = attachment;
+    }
+
+    /**
+     * Returns the address of the recipient of the email.
+     * @return The address of the recipient of the email.
+     */
+    public String getRecipient() {
+        return to;
+    }
+
+    /**
+     * Returns the address of the recipient of a copy of the email.
+     * @return The address of the recipient of a copy of the email.
+     */
+    public String getCc() {
+        return cc;
+    }
+
+    /**
+     * Returns the address of the secret recipient of a copy of the email.
+     * @return The address of the secret recipient of a copy of the email.
+     */
+    public String getBcc() {
+        return bcc;
+    }
+
+    /**
+     * Returns the subject of the email.
+     * @return The subject of the email.
+     */
+    public String getSubject() {
+        return subject;
+    }
+
+    /**
+     * Returns the body of the email.
+     * @return The body of the email.
+     */
+    public String getBody() {
+        return body;
+    }
+
+    /**
+     * Returns the content type of the email.
+     * @return The content type of the email.
+     */
+    public String getContentType() {
+        return contentType;
+    }
+
+    /**
+     * Returns the attachment of the email.
+     * @return The attachment of the email.
+     */
+    public String getAttachment() {
+        return attachment;
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/EmailActionBuilder.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/EmailActionBuilder.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/EmailActionBuilder.java
new file mode 100644
index 0000000..fb78f97
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/EmailActionBuilder.java
@@ -0,0 +1,234 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.oozie.fluentjob.api.ModifyOnce;
+
+/**
+ * A builder class for {@link EmailAction}.
+ * <p>
+ * The properties of the builder can only be set once, an attempt to set them a second time will trigger
+ * an {@link IllegalStateException}. The properties that are lists are an exception to this rule, of course multiple
+ * elements can be added / removed.
+ * <p>
+ * Builder instances can be used to build several elements, although properties already set cannot be changed after
+ * a call to {@link EmailActionBuilder#build} either.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class EmailActionBuilder extends NodeBuilderBaseImpl<EmailActionBuilder> implements Builder<EmailAction> {
+    private final ModifyOnce<String> to;
+    private final ModifyOnce<String> cc;
+    private final ModifyOnce<String> bcc;
+    private final ModifyOnce<String> subject;
+    private final ModifyOnce<String> body;
+    private final ModifyOnce<String> contentType;
+    private final ModifyOnce<String> attachment;
+
+    /**
+     * Creates and returns an empty builder.
+     * @return An empty builder.
+     */
+    public static EmailActionBuilder create() {
+        final ModifyOnce<String> to = new ModifyOnce<>();
+        final ModifyOnce<String> cc = new ModifyOnce<>();
+        final ModifyOnce<String> bcc = new ModifyOnce<>();
+        final ModifyOnce<String> subject = new ModifyOnce<>();
+        final ModifyOnce<String> body = new ModifyOnce<>();
+        final ModifyOnce<String> contentType = new ModifyOnce<>();
+        final ModifyOnce<String> attachment = new ModifyOnce<>();
+
+        return new EmailActionBuilder(
+                null,
+                to,
+                cc,
+                bcc,
+                subject,
+                body,
+                contentType,
+                attachment);
+    }
+
+    /**
+     * Create and return a new {@link EmailActionBuilder} that is based on an already built
+     * {@link EmailAction} object. The properties of the builder will initially be the same as those of the
+     * provided {@link EmailAction} object, but it is possible to modify them once.
+     * @param action The {@link EmailAction} object on which this {@link EmailActionBuilder} will be based.
+     * @return A new {@link EmailActionBuilder} that is based on a previously built {@link EmailAction} object.
+     */
+    public static EmailActionBuilder createFromExistingAction(final EmailAction action) {
+        final ModifyOnce<String> to = new ModifyOnce<>(action.getRecipient());
+        final ModifyOnce<String> cc = new ModifyOnce<>(action.getCc());
+        final ModifyOnce<String> bcc = new ModifyOnce<>(action.getBcc());
+        final ModifyOnce<String> subject = new ModifyOnce<>(action.getSubject());
+        final ModifyOnce<String> body = new ModifyOnce<>(action.getBody());
+        final ModifyOnce<String> contentType = new ModifyOnce<>(action.getContentType());
+        final ModifyOnce<String> attachment = new ModifyOnce<>(action.getAttachment());
+
+        return new EmailActionBuilder(
+                action,
+                to,
+                cc,
+                bcc,
+                subject,
+                body,
+                contentType,
+                attachment);
+    }
+
+    public static EmailActionBuilder createFromExistingAction(final Node action) {
+        final ModifyOnce<String> to = new ModifyOnce<>();
+        final ModifyOnce<String> cc = new ModifyOnce<>();
+        final ModifyOnce<String> bcc = new ModifyOnce<>();
+        final ModifyOnce<String> subject = new ModifyOnce<>();
+        final ModifyOnce<String> body = new ModifyOnce<>();
+        final ModifyOnce<String> contentType = new ModifyOnce<>();
+        final ModifyOnce<String> attachment = new ModifyOnce<>();
+
+        return new EmailActionBuilder(
+                action,
+                to,
+                cc,
+                bcc,
+                subject,
+                body,
+                contentType,
+                attachment);
+    }
+
+    EmailActionBuilder(final Node action,
+                       final ModifyOnce<String> to,
+                       final ModifyOnce<String> cc,
+                       final ModifyOnce<String> bcc,
+                       final ModifyOnce<String> subject,
+                       final ModifyOnce<String> body,
+                       final ModifyOnce<String> contentType,
+                       final ModifyOnce<String> attachment) {
+        super(action);
+        this.to = to;
+        this.cc = cc;
+        this.bcc = bcc;
+        this.subject = subject;
+        this.body = body;
+        this.contentType = contentType;
+        this.attachment = attachment;
+    }
+
+    /**
+     * Sets the address of the recipient of the email.
+     * @param to the recipient in To: field
+     * @return This builder.
+     */
+    public EmailActionBuilder withRecipient(final String to) {
+        this.to.set(to);
+        return this;
+    }
+
+    /**
+     * Sets the address of the recipient of a copy of the email.
+     * @param cc the recipient in CC: field
+     * @return This builder.
+     */
+    public EmailActionBuilder withCc(final String cc) {
+        this.cc.set(cc);
+        return this;
+    }
+
+    /**
+     * Sets the address of the secret recipient of a copy of the email.
+     * @param bcc the recipient in BCC: field
+     * @return This builder.
+     */
+    public EmailActionBuilder withBcc(final String bcc) {
+        this.bcc.set(bcc);
+        return this;
+    }
+
+    /**
+     * Sets the subject of the email.
+     * @param subject the email subject
+     * @return This builder.
+     */
+    public EmailActionBuilder withSubject(final String subject) {
+        this.subject.set(subject);
+        return this;
+    }
+
+    /**
+     * Sets the body of the email.
+     * @param body the email body
+     * @return This builder.
+     */
+    public EmailActionBuilder withBody(final String body) {
+        this.body.set(body);
+        return this;
+    }
+
+    /**
+     * Sets the content type of the email.
+     * @param contentType the email content type
+     * @return This builder
+     */
+    public EmailActionBuilder withContentType(final String contentType) {
+        this.contentType.set(contentType);
+        return this;
+    }
+
+    /**
+     * Sets the attachment of the email.
+     * @param attachment the email attachment path
+     * @return This builder.
+     */
+    public EmailActionBuilder withAttachment(final String attachment) {
+        this.attachment.set(attachment);
+        return this;
+    }
+
+    /**
+     * Creates a new {@link EmailAction} object with the properties stores in this builder.
+     * The new {@link EmailAction} object is independent of this builder and the builder can be used to build
+     * new instances.
+     * @return A new {@link EmailAction} object with the properties stored in this builder.
+     */
+    @Override
+    public EmailAction build() {
+        final Node.ConstructionData constructionData = getConstructionData();
+
+        final EmailAction instance = new EmailAction(
+                constructionData,
+                to.get(),
+                cc.get(),
+                bcc.get(),
+                subject.get(),
+                body.get(),
+                contentType.get(),
+                attachment.get());
+
+        addAsChildToAllParents(instance);
+
+        return instance;
+    }
+
+    @Override
+    protected EmailActionBuilder getRuntimeSelfReference() {
+        return this;
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/ErrorHandler.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/ErrorHandler.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/ErrorHandler.java
new file mode 100644
index 0000000..6129b9f
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/ErrorHandler.java
@@ -0,0 +1,78 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * A class encapsulating an action so that it can be used as an error handler in a workflow.
+ *
+ * In an Oozie workflow definition (XML), every action has an "ok-transition" that is taken if the action completed
+ * successfully and an "error-transition" that is taken if the action failed. In this API, the dependency relations
+ * specified will be translated into "ok-transitions".
+ *
+ * If you would like to provide some error handling in case of action failure, you should add an {@link ErrorHandler}
+ * to the {@link Node} representing the action. The error handler action will be added as the "error-transition" of
+ * the original action in the generated Oozie workflow XML. Both the "ok-transition" and the "error-transition" of the
+ * error handler action itself will lead to an autogenerated kill node.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class ErrorHandler {
+    private final Node handlerNode;
+
+    /**
+     * Creates a new {@link ErrorHandler}. The provided builder is used to build the underlying error handler action.
+     * The builder should be in a state where no parents are specified, otherwise an exception is thrown.
+     * @param builder The builder that is used to build the underlying error handler node.
+     * @return A new {@link ErrorHandler}.
+     *
+     * @throws IllegalStateException if the provided builder has parents registered.
+     */
+    public static ErrorHandler buildAsErrorHandler(final Builder<? extends Node> builder) {
+        final Node handlerNode = builder.build();
+        return new ErrorHandler(handlerNode);
+    }
+
+    private ErrorHandler(final Node handlerNode) {
+        final boolean hasParents = !handlerNode.getAllParents().isEmpty();
+        final boolean hasChildren = !handlerNode.getAllChildren().isEmpty();
+        Preconditions.checkState(!hasParents && !hasChildren, "Error handler nodes cannot have parents or children.");
+
+        this.handlerNode = handlerNode;
+    }
+
+    /**
+     * Returns the name of the error handler action.
+     * @return The name of the error handler action.
+     */
+    public String getName() {
+        return handlerNode.getName();
+    }
+
+    /**
+     * Returns the error handler action node.
+     * @return The error handler action node.
+     */
+    public Node getHandlerNode() {
+        return handlerNode;
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/FSAction.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/FSAction.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/FSAction.java
new file mode 100644
index 0000000..5d10e80
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/FSAction.java
@@ -0,0 +1,137 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+import java.util.List;
+import java.util.Map;
+
+/**
+ * A class representing the Oozie file system action.
+ * Instances of this class should be built using the builder {@link FSActionBuilder}.
+ *
+ * The properties of the builder can only be set once, an attempt to set them a second time will trigger
+ * an {@link IllegalStateException}.
+ *
+ * Builder instances can be used to build several elements, although properties already set cannot be changed after
+ * a call to {@link FSActionBuilder#build} either.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class FSAction extends Node implements HasAttributes {
+    private final ActionAttributes attributes;
+
+    FSAction(final Node.ConstructionData constructionData,
+             final ActionAttributes attributes) {
+        super(constructionData);
+
+        this.attributes = attributes;
+    }
+
+    /**
+     * Returns the name node used by this {@link FSAction}.
+     * @return The name node used by this {@link FSAction}.
+     */
+    public String getNameNode() {
+        return attributes.getNameNode();
+    }
+
+    /**
+     * Returns the list of job XMLs used by this {@link FSAction}.
+     * @return The list of job XMLs used by this {@link FSAction}.
+     */
+    public List<String> getJobXmls() {
+        return attributes.getJobXmls();
+    }
+
+    /**
+     * Returns the value associated with the provided configuration property name.
+     * @param property The name of the configuration property for which the value will be returned.
+     * @return The value associated with the provided configuration property name.
+     */
+    public String getConfigProperty(final String property) {
+        return attributes.getConfiguration().get(property);
+    }
+
+    /**
+     * Returns all configuration properties of this {@link FSAction} as a map.
+     * @return All configuration properties of this {@link FSAction} as a map.
+     */
+    public Map<String, String> getConfiguration() {
+        return attributes.getConfiguration();
+    }
+
+    /**
+     * Returns the {@link Delete} objects that specify which directories or files will be deleted when running this action.
+     * @return The {@link Delete} objects that specify which directories or files will be deleted when running this action.
+     */
+    public List<Delete> getDeletes() {
+        return attributes.getDeletes();
+    }
+
+    /**
+     * Returns the {@link Mkdir} objects that specify which directories will be created when running this action.
+     * @return The {@link Mkdir} objects that specify which directories will be created when running this action.
+     */
+    public List<Mkdir> getMkdirs() {
+        return attributes.getMkdirs();
+    }
+
+    /**
+     * Returns the {@link Move} objects that specify which directories or files will be moved and where when running this action.
+     * @return The {@link Move} objects that specify which directories or files will be moved and where when running this action.
+     */
+    public List<Move> getMoves() {
+        return attributes.getMoves();
+    }
+
+    /**
+     * Returns the {@link Chmod} objects that specify the directories or files the permission of which will be changed
+     * and to what when running this action.
+     * @return The {@link Chmod} objects that specify the directories or files the permission of which will be changed
+     *         and to what when running this action.
+     */
+    public List<Chmod> getChmods() {
+        return attributes.getChmods();
+    }
+
+    /**
+     * Returns the {@link Touchz} objects that specify which files will be touched when running this action.
+     * @return The {@link Touchz} objects that specify which files will be touched when running this action.
+     */
+    public List<Touchz> getTouchzs() {
+        return attributes.getTouchzs();
+    }
+
+    /**
+     * Returns the {@link Chgrp} objects that specify the directories or files the owner / group of which will be changed
+     * and to what when running this action.
+     * @return The {@link Chgrp} objects that specify the directories or files the owner / group of which will be changed
+     *         and to what when running this action.
+     */
+    public List<Chgrp> getChgrps() {
+        return attributes.getChgrps();
+    }
+
+    public ActionAttributes getAttributes() {
+        return attributes;
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/FSActionBuilder.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/FSActionBuilder.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/FSActionBuilder.java
new file mode 100644
index 0000000..e70b061
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/FSActionBuilder.java
@@ -0,0 +1,323 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * A builder class for {@link FSAction}.
+ * <p>
+ * The properties of the builder can only be set once, an attempt to set them a second time will trigger
+ * an {@link IllegalStateException}. The properties that are lists are an exception to this rule, of course multiple
+ * elements can be added / removed.
+ * <p>
+ * Builder instances can be used to build several elements, although properties already set cannot be changed after
+ * a call to {@link FSActionBuilder#build} either.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class FSActionBuilder extends NodeBuilderBaseImpl<FSActionBuilder> implements Builder<FSAction> {
+    private final ActionAttributesBuilder attributesBuilder;
+
+    /**
+     * Creates and returns an empty builder.
+     * @return An empty builder.
+     */
+    public static FSActionBuilder create() {
+        final ActionAttributesBuilder builder = ActionAttributesBuilder.create();
+
+        return new FSActionBuilder(
+                null,
+                builder);
+    }
+
+    /**
+     * Create and return a new {@link FSActionBuilder} that is based on an already built
+     * {@link FSAction} object. The properties of the builder will initially be the same as those of the
+     * provided {@link FSAction} object, but it is possible to modify them once.
+     * @param action The {@link FSAction} object on which this {@link FSActionBuilder} will be based.
+     * @return A new {@link FSActionBuilder} that is based on a previously built {@link FSAction} object.
+     */
+    public static FSActionBuilder createFromExistingAction(final Node action) {
+        final ActionAttributesBuilder builder = ActionAttributesBuilder.createFromAction(action);
+
+        return new FSActionBuilder(
+                action,
+                builder);
+    }
+
+    FSActionBuilder(final Node action,
+                    final ActionAttributesBuilder attributesBuilder) {
+        super(action);
+
+        this.attributesBuilder = attributesBuilder;
+    }
+
+    /**
+     * Registers a name node.
+     * @param nameNode The string representing the name node.
+     * @return this
+     * @throws IllegalStateException if a name node has already been set on this builder.
+     */
+    public FSActionBuilder withNameNode(final String nameNode) {
+        attributesBuilder.withNameNode(nameNode);
+        return this;
+    }
+
+    /**
+     * Registers a job XML with this builder.
+     * @param jobXml The job XML to register.
+     * @return this
+     */
+    public FSActionBuilder withJobXml(final String jobXml) {
+        attributesBuilder.withJobXml(jobXml);
+        return this;
+    }
+
+    /**
+     * Removes a job XML if it is registered with this builder, otherwise does nothing.
+     * @param jobXml The job XML to remove.
+     * @return this
+     */
+    public FSActionBuilder withoutJobXml(final String jobXml) {
+        attributesBuilder.withoutJobXml(jobXml);
+        return this;
+    }
+
+    /**
+     * Removes all job XMLs that are registered with this builder.
+     * @return this
+     */
+    public FSActionBuilder clearJobXmls() {
+        attributesBuilder.clearJobXmls();
+        return this;
+    }
+
+    /**
+     * Registers a configuration property (a key-value pair) with this builder. If the provided key has already been
+     * set on this builder, an exception is thrown. Setting a key to null means deleting it.
+     * @param key The name of the property to set.
+     * @param value The value of the property to set.
+     * @return this
+     * @throws IllegalStateException if the provided key has already been set on this builder.
+     */
+    public FSActionBuilder withConfigProperty(final String key, final String value) {
+        attributesBuilder.withConfigProperty(key, value);
+        return this;
+    }
+
+    /**
+     * Registers a {@link Delete} object with this builder.
+     * @param delete The {@link Delete} object to register.
+     * @return this
+     */
+    public FSActionBuilder withDelete(final Delete delete) {
+        attributesBuilder.withDelete(delete);
+        return this;
+    }
+
+    /**
+     * Removes a {@link Delete} object if it is registered with this builder, otherwise does nothing.
+     * @param delete The {@link Delete} object to remove.
+     * @return this
+     */
+    public FSActionBuilder withoutDelete(final Delete delete) {
+        attributesBuilder.withoutDelete(delete);
+        return this;
+    }
+
+    /**
+     * Removes all {@link Delete} objects that are registered with this builder.
+     * @return this
+     */
+    public FSActionBuilder clearDeletes() {
+        attributesBuilder.clearDeletes();
+        return this;
+    }
+
+    /**
+     * Registers a {@link Mkdir} object with this builder.
+     * @param mkdir The {@link Mkdir} object to register.
+     * @return this
+     */
+    public FSActionBuilder withMkdir(final Mkdir mkdir) {
+        attributesBuilder.withMkdir(mkdir);
+        return this;
+    }
+
+    /**
+     * Removes a {@link Mkdir} object if it is registered with this builder, otherwise does nothing.
+     * @param mkdir The {@link Mkdir} object to remove.
+     * @return this
+     */
+    public FSActionBuilder withoutMkdir(final Mkdir mkdir) {
+        attributesBuilder.withoutMkdir(mkdir);
+        return this;
+    }
+
+    /**
+     * Removes all {@link Mkdir} objects that are registered with this builder.
+     * @return this
+     */
+    public FSActionBuilder clearMkdirs() {
+        attributesBuilder.clearMkdirs();
+        return this;
+    }
+
+    /**
+     * Registers a {@link Move} object with this builder.
+     * @param move The {@link Move} object to register.
+     * @return this
+     */
+    public FSActionBuilder withMove(final Move move) {
+        attributesBuilder.withMove(move);
+        return this;
+    }
+
+    /**
+     * Removes a {@link Move} object if it is registered with this builder, otherwise does nothing.
+     * @param move The {@link Move} object to remove.
+     * @return this
+     */
+    public FSActionBuilder withoutMove(final Move move) {
+        attributesBuilder.withoutMove(move);
+        return this;
+    }
+
+    /**
+     * Removes all {@link Move} objects that are registered with this builder.
+     * @return this
+     */
+    public FSActionBuilder clearMoves() {
+        attributesBuilder.clearMoves();
+        return this;
+    }
+
+    /**
+     * Registers a {@link Chmod} object with this builder.
+     * @param chmod The {@link Chmod} object to register.
+     * @return this
+     */
+    public FSActionBuilder withChmod(final Chmod chmod) {
+        attributesBuilder.withChmod(chmod);
+        return this;
+    }
+
+    /**
+     * Removes a {@link Chmod} object if it is registered with this builder, otherwise does nothing.
+     * @param chmod The {@link Chmod} object to remove.
+     * @return this
+     */
+    public FSActionBuilder withoutChmod(final Chmod chmod) {
+        attributesBuilder.withoutChmod(chmod);
+        return this;
+    }
+
+    /**
+     * Removes all {@link Chmod} objects that are registered with this builder.
+     * @return this
+     */
+    public FSActionBuilder clearChmods() {
+        attributesBuilder.clearChmods();
+        return this;
+    }
+
+    /**
+     * Registers a {@link Touchz} object with this builder.
+     * @param touchz The {@link Touchz} object to register.
+     * @return this
+     */
+    public FSActionBuilder withTouchz(final Touchz touchz) {
+        attributesBuilder.withTouchz(touchz);
+        return this;
+    }
+
+    /**
+     * Removes a {@link Touchz} object if it is registered with this builder, otherwise does nothing.
+     * @param touchz The {@link Touchz} object to remove.
+     * @return this
+     */
+    public FSActionBuilder withoutTouchz(final Touchz touchz) {
+        attributesBuilder.withoutTouchz(touchz);
+        return this;
+    }
+
+    /**
+     * Removes all {@link Touchz} objects that are registered with this builder.
+     * @return this
+     */
+    public FSActionBuilder clearTouchzs() {
+        attributesBuilder.clearTouchzs();
+        return this;
+    }
+
+    /**
+     * Registers a {@link Chgrp} object with this builder.
+     * @param chgrp The {@link Chgrp} object to register.
+     * @return this
+     */
+    public FSActionBuilder withChgrp(final Chgrp chgrp) {
+        attributesBuilder.withChgrp(chgrp);
+        return this;
+    }
+
+    /**
+     * Removes a {@link Chgrp} object if it is registered with this builder, otherwise does nothing.
+     * @param chgrp The {@link Chgrp} object to remove.
+     * @return this
+     */
+    public FSActionBuilder withoutChgrp(final Chgrp chgrp) {
+        attributesBuilder.withoutChgrp(chgrp);
+        return this;
+    }
+
+    /**
+     * Removes all {@link Chgrp} objects that are registered with this builder.
+     * @return this
+     */
+    public FSActionBuilder clearChgrps() {
+        attributesBuilder.clearChgrps();
+        return this;
+    }
+
+    /**
+     * Creates a new {@link FSAction} object with the properties stores in this builder.
+     * The new {@link FSAction} object is independent of this builder and the builder can be used to build
+     * new instances.
+     * @return A new {@link FSAction} object with the properties stored in this builder.
+     */
+    @Override
+    public FSAction build() {
+        final Node.ConstructionData constructionData = getConstructionData();
+
+        final FSAction instance = new FSAction(
+                constructionData,
+                attributesBuilder.build());
+
+        addAsChildToAllParents(instance);
+
+        return instance;
+    }
+
+    @Override
+    protected FSActionBuilder getRuntimeSelfReference() {
+        return this;
+    }
+}


[16/16] oozie git commit: OOZIE-2339 [fluent-job] Minimum Viable Fluent Job API (daniel.becker, andras.piros via rkanter, gezapeti, pbacsko)

Posted by an...@apache.org.
OOZIE-2339 [fluent-job] Minimum Viable Fluent Job API (daniel.becker, andras.piros via rkanter, gezapeti, pbacsko)


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

Branch: refs/heads/master
Commit: 8a0a6487dcf9ef2e4489b5bdeea2360e12d91b14
Parents: f8cbce6
Author: Andras Piros <an...@cloudera.com>
Authored: Tue Jun 19 11:11:06 2018 +0200
Committer: Andras Piros <an...@cloudera.com>
Committed: Tue Jun 19 11:13:28 2018 +0200

----------------------------------------------------------------------
 client/pom.xml                                  |   9 +
 .../java/org/apache/oozie/cli/OozieCLI.java     | 115 ++-
 .../org/apache/oozie/client/ApiJarFactory.java  | 117 +++
 .../org/apache/oozie/client/ApiJarLoader.java   |  78 ++
 .../org/apache/oozie/client/OozieClient.java    |  82 +-
 core/pom.xml                                    |  11 +
 .../org/apache/oozie/BaseLocalOozieClient.java  |  13 +-
 .../java/org/apache/oozie/local/LocalOozie.java |   4 +
 .../apache/oozie/servlet/BaseJobServlet.java    |   3 +-
 .../apache/oozie/servlet/BaseJobsServlet.java   |  43 +
 .../apache/oozie/servlet/ServletUtilities.java  |   4 +-
 .../org/apache/oozie/servlet/V0JobsServlet.java |   5 +
 .../org/apache/oozie/servlet/V1JobsServlet.java | 141 ++-
 .../apache/oozie/servlet/V2ValidateServlet.java |   5 +-
 core/src/main/resources/oozie-default.xml       |  11 +
 .../org/apache/oozie/client/TestOozieCLI.java   | 191 +++-
 .../apache/oozie/servlet/TestV1JobsServlet.java |  12 +-
 .../apache/oozie/test/MiniOozieTestCase.java    |   8 +-
 docs/src/site/twiki/DG_CommandLineTool.twiki    | 139 +++
 .../site/twiki/DG_CustomActionExecutor.twiki    |   3 +
 docs/src/site/twiki/DG_Examples.twiki           |   7 +
 docs/src/site/twiki/DG_FluentJobAPI.twiki       | 376 ++++++++
 examples/pom.xml                                |   6 +
 .../example/fluentjob/CredentialsRetrying.java  |  93 ++
 .../apache/oozie/example/fluentjob/Global.java  |  49 +
 .../oozie/example/fluentjob/JavaMain.java       |  56 ++
 .../example/fluentjob/MultipleShellActions.java |  79 ++
 .../oozie/example/fluentjob/Parameters.java     |  39 +
 .../apache/oozie/example/fluentjob/Shell.java   |  67 ++
 .../apache/oozie/example/fluentjob/Spark.java   |  61 ++
 findbugs-filter.xml                             |  12 +
 fluent-job/fluent-job-api/pom.xml               | 212 +++++
 .../apache/oozie/fluentjob/api/Condition.java   | 102 ++
 .../oozie/fluentjob/api/GraphVisualization.java | 117 +++
 .../apache/oozie/fluentjob/api/ModifyOnce.java  |  67 ++
 .../fluentjob/api/action/ActionAttributes.java  | 258 +++++
 .../api/action/ActionAttributesBuilder.java     | 567 +++++++++++
 .../oozie/fluentjob/api/action/Builder.java     |  31 +
 .../oozie/fluentjob/api/action/ChFSBase.java    |  81 ++
 .../fluentjob/api/action/ChFSBaseBuilder.java   |  95 ++
 .../oozie/fluentjob/api/action/Chgrp.java       |  53 ++
 .../fluentjob/api/action/ChgrpBuilder.java      |  68 ++
 .../oozie/fluentjob/api/action/Chmod.java       |  52 +
 .../fluentjob/api/action/ChmodBuilder.java      |  68 ++
 .../oozie/fluentjob/api/action/Delete.java      |  60 ++
 .../fluentjob/api/action/DistcpAction.java      |  80 ++
 .../api/action/DistcpActionBuilder.java         | 118 +++
 .../oozie/fluentjob/api/action/EmailAction.java | 118 +++
 .../api/action/EmailActionBuilder.java          | 234 +++++
 .../fluentjob/api/action/ErrorHandler.java      |  78 ++
 .../oozie/fluentjob/api/action/FSAction.java    | 137 +++
 .../fluentjob/api/action/FSActionBuilder.java   | 323 +++++++
 .../fluentjob/api/action/HasAttributes.java     |  31 +
 .../oozie/fluentjob/api/action/Hive2Action.java |  65 ++
 .../api/action/Hive2ActionBuilder.java          | 259 +++++
 .../oozie/fluentjob/api/action/HiveAction.java  |  56 ++
 .../fluentjob/api/action/HiveActionBuilder.java | 242 +++++
 .../oozie/fluentjob/api/action/JavaAction.java  | 119 +++
 .../fluentjob/api/action/JavaActionBuilder.java | 235 +++++
 .../oozie/fluentjob/api/action/Launcher.java    |  81 ++
 .../fluentjob/api/action/LauncherBuilder.java   |  92 ++
 .../fluentjob/api/action/MapReduceAction.java   | 138 +++
 .../api/action/MapReduceActionBuilder.java      | 256 +++++
 .../oozie/fluentjob/api/action/Mkdir.java       |  48 +
 .../apache/oozie/fluentjob/api/action/Move.java |  58 ++
 .../apache/oozie/fluentjob/api/action/Node.java | 282 ++++++
 .../api/action/NodeBuilderBaseImpl.java         | 340 +++++++
 .../oozie/fluentjob/api/action/PigAction.java   | 107 +++
 .../fluentjob/api/action/PigActionBuilder.java  | 215 +++++
 .../oozie/fluentjob/api/action/Pipes.java       | 106 +++
 .../fluentjob/api/action/PipesBuilder.java      | 127 +++
 .../oozie/fluentjob/api/action/Prepare.java     |  58 ++
 .../fluentjob/api/action/PrepareBuilder.java    |  85 ++
 .../oozie/fluentjob/api/action/ShellAction.java | 111 +++
 .../api/action/ShellActionBuilder.java          | 220 +++++
 .../oozie/fluentjob/api/action/SparkAction.java | 134 +++
 .../api/action/SparkActionBuilder.java          | 261 +++++
 .../oozie/fluentjob/api/action/SqoopAction.java |  99 ++
 .../api/action/SqoopActionBuilder.java          | 186 ++++
 .../oozie/fluentjob/api/action/SshAction.java   |  74 ++
 .../fluentjob/api/action/SshActionBuilder.java  | 136 +++
 .../oozie/fluentjob/api/action/Streaming.java   |  98 ++
 .../fluentjob/api/action/StreamingBuilder.java  | 117 +++
 .../fluentjob/api/action/SubWorkflowAction.java |  88 ++
 .../api/action/SubWorkflowActionBuilder.java    | 159 ++++
 .../oozie/fluentjob/api/action/Touchz.java      |  47 +
 .../fluentjob/api/dag/DagNodeWithCondition.java | 108 +++
 .../oozie/fluentjob/api/dag/Decision.java       | 190 ++++
 .../oozie/fluentjob/api/dag/DecisionJoin.java   |  51 +
 .../org/apache/oozie/fluentjob/api/dag/End.java | 121 +++
 .../oozie/fluentjob/api/dag/ExplicitNode.java   | 150 +++
 .../apache/oozie/fluentjob/api/dag/Fork.java    | 142 +++
 .../apache/oozie/fluentjob/api/dag/Graph.java   | 872 +++++++++++++++++
 .../apache/oozie/fluentjob/api/dag/Join.java    |  36 +
 .../fluentjob/api/dag/JoiningNodeBase.java      | 126 +++
 .../oozie/fluentjob/api/dag/NodeBase.java       | 104 ++
 .../apache/oozie/fluentjob/api/dag/Start.java   | 120 +++
 .../fluentjob/api/factory/WorkflowFactory.java  |  35 +
 .../api/mapping/BooleanToFLAGConverter.java     |  50 +
 .../mapping/BooleanToShellFLAGConverter.java    |  50 +
 .../api/mapping/BooleanToSshFLAGConverter.java  |  50 +
 .../api/mapping/CredentialsConverter.java       |  94 ++
 .../api/mapping/DecisionConverter.java          | 122 +++
 .../mapping/DistcpConfigurationConverter.java   |  81 ++
 .../api/mapping/DistcpPrepareConverter.java     |  97 ++
 .../api/mapping/DozerBeanMapperSingleton.java   |  50 +
 .../api/mapping/ExplicitNodeConverter.java      | 296 ++++++
 .../fluentjob/api/mapping/ForkConverter.java    |  76 ++
 .../fluentjob/api/mapping/GlobalConverter.java  | 100 ++
 .../oozie/fluentjob/api/mapping/GraphNodes.java |  86 ++
 .../GraphNodesToWORKFLOWAPPConverter.java       | 257 +++++
 .../mapping/GraphToWORKFLOWAPPConverter.java    |  67 ++
 .../mapping/Hive2ConfigurationConverter.java    |  81 ++
 .../api/mapping/Hive2LauncherConverter.java     |  74 ++
 .../api/mapping/Hive2PrepareConverter.java      |  97 ++
 .../api/mapping/HiveConfigurationConverter.java |  81 ++
 .../api/mapping/HiveLauncherConverter.java      |  74 ++
 .../api/mapping/HivePrepareConverter.java       |  97 ++
 .../InlineWorkflowConfigurationConverter.java   |  81 ++
 .../InlineWorkflowLauncherConverter.java        |  74 ++
 .../mapping/InlineWorkflowPrepareConverter.java |  97 ++
 .../fluentjob/api/mapping/JoinConverter.java    |  63 ++
 .../MapToConfigurationPropertyConverter.java    |  75 ++
 .../api/mapping/ParametersConverter.java        |  73 ++
 .../fluentjob/api/mapping/RealChildLocator.java |  35 +
 .../mapping/ShellConfigurationConverter.java    |  81 ++
 .../api/mapping/ShellLauncherConverter.java     |  74 ++
 .../api/mapping/ShellPrepareConverter.java      |  97 ++
 .../mapping/SparkConfigurationConverter.java    |  81 ++
 .../api/mapping/SparkLauncherConverter.java     |  74 ++
 .../api/mapping/SparkPrepareConverter.java      |  97 ++
 .../mapping/SqoopConfigurationConverter.java    |  81 ++
 .../api/mapping/SqoopLauncherConverter.java     |  74 ++
 .../api/mapping/SqoopPrepareConverter.java      |  97 ++
 .../fluentjob/api/mapping/StartConverter.java   |  50 +
 .../api/serialization/WorkflowMarshaller.java   | 105 +++
 .../api/workflow/ConfigurationEntry.java        |  61 ++
 .../fluentjob/api/workflow/Credential.java      |  60 ++
 .../api/workflow/CredentialBuilder.java         |  89 ++
 .../fluentjob/api/workflow/Credentials.java     |  48 +
 .../api/workflow/CredentialsBuilder.java        |  77 ++
 .../oozie/fluentjob/api/workflow/Global.java    |  67 ++
 .../fluentjob/api/workflow/GlobalBuilder.java   |  94 ++
 .../oozie/fluentjob/api/workflow/Parameter.java |  57 ++
 .../fluentjob/api/workflow/Parameters.java      |  48 +
 .../api/workflow/ParametersBuilder.java         |  66 ++
 .../oozie/fluentjob/api/workflow/Workflow.java  | 141 +++
 .../fluentjob/api/workflow/WorkflowBuilder.java | 204 ++++
 .../src/main/resources/action_mappings.xml      | 821 ++++++++++++++++
 .../src/main/resources/checkstyle-header.txt    |  17 +
 .../src/main/resources/checkstyle.xml           |  41 +
 .../src/main/resources/dozer_config.xml         |  94 ++
 .../resources/mappingGraphToWORKFLOWAPP.xml     |  50 +
 .../fluent-job-api/src/main/xjb/bindings.xml    | 166 ++++
 .../apache/oozie/fluentjob/api/NodesToPng.java  |  52 +
 .../oozie/fluentjob/api/TestCondition.java      |  43 +
 .../oozie/fluentjob/api/TestModifyOnce.java     |  52 +
 .../api/action/TestActionAttributesBuilder.java | 713 ++++++++++++++
 .../fluentjob/api/action/TestChBaseBuilder.java | 127 +++
 .../fluentjob/api/action/TestChgrpBuilder.java  |  53 ++
 .../fluentjob/api/action/TestChmodBuilder.java  |  53 ++
 .../oozie/fluentjob/api/action/TestDelete.java  |  36 +
 .../api/action/TestDistcpActionBuilder.java     | 213 +++++
 .../api/action/TestEmailActionBuilder.java      | 197 ++++
 .../fluentjob/api/action/TestErrorHandler.java  |  51 +
 .../api/action/TestFSActionBuilder.java         | 469 +++++++++
 .../api/action/TestHive2ActionBuilder.java      | 224 +++++
 .../api/action/TestHiveActionBuilder.java       | 220 +++++
 .../api/action/TestJavaActionBuilder.java       | 225 +++++
 .../api/action/TestLauncherBuilder.java         |  44 +
 .../api/action/TestMapReduceActionBuilder.java  | 392 ++++++++
 .../oozie/fluentjob/api/action/TestMove.java    |  36 +
 .../api/action/TestNodeBuilderBaseImpl.java     | 523 +++++++++++
 .../api/action/TestPigActionBuilder.java        | 218 +++++
 .../fluentjob/api/action/TestPipesBuilder.java  | 168 ++++
 .../api/action/TestPrepareBuilder.java          | 105 +++
 .../api/action/TestShellActionBuilder.java      | 223 +++++
 .../api/action/TestSparkActionBuilder.java      | 229 +++++
 .../api/action/TestSqoopActionBuilder.java      | 219 +++++
 .../api/action/TestSshActionBuilder.java        | 147 +++
 .../api/action/TestStreamingBuilder.java        | 127 +++
 .../api/action/TestSubWorkflowBuilder.java      | 185 ++++
 .../oozie/fluentjob/api/action/TestTouchz.java  |  33 +
 .../oozie/fluentjob/api/dag/TestDecision.java   | 210 +++++
 .../fluentjob/api/dag/TestDecisionJoin.java     |  36 +
 .../apache/oozie/fluentjob/api/dag/TestEnd.java | 117 +++
 .../fluentjob/api/dag/TestExplicitNode.java     | 156 +++
 .../oozie/fluentjob/api/dag/TestFork.java       | 156 +++
 .../oozie/fluentjob/api/dag/TestGraph.java      | 941 +++++++++++++++++++
 .../oozie/fluentjob/api/dag/TestJoin.java       |  52 +
 .../fluentjob/api/dag/TestJoiningNodeBase.java  | 157 ++++
 .../oozie/fluentjob/api/dag/TestNodeBase.java   |  40 +
 .../oozie/fluentjob/api/dag/TestStart.java      | 113 +++
 .../api/factory/SimpleWorkflowFactory.java      |  65 ++
 .../api/mapping/SourceDataFactory.java          |  48 +
 .../mapping/TestActionAttributesMapping.java    | 100 ++
 .../api/mapping/TestConfigurationMapping.java   |  45 +
 .../api/mapping/TestControlNodeMappingBase.java |  27 +
 .../api/mapping/TestCredentialsMapping.java     |  37 +
 .../api/mapping/TestDecisionMapping.java        | 113 +++
 .../api/mapping/TestDeleteMapping.java          |  38 +
 .../api/mapping/TestDistcpActionMapping.java    |  66 ++
 .../api/mapping/TestEmailActionMapping.java     |  60 ++
 .../fluentjob/api/mapping/TestEndMapping.java   |  37 +
 .../api/mapping/TestExplicitNodeMapping.java    |  47 +
 .../api/mapping/TestFSActionMapping.java        |  77 ++
 .../fluentjob/api/mapping/TestForkMapping.java  |  70 ++
 .../api/mapping/TestGlobalMapping.java          |  56 ++
 .../fluentjob/api/mapping/TestGraphMapping.java | 255 +++++
 .../api/mapping/TestHive2ActionMapping.java     |  84 ++
 .../api/mapping/TestHiveActionMapping.java      |  84 ++
 .../api/mapping/TestJavaActionMapping.java      |  95 ++
 .../fluentjob/api/mapping/TestJoinMapping.java  |  64 ++
 .../api/mapping/TestMapReduceActionMapping.java |  96 ++
 .../fluentjob/api/mapping/TestMappings.java     |  59 ++
 .../fluentjob/api/mapping/TestMkdirMapping.java |  38 +
 .../api/mapping/TestParametersMapping.java      |  47 +
 .../api/mapping/TestPigActionMapping.java       |  84 ++
 .../fluentjob/api/mapping/TestPipesMapping.java |  57 ++
 .../api/mapping/TestPrepareMapping.java         |  65 ++
 .../api/mapping/TestShellActionMapping.java     |  92 ++
 .../api/mapping/TestSparkActionMapping.java     |  98 ++
 .../api/mapping/TestSqoopActionMapping.java     |  89 ++
 .../api/mapping/TestSshActionMapping.java       |  59 ++
 .../fluentjob/api/mapping/TestStartMapping.java |  61 ++
 .../api/mapping/TestStreamingMapping.java       |  64 ++
 .../mapping/TestSubWorkflowActionMapping.java   |  47 +
 .../mapping/TestWorkflowAttributesMapping.java  |  53 ++
 .../api/workflow/TestCredentialBuilder.java     |  56 ++
 .../api/workflow/TestCredentialsBuilder.java    |  66 ++
 .../api/workflow/TestGlobalBuilder.java         |  63 ++
 .../api/workflow/TestParametersBuilder.java     |  80 ++
 .../api/workflow/TestWorkflowBuilder.java       | 258 +++++
 fluent-job/fluent-job-client/pom.xml            |  99 ++
 .../jobs/client/jaxb/TestJAXBWorkflow.java      | 435 +++++++++
 .../jobs/client/minitest/TestDistcpAction.java  |  89 ++
 .../jobs/client/minitest/TestEmailAction.java   |  73 ++
 .../jobs/client/minitest/TestFSAction.java      |  72 ++
 .../jobs/client/minitest/TestHive2Action.java   |  83 ++
 .../jobs/client/minitest/TestHiveAction.java    |  81 ++
 .../jobs/client/minitest/TestJavaAction.java    |  92 ++
 .../client/minitest/TestMapReduceAction.java    |  76 ++
 .../jobs/client/minitest/TestPigAction.java     |  81 ++
 .../jobs/client/minitest/TestShellAction.java   |  90 ++
 .../jobs/client/minitest/TestSparkAction.java   |  87 ++
 .../jobs/client/minitest/TestSqoopAction.java   |  80 ++
 .../jobs/client/minitest/TestSshAction.java     |  74 ++
 .../src/test/resources/workflow-all-actions.xml | 234 +++++
 .../resources/workflow-mapreduce-action.xml     |  63 ++
 fluent-job/pom.xml                              |  62 ++
 minitest/pom.xml                                |  18 +-
 .../org/apache/oozie/test/TestWorkflow.java     | 159 +---
 .../org/apache/oozie/test/WorkflowTestCase.java | 259 +++++
 pom.xml                                         |  72 +-
 release-log.txt                                 |   1 +
 255 files changed, 29670 insertions(+), 198 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/client/pom.xml
----------------------------------------------------------------------
diff --git a/client/pom.xml b/client/pom.xml
index 7d36e2d..118f1c3 100644
--- a/client/pom.xml
+++ b/client/pom.xml
@@ -95,6 +95,15 @@
             <artifactId>xercesImpl</artifactId>
             <scope>compile</scope>
         </dependency>
+        <dependency>
+            <groupId>${project.groupId}</groupId>
+            <artifactId>oozie-fluent-job-api</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>com.google.code.findbugs</groupId>
+            <artifactId>annotations</artifactId>
+            <scope>provided</scope>
+        </dependency>
     </dependencies>
 
     <build>

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/client/src/main/java/org/apache/oozie/cli/OozieCLI.java
----------------------------------------------------------------------
diff --git a/client/src/main/java/org/apache/oozie/cli/OozieCLI.java b/client/src/main/java/org/apache/oozie/cli/OozieCLI.java
index 08e2b91..bc234e3 100644
--- a/client/src/main/java/org/apache/oozie/cli/OozieCLI.java
+++ b/client/src/main/java/org/apache/oozie/cli/OozieCLI.java
@@ -19,13 +19,16 @@
 package org.apache.oozie.cli;
 
 import com.google.common.annotations.VisibleForTesting;
+import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.Option;
 import org.apache.commons.cli.OptionBuilder;
 import org.apache.commons.cli.OptionGroup;
 import org.apache.commons.cli.Options;
 import org.apache.commons.cli.ParseException;
+import org.apache.commons.io.FilenameUtils;
 import org.apache.oozie.BuildInfo;
+import org.apache.oozie.client.ApiJarLoader;
 import org.apache.oozie.client.AuthOozieClient;
 import org.apache.oozie.client.BulkResponse;
 import org.apache.oozie.client.BundleJob;
@@ -40,6 +43,8 @@ import org.apache.oozie.client.XOozieClient;
 import org.apache.oozie.client.rest.JsonTags;
 import org.apache.oozie.client.rest.JsonToBean;
 import org.apache.oozie.client.rest.RestConstants;
+import org.apache.oozie.fluentjob.api.serialization.WorkflowMarshaller;
+import org.apache.oozie.fluentjob.api.workflow.Workflow;
 import org.json.simple.JSONArray;
 import org.json.simple.JSONObject;
 import org.w3c.dom.DOMException;
@@ -50,6 +55,7 @@ import org.w3c.dom.NodeList;
 import org.w3c.dom.Text;
 import org.xml.sax.SAXException;
 
+import javax.xml.bind.JAXBException;
 import javax.xml.parsers.DocumentBuilder;
 import javax.xml.parsers.DocumentBuilderFactory;
 import javax.xml.parsers.ParserConfigurationException;
@@ -59,6 +65,10 @@ import java.io.FileReader;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.PrintStream;
+import java.lang.reflect.InvocationTargetException;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
 import java.text.SimpleDateFormat;
 import java.util.ArrayList;
 import java.util.Date;
@@ -113,6 +123,9 @@ public class OozieCLI {
     public static final String LOG_OPTION = "log";
     public static final String ERROR_LOG_OPTION = "errorlog";
     public static final String AUDIT_LOG_OPTION = "auditlog";
+    public static final String VALIDATE_JAR_OPTION = "validatejar";
+    public static final String SUBMIT_JAR_OPTION = "submitjar";
+    public static final String RUN_JAR_OPTION = "runjar";
 
     public static final String ACTION_OPTION = "action";
     public static final String DEFINITION_OPTION = "definition";
@@ -344,6 +357,9 @@ public class OozieCLI {
         Option log = new Option(LOG_OPTION, true, "job log");
         Option errorlog = new Option(ERROR_LOG_OPTION, true, "job error log");
         Option auditlog = new Option(AUDIT_LOG_OPTION, true, "job audit log");
+        final Option generateAndCheck = new Option(VALIDATE_JAR_OPTION, true, "generate and check job definition");
+        final Option generateAndSubmit = new Option(SUBMIT_JAR_OPTION, true, "generate and submit job definition");
+        final Option generateAndRun = new Option(RUN_JAR_OPTION, true, "generate and run job definition");
         Option logFilter = new Option(
                 RestConstants.LOG_FILTER_OPTION, true,
                 "job log search parameter. Can be specified as -logfilter opt1=val1;opt2=val1;opt3=val1. "
@@ -407,6 +423,9 @@ public class OozieCLI {
         actions.addOption(log);
         actions.addOption(errorlog);
         actions.addOption(auditlog);
+        actions.addOption(generateAndCheck);
+        actions.addOption(generateAndSubmit);
+        actions.addOption(generateAndRun);
         actions.addOption(definition);
         actions.addOption(config_content);
         actions.addOption(ignore);
@@ -1348,13 +1367,105 @@ public class OozieCLI {
                 wc.getCoordActionMissingDependencies(commandLine.getOptionValue(COORD_ACTION_MISSING_DEPENDENCIES),
                         actions, dates, System.out);
             }
-
+            else if (options.contains(VALIDATE_JAR_OPTION)) {
+                checkApiJar(wc, commandLine, options.contains(VERBOSE_OPTION));
+            }
+            else if (options.contains(SUBMIT_JAR_OPTION)) {
+                submitApiJar(wc, commandLine, options.contains(VERBOSE_OPTION));
+            }
+            else if (options.contains(RUN_JAR_OPTION)) {
+                runApiJar(wc, commandLine, options.contains(VERBOSE_OPTION));
+            }
         }
-        catch (OozieClientException ex) {
+        catch (final OozieClientException ex) {
             throw new OozieCLIException(ex.toString(), ex);
         }
     }
 
+    private void checkApiJar(final XOozieClient wc, final CommandLine commandLine, final boolean verbose)
+            throws OozieClientException {
+        final String apiJarPath = commandLine.getOptionValue(VALIDATE_JAR_OPTION);
+        logIfVerbose(verbose, "Checking API jar: " + apiJarPath);
+
+        final String generatedXml = loadApiJarAndGenerateXml(apiJarPath, verbose);
+
+        final Path workflowXml;
+        try {
+            workflowXml = Files.createTempFile("workflow", ".xml");
+            Files.write(workflowXml, generatedXml.getBytes(StandardCharsets.UTF_8));
+
+            logIfVerbose(verbose, "API jar was written to " + workflowXml.toString());
+        }
+        catch (final IOException e) {
+            throw new OozieClientException(e.getMessage(), e);
+        }
+
+        logIfVerbose(verbose, "Servlet response is: ");
+        System.out.println(wc.validateXML(workflowXml.toString()));
+
+        logIfVerbose(verbose, "API jar is valid.");
+    }
+
+    private void logIfVerbose(final boolean verbose, final String message) {
+        if (verbose) {
+            System.out.println(message);
+        }
+    }
+
+    @SuppressFBWarnings(value = {"PATH_TRAVERSAL_IN", "WEAK_FILENAMEUTILS"},
+            justification = "FilenameUtils is used to filter user input. JDK8+ is used.")
+    private String loadApiJarAndGenerateXml(final String apiJarPath, final boolean verbose) throws OozieClientException {
+        final String generatedXml;
+        try {
+            logIfVerbose(verbose, "Loading API jar " + apiJarPath);
+
+            final Workflow generatedWorkflow = new ApiJarLoader(new File(
+                    FilenameUtils.getFullPath(apiJarPath) + FilenameUtils.getName(apiJarPath)))
+                    .loadAndGenerate();
+            generatedXml = WorkflowMarshaller.marshal(generatedWorkflow);
+
+            logIfVerbose(verbose, "Workflow job definition generated from API jar: \n" + generatedXml);
+        }
+        catch (final IOException | ClassNotFoundException | IllegalAccessException | NoSuchMethodException |
+                InvocationTargetException | InstantiationException | JAXBException e) {
+            throw new OozieClientException(e.getMessage(), e);
+        }
+
+        return generatedXml;
+    }
+
+    private void submitApiJar(final XOozieClient wc, final CommandLine commandLine, final boolean verbose)
+            throws OozieClientException {
+        final String apiJarPath = commandLine.getOptionValue(SUBMIT_JAR_OPTION);
+        logIfVerbose(verbose, "Submitting a job based on API jar: " + apiJarPath);
+
+        try {
+            System.out.println(JOB_ID_PREFIX + wc.submit(getConfiguration(wc, commandLine),
+                    loadApiJarAndGenerateXml(apiJarPath, verbose)));
+        }
+        catch (final IOException e) {
+            throw new OozieClientException(e.getMessage(), e);
+        }
+
+        logIfVerbose(verbose, "Job based on API jar submitted successfully.");
+    }
+
+    private void runApiJar(final XOozieClient wc, final CommandLine commandLine, final boolean verbose)
+            throws OozieClientException {
+        final String apiJarPath = commandLine.getOptionValue(RUN_JAR_OPTION);
+        logIfVerbose(verbose, "Running a job based on API jar: " + apiJarPath);
+
+        try {
+            System.out.println(JOB_ID_PREFIX + wc.run(getConfiguration(wc, commandLine),
+                    loadApiJarAndGenerateXml(apiJarPath, verbose)));
+        }
+        catch (final IOException e) {
+            throw new OozieClientException(e.getMessage(), e);
+        }
+
+        logIfVerbose(verbose, "Job based on API jar run successfully.");
+    }
+
     @VisibleForTesting
     void printCoordJob(CoordinatorJob coordJob, String timeZoneId, boolean verbose) {
         System.out.println("Job ID : " + coordJob.getId());

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/client/src/main/java/org/apache/oozie/client/ApiJarFactory.java
----------------------------------------------------------------------
diff --git a/client/src/main/java/org/apache/oozie/client/ApiJarFactory.java b/client/src/main/java/org/apache/oozie/client/ApiJarFactory.java
new file mode 100644
index 0000000..bb85d67
--- /dev/null
+++ b/client/src/main/java/org/apache/oozie/client/ApiJarFactory.java
@@ -0,0 +1,117 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.client;
+
+import com.google.common.base.Preconditions;
+import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
+import org.apache.commons.io.FilenameUtils;
+import org.apache.commons.io.IOUtils;
+import org.apache.oozie.fluentjob.api.factory.WorkflowFactory;
+
+import java.io.BufferedInputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.util.jar.Attributes;
+import java.util.jar.JarEntry;
+import java.util.jar.JarFile;
+import java.util.jar.JarOutputStream;
+import java.util.jar.Manifest;
+
+/**
+ * Given a folder where the {@code .class} file containing a {@link WorkflowFactory} subclass persist, a name, and the
+ * exact {@code Class} reference, creates a {@code .jar} file containing {@code Main-Class} manifest attribute
+ * pointing to {@code apiFactoryClass}.
+ * <p>
+ * Used mainly by unit tests, this class helps to dynamically assemble Fluent Job API {@code .jar} files.
+ */
+class ApiJarFactory {
+    private final File classFolder;
+    private final File jarFolder;
+    private final String apiJarName;
+    private final Class<? extends WorkflowFactory> apiFactoryClass;
+
+    ApiJarFactory(final File classFolder,
+                         File jarFolder, final Class<? extends WorkflowFactory> apiFactoryClass, final String apiJarName) {
+        Preconditions.checkNotNull(classFolder, "classFolder should be set");
+        Preconditions.checkNotNull(jarFolder, "jarFolder should be set");
+        Preconditions.checkNotNull(apiJarName, "apiJarName should be set");
+        Preconditions.checkNotNull(apiFactoryClass, "apiFactoryClass should be set");
+        Preconditions.checkState(WorkflowFactory.class.isAssignableFrom(apiFactoryClass),
+                String.format("%s should be a %s", apiFactoryClass.getName(), WorkflowFactory.class.getName()));
+
+        this.classFolder = classFolder;
+        this.jarFolder = jarFolder;
+        this.apiJarName = apiJarName;
+        this.apiFactoryClass = apiFactoryClass;
+    }
+
+    @SuppressFBWarnings(value = {"PATH_TRAVERSAL_OUT", "WEAK_FILENAMEUTILS"},
+            justification = "FilenameUtils is used to filter user output. JDK8+ is used.")
+    JarFile create() throws IOException {
+        final Manifest manifest = new Manifest();
+        manifest.getMainAttributes().put(Attributes.Name.MANIFEST_VERSION, "1.0");
+        manifest.getMainAttributes().put(Attributes.Name.MAIN_CLASS, apiFactoryClass.getName());
+
+        final String apiJarPath = jarFolder + File.separator + FilenameUtils.getName(apiJarName);
+        try (final JarOutputStream target = new JarOutputStream(
+                new FileOutputStream(FilenameUtils.getFullPath(apiJarPath) + FilenameUtils.getName(apiJarPath)), manifest)) {
+            addWorkflowJarEntry(classFolder, target);
+        }
+
+        return new JarFile(apiJarPath);
+    }
+
+    private void addWorkflowJarEntry(final File source, final JarOutputStream target) throws IOException {
+        if (source.isDirectory()) {
+            String name = source.getPath().replace("\\", "/");
+            if (!name.isEmpty()) {
+                if (!name.endsWith("/")) {
+                    name += "/";
+                }
+                final JarEntry entry = new JarEntry(name);
+                entry.setTime(source.lastModified());
+                target.putNextEntry(entry);
+                target.closeEntry();
+            }
+
+            final File[] nestedFiles = source.listFiles();
+            if (nestedFiles == null) {
+                return;
+            }
+
+            for (final File nestedFile : nestedFiles) {
+                addWorkflowJarEntry(nestedFile, target);
+            }
+
+            return;
+        }
+
+        try (final BufferedInputStream in = new BufferedInputStream(new FileInputStream(source))) {
+
+            final JarEntry entry = new JarEntry(source.getPath().replace("\\", "/"));
+            entry.setTime(source.lastModified());
+            target.putNextEntry(entry);
+
+            IOUtils.copy(in, target);
+            target.closeEntry();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/client/src/main/java/org/apache/oozie/client/ApiJarLoader.java
----------------------------------------------------------------------
diff --git a/client/src/main/java/org/apache/oozie/client/ApiJarLoader.java b/client/src/main/java/org/apache/oozie/client/ApiJarLoader.java
new file mode 100644
index 0000000..6ecf485
--- /dev/null
+++ b/client/src/main/java/org/apache/oozie/client/ApiJarLoader.java
@@ -0,0 +1,78 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.client;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
+import org.apache.oozie.fluentjob.api.factory.WorkflowFactory;
+import org.apache.oozie.fluentjob.api.workflow.Workflow;
+
+import java.io.File;
+import java.io.IOException;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.net.URL;
+import java.net.URLClassLoader;
+import java.util.jar.Attributes;
+import java.util.jar.JarFile;
+
+/**
+ * Given a Fluent Job API {@code .jar} file compiled by the user this class loads the {@code Class} instance defined by
+ * {@code Main-Class} manifest attribute, calls its {@code Workflow create()} method, and returns its output to the caller.
+ */
+public class ApiJarLoader {
+    private final File apiJarFile;
+
+    public ApiJarLoader(final File apiJarFile) {
+        Preconditions.checkArgument(apiJarFile.isFile(), "Fluent Job API JAR [%s] should be a file", apiJarFile.toString());
+        Preconditions.checkArgument(apiJarFile.getName().endsWith(".jar"), "Fluent Job API JAR [%s] should be a JAR file",
+                apiJarFile.toString());
+
+        this.apiJarFile = apiJarFile;
+    }
+
+    public Workflow loadAndGenerate() throws IOException, ClassNotFoundException, NoSuchMethodException, IllegalAccessException,
+            InstantiationException, InvocationTargetException {
+        final String mainClassName = getMainClassName();
+        Preconditions.checkState(!Strings.isNullOrEmpty(mainClassName),
+                "Fluent Job API JAR should have a Main-Class defined in MANIFEST.MF");
+
+        final URLClassLoader workflowFactoryClassLoader = URLClassLoader.newInstance(new URL[]{apiJarFile.toURI().toURL()});
+        final Class mainClass = workflowFactoryClassLoader.loadClass(mainClassName);
+
+        Preconditions.checkNotNull(mainClass, "Fluent Job API JAR file should have a main class");
+        Preconditions.checkState(WorkflowFactory.class.isAssignableFrom(mainClass),
+                "Fluent Job API JAR main class should be an " + WorkflowFactory.class.getName());
+
+        @SuppressWarnings("unchecked")
+        final Method mainMethod = mainClass.getMethod("create");
+        Preconditions.checkState(Workflow.class.isAssignableFrom(mainMethod.getReturnType()),
+                "Fluent Job API JAR file's main class's create() method should return a " + Workflow.class.getName());
+
+        return (Workflow) mainMethod.invoke(mainClass.newInstance());
+    }
+
+    private String getMainClassName() throws IOException {
+        try (final JarFile apiJar = new JarFile(apiJarFile)) {
+            Preconditions.checkNotNull(apiJar.getManifest(), "Fluent Job API JAR doesn't have MANIFEST.MF");
+
+            return apiJar.getManifest().getMainAttributes().getValue(Attributes.Name.MAIN_CLASS);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/client/src/main/java/org/apache/oozie/client/OozieClient.java
----------------------------------------------------------------------
diff --git a/client/src/main/java/org/apache/oozie/client/OozieClient.java b/client/src/main/java/org/apache/oozie/client/OozieClient.java
index 4c81756..2cc1692 100644
--- a/client/src/main/java/org/apache/oozie/client/OozieClient.java
+++ b/client/src/main/java/org/apache/oozie/client/OozieClient.java
@@ -20,6 +20,9 @@ package org.apache.oozie.client;
 
 import com.fasterxml.jackson.core.type.TypeReference;
 import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.base.Strings;
+import com.google.common.collect.Lists;
+import org.apache.commons.io.IOUtils;
 import com.google.common.collect.Lists;
 import org.apache.oozie.BuildInfo;
 import org.apache.oozie.cli.ValidationUtil;
@@ -195,6 +198,8 @@ public class OozieClient {
 
     public static final String FILTER_SORT_BY = "sortby";
 
+    public static final String CONFIG_KEY_GENERATED_XML = "oozie.jobs.api.generated.xml";
+
     public enum SORT_BY {
         createdTime("createdTimestamp"), lastModifiedTime("lastModifiedTimestamp");
         private final String fullname;
@@ -424,11 +429,7 @@ public class OozieClient {
      */
     public Properties createConfiguration() {
         Properties conf = new Properties();
-        String userName = USER_NAME_TL.get();
-        if (userName == null) {
-            userName = System.getProperty("user.name");
-        }
-        conf.setProperty(USER_NAME, userName);
+        conf.setProperty(USER_NAME, getUserName());
         return conf;
     }
 
@@ -690,36 +691,54 @@ public class OozieClient {
 
     private class JobSubmit extends ClientCallable<String> {
         private final Properties conf;
-
-        JobSubmit(Properties conf, boolean start) {
-            super("POST", RestConstants.JOBS, "", (start) ? prepareParams(RestConstants.ACTION_PARAM,
-                    RestConstants.JOB_ACTION_START) : prepareParams());
+        private final String generatedXml;
+
+        JobSubmit(final Properties conf, final boolean start, final String generatedXml) {
+            super("POST", RestConstants.JOBS, "",
+                    (start)
+                            ? prepareParams(RestConstants.ACTION_PARAM,
+                                    RestConstants.JOB_ACTION_START,
+                                    RestConstants.USER_PARAM,
+                                    getUserName())
+                            : prepareParams(RestConstants.USER_PARAM,
+                                    getUserName()));
             this.conf = notNull(conf, "conf");
+            this.generatedXml = generatedXml;
         }
 
         JobSubmit(String jobId, Properties conf) {
             super("PUT", RestConstants.JOB, notEmpty(jobId, "jobId"), prepareParams(RestConstants.ACTION_PARAM,
-                    RestConstants.JOB_ACTION_RERUN));
+                    RestConstants.JOB_ACTION_RERUN, RestConstants.USER_PARAM, getUserName()));
             this.conf = notNull(conf, "conf");
+            this.generatedXml = null;
         }
 
         public JobSubmit(Properties conf, String jobActionDryrun) {
             super("POST", RestConstants.JOBS, "", prepareParams(RestConstants.ACTION_PARAM,
-                    RestConstants.JOB_ACTION_DRYRUN));
+                    RestConstants.JOB_ACTION_DRYRUN, RestConstants.USER_PARAM, getUserName()));
             this.conf = notNull(conf, "conf");
+            this.generatedXml = null;
         }
 
         @Override
         protected String call(HttpURLConnection conn) throws IOException, OozieClientException {
             conn.setRequestProperty("content-type", RestConstants.XML_CONTENT_TYPE);
+
+            if (!Strings.isNullOrEmpty(generatedXml)) {
+                conf.setProperty(CONFIG_KEY_GENERATED_XML, generatedXml);
+            }
+
             writeToXml(conf, conn.getOutputStream());
+
             if (conn.getResponseCode() == HttpURLConnection.HTTP_CREATED) {
                 JSONObject json = (JSONObject) JSONValue.parse(new InputStreamReader(conn.getInputStream()));
                 return (String) json.get(JsonTags.JOB_ID);
             }
+
             if (conn.getResponseCode() != HttpURLConnection.HTTP_OK) {
                 handleError(conn);
             }
+
             return null;
         }
     }
@@ -732,7 +751,11 @@ public class OozieClient {
      * @throws OozieClientException thrown if the job could not be submitted.
      */
     public String submit(Properties conf) throws OozieClientException {
-        return (new JobSubmit(conf, false)).call();
+        return (new JobSubmit(conf, false, null)).call();
+    }
+
+    public String submit(final Properties conf, final String generatedXml) throws OozieClientException {
+        return (new JobSubmit(conf, false, generatedXml)).call();
     }
 
     private class JobAction extends ClientCallable<Void> {
@@ -875,7 +898,11 @@ public class OozieClient {
      * @throws OozieClientException thrown if the job could not be submitted.
      */
     public String run(Properties conf) throws OozieClientException {
-        return (new JobSubmit(conf, true)).call();
+        return (new JobSubmit(conf, true, null)).call();
+    }
+
+    public String run(final Properties conf, final String generatedXml) throws OozieClientException {
+        return (new JobSubmit(conf, true, generatedXml)).call();
     }
 
     /**
@@ -2188,7 +2215,7 @@ public class OozieClient {
         String file = null;
 
         ValidateXML(String file, String user) {
-            super("POST", RestConstants.VALIDATE, "",
+            super("POST", WS_PROTOCOL_VERSION, RestConstants.VALIDATE, "",
                     prepareParams(RestConstants.FILE_PARAM, file, RestConstants.USER_PARAM, user));
             this.file = file;
         }
@@ -2219,6 +2246,21 @@ public class OozieClient {
         }
     }
 
+    private class SubmitXML extends ClientCallable<String> {
+        SubmitXML(final String xml, final String user) {
+            super("POST",
+                    WS_PROTOCOL_VERSION,
+                    RestConstants.ACTION_PARAM,
+                    RestConstants.JOB_ACTION_SUBMIT,
+                    prepareParams());
+        }
+
+        @Override
+        protected String call(final HttpURLConnection conn) throws IOException, OozieClientException {
+            conn.setRequestProperty("content-type", RestConstants.XML_CONTENT_TYPE);
+            return null;
+        }
+    }
 
     private  class UpdateSharelib extends ClientCallable<String> {
 
@@ -2405,11 +2447,15 @@ public class OozieClient {
             }
             fileName = f.getAbsolutePath();
         }
-        String user = USER_NAME_TL.get();
-        if (user == null) {
-            user = System.getProperty("user.name");
+        return new ValidateXML(fileName, getUserName()).call();
+    }
+
+    private String getUserName() {
+        String userName = USER_NAME_TL.get();
+        if (userName == null) {
+            userName = System.getProperty("user.name");
         }
-        return new ValidateXML(fileName, user).call();
+        return userName;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/core/pom.xml
----------------------------------------------------------------------
diff --git a/core/pom.xml b/core/pom.xml
index 45880a6..a5a776c 100644
--- a/core/pom.xml
+++ b/core/pom.xml
@@ -404,6 +404,17 @@
         </dependency>
 
         <dependency>
+            <groupId>org.apache.oozie</groupId>
+            <artifactId>oozie-fluent-job-api</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.oozie</groupId>
+            <artifactId>oozie-fluent-job-api</artifactId>
+            <classifier>tests</classifier>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
             <groupId>org.mockito</groupId>
             <artifactId>mockito-all</artifactId>
             <scope>test</scope>

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/core/src/main/java/org/apache/oozie/BaseLocalOozieClient.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/oozie/BaseLocalOozieClient.java b/core/src/main/java/org/apache/oozie/BaseLocalOozieClient.java
index 38fb006..376ea11 100644
--- a/core/src/main/java/org/apache/oozie/BaseLocalOozieClient.java
+++ b/core/src/main/java/org/apache/oozie/BaseLocalOozieClient.java
@@ -29,6 +29,7 @@ import org.apache.oozie.client.OozieClientException;
 import org.apache.oozie.client.WorkflowAction;
 import org.apache.oozie.client.WorkflowJob;
 import org.apache.oozie.client.rest.RestConstants;
+import org.apache.oozie.servlet.V2ValidateServlet;
 import org.apache.oozie.util.XConfiguration;
 import org.json.simple.JSONObject;
 
@@ -555,8 +556,16 @@ abstract class BaseLocalOozieClient extends OozieClient {
     }
 
     @Override
-    public String validateXML(String file) throws OozieClientException {
-        return throwNoOp();
+    public String validateXML(final String xmlContent) throws OozieClientException {
+        final V2ValidateServlet validateServlet = new V2ValidateServlet();
+
+        try {
+            validateServlet.validate(xmlContent);
+            return V2ValidateServlet.VALID_WORKFLOW_APP;
+        }
+        catch (final Exception e) {
+            throw new OozieClientException("Cannot validate XML.", e);
+        }
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/core/src/main/java/org/apache/oozie/local/LocalOozie.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/oozie/local/LocalOozie.java b/core/src/main/java/org/apache/oozie/local/LocalOozie.java
index 9ab646c..6475f33 100644
--- a/core/src/main/java/org/apache/oozie/local/LocalOozie.java
+++ b/core/src/main/java/org/apache/oozie/local/LocalOozie.java
@@ -87,6 +87,10 @@ public class LocalOozie {
         XLog.getLog(LocalOozie.class).info("LocalOozie started callback set to [{0}]", callbackUrl);
     }
 
+    public static boolean isStarted() {
+        return localOozieActive;
+    }
+
     /**
      * Stop LocalOozie.
      */

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/core/src/main/java/org/apache/oozie/servlet/BaseJobServlet.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/oozie/servlet/BaseJobServlet.java b/core/src/main/java/org/apache/oozie/servlet/BaseJobServlet.java
index e1bd3cf..3145650 100644
--- a/core/src/main/java/org/apache/oozie/servlet/BaseJobServlet.java
+++ b/core/src/main/java/org/apache/oozie/servlet/BaseJobServlet.java
@@ -257,7 +257,8 @@ public abstract class BaseJobServlet extends JsonRestServlet {
                     throw new XServletException(HttpServletResponse.SC_BAD_REQUEST, ErrorCode.E0405);
                 }
             }
-            ServletUtilities.ValidateAppPath(wfPath, coordPath, bundlePath);
+
+            ServletUtilities.validateAppPath(wfPath, coordPath, bundlePath);
 
             if (wfPath != null) {
                 auth.authorizeForApp(user, acl, wfPath, "workflow.xml", conf);

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/core/src/main/java/org/apache/oozie/servlet/BaseJobsServlet.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/oozie/servlet/BaseJobsServlet.java b/core/src/main/java/org/apache/oozie/servlet/BaseJobsServlet.java
index d4b0871..8df1443 100644
--- a/core/src/main/java/org/apache/oozie/servlet/BaseJobsServlet.java
+++ b/core/src/main/java/org/apache/oozie/servlet/BaseJobsServlet.java
@@ -18,26 +18,33 @@
 
 package org.apache.oozie.servlet;
 
+import java.io.File;
 import java.io.IOException;
 import java.util.Arrays;
+import java.util.Date;
 
 import javax.servlet.ServletException;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 
+import com.google.common.base.Strings;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.oozie.ErrorCode;
 import org.apache.oozie.client.OozieClient;
+import org.apache.oozie.client.XOozieClient;
 import org.apache.oozie.client.rest.RestConstants;
+import org.apache.oozie.service.ConfigurationService;
 import org.apache.oozie.service.Services;
 import org.apache.oozie.service.AuthorizationException;
 import org.apache.oozie.service.AuthorizationService;
 import org.apache.oozie.util.JobUtils;
 import org.apache.oozie.util.JobsFilterUtils;
 import org.apache.oozie.util.XConfiguration;
+import org.apache.oozie.util.XLog;
 import org.json.simple.JSONObject;
 
 public abstract class BaseJobsServlet extends JsonRestServlet {
+    private static final XLog LOG = XLog.getLog(BaseJobsServlet.class);
 
     private static final JsonRestServlet.ResourceInfo RESOURCES_INFO[] = new JsonRestServlet.ResourceInfo[1];
 
@@ -96,7 +103,15 @@ public abstract class BaseJobsServlet extends JsonRestServlet {
         if (!requestUser.equals(UNDEF)) {
             conf.set(OozieClient.USER_NAME, requestUser);
         }
+
+        final String fsUser = request.getParameter(RestConstants.USER_PARAM) == null
+                ? conf.get(OozieClient.USER_NAME)
+                : request.getParameter(RestConstants.USER_PARAM);
+
+        checkAndWriteApplicationXMLToHDFS(fsUser, ensureJobApplicationPath(conf));
+
         BaseJobServlet.checkAuthorizationForApp(conf);
+
         JobUtils.normalizeAppPath(conf.get(OozieClient.USER_NAME), conf.get(OozieClient.GROUP_NAME), conf);
 
         JSONObject json = submitJob(request, conf);
@@ -104,6 +119,34 @@ public abstract class BaseJobsServlet extends JsonRestServlet {
         sendJsonResponse(response, HttpServletResponse.SC_CREATED, json);
     }
 
+    private XConfiguration ensureJobApplicationPath(final XConfiguration configuration) {
+        if (!Strings.isNullOrEmpty(configuration.get(XOozieClient.IS_PROXY_SUBMISSION))
+                && Boolean.valueOf(configuration.get(XOozieClient.IS_PROXY_SUBMISSION))) {
+            LOG.debug("Proxy submission in progress, no need to set application path.");
+            return configuration;
+        }
+
+        if (Strings.isNullOrEmpty(configuration.get(OozieClient.APP_PATH))
+                && Strings.isNullOrEmpty(configuration.get(OozieClient.LIBPATH))
+                && Strings.isNullOrEmpty(configuration.get(OozieClient.COORDINATOR_APP_PATH))
+                && Strings.isNullOrEmpty(configuration.get(OozieClient.BUNDLE_APP_PATH))) {
+            final String generatedJobApplicationPath = ConfigurationService.get("oozie.fluent-job-api.generated.path")
+                    + File.separator + "gen_app_" + new Date().getTime();
+            LOG.debug("Parameters [{0}], [{1}], [{2}], and [{3}] were all missing, setting to generated path [{4}]",
+                    OozieClient.APP_PATH,
+                    OozieClient.LIBPATH,
+                    OozieClient.COORDINATOR_APP_PATH,
+                    OozieClient.BUNDLE_APP_PATH,
+                    generatedJobApplicationPath);
+            configuration.set(OozieClient.APP_PATH, generatedJobApplicationPath);
+        }
+
+        return configuration;
+    }
+
+    protected abstract void checkAndWriteApplicationXMLToHDFS(final String requestUser, final Configuration conf)
+            throws XServletException;
+
     /**
      * Return information about jobs.
      */

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/core/src/main/java/org/apache/oozie/servlet/ServletUtilities.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/oozie/servlet/ServletUtilities.java b/core/src/main/java/org/apache/oozie/servlet/ServletUtilities.java
index 3eb32d5..e6c2670 100644
--- a/core/src/main/java/org/apache/oozie/servlet/ServletUtilities.java
+++ b/core/src/main/java/org/apache/oozie/servlet/ServletUtilities.java
@@ -32,7 +32,7 @@ public class ServletUtilities {
      * @param coordPath coordinator app path
      * @throws XServletException if either path is not valid
      */
-    protected static void ValidateAppPath(String wfPath, String coordPath) throws XServletException {
+    protected static void validateAppPath(String wfPath, String coordPath) throws XServletException {
         if (wfPath != null && coordPath != null) {
             throw new XServletException(HttpServletResponse.SC_BAD_REQUEST, ErrorCode.E0302,
                     "multiple app paths specified, only one is allowed");
@@ -55,7 +55,7 @@ public class ServletUtilities {
      * @param bundlePath bundle app path
      * @throws XServletException if either path is not valid
      */
-    protected static void ValidateAppPath(String wfPath, String coordPath, String bundlePath) throws XServletException {
+    protected static void validateAppPath(String wfPath, String coordPath, String bundlePath) throws XServletException {
         int n = 0;
 
         if (wfPath != null) {

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/core/src/main/java/org/apache/oozie/servlet/V0JobsServlet.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/oozie/servlet/V0JobsServlet.java b/core/src/main/java/org/apache/oozie/servlet/V0JobsServlet.java
index 1d80094..0ff9c6a 100644
--- a/core/src/main/java/org/apache/oozie/servlet/V0JobsServlet.java
+++ b/core/src/main/java/org/apache/oozie/servlet/V0JobsServlet.java
@@ -119,6 +119,11 @@ public class V0JobsServlet extends BaseJobsServlet {
     }
 
 
+    @Override
+    protected void checkAndWriteApplicationXMLToHDFS(String requestUser, Configuration conf) throws XServletException {
+        // NOP
+    }
+
     /**
      * service implementation to bulk kill jobs
      * @param request

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/core/src/main/java/org/apache/oozie/servlet/V1JobsServlet.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/oozie/servlet/V1JobsServlet.java b/core/src/main/java/org/apache/oozie/servlet/V1JobsServlet.java
index c1ca65f..5cfcf57 100644
--- a/core/src/main/java/org/apache/oozie/servlet/V1JobsServlet.java
+++ b/core/src/main/java/org/apache/oozie/servlet/V1JobsServlet.java
@@ -18,7 +18,13 @@
 
 package org.apache.oozie.servlet;
 
+import java.io.File;
 import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.io.StringReader;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
 import java.util.HashSet;
 import java.util.List;
@@ -27,7 +33,12 @@ import java.util.Set;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 
+import com.google.common.base.Strings;
+import com.google.common.collect.Lists;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.oozie.BaseEngineException;
 import org.apache.oozie.BulkResponseInfo;
 import org.apache.oozie.BundleEngine;
@@ -43,19 +54,24 @@ import org.apache.oozie.OozieJsonFactory;
 import org.apache.oozie.WorkflowsInfo;
 import org.apache.oozie.cli.OozieCLI;
 import org.apache.oozie.client.OozieClient;
+import org.apache.oozie.client.XOozieClient;
 import org.apache.oozie.client.rest.BulkResponseImpl;
 import org.apache.oozie.client.rest.JsonTags;
 import org.apache.oozie.client.rest.RestConstants;
 import org.apache.oozie.service.BundleEngineService;
 import org.apache.oozie.service.CoordinatorEngineService;
 import org.apache.oozie.service.DagEngineService;
+import org.apache.oozie.service.HadoopAccessorException;
+import org.apache.oozie.service.HadoopAccessorService;
 import org.apache.oozie.service.Services;
+import org.apache.oozie.util.IOUtils;
 import org.apache.oozie.util.XLog;
 import org.apache.oozie.util.XmlUtils;
 import org.json.simple.JSONArray;
 import org.json.simple.JSONObject;
 
 public class V1JobsServlet extends BaseJobsServlet {
+    private static final XLog LOG = XLog.getLog(V1JobsServlet.class);
 
     private static final String INSTRUMENTATION_NAME = "v1jobs";
     private static final Set<String> httpJobType = new HashSet<String>(){{
@@ -84,7 +100,7 @@ public class V1JobsServlet extends BaseJobsServlet {
             String coordPath = conf.get(OozieClient.COORDINATOR_APP_PATH);
             String bundlePath = conf.get(OozieClient.BUNDLE_APP_PATH);
 
-            ServletUtilities.ValidateAppPath(wfPath, coordPath, bundlePath);
+            ServletUtilities.validateAppPath(wfPath, coordPath, bundlePath);
 
             if (wfPath != null) {
                 json = submitWorkflowJob(request, conf);
@@ -108,6 +124,129 @@ public class V1JobsServlet extends BaseJobsServlet {
         return json;
     }
 
+    @Override
+    protected void checkAndWriteApplicationXMLToHDFS(final String userName, final Configuration conf) throws XServletException {
+        if (!Strings.isNullOrEmpty(conf.get(XOozieClient.IS_PROXY_SUBMISSION))
+                && Boolean.valueOf(conf.get(XOozieClient.IS_PROXY_SUBMISSION))) {
+            LOG.debug("Proxy submission in progress, no need to write application XML.");
+            return;
+        }
+
+        final List<String> appPathsWithFileNames;
+        if (!findAppPathsWithFileNames(conf.get(OozieClient.APP_PATH), "workflow.xml").isEmpty()) {
+            appPathsWithFileNames = findAppPathsWithFileNames(conf.get(OozieClient.APP_PATH), "workflow.xml");
+        }
+        else if (!findAppPathsWithFileNames(conf.get(OozieClient.LIBPATH), "workflow.xml").isEmpty()) {
+            appPathsWithFileNames = findAppPathsWithFileNames(conf.get(OozieClient.LIBPATH), "workflow.xml");
+        }
+        else if (!findAppPathsWithFileNames(conf.get(OozieClient.COORDINATOR_APP_PATH), "coordinator.xml").isEmpty()) {
+            appPathsWithFileNames = findAppPathsWithFileNames(conf.get(OozieClient.COORDINATOR_APP_PATH), "coordinator.xml");
+        }
+        else {
+            appPathsWithFileNames = findAppPathsWithFileNames(conf.get(OozieClient.BUNDLE_APP_PATH), "bundle.xml");
+        }
+
+        LOG.debug("Checking whether XML exists on HDFS. [appPathsWithFileNames={0}]", appPathsWithFileNames);
+
+        for (final String appPathWithFileName : appPathsWithFileNames) {
+            if (existsOnDFS(userName, appPathWithFileName)) {
+                return;
+            }
+        }
+
+        for (final String appPathWithFileName : appPathsWithFileNames) {
+            final String sourceContent = conf.get(OozieClient.CONFIG_KEY_GENERATED_XML);
+            if (sourceContent == null) {
+                throw new XServletException(HttpServletResponse.SC_BAD_REQUEST, ErrorCode.E0307,
+                        String.format("Configuration entry %s not present", OozieClient.CONFIG_KEY_GENERATED_XML));
+            }
+
+            if (tryCreateOnDFS(userName, appPathWithFileName, sourceContent)) {
+                return;
+            }
+        }
+
+        throw new XServletException(HttpServletResponse.SC_BAD_REQUEST, ErrorCode.E0307,
+                String.format("Could not create on HDFS any of the missing application XMLs [%s]",
+                        appPathsWithFileNames));
+    }
+
+    private List<String> findAppPathsWithFileNames(final String appPaths, final String defaultFileName) {
+        final List<String> appPathsWithFileNames = Lists.newArrayList();
+
+        if (Strings.isNullOrEmpty(appPaths)) {
+            return appPathsWithFileNames;
+        }
+
+        for (final String appPath : appPaths.split(",")) {
+            if (appPath.endsWith(".xml")) {
+                appPathsWithFileNames.add(appPath);
+            }
+            else {
+                appPathsWithFileNames.add(appPath + File.separator + defaultFileName);
+            }
+        }
+
+        return appPathsWithFileNames;
+    }
+
+    private boolean existsOnDFS(final String userName, final String appPathWithFileName) throws XServletException {
+        try {
+            final URI uri = new URI(appPathWithFileName);
+            final HadoopAccessorService has = Services.get().get(HadoopAccessorService.class);
+            final Configuration fsConf = has.createConfiguration(uri.getAuthority());
+            final FileSystem dfs = has.createFileSystem(userName, uri, fsConf);
+
+            final Path path = new Path(uri.getPath());
+
+            if (dfs.exists(path)) {
+                if (!dfs.isFile(path)) {
+                    final String errorMessage = String.format("HDFS path [%s] exists but is not a file.", path.toString());
+                    LOG.error(errorMessage);
+                    throw new XServletException(HttpServletResponse.SC_BAD_REQUEST, ErrorCode.E0307, errorMessage);
+                }
+
+                LOG.debug("HDFS path [{0}] is an existing file, no need to create.", path.toString());
+                return true;
+            }
+
+            LOG.debug("HDFS path [{0}] is not an existing file.", path.toString());
+            return false;
+        }
+        catch (final URISyntaxException | IOException | HadoopAccessorException e) {
+            throw new XServletException(HttpServletResponse.SC_BAD_REQUEST, ErrorCode.E0307,
+                    String.format("Could not check whether file [%s] exists on HDFS. Error message: %s",
+                            appPathWithFileName,  e.getMessage()));
+        }
+    }
+
+    private boolean tryCreateOnDFS(final String userName, final String appPathWithFileName, final String sourceContent) {
+        try {
+            final URI uri = new URI(appPathWithFileName);
+            final HadoopAccessorService has = Services.get().get(HadoopAccessorService.class);
+            final Configuration fsConf = has.createConfiguration(uri.getAuthority());
+            final FileSystem dfs = has.createFileSystem(userName, uri, fsConf);
+
+            final Path path = new Path(uri.getPath());
+
+            LOG.debug("HDFS path [{0}] does not exist, will try to create.", path.toString());
+
+            try (final FSDataOutputStream target = dfs.create(path)) {
+                LOG.debug("HDFS path [{0}] created.", path.toString());
+
+                IOUtils.copyCharStream(new StringReader(sourceContent), new OutputStreamWriter(target, StandardCharsets.UTF_8));
+            }
+
+            LOG.debug("XML written to HDFS file [{0}].", path.toString());
+
+            return true;
+        }
+        catch (final URISyntaxException | IOException | HadoopAccessorException e) {
+            LOG.warn("Could not write XML [%s] to HDFS. Error message: %s", appPathWithFileName, e.getMessage());
+            return false;
+        }
+    }
+
     /**
      * v1 service implementation to get a JSONObject representation of a job from its external ID
      */

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/core/src/main/java/org/apache/oozie/servlet/V2ValidateServlet.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/oozie/servlet/V2ValidateServlet.java b/core/src/main/java/org/apache/oozie/servlet/V2ValidateServlet.java
index b86fa6a..36a9de2 100644
--- a/core/src/main/java/org/apache/oozie/servlet/V2ValidateServlet.java
+++ b/core/src/main/java/org/apache/oozie/servlet/V2ValidateServlet.java
@@ -53,6 +53,7 @@ public class V2ValidateServlet extends JsonRestServlet {
             new ResourceInfo("", Arrays.asList("POST"), Arrays.asList(
                     new ParameterInfo(RestConstants.FILE_PARAM, String.class, true, Arrays.asList("POST")),
                     new ParameterInfo(RestConstants.USER_PARAM, String.class, true, Arrays.asList("POST"))));
+    public static final String VALID_WORKFLOW_APP = "Valid workflow-app";
 
 
     public V2ValidateServlet() {
@@ -98,12 +99,12 @@ public class V2ValidateServlet extends JsonRestServlet {
                     file + ", " + e.toString());
         }
 
-        JSONObject json = createJSON("Valid workflow-app");
+        JSONObject json = createJSON(VALID_WORKFLOW_APP);
         startCron();
         sendJsonResponse(response, HttpServletResponse.SC_OK, json);
     }
 
-    private void validate(String xml) throws Exception{
+    public void validate(String xml) throws Exception {
         SchemaService schemaService = Services.get().get(SchemaService.class);
         Schema[] schemas = {schemaService.getSchema(SchemaService.SchemaName.WORKFLOW),
                 schemaService.getSchema(SchemaService.SchemaName.COORDINATOR),

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/core/src/main/resources/oozie-default.xml
----------------------------------------------------------------------
diff --git a/core/src/main/resources/oozie-default.xml b/core/src/main/resources/oozie-default.xml
index 8d7465c..ff1820c 100644
--- a/core/src/main/resources/oozie-default.xml
+++ b/core/src/main/resources/oozie-default.xml
@@ -3377,5 +3377,16 @@ will be the requeue interval for the actions which are waiting for a long time w
         </description>
     </property>
 
+    <property>
+        <name>oozie.fluent-job-api.generated.path</name>
+        <value>/user/${user.name}/oozie-fluent-job-api-generated</value>
+        <description>
+            HDFS path to store workflow / coordinator / bundle definitions generated by fluent-job-api artifact.
+            The XML files are first generated out of the fluent-job-api JARs submitted by the user at command line, then stored
+            under this HDFS folder structure for later retrieval / resubmit / check.
+            Note that the submitting user needs r/w permissions under this HDFS folder.
+            Note further that this folder structure, when does not exist, will be created.
+        </description>
+    </property>
 
 </configuration>

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/core/src/test/java/org/apache/oozie/client/TestOozieCLI.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/oozie/client/TestOozieCLI.java b/core/src/test/java/org/apache/oozie/client/TestOozieCLI.java
index 3395cc2..56639b0 100644
--- a/core/src/test/java/org/apache/oozie/client/TestOozieCLI.java
+++ b/core/src/test/java/org/apache/oozie/client/TestOozieCLI.java
@@ -23,18 +23,30 @@ import java.io.DataOutputStream;
 import java.io.File;
 import java.io.FileOutputStream;
 import java.io.FileWriter;
+import java.io.IOException;
+import java.io.InputStream;
 import java.io.OutputStream;
 import java.io.PrintStream;
 import java.io.StringReader;
+import java.util.Enumeration;
 import java.util.Properties;
 import java.util.concurrent.Callable;
-
+import java.util.jar.JarEntry;
+import java.util.jar.JarFile;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
+import com.google.common.io.Files;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.io.filefilter.FileFilterUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.oozie.BuildInfo;
 import org.apache.oozie.cli.CLIParser;
 import org.apache.oozie.cli.OozieCLI;
 import org.apache.oozie.client.rest.RestConstants;
+import org.apache.oozie.fluentjob.api.factory.SimpleWorkflowFactory;
+import org.apache.oozie.fluentjob.api.factory.WorkflowFactory;
 import org.apache.oozie.service.InstrumentationService;
 import org.apache.oozie.service.MetricsInstrumentationService;
 import org.apache.oozie.service.Services;
@@ -51,7 +63,6 @@ import org.apache.oozie.servlet.V2JobServlet;
 import org.apache.oozie.servlet.V2ValidateServlet;
 import org.apache.oozie.util.IOUtils;
 import org.apache.oozie.util.XConfiguration;
-import org.json.simple.JSONValue;
 
 //hardcoding options instead using constants on purpose, to detect changes to option names if any and correct docs.
 public class TestOozieCLI extends DagServletTestCase {
@@ -84,7 +95,9 @@ public class TestOozieCLI extends DagServletTestCase {
     private String createConfigFile(String appPath) throws Exception {
         String path = getTestCaseDir() + "/" + getName() + ".xml";
         Configuration conf = new Configuration(false);
-        conf.set(OozieClient.APP_PATH, appPath);
+        if (!Strings.isNullOrEmpty(appPath)) {
+            conf.set(OozieClient.APP_PATH, appPath);
+        }
         conf.set(OozieClient.RERUN_SKIP_NODES, "node");
 
         OutputStream os = new FileOutputStream(path);
@@ -1711,6 +1724,178 @@ public class TestOozieCLI extends DagServletTestCase {
         });
     }
 
+    public void testValidateJar() throws Exception {
+        final JarFile workflowApiJar = createWorkflowApiJar();
+
+        runTest(END_POINTS, SERVLET_CLASSES, IS_SECURITY_ENABLED, new Callable<Void>() {
+            @Override
+            public Void call() throws Exception {
+                String oozieUrl = getContextURL();
+
+                String[] args = new String[]{"job",
+                        "-validatejar", workflowApiJar.getName(),
+                        "-oozie", oozieUrl,
+                        "-verbose"};
+                assertEquals(0, new OozieCLI().run(args));
+
+                return null;
+            }
+        });
+    }
+
+    public void testSubmitJar() throws Exception {
+        final JarFile workflowApiJar = createWorkflowApiJar();
+
+        runTest(END_POINTS, SERVLET_CLASSES, IS_SECURITY_ENABLED, new Callable<Void>() {
+            @Override
+            public Void call() throws Exception {
+                final int wfCount = MockDagEngineService.INIT_WF_COUNT;
+
+                final String oozieUrl = getContextURL();
+                final Path appPath = new Path(getFsTestCaseDir(), "app");
+                getFileSystem().mkdirs(appPath);
+
+                final String[] args = new String[]{"job",
+                        "-submitjar", workflowApiJar.getName(),
+                        "-oozie", oozieUrl,
+                        "-config", createConfigFile(appPath.toString()),
+                        "-verbose"};
+                assertEquals(0, new OozieCLI().run(args));
+                assertEquals("submit", MockDagEngineService.did);
+                assertFalse(MockDagEngineService.started.get(wfCount));
+
+                return null;
+            }
+        });
+    }
+
+    public void testSubmitJarWithoutAppPath() throws Exception {
+        final JarFile workflowApiJar = createWorkflowApiJar();
+
+        runTest(END_POINTS, SERVLET_CLASSES, IS_SECURITY_ENABLED, new Callable<Void>() {
+            @Override
+            public Void call() throws Exception {
+                final int wfCount = MockDagEngineService.INIT_WF_COUNT;
+
+                final String oozieUrl = getContextURL();
+                final Path appPath = new Path(getFsTestCaseDir(), "app");
+                getFileSystem().mkdirs(appPath);
+
+                final String[] args = new String[]{"job",
+                        "-submitjar", workflowApiJar.getName(),
+                        "-oozie", oozieUrl,
+                        "-config", createConfigFile(null),
+                        "-verbose"};
+                assertEquals(0, new OozieCLI().run(args));
+                assertEquals("submit", MockDagEngineService.did);
+                assertFalse(MockDagEngineService.started.get(wfCount));
+
+                return null;
+            }
+        });
+    }
+
+    public void testRunJar() throws Exception {
+        final JarFile workflowApiJar = createWorkflowApiJar();
+
+        runTest(END_POINTS, SERVLET_CLASSES, IS_SECURITY_ENABLED, new Callable<Void>() {
+            @Override
+            public Void call() throws Exception {
+                final int wfCount = MockDagEngineService.INIT_WF_COUNT;
+
+                final String oozieUrl = getContextURL();
+                final Path appPath = new Path(getFsTestCaseDir(), "app");
+                getFileSystem().mkdirs(appPath);
+
+                final String[] args = new String[]{"job",
+                        "-runjar", workflowApiJar.getName(),
+                        "-oozie", oozieUrl,
+                        "-config", createConfigFile(appPath.toString()),
+                        "-verbose"};
+                assertEquals(0, new OozieCLI().run(args));
+                assertEquals("submit", MockDagEngineService.did);
+                assertTrue(MockDagEngineService.started.get(wfCount));
+
+                return null;
+            }
+        });
+    }
+
+    public void testRunJarWithoutAppPath() throws Exception {
+        final JarFile workflowApiJar = createWorkflowApiJar();
+
+        runTest(END_POINTS, SERVLET_CLASSES, IS_SECURITY_ENABLED, new Callable<Void>() {
+            @Override
+            public Void call() throws Exception {
+                final int wfCount = MockDagEngineService.INIT_WF_COUNT;
+
+                final String oozieUrl = getContextURL();
+                final Path appPath = new Path(getFsTestCaseDir(), "app");
+                getFileSystem().mkdirs(appPath);
+
+                final String[] args = new String[]{"job",
+                        "-runjar", workflowApiJar.getName(),
+                        "-oozie", oozieUrl,
+                        "-config", createConfigFile(null),
+                        "-verbose"};
+                assertEquals(0, new OozieCLI().run(args));
+                assertEquals("submit", MockDagEngineService.did);
+                assertTrue(MockDagEngineService.started.get(wfCount));
+
+                return null;
+            }
+        });
+    }
+
+    private JarFile createWorkflowApiJar() throws IOException {
+        final String workflowApiJarName = "workflow-api-jar.jar";
+        final File targetFolder = Files.createTempDir();
+        final File classFolder = new File(targetFolder.getPath() + File.separator + "classes");
+        Preconditions.checkState(classFolder.mkdir(), "could not create classFolder %s", classFolder);
+        final File jarFolder = new File(targetFolder.getPath() + File.separator + "jar");
+        Preconditions.checkState(jarFolder.mkdir(), "could not create jarFolder %s", jarFolder);
+        final Class<? extends WorkflowFactory> workflowFactoryClass = SimpleWorkflowFactory.class;
+        final String workflowFactoryClassName = workflowFactoryClass.getSimpleName() + ".class";
+        final String workflowFactoryClassLocation =
+                workflowFactoryClass.getProtectionDomain().getCodeSource().getLocation().getFile();
+
+        if (workflowFactoryClassLocation.endsWith(".jar")) {
+            // Maven / JAR file based test execution
+            extractClassFromJar(new JarFile(workflowFactoryClassLocation), workflowFactoryClassName, classFolder);
+        }
+        else {
+            // IDE / class file based test execution
+            FileUtils.copyDirectory(
+                    new File(workflowFactoryClassLocation),
+                    classFolder,
+                    FileFilterUtils.or(FileFilterUtils.directoryFileFilter(),
+                            FileFilterUtils.nameFileFilter(workflowFactoryClassName)));
+        }
+
+        return new ApiJarFactory(targetFolder, jarFolder, workflowFactoryClass, workflowApiJarName).create();
+    }
+
+    private void extractClassFromJar(final JarFile jarFile, final String filterClassName, final File targetFolder)
+            throws IOException {
+        final Enumeration<JarEntry> entries = jarFile.entries();
+        while (entries.hasMoreElements()) {
+            final JarEntry entry = entries.nextElement();
+            if (entry.getName().endsWith(File.separator + filterClassName)) {
+                final String relativeClassFolderName = entry.getName().substring(0, entry.getName().lastIndexOf(File.separator));
+                final String classFileName =
+                        entry.getName().substring(entry.getName().lastIndexOf(File.separator) + File.separator.length());
+                final File classFolder = new File(targetFolder + File.separator + relativeClassFolderName);
+                Preconditions.checkState(classFolder.mkdirs(), "could not create classFolder %s", classFolder);
+
+                try (final InputStream entryStream = jarFile.getInputStream(entry);
+                final FileOutputStream filterClassStream = new FileOutputStream(
+                        new File(classFolder + File.separator + classFileName))) {
+                    IOUtils.copyStream(entryStream, filterClassStream);
+                }
+            }
+        }
+    }
+
     private String runOozieCLIAndGetStdout(String[] args) {
         PrintStream original = System.out;
         ByteArrayOutputStream baos = new ByteArrayOutputStream();

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/core/src/test/java/org/apache/oozie/servlet/TestV1JobsServlet.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/oozie/servlet/TestV1JobsServlet.java b/core/src/test/java/org/apache/oozie/servlet/TestV1JobsServlet.java
index fb481e3..589d859 100644
--- a/core/src/test/java/org/apache/oozie/servlet/TestV1JobsServlet.java
+++ b/core/src/test/java/org/apache/oozie/servlet/TestV1JobsServlet.java
@@ -72,7 +72,7 @@ public class TestV1JobsServlet extends DagServletTestCase {
                 jobConf.set(OozieClient.USER_NAME, getTestUser());
                 jobConf.set(OozieClient.APP_PATH, appPath);
 
-                Map<String, String> params = new HashMap<String, String>();
+                Map<String, String> params = new HashMap<>();
                 URL url = createURL("", params);
                 HttpURLConnection conn = (HttpURLConnection) url.openConnection();
                 conn.setRequestMethod("POST");
@@ -90,7 +90,7 @@ public class TestV1JobsServlet extends DagServletTestCase {
                 jobConf.set(OozieClient.USER_NAME, getTestUser());
                 jobConf.set(OozieClient.APP_PATH, appPath);
 
-                params = new HashMap<String, String>();
+                params = new HashMap<>();
                 params.put(RestConstants.ACTION_PARAM, RestConstants.JOB_ACTION_START);
                 url = createURL("", params);
                 conn = (HttpURLConnection) url.openConnection();
@@ -112,7 +112,7 @@ public class TestV1JobsServlet extends DagServletTestCase {
                 jobConf = new XConfiguration();
                 jobConf.set(OozieClient.USER_NAME, getTestUser());
 
-                params = new HashMap<String, String>();
+                params = new HashMap<>();
                 url = createURL("", params);
                 conn = (HttpURLConnection) url.openConnection();
                 conn.setRequestMethod("POST");
@@ -129,7 +129,7 @@ public class TestV1JobsServlet extends DagServletTestCase {
                 jobConf.set(OozieClient.USER_NAME, getTestUser());
                 jobConf.set(OozieClient.LIBPATH, libPath1.toString());
 
-                params = new HashMap<String, String>();
+                params = new HashMap<>();
                 url = createURL("", params);
                 conn = (HttpURLConnection) url.openConnection();
                 conn.setRequestMethod("POST");
@@ -137,7 +137,6 @@ public class TestV1JobsServlet extends DagServletTestCase {
                 conn.setDoOutput(true);
                 jobConf.writeXml(conn.getOutputStream());
                 assertEquals(HttpServletResponse.SC_CREATED, conn.getResponseCode());
-                assertEquals(HttpServletResponse.SC_CREATED, conn.getResponseCode());
                 obj = (JSONObject) JSONValue.parse(new InputStreamReader(conn.getInputStream()));
                 assertEquals(MockDagEngineService.JOB_ID + wfCount + MockDagEngineService.JOB_ID_END,
                              obj.get(JsonTags.JOB_ID));
@@ -150,7 +149,7 @@ public class TestV1JobsServlet extends DagServletTestCase {
                 jobConf.set(OozieClient.USER_NAME, getTestUser());
                 jobConf.set(OozieClient.LIBPATH, libPath1.toString() + "," + libPath2.toString());
 
-                params = new HashMap<String, String>();
+                params = new HashMap<>();
                 url = createURL("", params);
                 conn = (HttpURLConnection) url.openConnection();
                 conn.setRequestMethod("POST");
@@ -158,7 +157,6 @@ public class TestV1JobsServlet extends DagServletTestCase {
                 conn.setDoOutput(true);
                 jobConf.writeXml(conn.getOutputStream());
                 assertEquals(HttpServletResponse.SC_CREATED, conn.getResponseCode());
-                assertEquals(HttpServletResponse.SC_CREATED, conn.getResponseCode());
                 obj = (JSONObject) JSONValue.parse(new InputStreamReader(conn.getInputStream()));
                 assertEquals(MockDagEngineService.JOB_ID + wfCount + MockDagEngineService.JOB_ID_END,
                              obj.get(JsonTags.JOB_ID));

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/core/src/test/java/org/apache/oozie/test/MiniOozieTestCase.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/oozie/test/MiniOozieTestCase.java b/core/src/test/java/org/apache/oozie/test/MiniOozieTestCase.java
index 0a030d6..46698d1 100644
--- a/core/src/test/java/org/apache/oozie/test/MiniOozieTestCase.java
+++ b/core/src/test/java/org/apache/oozie/test/MiniOozieTestCase.java
@@ -61,12 +61,16 @@ public abstract class MiniOozieTestCase extends XFsTestCase {
     protected void setUp() throws Exception {
         System.setProperty("hadoop20", "true");
         super.setUp();
-        LocalOozie.start();
+        if (!LocalOozie.isStarted()) {
+            LocalOozie.start();
+        }
     }
 
     @Override
     protected void tearDown() throws Exception {
-        LocalOozie.stop();
+        if (LocalOozie.isStarted()) {
+            LocalOozie.stop();
+        }
         super.tearDown();
     }
 

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/docs/src/site/twiki/DG_CommandLineTool.twiki
----------------------------------------------------------------------
diff --git a/docs/src/site/twiki/DG_CommandLineTool.twiki b/docs/src/site/twiki/DG_CommandLineTool.twiki
index 9a17459..d1d302a 100644
--- a/docs/src/site/twiki/DG_CommandLineTool.twiki
+++ b/docs/src/site/twiki/DG_CommandLineTool.twiki
@@ -106,6 +106,16 @@ oozie job <OPTIONS>           : job operations
           -slaenable            enables sla alerts for the job and its children
           -slachange            Update sla param for jobs, supported param are should-start, should-end and max-duration
           -retries              Get information of the retry attempts for a given workflow action.
+          -apijarcheck <jar>    based on the supplied Fluent Job API jar, a workflow definition XML is generated and checked whether
+                                it's a valid Oozie workflow. Output is whether the generated workflow is a valid one
+          -apijarsubmit <jar>   based on the supplied Fluent Job API jar, a workflow definition XML is generated and submitted. When
+                                the parameter =oozie.wf.application.path= isn't supplied on the command line, an HDFS location with
+                                the prefix defined in
+                                =oozie-site.xml#oozie.client.jobs.application.generated.path= is used. Output is the workflow ID
+          -apijarrun <jar>      based on the supplied Fluent Job API jar, a workflow definition XML is generated and run. When the
+                                parameter =oozie.wf.application.path= isn't supplied on the command line, an HDFS location with the
+                                prefix defined in
+                                =oozie-site.xml#oozie.client.jobs.application.generated.path= is used. Output is the workflow ID
 
 </verbatim>
 
@@ -1033,6 +1043,135 @@ Pending Dependencies :
 $
 </verbatim>
 
+---+++ Checking a workflow definition generated by a Fluent Job API jar file
+
+Since Oozie 5.1.0.
+
+Generate a workflow definition given the Fluent Job API jar file supplied at command line, and check for its correctness.
+
+*Preconditions:*
+   * the Fluent Job API jar file has to be present and readable by the current user at the local path provided
+   * the folder containing the Fluent Job API jar file provided has to be writable by the current user, since the generated workflow
+   definition is written there
+
+If the =-verbose= option is provided, a lot more debugging output, including the generated workflow definition, is given.
+
+For more information what an Fluent Job API jar file is, how to build it etc.,
+refer to [[DG_FluentJobAPI#AE.A_Appendix_A_API_JAR_format][Fluent Job API - API JAR format]].
+
+*Example:*
+
+<verbatim>
+$ oozie job -oozie http://localhost:11000/oozie -validatejar /tmp/workflow-api-jar.jar
+Valid workflow-app
+</verbatim>
+
+*Example (verbose):*
+
+<verbatim>
+$ oozie job -oozie http://localhost:11000/oozie -validatejar /tmp/workflow-api-jar.jar -verbose
+Checking API jar:/tmp/workflow-api-jar.jar
+Loading API jar /tmp/workflow-api-jar.jar
+Workflow job definition generated from API jar:
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+<workflow:workflow-app xmlns:workflow="uri:oozie:workflow:1.0" ... name="shell-example">
+...
+</workflow:workflow-app>
+.
+API jar is written to /tmp/workflow1876390751841950810.xml
+Servlet response is:
+Valid workflow-app
+API jar is valid.
+</verbatim>
+
+---+++ Submitting a workflow definition generated by a Fluent Job API jar file
+
+Since Oozie 5.1.0.
+
+Generate a workflow definition given the Fluent Job API jar file supplied at command line, write it to a provided or generated HDFS
+location, and submit.
+
+*Preconditions:*
+   * all the parameters that are present in the workflow definition have to be supplied either as command line parameters or via
+   =job.properties= passed along with the =-config= option
+   * the Fluent Job API jar file has to be present and readable by the current user at the local path provided
+   * the folder containing the Fluent Job API jar file provided has to be writable by the current user, since the generated workflow
+   definition is written there
+   * the HDFS folder either given by =-Doozie.wf.application.path= command line parameter, or in its absence contained by
+   =oozie-site.xml#oozie.client.jobs.application.generated.path= has to be writable by the current user
+
+If the =-verbose= option is provided, a lot more debugging output, including the generated workflow definition, is given.
+
+For more information what an Fluent Job API jar file is, how to build it etc., refer to
+[[DG_FluentJobAPI#AE.A_Appendix_A_API_JAR_format][Fluent Job API - API JAR format]].
+
+*Example:*
+
+<verbatim>
+$ oozie job -oozie http://localhost:11000/oozie -submitjar /tmp/workflow-api-jar.jar -config /tmp/job.properties
+job: 0000009-180107110323219-oozie-oozi-W
+</verbatim>
+
+*Example (verbose):*
+
+<verbatim>
+$ oozie job -oozie http://localhost:11000/oozie -submitjar /tmp/workflow-api-jar.jar -config /tmp/job.properties -verbose
+Submitting a job based on API jar: /tmp/workflow-api-jar.jar
+Loading API jar /tmp/workflow-api-jar.jar
+Workflow job definition generated from API jar:
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+<workflow:workflow-app xmlns:workflow="uri:oozie:workflow:1.0" ... name="shell-example">
+...
+</workflow:workflow-app>
+.
+job: 0000010-180107110323219-oozie-oozi-W
+Job based on API jar submitted successfully.
+</verbatim>
+
+---+++ Running a workflow definition generated by a Fluent Job API jar file
+
+Since Oozie 5.1.0.
+
+Generate a workflow definition given the Fluent Job API jar file supplied at command line, write it to a provided or generated HDFS
+location, submit and run.
+
+*Preconditions:*
+   * all the parameters that are present in the workflow definition have to be supplied either as command line parameters or via
+   =job.properties= passed along with the =-config= option
+   * the Fluent Job API jar file has to be present and readable by the current user at the local path provided
+   * the folder containing the Fluent Job API jar file provided has to be writable by the current user, since the generated workflow
+   definition is written there
+   * the HDFS folder either given by =-Doozie.wf.application.path= command line parameter, or in its absence contained by
+   =oozie-site.xml#oozie.client.jobs.application.generated.path= has to be writable by the current user
+
+If the =-verbose= option is provided, a lot more debugging output, including the generated workflow definition, is given.
+
+For more information what an Fluent Job API jar file is, how to build it etc., refer to
+[[DG_FluentJobAPI#AE.A_Appendix_A_API_JAR_format][Fluent Job API - API JAR format]].
+
+*Example:*
+
+<verbatim>
+$ oozie job -oozie http://localhost:11000/oozie -runjar /tmp/workflow-api-jar.jar -config /tmp/job.properties
+job: 0000011-180107110323219-oozie-oozi-W
+</verbatim>
+
+*Example (verbose):*
+
+<verbatim>
+$ oozie job -oozie http://localhost:11000/oozie -runjar /tmp/workflow-api-jar.jar -config /tmp/job.properties -verbose
+Submitting a job based on API jar: /tmp/workflow-api-jar.jar
+Loading API jar /tmp/workflow-api-jar.jar
+Workflow job definition generated from API jar:
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+<workflow:workflow-app xmlns:workflow="uri:oozie:workflow:1.0" ... name="shell-example">
+...
+</workflow:workflow-app>
+.
+job: 0000010-180107110323219-oozie-oozi-W
+Job based on API jar run successfully.
+</verbatim>
+
 ---++ Jobs Operations
 
 ---+++ Checking the Status of multiple Workflow Jobs


[07/16] oozie git commit: OOZIE-2339 [fluent-job] Minimum Viable Fluent Job API (daniel.becker, andras.piros via rkanter, gezapeti, pbacsko)

Posted by an...@apache.org.
http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestActionAttributesBuilder.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestActionAttributesBuilder.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestActionAttributesBuilder.java
new file mode 100644
index 0000000..ad8d294
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestActionAttributesBuilder.java
@@ -0,0 +1,713 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import com.google.common.collect.ImmutableMap;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class TestActionAttributesBuilder {
+    @Rule
+    public final ExpectedException expectedException = ExpectedException.none();
+
+    private static final String NAME_NODE = "${nameNode}";
+    private static final String EXAMPLE_DIR = "/path/to/directory";
+    private static final String CONFIG_CLASS = "AnyConfigClass.class";
+    private static final String[] JOB_XMLS = {"jobXml1.xml", "jobXml2.xml", "jobXml3.xml", "jobXml4.xml"};
+    private static final String[] FILES = {"file1.xml", "file2.xml", "file3.xml", "file4.xml"};
+    private static final String[] ARCHIVES = {"archive1.jar", "archive2.jar", "archive3.jar", "archive4.jar"};
+
+    private static final String MAPRED_JOB_QUEUE_NAME = "mapred.job.queue.name";
+    private static final String DEFAULT = "default";
+
+    private static final ImmutableMap<String, String> CONFIG_EXAMPLE = getConfigExample();
+
+    private static ImmutableMap<String, String> getConfigExample() {
+        final ImmutableMap.Builder<String, String> configExampleBuilder = new ImmutableMap.Builder<>();
+
+        final String[] keys = {"mapred.map.tasks", "mapred.input.dir", "mapred.output.dir"};
+        final String[] values = {"1", "${inputDir}", "${outputDir}"};
+
+        for (int i = 0; i < keys.length; ++i) {
+            configExampleBuilder.put(keys[i], values[i]);
+        }
+
+        return configExampleBuilder.build();
+    }
+
+    private static final Delete[] DELETES = {new Delete("path0", null),
+            new Delete("path1", true),
+            new Delete("path2", false),
+            new Delete("path3", null)
+    };
+
+    private static final Mkdir[] MKDIRS = {new Mkdir("path0"),
+            new Mkdir("path1"),
+            new Mkdir("path2"),
+            new Mkdir("path3")
+    };
+
+    private static final Move[] MOVES = {new Move("source0", "target0"),
+            new Move("source1", "target1"),
+            new Move("source2", "target2"),
+            new Move("source3", "target3")
+    };
+
+    private static final Chmod[] CHMODS = {new ChmodBuilder().withPermissions("711").build(),
+            new ChmodBuilder().withPermissions("511").build(),
+            new ChmodBuilder().withPermissions("551").build(),
+            new ChmodBuilder().withPermissions("755").build()
+    };
+
+    private static final Touchz[] TOUCHZS = {new Touchz("path0"),
+            new Touchz("path1"),
+            new Touchz("path2"),
+            new Touchz("path3")
+    };
+
+    private static final Chgrp[] CHGRPS = {new ChgrpBuilder().withGroup("user0").build(),
+            new ChgrpBuilder().withGroup("user1").build(),
+            new ChgrpBuilder().withGroup("user2").build(),
+            new ChgrpBuilder().withGroup("user3").build()
+    };
+
+    private static final String RESOURCE_MANAGER = "${resourceManager}";
+
+    private static final Launcher LAUNCHER = new LauncherBuilder()
+            .withMemoryMb(1024)
+            .withVCores(2)
+            .withQueue(DEFAULT)
+            .withSharelib(DEFAULT)
+            .withViewAcl(DEFAULT)
+            .withModifyAcl(DEFAULT)
+            .build();
+
+    private ActionAttributesBuilder getBuilderInstance() {
+        return ActionAttributesBuilder.create();
+    }
+
+    @Test
+    public void testFromExistingBuilder() {
+        final ActionAttributesBuilder builder = getBuilderInstance();
+        builder.withNameNode(NAME_NODE);
+
+        final ActionAttributes fromExisting = ActionAttributesBuilder
+                .createFromExisting(builder.build())
+                .build();
+
+        assertEquals(NAME_NODE, fromExisting.getNameNode());
+    }
+
+    @Test
+    public void testNameNodeAdded() {
+        final ActionAttributesBuilder builder = getBuilderInstance();
+        builder.withNameNode(NAME_NODE);
+
+        final ActionAttributes attributes = builder.build();
+        assertEquals(NAME_NODE, attributes.getNameNode());
+    }
+
+    @Test
+    public void testNameNodeAddedTwiceThrows() {
+        final ActionAttributesBuilder builder = getBuilderInstance();
+        builder.withNameNode(NAME_NODE);
+
+        expectedException.expect(IllegalStateException.class);
+        builder.withNameNode("any_string");
+    }
+
+    @Test
+    public void testPrepareAdded() {
+        final ActionAttributesBuilder builder = getBuilderInstance();
+        builder.withPrepare(new PrepareBuilder().withDelete(EXAMPLE_DIR).build());
+
+        final ActionAttributes attributes = builder.build();
+        assertEquals(EXAMPLE_DIR, attributes.getPrepare().getDeletes().get(0).getPath());
+    }
+
+    @Test
+    public void testPrepareAddedTwiceThrows() {
+        final ActionAttributesBuilder builder = getBuilderInstance();
+        builder.withPrepare(new PrepareBuilder().withDelete(EXAMPLE_DIR).build());
+
+        expectedException.expect(IllegalStateException.class);
+        builder.withPrepare(new PrepareBuilder().withDelete("any_directory").build());
+    }
+
+    @Test
+    public void testStreamingAdded() {
+        final Streaming streaming = new StreamingBuilder().withMapper("mapper.sh").withReducer("reducer.sh").build();
+
+        final ActionAttributesBuilder builder = getBuilderInstance();
+        builder.withStreaming(streaming);
+
+        final ActionAttributes attributes = builder.build();
+        assertEquals(streaming, attributes.getStreaming());
+    }
+
+    @Test
+    public void testStreamingAddedTwiceThrows() {
+        final Streaming streaming1= new StreamingBuilder().withMapper("mapper1.sh").withReducer("reducer1.sh").build();
+        final Streaming streaming2 = new StreamingBuilder().withMapper("mapper2.sh").withReducer("reducer2.sh").build();
+
+        final ActionAttributesBuilder builder = getBuilderInstance();
+        builder.withStreaming(streaming1);
+
+        expectedException.expect(IllegalStateException.class);
+        builder.withStreaming(streaming2);
+    }
+
+    @Test
+    public void testPipesAdded() {
+        final Pipes pipes = new PipesBuilder().withMap("map").withReduce("reduce").build();
+
+        final ActionAttributesBuilder builder = getBuilderInstance();
+        builder.withPipes(pipes);
+
+        final ActionAttributes attributes = builder.build();
+        assertEquals(pipes, attributes.getPipes());
+    }
+
+    @Test
+    public void testPipesAddedTwiceThrows() {
+        final Pipes pipes1 = new PipesBuilder().withMap("map1").withReduce("reduce1").build();
+        final Pipes pipes2 = new PipesBuilder().withMap("map2").withReduce("reduce2").build();
+
+        final ActionAttributesBuilder builder = getBuilderInstance();
+        builder.withPipes(pipes1);
+
+        expectedException.expect(IllegalStateException.class);
+        builder.withPipes(pipes2);
+    }
+
+    @Test
+    public void testConfigClassAdded() {
+        final ActionAttributesBuilder builder = getBuilderInstance();
+        builder.withConfigClass(CONFIG_CLASS);
+
+        final ActionAttributes attributes = builder.build();
+        assertEquals(CONFIG_CLASS, attributes.getConfigClass());
+    }
+
+    @Test
+    public void testConfigClassAddedTwiceThrows() {
+        final ActionAttributesBuilder builder = getBuilderInstance();
+        builder.withConfigClass(CONFIG_CLASS);
+
+        expectedException.expect(IllegalStateException.class);
+        builder.withConfigClass("AnyClass");
+    }
+
+    @Test
+    public void testSeveralJobXmlsAdded() {
+        final ActionAttributesBuilder builder = getBuilderInstance();
+
+        for (final String jobXml : JOB_XMLS) {
+            builder.withJobXml(jobXml);
+        }
+
+        final ActionAttributes attributes = builder.build();
+
+        final List<String> jobXmlsList = attributes.getJobXmls();
+        assertEquals(JOB_XMLS.length, jobXmlsList.size());
+
+        for (int i = 0; i < JOB_XMLS.length; ++i) {
+            assertEquals(JOB_XMLS[i], jobXmlsList.get(i));
+        }
+    }
+
+    @Test
+    public void testWithoutJobXmls() {
+        final ActionAttributesBuilder builder = getBuilderInstance();
+
+        for (final String jobXml : JOB_XMLS) {
+            builder.withJobXml(jobXml);
+        }
+
+        builder.withoutJobXml(JOB_XMLS[0]);
+
+        final ActionAttributes attributes = builder.build();
+
+        final List<String> jobXmlsList = attributes.getJobXmls();
+        final String[] remainingJobXmls = Arrays.copyOfRange(JOB_XMLS, 1, JOB_XMLS.length);
+        assertEquals(remainingJobXmls.length, jobXmlsList.size());
+
+        for (int i = 0; i < remainingJobXmls.length; ++i) {
+            assertEquals(remainingJobXmls[i], jobXmlsList.get(i));
+        }
+    }
+
+    @Test
+    public void testClearJobXmls() {
+        final ActionAttributesBuilder builder = getBuilderInstance();
+
+        for (final String jobXml : JOB_XMLS) {
+            builder.withJobXml(jobXml);
+        }
+
+        builder.clearJobXmls();
+
+        final ActionAttributes attributes = builder.build();
+
+        final List<String> jobXmlsList = attributes.getJobXmls();
+        assertEquals(0, jobXmlsList.size());
+    }
+
+
+
+    @Test
+    public void testConfigPropertyAdded() {
+        final ActionAttributesBuilder builder = getBuilderInstance();
+        builder.withConfigProperty(MAPRED_JOB_QUEUE_NAME, DEFAULT);
+
+        final ActionAttributes attributes = builder.build();
+        assertEquals(DEFAULT, attributes.getConfiguration().get(MAPRED_JOB_QUEUE_NAME));
+    }
+
+    @Test
+    public void testSeveralConfigPropertiesAdded() {
+        final ActionAttributesBuilder builder = getBuilderInstance();
+
+        for (final Map.Entry<String, String> entry : CONFIG_EXAMPLE.entrySet()) {
+            builder.withConfigProperty(entry.getKey(), entry.getValue());
+        }
+
+        final ActionAttributes attributes = builder.build();
+
+        for (final Map.Entry<String, String> entry : CONFIG_EXAMPLE.entrySet()) {
+            assertEquals(entry.getValue(), attributes.getConfiguration().get(entry.getKey()));
+        }
+
+        assertEquals(CONFIG_EXAMPLE, attributes.getConfiguration());
+    }
+
+    @Test
+    public void testSameConfigPropertyAddedTwiceThrows() {
+        final ActionAttributesBuilder builder = getBuilderInstance();
+        builder.withConfigProperty(MAPRED_JOB_QUEUE_NAME, DEFAULT);
+
+        expectedException.expect(IllegalStateException.class);
+        builder.withConfigProperty(MAPRED_JOB_QUEUE_NAME, DEFAULT);
+    }
+
+    @Test
+    public void testSeveralFilesAdded() {
+        final ActionAttributesBuilder builder = getBuilderInstance();
+
+        for (final String file : FILES) {
+            builder.withFile(file);
+        }
+
+        final ActionAttributes attributes = builder.build();
+
+        final List<String> filesList = attributes.getFiles();
+        assertEquals(FILES.length, filesList.size());
+
+        for (int i = 0; i < FILES.length; ++i) {
+            assertEquals(FILES[i], filesList.get(i));
+        }
+    }
+
+    @Test
+    public void testRemoveFiles() {
+        final ActionAttributesBuilder builder = getBuilderInstance();
+
+        for (final String file : FILES) {
+            builder.withFile(file);
+        }
+
+        builder.withoutFile(FILES[0]);
+
+        final ActionAttributes attributes = builder.build();
+
+        final List<String> filesList = attributes.getFiles();
+        final String[] remainingFiles = Arrays.copyOfRange(FILES, 1, FILES.length);
+        assertEquals(remainingFiles.length, filesList.size());
+
+        for (int i = 0; i < remainingFiles.length; ++i) {
+            assertEquals(remainingFiles[i], filesList.get(i));
+        }
+    }
+
+    @Test
+    public void testClearFiles() {
+        final ActionAttributesBuilder builder = getBuilderInstance();
+
+        for (final String file : FILES) {
+            builder.withFile(file);
+        }
+
+        builder.clearFiles();
+
+        final ActionAttributes attributes = builder.build();
+
+        final List<String> filesList = attributes.getFiles();
+        assertEquals(0, filesList.size());
+    }
+
+    @Test
+    public void testSeveralArchivesAdded() {
+        final ActionAttributesBuilder builder = getBuilderInstance();
+
+        for (final String archive : ARCHIVES) {
+            builder.withArchive(archive);
+        }
+
+        final ActionAttributes attributes = builder.build();
+
+        final List<String> filesList = attributes.getArchives();
+        assertEquals(ARCHIVES.length, filesList.size());
+
+        for (int i = 0; i < ARCHIVES.length; ++i) {
+            assertEquals(ARCHIVES[i], filesList.get(i));
+        }
+    }
+
+    @Test
+    public void testRemoveArchives() {
+        final ActionAttributesBuilder builder = getBuilderInstance();
+
+        for (final String archive : ARCHIVES) {
+            builder.withArchive(archive);
+        }
+
+        builder.withoutArchive(ARCHIVES[0]);
+
+        final ActionAttributes attributes = builder.build();
+
+        final List<String> archivesList = attributes.getArchives();
+        final String[] remainingArchives = Arrays.copyOfRange(ARCHIVES, 1, ARCHIVES.length);
+        assertEquals(remainingArchives.length, archivesList.size());
+
+        for (int i = 0; i < remainingArchives.length; ++i) {
+            assertEquals(remainingArchives[i], archivesList.get(i));
+        }
+    }
+
+    @Test
+    public void testClearArchives() {
+        final ActionAttributesBuilder builder = getBuilderInstance();
+
+        for (final String archive : ARCHIVES) {
+            builder.withArchive(archive);
+        }
+
+        builder.clearArchives();
+
+        final ActionAttributes attributes = builder.build();
+
+        final List<String> archivesList = attributes.getArchives();
+        assertEquals(0, archivesList.size());
+    }
+
+    @Test
+    public void testSeveralDeletesAdded() {
+        final ActionAttributesBuilder builder = getBuilderInstance();
+
+        for (final Delete delete : DELETES) {
+            builder.withDelete(delete);
+        }
+
+        final ActionAttributes attributes = builder.build();
+
+        assertEquals(Arrays.asList(DELETES), attributes.getDeletes());
+    }
+
+    @Test
+    public void testWithoutDelete() {
+        final ActionAttributesBuilder builder = getBuilderInstance();
+
+        for (final Delete delete : DELETES) {
+            builder.withDelete(delete);
+        }
+
+        builder.withoutDelete(DELETES[0]);
+
+        final ActionAttributes attributes = builder.build();
+
+        final List<Delete> expectedDeletes = Arrays.asList(DELETES).subList(1, DELETES.length);
+        assertEquals(expectedDeletes, attributes.getDeletes());
+    }
+
+    @Test
+    public void testClearDeletes() {
+        final ActionAttributesBuilder builder = getBuilderInstance();
+
+        for (final Delete delete : DELETES) {
+            builder.withDelete(delete);
+        }
+
+        builder.clearDeletes();
+
+        final ActionAttributes attributes = builder.build();
+
+        assertTrue(attributes.getDeletes().isEmpty());
+    }
+
+    @Test
+    public void testSeveralMkdirsAdded() {
+        final ActionAttributesBuilder builder = getBuilderInstance();
+
+        for (final Mkdir mkdir : MKDIRS) {
+            builder.withMkdir(mkdir);
+        }
+
+        final ActionAttributes attributes = builder.build();
+
+        assertEquals(Arrays.asList(MKDIRS), attributes.getMkdirs());
+    }
+
+    @Test
+    public void testWithoutMkdir() {
+        final ActionAttributesBuilder builder = getBuilderInstance();
+
+        for (final Mkdir mkdir : MKDIRS) {
+            builder.withMkdir(mkdir);
+        }
+
+        builder.withoutMkdir(MKDIRS[0]);
+
+        final ActionAttributes attributes = builder.build();
+
+        final List<Mkdir> expectedMkdirs = Arrays.asList(MKDIRS).subList(1, MKDIRS.length);
+        assertEquals(expectedMkdirs, attributes.getMkdirs());
+    }
+
+    @Test
+    public void testClearMkdirs() {
+        final ActionAttributesBuilder builder = getBuilderInstance();
+
+        for (final Mkdir mkdir : MKDIRS) {
+            builder.withMkdir(mkdir);
+        }
+
+        builder.clearMkdirs();
+
+        final ActionAttributes attributes = builder.build();
+
+        assertTrue(attributes.getMkdirs().isEmpty());
+    }
+
+    @Test
+    public void testSeveralMovesAdded() {
+        final ActionAttributesBuilder builder = getBuilderInstance();
+
+        for (final Move move : MOVES) {
+            builder.withMove(move);
+        }
+
+        final ActionAttributes attributes = builder.build();
+
+        assertEquals(Arrays.asList(MOVES), attributes.getMoves());
+    }
+
+    @Test
+    public void testWithoutMove() {
+        final ActionAttributesBuilder builder = getBuilderInstance();
+
+        for (final Move move : MOVES) {
+            builder.withMove(move);
+        }
+
+        builder.withoutMove(MOVES[0]);
+
+        final ActionAttributes attributes = builder.build();
+
+        final List<Move> expectedMoves = Arrays.asList(MOVES).subList(1, MOVES.length);
+        assertEquals(expectedMoves, attributes.getMoves());
+    }
+
+    @Test
+    public void testClearMoves() {
+        final ActionAttributesBuilder builder = getBuilderInstance();
+
+        for (final Move move : MOVES) {
+            builder.withMove(move);
+        }
+
+        builder.clearMoves();
+
+        final ActionAttributes attributes = builder.build();
+
+        assertTrue(attributes.getMoves().isEmpty());
+    }
+
+    @Test
+    public void testSeveralChmodsAdded() {
+        final ActionAttributesBuilder builder = getBuilderInstance();
+
+        for (final Chmod chmod : CHMODS) {
+            builder.withChmod(chmod);
+        }
+
+        final ActionAttributes attributes = builder.build();
+
+        assertEquals(Arrays.asList(CHMODS), attributes.getChmods());
+    }
+
+    @Test
+    public void testWithoutChmod() {
+        final ActionAttributesBuilder builder = getBuilderInstance();
+
+        for (final Chmod chmod : CHMODS) {
+            builder.withChmod(chmod);
+        }
+
+        builder.withoutChmod(CHMODS[0]);
+
+        final ActionAttributes attributes = builder.build();
+
+        final List<Chmod> expectedChmods = Arrays.asList(CHMODS).subList(1, CHMODS.length);
+        assertEquals(expectedChmods, attributes.getChmods());
+    }
+
+    @Test
+    public void testClearChmods() {
+        final ActionAttributesBuilder builder = getBuilderInstance();
+
+        for (final Chmod chmod : CHMODS) {
+            builder.withChmod(chmod);
+        }
+
+        builder.clearChmods();
+
+        final ActionAttributes attributes = builder.build();
+
+        assertTrue(attributes.getChmods().isEmpty());
+    }
+
+    @Test
+    public void testSeveralTouchzsAdded() {
+        final ActionAttributesBuilder builder = getBuilderInstance();
+
+        for (final Touchz touchz : TOUCHZS) {
+            builder.withTouchz(touchz);
+        }
+
+        final ActionAttributes attributes = builder.build();
+
+        assertEquals(Arrays.asList(TOUCHZS), attributes.getTouchzs());
+    }
+
+    @Test
+    public void testWithoutTouchz() {
+        final ActionAttributesBuilder builder = getBuilderInstance();
+
+        for (final Touchz touchz : TOUCHZS) {
+            builder.withTouchz(touchz);
+        }
+
+        builder.withoutTouchz(TOUCHZS[0]);
+
+        final ActionAttributes attributes = builder.build();
+
+        final List<Touchz> expectedTouchzs = Arrays.asList(TOUCHZS).subList(1, TOUCHZS.length);
+        assertEquals(expectedTouchzs, attributes.getTouchzs());
+    }
+
+    @Test
+    public void testClearTouchzs() {
+        final ActionAttributesBuilder builder = getBuilderInstance();
+
+        for (final Touchz touchz : TOUCHZS) {
+            builder.withTouchz(touchz);
+        }
+
+        builder.clearTouchzs();
+
+        final ActionAttributes attributes = builder.build();
+
+        assertTrue(attributes.getTouchzs().isEmpty());
+    }
+
+    @Test
+    public void testSeveralChgrpsAdded() {
+        final ActionAttributesBuilder builder = getBuilderInstance();
+
+        for (final Chgrp chgrp : CHGRPS) {
+            builder.withChgrp(chgrp);
+        }
+
+        final ActionAttributes attributes = builder.build();
+
+        assertEquals(Arrays.asList(CHGRPS), attributes.getChgrps());
+    }
+
+    @Test
+    public void testWithoutChgrp() {
+        final ActionAttributesBuilder builder = getBuilderInstance();
+
+        for (final Chgrp chgrp : CHGRPS) {
+            builder.withChgrp(chgrp);
+        }
+
+        builder.withoutChgrp(CHGRPS[0]);
+
+        final ActionAttributes attributes = builder.build();
+
+        final List<Chgrp> expectedChgrps = Arrays.asList(CHGRPS).subList(1, CHGRPS.length);
+        assertEquals(expectedChgrps, attributes.getChgrps());
+    }
+
+    @Test
+    public void testClearChgrps() {
+        final ActionAttributesBuilder builder = getBuilderInstance();
+
+        for (final Chgrp chgrp : CHGRPS) {
+            builder.withChgrp(chgrp);
+        }
+
+        builder.clearChgrps();
+
+        final ActionAttributes attributes = builder.build();
+
+        assertTrue(attributes.getChgrps().isEmpty());
+    }
+
+    @Test
+    public void testResourceManagerAdded() {
+        final ActionAttributesBuilder builder = getBuilderInstance();
+        builder.withResourceManager(RESOURCE_MANAGER);
+
+        final ActionAttributes attributes = builder.build();
+        assertEquals(RESOURCE_MANAGER, attributes.getResourceManager());
+    }
+
+    @Test
+    public void testLauncherAdded() {
+        final ActionAttributesBuilder builder = getBuilderInstance();
+        builder.withLauncher(LAUNCHER);
+
+        final ActionAttributes attributes = builder.build();
+        assertEquals(LAUNCHER.getMemoryMb(), attributes.getLauncher().getMemoryMb());
+        assertEquals(LAUNCHER.getVCores(), attributes.getLauncher().getVCores());
+        assertEquals(LAUNCHER.getQueue(), attributes.getLauncher().getQueue());
+        assertEquals(LAUNCHER.getSharelib(), attributes.getLauncher().getSharelib());
+        assertEquals(LAUNCHER.getViewAcl(), attributes.getLauncher().getViewAcl());
+        assertEquals(LAUNCHER.getModifyAcl(), attributes.getLauncher().getModifyAcl());
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestChBaseBuilder.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestChBaseBuilder.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestChBaseBuilder.java
new file mode 100644
index 0000000..87b8543
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestChBaseBuilder.java
@@ -0,0 +1,127 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import static org.junit.Assert.assertEquals;
+
+public abstract class TestChBaseBuilder<A extends ChFSBase, B extends ChFSBaseBuilder<B> & Builder<A>> {
+    @Rule
+    public final ExpectedException expectedException = ExpectedException.none();
+
+    protected abstract B getBuilderInstance();
+
+    @Test
+    public void testSetRecursive() {
+        final B builder = getBuilderInstance();
+        builder.setRecursive();
+
+        final A chmod = builder.build();
+        assertEquals(true, chmod.isRecursive());
+    }
+
+    @Test
+    public void testSetRecursiveCalledTwiceThrows() {
+        final B builder = getBuilderInstance();
+        builder.setRecursive();
+
+        expectedException.expect(IllegalStateException.class);
+        builder.setRecursive();
+    }
+
+    @Test
+    public void testSetNonRecursive() {
+        final B builder = getBuilderInstance();
+        builder.setNonRecursive();
+
+        final A chmod = builder.build();
+        assertEquals(false, chmod.isRecursive());
+    }
+
+    @Test
+    public void testSetNonRecursiveCalledTwiceThrows() {
+        final B builder = getBuilderInstance();
+        builder.setNonRecursive();
+
+        expectedException.expect(IllegalStateException.class);
+        builder.setNonRecursive();
+    }
+
+    @Test
+    public void testSetRecursiveCalledAfterSetNonRecursive() {
+        final B builder = getBuilderInstance();
+        builder.setNonRecursive();
+
+        expectedException.expect(IllegalStateException.class);
+        builder.setRecursive();
+    }
+
+    @Test
+    public void testSetNonRecursiveCalledAfterSetRecursive() {
+        final B builder = getBuilderInstance();
+        builder.setRecursive();
+
+        expectedException.expect(IllegalStateException.class);
+        builder.setNonRecursive();
+    }
+
+    @Test
+    public void testWithPath() {
+        final String path = "path";
+
+        final B builder = getBuilderInstance();
+        builder.withPath(path);
+
+        final A chmod = builder.build();
+        assertEquals(path, chmod.getPath());
+    }
+
+    @Test
+    public void testWithPathCalledTwiceThrows() {
+        final String path1 = "path1";
+        final String path2 = "path2";
+
+        final B builder = getBuilderInstance();
+        builder.withPath(path1);
+
+        expectedException.expect(IllegalStateException.class);
+        builder.withPath(path2);
+    }
+
+    @Test
+    public void testSetDirFiles() {
+        final B builder = getBuilderInstance();
+        builder.setDirFiles(false);
+
+        final A chmod = builder.build();
+        assertEquals("false", chmod.getDirFiles());
+    }
+
+    @Test
+    public void testSetDirFilesCalledTwiceThrows() {
+        final B builder = getBuilderInstance();
+        builder.setDirFiles(false);
+
+        expectedException.expect(IllegalStateException.class);
+        builder.setDirFiles(true);
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestChgrpBuilder.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestChgrpBuilder.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestChgrpBuilder.java
new file mode 100644
index 0000000..2f8a0fc
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestChgrpBuilder.java
@@ -0,0 +1,53 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestChgrpBuilder extends TestChBaseBuilder<Chgrp, ChgrpBuilder> {
+    @Override
+    protected ChgrpBuilder getBuilderInstance() {
+        return new ChgrpBuilder();
+    }
+
+    @Test
+    public void testWithGroup() {
+        final String group = "root";
+
+        final ChgrpBuilder builder = new ChgrpBuilder();
+        builder.withGroup(group);
+
+        final Chgrp chgrp = builder.build();
+        assertEquals(group, chgrp.getGroup());
+    }
+
+    @Test
+    public void testWithPermissionsCalledTwiceThrows() {
+        final String group1 = "group1";
+        final String group2 = "group1";
+
+        final ChgrpBuilder builder = new ChgrpBuilder();
+        builder.withGroup(group1);
+
+        expectedException.expect(IllegalStateException.class);
+        builder.withGroup(group2);
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestChmodBuilder.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestChmodBuilder.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestChmodBuilder.java
new file mode 100644
index 0000000..084f146
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestChmodBuilder.java
@@ -0,0 +1,53 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestChmodBuilder extends TestChBaseBuilder<Chmod, ChmodBuilder> {
+    @Override
+    protected ChmodBuilder getBuilderInstance() {
+        return new ChmodBuilder();
+    }
+
+    @Test
+    public void testWithPermissions() {
+        final String permission = "-rwxrw-rw-";
+
+        final ChmodBuilder builder = new ChmodBuilder();
+        builder.withPermissions(permission);
+
+        final Chmod chmod = builder.build();
+        assertEquals(permission, chmod.getPermissions());
+    }
+
+    @Test
+    public void testWithPermissionsCalledTwiceThrows() {
+        final String permission1 = "-rwxrw-rw-";
+        final String permission2 = "-rwxrw-r-";
+
+        final ChmodBuilder builder = new ChmodBuilder();
+        builder.withPermissions(permission1);
+
+        expectedException.expect(IllegalStateException.class);
+        builder.withPermissions(permission2);
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestDelete.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestDelete.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestDelete.java
new file mode 100644
index 0000000..402e016
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestDelete.java
@@ -0,0 +1,36 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestDelete {
+    @Test
+    public void testPathAndSkipTrashAreCorrect() {
+        final String path = "path/to/location";
+        final Boolean skipTrash = null;
+
+        final Delete delete = new Delete(path, skipTrash);
+
+        assertEquals(path, delete.getPath());
+        assertEquals(skipTrash, delete.getSkipTrash());
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestDistcpActionBuilder.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestDistcpActionBuilder.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestDistcpActionBuilder.java
new file mode 100644
index 0000000..b0e8e0c
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestDistcpActionBuilder.java
@@ -0,0 +1,213 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestDistcpActionBuilder extends TestNodeBuilderBaseImpl<DistcpAction, DistcpActionBuilder> {
+    private static final String NAME = "distcp-name";
+    private static final String RESOURCE_MANAGER = "${resourceManager}";
+    private static final String NAME_NODE = "${nameNode}";
+    private static final String EXAMPLE_DIR = "/path/to/directory";
+    private static final String[] ARGS = {"arg1", "arg2", "arg3"};
+
+    private static final String MAPRED_JOB_QUEUE_NAME = "mapred.job.queue.name";
+    private static final String DEFAULT = "default";
+
+    @Override
+    protected DistcpActionBuilder getBuilderInstance() {
+        return DistcpActionBuilder.create();
+    }
+
+    @Override
+    protected DistcpActionBuilder getBuilderInstance(final DistcpAction action) {
+        return DistcpActionBuilder.createFromExistingAction(action);
+    }
+
+    @Test
+    public void testResourceManagerAdded() {
+        final DistcpActionBuilder builder = getBuilderInstance();
+        builder.withResourceManager(RESOURCE_MANAGER);
+
+        final DistcpAction action = builder.build();
+        assertEquals(RESOURCE_MANAGER, action.getResourceManager());
+    }
+
+    @Test
+    public void testNameNodeAdded() {
+        final DistcpActionBuilder builder = getBuilderInstance();
+        builder.withNameNode(NAME_NODE);
+
+        final DistcpAction action = builder.build();
+        assertEquals(NAME_NODE, action.getNameNode());
+    }
+
+    @Test
+    public void testPrepareAdded() {
+        final DistcpActionBuilder builder = getBuilderInstance();
+        builder.withPrepare(new PrepareBuilder().withDelete(EXAMPLE_DIR).build());
+
+        final DistcpAction action = builder.build();
+        assertEquals(EXAMPLE_DIR, action.getPrepare().getDeletes().get(0).getPath());
+    }
+
+    @Test
+    public void testSameConfigPropertyAddedTwiceThrows() {
+        final DistcpActionBuilder builder = getBuilderInstance();
+        builder.withConfigProperty(MAPRED_JOB_QUEUE_NAME, DEFAULT);
+
+        expectedException.expect(IllegalStateException.class);
+        builder.withConfigProperty(MAPRED_JOB_QUEUE_NAME, DEFAULT);
+    }
+
+    @Test
+    public void testJavaOptsAdded() {
+        final DistcpActionBuilder builder = getBuilderInstance();
+        builder.withJavaOpts("-Dopt1 -Dopt2");
+
+        final DistcpAction action = builder.build();
+        assertEquals("-Dopt1 -Dopt2", action.getJavaOpts());
+    }
+
+    @Test
+    public void testSeveralArgsAdded() {
+        final DistcpActionBuilder builder = getBuilderInstance();
+
+        for (final String arg : ARGS) {
+            builder.withArg(arg);
+        }
+
+        final DistcpAction action = builder.build();
+
+        final List<String> argList = action.getArgs();
+        assertEquals(ARGS.length, argList.size());
+
+        for (int i = 0; i < ARGS.length; ++i) {
+            assertEquals(ARGS[i], argList.get(i));
+        }
+    }
+
+    @Test
+    public void testRemoveArgs() {
+        final DistcpActionBuilder builder = getBuilderInstance();
+
+        for (final String file : ARGS) {
+            builder.withArg(file);
+        }
+
+        builder.withoutArg(ARGS[0]);
+
+        final DistcpAction action = builder.build();
+
+        final List<String> argList = action.getArgs();
+        final String[] remainingArgs = Arrays.copyOfRange(ARGS, 1, ARGS.length);
+        assertEquals(remainingArgs.length, argList.size());
+
+        for (int i = 0; i < remainingArgs.length; ++i) {
+            assertEquals(remainingArgs[i], argList.get(i));
+        }
+    }
+
+    @Test
+    public void testClearArgs() {
+        final DistcpActionBuilder builder = getBuilderInstance();
+
+        for (final String file : ARGS) {
+            builder.withArg(file);
+        }
+
+        builder.clearArgs();
+
+        final DistcpAction mrAction = builder.build();
+
+        final List<String> argList = mrAction.getArgs();
+        assertEquals(0, argList.size());
+    }
+    @Test
+    public void testFromExistingActionDistcpSpecific() {
+        final DistcpActionBuilder builder = getBuilderInstance();
+
+        builder.withName(NAME)
+                .withNameNode(NAME_NODE)
+                .withConfigProperty(MAPRED_JOB_QUEUE_NAME, DEFAULT)
+                .withArg(ARGS[0])
+                .withArg(ARGS[1]);
+
+        final DistcpAction action = builder.build();
+
+        final DistcpActionBuilder fromExistingBuilder = getBuilderInstance(action);
+
+        final String newName = "fromExisting_" + NAME;
+        fromExistingBuilder.withName(newName)
+                .withoutArg(ARGS[1])
+                .withArg(ARGS[2]);
+
+        final DistcpAction modifiedAction = fromExistingBuilder.build();
+
+        assertEquals(newName, modifiedAction.getName());
+        assertEquals(action.getNameNode(), modifiedAction.getNameNode());
+
+        final Map<String, String> expectedConfiguration = new LinkedHashMap<>();
+        expectedConfiguration.put(MAPRED_JOB_QUEUE_NAME, DEFAULT);
+        assertEquals(expectedConfiguration, modifiedAction.getConfiguration());
+
+        assertEquals(Arrays.asList(ARGS[0], ARGS[2]), modifiedAction.getArgs());
+    }
+
+    @Test
+    public void testFromOtherAction() {
+        final ShellAction parent = ShellActionBuilder.create()
+                .withName("parent")
+                .withExecutable("echo")
+                .build();
+
+        final ShellAction otherAction = ShellActionBuilder.createFromExistingAction(parent)
+                .withName("shell")
+                .withParent(parent)
+                .withNameNode(NAME_NODE)
+                .withConfigProperty(MAPRED_JOB_QUEUE_NAME, DEFAULT)
+                .withArgument(ARGS[0])
+                .withArgument(ARGS[1])
+                .build();
+
+        final String newName = "fromExisting_" + NAME;
+        final DistcpAction fromOtherAction = DistcpActionBuilder.createFromExistingAction(otherAction)
+                .withName(newName)
+                .withoutArg(ARGS[1])
+                .withArg(ARGS[2])
+                .build();
+
+        assertEquals(newName, fromOtherAction.getName());
+        assertEquals(otherAction.getNameNode(), fromOtherAction.getNameNode());
+        assertEquals(parent, fromOtherAction.getParentsWithoutConditions().get(0));
+
+        final Map<String, String> expectedConfiguration = new LinkedHashMap<>();
+        expectedConfiguration.put(MAPRED_JOB_QUEUE_NAME, DEFAULT);
+        assertEquals(expectedConfiguration, fromOtherAction.getConfiguration());
+
+        assertEquals(Arrays.asList(ARGS[0], ARGS[2]), fromOtherAction.getArgs());
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestEmailActionBuilder.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestEmailActionBuilder.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestEmailActionBuilder.java
new file mode 100644
index 0000000..c3ba356
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestEmailActionBuilder.java
@@ -0,0 +1,197 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestEmailActionBuilder extends TestNodeBuilderBaseImpl<EmailAction, EmailActionBuilder> {
+    @Override
+    protected EmailActionBuilder getBuilderInstance() {
+        return EmailActionBuilder.create();
+    }
+
+    @Override
+    protected EmailActionBuilder getBuilderInstance(EmailAction action) {
+        return EmailActionBuilder.createFromExistingAction(action);
+    }
+
+    @Test
+    public void testRecipientAdded() {
+        final String recipient = "recipient@something.com";
+
+        final EmailActionBuilder builder = getBuilderInstance();
+        builder.withRecipient(recipient);
+
+        final EmailAction emailAction = builder.build();
+        assertEquals(recipient, emailAction.getRecipient());
+    }
+
+    @Test
+    public void testRecipientAddedTwiceThrows() {
+        final EmailActionBuilder builder = getBuilderInstance();
+        builder.withRecipient("some.recipient@something.com");
+
+        expectedException.expect(IllegalStateException.class);
+        builder.withRecipient("any.recipient@something.com");
+    }
+
+    @Test
+    public void testCcAdded() {
+        final String cc = "recipient@something.com";
+
+        final EmailActionBuilder builder = getBuilderInstance();
+        builder.withCc(cc);
+
+        final EmailAction emailAction = builder.build();
+        assertEquals(cc, emailAction.getCc());
+    }
+
+    @Test
+    public void testCcAddedTwiceThrows() {
+        final EmailActionBuilder builder = getBuilderInstance();
+        builder.withCc("some.recipient@something.com");
+
+        expectedException.expect(IllegalStateException.class);
+        builder.withCc("any.recipient@something.com");
+    }
+
+    @Test
+    public void testBccAdded() {
+        final String bcc = "recipient@something.com";
+
+        final EmailActionBuilder builder = getBuilderInstance();
+        builder.withBcc(bcc);
+
+        final EmailAction emailAction = builder.build();
+        assertEquals(bcc, emailAction.getBcc());
+    }
+
+    @Test
+    public void testBccAddedTwiceThrows() {
+        final EmailActionBuilder builder = getBuilderInstance();
+        builder.withBcc("some.recipient@something.com");
+
+        expectedException.expect(IllegalStateException.class);
+        builder.withBcc("any.recipient@something.com");
+    }
+
+    @Test
+    public void testSubjectAdded() {
+        final String subject = "Subject";
+
+        final EmailActionBuilder builder = getBuilderInstance();
+        builder.withSubject(subject);
+
+        final EmailAction emailAction = builder.build();
+        assertEquals(subject, emailAction.getSubject());
+    }
+
+    @Test
+    public void testSubjectAddedTwiceThrows() {
+        final EmailActionBuilder builder = getBuilderInstance();
+        builder.withSubject("Subject");
+
+        expectedException.expect(IllegalStateException.class);
+        builder.withSubject("Any subject");
+    }
+
+    @Test
+    public void testBodyAdded() {
+        final String body = "Email body.";
+
+        final EmailActionBuilder builder = getBuilderInstance();
+        builder.withBody(body);
+
+        final EmailAction emailAction = builder.build();
+        assertEquals(body, emailAction.getBody());
+    }
+
+    @Test
+    public void testBodyAddedTwiceThrows() {
+        final EmailActionBuilder builder = getBuilderInstance();
+        builder.withBody("Email body.");
+
+        expectedException.expect(IllegalStateException.class);
+        builder.withBody("Any email body.");
+    }
+
+    @Test
+    public void testContentTypeAdded() {
+        final String contentType = "content_type";
+
+        final EmailActionBuilder builder = getBuilderInstance();
+        builder.withContentType(contentType);
+
+        final EmailAction emailAction = builder.build();
+        assertEquals(contentType, emailAction.getContentType());
+    }
+
+    @Test
+    public void testContentTypeAddedTwiceThrows() {
+        final EmailActionBuilder builder = getBuilderInstance();
+        builder.withContentType("content_type");
+
+        expectedException.expect(IllegalStateException.class);
+        builder.withContentType("any_content_type");
+    }
+
+    @Test
+    public void testAttachmentAdded() {
+        final String attachment = "attachment";
+
+        final EmailActionBuilder builder = getBuilderInstance();
+        builder.withAttachment(attachment);
+
+        final EmailAction emailAction = builder.build();
+        assertEquals(attachment, emailAction.getAttachment());
+    }
+
+    @Test
+    public void testAttachmentAddedTwiceThrows() {
+        final EmailActionBuilder builder = getBuilderInstance();
+        builder.withAttachment("attachment");
+
+        expectedException.expect(IllegalStateException.class);
+        builder.withAttachment("any_attachment");
+    }
+
+    @Test
+    public void testFromOtherAction() {
+        final ShellAction parent = ShellActionBuilder.create()
+                .withName("parent")
+                .withExecutable("echo")
+                .build();
+
+        final ShellAction otherAction = ShellActionBuilder.createFromExistingAction(parent)
+                .withName("shell")
+                .withParent(parent)
+                .build();
+
+        final EmailAction fromOtherAction = EmailActionBuilder.createFromExistingAction(otherAction)
+                .withName("email")
+                .withBody("body")
+                .build();
+
+        assertEquals(parent, fromOtherAction.getParentsWithoutConditions().get(0));
+        assertEquals("email", fromOtherAction.getName());
+        assertEquals("body", fromOtherAction.getBody());
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestErrorHandler.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestErrorHandler.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestErrorHandler.java
new file mode 100644
index 0000000..2fe01a2
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestErrorHandler.java
@@ -0,0 +1,51 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestErrorHandler {
+    @Rule
+    public final ExpectedException expectedException = ExpectedException.none();
+
+    @Test
+    public void testNameIsCorrect() {
+        final String name = "error-handler";
+        final Builder<MapReduceAction> handlerBuilder = MapReduceActionBuilder.create().withName(name);
+
+        final ErrorHandler errorHandler = ErrorHandler.buildAsErrorHandler(handlerBuilder);
+        assertEquals(name, errorHandler.getName());
+    }
+
+    @Test
+    public void testIfThereAreParentsThenThrows() {
+        final String name = "error-handler";
+        final Node parent = MapReduceActionBuilder.create().withName("parent").build();
+        final Builder<MapReduceAction> handlerBuilder = MapReduceActionBuilder.create()
+                .withName(name)
+                .withParent(parent);
+
+        expectedException.expect(IllegalStateException.class);
+        ErrorHandler.buildAsErrorHandler(handlerBuilder);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestFSActionBuilder.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestFSActionBuilder.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestFSActionBuilder.java
new file mode 100644
index 0000000..d83ceed
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestFSActionBuilder.java
@@ -0,0 +1,469 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class TestFSActionBuilder extends TestNodeBuilderBaseImpl<FSAction, FSActionBuilder> {
+    private static final String NAME_NODE = "${nameNode}";
+    private static final String[] JOB_XMLS = {"jobXml1.xml", "jobXml2.xml", "jobXml3.xml", "jobXml4.xml"};
+    private static final Delete[] DELETES = {new Delete("path0", null),
+                                             new Delete("path1", true),
+                                             new Delete("path2", false),
+                                             new Delete("path3", null)
+                                            };
+
+    private static final Mkdir[] MKDIRS = {new Mkdir("path0"),
+                                           new Mkdir("path1"),
+                                           new Mkdir("path2"),
+                                           new Mkdir("path3")
+                                          };
+
+    private static final Move[] MOVES = {new Move("source0", "target0"),
+                                         new Move("source1", "target1"),
+                                         new Move("source2", "target2"),
+                                         new Move("source3", "target3")
+                                        };
+
+    private static final Chmod[] CHMODS = {new ChmodBuilder().withPermissions("711").build(),
+                                           new ChmodBuilder().withPermissions("511").build(),
+                                           new ChmodBuilder().withPermissions("551").build(),
+                                           new ChmodBuilder().withPermissions("755").build()
+                                          };
+
+    private static final Touchz[] TOUCHZS = {new Touchz("path0"),
+                                             new Touchz("path1"),
+                                             new Touchz("path2"),
+                                             new Touchz("path3")
+                                            };
+
+    private static final Chgrp[] CHGRPS = {new ChgrpBuilder().withGroup("user0").build(),
+                                           new ChgrpBuilder().withGroup("user1").build(),
+                                           new ChgrpBuilder().withGroup("user2").build(),
+                                           new ChgrpBuilder().withGroup("user3").build()
+                                          };
+
+    private static final String MAPRED_JOB_QUEUE_NAME = "mapred.job.queue.name";
+    private static final String DEFAULT = "default";
+
+    @Override
+    protected FSActionBuilder getBuilderInstance() {
+        return FSActionBuilder.create();
+    }
+
+    @Override
+    protected FSActionBuilder getBuilderInstance(FSAction action) {
+        return FSActionBuilder.createFromExistingAction(action);
+    }
+
+    @Test
+    public void testNameNodeAdded() {
+        final FSActionBuilder builder = getBuilderInstance();
+        builder.withNameNode(NAME_NODE);
+
+        final FSAction mrAction = builder.build();
+        assertEquals(NAME_NODE, mrAction.getNameNode());
+    }
+
+    @Test
+    public void testNameNodeAddedTwiceThrows() {
+        final FSActionBuilder builder = getBuilderInstance();
+        builder.withNameNode(NAME_NODE);
+
+        expectedException.expect(IllegalStateException.class);
+        builder.withNameNode("any_string");
+    }
+
+    @Test
+    public void testSeveralJobXmlsAdded() {
+        final FSActionBuilder builder = getBuilderInstance();
+
+        for (final String jobXml : JOB_XMLS) {
+            builder.withJobXml(jobXml);
+        }
+
+        final FSAction fsAction = builder.build();
+
+        final List<String> jobXmlsList = fsAction.getJobXmls();
+        assertEquals(JOB_XMLS.length, jobXmlsList.size());
+
+        for (int i = 0; i < JOB_XMLS.length; ++i) {
+            assertEquals(JOB_XMLS[i], jobXmlsList.get(i));
+        }
+    }
+
+    @Test
+    public void testWithoutJobXmls() {
+        final FSActionBuilder builder = getBuilderInstance();
+
+        for (final String jobXml : JOB_XMLS) {
+            builder.withJobXml(jobXml);
+        }
+
+        builder.withoutJobXml(JOB_XMLS[0]);
+
+        final FSAction fsAction = builder.build();
+
+        final List<String> jobXmlsList = fsAction.getJobXmls();
+        final String[] remainingJobXmls = Arrays.copyOfRange(JOB_XMLS, 1, JOB_XMLS.length);
+        assertEquals(remainingJobXmls.length, jobXmlsList.size());
+
+        for (int i = 0; i < remainingJobXmls.length; ++i) {
+            assertEquals(remainingJobXmls[i], jobXmlsList.get(i));
+        }
+    }
+
+    @Test
+    public void testClearJobXmls() {
+        final FSActionBuilder builder = getBuilderInstance();
+
+        for (final String jobXml : JOB_XMLS) {
+            builder.withJobXml(jobXml);
+        }
+
+        builder.clearJobXmls();
+
+        final FSAction fsAction = builder.build();
+
+        final List<String> jobXmlsList = fsAction.getJobXmls();
+        assertEquals(0, jobXmlsList.size());
+    }
+
+    @Test
+    public void testSameConfigPropertyAddedTwiceThrows() {
+        final FSActionBuilder builder = getBuilderInstance();
+        builder.withConfigProperty(MAPRED_JOB_QUEUE_NAME, DEFAULT);
+
+        expectedException.expect(IllegalStateException.class);
+        builder.withConfigProperty(MAPRED_JOB_QUEUE_NAME, DEFAULT);
+    }
+
+    @Test
+    public void testSeveralDeletesAdded() {
+        final FSActionBuilder builder = getBuilderInstance();
+
+        for (final Delete delete : DELETES) {
+            builder.withDelete(delete);
+        }
+
+        final FSAction fsAction = builder.build();
+
+        assertEquals(Arrays.asList(DELETES), fsAction.getDeletes());
+    }
+
+    @Test
+    public void testWithoutDelete() {
+        final FSActionBuilder builder = getBuilderInstance();
+
+        for (final Delete delete : DELETES) {
+            builder.withDelete(delete);
+        }
+
+        builder.withoutDelete(DELETES[0]);
+
+        final FSAction fsAction = builder.build();
+
+        final List<Delete> expectedDeletes = Arrays.asList(DELETES).subList(1, DELETES.length);
+        assertEquals(expectedDeletes, fsAction.getDeletes());
+    }
+
+    @Test
+    public void testClearDeletes() {
+        final FSActionBuilder builder = getBuilderInstance();
+
+        for (final Delete delete : DELETES) {
+            builder.withDelete(delete);
+        }
+
+        builder.clearDeletes();
+
+        final FSAction fsAction = builder.build();
+
+        assertTrue(fsAction.getDeletes().isEmpty());
+    }
+
+    @Test
+    public void testSeveralMkdirsAdded() {
+        final FSActionBuilder builder = getBuilderInstance();
+
+        for (final Mkdir mkdir : MKDIRS) {
+            builder.withMkdir(mkdir);
+        }
+
+        final FSAction fsAction = builder.build();
+
+        assertEquals(Arrays.asList(MKDIRS), fsAction.getMkdirs());
+    }
+
+    @Test
+    public void testWithoutMkdir() {
+        final FSActionBuilder builder = getBuilderInstance();
+
+        for (final Mkdir mkdir : MKDIRS) {
+            builder.withMkdir(mkdir);
+        }
+
+        builder.withoutMkdir(MKDIRS[0]);
+
+        final FSAction fsAction = builder.build();
+
+        final List<Mkdir> expectedMkdirs = Arrays.asList(MKDIRS).subList(1, MKDIRS.length);
+        assertEquals(expectedMkdirs, fsAction.getMkdirs());
+    }
+
+    @Test
+    public void testClearMkdirs() {
+        final FSActionBuilder builder = getBuilderInstance();
+
+        for (final Mkdir mkdir : MKDIRS) {
+            builder.withMkdir(mkdir);
+        }
+
+        builder.clearMkdirs();
+
+        final FSAction fsAction = builder.build();
+
+        assertTrue(fsAction.getMkdirs().isEmpty());
+    }
+
+    @Test
+    public void testSeveralMovesAdded() {
+        final FSActionBuilder builder = getBuilderInstance();
+
+        for (final Move move : MOVES) {
+            builder.withMove(move);
+        }
+
+        final FSAction fsAction = builder.build();
+
+        assertEquals(Arrays.asList(MOVES), fsAction.getMoves());
+    }
+
+    @Test
+    public void testWithoutMove() {
+        final FSActionBuilder builder = getBuilderInstance();
+
+        for (final Move move : MOVES) {
+            builder.withMove(move);
+        }
+
+        builder.withoutMove(MOVES[0]);
+
+        final FSAction fsAction = builder.build();
+
+        final List<Move> expectedMoves = Arrays.asList(MOVES).subList(1, MOVES.length);
+        assertEquals(expectedMoves, fsAction.getMoves());
+    }
+
+    @Test
+    public void testClearMoves() {
+        final FSActionBuilder builder = getBuilderInstance();
+
+        for (final Move move : MOVES) {
+            builder.withMove(move);
+        }
+
+        builder.clearMoves();
+
+        final FSAction fsAction = builder.build();
+
+        assertTrue(fsAction.getMoves().isEmpty());
+    }
+
+    @Test
+    public void testSeveralChmodsAdded() {
+        final FSActionBuilder builder = getBuilderInstance();
+
+        for (final Chmod chmod : CHMODS) {
+            builder.withChmod(chmod);
+        }
+
+        final FSAction fsAction = builder.build();
+
+        assertEquals(Arrays.asList(CHMODS), fsAction.getChmods());
+    }
+
+    @Test
+    public void testWithoutChmod() {
+        final FSActionBuilder builder = getBuilderInstance();
+
+        for (final Chmod chmod : CHMODS) {
+            builder.withChmod(chmod);
+        }
+
+        builder.withoutChmod(CHMODS[0]);
+
+        final FSAction fsAction = builder.build();
+
+        final List<Chmod> expectedChmods = Arrays.asList(CHMODS).subList(1, CHMODS.length);
+        assertEquals(expectedChmods, fsAction.getChmods());
+    }
+
+    @Test
+    public void testClearChmods() {
+        final FSActionBuilder builder = getBuilderInstance();
+
+        for (final Chmod chmod : CHMODS) {
+            builder.withChmod(chmod);
+        }
+
+        builder.clearChmods();
+
+        final FSAction fsAction = builder.build();
+
+        assertTrue(fsAction.getChmods().isEmpty());
+    }
+
+    @Test
+    public void testSeveralTouchzsAdded() {
+        final FSActionBuilder builder = getBuilderInstance();
+
+        for (final Touchz touchz : TOUCHZS) {
+            builder.withTouchz(touchz);
+        }
+
+        final FSAction fsAction = builder.build();
+
+        assertEquals(Arrays.asList(TOUCHZS), fsAction.getTouchzs());
+    }
+
+    @Test
+    public void testWithoutTouchz() {
+        final FSActionBuilder builder = getBuilderInstance();
+
+        for (final Touchz touchz : TOUCHZS) {
+            builder.withTouchz(touchz);
+        }
+
+        builder.withoutTouchz(TOUCHZS[0]);
+
+        final FSAction fsAction = builder.build();
+
+        final List<Touchz> expectedTouchzs = Arrays.asList(TOUCHZS).subList(1, TOUCHZS.length);
+        assertEquals(expectedTouchzs, fsAction.getTouchzs());
+    }
+
+    @Test
+    public void testClearTouchzs() {
+        final FSActionBuilder builder = getBuilderInstance();
+
+        for (final Touchz touchz : TOUCHZS) {
+            builder.withTouchz(touchz);
+        }
+
+        builder.clearTouchzs();
+
+        final FSAction fsAction = builder.build();
+
+        assertTrue(fsAction.getTouchzs().isEmpty());
+    }
+
+    @Test
+    public void testSeveralChgrpsAdded() {
+        final FSActionBuilder builder = getBuilderInstance();
+
+        for (final Chgrp chgrp : CHGRPS) {
+            builder.withChgrp(chgrp);
+        }
+
+        final FSAction fsAction = builder.build();
+
+        assertEquals(Arrays.asList(CHGRPS), fsAction.getChgrps());
+    }
+
+    @Test
+    public void testWithoutChgrp() {
+        final FSActionBuilder builder = getBuilderInstance();
+
+        for (final Chgrp chgrp : CHGRPS) {
+            builder.withChgrp(chgrp);
+        }
+
+        builder.withoutChgrp(CHGRPS[0]);
+
+        final FSAction fsAction = builder.build();
+
+        final List<Chgrp> expectedChgrps = Arrays.asList(CHGRPS).subList(1, CHGRPS.length);
+        assertEquals(expectedChgrps, fsAction.getChgrps());
+    }
+
+    @Test
+    public void testClearChgrps() {
+        final FSActionBuilder builder = getBuilderInstance();
+
+        for (final Chgrp chgrp : CHGRPS) {
+            builder.withChgrp(chgrp);
+        }
+
+        builder.clearChgrps();
+
+        final FSAction fsAction = builder.build();
+
+        assertTrue(fsAction.getChgrps().isEmpty());
+    }
+
+    @Test
+    public void testFromExistingFSAction() {
+        final String nameNode = "${nameNode}";
+
+        final FSActionBuilder builder = getBuilderInstance();
+        builder.withNameNode(nameNode)
+                .withConfigProperty(MAPRED_JOB_QUEUE_NAME, DEFAULT);
+
+        final FSAction action = builder.build();
+
+        final FSActionBuilder fromExistingBuilder = getBuilderInstance(action);
+
+        final FSAction modifiedAction = fromExistingBuilder.build();
+        assertEquals(nameNode, modifiedAction.getNameNode());
+
+        final Map<String, String> expectedConfiguration = new LinkedHashMap<>();
+        expectedConfiguration.put(MAPRED_JOB_QUEUE_NAME, DEFAULT);
+        assertEquals(expectedConfiguration, modifiedAction.getConfiguration());
+    }
+
+    @Test
+    public void testFromEmailAction() {
+        final EmailAction parent = EmailActionBuilder.create()
+                .withName("parent")
+                .build();
+
+        final EmailAction other = EmailActionBuilder.createFromExistingAction(parent)
+                .withName("other")
+                .withParent(parent)
+                .build();
+
+        final FSAction fromEmail = FSActionBuilder.createFromExistingAction(other)
+                .withName("fs")
+                .withNameNode("${nameNode}")
+                .build();
+
+        assertEquals(parent, fromEmail.getParentsWithoutConditions().get(0));
+        assertEquals("fs", fromEmail.getName());
+        assertEquals("${nameNode}", fromEmail.getNameNode());
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestHive2ActionBuilder.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestHive2ActionBuilder.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestHive2ActionBuilder.java
new file mode 100644
index 0000000..0893c18
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestHive2ActionBuilder.java
@@ -0,0 +1,224 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestHive2ActionBuilder extends TestNodeBuilderBaseImpl<Hive2Action, Hive2ActionBuilder> {
+    private static final String NAME = "hive2-name";
+    private static final String NAME_NODE = "${nameNode}";
+    private static final String EXAMPLE_DIR = "/path/to/directory";
+    private static final String[] ARGS = {"arg1", "arg2", "arg3"};
+    private static final String MAPRED_JOB_QUEUE_NAME = "mapred.job.queue.name";
+    private static final String DEFAULT = "default";
+    private static final String RESOURCE_MANAGER = "${resourceManager}";
+    private static final String PATH_TO_DELETE = "/path/to/delete";
+    private static final String PATH_TO_MKDIR = "/path/to/mkdir";
+
+    @Override
+    protected Hive2ActionBuilder getBuilderInstance() {
+        return Hive2ActionBuilder.create();
+    }
+
+    @Override
+    protected Hive2ActionBuilder getBuilderInstance(final Hive2Action action) {
+        return Hive2ActionBuilder.createFromExistingAction(action);
+    }
+
+    @Test
+    public void testResourceManagerAdded() {
+        final Hive2ActionBuilder builder = getBuilderInstance();
+        builder.withResourceManager(RESOURCE_MANAGER);
+
+        final Hive2Action action = builder.build();
+        assertEquals(RESOURCE_MANAGER, action.getResourceManager());
+    }
+
+    @Test
+    public void testNameNodeAdded() {
+        final Hive2ActionBuilder builder = getBuilderInstance();
+        builder.withNameNode(NAME_NODE);
+
+        final Hive2Action action = builder.build();
+        assertEquals(NAME_NODE, action.getNameNode());
+    }
+
+    @Test
+    public void testPrepareAdded() {
+        final Hive2ActionBuilder builder = getBuilderInstance();
+        builder.withPrepare(new PrepareBuilder().withDelete(EXAMPLE_DIR).build());
+
+        final Hive2Action action = builder.build();
+        assertEquals(EXAMPLE_DIR, action.getPrepare().getDeletes().get(0).getPath());
+    }
+
+    @Test
+    public void testSameConfigPropertyAddedTwiceThrows() {
+        final Hive2ActionBuilder builder = getBuilderInstance();
+        builder.withConfigProperty(MAPRED_JOB_QUEUE_NAME, DEFAULT);
+
+        expectedException.expect(IllegalStateException.class);
+        builder.withConfigProperty(MAPRED_JOB_QUEUE_NAME, DEFAULT);
+    }
+
+    @Test
+    public void testSeveralArgsAdded() {
+        final Hive2ActionBuilder builder = getBuilderInstance();
+
+        for (final String arg : ARGS) {
+            builder.withArg(arg);
+        }
+
+        final Hive2Action action = builder.build();
+
+        final List<String> argList = action.getArgs();
+        assertEquals(ARGS.length, argList.size());
+
+        for (int i = 0; i < ARGS.length; ++i) {
+            assertEquals(ARGS[i], argList.get(i));
+        }
+    }
+
+    @Test
+    public void testRemoveArgs() {
+        final Hive2ActionBuilder builder = getBuilderInstance();
+
+        for (final String file : ARGS) {
+            builder.withArg(file);
+        }
+
+        builder.withoutArg(ARGS[0]);
+
+        final Hive2Action action = builder.build();
+
+        final List<String> argList = action.getArgs();
+        final String[] remainingArgs = Arrays.copyOfRange(ARGS, 1, ARGS.length);
+        assertEquals(remainingArgs.length, argList.size());
+
+        for (int i = 0; i < remainingArgs.length; ++i) {
+            assertEquals(remainingArgs[i], argList.get(i));
+        }
+    }
+
+    @Test
+    public void testClearArgs() {
+        final Hive2ActionBuilder builder = getBuilderInstance();
+
+        for (final String file : ARGS) {
+            builder.withArg(file);
+        }
+
+        builder.clearArgs();
+
+        final Hive2Action action = builder.build();
+
+        final List<String> argList = action.getArgs();
+        assertEquals(0, argList.size());
+    }
+
+    @Test
+    public void testFromExistingHive2Action() {
+        final Hive2ActionBuilder builder = getBuilderInstance();
+
+        builder.withName(NAME)
+                .withResourceManager(RESOURCE_MANAGER)
+                .withNameNode(NAME_NODE)
+                .withConfigProperty(MAPRED_JOB_QUEUE_NAME, DEFAULT)
+                .withPrepare(new PrepareBuilder()
+                        .withDelete(PATH_TO_DELETE)
+                        .withMkdir(PATH_TO_MKDIR)
+                        .build())
+                .withLauncher(new LauncherBuilder()
+                        .withMemoryMb(1024L)
+                        .withVCores(2L)
+                        .withQueue(DEFAULT)
+                        .withSharelib(DEFAULT)
+                        .withViewAcl(DEFAULT)
+                        .withModifyAcl(DEFAULT)
+                        .build())
+                .withArg(ARGS[0])
+                .withArg(ARGS[1])
+                .withJdbcUrl(DEFAULT)
+                .withPassword(DEFAULT)
+                .withQuery(DEFAULT)
+                .withArchive(DEFAULT)
+                .withFile(DEFAULT);
+
+        final Hive2Action action = builder.build();
+
+        final Hive2ActionBuilder fromExistingBuilder = getBuilderInstance(action);
+
+        final String newName = "fromExisting_" + NAME;
+        fromExistingBuilder.withName(newName)
+                .withoutArg(ARGS[1])
+                .withArg(ARGS[2]);
+
+        final Hive2Action modifiedAction = fromExistingBuilder.build();
+
+        assertEquals(newName, modifiedAction.getName());
+        assertEquals(action.getNameNode(), modifiedAction.getNameNode());
+
+        final Map<String, String> expectedConfiguration = new LinkedHashMap<>();
+        expectedConfiguration.put(MAPRED_JOB_QUEUE_NAME, DEFAULT);
+        assertEquals(expectedConfiguration, modifiedAction.getConfiguration());
+
+        assertEquals(Arrays.asList(ARGS[0], ARGS[2]), modifiedAction.getArgs());
+
+        assertEquals(PATH_TO_DELETE, modifiedAction.getPrepare().getDeletes().get(0).getPath());
+        assertEquals(PATH_TO_MKDIR, modifiedAction.getPrepare().getMkdirs().get(0).getPath());
+
+        assertEquals(1024L, modifiedAction.getLauncher().getMemoryMb());
+        assertEquals(2L, modifiedAction.getLauncher().getVCores());
+        assertEquals(DEFAULT, modifiedAction.getLauncher().getQueue());
+        assertEquals(DEFAULT, modifiedAction.getLauncher().getSharelib());
+        assertEquals(DEFAULT, modifiedAction.getLauncher().getViewAcl());
+        assertEquals(DEFAULT, modifiedAction.getLauncher().getModifyAcl());
+
+        assertEquals(action.getJdbcUrl(), modifiedAction.getJdbcUrl());
+        assertEquals(action.getPassword(), modifiedAction.getPassword());
+        assertEquals(action.getScript(), modifiedAction.getScript());
+        assertEquals(action.getQuery(), modifiedAction.getQuery());
+    }
+
+    @Test
+    public void testFromOtherAction() {
+        final ShellAction parent = ShellActionBuilder.create()
+                .withName("parent")
+                .build();
+
+        final ShellAction otherAction = ShellActionBuilder.createFromExistingAction(parent)
+                .withName("shell")
+                .withParent(parent)
+                .build();
+
+        final Hive2Action fromOtherAction = Hive2ActionBuilder.createFromExistingAction(otherAction)
+                .withName("hive2")
+                .build();
+
+        assertEquals("hive2", fromOtherAction.getName());
+        assertEquals(parent, fromOtherAction.getParentsWithoutConditions().get(0));
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestHiveActionBuilder.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestHiveActionBuilder.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestHiveActionBuilder.java
new file mode 100644
index 0000000..4c9995a
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestHiveActionBuilder.java
@@ -0,0 +1,220 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestHiveActionBuilder extends TestNodeBuilderBaseImpl<HiveAction, HiveActionBuilder> {
+    private static final String NAME = "hive-name";
+    private static final String NAME_NODE = "${nameNode}";
+    private static final String EXAMPLE_DIR = "/path/to/directory";
+    private static final String[] ARGS = {"arg1", "arg2", "arg3"};
+    private static final String MAPRED_JOB_QUEUE_NAME = "mapred.job.queue.name";
+    private static final String DEFAULT = "default";
+    private static final String RESOURCE_MANAGER = "${resourceManager}";
+    private static final String PATH_TO_DELETE = "/path/to/delete";
+    private static final String PATH_TO_MKDIR = "/path/to/mkdir";
+
+    @Override
+    protected HiveActionBuilder getBuilderInstance() {
+        return HiveActionBuilder.create();
+    }
+
+    @Override
+    protected HiveActionBuilder getBuilderInstance(final HiveAction action) {
+        return HiveActionBuilder.createFromExistingAction(action);
+    }
+
+    @Test
+    public void testResourceManagerAdded() {
+        final HiveActionBuilder builder = getBuilderInstance();
+        builder.withResourceManager(RESOURCE_MANAGER);
+
+        final HiveAction action = builder.build();
+        assertEquals(RESOURCE_MANAGER, action.getResourceManager());
+    }
+
+    @Test
+    public void testNameNodeAdded() {
+        final HiveActionBuilder builder = getBuilderInstance();
+        builder.withNameNode(NAME_NODE);
+
+        final HiveAction action = builder.build();
+        assertEquals(NAME_NODE, action.getNameNode());
+    }
+
+    @Test
+    public void testPrepareAdded() {
+        final HiveActionBuilder builder = getBuilderInstance();
+        builder.withPrepare(new PrepareBuilder().withDelete(EXAMPLE_DIR).build());
+
+        final HiveAction action = builder.build();
+        assertEquals(EXAMPLE_DIR, action.getPrepare().getDeletes().get(0).getPath());
+    }
+
+    @Test
+    public void testSameConfigPropertyAddedTwiceThrows() {
+        final HiveActionBuilder builder = getBuilderInstance();
+        builder.withConfigProperty(MAPRED_JOB_QUEUE_NAME, DEFAULT);
+
+        expectedException.expect(IllegalStateException.class);
+        builder.withConfigProperty(MAPRED_JOB_QUEUE_NAME, DEFAULT);
+    }
+
+    @Test
+    public void testSeveralArgsAdded() {
+        final HiveActionBuilder builder = getBuilderInstance();
+
+        for (final String arg : ARGS) {
+            builder.withArg(arg);
+        }
+
+        final HiveAction action = builder.build();
+
+        final List<String> argList = action.getArgs();
+        assertEquals(ARGS.length, argList.size());
+
+        for (int i = 0; i < ARGS.length; ++i) {
+            assertEquals(ARGS[i], argList.get(i));
+        }
+    }
+
+    @Test
+    public void testRemoveArgs() {
+        final HiveActionBuilder builder = getBuilderInstance();
+
+        for (final String file : ARGS) {
+            builder.withArg(file);
+        }
+
+        builder.withoutArg(ARGS[0]);
+
+        final HiveAction action = builder.build();
+
+        final List<String> argList = action.getArgs();
+        final String[] remainingArgs = Arrays.copyOfRange(ARGS, 1, ARGS.length);
+        assertEquals(remainingArgs.length, argList.size());
+
+        for (int i = 0; i < remainingArgs.length; ++i) {
+            assertEquals(remainingArgs[i], argList.get(i));
+        }
+    }
+
+    @Test
+    public void testClearArgs() {
+        final HiveActionBuilder builder = getBuilderInstance();
+
+        for (final String file : ARGS) {
+            builder.withArg(file);
+        }
+
+        builder.clearArgs();
+
+        final HiveAction action = builder.build();
+
+        final List<String> argList = action.getArgs();
+        assertEquals(0, argList.size());
+    }
+
+    @Test
+    public void testFromExistingHiveAction() {
+        final HiveActionBuilder builder = getBuilderInstance();
+
+        builder.withName(NAME)
+                .withResourceManager(RESOURCE_MANAGER)
+                .withNameNode(NAME_NODE)
+                .withConfigProperty(MAPRED_JOB_QUEUE_NAME, DEFAULT)
+                .withPrepare(new PrepareBuilder()
+                        .withDelete(PATH_TO_DELETE)
+                        .withMkdir(PATH_TO_MKDIR)
+                        .build())
+                .withLauncher(new LauncherBuilder()
+                        .withMemoryMb(1024L)
+                        .withVCores(2L)
+                        .withQueue(DEFAULT)
+                        .withSharelib(DEFAULT)
+                        .withViewAcl(DEFAULT)
+                        .withModifyAcl(DEFAULT)
+                        .build())
+                .withArg(ARGS[0])
+                .withArg(ARGS[1])
+                .withQuery(DEFAULT)
+                .withArchive(DEFAULT)
+                .withFile(DEFAULT);
+
+        final HiveAction action = builder.build();
+
+        final HiveActionBuilder fromExistingBuilder = getBuilderInstance(action);
+
+        final String newName = "fromExisting_" + NAME;
+        fromExistingBuilder.withName(newName)
+                .withoutArg(ARGS[1])
+                .withArg(ARGS[2]);
+
+        final HiveAction modifiedAction = fromExistingBuilder.build();
+
+        assertEquals(newName, modifiedAction.getName());
+        assertEquals(action.getNameNode(), modifiedAction.getNameNode());
+
+        final Map<String, String> expectedConfiguration = new LinkedHashMap<>();
+        expectedConfiguration.put(MAPRED_JOB_QUEUE_NAME, DEFAULT);
+        assertEquals(expectedConfiguration, modifiedAction.getConfiguration());
+
+        assertEquals(Arrays.asList(ARGS[0], ARGS[2]), modifiedAction.getArgs());
+
+        assertEquals(PATH_TO_DELETE, modifiedAction.getPrepare().getDeletes().get(0).getPath());
+        assertEquals(PATH_TO_MKDIR, modifiedAction.getPrepare().getMkdirs().get(0).getPath());
+
+        assertEquals(1024L, modifiedAction.getLauncher().getMemoryMb());
+        assertEquals(2L, modifiedAction.getLauncher().getVCores());
+        assertEquals(DEFAULT, modifiedAction.getLauncher().getQueue());
+        assertEquals(DEFAULT, modifiedAction.getLauncher().getSharelib());
+        assertEquals(DEFAULT, modifiedAction.getLauncher().getViewAcl());
+        assertEquals(DEFAULT, modifiedAction.getLauncher().getModifyAcl());
+
+        assertEquals(action.getScript(), modifiedAction.getScript());
+        assertEquals(action.getQuery(), modifiedAction.getQuery());
+    }
+
+    @Test
+    public void testFromOtherAction() {
+        final ShellAction parent = ShellActionBuilder.create()
+                .withName("parent")
+                .build();
+
+        final ShellAction otherAction = ShellActionBuilder.createFromExistingAction(parent)
+                .withName("shell")
+                .withParent(parent)
+                .build();
+
+        final HiveAction fromOtherAction = HiveActionBuilder.createFromExistingAction(otherAction)
+                .withName("hive")
+                .build();
+
+        assertEquals("hive", fromOtherAction.getName());
+        assertEquals(parent, fromOtherAction.getParentsWithoutConditions().get(0));
+    }
+}
\ No newline at end of file


[02/16] oozie git commit: OOZIE-2339 [fluent-job] Minimum Viable Fluent Job API (daniel.becker, andras.piros via rkanter, gezapeti, pbacsko)

Posted by an...@apache.org.
http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestSubWorkflowActionMapping.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestSubWorkflowActionMapping.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestSubWorkflowActionMapping.java
new file mode 100644
index 0000000..10f673f
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestSubWorkflowActionMapping.java
@@ -0,0 +1,47 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.mapping;
+
+import org.apache.oozie.fluentjob.api.generated.workflow.SUBWORKFLOW;
+import org.apache.oozie.fluentjob.api.action.SubWorkflowAction;
+import org.apache.oozie.fluentjob.api.action.SubWorkflowActionBuilder;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+public class TestSubWorkflowActionMapping {
+
+    @Test
+    public void testMappingSubWorkflowAction() {
+        final String appPath = "/path/to/app";
+
+        final SubWorkflowAction action = SubWorkflowActionBuilder.create()
+                .withAppPath(appPath)
+                .withPropagatingConfiguration()
+                .withConfigProperty("propertyName", "propertyValue")
+                .build();
+
+        final SUBWORKFLOW subWorkflowAction = DozerBeanMapperSingleton.instance().map(action, SUBWORKFLOW.class);
+
+        assertEquals(appPath, subWorkflowAction.getAppPath());
+        assertNotNull(subWorkflowAction.getPropagateConfiguration());
+        assertNotNull(subWorkflowAction.getConfiguration());
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestWorkflowAttributesMapping.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestWorkflowAttributesMapping.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestWorkflowAttributesMapping.java
new file mode 100644
index 0000000..24cbb64
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestWorkflowAttributesMapping.java
@@ -0,0 +1,53 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.mapping;
+
+import org.apache.oozie.fluentjob.api.dag.Graph;
+import org.apache.oozie.fluentjob.api.generated.workflow.WORKFLOWAPP;
+import org.apache.oozie.fluentjob.api.workflow.Workflow;
+import org.apache.oozie.fluentjob.api.workflow.WorkflowBuilder;
+import org.junit.Test;
+
+import static org.junit.Assert.assertNull;
+
+public class TestWorkflowAttributesMapping {
+
+    private final SourceDataFactory factory = new SourceDataFactory();
+
+    @Test
+    public void testMappingNoCredentialsToWorkflow() {
+        final Workflow source = new WorkflowBuilder()
+                .build();
+
+        final WORKFLOWAPP destination = DozerBeanMapperSingleton.instance().map(new Graph(source), WORKFLOWAPP.class);
+
+        assertNull(destination.getCredentials());
+    }
+
+    @Test
+    public void testMappingCredentialsAsWorkflowParameters() {
+        final Workflow source = new WorkflowBuilder()
+                .withCredentials(factory.createCredentials())
+                .build();
+
+        final WORKFLOWAPP destination = DozerBeanMapperSingleton.instance().map(new Graph(source), WORKFLOWAPP.class);
+
+        factory.assertCredentials(destination.getCredentials());
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/workflow/TestCredentialBuilder.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/workflow/TestCredentialBuilder.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/workflow/TestCredentialBuilder.java
new file mode 100644
index 0000000..06c5b3d
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/workflow/TestCredentialBuilder.java
@@ -0,0 +1,56 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.workflow;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestCredentialBuilder {
+
+    @Test
+    public void testCreate() {
+        final Credential credential = CredentialBuilder.create()
+                .withName("hive2")
+                .withType("hive")
+                .withConfigurationEntry("jdbcUrl", "jdbc://localhost/hive")
+                .build();
+
+        assertEquals("hive2", credential.getName());
+        assertEquals("hive", credential.getType());
+        assertEquals("jdbcUrl", credential.getConfigurationEntries().get(0).getName());
+        assertEquals("jdbc://localhost/hive", credential.getConfigurationEntries().get(0).getValue());
+    }
+
+    @Test
+    public void testCreateFromExisting() {
+        final Credential credential = CredentialBuilder.create()
+                .withName("hive2")
+                .withType("hive")
+                .withConfigurationEntry("jdbcUrl", "jdbc://localhost/hive")
+                .build();
+
+        final Credential fromExisting = CredentialBuilder.createFromExisting(credential).build();
+
+        assertEquals("hive2", fromExisting.getName());
+        assertEquals("hive", fromExisting.getType());
+        assertEquals("jdbcUrl", fromExisting.getConfigurationEntries().get(0).getName());
+        assertEquals("jdbc://localhost/hive", fromExisting.getConfigurationEntries().get(0).getValue());
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/workflow/TestCredentialsBuilder.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/workflow/TestCredentialsBuilder.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/workflow/TestCredentialsBuilder.java
new file mode 100644
index 0000000..4d3dce3
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/workflow/TestCredentialsBuilder.java
@@ -0,0 +1,66 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.workflow;
+
+import com.google.common.collect.Lists;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestCredentialsBuilder {
+
+    @Test
+    public void testCreate() {
+        final Credentials credentials = CredentialsBuilder.create()
+                .withCredential("hive2",
+                        "hive",
+                        Lists.newArrayList(
+                                new ConfigurationEntry("jdbcUrl", "jdbc://localhost/hive")))
+                .build();
+
+        assertEquals("hive2", credentials.getCredentials().get(0).getName());
+        assertEquals("hive", credentials.getCredentials().get(0).getType());
+        assertEquals("jdbcUrl", credentials.getCredentials().get(0).getConfigurationEntries().get(0).getName());
+        assertEquals("jdbc://localhost/hive", credentials.getCredentials().get(0).getConfigurationEntries().get(0).getValue());
+    }
+
+    @Test
+    public void testCreateFromExisting() {
+        final Credentials credentials = CredentialsBuilder.create()
+                .withCredential("hive2",
+                        "hive",
+                        Lists.newArrayList(
+                                new ConfigurationEntry("jdbcUrl", "jdbc://localhost/hive")))
+                .build();
+
+        final Credentials fromExisting = CredentialsBuilder.createFromExisting(credentials)
+                .withCredential("hbase",
+                        "hbase")
+                .build();
+
+        assertEquals("hive2", fromExisting.getCredentials().get(0).getName());
+        assertEquals("hive", fromExisting.getCredentials().get(0).getType());
+        assertEquals("jdbcUrl", fromExisting.getCredentials().get(0).getConfigurationEntries().get(0).getName());
+        assertEquals("jdbc://localhost/hive", fromExisting.getCredentials().get(0).getConfigurationEntries().get(0).getValue());
+
+        assertEquals("hbase", fromExisting.getCredentials().get(1).getName());
+        assertEquals("hbase", fromExisting.getCredentials().get(1).getType());
+        assertEquals(0, fromExisting.getCredentials().get(1).getConfigurationEntries().size());
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/workflow/TestGlobalBuilder.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/workflow/TestGlobalBuilder.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/workflow/TestGlobalBuilder.java
new file mode 100644
index 0000000..c417170
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/workflow/TestGlobalBuilder.java
@@ -0,0 +1,63 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.workflow;
+
+import org.apache.oozie.fluentjob.api.action.LauncherBuilder;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
+public class TestGlobalBuilder {
+
+    public static final String DEFAULT = "default";
+
+    @Test
+    public void testAfterCopyFieldsAreSetCorrectly() {
+        final Global original = GlobalBuilder.create()
+                .withResourceManager(DEFAULT)
+                .withNameNode(DEFAULT)
+                .withJobXml(DEFAULT)
+                .withConfigProperty("key1", "value1")
+                .withLauncher(new LauncherBuilder()
+                        .withMemoryMb(1024L)
+                        .withVCores(1L)
+                        .build())
+                .build();
+
+        assertEquals(DEFAULT, original.getResourceManager());
+        assertEquals(DEFAULT, original.getNameNode());
+        assertEquals(DEFAULT, original.getJobXmls().get(0));
+        assertEquals("value1", original.getConfigProperty("key1"));
+        assertEquals(1024L, original.getLauncher().getMemoryMb());
+        assertEquals(1L, original.getLauncher().getVCores());
+
+        final Global copied = GlobalBuilder.createFromExisting(original)
+                .withoutJobXml(DEFAULT)
+                .withConfigProperty("key1", null)
+                .build();
+
+        assertEquals(DEFAULT, copied.getResourceManager());
+        assertEquals(DEFAULT, copied.getNameNode());
+        assertEquals(0, copied.getJobXmls().size());
+        assertNull(copied.getConfigProperty("key1"));
+        assertEquals(1024L, copied.getLauncher().getMemoryMb());
+        assertEquals(1L, copied.getLauncher().getVCores());
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/workflow/TestParametersBuilder.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/workflow/TestParametersBuilder.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/workflow/TestParametersBuilder.java
new file mode 100644
index 0000000..f9405e6
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/workflow/TestParametersBuilder.java
@@ -0,0 +1,80 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.workflow;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
+public class TestParametersBuilder {
+    private ParametersBuilder builder;
+
+    @Before
+    public void setUp() {
+        this.builder = ParametersBuilder.create();
+    }
+
+    @Test
+    public void testWithoutDescription() {
+        final Parameters parameters = builder
+                .withParameter("name1", "value1")
+                .withParameter("name2", "value2")
+                .build();
+
+        assertEquals("name1", parameters.getParameters().get(0).getName());
+        assertEquals("value1", parameters.getParameters().get(0).getValue());
+        assertNull(parameters.getParameters().get(0).getDescription());
+        assertEquals("name2", parameters.getParameters().get(1).getName());
+        assertEquals("value2", parameters.getParameters().get(1).getValue());
+        assertNull(parameters.getParameters().get(1).getDescription());
+    }
+
+    @Test
+    public void testWithDescription() {
+        final Parameters parameters = builder
+                .withParameter("name1", "value1", "description1")
+                .withParameter("name2", "value2", "description2")
+                .build();
+
+        assertEquals("name1", parameters.getParameters().get(0).getName());
+        assertEquals("value1", parameters.getParameters().get(0).getValue());
+        assertEquals("description1", parameters.getParameters().get(0).getDescription());
+        assertEquals("name2", parameters.getParameters().get(1).getName());
+        assertEquals("value2", parameters.getParameters().get(1).getValue());
+        assertEquals("description2", parameters.getParameters().get(1).getDescription());
+    }
+
+    @Test
+    public void testCreateFromExisting() {
+        final Parameters existing = builder
+                .withParameter("name1", "value1")
+                .withParameter("name2", "value2")
+                .build();
+
+        final Parameters fromExisting = ParametersBuilder.createFromExisting(existing)
+                .withParameter("name3", "value3")
+                .build();
+
+        assertEquals("value1", fromExisting.getParameters().get(0).getValue());
+        assertEquals("value2", fromExisting.getParameters().get(1).getValue());
+        assertEquals("value3", fromExisting.getParameters().get(2).getValue());
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/workflow/TestWorkflowBuilder.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/workflow/TestWorkflowBuilder.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/workflow/TestWorkflowBuilder.java
new file mode 100644
index 0000000..29caf27
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/workflow/TestWorkflowBuilder.java
@@ -0,0 +1,258 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.workflow;
+
+import com.google.common.collect.Lists;
+import org.apache.oozie.fluentjob.api.action.MapReduceActionBuilder;
+import org.apache.oozie.fluentjob.api.action.Node;
+import org.apache.oozie.fluentjob.api.action.ShellAction;
+import org.apache.oozie.fluentjob.api.action.ShellActionBuilder;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Set;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
+public class TestWorkflowBuilder {
+    private static final String NAME = "workflow-name";
+
+    @Rule
+    public final ExpectedException expectedException = ExpectedException.none();
+
+    @Test
+    public void testAddName() {
+        final WorkflowBuilder builder = new WorkflowBuilder();
+        builder.withName(NAME);
+
+        final Workflow workflow = builder.build();
+
+        assertEquals(NAME, workflow.getName());
+    }
+
+    @Test
+    public void testAddDagTrivial() {
+        final Node mrAction1 = MapReduceActionBuilder.create()
+                .withName("mr1")
+                .withNameNode("${nameNode}")
+                .withResourceManager("${resourceManager}")
+                .withConfigProperty("mapred.output.dir", "${outputDir}")
+                .build();
+
+        final Node mrAction2 = MapReduceActionBuilder.create()
+                .withName("mr2")
+                .withNameNode("${nameNode}")
+                .withResourceManager("${resourceManager}")
+                .withConfigProperty("mapred.output.dir", "${outputDir}")
+                .build();
+
+        final WorkflowBuilder builder = new WorkflowBuilder();
+
+        builder.withDagContainingNode(mrAction1)
+                .withDagContainingNode(mrAction2);
+
+        final Workflow workflow = builder.build();
+
+        final Set<Node> expectedRoots = new HashSet<>(Arrays.asList(mrAction1, mrAction2));
+        assertEquals(expectedRoots, workflow.getRoots());
+
+        final Set<Node> expectedNodes = new HashSet<>(Arrays.asList(mrAction1, mrAction2));
+        assertEquals(expectedNodes, workflow.getNodes());
+    }
+
+    @Test
+    public void testAddDagFindRoots() {
+        final Node mrAction1 = MapReduceActionBuilder.create()
+                .withName("mr1")
+                .build();
+
+        final Node mrAction2 = MapReduceActionBuilder.create()
+                .withName("mr2")
+                .build();
+
+        final Node mrAction3 = MapReduceActionBuilder.create()
+                .withName("mr3")
+                .withParent(mrAction1)
+                .withParent(mrAction2)
+                .build();
+
+        final WorkflowBuilder builder = new WorkflowBuilder();
+
+        builder.withDagContainingNode(mrAction3);
+
+        final Workflow workflow = builder.build();
+
+        final Set<Node> expectedRoots = new HashSet<>(Arrays.asList(mrAction1, mrAction2));
+        assertEquals(expectedRoots, workflow.getRoots());
+
+        final Set<Node> expectedNodes = new HashSet<>(Arrays.asList(mrAction1, mrAction2, mrAction3));
+        assertEquals(expectedNodes, workflow.getNodes());
+    }
+
+    @Test
+    public void testAddDagThrowOnDuplicateNodeNames() {
+        final Node mrAction = MapReduceActionBuilder.create()
+                .withName("mr-action")
+                .build();
+
+        final Node mrActionWithTheSameName = MapReduceActionBuilder.create()
+                .withName("mr-action")
+                .build();
+
+        final WorkflowBuilder builder = new WorkflowBuilder();
+        builder.withName(NAME)
+                .withDagContainingNode(mrAction)
+                .withDagContainingNode(mrActionWithTheSameName);
+
+        expectedException.expect(IllegalArgumentException.class);
+        builder.build();
+    }
+
+    @Test
+    public void testAddDagWithConditionalChildrenAndConditionalParents() {
+        final String condition = "condition";
+
+        final Node mrAction1 = MapReduceActionBuilder.create()
+                .withName("mr1")
+                .build();
+
+        final Node mrAction2 = MapReduceActionBuilder.create()
+                .withName("mr2")
+                .build();
+
+        final Node mrAction3 = MapReduceActionBuilder.create()
+                .withName("mr3")
+                .withParentWithCondition(mrAction1, condition)
+                .withParent(mrAction2)
+                .build();
+        final Node mrAction4 = MapReduceActionBuilder.create()
+                .withName("mr4")
+                .withParentWithCondition(mrAction3, condition)
+                .build();
+        final Node mrAction5 = MapReduceActionBuilder.create()
+                .withName("mr5")
+                .withParentWithCondition(mrAction3, condition)
+                .build();
+
+        final WorkflowBuilder builder = new WorkflowBuilder();
+
+        builder.withDagContainingNode(mrAction3);
+
+        final Workflow workflow = builder.build();
+
+        final Set<Node> expectedRoots = new HashSet<>(Arrays.asList(mrAction1, mrAction2));
+        assertEquals(expectedRoots, workflow.getRoots());
+
+        final Set<Node> expectedNodes = new HashSet<>(Arrays.asList(mrAction1, mrAction2, mrAction3, mrAction4, mrAction5));
+        assertEquals(expectedNodes, workflow.getNodes());
+    }
+
+    @Test
+    public void testAddMixedParameters() {
+        final Workflow workflow = new WorkflowBuilder()
+                .withParameter("name1", "value1")
+                .withParameter("name2", "value2", "description2")
+                .build();
+
+        assertEquals("name1", workflow.getParameters().getParameters().get(0).getName());
+        assertEquals("value1", workflow.getParameters().getParameters().get(0).getValue());
+        assertNull(workflow.getParameters().getParameters().get(0).getDescription());
+        assertEquals("name2", workflow.getParameters().getParameters().get(1).getName());
+        assertEquals("value2", workflow.getParameters().getParameters().get(1).getValue());
+        assertEquals("description2", workflow.getParameters().getParameters().get(1).getDescription());
+    }
+
+    @Test
+    public void testAddGlobal() {
+        final Workflow workflow = new WorkflowBuilder()
+                .withGlobal(GlobalBuilder.create()
+                        .withConfigProperty("key1", "value1")
+                        .build())
+                .build();
+
+        assertEquals("value1", workflow.getGlobal().getConfigProperty("key1"));
+    }
+
+    @Test
+    public void testAddCredentials() {
+        final Workflow workflow = new WorkflowBuilder()
+                .withCredentials(CredentialsBuilder.create()
+                        .withCredential("hbase", "hbase")
+                        .build())
+                .build();
+
+        assertEquals("hbase", workflow.getCredentials().getCredentials().get(0).getName());
+        assertEquals("hbase", workflow.getCredentials().getCredentials().get(0).getType());
+    }
+
+    @Test
+    public void testOmittedCredentialsAreAutogenerated() {
+        final Credential hbaseCredential = CredentialBuilder.create()
+                .withName("hbase")
+                .withType("hbase")
+                .build();
+
+        final ShellAction hbaseAction = ShellActionBuilder.create()
+                .withName("hbase")
+                .withCredential(hbaseCredential)
+                .build();
+
+        final Workflow workflow = new WorkflowBuilder()
+                .withDagContainingNode(hbaseAction)
+                .build();
+
+        assertEquals(1, workflow.getCredentials().getCredentials().size());
+        assertEquals("hbase", workflow.getCredentials().getCredentials().get(0).getName());
+        assertEquals("hbase", workflow.getCredentials().getCredentials().get(0).getType());
+    }
+
+    @Test
+    public void testOverrideCredentialsPreviouslyAddedInActionsOnWorkflowLevel() {
+        final Credential hbaseCredential = CredentialBuilder.create()
+                .withName("hbase")
+                .withType("hbase")
+                .build();
+
+        final ShellAction hbaseAction = ShellActionBuilder.create()
+                .withName("hbase")
+                .withCredential(hbaseCredential)
+                .build();
+
+        final Workflow workflow = new WorkflowBuilder()
+                .withDagContainingNode(hbaseAction)
+                .withCredentials(CredentialsBuilder.create()
+                        .withCredential("hive2", "hive2", Lists.newArrayList(
+                                new ConfigurationEntry("jdbcUrl", "jdbc://localhost/hive")))
+                        .build())
+                .build();
+
+        assertEquals(1, workflow.getCredentials().getCredentials().size());
+        assertEquals("hive2", workflow.getCredentials().getCredentials().get(0).getName());
+        assertEquals("hive2", workflow.getCredentials().getCredentials().get(0).getType());
+        assertEquals(1, workflow.getCredentials().getCredentials().get(0).getConfigurationEntries().size());
+        assertEquals("jdbcUrl",
+                workflow.getCredentials().getCredentials().get(0).getConfigurationEntries().get(0).getName());
+        assertEquals("jdbc://localhost/hive",
+                workflow.getCredentials().getCredentials().get(0).getConfigurationEntries().get(0).getValue());
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-client/pom.xml
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-client/pom.xml b/fluent-job/fluent-job-client/pom.xml
new file mode 100644
index 0000000..612fabb
--- /dev/null
+++ b/fluent-job/fluent-job-client/pom.xml
@@ -0,0 +1,99 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you under the Apache License, Version 2.0 (the
+  "License"); you may not use this file except in compliance
+  with the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <parent>
+        <artifactId>oozie-fluent-job</artifactId>
+        <groupId>org.apache.oozie</groupId>
+        <version>5.1.0-SNAPSHOT</version>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+
+    <artifactId>oozie-fluent-job-client</artifactId>
+    <version>5.1.0-SNAPSHOT</version>
+    <description>Apache Oozie Fluent Job Client</description>
+    <name>Apache Oozie Fluent Job Client</name>
+    <packaging>jar</packaging>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.oozie</groupId>
+            <artifactId>oozie-core</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.oozie</groupId>
+            <artifactId>oozie-core</artifactId>
+            <type>test-jar</type>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.derby</groupId>
+            <artifactId>derby</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.oozie</groupId>
+            <artifactId>oozie-sharelib-hive</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-client</artifactId>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-minicluster</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.oozie.test</groupId>
+            <artifactId>oozie-mini</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.oozie.test</groupId>
+            <artifactId>oozie-mini</artifactId>
+            <classifier>tests</classifier>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.oozie</groupId>
+            <artifactId>oozie-client</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.mockito</groupId>
+            <artifactId>mockito-all</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.xmlunit</groupId>
+            <artifactId>xmlunit-core</artifactId>
+            <scope>test</scope>
+        </dependency>
+    </dependencies>
+</project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-client/src/test/java/org/apache/oozie/jobs/client/jaxb/TestJAXBWorkflow.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-client/src/test/java/org/apache/oozie/jobs/client/jaxb/TestJAXBWorkflow.java b/fluent-job/fluent-job-client/src/test/java/org/apache/oozie/jobs/client/jaxb/TestJAXBWorkflow.java
new file mode 100644
index 0000000..47adb11
--- /dev/null
+++ b/fluent-job/fluent-job-client/src/test/java/org/apache/oozie/jobs/client/jaxb/TestJAXBWorkflow.java
@@ -0,0 +1,435 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.jobs.client.jaxb;
+
+import org.apache.oozie.fluentjob.api.generated.workflow.ACTION;
+import org.apache.oozie.fluentjob.api.generated.workflow.ACTIONTRANSITION;
+import org.apache.oozie.fluentjob.api.generated.workflow.CONFIGURATION;
+import org.apache.oozie.fluentjob.api.generated.workflow.DELETE;
+import org.apache.oozie.fluentjob.api.generated.workflow.END;
+import org.apache.oozie.fluentjob.api.generated.workflow.KILL;
+import org.apache.oozie.fluentjob.api.generated.workflow.MAPREDUCE;
+import org.apache.oozie.fluentjob.api.generated.workflow.ObjectFactory;
+import org.apache.oozie.fluentjob.api.generated.workflow.PREPARE;
+import org.apache.oozie.fluentjob.api.generated.workflow.START;
+import org.apache.oozie.fluentjob.api.generated.workflow.WORKFLOWAPP;
+
+import org.junit.Test;
+import org.xml.sax.SAXException;
+import org.xmlunit.builder.DiffBuilder;
+import org.xmlunit.builder.Input;
+import org.xmlunit.diff.Comparison;
+import org.xmlunit.diff.ComparisonResult;
+import org.xmlunit.diff.ComparisonType;
+import org.xmlunit.diff.Diff;
+import org.xmlunit.diff.DifferenceEvaluator;
+import org.xmlunit.diff.DifferenceEvaluators;
+
+import javax.xml.XMLConstants;
+import javax.xml.bind.JAXBContext;
+import javax.xml.bind.JAXBElement;
+import javax.xml.bind.JAXBException;
+import javax.xml.bind.Marshaller;
+import javax.xml.bind.Unmarshaller;
+import javax.xml.parsers.ParserConfigurationException;
+import javax.xml.transform.Source;
+import javax.xml.transform.stream.StreamSource;
+import javax.xml.validation.Schema;
+import javax.xml.validation.SchemaFactory;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+import java.net.URISyntaxException;
+import java.net.URL;
+import java.nio.charset.Charset;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
+/**
+ * This class tests whether the workflow.xml files are parsed correctly into JAXB objects and whether the JAXB objects
+ * are serialized correctly to xml.
+ */
+public class TestJAXBWorkflow {
+    private static final String GENERATED_PACKAGES_ALL = "org.apache.oozie.fluentjob.api.generated.action.distcp:" +
+            "org.apache.oozie.fluentjob.api.generated.action.email:" +
+            "org.apache.oozie.fluentjob.api.generated.action.hive2:" +
+            "org.apache.oozie.fluentjob.api.generated.action.hive:" +
+            "org.apache.oozie.fluentjob.api.generated.sla:" +
+            "org.apache.oozie.fluentjob.api.generated.workflow:" +
+            "org.apache.oozie.fluentjob.api.generated.action.shell:" +
+            "org.apache.oozie.fluentjob.api.generated.action.spark:" +
+            "org.apache.oozie.fluentjob.api.generated.action.sqoop:" +
+            "org.apache.oozie.fluentjob.api.generated.action.ssh";
+    private static final String GENERATED_PACKAGES_WORKFLOW = "org.apache.oozie.fluentjob.api.generated.workflow";
+    private static final String WORKFLOW_MAPREDUCE_ACTION = "/workflow-mapreduce-action.xml";
+    private static final String WORKFLOW_ALL_ACTIONS = "/workflow-all-actions.xml";
+
+    /**
+     * Tests whether a workflow.xml object is parsed correctly into a JAXB element tree by checking some of the main
+     * properties.
+     * @throws SAXException If a SAX error occurs during parsing the schema file.
+     * @throws JAXBException If an error was encountered while creating the <tt>JAXBContext</tt> or the
+     *         <tt>Unmarshaller</tt> objects.
+     */
+    @Test
+    public void whenWorkflowXmlWithAllActionTypesIsUnmarshalledAttributesArePreserved()
+            throws SAXException, JAXBException, URISyntaxException {
+        final WORKFLOWAPP wf = unmarshalWorkflowWithAllActionTypes();
+
+        assertEquals("jaxb-example-wf", wf.getName());
+        assertEquals("mr-node", wf.getStart().getTo());
+        assertEquals("end", wf.getEnd().getName());
+
+        final List<Object> actions = wf.getDecisionOrForkOrJoin();
+
+        final KILL kill = (KILL) actions.get(9);
+        assertKill(kill);
+
+        final MAPREDUCE mr = ((ACTION) actions.get(0)).getMapReduce();
+        assertMapReduce(mr);
+
+        final org.apache.oozie.fluentjob.api.generated.action.distcp.ACTION distcp =
+                (org.apache.oozie.fluentjob.api.generated.action.distcp.ACTION)
+                        ((JAXBElement<?>) ((ACTION) actions.get(1)).getOther()).getValue();
+        assertDistcp(distcp);
+
+        final org.apache.oozie.fluentjob.api.generated.action.email.ACTION email =
+                (org.apache.oozie.fluentjob.api.generated.action.email.ACTION)
+                        ((JAXBElement<?>) ((ACTION) actions.get(2)).getOther()).getValue();
+        assertEmail(email);
+
+        final org.apache.oozie.fluentjob.api.generated.action.hive2.ACTION hive2 =
+                (org.apache.oozie.fluentjob.api.generated.action.hive2.ACTION)
+                        ((JAXBElement<?>) ((ACTION) actions.get(3)).getOther()).getValue();
+        assertHive2(hive2);
+
+        final org.apache.oozie.fluentjob.api.generated.action.hive.ACTION hive =
+                (org.apache.oozie.fluentjob.api.generated.action.hive.ACTION)
+                        ((JAXBElement<?>) ((ACTION) actions.get(4)).getOther()).getValue();
+        assertHive(hive);
+
+        final org.apache.oozie.fluentjob.api.generated.action.shell.ACTION shell =
+                (org.apache.oozie.fluentjob.api.generated.action.shell.ACTION)
+                        ((JAXBElement<?>) ((ACTION) actions.get(5)).getOther()).getValue();
+        assertShell(shell);
+
+        final org.apache.oozie.fluentjob.api.generated.action.spark.ACTION spark =
+                (org.apache.oozie.fluentjob.api.generated.action.spark.ACTION)
+                        ((JAXBElement<?>) ((ACTION) actions.get(6)).getOther()).getValue();
+        assertSpark(spark);
+
+        final org.apache.oozie.fluentjob.api.generated.action.sqoop.ACTION sqoop =
+                (org.apache.oozie.fluentjob.api.generated.action.sqoop.ACTION)
+                        ((JAXBElement<?>) ((ACTION) actions.get(7)).getOther()).getValue();
+        assertSqoop(sqoop);
+
+        final org.apache.oozie.fluentjob.api.generated.action.ssh.ACTION ssh =
+                (org.apache.oozie.fluentjob.api.generated.action.ssh.ACTION)
+                        ((JAXBElement<?>) ((ACTION) actions.get(8)).getOther()).getValue();
+        assertSsh(ssh);
+    }
+
+    private void assertKill(final KILL kill) {
+        assertEquals("fail", kill.getName());
+        assertEquals("Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]", kill.getMessage());
+    }
+
+    private void assertMapReduce(final MAPREDUCE mr) {
+        final PREPARE prepare = mr.getPrepare();
+        assertEquals(0, prepare.getMkdir().size());
+
+        final List<DELETE> deleteList = prepare.getDelete();
+        assertEquals(1, deleteList.size());
+
+        final DELETE delete = deleteList.get(0);
+        assertEquals("${nameNode}/user/${wf:user()}/${examplesRoot}/output-data/${outputDir}", delete.getPath());
+
+        final CONFIGURATION conf = mr.getConfiguration();
+        final List<CONFIGURATION.Property> properties = conf.getProperty();
+
+        final CONFIGURATION.Property mapper = properties.get(1);
+        assertEquals("mapred.mapper.class", mapper.getName());
+        assertEquals("org.apache.oozie.example.SampleMapper", mapper.getValue());
+    }
+
+    private void assertDistcp(final org.apache.oozie.fluentjob.api.generated.action.distcp.ACTION distcp) {
+        assertEquals(1, distcp.getPrepare().getDelete().size());
+        assertEquals(1, distcp.getPrepare().getMkdir().size());
+        assertEquals(2, distcp.getConfiguration().getProperty().size());
+        assertEquals(2, distcp.getArg().size());
+    }
+
+    private void assertEmail(final org.apache.oozie.fluentjob.api.generated.action.email.ACTION email) {
+        assertEquals("foo@bar.com", email.getTo());
+        assertEquals("foo", email.getSubject());
+        assertEquals("bar", email.getBody());
+    }
+
+    private void assertHive2(final org.apache.oozie.fluentjob.api.generated.action.hive2.ACTION hive2) {
+        assertEquals(1, hive2.getPrepare().getDelete().size());
+        assertEquals(1, hive2.getConfiguration().getProperty().size());
+        assertEquals(2, hive2.getParam().size());
+    }
+
+    private void assertHive(final org.apache.oozie.fluentjob.api.generated.action.hive.ACTION hive) {
+        assertEquals(1, hive.getPrepare().getDelete().size());
+        assertEquals(1, hive.getConfiguration().getProperty().size());
+        assertEquals(2, hive.getParam().size());
+    }
+
+    private void assertShell(final org.apache.oozie.fluentjob.api.generated.action.shell.ACTION shell) {
+        assertEquals("echo", shell.getExec());
+        assertEquals(1, shell.getArgument().size());
+    }
+
+    private void assertSpark(final org.apache.oozie.fluentjob.api.generated.action.spark.ACTION spark) {
+        assertEquals(1, spark.getPrepare().getDelete().size());
+        assertEquals(1, spark.getConfiguration().getProperty().size());
+        assertEquals(2, spark.getArg().size());
+    }
+
+    private void assertSqoop(final org.apache.oozie.fluentjob.api.generated.action.sqoop.ACTION sqoop) {
+        assertEquals(1, sqoop.getPrepare().getDelete().size());
+        assertEquals(1, sqoop.getConfiguration().getProperty().size());
+    }
+
+    private void assertSsh(final org.apache.oozie.fluentjob.api.generated.action.ssh.ACTION ssh) {
+        assertEquals("foo@bar.com", ssh.getHost());
+        assertEquals("uploaddata", ssh.getCommand());
+        assertEquals(2, ssh.getArgs().size());
+    }
+
+    /**
+     * Tests whether a programmatically built JAXB element tree is serialized correctly to xml.
+     *
+     * @throws JAXBException If an error was encountered while creating the <tt>JAXBContext</tt>
+     *         or during the marshalling.
+     */
+    @Test
+    public void marshallingWorkflowProducesCorrectXml() throws JAXBException, URISyntaxException, IOException,
+                                                 ParserConfigurationException, SAXException {
+        final WORKFLOWAPP programmaticallyCreatedWfApp = getWfApp();
+        final String outputXml = marshalWorkflowApp(programmaticallyCreatedWfApp, GENERATED_PACKAGES_WORKFLOW);
+
+        final Diff diff = DiffBuilder.compare(Input.fromURL(getClass().getResource(WORKFLOW_MAPREDUCE_ACTION)))
+                .withTest(Input.fromString(outputXml))
+                .ignoreComments()
+                .withDifferenceEvaluator(DifferenceEvaluators.chain(
+                        DifferenceEvaluators.Default,
+                        new IgnoreWhitespaceInTextValueDifferenceEvaluator(),
+                        new IgnoreNamespacePrefixDifferenceEvaluator()))
+                .build();
+
+        assertFalse(diff.hasDifferences());
+    }
+
+    @Test
+    public void testMarshallingWorkflowWithAllActionTypesWorks() throws JAXBException, SAXException,
+            URISyntaxException, UnsupportedEncodingException {
+        final WORKFLOWAPP wf = unmarshalWorkflowWithAllActionTypes();
+        final String outputXml = marshalWorkflowApp(wf, GENERATED_PACKAGES_ALL);
+
+        final Diff diff = DiffBuilder.compare(Input.fromURL(getClass().getResource(WORKFLOW_ALL_ACTIONS)))
+                .withTest(Input.fromString(outputXml))
+                .ignoreComments()
+                .withDifferenceEvaluator(DifferenceEvaluators.chain(
+                        DifferenceEvaluators.Default,
+                        new IgnoreWhitespaceInTextValueDifferenceEvaluator(),
+                        new IgnoreNamespacePrefixDifferenceEvaluator()))
+                .build();
+
+        assertFalse("unmarshalled and marshalled workflow XMLs differ", diff.hasDifferences());
+    }
+
+    private static class IgnoreWhitespaceInTextValueDifferenceEvaluator implements DifferenceEvaluator {
+        @Override
+        public ComparisonResult evaluate(final Comparison comparison, final ComparisonResult comparisonResult) {
+            // We want to ignore whitespace differences in TEXT_VALUE comparisons but not anywhere else,
+            // for example not in attribute names.
+            if (isTextValueComparison(comparison) && expectedAndActualValueTrimmedAreEqual(comparison)) {
+                return ComparisonResult.EQUAL;
+            } else {
+                return comparisonResult;
+            }
+        }
+
+        private boolean isTextValueComparison(final Comparison comparison) {
+            return comparison.getType().equals(ComparisonType.TEXT_VALUE);
+        }
+
+        private boolean expectedAndActualValueTrimmedAreEqual(final Comparison comparison) {
+            final String expectedNodeValue = comparison.getControlDetails().getTarget().getNodeValue();
+            final String actualNodeValue = comparison.getTestDetails().getTarget().getNodeValue();
+
+            if (expectedNodeValue == null || actualNodeValue == null) {
+                return false;
+            }
+
+            return expectedNodeValue.trim().equals(actualNodeValue.trim());
+        }
+    }
+
+    private static class IgnoreNamespacePrefixDifferenceEvaluator implements DifferenceEvaluator {
+
+        @Override
+        public ComparisonResult evaluate(final Comparison comparison, final ComparisonResult comparisonResult) {
+            if (isElementNodeComparison(comparison)) {
+                return ComparisonResult.EQUAL;
+            }
+
+            return comparisonResult;
+        }
+
+        private boolean isElementNodeComparison(final Comparison comparison) {
+            return comparison.getType().equals(ComparisonType.NAMESPACE_PREFIX);
+        }
+    }
+
+    private WORKFLOWAPP unmarshalWorkflowWithAllActionTypes() throws SAXException, JAXBException, URISyntaxException {
+        final JAXBContext jc = JAXBContext.newInstance(GENERATED_PACKAGES_ALL);
+        final Unmarshaller u = jc.createUnmarshaller();
+        final Schema wfSchema = getSchema();
+        u.setSchema(wfSchema);
+
+        final URL wfUrl = getClass().getResource(WORKFLOW_ALL_ACTIONS);
+        final JAXBElement element = (JAXBElement) u.unmarshal(wfUrl);
+
+        return (WORKFLOWAPP) element.getValue();
+    }
+
+    private Schema getSchema() throws SAXException, URISyntaxException {
+        final SchemaFactory sf = SchemaFactory.newInstance(XMLConstants.W3C_XML_SCHEMA_NS_URI);
+        return sf.newSchema(new Source [] {
+                getStreamSource("/oozie-common-1.0.xsd"),
+                getStreamSource("/distcp-action-1.0.xsd"),
+                getStreamSource("/email-action-0.2.xsd"),
+                getStreamSource("/hive2-action-1.0.xsd"),
+                getStreamSource("/hive-action-1.0.xsd"),
+                getStreamSource("/oozie-sla-0.2.xsd"),
+                getStreamSource("/oozie-workflow-1.0.xsd"),
+                getStreamSource("/shell-action-1.0.xsd"),
+                getStreamSource("/spark-action-1.0.xsd"),
+                getStreamSource("/sqoop-action-1.0.xsd"),
+                getStreamSource("/ssh-action-0.2.xsd")
+        });
+    }
+
+    private Source getStreamSource(final String resourceURI) throws URISyntaxException {
+        return new StreamSource(getClass().getResource(resourceURI).toExternalForm());
+    }
+
+    private String marshalWorkflowApp(final WORKFLOWAPP wfApp, final String packages)
+            throws JAXBException, UnsupportedEncodingException {
+        final JAXBElement wfElement = new ObjectFactory().createWorkflowApp(wfApp);
+
+        final JAXBContext jc = JAXBContext.newInstance(packages);
+        final Marshaller m =  jc.createMarshaller();
+        m.setProperty(Marshaller.JAXB_FORMATTED_OUTPUT, Boolean.TRUE);
+        final ByteArrayOutputStream out = new ByteArrayOutputStream();
+        m.marshal(wfElement, out);
+
+        return out.toString(Charset.defaultCharset().toString());
+    }
+
+    private WORKFLOWAPP getWfApp() {
+        final START start = new START();
+        start.setTo("mr-node");
+
+        final KILL kill = new KILL();
+        kill.setName("fail");
+        kill.setMessage("Map/Reduce failed, error message[${wf:errorMessage(wf:lastErrorNode())}]");
+
+        final END end = new END();
+        end.setName("end");
+
+        final WORKFLOWAPP wfApp = new WORKFLOWAPP();
+        wfApp.setName("jaxb-example-wf");
+        wfApp.setStart(start);
+        wfApp.getDecisionOrForkOrJoin().add(getMapReduceAction());
+        wfApp.getDecisionOrForkOrJoin().add(kill);
+        wfApp.setEnd(end);
+
+        return wfApp;
+    }
+
+    private ACTION getMapReduceAction() {
+        final ACTION action = new ACTION();
+
+        action.setName("mr-node");
+        action.setMapReduce(getMapreduce());
+
+        final ACTIONTRANSITION okTransition = new ACTIONTRANSITION();
+        okTransition.setTo("end");
+        action.setOk(okTransition);
+
+        final ACTIONTRANSITION errorTransition = new ACTIONTRANSITION();
+        errorTransition.setTo("fail");
+        action.setError(errorTransition);
+
+        return action;
+    }
+
+    private MAPREDUCE getMapreduce() {
+        final MAPREDUCE mr = new MAPREDUCE();
+
+        mr.setResourceManager("${resourceManager}");
+        mr.setNameNode("${nameNode}");
+        mr.setPrepare(getPrepare());
+        mr.setConfiguration(getConfiguration());
+
+        return mr;
+    }
+
+    private CONFIGURATION getConfiguration() {
+        final String[][] nameValuePairs = {
+                {"mapred.job.queue.name", "${queueName}"},
+                {"mapred.mapper.class", "org.apache.oozie.example.SampleMapper"},
+                {"mapred.reducer.class", "org.apache.oozie.example.SampleReducer"},
+                {"mapred.map.tasks", "1"},
+                {"mapred.input.dir", "/user/${wf:user()}/${examplesRoot}/input-data/text"},
+                {"mapred.output.dir", "/user/${wf:user()}/${examplesRoot}/output-data/${outputDir}"}
+        };
+
+        final CONFIGURATION config = new CONFIGURATION();
+        final List<CONFIGURATION.Property> properties = config.getProperty();
+
+        for (final String[] pair : nameValuePairs) {
+            final CONFIGURATION.Property property = new CONFIGURATION.Property();
+            property.setName(pair[0]);
+            property.setValue(pair[1]);
+
+            properties.add(property);
+        }
+
+        return config;
+    }
+
+    private PREPARE getPrepare() {
+        final PREPARE prepare = new PREPARE();
+
+        final DELETE delete = new DELETE();
+        delete.setPath("${nameNode}/user/${wf:user()}/${examplesRoot}/output-data/${outputDir}");
+
+        prepare.getDelete().add(delete);
+
+        return prepare;
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-client/src/test/java/org/apache/oozie/jobs/client/minitest/TestDistcpAction.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-client/src/test/java/org/apache/oozie/jobs/client/minitest/TestDistcpAction.java b/fluent-job/fluent-job-client/src/test/java/org/apache/oozie/jobs/client/minitest/TestDistcpAction.java
new file mode 100644
index 0000000..e377d5c
--- /dev/null
+++ b/fluent-job/fluent-job-client/src/test/java/org/apache/oozie/jobs/client/minitest/TestDistcpAction.java
@@ -0,0 +1,89 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.jobs.client.minitest;
+
+import org.apache.oozie.client.OozieClientException;
+import org.apache.oozie.fluentjob.api.GraphVisualization;
+import org.apache.oozie.fluentjob.api.action.DistcpAction;
+import org.apache.oozie.fluentjob.api.action.DistcpActionBuilder;
+import org.apache.oozie.fluentjob.api.action.Prepare;
+import org.apache.oozie.fluentjob.api.action.PrepareBuilder;
+import org.apache.oozie.fluentjob.api.action.SshActionBuilder;
+import org.apache.oozie.fluentjob.api.dag.Graph;
+import org.apache.oozie.fluentjob.api.serialization.WorkflowMarshaller;
+import org.apache.oozie.fluentjob.api.workflow.Workflow;
+import org.apache.oozie.fluentjob.api.workflow.WorkflowBuilder;
+import org.apache.oozie.test.WorkflowTestCase;
+
+import javax.xml.bind.JAXBException;
+import java.io.IOException;
+
+public class TestDistcpAction extends WorkflowTestCase {
+    public void testForkedDistcpActions() throws IOException, JAXBException, OozieClientException {
+        final Prepare prepare = new PrepareBuilder()
+                .withDelete("hdfs://localhost:8020/user/${wf:user()}/examples/output")
+                .build();
+
+        final DistcpAction parent = DistcpActionBuilder.create()
+                .withResourceManager(getJobTrackerUri())
+                .withNameNode(getNameNodeUri())
+                .withPrepare(prepare)
+                .withConfigProperty("mapred.job.queue.name", "default")
+                .withJavaOpts("-Dopt1 -Dopt2")
+                .withArg("arg1")
+                .build();
+
+        //  We are reusing the definition of parent and only modifying and adding what is different.
+        final DistcpAction leftChild = DistcpActionBuilder.createFromExistingAction(parent)
+                .withParent(parent)
+                .withoutArg("arg1")
+                .withArg("arg2")
+                .build();
+
+        final DistcpAction rightChild = DistcpActionBuilder.createFromExistingAction(leftChild)
+                .withoutArg("arg2")
+                .withArg("arg3")
+                .build();
+
+        final Workflow workflow = new WorkflowBuilder()
+                .withName("simple-distcp-example")
+                .withDagContainingNode(parent).build();
+
+        SshActionBuilder.create()
+                .withParent(leftChild)
+                .withParent(rightChild)
+                .withHost("localhost")
+                .withCommand("pwd")
+                .build();
+
+        final String xml = WorkflowMarshaller.marshal(workflow);
+
+        System.out.println(xml);
+
+        GraphVisualization.workflowToPng(workflow, "simple-distcp-example-workflow.png");
+
+        final Graph intermediateGraph = new Graph(workflow);
+
+        GraphVisualization.graphToPng(intermediateGraph, "simple-distcp-example-graph.png");
+
+        log.debug("Workflow XML is:\n{0}", xml);
+
+        validate(xml);
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-client/src/test/java/org/apache/oozie/jobs/client/minitest/TestEmailAction.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-client/src/test/java/org/apache/oozie/jobs/client/minitest/TestEmailAction.java b/fluent-job/fluent-job-client/src/test/java/org/apache/oozie/jobs/client/minitest/TestEmailAction.java
new file mode 100644
index 0000000..d46cee2
--- /dev/null
+++ b/fluent-job/fluent-job-client/src/test/java/org/apache/oozie/jobs/client/minitest/TestEmailAction.java
@@ -0,0 +1,73 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.jobs.client.minitest;
+
+import org.apache.oozie.client.OozieClientException;
+import org.apache.oozie.fluentjob.api.GraphVisualization;
+import org.apache.oozie.fluentjob.api.action.EmailActionBuilder;
+import org.apache.oozie.fluentjob.api.action.Node;
+import org.apache.oozie.fluentjob.api.dag.Graph;
+import org.apache.oozie.fluentjob.api.serialization.WorkflowMarshaller;
+import org.apache.oozie.fluentjob.api.workflow.Workflow;
+import org.apache.oozie.fluentjob.api.workflow.WorkflowBuilder;
+import org.apache.oozie.test.WorkflowTestCase;
+
+import javax.xml.bind.JAXBException;
+import java.io.IOException;
+
+public class TestEmailAction extends WorkflowTestCase {
+
+    public void testForkedEmailActions() throws JAXBException, IOException, OozieClientException {
+        final Node parent = EmailActionBuilder.create()
+                .withRecipient("somebody@apache.org")
+                .withSubject("Subject")
+                .withBody("This is a wonderful e-mail.")
+                .build();
+
+        EmailActionBuilder.create()
+                .withParent(parent)
+                .withRecipient("somebody.else@apache.org")
+                .withSubject("Re: Subject")
+                .withBody("This is an even more wonderful e-mail.")
+                .build();
+
+        EmailActionBuilder.create()
+                .withParent(parent)
+                .withRecipient("somebody@apache.org")
+                .withSubject("Re: Subject")
+                .withBody("No, this is the most wonderful e-mail.")
+                .build();
+
+        final Workflow workflow = new WorkflowBuilder()
+                .withName("simple-email-example")
+                .withDagContainingNode(parent).build();
+
+        GraphVisualization.workflowToPng(workflow, "simple-email-example-workflow.png");
+
+        final Graph intermediateGraph = new Graph(workflow);
+
+        GraphVisualization.graphToPng(intermediateGraph, "simple-email-example-graph.png");
+
+        final String xml = WorkflowMarshaller.marshal(workflow);
+
+        log.debug("Workflow XML is:\n{0}", xml);
+
+        validate(xml);
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-client/src/test/java/org/apache/oozie/jobs/client/minitest/TestFSAction.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-client/src/test/java/org/apache/oozie/jobs/client/minitest/TestFSAction.java b/fluent-job/fluent-job-client/src/test/java/org/apache/oozie/jobs/client/minitest/TestFSAction.java
new file mode 100644
index 0000000..947360f
--- /dev/null
+++ b/fluent-job/fluent-job-client/src/test/java/org/apache/oozie/jobs/client/minitest/TestFSAction.java
@@ -0,0 +1,72 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.jobs.client.minitest;
+
+import org.apache.oozie.client.OozieClientException;
+import org.apache.oozie.fluentjob.api.GraphVisualization;
+import org.apache.oozie.fluentjob.api.action.Delete;
+import org.apache.oozie.fluentjob.api.action.FSAction;
+import org.apache.oozie.fluentjob.api.action.FSActionBuilder;
+import org.apache.oozie.fluentjob.api.action.Mkdir;
+import org.apache.oozie.fluentjob.api.dag.Graph;
+import org.apache.oozie.fluentjob.api.serialization.WorkflowMarshaller;
+import org.apache.oozie.fluentjob.api.workflow.Workflow;
+import org.apache.oozie.fluentjob.api.workflow.WorkflowBuilder;
+import org.apache.oozie.test.WorkflowTestCase;
+
+import javax.xml.bind.JAXBException;
+import java.io.IOException;
+import java.util.Date;
+
+public class TestFSAction extends WorkflowTestCase {
+
+    public void testTwoFSActions() throws JAXBException, IOException, OozieClientException {
+        final String hdfsPath = getFsTestCaseDir() + "/user/${wf:user()}/examples/output_" + new Date().getTime();
+
+        final Delete delete = new Delete(hdfsPath, true);
+
+        final Mkdir mkdir = new Mkdir(hdfsPath);
+
+        final FSAction parent = FSActionBuilder.create()
+                .withNameNode(getNameNodeUri())
+                .withDelete(delete)
+                .withMkdir(mkdir)
+                .build();
+
+        FSActionBuilder.createFromExistingAction(parent)
+                .withParent(parent)
+                .build();
+
+        final Workflow workflow = new WorkflowBuilder()
+                .withName("simple-fs-example")
+                .withDagContainingNode(parent).build();
+
+        final String xml = WorkflowMarshaller.marshal(workflow);
+
+        log.debug("Workflow XML is:\n{0}", xml);
+
+        GraphVisualization.workflowToPng(workflow, "simple-fs-example-workflow.png");
+
+        final Graph intermediateGraph = new Graph(workflow);
+
+        GraphVisualization.graphToPng(intermediateGraph, "simple-fs-example-graph.png");
+
+        validate(xml);
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-client/src/test/java/org/apache/oozie/jobs/client/minitest/TestHive2Action.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-client/src/test/java/org/apache/oozie/jobs/client/minitest/TestHive2Action.java b/fluent-job/fluent-job-client/src/test/java/org/apache/oozie/jobs/client/minitest/TestHive2Action.java
new file mode 100644
index 0000000..bb4fa98
--- /dev/null
+++ b/fluent-job/fluent-job-client/src/test/java/org/apache/oozie/jobs/client/minitest/TestHive2Action.java
@@ -0,0 +1,83 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.jobs.client.minitest;
+
+import org.apache.oozie.client.OozieClientException;
+import org.apache.oozie.fluentjob.api.GraphVisualization;
+import org.apache.oozie.fluentjob.api.action.Hive2Action;
+import org.apache.oozie.fluentjob.api.action.Hive2ActionBuilder;
+import org.apache.oozie.fluentjob.api.action.Prepare;
+import org.apache.oozie.fluentjob.api.action.PrepareBuilder;
+import org.apache.oozie.fluentjob.api.dag.Graph;
+import org.apache.oozie.fluentjob.api.serialization.WorkflowMarshaller;
+import org.apache.oozie.fluentjob.api.workflow.Workflow;
+import org.apache.oozie.fluentjob.api.workflow.WorkflowBuilder;
+import org.apache.oozie.test.WorkflowTestCase;
+
+import javax.xml.bind.JAXBException;
+import java.io.IOException;
+
+public class TestHive2Action extends WorkflowTestCase {
+    public void testForkedHive2Actions() throws IOException, JAXBException, OozieClientException {
+        final Prepare prepare = new PrepareBuilder()
+                .withDelete("hdfs://localhost:8020/user/${wf:user()}/examples/output")
+                .build();
+
+        final Hive2Action parent = Hive2ActionBuilder.create()
+                .withResourceManager(getJobTrackerUri())
+                .withNameNode(getNameNodeUri())
+                .withPrepare(prepare)
+                .withConfigProperty("mapred.job.queue.name", "default")
+                .withArg("arg1")
+                .withJdbcUrl("jdbc://hive2")
+                .withPassword("secret")
+                .withScript("hive2.sql")
+                .build();
+
+        //  We are reusing the definition of parent and only modifying and adding what is different.
+        final Hive2Action leftChild = Hive2ActionBuilder.createFromExistingAction(parent)
+                .withParent(parent)
+                .withoutArg("arg1")
+                .withArg("arg2")
+                .build();
+
+        Hive2ActionBuilder.createFromExistingAction(leftChild)
+                .withoutArg("arg2")
+                .withArg("arg3")
+                .build();
+
+        final Workflow workflow = new WorkflowBuilder()
+                .withName("simple-hive2-example")
+                .withDagContainingNode(parent).build();
+
+        final String xml = WorkflowMarshaller.marshal(workflow);
+
+        System.out.println(xml);
+
+        GraphVisualization.workflowToPng(workflow, "simple-hive2-example-workflow.png");
+
+        final Graph intermediateGraph = new Graph(workflow);
+
+        GraphVisualization.graphToPng(intermediateGraph, "simple-hive2-example-graph.png");
+
+        log.debug("Workflow XML is:\n{0}", xml);
+
+        validate(xml);
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-client/src/test/java/org/apache/oozie/jobs/client/minitest/TestHiveAction.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-client/src/test/java/org/apache/oozie/jobs/client/minitest/TestHiveAction.java b/fluent-job/fluent-job-client/src/test/java/org/apache/oozie/jobs/client/minitest/TestHiveAction.java
new file mode 100644
index 0000000..017487d
--- /dev/null
+++ b/fluent-job/fluent-job-client/src/test/java/org/apache/oozie/jobs/client/minitest/TestHiveAction.java
@@ -0,0 +1,81 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.jobs.client.minitest;
+
+import org.apache.oozie.client.OozieClientException;
+import org.apache.oozie.fluentjob.api.GraphVisualization;
+import org.apache.oozie.fluentjob.api.action.HiveAction;
+import org.apache.oozie.fluentjob.api.action.HiveActionBuilder;
+import org.apache.oozie.fluentjob.api.action.Prepare;
+import org.apache.oozie.fluentjob.api.action.PrepareBuilder;
+import org.apache.oozie.fluentjob.api.dag.Graph;
+import org.apache.oozie.fluentjob.api.serialization.WorkflowMarshaller;
+import org.apache.oozie.fluentjob.api.workflow.Workflow;
+import org.apache.oozie.fluentjob.api.workflow.WorkflowBuilder;
+import org.apache.oozie.test.WorkflowTestCase;
+
+import javax.xml.bind.JAXBException;
+import java.io.IOException;
+
+public class TestHiveAction extends WorkflowTestCase {
+    public void testForkedHiveActions() throws IOException, JAXBException, OozieClientException {
+        final Prepare prepare = new PrepareBuilder()
+                .withDelete("hdfs://localhost:8020/user/${wf:user()}/examples/output")
+                .build();
+
+        final HiveAction parent = HiveActionBuilder.create()
+                .withResourceManager(getJobTrackerUri())
+                .withNameNode(getNameNodeUri())
+                .withPrepare(prepare)
+                .withConfigProperty("mapred.job.queue.name", "default")
+                .withArg("arg1")
+                .withScript("hive2.sql")
+                .build();
+
+        //  We are reusing the definition of parent and only modifying and adding what is different.
+        final HiveAction leftChild = HiveActionBuilder.createFromExistingAction(parent)
+                .withParent(parent)
+                .withoutArg("arg1")
+                .withArg("arg2")
+                .build();
+
+        HiveActionBuilder.createFromExistingAction(leftChild)
+                .withoutArg("arg2")
+                .withArg("arg3")
+                .build();
+
+        final Workflow workflow = new WorkflowBuilder()
+                .withName("simple-hive-example")
+                .withDagContainingNode(parent).build();
+
+        final String xml = WorkflowMarshaller.marshal(workflow);
+
+        System.out.println(xml);
+
+        GraphVisualization.workflowToPng(workflow, "simple-hive-example-workflow.png");
+
+        final Graph intermediateGraph = new Graph(workflow);
+
+        GraphVisualization.graphToPng(intermediateGraph, "simple-hive-example-graph.png");
+
+        log.debug("Workflow XML is:\n{0}", xml);
+
+        validate(xml);
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-client/src/test/java/org/apache/oozie/jobs/client/minitest/TestJavaAction.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-client/src/test/java/org/apache/oozie/jobs/client/minitest/TestJavaAction.java b/fluent-job/fluent-job-client/src/test/java/org/apache/oozie/jobs/client/minitest/TestJavaAction.java
new file mode 100644
index 0000000..397144b
--- /dev/null
+++ b/fluent-job/fluent-job-client/src/test/java/org/apache/oozie/jobs/client/minitest/TestJavaAction.java
@@ -0,0 +1,92 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.jobs.client.minitest;
+
+import org.apache.oozie.client.OozieClientException;
+import org.apache.oozie.fluentjob.api.GraphVisualization;
+import org.apache.oozie.fluentjob.api.action.JavaAction;
+import org.apache.oozie.fluentjob.api.action.JavaActionBuilder;
+import org.apache.oozie.fluentjob.api.action.Prepare;
+import org.apache.oozie.fluentjob.api.action.PrepareBuilder;
+import org.apache.oozie.fluentjob.api.dag.Graph;
+import org.apache.oozie.fluentjob.api.serialization.WorkflowMarshaller;
+import org.apache.oozie.fluentjob.api.workflow.Workflow;
+import org.apache.oozie.fluentjob.api.workflow.WorkflowBuilder;
+import org.apache.oozie.test.WorkflowTestCase;
+
+import javax.xml.bind.JAXBException;
+import java.io.IOException;
+
+public class TestJavaAction extends WorkflowTestCase {
+    public void testForkedJavaActions() throws IOException, JAXBException, OozieClientException {
+        final Prepare prepare = new PrepareBuilder()
+                .withDelete("hdfs://localhost:8020/user/${wf:user()}/examples/output")
+                .build();
+
+        final JavaAction parent = JavaActionBuilder.create()
+                .withResourceManager(getJobTrackerUri())
+                .withNameNode(getNameNodeUri())
+                .withPrepare(prepare)
+                .withConfigProperty("mapred.job.queue.name", "default")
+                .withArg("arg1")
+                .withMainClass("org.apache.oozie.MyFirstMainClass")
+                .withJavaOptsString("-Dopt1a -Dopt1b")
+                .withCaptureOutput(true)
+                .build();
+
+        //  We are reusing the definition of parent and only modifying and adding what is different.
+        final JavaAction leftChild = JavaActionBuilder.createFromExistingAction(parent)
+                .withParent(parent)
+                .withoutArg("arg1")
+                .withArg("arg2")
+                .withJavaOptsString(null)
+                .withJavaOpt("-Dopt2a")
+                .withJavaOpt("-Dopt2b")
+                .withCaptureOutput(false)
+                .build();
+
+        JavaActionBuilder.createFromExistingAction(leftChild)
+                .withoutArg("arg2")
+                .withArg("arg3")
+                .withJavaOptsString(null)
+                .withoutJavaOpt("-Dopt2a")
+                .withoutJavaOpt("-Dopt2b")
+                .withJavaOpt("-Dopt3a")
+                .withJavaOpt("-Dopt3b")
+                .build();
+
+        final Workflow workflow = new WorkflowBuilder()
+                .withName("simple-java-example")
+                .withDagContainingNode(parent).build();
+
+        final String xml = WorkflowMarshaller.marshal(workflow);
+
+        System.out.println(xml);
+
+        GraphVisualization.workflowToPng(workflow, "simple-java-example-workflow.png");
+
+        final Graph intermediateGraph = new Graph(workflow);
+
+        GraphVisualization.graphToPng(intermediateGraph, "simple-java-example-graph.png");
+
+        log.debug("Workflow XML is:\n{0}", xml);
+
+        validate(xml);
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-client/src/test/java/org/apache/oozie/jobs/client/minitest/TestMapReduceAction.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-client/src/test/java/org/apache/oozie/jobs/client/minitest/TestMapReduceAction.java b/fluent-job/fluent-job-client/src/test/java/org/apache/oozie/jobs/client/minitest/TestMapReduceAction.java
new file mode 100644
index 0000000..573208b
--- /dev/null
+++ b/fluent-job/fluent-job-client/src/test/java/org/apache/oozie/jobs/client/minitest/TestMapReduceAction.java
@@ -0,0 +1,76 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.jobs.client.minitest;
+
+import org.apache.oozie.client.OozieClientException;
+import org.apache.oozie.fluentjob.api.GraphVisualization;
+import org.apache.oozie.fluentjob.api.action.MapReduceAction;
+import org.apache.oozie.fluentjob.api.action.MapReduceActionBuilder;
+import org.apache.oozie.fluentjob.api.action.Prepare;
+import org.apache.oozie.fluentjob.api.action.PrepareBuilder;
+import org.apache.oozie.fluentjob.api.dag.Graph;
+import org.apache.oozie.fluentjob.api.serialization.WorkflowMarshaller;
+import org.apache.oozie.fluentjob.api.workflow.Workflow;
+import org.apache.oozie.fluentjob.api.workflow.WorkflowBuilder;
+import org.apache.oozie.test.WorkflowTestCase;
+
+import javax.xml.bind.JAXBException;
+import java.io.IOException;
+
+public class TestMapReduceAction extends WorkflowTestCase {
+    public void testForkedMapReduceActions() throws IOException, JAXBException, OozieClientException {
+        final Prepare prepare = new PrepareBuilder()
+                .withDelete("hdfs://localhost:8020/user/${wf:user()}/examples/output")
+                .build();
+
+        final MapReduceAction parent = MapReduceActionBuilder.create()
+                .withResourceManager(getJobTrackerUri())
+                .withNameNode(getNameNodeUri())
+                .withPrepare(prepare)
+                .withConfigProperty("mapred.job.queue.name", "default")
+                .withConfigProperty("mapred.mapper.class", "org.apache.hadoop.mapred.lib.IdentityMapper")
+                .withConfigProperty("mapred.input.dir", "/user/${wf:user()}/examples/input")
+                .withConfigProperty("mapred.output.dir", "/user/${wf:user()}/examples/output")
+                .build();
+
+        //  We are reusing the definition of mrAction1 and only modifying and adding what is different.
+        final MapReduceAction leftChild = MapReduceActionBuilder.createFromExistingAction(parent)
+                .withParent(parent)
+                .build();
+
+        MapReduceActionBuilder.createFromExistingAction(leftChild)
+                .build();
+
+        final Workflow workflow = new WorkflowBuilder()
+                .withName("simple-map-reduce-example")
+                .withDagContainingNode(parent).build();
+
+        final String xml = WorkflowMarshaller.marshal(workflow);
+
+        System.out.println(xml);
+
+        GraphVisualization.workflowToPng(workflow, "simple-map-reduce-example-workflow.png");
+
+        final Graph intermediateGraph = new Graph(workflow);
+
+        GraphVisualization.graphToPng(intermediateGraph, "simple-map-reduce-example-graph.png");
+
+        validate(xml);
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-client/src/test/java/org/apache/oozie/jobs/client/minitest/TestPigAction.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-client/src/test/java/org/apache/oozie/jobs/client/minitest/TestPigAction.java b/fluent-job/fluent-job-client/src/test/java/org/apache/oozie/jobs/client/minitest/TestPigAction.java
new file mode 100644
index 0000000..b4daf33
--- /dev/null
+++ b/fluent-job/fluent-job-client/src/test/java/org/apache/oozie/jobs/client/minitest/TestPigAction.java
@@ -0,0 +1,81 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.jobs.client.minitest;
+
+import org.apache.oozie.client.OozieClientException;
+import org.apache.oozie.fluentjob.api.GraphVisualization;
+import org.apache.oozie.fluentjob.api.action.PigAction;
+import org.apache.oozie.fluentjob.api.action.PigActionBuilder;
+import org.apache.oozie.fluentjob.api.action.Prepare;
+import org.apache.oozie.fluentjob.api.action.PrepareBuilder;
+import org.apache.oozie.fluentjob.api.dag.Graph;
+import org.apache.oozie.fluentjob.api.serialization.WorkflowMarshaller;
+import org.apache.oozie.fluentjob.api.workflow.Workflow;
+import org.apache.oozie.fluentjob.api.workflow.WorkflowBuilder;
+import org.apache.oozie.test.WorkflowTestCase;
+
+import javax.xml.bind.JAXBException;
+import java.io.IOException;
+
+public class TestPigAction extends WorkflowTestCase {
+    public void testForkedPigActions() throws IOException, JAXBException, OozieClientException {
+        final Prepare prepare = new PrepareBuilder()
+                .withDelete("hdfs://localhost:8020/user/${wf:user()}/examples/output")
+                .build();
+
+        final PigAction parent = PigActionBuilder.create()
+                .withResourceManager(getJobTrackerUri())
+                .withNameNode(getNameNodeUri())
+                .withPrepare(prepare)
+                .withConfigProperty("mapred.job.queue.name", "default")
+                .withArg("arg1")
+                .withScript("pig.sql")
+                .build();
+
+        //  We are reusing the definition of parent and only modifying and adding what is different.
+        final PigAction leftChild = PigActionBuilder.createFromExistingAction(parent)
+                .withParent(parent)
+                .withoutArg("arg1")
+                .withArg("arg2")
+                .build();
+
+        PigActionBuilder.createFromExistingAction(leftChild)
+                .withoutArg("arg2")
+                .withArg("arg3")
+                .build();
+
+        final Workflow workflow = new WorkflowBuilder()
+                .withName("simple-pig-example")
+                .withDagContainingNode(parent).build();
+
+        final String xml = WorkflowMarshaller.marshal(workflow);
+
+        System.out.println(xml);
+
+        GraphVisualization.workflowToPng(workflow, "simple-pig-example-workflow.png");
+
+        final Graph intermediateGraph = new Graph(workflow);
+
+        GraphVisualization.graphToPng(intermediateGraph, "simple-pig-example-graph.png");
+
+        log.debug("Workflow XML is:\n{0}", xml);
+
+        validate(xml);
+    }
+}


[03/16] oozie git commit: OOZIE-2339 [fluent-job] Minimum Viable Fluent Job API (daniel.becker, andras.piros via rkanter, gezapeti, pbacsko)

Posted by an...@apache.org.
http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestExplicitNodeMapping.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestExplicitNodeMapping.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestExplicitNodeMapping.java
new file mode 100644
index 0000000..47eb9c3
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestExplicitNodeMapping.java
@@ -0,0 +1,47 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.mapping;
+
+import org.apache.oozie.fluentjob.api.generated.workflow.ACTION;
+import org.apache.oozie.fluentjob.api.action.MapReduceAction;
+import org.apache.oozie.fluentjob.api.action.MapReduceActionBuilder;
+import org.apache.oozie.fluentjob.api.dag.End;
+import org.apache.oozie.fluentjob.api.dag.ExplicitNode;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+public class TestExplicitNodeMapping {
+    @Test
+    public void testMappingExplicitNode() {
+        final MapReduceAction mrAction = MapReduceActionBuilder.create().withName("map-reduce-action").build();
+        final ExplicitNode node = new ExplicitNode(mrAction.getName(), mrAction);
+
+        final End end = new End("end");
+
+        end.addParent(node);
+
+        final ACTION action = DozerBeanMapperSingleton.instance().map(node, ACTION.class);
+
+        assertEquals(mrAction.getName(), action.getName());
+        assertEquals(end.getName(), action.getOk().getTo());
+        assertNotNull(action.getMapReduce());
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestFSActionMapping.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestFSActionMapping.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestFSActionMapping.java
new file mode 100644
index 0000000..c43ca0a
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestFSActionMapping.java
@@ -0,0 +1,77 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.mapping;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.oozie.fluentjob.api.generated.workflow.FS;
+import org.apache.oozie.fluentjob.api.action.Chgrp;
+import org.apache.oozie.fluentjob.api.action.ChgrpBuilder;
+import org.apache.oozie.fluentjob.api.action.Chmod;
+import org.apache.oozie.fluentjob.api.action.ChmodBuilder;
+import org.apache.oozie.fluentjob.api.action.Delete;
+import org.apache.oozie.fluentjob.api.action.FSAction;
+import org.apache.oozie.fluentjob.api.action.FSActionBuilder;
+import org.apache.oozie.fluentjob.api.action.Mkdir;
+import org.apache.oozie.fluentjob.api.action.Move;
+import org.apache.oozie.fluentjob.api.action.Touchz;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestFSActionMapping {
+    @Test
+    public void testMappingFSAction() {
+        final String nameNode = "${nameNode}";
+        final ImmutableList<String> jobXmls = new ImmutableList.Builder<String>().add("jobXml1").add("jobXml2").build();
+
+        final Delete delete = new Delete("path", true);
+        final Mkdir mkdir = new Mkdir("path");
+        final Move move = new Move("from", "to");
+        final Chmod chmod = new ChmodBuilder().withPermissions("511").build();
+        final Touchz touchz = new Touchz("path");
+        final Chgrp chgrp = new ChgrpBuilder().withGroup("user:group:").build();
+
+        final FSActionBuilder builder = FSActionBuilder.create()
+                .withNameNode(nameNode);
+
+        for (final String jobXml : jobXmls) {
+            builder.withJobXml(jobXml);
+        }
+
+        builder.withDelete(delete)
+                .withMkdir(mkdir)
+                .withMove(move)
+                .withChmod(chmod)
+                .withTouchz(touchz)
+                .withChgrp(chgrp);
+
+        final FSAction action = builder.build();
+
+        final FS fsAction = DozerBeanMapperSingleton.instance().map(action, FS.class);
+
+        assertEquals(nameNode, fsAction.getNameNode());
+        assertEquals(jobXmls, fsAction.getJobXml());
+
+        final List<Object> expectedList = Arrays.asList(delete, mkdir, move, chmod, touchz, chgrp);
+        assertEquals(expectedList.size(), fsAction.getDeleteOrMkdirOrMove().size());
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestForkMapping.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestForkMapping.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestForkMapping.java
new file mode 100644
index 0000000..8526744
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestForkMapping.java
@@ -0,0 +1,70 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.mapping;
+
+import org.apache.oozie.fluentjob.api.generated.workflow.FORK;
+import org.apache.oozie.fluentjob.api.generated.workflow.FORKTRANSITION;
+import org.apache.oozie.fluentjob.api.dag.Decision;
+import org.apache.oozie.fluentjob.api.dag.DecisionJoin;
+import org.apache.oozie.fluentjob.api.dag.ExplicitNode;
+import org.apache.oozie.fluentjob.api.dag.Fork;
+import org.apache.oozie.fluentjob.api.dag.NodeBase;
+import org.junit.Test;
+
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestForkMapping extends TestControlNodeMappingBase {
+    @Test
+    public void testMappingFork() {
+        final String name = "fork";
+        final Fork fork = new Fork(name);
+
+        final NodeBase child1 = new ExplicitNode("child1", null);
+        final NodeBase child2 = new ExplicitNode("child2", null);
+
+        child1.addParent(fork);
+        child2.addParent(fork);
+
+        final FORK mappedFork = DozerBeanMapperSingleton.instance().map(fork, FORK.class);
+
+        assertEquals(name, mappedFork.getName());
+
+        final List<FORKTRANSITION> transitions = mappedFork.getPath();
+        assertEquals(child1.getName(), transitions.get(0).getStart());
+        assertEquals(child2.getName(), transitions.get(1).getStart());
+    }
+
+    @Test
+    public void testMappingForkWithDecisionJoin() {
+        final String childName = "child";
+        final Fork fork = new Fork("fork");
+
+        final NodeBase decisionJoin = new DecisionJoin("decisionJoin", new Decision("decision"));
+        decisionJoin.addParent(fork);
+
+        final NodeBase child = new ExplicitNode(childName, null);
+        child.addParent(decisionJoin);
+
+        final FORK mappedFork = DozerBeanMapperSingleton.instance().map(fork, FORK.class);
+
+        assertEquals(childName, mappedFork.getPath().get(0).getStart());
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestGlobalMapping.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestGlobalMapping.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestGlobalMapping.java
new file mode 100644
index 0000000..8501813
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestGlobalMapping.java
@@ -0,0 +1,56 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.mapping;
+
+import org.apache.oozie.fluentjob.api.generated.workflow.GLOBAL;
+import org.apache.oozie.fluentjob.api.action.LauncherBuilder;
+import org.apache.oozie.fluentjob.api.generated.workflow.GLOBAL;
+import org.apache.oozie.fluentjob.api.workflow.Global;
+import org.apache.oozie.fluentjob.api.workflow.GlobalBuilder;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestGlobalMapping {
+    private static final String DEFAULT = "default";
+
+    @Test
+    public void testMappingGlobal() {
+        final Global source = GlobalBuilder.create()
+                .withResourceManager(DEFAULT)
+                .withNameNode(DEFAULT)
+                .withJobXml(DEFAULT)
+                .withConfigProperty("key1", "value1")
+                .withLauncher(new LauncherBuilder()
+                        .withMemoryMb(1024L)
+                        .withVCores(1L)
+                        .build())
+                .build();
+
+        final GLOBAL destination = DozerBeanMapperSingleton.instance().map(source, GLOBAL.class);
+
+        assertEquals(DEFAULT, destination.getResourceManager());
+        assertEquals(DEFAULT, destination.getNameNode());
+        assertEquals(DEFAULT, destination.getJobXml().get(0));
+        assertEquals("key1", destination.getConfiguration().getProperty().get(0).getName());
+        assertEquals("value1", destination.getConfiguration().getProperty().get(0).getValue());
+        assertEquals(1024L, destination.getLauncher().getMemoryMbOrVcoresOrJavaOpts().get(0).getValue());
+        assertEquals(1L, destination.getLauncher().getMemoryMbOrVcoresOrJavaOpts().get(1).getValue());
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestGraphMapping.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestGraphMapping.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestGraphMapping.java
new file mode 100644
index 0000000..0563ec6
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestGraphMapping.java
@@ -0,0 +1,255 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.mapping;
+
+import org.apache.oozie.fluentjob.api.Condition;
+import org.apache.oozie.fluentjob.api.action.EmailActionBuilder;
+import org.apache.oozie.fluentjob.api.action.ErrorHandler;
+import org.apache.oozie.fluentjob.api.action.FSAction;
+import org.apache.oozie.fluentjob.api.action.FSActionBuilder;
+import org.apache.oozie.fluentjob.api.action.MapReduceAction;
+import org.apache.oozie.fluentjob.api.action.MapReduceActionBuilder;
+import org.apache.oozie.fluentjob.api.action.Node;
+import org.apache.oozie.fluentjob.api.generated.workflow.ACTION;
+import org.apache.oozie.fluentjob.api.generated.workflow.ACTIONTRANSITION;
+import org.apache.oozie.fluentjob.api.generated.workflow.CASE;
+import org.apache.oozie.fluentjob.api.generated.workflow.DECISION;
+import org.apache.oozie.fluentjob.api.generated.workflow.DEFAULT;
+import org.apache.oozie.fluentjob.api.generated.workflow.END;
+import org.apache.oozie.fluentjob.api.generated.workflow.FORK;
+import org.apache.oozie.fluentjob.api.generated.workflow.FORKTRANSITION;
+import org.apache.oozie.fluentjob.api.generated.workflow.JOIN;
+import org.apache.oozie.fluentjob.api.generated.workflow.KILL;
+import org.apache.oozie.fluentjob.api.generated.workflow.ObjectFactory;
+import org.apache.oozie.fluentjob.api.generated.workflow.START;
+import org.apache.oozie.fluentjob.api.generated.workflow.SWITCH;
+import org.apache.oozie.fluentjob.api.generated.workflow.WORKFLOWAPP;
+import org.apache.oozie.fluentjob.api.dag.Decision;
+import org.apache.oozie.fluentjob.api.dag.DecisionJoin;
+import org.apache.oozie.fluentjob.api.dag.End;
+import org.apache.oozie.fluentjob.api.dag.ExplicitNode;
+import org.apache.oozie.fluentjob.api.dag.Fork;
+import org.apache.oozie.fluentjob.api.dag.Graph;
+import org.apache.oozie.fluentjob.api.dag.Join;
+import org.apache.oozie.fluentjob.api.dag.Start;
+import org.apache.oozie.fluentjob.api.workflow.Workflow;
+import org.apache.oozie.fluentjob.api.workflow.WorkflowBuilder;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestGraphMapping {
+    private static final ObjectFactory OBJECT_FACTORY = new ObjectFactory();
+
+    @Test
+    public void testMappingGraphFromWorkflow() {
+        final String errorHandlerName = "error-handler";
+
+        final EmailActionBuilder emailBuilder = EmailActionBuilder.create().withName(errorHandlerName)
+                .withRecipient("somebody@workplace.com")
+                .withSubject("Subject")
+                .withBody("Email body.");
+
+        final ErrorHandler errorHandler = ErrorHandler.buildAsErrorHandler(emailBuilder);
+
+        final MapReduceAction mrAction = MapReduceActionBuilder.create()
+                .withName("map-reduce-action")
+                .build();
+        final FSAction fsAction = FSActionBuilder.create()
+                .withName("fs-action")
+                .withParent(mrAction)
+                .withErrorHandler(errorHandler)
+                .build();
+
+        final Workflow workflow = new WorkflowBuilder().withName("workflow-name").withDagContainingNode(mrAction).build();
+
+        final Graph graph = new Graph(workflow);
+
+        final WORKFLOWAPP workflowapp = DozerBeanMapperSingleton.instance().map(graph, WORKFLOWAPP.class);
+
+        final WORKFLOWAPP expectedWorkflowapp = OBJECT_FACTORY.createWORKFLOWAPP();
+        expectedWorkflowapp.setName(workflow.getName());
+
+        final START start = OBJECT_FACTORY.createSTART();
+        start.setTo(mrAction.getName());
+        expectedWorkflowapp.setStart(start);
+
+        final END end = OBJECT_FACTORY.createEND();
+        end.setName("end");
+        expectedWorkflowapp.setEnd(end);
+
+        final List<Object> actions = expectedWorkflowapp.getDecisionOrForkOrJoin();
+
+        final ACTION actionMr = convertEmailActionByHand((ExplicitNode) graph.getNodeByName(mrAction.getName()));
+
+        final ACTION actionFs = convertEmailActionByHand((ExplicitNode) graph.getNodeByName(fsAction.getName()));
+
+        final ACTIONTRANSITION error = actionFs.getError();
+        error.setTo(errorHandlerName);
+
+        final Node emailErrorHandlerNode = emailBuilder.build();
+        final ExplicitNode emailErrorHandlerExplicitNode
+                = new ExplicitNode(emailErrorHandlerNode.getName(), emailErrorHandlerNode);
+        final ACTION errorHandlerAction = convertEmailActionByHand(emailErrorHandlerExplicitNode);
+
+        final KILL kill = createKill();
+
+        final ACTIONTRANSITION okAndError = OBJECT_FACTORY.createACTIONTRANSITION();
+        okAndError.setTo(kill.getName());
+
+        errorHandlerAction.setOk(okAndError);
+        errorHandlerAction.setError(okAndError);
+
+        actions.add(kill);
+        actions.add(actionMr);
+        actions.add(errorHandlerAction);
+        actions.add(actionFs);
+
+        assertEquals(expectedWorkflowapp, workflowapp);
+    }
+
+    @Test
+    public void testMappingGraphFromNodes() {
+        final String workflowName = "test-workflow";
+        final String condition = "condition";
+
+        final ExplicitNode A = new ExplicitNode("A", EmailActionBuilder.create().build());
+        final ExplicitNode B = new ExplicitNode("B", EmailActionBuilder.create().build());
+        final ExplicitNode C = new ExplicitNode("C", EmailActionBuilder.create().build());
+        final ExplicitNode D = new ExplicitNode("D", EmailActionBuilder.create().build());
+        final ExplicitNode E = new ExplicitNode("E", EmailActionBuilder.create().build());
+
+        final Start start = new Start("start");
+        final End end = new End("end");
+        final Fork fork = new Fork("fork1");
+        final Join join = new Join("join1", fork);
+        final Decision decision = new Decision("decision");
+        final DecisionJoin decisionJoin = new DecisionJoin("decisionJoin", decision);
+
+        // TODO: Unfortunately the order of the elements counts.
+        end.addParent(join);
+        join.addParent(decisionJoin);
+        join.addParent(C);
+        decisionJoin.addParent(D);
+        decisionJoin.addParent(E);
+        D.addParentWithCondition(decision, Condition.actualCondition(condition));
+        E.addParentDefaultConditional(decision);
+        decision.addParent(B);
+        B.addParent(fork);
+        C.addParent(fork);
+        fork.addParent(A);
+        A.addParent(start);
+
+        final GraphNodes graphNodes = new GraphNodes(workflowName,
+                null,
+                null,
+                null,
+                start,
+                end,
+                Arrays.asList(A, B, C, D, E, fork, join, decision, decisionJoin));
+
+
+        final WORKFLOWAPP expectedWorkflowApp = OBJECT_FACTORY.createWORKFLOWAPP();
+        expectedWorkflowApp.setName(workflowName);
+
+        final START startJaxb = OBJECT_FACTORY.createSTART();
+        startJaxb.setTo(start.getChild().getName());
+        expectedWorkflowApp.setStart(startJaxb);
+
+        final END endJaxb = OBJECT_FACTORY.createEND();
+        endJaxb.setName(end.getName());
+        expectedWorkflowApp.setEnd(endJaxb);
+
+        final List<Object> nodesInWorkflowApp = expectedWorkflowApp.getDecisionOrForkOrJoin();
+
+        final FORK forkJaxb = OBJECT_FACTORY.createFORK();
+        forkJaxb.setName(fork.getName());
+        final List<FORKTRANSITION> transitions = forkJaxb.getPath();
+        final FORKTRANSITION transitionB = OBJECT_FACTORY.createFORKTRANSITION();
+        transitionB.setStart(B.getName());
+        final FORKTRANSITION transitionC = OBJECT_FACTORY.createFORKTRANSITION();
+        transitionC.setStart(C.getName());
+        transitions.add(transitionB);
+        transitions.add(transitionC);
+
+        final ACTION actionA = convertEmailActionByHand(A);
+
+        final ACTION actionB = convertEmailActionByHand(B);
+
+        final ACTION actionC = convertEmailActionByHand(C);
+
+        final DECISION decisionJaxb = OBJECT_FACTORY.createDECISION();
+        decisionJaxb.setName(decision.getName());
+        final SWITCH _switch = OBJECT_FACTORY.createSWITCH();
+        final List<CASE> cases = _switch.getCase();
+        final CASE _case = OBJECT_FACTORY.createCASE();
+        _case.setTo(D.getName());
+        _case.setValue(condition);
+        cases.add(_case);
+        final DEFAULT _default = OBJECT_FACTORY.createDEFAULT();
+        _default.setTo(E.getName());
+        _switch.setDefault(_default);
+        decisionJaxb.setSwitch(_switch);
+
+        final ACTION actionD = convertEmailActionByHand(D);
+
+        final ACTION actionE = convertEmailActionByHand(E);
+
+        final JOIN joinJaxb = OBJECT_FACTORY.createJOIN();
+        joinJaxb.setName(join.getName());
+        joinJaxb.setTo(end.getName());
+
+        // TODO: Unfortunately the order of the elements counts.
+        nodesInWorkflowApp.add(createKill());
+        nodesInWorkflowApp.add(actionA);
+        nodesInWorkflowApp.add(actionB);
+        nodesInWorkflowApp.add(actionC);
+        nodesInWorkflowApp.add(actionD);
+        nodesInWorkflowApp.add(actionE);
+        nodesInWorkflowApp.add(forkJaxb);
+        nodesInWorkflowApp.add(joinJaxb);
+        nodesInWorkflowApp.add(decisionJaxb);
+
+        final WORKFLOWAPP workflowapp = DozerBeanMapperSingleton.instance().map(graphNodes, WORKFLOWAPP.class);
+
+        assertEquals(expectedWorkflowApp, workflowapp);
+    }
+
+    private ACTION convertEmailActionByHand(final ExplicitNode node) {
+        final ACTION action = DozerBeanMapperSingleton.instance().map(node, ACTION.class);
+
+        final ACTIONTRANSITION error = OBJECT_FACTORY.createACTIONTRANSITION();
+        error.setTo(createKill().getName());
+        action.setError(error);
+
+        return action;
+    }
+
+    private KILL createKill() {
+        final KILL kill = OBJECT_FACTORY.createKILL();
+
+        kill.setName("kill");
+        kill.setMessage("Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]");
+
+        return kill;
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestHive2ActionMapping.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestHive2ActionMapping.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestHive2ActionMapping.java
new file mode 100644
index 0000000..a807e60
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestHive2ActionMapping.java
@@ -0,0 +1,84 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.mapping;
+
+import org.apache.oozie.fluentjob.api.action.Hive2Action;
+import org.apache.oozie.fluentjob.api.action.Hive2ActionBuilder;
+import org.apache.oozie.fluentjob.api.action.LauncherBuilder;
+import org.apache.oozie.fluentjob.api.action.PrepareBuilder;
+import org.apache.oozie.fluentjob.api.generated.action.hive2.ACTION;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+public class TestHive2ActionMapping {
+
+    @Test
+    public void testMappingHive2Action() {
+        final String resourceManager = "${resourceManager}";
+        final String nameNode = "${nameNode}";
+        final List<String> args = Arrays.asList("arg1", "arg2");
+
+        final Hive2ActionBuilder builder = Hive2ActionBuilder.create();
+
+        builder.withResourceManager(resourceManager)
+                .withNameNode(nameNode)
+                .withPrepare(new PrepareBuilder()
+                        .withDelete("/path/to/delete")
+                        .withMkdir("/path/to/mkdir")
+                        .build())
+                .withLauncher(new LauncherBuilder()
+                        .withMemoryMb(1024)
+                        .withVCores(2)
+                        .withQueue("default")
+                        .withSharelib("default")
+                        .withViewAcl("default")
+                        .withModifyAcl("default")
+                        .build());
+
+        for (final String arg : args) {
+            builder.withArg(arg);
+        }
+
+        builder.withConfigProperty("propertyName1", "propertyValue1")
+                .withConfigProperty("propertyName2", "propertyValue2");
+
+        final Hive2Action action = builder.build();
+
+        final ACTION hive2 = DozerBeanMapperSingleton.instance().map(action, ACTION.class);
+
+        assertEquals(resourceManager, hive2.getResourceManager());
+        assertEquals(nameNode, hive2.getNameNode());
+        assertNotNull(hive2.getPrepare());
+        assertEquals("/path/to/delete", hive2.getPrepare().getDelete().get(0).getPath());
+        assertEquals("/path/to/mkdir", hive2.getPrepare().getMkdir().get(0).getPath());
+        assertNotNull(hive2.getConfiguration());
+        assertEquals(args, hive2.getArgument());
+        assertEquals(1024L, hive2.getLauncher().getMemoryMbOrVcoresOrJavaOpts().get(0).getValue());
+        assertEquals(2L, hive2.getLauncher().getMemoryMbOrVcoresOrJavaOpts().get(1).getValue());
+        assertEquals("default", hive2.getLauncher().getMemoryMbOrVcoresOrJavaOpts().get(2).getValue());
+        assertEquals("default", hive2.getLauncher().getMemoryMbOrVcoresOrJavaOpts().get(3).getValue());
+        assertEquals("default", hive2.getLauncher().getMemoryMbOrVcoresOrJavaOpts().get(4).getValue());
+        assertEquals("default", hive2.getLauncher().getMemoryMbOrVcoresOrJavaOpts().get(5).getValue());
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestHiveActionMapping.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestHiveActionMapping.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestHiveActionMapping.java
new file mode 100644
index 0000000..95e6da3
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestHiveActionMapping.java
@@ -0,0 +1,84 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.mapping;
+
+import org.apache.oozie.fluentjob.api.action.HiveAction;
+import org.apache.oozie.fluentjob.api.action.HiveActionBuilder;
+import org.apache.oozie.fluentjob.api.action.LauncherBuilder;
+import org.apache.oozie.fluentjob.api.action.PrepareBuilder;
+import org.apache.oozie.fluentjob.api.generated.action.hive.ACTION;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+public class TestHiveActionMapping {
+
+    @Test
+    public void testMappingHiveAction() {
+        final String resourceManager = "${resourceManager}";
+        final String nameNode = "${nameNode}";
+        final List<String> args = Arrays.asList("arg1", "arg2");
+
+        final HiveActionBuilder builder = HiveActionBuilder.create();
+
+        builder.withResourceManager(resourceManager)
+                .withNameNode(nameNode)
+                .withPrepare(new PrepareBuilder()
+                        .withDelete("/path/to/delete")
+                        .withMkdir("/path/to/mkdir")
+                        .build())
+                .withLauncher(new LauncherBuilder()
+                        .withMemoryMb(1024)
+                        .withVCores(2)
+                        .withQueue("default")
+                        .withSharelib("default")
+                        .withViewAcl("default")
+                        .withModifyAcl("default")
+                        .build());
+
+        for (final String arg : args) {
+            builder.withArg(arg);
+        }
+
+        builder.withConfigProperty("propertyName1", "propertyValue1")
+                .withConfigProperty("propertyName2", "propertyValue2");
+
+        final HiveAction action = builder.build();
+
+        final ACTION hive = DozerBeanMapperSingleton.instance().map(action, ACTION.class);
+
+        assertEquals(resourceManager, hive.getResourceManager());
+        assertEquals(nameNode, hive.getNameNode());
+        assertNotNull(hive.getPrepare());
+        assertEquals("/path/to/delete", hive.getPrepare().getDelete().get(0).getPath());
+        assertEquals("/path/to/mkdir", hive.getPrepare().getMkdir().get(0).getPath());
+        assertNotNull(hive.getConfiguration());
+        assertEquals(args, hive.getArgument());
+        assertEquals(1024L, hive.getLauncher().getMemoryMbOrVcoresOrJavaOpts().get(0).getValue());
+        assertEquals(2L, hive.getLauncher().getMemoryMbOrVcoresOrJavaOpts().get(1).getValue());
+        assertEquals("default", hive.getLauncher().getMemoryMbOrVcoresOrJavaOpts().get(2).getValue());
+        assertEquals("default", hive.getLauncher().getMemoryMbOrVcoresOrJavaOpts().get(3).getValue());
+        assertEquals("default", hive.getLauncher().getMemoryMbOrVcoresOrJavaOpts().get(4).getValue());
+        assertEquals("default", hive.getLauncher().getMemoryMbOrVcoresOrJavaOpts().get(5).getValue());
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestJavaActionMapping.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestJavaActionMapping.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestJavaActionMapping.java
new file mode 100644
index 0000000..ba79298
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestJavaActionMapping.java
@@ -0,0 +1,95 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.mapping;
+
+import org.apache.oozie.fluentjob.api.generated.workflow.JAVA;
+import org.apache.oozie.fluentjob.api.action.JavaAction;
+import org.apache.oozie.fluentjob.api.action.JavaActionBuilder;
+import org.apache.oozie.fluentjob.api.action.LauncherBuilder;
+import org.apache.oozie.fluentjob.api.action.PrepareBuilder;
+import org.apache.oozie.fluentjob.api.generated.workflow.JAVA;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+public class TestJavaActionMapping {
+
+    public static final String DEFAULT = "default";
+
+    @Test
+    public void testMappingJavaAction() {
+        final String resourceManager = "${resourceManager}";
+        final String nameNode = "${nameNode}";
+        final List<String> args = Arrays.asList("arg1", "arg2");
+
+        final JavaActionBuilder builder = JavaActionBuilder.create();
+
+        builder.withResourceManager(resourceManager)
+                .withNameNode(nameNode)
+                .withPrepare(new PrepareBuilder()
+                        .withDelete("/path/to/delete")
+                        .withMkdir("/path/to/mkdir")
+                        .build())
+                .withLauncher(new LauncherBuilder()
+                        .withMemoryMb(1024)
+                        .withVCores(2)
+                        .withQueue(DEFAULT)
+                        .withSharelib(DEFAULT)
+                        .withViewAcl(DEFAULT)
+                        .withModifyAcl(DEFAULT)
+                        .build())
+                .withMainClass(DEFAULT)
+                .withJavaOptsString(DEFAULT)
+                .withJavaOpt(DEFAULT)
+                .withCaptureOutput(true);
+
+        for (final String arg : args) {
+            builder.withArg(arg);
+        }
+
+        builder.withConfigProperty("propertyName1", "propertyValue1")
+                .withConfigProperty("propertyName2", "propertyValue2");
+
+        final JavaAction action = builder.build();
+
+        final JAVA java = DozerBeanMapperSingleton.instance().map(action, JAVA.class);
+
+        assertEquals(resourceManager, java.getResourceManager());
+        assertEquals(nameNode, java.getNameNode());
+        assertNotNull(java.getPrepare());
+        assertEquals("/path/to/delete", java.getPrepare().getDelete().get(0).getPath());
+        assertEquals("/path/to/mkdir", java.getPrepare().getMkdir().get(0).getPath());
+        assertNotNull(java.getConfiguration());
+        assertEquals(args, java.getArg());
+        assertEquals(1024L, java.getLauncher().getMemoryMbOrVcoresOrJavaOpts().get(0).getValue());
+        assertEquals(2L, java.getLauncher().getMemoryMbOrVcoresOrJavaOpts().get(1).getValue());
+        assertEquals(DEFAULT, java.getLauncher().getMemoryMbOrVcoresOrJavaOpts().get(2).getValue());
+        assertEquals(DEFAULT, java.getLauncher().getMemoryMbOrVcoresOrJavaOpts().get(3).getValue());
+        assertEquals(DEFAULT, java.getLauncher().getMemoryMbOrVcoresOrJavaOpts().get(4).getValue());
+        assertEquals(DEFAULT, java.getLauncher().getMemoryMbOrVcoresOrJavaOpts().get(5).getValue());
+        assertEquals(DEFAULT, java.getMainClass());
+        assertEquals(DEFAULT, java.getJavaOpts());
+        assertEquals(DEFAULT, java.getJavaOpt().get(0));
+        assertNotNull(java.getCaptureOutput());
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestJoinMapping.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestJoinMapping.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestJoinMapping.java
new file mode 100644
index 0000000..fd8184d
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestJoinMapping.java
@@ -0,0 +1,64 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.mapping;
+
+import org.apache.oozie.fluentjob.api.generated.workflow.JOIN;
+import org.apache.oozie.fluentjob.api.dag.Decision;
+import org.apache.oozie.fluentjob.api.dag.DecisionJoin;
+import org.apache.oozie.fluentjob.api.dag.ExplicitNode;
+import org.apache.oozie.fluentjob.api.dag.Fork;
+import org.apache.oozie.fluentjob.api.dag.Join;
+import org.apache.oozie.fluentjob.api.dag.NodeBase;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestJoinMapping extends TestControlNodeMappingBase {
+    @Test
+    public void testMappingJoin() {
+        final String joinName = "join";
+        final String childName = "child";
+        final Join join = new Join(joinName, new Fork("fork"));
+
+        final NodeBase child = new ExplicitNode(childName, null);
+
+        child.addParent(join);
+
+        final JOIN mappedJoin = DozerBeanMapperSingleton.instance().map(join, JOIN.class);
+
+        assertEquals(joinName, mappedJoin.getName());
+        assertEquals(childName, mappedJoin.getTo());
+    }
+
+    @Test
+    public void testMappingJoinWithDecisionJoin() {
+        final String childName = "child";
+        final Join join = new Join("join", new Fork("fork"));
+
+        final NodeBase decisionJoin = new DecisionJoin("decisionJoin", new Decision("decision"));
+        decisionJoin.addParent(join);
+
+        final NodeBase child = new ExplicitNode(childName, null);
+        child.addParent(decisionJoin);
+
+        final JOIN mappedJoin = DozerBeanMapperSingleton.instance().map(join, JOIN.class);
+
+        assertEquals(childName, mappedJoin.getTo());
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestMapReduceActionMapping.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestMapReduceActionMapping.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestMapReduceActionMapping.java
new file mode 100644
index 0000000..356fe7d
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestMapReduceActionMapping.java
@@ -0,0 +1,96 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.mapping;
+
+import org.apache.oozie.fluentjob.api.generated.workflow.MAPREDUCE;
+import org.apache.oozie.fluentjob.api.action.MapReduceAction;
+import org.apache.oozie.fluentjob.api.action.MapReduceActionBuilder;
+import org.apache.oozie.fluentjob.api.action.PipesBuilder;
+import org.apache.oozie.fluentjob.api.action.PrepareBuilder;
+import org.apache.oozie.fluentjob.api.action.StreamingBuilder;
+import org.apache.oozie.fluentjob.api.generated.workflow.MAPREDUCE;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import java.util.Arrays;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+
+public class TestMapReduceActionMapping {
+    @Rule
+    public final ExpectedException expectedException = ExpectedException.none();
+
+    @Test
+    public void testMappingMapReduceAction() {
+        final String resourceManager = "${resourceManager}";
+        final String nameNode = "${nameNode}";
+
+        final List<String> jobXmls = Arrays.asList("job1.xml", "job2.xml");
+
+        final String configClass = "${configClass}";
+
+        final List<String> files = Arrays.asList("file1", "file2");
+
+        final List<String> archives = Arrays.asList("archive1", "archive2");
+
+        final MapReduceActionBuilder builder = MapReduceActionBuilder.create();
+
+        builder.withResourceManager(resourceManager)
+                .withNameNode(nameNode)
+                .withPrepare(new PrepareBuilder().build())
+                .withStreaming(new StreamingBuilder().build())
+                .withPipes(new PipesBuilder().build());
+
+        for (final String jobXml : jobXmls) {
+            builder.withJobXml(jobXml);
+        }
+
+        builder.withConfigProperty("propertyName1", "propertyValue1")
+                .withConfigProperty("propertyName2", "propertyValue2");
+
+        builder.withConfigClass(configClass);
+
+        for (final String file : files) {
+            builder.withFile(file);
+        }
+
+        for (final String archive : archives) {
+            builder.withArchive(archive);
+        }
+
+        final MapReduceAction action = builder.build();
+
+        final MAPREDUCE mapreduce = DozerBeanMapperSingleton.instance().map(action, MAPREDUCE.class);
+
+        assertEquals(resourceManager, mapreduce.getResourceManager());
+        assertEquals(nameNode, mapreduce.getNameNode());
+        assertNotNull(mapreduce.getPrepare());
+        assertNotNull(mapreduce.getStreaming());
+        assertNotNull(mapreduce.getPipes());
+        assertEquals(jobXmls, mapreduce.getJobXml());
+        assertNotNull(mapreduce.getConfiguration());
+        assertEquals(configClass, mapreduce.getConfigClass());
+        assertEquals(files, mapreduce.getFile());
+        assertEquals(archives, mapreduce.getArchive());
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestMappings.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestMappings.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestMappings.java
new file mode 100644
index 0000000..dd844be
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestMappings.java
@@ -0,0 +1,59 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.mapping;
+
+import org.apache.oozie.fluentjob.api.generated.workflow.WORKFLOWAPP;
+import org.apache.oozie.fluentjob.api.action.MapReduceAction;
+import org.apache.oozie.fluentjob.api.action.MapReduceActionBuilder;
+import org.apache.oozie.fluentjob.api.dag.Graph;
+import org.apache.oozie.fluentjob.api.workflow.Workflow;
+import org.apache.oozie.fluentjob.api.workflow.WorkflowBuilder;
+import org.dozer.DozerBeanMapper;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestMappings {
+
+    @Test
+    public void whenWorkflowWithOneMRActionIsCreatedWORKFLOWAPPIsMappedCorrectly() {
+        final MapReduceAction mr1 = MapReduceActionBuilder.create().withName("mr1").build();
+
+        final Workflow workflow = new WorkflowBuilder()
+                .withName("Workflow_to_map")
+                .withDagContainingNode(mr1)
+                .build();
+        final Graph graph = new Graph(workflow);
+
+        final List<String> mappingFiles = new ArrayList<>();
+        mappingFiles.add("dozer_config.xml");
+        mappingFiles.add("mappingGraphToWORKFLOWAPP.xml");
+        mappingFiles.add("action_mappings.xml");
+
+        final DozerBeanMapper mapper = new DozerBeanMapper();
+        mapper.setMappingFiles(mappingFiles);
+
+        final WORKFLOWAPP workflowapp = mapper.map(graph, WORKFLOWAPP.class);
+
+        assertEquals("API and JAXB workflows should have the same names", workflow.getName(), workflowapp.getName());
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestMkdirMapping.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestMkdirMapping.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestMkdirMapping.java
new file mode 100644
index 0000000..b19bfec
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestMkdirMapping.java
@@ -0,0 +1,38 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.mapping;
+
+import org.apache.oozie.fluentjob.api.generated.workflow.MKDIR;
+import org.apache.oozie.fluentjob.api.action.Mkdir;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestMkdirMapping {
+
+    @Test
+    public void testMappingMkdir() {
+        final String path = "path/to/location";
+        final Mkdir mkdir = new Mkdir(path);
+
+        final MKDIR mkdirJAXB = DozerBeanMapperSingleton.instance().map(mkdir, MKDIR.class);
+
+        assertEquals(path, mkdirJAXB.getPath());
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestParametersMapping.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestParametersMapping.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestParametersMapping.java
new file mode 100644
index 0000000..902d02e
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestParametersMapping.java
@@ -0,0 +1,47 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.mapping;
+
+import org.apache.oozie.fluentjob.api.generated.workflow.PARAMETERS;
+import org.apache.oozie.fluentjob.api.workflow.Parameters;
+import org.apache.oozie.fluentjob.api.workflow.ParametersBuilder;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
+public class TestParametersMapping {
+
+    @Test
+    public void testMappingParameters() {
+        final Parameters source = ParametersBuilder.create()
+                .withParameter("name1", "value1")
+                .withParameter("name2", "value2", "description2")
+                .build();
+
+        final PARAMETERS destination = DozerBeanMapperSingleton.instance().map(source, PARAMETERS.class);
+
+        assertEquals("name1", destination.getProperty().get(0).getName());
+        assertEquals("value1", destination.getProperty().get(0).getValue());
+        assertNull(destination.getProperty().get(0).getDescription());
+        assertEquals("name2", destination.getProperty().get(1).getName());
+        assertEquals("value2", destination.getProperty().get(1).getValue());
+        assertEquals("description2", destination.getProperty().get(1).getDescription());
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestPigActionMapping.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestPigActionMapping.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestPigActionMapping.java
new file mode 100644
index 0000000..daf67ca
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestPigActionMapping.java
@@ -0,0 +1,84 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.mapping;
+
+import org.apache.oozie.fluentjob.api.action.LauncherBuilder;
+import org.apache.oozie.fluentjob.api.action.PigAction;
+import org.apache.oozie.fluentjob.api.action.PigActionBuilder;
+import org.apache.oozie.fluentjob.api.action.PrepareBuilder;
+import org.apache.oozie.fluentjob.api.generated.workflow.PIG;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+public class TestPigActionMapping {
+
+    @Test
+    public void testMappingPigAction() {
+        final String resourceManager = "${resourceManager}";
+        final String nameNode = "${nameNode}";
+        final List<String> args = Arrays.asList("arg1", "arg2");
+
+        final PigActionBuilder builder = PigActionBuilder.create();
+
+        builder.withResourceManager(resourceManager)
+                .withNameNode(nameNode)
+                .withPrepare(new PrepareBuilder()
+                        .withDelete("/path/to/delete")
+                        .withMkdir("/path/to/mkdir")
+                        .build())
+                .withLauncher(new LauncherBuilder()
+                        .withMemoryMb(1024)
+                        .withVCores(2)
+                        .withQueue("default")
+                        .withSharelib("default")
+                        .withViewAcl("default")
+                        .withModifyAcl("default")
+                        .build());
+
+        for (final String arg : args) {
+            builder.withArg(arg);
+        }
+
+        builder.withConfigProperty("propertyName1", "propertyValue1")
+                .withConfigProperty("propertyName2", "propertyValue2");
+
+        final PigAction action = builder.build();
+
+        final PIG pig = DozerBeanMapperSingleton.instance().map(action, PIG.class);
+
+        assertEquals(resourceManager, pig.getResourceManager());
+        assertEquals(nameNode, pig.getNameNode());
+        assertNotNull(pig.getPrepare());
+        assertEquals("/path/to/delete", pig.getPrepare().getDelete().get(0).getPath());
+        assertEquals("/path/to/mkdir", pig.getPrepare().getMkdir().get(0).getPath());
+        assertNotNull(pig.getConfiguration());
+        assertEquals(args, pig.getArgument());
+        assertEquals(1024L, pig.getLauncher().getMemoryMbOrVcoresOrJavaOpts().get(0).getValue());
+        assertEquals(2L, pig.getLauncher().getMemoryMbOrVcoresOrJavaOpts().get(1).getValue());
+        assertEquals("default", pig.getLauncher().getMemoryMbOrVcoresOrJavaOpts().get(2).getValue());
+        assertEquals("default", pig.getLauncher().getMemoryMbOrVcoresOrJavaOpts().get(3).getValue());
+        assertEquals("default", pig.getLauncher().getMemoryMbOrVcoresOrJavaOpts().get(4).getValue());
+        assertEquals("default", pig.getLauncher().getMemoryMbOrVcoresOrJavaOpts().get(5).getValue());
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestPipesMapping.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestPipesMapping.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestPipesMapping.java
new file mode 100644
index 0000000..9cbab8f
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestPipesMapping.java
@@ -0,0 +1,57 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.mapping;
+
+import org.apache.oozie.fluentjob.api.generated.workflow.PIPES;
+import org.apache.oozie.fluentjob.api.action.Pipes;
+import org.apache.oozie.fluentjob.api.action.PipesBuilder;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestPipesMapping {
+
+    @Test
+    public void testMappingPipes() {
+        final String map = "map";
+        final String reduce = "reduce";
+        final String inputformat = "inputformat";
+        final String partitioner = "partitioner";
+        final String writer = "writer";
+        final String program = "program";
+
+        final Pipes pipes = new PipesBuilder()
+                .withMap(map)
+                .withReduce(reduce)
+                .withInputformat(inputformat)
+                .withPartitioner(partitioner)
+                .withWriter(writer)
+                .withProgram(program)
+                .build();
+
+        final PIPES pipesJAXB = DozerBeanMapperSingleton.instance().map(pipes, PIPES.class);
+
+        assertEquals(map, pipesJAXB.getMap());
+        assertEquals(reduce, pipesJAXB.getReduce());
+        assertEquals(inputformat, pipesJAXB.getInputformat());
+        assertEquals(partitioner, pipesJAXB.getPartitioner());
+        assertEquals(writer, pipesJAXB.getWriter());
+        assertEquals(program, pipesJAXB.getProgram());
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestPrepareMapping.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestPrepareMapping.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestPrepareMapping.java
new file mode 100644
index 0000000..d4c343f
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestPrepareMapping.java
@@ -0,0 +1,65 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.mapping;
+
+import org.apache.oozie.fluentjob.api.generated.workflow.DELETE;
+import org.apache.oozie.fluentjob.api.generated.workflow.MKDIR;
+import org.apache.oozie.fluentjob.api.generated.workflow.PREPARE;
+import org.apache.oozie.fluentjob.api.action.Prepare;
+import org.apache.oozie.fluentjob.api.action.PrepareBuilder;
+import org.junit.Test;
+
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestPrepareMapping {
+
+    @Test
+    public void testMappingPrepare() {
+        final String deletePath1 = "path/to/delete/location1";
+        final String deletePath2 = "path/to/delete/location2";
+
+        final String mkdirPath1 = "path/to/mkdir/location1";
+        final String mkdirPath2 = "path/to/mkdir/location2";
+
+        final Prepare prepare = new PrepareBuilder()
+                .withDelete(deletePath1, false)
+                .withDelete(deletePath2, false)
+                .withMkdir(mkdirPath1)
+                .withMkdir(mkdirPath2)
+                .build();
+
+        final PREPARE prepareJAXB = DozerBeanMapperSingleton.instance().map(prepare, PREPARE.class);
+
+        final List<DELETE> deletesJAXB = prepareJAXB.getDelete();
+        final DELETE delete1JAXB = deletesJAXB.get(0);
+        final DELETE delete2JAXB = deletesJAXB.get(1);
+
+        final List<MKDIR> mkdirsJAXB = prepareJAXB.getMkdir();
+        final MKDIR mkdir1JAXB = mkdirsJAXB.get(0);
+        final MKDIR mkdir2JAXB = mkdirsJAXB.get(1);
+
+        assertEquals(deletePath1, delete1JAXB.getPath());
+        assertEquals(deletePath2, delete2JAXB.getPath());
+
+        assertEquals(mkdirPath1, mkdir1JAXB.getPath());
+        assertEquals(mkdirPath2, mkdir2JAXB.getPath());
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestShellActionMapping.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestShellActionMapping.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestShellActionMapping.java
new file mode 100644
index 0000000..20972ca
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestShellActionMapping.java
@@ -0,0 +1,92 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.mapping;
+
+import org.apache.oozie.fluentjob.api.action.LauncherBuilder;
+import org.apache.oozie.fluentjob.api.action.PrepareBuilder;
+import org.apache.oozie.fluentjob.api.action.ShellAction;
+import org.apache.oozie.fluentjob.api.action.ShellActionBuilder;
+import org.apache.oozie.fluentjob.api.generated.action.shell.ACTION;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+public class TestShellActionMapping {
+
+    public static final String DEFAULT = "default";
+
+    @Test
+    public void testMappingShellAction() {
+        final String resourceManager = "${resourceManager}";
+        final String nameNode = "${nameNode}";
+        final List<String> args = Arrays.asList("arg1", "arg2");
+
+        final ShellActionBuilder builder = ShellActionBuilder.create();
+
+        builder.withResourceManager(resourceManager)
+                .withNameNode(nameNode)
+                .withPrepare(new PrepareBuilder()
+                        .withDelete("/path/to/delete")
+                        .withMkdir("/path/to/mkdir")
+                        .build())
+                .withLauncher(new LauncherBuilder()
+                        .withMemoryMb(1024)
+                        .withVCores(2)
+                        .withQueue(DEFAULT)
+                        .withSharelib(DEFAULT)
+                        .withViewAcl(DEFAULT)
+                        .withModifyAcl(DEFAULT)
+                        .build())
+                .withExecutable(DEFAULT)
+                .withEnvironmentVariable(DEFAULT)
+                .withCaptureOutput(true);
+
+        for (final String arg : args) {
+            builder.withArgument(arg);
+        }
+
+        builder.withConfigProperty("propertyName1", "propertyValue1")
+                .withConfigProperty("propertyName2", "propertyValue2");
+
+        final ShellAction action = builder.build();
+
+        final ACTION shell = DozerBeanMapperSingleton.instance().map(action, ACTION.class);
+
+        assertEquals(resourceManager, shell.getResourceManager());
+        assertEquals(nameNode, shell.getNameNode());
+        assertNotNull(shell.getPrepare());
+        assertEquals("/path/to/delete", shell.getPrepare().getDelete().get(0).getPath());
+        assertEquals("/path/to/mkdir", shell.getPrepare().getMkdir().get(0).getPath());
+        assertNotNull(shell.getConfiguration());
+        assertEquals(args, shell.getArgument());
+        assertEquals(1024L, shell.getLauncher().getMemoryMbOrVcoresOrJavaOpts().get(0).getValue());
+        assertEquals(2L, shell.getLauncher().getMemoryMbOrVcoresOrJavaOpts().get(1).getValue());
+        assertEquals(DEFAULT, shell.getLauncher().getMemoryMbOrVcoresOrJavaOpts().get(2).getValue());
+        assertEquals(DEFAULT, shell.getLauncher().getMemoryMbOrVcoresOrJavaOpts().get(3).getValue());
+        assertEquals(DEFAULT, shell.getLauncher().getMemoryMbOrVcoresOrJavaOpts().get(4).getValue());
+        assertEquals(DEFAULT, shell.getLauncher().getMemoryMbOrVcoresOrJavaOpts().get(5).getValue());
+        assertEquals(DEFAULT, shell.getExec());
+        assertEquals(DEFAULT, shell.getEnvVar().get(0));
+        assertNotNull(shell.getCaptureOutput());
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestSparkActionMapping.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestSparkActionMapping.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestSparkActionMapping.java
new file mode 100644
index 0000000..2b003b7
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestSparkActionMapping.java
@@ -0,0 +1,98 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.mapping;
+
+import org.apache.oozie.fluentjob.api.action.LauncherBuilder;
+import org.apache.oozie.fluentjob.api.action.PrepareBuilder;
+import org.apache.oozie.fluentjob.api.action.SparkAction;
+import org.apache.oozie.fluentjob.api.action.SparkActionBuilder;
+import org.apache.oozie.fluentjob.api.generated.action.spark.ACTION;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+public class TestSparkActionMapping {
+
+    public static final String DEFAULT = "default";
+
+    @Test
+    public void testMappingSparkAction() {
+        final String resourceManager = "${resourceManager}";
+        final String nameNode = "${nameNode}";
+        final List<String> args = Arrays.asList("arg1", "arg2");
+
+        final SparkActionBuilder builder = SparkActionBuilder.create();
+
+        builder.withResourceManager(resourceManager)
+                .withNameNode(nameNode)
+                .withPrepare(new PrepareBuilder()
+                        .withDelete("/path/to/delete")
+                        .withMkdir("/path/to/mkdir")
+                        .build())
+                .withLauncher(new LauncherBuilder()
+                        .withMemoryMb(1024)
+                        .withVCores(2)
+                        .withQueue(DEFAULT)
+                        .withSharelib(DEFAULT)
+                        .withViewAcl(DEFAULT)
+                        .withModifyAcl(DEFAULT)
+                        .build())
+                .withMaster(DEFAULT)
+                .withMode(DEFAULT)
+                .withActionName(DEFAULT)
+                .withActionClass(DEFAULT)
+                .withJar(DEFAULT)
+                .withSparkOpts(DEFAULT);
+
+        for (final String arg : args) {
+            builder.withArg(arg);
+        }
+
+        builder.withConfigProperty("propertyName1", "propertyValue1")
+                .withConfigProperty("propertyName2", "propertyValue2");
+
+        final SparkAction action = builder.build();
+
+        final ACTION spark = DozerBeanMapperSingleton.instance().map(action, ACTION.class);
+
+        assertEquals(resourceManager, spark.getResourceManager());
+        assertEquals(nameNode, spark.getNameNode());
+        assertNotNull(spark.getPrepare());
+        assertEquals("/path/to/delete", spark.getPrepare().getDelete().get(0).getPath());
+        assertEquals("/path/to/mkdir", spark.getPrepare().getMkdir().get(0).getPath());
+        assertNotNull(spark.getConfiguration());
+        assertEquals(args, spark.getArg());
+        assertEquals(1024L, spark.getLauncher().getMemoryMbOrVcoresOrJavaOpts().get(0).getValue());
+        assertEquals(2L, spark.getLauncher().getMemoryMbOrVcoresOrJavaOpts().get(1).getValue());
+        assertEquals(DEFAULT, spark.getLauncher().getMemoryMbOrVcoresOrJavaOpts().get(2).getValue());
+        assertEquals(DEFAULT, spark.getLauncher().getMemoryMbOrVcoresOrJavaOpts().get(3).getValue());
+        assertEquals(DEFAULT, spark.getLauncher().getMemoryMbOrVcoresOrJavaOpts().get(4).getValue());
+        assertEquals(DEFAULT, spark.getLauncher().getMemoryMbOrVcoresOrJavaOpts().get(5).getValue());
+        assertEquals(DEFAULT, spark.getMaster());
+        assertEquals(DEFAULT, spark.getMode());
+        assertEquals(DEFAULT, spark.getName());
+        assertEquals(DEFAULT, spark.getClazz());
+        assertEquals(DEFAULT, spark.getJar());
+        assertEquals(DEFAULT, spark.getSparkOpts());
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestSqoopActionMapping.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestSqoopActionMapping.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestSqoopActionMapping.java
new file mode 100644
index 0000000..75348e8
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestSqoopActionMapping.java
@@ -0,0 +1,89 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.mapping;
+
+import org.apache.oozie.fluentjob.api.generated.action.sqoop.ACTION;
+import org.apache.oozie.fluentjob.api.action.LauncherBuilder;
+import org.apache.oozie.fluentjob.api.action.PrepareBuilder;
+import org.apache.oozie.fluentjob.api.action.SqoopAction;
+import org.apache.oozie.fluentjob.api.action.SqoopActionBuilder;
+import org.apache.oozie.fluentjob.api.generated.action.sqoop.ACTION;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+public class TestSqoopActionMapping {
+
+    public static final String DEFAULT = "default";
+
+    @Test
+    public void testMappingSqoopAction() {
+        final String resourceManager = "${resourceManager}";
+        final String nameNode = "${nameNode}";
+        final List<String> args = Arrays.asList("arg1", "arg2");
+
+        final SqoopActionBuilder builder = SqoopActionBuilder.create();
+
+        builder.withResourceManager(resourceManager)
+                .withNameNode(nameNode)
+                .withPrepare(new PrepareBuilder()
+                        .withDelete("/path/to/delete")
+                        .withMkdir("/path/to/mkdir")
+                        .build())
+                .withLauncher(new LauncherBuilder()
+                        .withMemoryMb(1024)
+                        .withVCores(2)
+                        .withQueue(DEFAULT)
+                        .withSharelib(DEFAULT)
+                        .withViewAcl(DEFAULT)
+                        .withModifyAcl(DEFAULT)
+                        .build())
+                .withCommand(DEFAULT);
+
+        for (final String arg : args) {
+            builder.withArgument(arg);
+        }
+
+        builder.withConfigProperty("propertyName1", "propertyValue1")
+                .withConfigProperty("propertyName2", "propertyValue2");
+
+        final SqoopAction action = builder.build();
+
+        final ACTION sqoop = DozerBeanMapperSingleton.instance().map(action, ACTION.class);
+
+        assertEquals(resourceManager, sqoop.getResourceManager());
+        assertEquals(nameNode, sqoop.getNameNode());
+        assertNotNull(sqoop.getPrepare());
+        assertEquals("/path/to/delete", sqoop.getPrepare().getDelete().get(0).getPath());
+        assertEquals("/path/to/mkdir", sqoop.getPrepare().getMkdir().get(0).getPath());
+        assertNotNull(sqoop.getConfiguration());
+        assertEquals(args, sqoop.getArg());
+        assertEquals(1024L, sqoop.getLauncher().getMemoryMbOrVcoresOrJavaOpts().get(0).getValue());
+        assertEquals(2L, sqoop.getLauncher().getMemoryMbOrVcoresOrJavaOpts().get(1).getValue());
+        assertEquals(DEFAULT, sqoop.getLauncher().getMemoryMbOrVcoresOrJavaOpts().get(2).getValue());
+        assertEquals(DEFAULT, sqoop.getLauncher().getMemoryMbOrVcoresOrJavaOpts().get(3).getValue());
+        assertEquals(DEFAULT, sqoop.getLauncher().getMemoryMbOrVcoresOrJavaOpts().get(4).getValue());
+        assertEquals(DEFAULT, sqoop.getLauncher().getMemoryMbOrVcoresOrJavaOpts().get(5).getValue());
+        assertEquals(DEFAULT, sqoop.getCommand());
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestSshActionMapping.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestSshActionMapping.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestSshActionMapping.java
new file mode 100644
index 0000000..e47fb1f
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestSshActionMapping.java
@@ -0,0 +1,59 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.mapping;
+
+import org.apache.oozie.fluentjob.api.action.SshAction;
+import org.apache.oozie.fluentjob.api.action.SshActionBuilder;
+import org.apache.oozie.fluentjob.api.generated.action.ssh.ACTION;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+public class TestSshActionMapping {
+
+    public static final String DEFAULT = "default";
+
+    @Test
+    public void testMappingSshAction() {
+        final List<String> args = Arrays.asList("arg1", "arg2");
+
+        final SshActionBuilder builder = SshActionBuilder.create();
+
+        builder.withHost(DEFAULT)
+                .withCommand(DEFAULT)
+                .withCaptureOutput(true);
+
+        for (final String arg : args) {
+            builder.withArg(arg);
+        }
+
+        final SshAction action = builder.build();
+
+        final ACTION ssh = DozerBeanMapperSingleton.instance().map(action, ACTION.class);
+
+        assertEquals(DEFAULT, ssh.getHost());
+        assertEquals(DEFAULT, ssh.getCommand());
+        assertEquals(args, ssh.getArgs());
+        assertNotNull(ssh.getCaptureOutput());
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestStartMapping.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestStartMapping.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestStartMapping.java
new file mode 100644
index 0000000..ffd477b
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestStartMapping.java
@@ -0,0 +1,61 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.mapping;
+
+import org.apache.oozie.fluentjob.api.generated.workflow.START;
+import org.apache.oozie.fluentjob.api.dag.Decision;
+import org.apache.oozie.fluentjob.api.dag.DecisionJoin;
+import org.apache.oozie.fluentjob.api.dag.ExplicitNode;
+import org.apache.oozie.fluentjob.api.dag.NodeBase;
+import org.apache.oozie.fluentjob.api.dag.Start;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestStartMapping extends TestControlNodeMappingBase {
+
+    @Test
+    public void testMappingStart() {
+        final String childName = "child";
+        final Start start = new Start("start");
+        final NodeBase child = new ExplicitNode(childName, null);
+
+        child.addParent(start);
+
+        final START mappedStart = DozerBeanMapperSingleton.instance().map(start, START.class);
+
+        assertEquals(childName, mappedStart.getTo());
+    }
+
+    @Test
+    public void testMappingStartWithDecisionJoin() {
+        final String childName = "child";
+        final Start start = new Start("start");
+
+        final NodeBase decisionJoin = new DecisionJoin("decisionJoin", new Decision("decision"));
+        decisionJoin.addParent(start);
+
+        final NodeBase child = new ExplicitNode(childName, null);
+        child.addParent(decisionJoin);
+
+        final START mappedStart = DozerBeanMapperSingleton.instance().map(start, START.class);
+
+        assertEquals(childName, mappedStart.getTo());
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestStreamingMapping.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestStreamingMapping.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestStreamingMapping.java
new file mode 100644
index 0000000..60509f5
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/mapping/TestStreamingMapping.java
@@ -0,0 +1,64 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.mapping;
+
+import org.apache.oozie.fluentjob.api.generated.workflow.STREAMING;
+import org.apache.oozie.fluentjob.api.action.Streaming;
+import org.apache.oozie.fluentjob.api.action.StreamingBuilder;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestStreamingMapping {
+
+    @Test
+    public void testMappingStreaming() {
+        final String mapper = "mapper";
+        final String reducer = "reducer";
+        final String recordReader = "recordReader";
+        final List<String> recordReaderMappings = Arrays.asList("mapping1", "mapping2");
+        final List<String> envs = Arrays.asList("env1", "env2");
+
+        final StreamingBuilder builder = new StreamingBuilder();
+        builder.withMapper(mapper)
+                .withReducer(reducer)
+                .withRecordReader(recordReader);
+
+        for (final String recordReaderMapping : recordReaderMappings) {
+            builder.withRecordReaderMapping(recordReaderMapping);
+        }
+
+        for (final String env : envs) {
+            builder.withEnv(env);
+        }
+
+        final Streaming streaming = builder.build();
+
+        final STREAMING streamingJAXB = DozerBeanMapperSingleton.instance().map(streaming, STREAMING.class);
+
+        assertEquals(mapper, streamingJAXB.getMapper());
+        assertEquals(reducer, streamingJAXB.getReducer());
+        assertEquals(recordReader, streamingJAXB.getRecordReader());
+        assertEquals(recordReaderMappings, streamingJAXB.getRecordReaderMapping());
+        assertEquals(envs, streamingJAXB.getEnv());
+    }
+}


[08/16] oozie git commit: OOZIE-2339 [fluent-job] Minimum Viable Fluent Job API (daniel.becker, andras.piros via rkanter, gezapeti, pbacsko)

Posted by an...@apache.org.
http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/workflow/Credential.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/workflow/Credential.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/workflow/Credential.java
new file mode 100644
index 0000000..4f199f5
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/workflow/Credential.java
@@ -0,0 +1,60 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.workflow;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+import java.util.List;
+
+/**
+ * Represents the {@code <credential>} element and its siblings inside workflow XML / XSD.
+ * <p>
+ * By assigning non-{@code null} field values, the resulting parent {@code <workflow>} will have its
+ * optional {@code <credential>} element and its siblings filled.
+ * <p>
+ * This class is used only as part of a {@link org.apache.oozie.fluentjob.api.workflow.Workflow}, isn't
+ * to be used alone with Jobs API.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class Credential {
+    private final String name;
+    private final String type;
+    private final ImmutableList<ConfigurationEntry> configurationEntries;
+
+    public Credential(final String name, final String type, final ImmutableList<ConfigurationEntry> configurationEntries) {
+        this.name = name;
+        this.type = type;
+        this.configurationEntries = configurationEntries;
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public String getType() {
+        return type;
+    }
+
+    public List<ConfigurationEntry> getConfigurationEntries() {
+        return configurationEntries;
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/workflow/CredentialBuilder.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/workflow/CredentialBuilder.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/workflow/CredentialBuilder.java
new file mode 100644
index 0000000..098bc13
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/workflow/CredentialBuilder.java
@@ -0,0 +1,89 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.workflow;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.oozie.fluentjob.api.ModifyOnce;
+import org.apache.oozie.fluentjob.api.action.Builder;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * A builder class for {@link Credential}.
+ * <p>
+ * The properties of the builder can only be set once, an attempt to set them a second time will trigger
+ * an {@link IllegalStateException}. The properties that are lists are an exception to this rule, of course multiple
+ * elements can be added / removed.
+ * <p>
+ * Builder instances can be used to build several elements, although properties already set cannot be changed after
+ * a call to {@link CredentialBuilder#build} either.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class CredentialBuilder implements Builder<Credential> {
+    private final ModifyOnce<String> name;
+    private final ModifyOnce<String> type;
+    private final List<ConfigurationEntry> configurationEntries;
+
+    public static CredentialBuilder create() {
+        final ModifyOnce<String> name = new ModifyOnce<>();
+        final ModifyOnce<String> type = new ModifyOnce<>();
+        final List<ConfigurationEntry> configurationEntries = new ArrayList<>();
+
+        return new CredentialBuilder(name, type, configurationEntries);
+    }
+
+    public static CredentialBuilder createFromExisting(final Credential credential) {
+        return new CredentialBuilder(new ModifyOnce<>(credential.getName()),
+                new ModifyOnce<>(credential.getType()),
+                new ArrayList<>(credential.getConfigurationEntries()));
+    }
+
+    private CredentialBuilder(final ModifyOnce<String> name,
+            final ModifyOnce<String> type,
+            final List<ConfigurationEntry> configurationEntries) {
+        this.name = name;
+        this.type = type;
+        this.configurationEntries = configurationEntries;
+    }
+
+    public CredentialBuilder withName(final String name) {
+        this.name.set(name);
+        return this;
+    }
+
+    public CredentialBuilder withType(final String type) {
+        this.type.set(type);
+        return this;
+    }
+
+    public CredentialBuilder withConfigurationEntry(final String name,
+                                                    final String description) {
+        this.configurationEntries.add(new ConfigurationEntry(name, description));
+        return this;
+    }
+
+    @Override
+    public Credential build() {
+        return new Credential(name.get(), type.get(), ImmutableList.copyOf(configurationEntries));
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/workflow/Credentials.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/workflow/Credentials.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/workflow/Credentials.java
new file mode 100644
index 0000000..319839b
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/workflow/Credentials.java
@@ -0,0 +1,48 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.workflow;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+import java.util.List;
+
+/**
+ * Represents the {@code <credentials>} element and its siblings inside workflow XML / XSD.
+ * <p>
+ * By assigning non-{@code null} field values, the resulting parent {@code <workflow>} will have its
+ * optional {@code <credentials>} element and its siblings filled.
+ * <p>
+ * This class is used only as part of a {@link org.apache.oozie.fluentjob.api.workflow.Workflow}, isn't
+ * to be used alone with Jobs API.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class Credentials {
+    private final ImmutableList<Credential> credentials;
+
+    public Credentials(final ImmutableList<Credential> credentials) {
+        this.credentials = credentials;
+    }
+
+    public List<Credential> getCredentials() {
+        return credentials;
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/workflow/CredentialsBuilder.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/workflow/CredentialsBuilder.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/workflow/CredentialsBuilder.java
new file mode 100644
index 0000000..980106c
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/workflow/CredentialsBuilder.java
@@ -0,0 +1,77 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.workflow;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.oozie.fluentjob.api.action.Builder;
+
+import java.util.List;
+
+/**
+ * A builder class for {@link Credentials}.
+ *
+ * The properties of the builder can only be set once, an attempt to set them a second time will trigger
+ * an {@link IllegalStateException}. The properties that are lists are an exception to this rule, of course multiple
+ * elements can be added / removed.
+ *
+ * Builder instances can be used to build several elements, although properties already set cannot be changed after
+ * a call to {@link CredentialsBuilder#build} either.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class CredentialsBuilder implements Builder<Credentials> {
+    private final ImmutableList.Builder<Credential> credentials;
+
+    public static CredentialsBuilder create() {
+        return new CredentialsBuilder(new ImmutableList.Builder<Credential>());
+    }
+
+    public static CredentialsBuilder createFromExisting(final Credentials credentials) {
+        return new CredentialsBuilder(new ImmutableList.Builder<Credential>().addAll(credentials.getCredentials()));
+    }
+
+    CredentialsBuilder(final ImmutableList.Builder<Credential> credentials) {
+        this.credentials = credentials;
+    }
+
+    public CredentialsBuilder withCredential(final String name,
+                                             final String value) {
+        this.credentials.add(new Credential(name, value, ImmutableList.<ConfigurationEntry>of()));
+        return this;
+    }
+
+    public CredentialsBuilder withCredential(final String name,
+                                             final String type,
+                                             final List<ConfigurationEntry> configurationEntries) {
+        this.credentials.add(new Credential(name, type, ImmutableList.copyOf(configurationEntries)));
+        return this;
+    }
+
+    public CredentialsBuilder withCredential(final Credential credential) {
+        this.credentials.add(credential);
+        return this;
+    }
+
+    @Override
+    public Credentials build() {
+        return new Credentials(credentials.build());
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/workflow/Global.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/workflow/Global.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/workflow/Global.java
new file mode 100644
index 0000000..49ae1de
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/workflow/Global.java
@@ -0,0 +1,67 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.workflow;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.oozie.fluentjob.api.action.ActionAttributes;
+import org.apache.oozie.fluentjob.api.action.HasAttributes;
+import org.apache.oozie.fluentjob.api.action.Launcher;
+
+import java.util.List;
+import java.util.Map;
+
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class Global implements HasAttributes {
+    private final ActionAttributes attributes;
+
+    Global(final ActionAttributes attributes) {
+        this.attributes = attributes;
+    }
+
+    public String getResourceManager() {
+        return attributes.getResourceManager();
+    }
+
+    public String getNameNode() {
+        return attributes.getNameNode();
+    }
+
+    public Launcher getLauncher() {
+        return attributes.getLauncher();
+    }
+
+    public List<String> getJobXmls() {
+        return attributes.getJobXmls();
+    }
+
+    public String getConfigProperty(final String property) {
+        return attributes.getConfiguration().get(property);
+    }
+
+    public Map<String, String> getConfiguration() {
+        return attributes.getConfiguration();
+    }
+
+    @Override
+    public ActionAttributes getAttributes() {
+        return attributes;
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/workflow/GlobalBuilder.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/workflow/GlobalBuilder.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/workflow/GlobalBuilder.java
new file mode 100644
index 0000000..8a80705
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/workflow/GlobalBuilder.java
@@ -0,0 +1,94 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.workflow;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.oozie.fluentjob.api.action.ActionAttributesBuilder;
+import org.apache.oozie.fluentjob.api.action.Builder;
+import org.apache.oozie.fluentjob.api.action.HasAttributes;
+import org.apache.oozie.fluentjob.api.action.Launcher;
+
+/**
+ * A builder class for {@link Global}.
+ *
+ * The properties of the builder can only be set once, an attempt to set them a second time will trigger
+ * an {@link IllegalStateException}. The properties that are lists are an exception to this rule, of course multiple
+ * elements can be added / removed.
+ *
+ * Builder instances can be used to build several elements, although properties already set cannot be changed after
+ * a call to {@link GlobalBuilder#build} either.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class GlobalBuilder implements Builder<Global> {
+    private final ActionAttributesBuilder attributesBuilder;
+
+    public static GlobalBuilder create() {
+        return new GlobalBuilder(ActionAttributesBuilder.create());
+    }
+
+    public static GlobalBuilder createFromExisting(final HasAttributes hasAttributes) {
+        return new GlobalBuilder(ActionAttributesBuilder.createFromExisting(hasAttributes.getAttributes()));
+    }
+
+    private GlobalBuilder(final ActionAttributesBuilder attributesBuilder) {
+        this.attributesBuilder = attributesBuilder;
+    }
+
+    public GlobalBuilder withResourceManager(final String resourceManager) {
+        attributesBuilder.withResourceManager(resourceManager);
+        return this;
+    }
+
+    public GlobalBuilder withNameNode(final String nameNode) {
+        attributesBuilder.withNameNode(nameNode);
+        return this;
+    }
+
+    public GlobalBuilder withLauncher(final Launcher launcher) {
+        attributesBuilder.withLauncher(launcher);
+        return this;
+    }
+
+    public GlobalBuilder withJobXml(final String jobXml) {
+        attributesBuilder.withJobXml(jobXml);
+        return this;
+    }
+
+    public GlobalBuilder withoutJobXml(final String jobXml) {
+        attributesBuilder.withoutJobXml(jobXml);
+        return this;
+    }
+
+    public GlobalBuilder clearJobXmls() {
+        attributesBuilder.clearJobXmls();
+        return this;
+    }
+
+    public GlobalBuilder withConfigProperty(final String key, final String value) {
+        attributesBuilder.withConfigProperty(key, value);
+        return this;
+    }
+
+    @Override
+    public Global build() {
+        return new Global(attributesBuilder.build());
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/workflow/Parameter.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/workflow/Parameter.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/workflow/Parameter.java
new file mode 100644
index 0000000..fb309d3
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/workflow/Parameter.java
@@ -0,0 +1,57 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.workflow;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Represents the {@code <parameter>} element and its siblings inside workflow XML / XSD.
+ * <p>
+ * By assigning non-{@code null} field values, the resulting parent {@code <workflow>} will have its
+ * optional {@code <parameter>} element and its siblings filled.
+ * <p>
+ * This class is used only as part of a {@link org.apache.oozie.fluentjob.api.workflow.Workflow}, isn't
+ * to be used alone with Jobs API.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class Parameter {
+    private final String name;
+    private final String value;
+    private final String description;
+
+    public Parameter(final String name, final String value, final String description) {
+        this.name = name;
+        this.value = value;
+        this.description = description;
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public String getValue() {
+        return value;
+    }
+
+    public String getDescription() {
+        return description;
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/workflow/Parameters.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/workflow/Parameters.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/workflow/Parameters.java
new file mode 100644
index 0000000..e34dd3c
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/workflow/Parameters.java
@@ -0,0 +1,48 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.workflow;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+import java.util.List;
+
+/**
+ * Represents the {@code <parameters>} element and its siblings inside workflow XML / XSD.
+ * <p>
+ * By assigning non-{@code null} field values, the resulting parent {@code <workflow>} will have its
+ * optional {@code <parameters>} element and its siblings filled.
+ * <p>
+ * This class is used only as part of a {@link org.apache.oozie.fluentjob.api.workflow.Workflow}, isn't
+ * to be used alone with Jobs API.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class Parameters {
+    private final ImmutableList<Parameter> parameters;
+
+    public Parameters(final ImmutableList<Parameter> parameters) {
+        this.parameters = parameters;
+    }
+
+    public List<Parameter> getParameters() {
+        return parameters;
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/workflow/ParametersBuilder.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/workflow/ParametersBuilder.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/workflow/ParametersBuilder.java
new file mode 100644
index 0000000..d522e90
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/workflow/ParametersBuilder.java
@@ -0,0 +1,66 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.workflow;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.oozie.fluentjob.api.action.Builder;
+
+/**
+ * A builder class for {@link Parameters}.
+ *
+ * The properties of the builder can only be set once, an attempt to set them a second time will trigger
+ * an {@link IllegalStateException}. The properties that are lists are an exception to this rule, of course multiple
+ * elements can be added / removed.
+ *
+ * Builder instances can be used to build several elements, although properties already set cannot be changed after
+ * a call to {@link ParametersBuilder#build} either.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class ParametersBuilder implements Builder<Parameters> {
+    private final ImmutableList.Builder<Parameter> parameters;
+
+    public static ParametersBuilder create() {
+        return new ParametersBuilder(new ImmutableList.Builder<Parameter>());
+    }
+
+    public static ParametersBuilder createFromExisting(final Parameters parameters) {
+        return new ParametersBuilder(new ImmutableList.Builder<Parameter>().addAll(parameters.getParameters()));
+    }
+
+    ParametersBuilder(final ImmutableList.Builder<Parameter> parameters) {
+        this.parameters = parameters;
+    }
+
+    public ParametersBuilder withParameter(final String name, final String value) {
+        return withParameter(name, value, null);
+    }
+
+    public ParametersBuilder withParameter(final String name, final String value, final String description) {
+        parameters.add(new Parameter(name, value, description));
+        return this;
+    }
+
+    @Override
+    public Parameters build() {
+        return new Parameters(parameters.build());
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/workflow/Workflow.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/workflow/Workflow.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/workflow/Workflow.java
new file mode 100644
index 0000000..7a4ee02
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/workflow/Workflow.java
@@ -0,0 +1,141 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.workflow;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Sets;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.oozie.fluentjob.api.action.Node;
+import org.apache.oozie.fluentjob.api.serialization.WorkflowMarshaller;
+
+import javax.xml.bind.JAXBException;
+import java.io.UnsupportedEncodingException;
+import java.util.HashSet;
+import java.util.LinkedHashSet;
+import java.util.LinkedList;
+import java.util.Queue;
+import java.util.Set;
+
+/**
+ * Being the topmost POJO in the Jobs API hierarchy, represents the {@code <workflow>} element
+ * and its siblings inside workflow XML / XSD.
+ * <p>
+ * By assigning non-{@code null} field values, the resulting {@code <workflow>} will have all its
+ * siblings filled.
+ * <p>
+ * This class is the root container POJO from a user's perspective to Jobs API.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class Workflow {
+    private final String name;
+    private final ImmutableSet<Node> nodes;
+    private final ImmutableSet<Node> roots;
+    private final Parameters parameters;
+    private final Global global;
+    private final Credentials credentials;
+
+    Workflow(final String name,
+             final ImmutableSet<Node> nodes,
+             final Parameters parameters,
+             final Global global,
+             final Credentials credentials) {
+        this.global = global;
+        checkUniqueNames(nodes);
+
+        this.name = name;
+        this.nodes = nodes;
+
+        final Set<Node> mutableRoots = findMutableRoots(nodes);
+
+        this.roots = ImmutableSet.copyOf(mutableRoots);
+
+        this.parameters = parameters;
+
+        this.credentials = credentials;
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public Set<Node> getNodes() {
+        return nodes;
+    }
+
+    public Set<Node> getAllNodes() {
+        final Set<Node> allNodes = Sets.newLinkedHashSet();
+        final Queue<Node> nodesToAdd = new LinkedList<>();
+        nodesToAdd.addAll(nodes);
+
+        while (!nodesToAdd.isEmpty()) {
+            final Node nodeToAdd = nodesToAdd.poll();
+            allNodes.add(nodeToAdd);
+
+            nodesToAdd.addAll(nodeToAdd.getAllChildren());
+        }
+
+        return ImmutableSet.copyOf(allNodes);
+    }
+
+    public Set<Node> getRoots() {
+        return roots;
+    }
+
+    public Parameters getParameters() {
+        return parameters;
+    }
+
+    public Global getGlobal() {
+        return global;
+    }
+
+    public Credentials getCredentials() {
+        return credentials;
+    }
+
+    private void checkUniqueNames(final Set<Node> nodes) {
+        final Set<String> names = new HashSet<>();
+
+        for (final Node node : nodes) {
+            Preconditions.checkArgument(!names.contains(node.getName()),
+                    String.format("Duplicate name '%s' found in workflow '%s'", node.getName(), getName()));
+
+            names.add(node.getName());
+        }
+    }
+
+    private Set<Node> findMutableRoots(ImmutableSet<Node> nodes) {
+        final Set<Node> mutableRoots = new LinkedHashSet<>();
+
+        for (final Node node : nodes) {
+            if (node.getAllParents().isEmpty()) {
+                mutableRoots.add(node);
+            }
+        }
+
+        return mutableRoots;
+    }
+
+    public String asXml() throws JAXBException, UnsupportedEncodingException {
+        return WorkflowMarshaller.marshal(this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/workflow/WorkflowBuilder.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/workflow/WorkflowBuilder.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/workflow/WorkflowBuilder.java
new file mode 100644
index 0000000..c3e588c
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/workflow/WorkflowBuilder.java
@@ -0,0 +1,204 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.workflow;
+
+import com.google.common.base.Strings;
+import com.google.common.collect.ImmutableSet;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.oozie.fluentjob.api.ModifyOnce;
+import org.apache.oozie.fluentjob.api.action.Builder;
+import org.apache.oozie.fluentjob.api.action.Node;
+
+import java.security.SecureRandom;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Queue;
+import java.util.Set;
+
+/**
+ * A builder class for {@link Workflow}.
+ * <p>
+ * The properties of the builder can only be set once, an attempt to set them a second time will trigger
+ * an {@link IllegalStateException}. The properties that are lists are an exception to this rule, of course multiple
+ * elements can be added / removed.
+ * <p>
+ * Builder instances can be used to build several elements, although properties already set cannot be changed after
+ * a call to {@link WorkflowBuilder#build} either.
+ * <p>
+ * Being the topmost {@link Builder} from the user's perspective, it serves as a main entry point to create
+ * Workflow API POJOs.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class WorkflowBuilder implements Builder<Workflow> {
+    private final ModifyOnce<String> name;
+    private final List<Node> addedActions;
+    private ParametersBuilder parametersBuilder;
+    private GlobalBuilder globalBuilder;
+    private CredentialsBuilder credentialsBuilder;
+
+    public WorkflowBuilder() {
+        this.name = new ModifyOnce<>();
+        this.addedActions = new ArrayList<>();
+    }
+
+    public WorkflowBuilder withName(final String name) {
+        this.name.set(name);
+        return this;
+    }
+
+    public WorkflowBuilder withDagContainingNode(final Node node) {
+        this.addedActions.add(node);
+        return this;
+    }
+
+    public WorkflowBuilder withParameter(final String name, final String value) {
+        ensureParametersBuilder();
+        this.parametersBuilder.withParameter(name, value);
+        return this;
+    }
+
+    public WorkflowBuilder withParameter(final String name, final String value, final String description) {
+        this.parametersBuilder.withParameter(name, value, description);
+        return this;
+    }
+
+    private void ensureParametersBuilder() {
+        if (this.parametersBuilder == null) {
+            this.parametersBuilder = ParametersBuilder.create();
+        }
+    }
+
+    public WorkflowBuilder withGlobal(final Global global) {
+        this.globalBuilder = GlobalBuilder.createFromExisting(global);
+        return this;
+    }
+
+    /**
+     * Adds a {@link Credentials} to the {@link Workflow}.
+     * <p>
+     * Note that an explicit call to {@code WorkflowBuilder#withCredentials(Credentials)} can be omitted, since if
+     * {@link WorkflowBuilder#credentialsBuilder} is emtpy by the time {@link WorkflowBuilder#build()} is called,
+     * {@link Workflow#credentials} is built based on all the {@link Node#getCredentials()} that have been added to that
+     * {@code Workflow} in beforehand.
+     * <p>
+     * Note also that when {@code WorkflowBuilder#withCredentials(Credentials)} is explicitly called, the {@code <workflowapp />}'s
+     * {@code <credential />} section is generated only by using the {@code Credentials} defined on the {@code Workflow} level.
+     * <p>
+     * This way, users can, if they want to, omit calling {@code WorkflowBuilder#withCredentials(Credentials)} by default, but can
+     * also override the autogenerated {@code <credentials />} section of {@code <workflowapp />} by explicitly calling that method
+     * after another call to {@link CredentialsBuilder#build()}.
+     * @param credentials the {@code Credentials} to add, if want to override.
+     * @return this
+     */
+    public WorkflowBuilder withCredentials(final Credentials credentials) {
+        this.credentialsBuilder = CredentialsBuilder.createFromExisting(credentials);
+        return this;
+    }
+
+    @Override
+    public Workflow build() {
+        ensureName();
+
+        final Set<Node> nodes = new HashSet<>();
+        for (final Node node : addedActions) {
+            if (!nodes.contains(node)) {
+                nodes.addAll(getNodesInDag(node));
+            }
+        }
+
+        final ImmutableSet.Builder<Node> builder = new ImmutableSet.Builder<>();
+        builder.addAll(nodes);
+
+        final Parameters parameters;
+        if (parametersBuilder != null) {
+            parameters = parametersBuilder.build();
+        }
+        else {
+            parameters = null;
+        }
+
+        final Global global;
+        if (globalBuilder != null) {
+            global = globalBuilder.build();
+        }
+        else {
+            global = null;
+        }
+
+        final Credentials credentials;
+        if (credentialsBuilder != null) {
+            credentials = credentialsBuilder.build();
+        }
+        else {
+            final CredentialsBuilder actionCredentialsBuilder = CredentialsBuilder.create();
+            for (final Node action : addedActions) {
+                for (final Credential actionCredential : action.getCredentials()) {
+                    actionCredentialsBuilder.withCredential(actionCredential);
+                }
+            }
+
+            final Credentials actionCredentials = actionCredentialsBuilder.build();
+            if (actionCredentials.getCredentials().size() > 0) {
+                credentials = actionCredentialsBuilder.build();
+            } else {
+                credentials = null;
+            }
+        }
+
+        return new Workflow(name.get(), builder.build(), parameters, global, credentials);
+    }
+
+    private void ensureName() {
+        if (Strings.isNullOrEmpty(this.name.get())) {
+            final String type = "workflow";
+            final int randomSuffix = new SecureRandom().nextInt(1_000_000_000);
+
+            this.name.set(String.format("%s-%d", type, randomSuffix));
+        }
+    }
+
+    private static Set<Node> getNodesInDag(final Node node) {
+        final Set<Node> visited = new HashSet<>();
+        final Queue<Node> queue = new ArrayDeque<>();
+        visited.add(node);
+        queue.add(node);
+
+        Node current;
+        while ((current = queue.poll()) != null) {
+            visit(current.getAllParents(), visited, queue);
+            visit(current.getAllChildren(), visited, queue);
+        }
+
+        return visited;
+    }
+
+    // TODO: encapsulate into a more specific nested class, e.g. DagWalker#walk
+    private static void visit(final List<Node> toVisit, final Set<Node> visited, final Queue<Node> queue) {
+        for (final Node node : toVisit) {
+            if (!visited.contains(node)) {
+                visited.add(node);
+                queue.add(node);
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/resources/action_mappings.xml
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/resources/action_mappings.xml b/fluent-job/fluent-job-api/src/main/resources/action_mappings.xml
new file mode 100644
index 0000000..a5f890e
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/resources/action_mappings.xml
@@ -0,0 +1,821 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you under the Apache License, Version 2.0 (the
+  "License"); you may not use this file except in compliance
+  with the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+
+<mappings xmlns="http://dozer.sourceforge.net"
+          xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+          xsi:schemaLocation="http://dozer.sourceforge.net
+          http://dozer.sourceforge.net/schema/beanmapping.xsd">
+
+    <mapping type="one-way">
+        <class-a>org.apache.oozie.fluentjob.api.action.Delete</class-a>
+        <class-b>org.apache.oozie.fluentjob.api.generated.workflow.DELETE</class-b>
+
+        <field>
+            <a>path</a>
+            <b>path</b>
+        </field>
+    </mapping>
+
+    <mapping type="one-way">
+        <class-a>org.apache.oozie.fluentjob.api.action.Mkdir</class-a>
+        <class-b>org.apache.oozie.fluentjob.api.generated.workflow.MKDIR</class-b>
+
+        <field>
+            <a>path</a>
+            <b>path</b>
+        </field>
+    </mapping>
+
+    <mapping type="one-way">
+        <class-a>org.apache.oozie.fluentjob.api.action.Prepare</class-a>
+        <class-b>org.apache.oozie.fluentjob.api.generated.workflow.PREPARE</class-b>
+
+        <field>
+            <a is-accessible="true">deletes</a>
+            <b is-accessible="true">delete</b>
+        </field>
+
+        <field>
+            <a is-accessible="true">mkdirs</a>
+            <b is-accessible="true">mkdir</b>
+        </field>
+    </mapping>
+
+    <mapping type="one-way">
+        <class-a>org.apache.oozie.fluentjob.api.action.Streaming</class-a>
+        <class-b>org.apache.oozie.fluentjob.api.generated.workflow.STREAMING</class-b>
+
+        <field>
+            <a>mapper</a>
+            <b>mapper</b>
+        </field>
+
+        <field>
+            <a>reducer</a>
+            <b>reducer</b>
+        </field>
+
+        <field>
+            <a>recordReader</a>
+            <b>recordReader</b>
+        </field>
+
+        <field>
+            <a>recordReaderMappings</a>
+            <b>recordReaderMapping</b>
+        </field>
+
+        <field>
+            <a>envs</a>
+            <b>env</b>
+        </field>
+    </mapping>
+
+    <mapping type="one-way">
+        <class-a>org.apache.oozie.fluentjob.api.action.Pipes</class-a>
+        <class-b>org.apache.oozie.fluentjob.api.generated.workflow.PIPES</class-b>
+
+        <field>
+            <a>map</a>
+            <b>map</b>
+        </field>
+
+        <field>
+            <a>reduce</a>
+            <b>reduce</b>
+        </field>
+
+        <field>
+            <a>inputformat</a>
+            <b>inputformat</b>
+        </field>
+
+        <field>
+            <a>partitioner</a>
+            <b>partitioner</b>
+        </field>
+
+        <field>
+            <a>writer</a>
+            <b>writer</b>
+        </field>
+
+        <field>
+            <a>program</a>
+            <b>program</b>
+        </field>
+    </mapping>
+
+    <mapping type="one-way">
+        <class-a>org.apache.oozie.fluentjob.api.action.MapReduceAction</class-a>
+        <class-b>org.apache.oozie.fluentjob.api.generated.workflow.MAPREDUCE</class-b>
+
+        <field>
+            <a>resourceManager</a>
+            <b>resourceManager</b>
+        </field>
+
+        <field>
+            <a>nameNode</a>
+            <b>nameNode</b>
+        </field>
+
+        <field>
+            <a>prepare</a>
+            <b>prepare</b>
+        </field>
+
+        <field>
+            <a>streaming</a>
+            <b>streaming</b>
+        </field>
+
+        <field>
+            <a>pipes</a>
+            <b>pipes</b>
+        </field>
+
+        <field>
+            <a>jobXmls</a>
+            <b>jobXml</b>
+        </field>
+
+        <field>
+            <a>configuration</a>
+            <b>configuration</b>
+        </field>
+
+        <field>
+            <a>configClass</a>
+            <b>configClass</b>
+        </field>
+
+        <field>
+            <a>files</a>
+            <b>file</b>
+        </field>
+
+        <field>
+            <a>archives</a>
+            <b>archive</b>
+        </field>
+    </mapping>
+
+    <mapping type="one-way">
+        <class-a>org.apache.oozie.fluentjob.api.action.DistcpAction</class-a>
+        <class-b>org.apache.oozie.fluentjob.api.generated.action.distcp.ACTION</class-b>
+
+        <field>
+            <a>resourceManager</a>
+            <b>resourceManager</b>
+        </field>
+
+        <field>
+            <a>nameNode</a>
+            <b>nameNode</b>
+        </field>
+
+        <field custom-converter="org.apache.oozie.fluentjob.api.mapping.DistcpPrepareConverter">
+            <a>prepare</a>
+            <b>prepare</b>
+        </field>
+
+        <field custom-converter="org.apache.oozie.fluentjob.api.mapping.DistcpConfigurationConverter">
+            <a>configuration</a>
+            <b>configuration</b>
+        </field>
+
+        <field>
+            <a>javaOpts</a>
+            <b>javaOpts</b>
+        </field>
+
+        <field>
+            <a>args</a>
+            <b>arg</b>
+        </field>
+    </mapping>
+
+    <mapping type="one-way">
+        <class-a>org.apache.oozie.fluentjob.api.action.HiveAction</class-a>
+        <class-b>org.apache.oozie.fluentjob.api.generated.action.hive.ACTION</class-b>
+
+        <field>
+            <a>resourceManager</a>
+            <b>resourceManager</b>
+        </field>
+
+        <field>
+            <a>nameNode</a>
+            <b>nameNode</b>
+        </field>
+
+        <field custom-converter="org.apache.oozie.fluentjob.api.mapping.HivePrepareConverter">
+            <a>prepare</a>
+            <b>prepare</b>
+        </field>
+
+        <field custom-converter="org.apache.oozie.fluentjob.api.mapping.HiveLauncherConverter">
+            <a>launcher</a>
+            <b>launcher</b>
+        </field>
+
+        <field custom-converter="org.apache.oozie.fluentjob.api.mapping.HiveConfigurationConverter">
+            <a>configuration</a>
+            <b>configuration</b>
+        </field>
+
+        <field>
+            <a>jobXmls</a>
+            <b>jobXml</b>
+        </field>
+
+        <field>
+            <a>script</a>
+            <b>script</b>
+        </field>
+
+        <field>
+            <a>query</a>
+            <b>query</b>
+        </field>
+
+        <field>
+            <a>params</a>
+            <b>param</b>
+        </field>
+
+        <field>
+            <a>args</a>
+            <b>argument</b>
+        </field>
+
+        <field>
+            <a>files</a>
+            <b>file</b>
+        </field>
+
+        <field>
+            <a>archives</a>
+            <b>archive</b>
+        </field>
+    </mapping>
+
+    <mapping type="one-way">
+        <class-a>org.apache.oozie.fluentjob.api.action.Hive2Action</class-a>
+        <class-b>org.apache.oozie.fluentjob.api.generated.action.hive2.ACTION</class-b>
+
+        <field>
+            <a>resourceManager</a>
+            <b>resourceManager</b>
+        </field>
+
+        <field>
+            <a>nameNode</a>
+            <b>nameNode</b>
+        </field>
+
+        <field custom-converter="org.apache.oozie.fluentjob.api.mapping.Hive2PrepareConverter">
+            <a>prepare</a>
+            <b>prepare</b>
+        </field>
+
+        <field custom-converter="org.apache.oozie.fluentjob.api.mapping.Hive2LauncherConverter">
+            <a>launcher</a>
+            <b>launcher</b>
+        </field>
+
+        <field custom-converter="org.apache.oozie.fluentjob.api.mapping.Hive2ConfigurationConverter">
+            <a>configuration</a>
+            <b>configuration</b>
+        </field>
+
+        <field>
+            <a>jobXmls</a>
+            <b>jobXml</b>
+        </field>
+
+        <field>
+            <a>jdbcUrl</a>
+            <b>jdbcUrl</b>
+        </field>
+
+        <field>
+            <a>password</a>
+            <b>password</b>
+        </field>
+
+        <field>
+            <a>script</a>
+            <b>script</b>
+        </field>
+
+        <field>
+            <a>query</a>
+            <b>query</b>
+        </field>
+
+        <field>
+            <a>params</a>
+            <b>param</b>
+        </field>
+
+        <field>
+            <a>args</a>
+            <b>argument</b>
+        </field>
+
+        <field>
+            <a>files</a>
+            <b>file</b>
+        </field>
+
+        <field>
+            <a>archives</a>
+            <b>archive</b>
+        </field>
+    </mapping>
+
+    <mapping type="one-way">
+        <class-a>org.apache.oozie.fluentjob.api.action.JavaAction</class-a>
+        <class-b>org.apache.oozie.fluentjob.api.generated.workflow.JAVA</class-b>
+
+        <field>
+            <a>resourceManager</a>
+            <b>resourceManager</b>
+        </field>
+
+        <field>
+            <a>nameNode</a>
+            <b>nameNode</b>
+        </field>
+
+        <field custom-converter="org.apache.oozie.fluentjob.api.mapping.InlineWorkflowPrepareConverter">
+            <a>prepare</a>
+            <b>prepare</b>
+        </field>
+
+        <field custom-converter="org.apache.oozie.fluentjob.api.mapping.InlineWorkflowLauncherConverter">
+            <a>launcher</a>
+            <b>launcher</b>
+        </field>
+
+        <field>
+            <a>jobXmls</a>
+            <b>jobXml</b>
+        </field>
+
+        <field custom-converter="org.apache.oozie.fluentjob.api.mapping.InlineWorkflowConfigurationConverter">
+            <a>configuration</a>
+            <b>configuration</b>
+        </field>
+
+        <field>
+            <a>mainClass</a>
+            <b>mainClass</b>
+        </field>
+
+        <field>
+            <a>javaOptsString</a>
+            <b>javaOpts</b>
+        </field>
+
+        <field>
+            <a>javaOpts</a>
+            <b>javaOpt</b>
+        </field>
+
+        <field>
+            <a>args</a>
+            <b>arg</b>
+        </field>
+
+        <field>
+            <a>files</a>
+            <b>file</b>
+        </field>
+
+        <field>
+            <a>archives</a>
+            <b>archive</b>
+        </field>
+
+        <field custom-converter="org.apache.oozie.fluentjob.api.mapping.BooleanToFLAGConverter">
+            <a get-method="isCaptureOutput">captureOutput</a>
+            <b>captureOutput</b>
+        </field>
+    </mapping>
+
+    <mapping type="one-way">
+        <class-a>org.apache.oozie.fluentjob.api.action.PigAction</class-a>
+        <class-b>org.apache.oozie.fluentjob.api.generated.workflow.PIG</class-b>
+
+        <field>
+            <a>resourceManager</a>
+            <b>resourceManager</b>
+        </field>
+
+        <field>
+            <a>nameNode</a>
+            <b>nameNode</b>
+        </field>
+
+        <field custom-converter="org.apache.oozie.fluentjob.api.mapping.InlineWorkflowPrepareConverter">
+            <a>prepare</a>
+            <b>prepare</b>
+        </field>
+
+        <field custom-converter="org.apache.oozie.fluentjob.api.mapping.InlineWorkflowLauncherConverter">
+            <a>launcher</a>
+            <b>launcher</b>
+        </field>
+
+        <field custom-converter="org.apache.oozie.fluentjob.api.mapping.InlineWorkflowConfigurationConverter">
+            <a>configuration</a>
+            <b>configuration</b>
+        </field>
+
+        <field>
+            <a>jobXmls</a>
+            <b>jobXml</b>
+        </field>
+
+        <field>
+            <a>script</a>
+            <b>script</b>
+        </field>
+
+        <field>
+            <a>params</a>
+            <b>param</b>
+        </field>
+
+        <field>
+            <a>args</a>
+            <b>argument</b>
+        </field>
+
+        <field>
+            <a>files</a>
+            <b>file</b>
+        </field>
+
+        <field>
+            <a>archives</a>
+            <b>archive</b>
+        </field>
+    </mapping>
+
+    <mapping type="one-way">
+        <class-a>org.apache.oozie.fluentjob.api.action.ShellAction</class-a>
+        <class-b>org.apache.oozie.fluentjob.api.generated.action.shell.ACTION</class-b>
+
+        <field>
+            <a>resourceManager</a>
+            <b>resourceManager</b>
+        </field>
+
+        <field>
+            <a>nameNode</a>
+            <b>nameNode</b>
+        </field>
+
+        <field custom-converter="org.apache.oozie.fluentjob.api.mapping.ShellPrepareConverter">
+            <a>prepare</a>
+            <b>prepare</b>
+        </field>
+
+        <field custom-converter="org.apache.oozie.fluentjob.api.mapping.ShellLauncherConverter">
+            <a>launcher</a>
+            <b>launcher</b>
+        </field>
+
+        <field custom-converter="org.apache.oozie.fluentjob.api.mapping.ShellConfigurationConverter">
+            <a>configuration</a>
+            <b>configuration</b>
+        </field>
+
+        <field>
+            <a>jobXmls</a>
+            <b>jobXml</b>
+        </field>
+
+        <field>
+            <a>executable</a>
+            <b>exec</b>
+        </field>
+
+        <field>
+            <a>environmentVariables</a>
+            <b>envVar</b>
+        </field>
+
+        <field>
+            <a>arguments</a>
+            <b>argument</b>
+        </field>
+
+        <field>
+            <a>files</a>
+            <b>file</b>
+        </field>
+
+        <field>
+            <a>archives</a>
+            <b>archive</b>
+        </field>
+
+        <field custom-converter="org.apache.oozie.fluentjob.api.mapping.BooleanToShellFLAGConverter">
+            <a get-method="isCaptureOutput">captureOutput</a>
+            <b>captureOutput</b>
+        </field>
+    </mapping>
+
+    <mapping type="one-way">
+        <class-a>org.apache.oozie.fluentjob.api.action.SparkAction</class-a>
+        <class-b>org.apache.oozie.fluentjob.api.generated.action.spark.ACTION</class-b>
+
+        <field>
+            <a>resourceManager</a>
+            <b>resourceManager</b>
+        </field>
+
+        <field>
+            <a>nameNode</a>
+            <b>nameNode</b>
+        </field>
+
+        <field custom-converter="org.apache.oozie.fluentjob.api.mapping.SparkPrepareConverter">
+            <a>prepare</a>
+            <b>prepare</b>
+        </field>
+
+        <field custom-converter="org.apache.oozie.fluentjob.api.mapping.SparkLauncherConverter">
+            <a>launcher</a>
+            <b>launcher</b>
+        </field>
+
+        <field custom-converter="org.apache.oozie.fluentjob.api.mapping.SparkConfigurationConverter">
+            <a>configuration</a>
+            <b>configuration</b>
+        </field>
+
+        <field>
+            <a>jobXmls</a>
+            <b>jobXml</b>
+        </field>
+
+        <field>
+            <a>master</a>
+            <b>master</b>
+        </field>
+
+        <field>
+            <a>mode</a>
+            <b>mode</b>
+        </field>
+
+        <field>
+            <a>actionName</a>
+            <b>name</b>
+        </field>
+
+        <field>
+            <a>actionClass</a>
+            <b>clazz</b>
+        </field>
+
+        <field>
+            <a>jar</a>
+            <b>jar</b>
+        </field>
+
+        <field>
+            <a>sparkOpts</a>
+            <b>sparkOpts</b>
+        </field>
+
+        <field>
+            <a>args</a>
+            <b>arg</b>
+        </field>
+
+        <field>
+            <a>files</a>
+            <b>file</b>
+        </field>
+
+        <field>
+            <a>archives</a>
+            <b>archive</b>
+        </field>
+    </mapping>
+
+    <mapping type="one-way">
+        <class-a>org.apache.oozie.fluentjob.api.action.SqoopAction</class-a>
+        <class-b>org.apache.oozie.fluentjob.api.generated.action.sqoop.ACTION</class-b>
+
+        <field>
+            <a>resourceManager</a>
+            <b>resourceManager</b>
+        </field>
+
+        <field>
+            <a>nameNode</a>
+            <b>nameNode</b>
+        </field>
+
+        <field custom-converter="org.apache.oozie.fluentjob.api.mapping.SqoopPrepareConverter">
+            <a>prepare</a>
+            <b>prepare</b>
+        </field>
+
+        <field custom-converter="org.apache.oozie.fluentjob.api.mapping.SqoopLauncherConverter">
+            <a>launcher</a>
+            <b>launcher</b>
+        </field>
+
+        <field custom-converter="org.apache.oozie.fluentjob.api.mapping.SqoopConfigurationConverter">
+            <a>configuration</a>
+            <b>configuration</b>
+        </field>
+
+        <field>
+            <a>jobXmls</a>
+            <b>jobXml</b>
+        </field>
+
+        <field>
+            <a>command</a>
+            <b>command</b>
+        </field>
+
+        <field>
+            <a>arguments</a>
+            <b>arg</b>
+        </field>
+
+        <field>
+            <a>files</a>
+            <b>file</b>
+        </field>
+
+        <field>
+            <a>archives</a>
+            <b>archive</b>
+        </field>
+    </mapping>
+
+    <mapping type="one-way">
+        <class-a>org.apache.oozie.fluentjob.api.action.SshAction</class-a>
+        <class-b>org.apache.oozie.fluentjob.api.generated.action.ssh.ACTION</class-b>
+
+        <field>
+            <a>host</a>
+            <b>host</b>
+        </field>
+
+        <field>
+            <a>command</a>
+            <b>command</b>
+        </field>
+
+        <field>
+            <a>args</a>
+            <b>args</b>
+        </field>
+
+        <field custom-converter="org.apache.oozie.fluentjob.api.mapping.BooleanToSshFLAGConverter">
+            <a get-method="isCaptureOutput">captureOutput</a>
+            <b>captureOutput</b>
+        </field>
+    </mapping>
+
+    <mapping type="one-way">
+        <class-a>org.apache.oozie.fluentjob.api.action.EmailAction</class-a>
+        <class-b>org.apache.oozie.fluentjob.api.generated.action.email.ACTION</class-b>
+
+        <field>
+            <a is-accessible="true">to</a>
+            <b>to</b>
+        </field>
+
+        <field>
+            <a>cc</a>
+            <b>cc</b>
+        </field>
+
+        <field>
+            <a>bcc</a>
+            <b>bcc</b>
+        </field>
+
+        <field>
+            <a>subject</a>
+            <b>subject</b>
+        </field>
+
+        <field>
+            <a>body</a>
+            <b>body</b>
+        </field>
+
+        <field>
+            <a>contentType</a>
+            <b>contentType</b>
+        </field>
+
+        <field>
+            <a>attachment</a>
+            <b>attachment</b>
+        </field>
+    </mapping>
+
+    <mapping type="one-way">
+        <class-a>org.apache.oozie.fluentjob.api.action.FSAction</class-a>
+        <class-b>org.apache.oozie.fluentjob.api.generated.workflow.FS</class-b>
+
+        <field>
+            <a>nameNode</a>
+            <b>nameNode</b>
+        </field>
+
+        <field>
+            <a>jobXmls</a>
+            <b>jobXml</b>
+        </field>
+
+        <field relationship-type="cumulative">
+            <a>deletes</a>
+            <b>deleteOrMkdirOrMove</b>
+            <b-hint>org.apache.oozie.fluentjob.api.generated.workflow.DELETE</b-hint>
+        </field>
+
+        <field relationship-type="cumulative">
+            <a>mkdirs</a>
+            <b>deleteOrMkdirOrMove</b>
+            <b-hint>org.apache.oozie.fluentjob.api.generated.workflow.MKDIR</b-hint>
+        </field>
+
+        <field relationship-type="cumulative">
+            <a>moves</a>
+            <b>deleteOrMkdirOrMove</b>
+            <b-hint>org.apache.oozie.fluentjob.api.generated.workflow.MOVE</b-hint>
+        </field>
+
+        <field relationship-type="cumulative">
+            <a>chmods</a>
+            <b>deleteOrMkdirOrMove</b>
+            <b-hint>org.apache.oozie.fluentjob.api.generated.workflow.CHMOD</b-hint>
+        </field>
+
+        <field relationship-type="cumulative">
+            <a>touchzs</a>
+            <b>deleteOrMkdirOrMove</b>
+            <b-hint>org.apache.oozie.fluentjob.api.generated.workflow.TOUCHZ</b-hint>
+        </field>
+
+        <field relationship-type="cumulative">
+            <a>chgrps</a>
+            <b>deleteOrMkdirOrMove</b>
+            <b-hint>org.apache.oozie.fluentjob.api.generated.workflow.CHGRP</b-hint>
+        </field>
+    </mapping>
+
+    <mapping type="one-way">
+        <class-a>org.apache.oozie.fluentjob.api.action.SubWorkflowAction</class-a>
+        <class-b>org.apache.oozie.fluentjob.api.generated.workflow.SUBWORKFLOW</class-b>
+
+        <field>
+            <a>appPath</a>
+            <b>appPath</b>
+        </field>
+
+        <field custom-converter="org.apache.oozie.fluentjob.api.mapping.BooleanToFLAGConverter">
+            <a get-method="isPropagatingConfiguration">propagateConfiguration</a>
+            <b>propagateConfiguration</b>
+        </field>
+
+        <field>
+            <a>configuration</a>
+            <b>configuration</b>
+        </field>
+    </mapping>
+</mappings>

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/resources/checkstyle-header.txt
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/resources/checkstyle-header.txt b/fluent-job/fluent-job-api/src/main/resources/checkstyle-header.txt
new file mode 100644
index 0000000..4247452
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/resources/checkstyle-header.txt
@@ -0,0 +1,17 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/resources/checkstyle.xml
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/resources/checkstyle.xml b/fluent-job/fluent-job-api/src/main/resources/checkstyle.xml
new file mode 100644
index 0000000..c6fca01
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/resources/checkstyle.xml
@@ -0,0 +1,41 @@
+<?xml version="1.0"?>
+<!DOCTYPE module PUBLIC "-//Puppy Crawl//DTD Check Configuration 1.2//EN" "http://www.puppycrawl.com/dtds/configuration_1_2.dtd">
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you under the Apache License, Version 2.0 (the
+  "License"); you may not use this file except in compliance
+  with the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+<module name="Checker">
+
+    <module name="RegexpSingleline">
+        <property name="severity" value="warning"/>
+        <property name="format" value="\s+$"/>
+        <property name="message" value="Line has trailing spaces."/>
+    </module>
+
+    <module name="Header">
+        <property name="headerFile" value="${checkstyle.header.file}"/>
+    </module>
+
+    <module name="TreeWalker">
+
+        <module name="LineLength">
+            <property name="severity" value="warning"/>
+            <property name="max" value="132"/>
+        </module>
+
+    </module>
+
+</module>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/resources/dozer_config.xml
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/resources/dozer_config.xml b/fluent-job/fluent-job-api/src/main/resources/dozer_config.xml
new file mode 100644
index 0000000..55714c9
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/resources/dozer_config.xml
@@ -0,0 +1,94 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you under the Apache License, Version 2.0 (the
+  "License"); you may not use this file except in compliance
+  with the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+
+<mappings xmlns="http://dozer.sourceforge.net"
+          xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+          xsi:schemaLocation="http://dozer.sourceforge.net
+          http://dozer.sourceforge.net/schema/beanmapping.xsd">
+
+    <configuration>
+        <stop-on-errors>true</stop-on-errors>
+        <wildcard>false</wildcard>
+
+        <custom-converters>
+            <converter type="org.apache.oozie.fluentjob.api.mapping.GraphToWORKFLOWAPPConverter" >
+                <class-a>org.apache.oozie.fluentjob.api.dag.Graph</class-a>
+                <class-b>org.apache.oozie.fluentjob.api.generated.workflow.WORKFLOWAPP</class-b>
+            </converter>
+
+            <converter type="org.apache.oozie.fluentjob.api.mapping.GraphNodesToWORKFLOWAPPConverter" >
+                <class-a>org.apache.oozie.fluentjob.api.mapping.GraphNodes</class-a>
+                <class-b>org.apache.oozie.fluentjob.api.generated.workflow.WORKFLOWAPP</class-b>
+            </converter>
+
+            <converter type="org.apache.oozie.fluentjob.api.mapping.DecisionConverter" >
+                <class-a>org.apache.oozie.fluentjob.api.dag.Decision</class-a>
+                <class-b>org.apache.oozie.fluentjob.api.generated.workflow.DECISION</class-b>
+            </converter>
+
+            <converter type="org.apache.oozie.fluentjob.api.mapping.StartConverter" >
+                <class-a>org.apache.oozie.fluentjob.api.dag.Start</class-a>
+                <class-b>org.apache.oozie.fluentjob.api.generated.workflow.START</class-b>
+            </converter>
+
+            <converter type="org.apache.oozie.fluentjob.api.mapping.ForkConverter" >
+                <class-a>org.apache.oozie.fluentjob.api.dag.Fork</class-a>
+                <class-b>org.apache.oozie.fluentjob.api.generated.workflow.FORK</class-b>
+            </converter>
+
+            <converter type="org.apache.oozie.fluentjob.api.mapping.JoinConverter" >
+                <class-a>org.apache.oozie.fluentjob.api.dag.Join</class-a>
+                <class-b>org.apache.oozie.fluentjob.api.generated.workflow.JOIN</class-b>
+            </converter>
+
+            <converter type="org.apache.oozie.fluentjob.api.mapping.MapToConfigurationPropertyConverter" >
+                <class-a>java.util.Map</class-a>
+                <class-b>org.apache.oozie.fluentjob.api.generated.workflow.CONFIGURATION</class-b>
+            </converter>
+
+            <converter type="org.apache.oozie.fluentjob.api.mapping.ExplicitNodeConverter" >
+                <class-a>org.apache.oozie.fluentjob.api.dag.ExplicitNode</class-a>
+                <class-b>org.apache.oozie.fluentjob.api.generated.workflow.ACTION</class-b>
+            </converter>
+
+            <converter type="org.apache.oozie.fluentjob.api.mapping.ParametersConverter" >
+                <class-a>org.apache.oozie.fluentjob.api.workflow.Parameters</class-a>
+                <class-b>org.apache.oozie.fluentjob.api.generated.workflow.PARAMETERS</class-b>
+            </converter>
+
+            <converter type="org.apache.oozie.fluentjob.api.mapping.GlobalConverter" >
+                <class-a>org.apache.oozie.fluentjob.api.workflow.Global</class-a>
+                <class-b>org.apache.oozie.fluentjob.api.generated.workflow.GLOBAL</class-b>
+            </converter>
+
+            <converter type="org.apache.oozie.fluentjob.api.mapping.InlineWorkflowLauncherConverter" >
+                <class-a>org.apache.oozie.fluentjob.api.action.Launcher</class-a>
+                <class-b>org.apache.oozie.fluentjob.api.generated.workflow.LAUNCHER</class-b>
+            </converter>
+
+            <converter type="org.apache.oozie.fluentjob.api.mapping.CredentialsConverter" >
+                <class-a>org.apache.oozie.fluentjob.api.workflow.Credentials</class-a>
+                <class-b>org.apache.oozie.fluentjob.api.generated.workflow.CREDENTIALS</class-b>
+            </converter>
+        </custom-converters>
+
+    </configuration>
+
+</mappings>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/resources/mappingGraphToWORKFLOWAPP.xml
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/resources/mappingGraphToWORKFLOWAPP.xml b/fluent-job/fluent-job-api/src/main/resources/mappingGraphToWORKFLOWAPP.xml
new file mode 100644
index 0000000..780f963
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/resources/mappingGraphToWORKFLOWAPP.xml
@@ -0,0 +1,50 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you under the Apache License, Version 2.0 (the
+  "License"); you may not use this file except in compliance
+  with the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+
+<mappings xmlns="http://dozer.sourceforge.net"
+          xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+          xsi:schemaLocation="http://dozer.sourceforge.net
+          http://dozer.sourceforge.net/schema/beanmapping.xsd">
+
+    <mapping type="one-way">
+        <class-a>org.apache.oozie.fluentjob.api.dag.End</class-a>
+        <class-b>org.apache.oozie.fluentjob.api.generated.workflow.END</class-b>
+
+        <field>
+            <a>name</a>
+            <b>name</b>
+        </field>
+    </mapping>
+
+    <mapping type="one-way">
+        <class-a>org.apache.oozie.fluentjob.api.dag.DagNodeWithCondition</class-a>
+        <class-b>org.apache.oozie.fluentjob.api.generated.workflow.CASE</class-b>
+
+        <field>
+            <a>condition.condition</a>
+            <b>value</b>
+        </field>
+
+        <field>
+            <a>node.name</a>
+            <b>to</b>
+        </field>
+    </mapping>
+</mappings>

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/xjb/bindings.xml
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/xjb/bindings.xml b/fluent-job/fluent-job-api/src/main/xjb/bindings.xml
new file mode 100644
index 0000000..48f6890
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/xjb/bindings.xml
@@ -0,0 +1,166 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you under the Apache License, Version 2.0 (the
+  "License"); you may not use this file except in compliance
+  with the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+
+<jaxb:bindings xmlns:xs="http://www.w3.org/2001/XMLSchema"
+        xmlns:jaxb="http://java.sun.com/xml/ns/jaxb" version="2.1"
+        xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+        xmlns:namespace="http://jaxb2-commons.dev.java.net/namespace-prefix"
+        xsi:schemaLocation="http://java.sun.com/xml/ns/jaxb http://java.sun.com/xml/ns/jaxb/bindingschema_2_1.xsd
+            http://jaxb2-commons.dev.java.net/namespace-prefix
+            https://raw.githubusercontent.com/Siggen/jaxb2-namespace-prefix/master/src/main/resources/prefix-namespace-schema.xsd">
+
+    <jaxb:bindings schemaLocation = "../../../../../client/src/main/resources/distcp-action-1.0.xsd">
+
+        <jaxb:schemaBindings>
+            <jaxb:package name = "org.apache.oozie.fluentjob.api.generated.action.distcp"/>
+        </jaxb:schemaBindings>
+
+        <jaxb:bindings>
+            <namespace:prefix name="distcp" />
+        </jaxb:bindings>
+
+    </jaxb:bindings>
+
+    <jaxb:bindings schemaLocation = "../../../../../client/src/main/resources/email-action-0.2.xsd">
+
+        <jaxb:schemaBindings>
+            <jaxb:package name = "org.apache.oozie.fluentjob.api.generated.action.email"/>
+        </jaxb:schemaBindings>
+
+        <jaxb:bindings>
+            <namespace:prefix name="email" />
+        </jaxb:bindings>
+
+    </jaxb:bindings>
+
+    <jaxb:bindings schemaLocation = "../../../../../client/src/main/resources/hive2-action-1.0.xsd">
+
+        <jaxb:schemaBindings>
+            <jaxb:package name = "org.apache.oozie.fluentjob.api.generated.action.hive2"/>
+        </jaxb:schemaBindings>
+
+        <jaxb:bindings>
+            <namespace:prefix name="hive2" />
+        </jaxb:bindings>
+
+    </jaxb:bindings>
+
+    <jaxb:bindings schemaLocation = "../../../../../client/src/main/resources/hive-action-1.0.xsd">
+
+        <jaxb:schemaBindings>
+            <jaxb:package name = "org.apache.oozie.fluentjob.api.generated.action.hive"/>
+        </jaxb:schemaBindings>
+
+        <jaxb:bindings>
+            <namespace:prefix name="hive" />
+        </jaxb:bindings>
+
+    </jaxb:bindings>
+
+    <!--<jaxb:bindings schemaLocation = "../../../../../client/src/main/resources/oozie-sla-0.2.xsd">-->
+
+        <!--<jaxb:schemaBindings>-->
+            <!--<jaxb:package name = "org.apache.oozie.fluentjob.api.generated.sla"/>-->
+        <!--</jaxb:schemaBindings>-->
+
+        <!--<jaxb:bindings>-->
+            <!--<namespace:prefix name="sla" />-->
+        <!--</jaxb:bindings>-->
+
+    <!--</jaxb:bindings>-->
+
+    <jaxb:bindings schemaLocation = "../../../../../client/src/main/resources/oozie-workflow-1.0.xsd">
+
+        <jaxb:schemaBindings>
+            <jaxb:package name = "org.apache.oozie.fluentjob.api.generated.workflow"/>
+        </jaxb:schemaBindings>
+
+        <jaxb:bindings>
+            <namespace:prefix name="workflow" />
+        </jaxb:bindings>
+
+        <jaxb:bindings node="//xs:complexType[@name='ACTION']/xs:sequence/xs:choice/xs:any">
+            <jaxb:property name="other" generateElementProperty="true"/>
+        </jaxb:bindings>
+
+        <jaxb:bindings node="//xs:complexType[@name='ACTION']/xs:sequence/xs:any[@namespace='uri:oozie:sla:0.1 uri:oozie:sla:0.2']">
+            <jaxb:bindings schemaLocation = "../../../../../client/src/main/resources/oozie-sla-0.2.xsd">
+
+                <jaxb:schemaBindings>
+                    <jaxb:package name = "org.apache.oozie.fluentjob.api.generated.sla"/>
+                </jaxb:schemaBindings>
+
+                <namespace:prefix name="sla" />
+            </jaxb:bindings>
+
+            <jaxb:property name="sla" generateElementProperty="true"/>
+        </jaxb:bindings>
+
+    </jaxb:bindings>
+
+    <jaxb:bindings schemaLocation = "../../../../../client/src/main/resources/shell-action-1.0.xsd">
+
+        <jaxb:schemaBindings>
+            <jaxb:package name = "org.apache.oozie.fluentjob.api.generated.action.shell"/>
+        </jaxb:schemaBindings>
+
+        <jaxb:bindings>
+            <namespace:prefix name="shell" />
+        </jaxb:bindings>
+
+    </jaxb:bindings>
+
+    <jaxb:bindings schemaLocation = "../../../../../client/src/main/resources/spark-action-1.0.xsd">
+
+        <jaxb:schemaBindings>
+            <jaxb:package name = "org.apache.oozie.fluentjob.api.generated.action.spark"/>
+        </jaxb:schemaBindings>
+
+        <jaxb:bindings>
+            <namespace:prefix name="spark" />
+        </jaxb:bindings>
+
+    </jaxb:bindings>
+
+    <jaxb:bindings schemaLocation = "../../../../../client/src/main/resources/sqoop-action-1.0.xsd">
+
+        <jaxb:schemaBindings>
+            <jaxb:package name = "org.apache.oozie.fluentjob.api.generated.action.sqoop"/>
+        </jaxb:schemaBindings>
+
+        <jaxb:bindings>
+            <namespace:prefix name="sqoop" />
+        </jaxb:bindings>
+
+    </jaxb:bindings>
+
+    <jaxb:bindings schemaLocation = "../../../../../client/src/main/resources/ssh-action-0.2.xsd">
+
+        <jaxb:schemaBindings>
+            <jaxb:package name = "org.apache.oozie.fluentjob.api.generated.action.ssh"/>
+        </jaxb:schemaBindings>
+
+        <jaxb:bindings>
+            <namespace:prefix name="ssh" />
+        </jaxb:bindings>
+
+    </jaxb:bindings>
+
+</jaxb:bindings>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/NodesToPng.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/NodesToPng.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/NodesToPng.java
new file mode 100644
index 0000000..4b97327
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/NodesToPng.java
@@ -0,0 +1,52 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api;
+
+import org.apache.oozie.fluentjob.api.dag.Graph;
+import org.apache.oozie.fluentjob.api.workflow.Workflow;
+import org.junit.rules.ExternalResource;
+
+import java.io.IOException;
+import java.text.DateFormat;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+
+/**
+ * Used in integration tests, eases the generation of visualized {@link Workflow} or {@link Graph} instances
+ * in {@code .png} format using a predefined datetime pattern name postfix.
+ */
+public class NodesToPng extends ExternalResource {
+    private final DateFormat df = new SimpleDateFormat("yyyyMMddhhmmss");
+    private Date testRunningOn;
+
+    @Override
+    protected void before() throws Throwable {
+        testRunningOn = new Date();
+    }
+
+    public void withWorkflow(final Workflow workflow) throws IOException {
+        final String fileName = String.format("%s-%s-workflow.png", df.format(testRunningOn), workflow.getName());
+        GraphVisualization.workflowToPng(workflow, fileName);
+    }
+
+    public void withGraph(final Graph graph) throws IOException {
+        final String fileName = String.format("%s-%s-graph.png", df.format(testRunningOn), graph.getName());
+        GraphVisualization.graphToPng(graph, fileName);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/TestCondition.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/TestCondition.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/TestCondition.java
new file mode 100644
index 0000000..5c0fab8
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/TestCondition.java
@@ -0,0 +1,43 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+public class TestCondition {
+    @Test
+    public void testActualCondition() {
+        final String conditionString = "some_condition";
+        final Condition actualCondition = Condition.actualCondition(conditionString);
+
+        assertEquals(conditionString, actualCondition.getCondition());
+        assertFalse(actualCondition.isDefault());
+    }
+
+    @Test
+    public void testDefaultCondition() {
+        final Condition defaultCondition = Condition.defaultCondition();
+
+        assertTrue(defaultCondition.isDefault());
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/TestModifyOnce.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/TestModifyOnce.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/TestModifyOnce.java
new file mode 100644
index 0000000..6e1f11f
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/TestModifyOnce.java
@@ -0,0 +1,52 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestModifyOnce {
+    @Rule
+    public final ExpectedException expectedException = ExpectedException.none();
+
+    @Test
+    public void testAllowOneModification() {
+        final String defaultValue = "Default value";
+        final ModifyOnce<String> instance = new ModifyOnce<>(defaultValue);
+
+        assertEquals(defaultValue, instance.get());
+
+        final String anotherValue = "Another value";
+        instance.set(anotherValue);
+
+        assertEquals(anotherValue, instance.get());
+    }
+
+    @Test
+    public void testThrowOnSecondModification() {
+        final ModifyOnce<String> instance = new ModifyOnce<>();
+        instance.set("First modification");
+
+        expectedException.expect(IllegalStateException.class);
+        instance.set("Second modification");
+    }
+}


[05/16] oozie git commit: OOZIE-2339 [fluent-job] Minimum Viable Fluent Job API (daniel.becker, andras.piros via rkanter, gezapeti, pbacsko)

Posted by an...@apache.org.
http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestSqoopActionBuilder.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestSqoopActionBuilder.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestSqoopActionBuilder.java
new file mode 100644
index 0000000..1bf9e32
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestSqoopActionBuilder.java
@@ -0,0 +1,219 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestSqoopActionBuilder extends TestNodeBuilderBaseImpl<SqoopAction, SqoopActionBuilder> {
+    private static final String NAME = "sqoop-name";
+    private static final String NAME_NODE = "${nameNode}";
+    private static final String EXAMPLE_DIR = "/path/to/directory";
+    private static final String[] ARGS = {"arg1", "arg2", "arg3"};
+    private static final String MAPRED_JOB_QUEUE_NAME = "mapred.job.queue.name";
+    private static final String DEFAULT = "default";
+    private static final String RESOURCE_MANAGER = "${resourceManager}";
+    private static final String PATH_TO_DELETE = "/path/to/delete";
+    private static final String PATH_TO_MKDIR = "/path/to/mkdir";
+
+    @Override
+    protected SqoopActionBuilder getBuilderInstance() {
+        return SqoopActionBuilder.create();
+    }
+
+    @Override
+    protected SqoopActionBuilder getBuilderInstance(final SqoopAction action) {
+        return SqoopActionBuilder.createFromExistingAction(action);
+    }
+
+    @Test
+    public void testResourceManagerAdded() {
+        final SqoopActionBuilder builder = getBuilderInstance();
+        builder.withResourceManager(RESOURCE_MANAGER);
+
+        final SqoopAction action = builder.build();
+        assertEquals(RESOURCE_MANAGER, action.getResourceManager());
+    }
+
+    @Test
+    public void testNameNodeAdded() {
+        final SqoopActionBuilder builder = getBuilderInstance();
+        builder.withNameNode(NAME_NODE);
+
+        final SqoopAction action = builder.build();
+        assertEquals(NAME_NODE, action.getNameNode());
+    }
+
+    @Test
+    public void testPrepareAdded() {
+        final SqoopActionBuilder builder = getBuilderInstance();
+        builder.withPrepare(new PrepareBuilder().withDelete(EXAMPLE_DIR).build());
+
+        final SqoopAction action = builder.build();
+        assertEquals(EXAMPLE_DIR, action.getPrepare().getDeletes().get(0).getPath());
+    }
+
+    @Test
+    public void testSameConfigPropertyAddedTwiceThrows() {
+        final SqoopActionBuilder builder = getBuilderInstance();
+        builder.withConfigProperty(MAPRED_JOB_QUEUE_NAME, DEFAULT);
+
+        expectedException.expect(IllegalStateException.class);
+        builder.withConfigProperty(MAPRED_JOB_QUEUE_NAME, DEFAULT);
+    }
+
+    @Test
+    public void testSeveralArgumentsAdded() {
+        final SqoopActionBuilder builder = getBuilderInstance();
+
+        for (final String arg : ARGS) {
+            builder.withArgument(arg);
+        }
+
+        final SqoopAction action = builder.build();
+
+        final List<String> argList = action.getArguments();
+        assertEquals(ARGS.length, argList.size());
+
+        for (int i = 0; i < ARGS.length; ++i) {
+            assertEquals(ARGS[i], argList.get(i));
+        }
+    }
+
+    @Test
+    public void testRemoveArguments() {
+        final SqoopActionBuilder builder = getBuilderInstance();
+
+        for (final String file : ARGS) {
+            builder.withArgument(file);
+        }
+
+        builder.withoutArgument(ARGS[0]);
+
+        final SqoopAction action = builder.build();
+
+        final List<String> argList = action.getArguments();
+        final String[] remainingArgs = Arrays.copyOfRange(ARGS, 1, ARGS.length);
+        assertEquals(remainingArgs.length, argList.size());
+
+        for (int i = 0; i < remainingArgs.length; ++i) {
+            assertEquals(remainingArgs[i], argList.get(i));
+        }
+    }
+
+    @Test
+    public void testClearArguments() {
+        final SqoopActionBuilder builder = getBuilderInstance();
+
+        for (final String file : ARGS) {
+            builder.withArgument(file);
+        }
+
+        builder.clearArguments();
+
+        final SqoopAction action = builder.build();
+
+        final List<String> argList = action.getArguments();
+        assertEquals(0, argList.size());
+    }
+
+    @Test
+    public void testFromExistingSqoopAction() {
+        final SqoopActionBuilder builder = getBuilderInstance();
+
+        builder.withName(NAME)
+                .withResourceManager(RESOURCE_MANAGER)
+                .withNameNode(NAME_NODE)
+                .withConfigProperty(MAPRED_JOB_QUEUE_NAME, DEFAULT)
+                .withPrepare(new PrepareBuilder()
+                        .withDelete(PATH_TO_DELETE)
+                        .withMkdir(PATH_TO_MKDIR)
+                        .build())
+                .withLauncher(new LauncherBuilder()
+                        .withMemoryMb(1024L)
+                        .withVCores(2L)
+                        .withQueue(DEFAULT)
+                        .withSharelib(DEFAULT)
+                        .withViewAcl(DEFAULT)
+                        .withModifyAcl(DEFAULT)
+                        .build())
+                .withCommand(DEFAULT)
+                .withArgument(ARGS[0])
+                .withArgument(ARGS[1])
+                .withArchive(DEFAULT)
+                .withFile(DEFAULT);
+
+        final SqoopAction action = builder.build();
+
+        final SqoopActionBuilder fromExistingBuilder = getBuilderInstance(action);
+
+        final String newName = "fromExisting_" + NAME;
+        fromExistingBuilder.withName(newName)
+                .withoutArgument(ARGS[1])
+                .withArgument(ARGS[2]);
+
+        final SqoopAction modifiedAction = fromExistingBuilder.build();
+
+        assertEquals(newName, modifiedAction.getName());
+        assertEquals(action.getNameNode(), modifiedAction.getNameNode());
+
+        final Map<String, String> expectedConfiguration = new LinkedHashMap<>();
+        expectedConfiguration.put(MAPRED_JOB_QUEUE_NAME, DEFAULT);
+        assertEquals(expectedConfiguration, modifiedAction.getConfiguration());
+
+        assertEquals(Arrays.asList(ARGS[0], ARGS[2]), modifiedAction.getArguments());
+
+        assertEquals(PATH_TO_DELETE, modifiedAction.getPrepare().getDeletes().get(0).getPath());
+        assertEquals(PATH_TO_MKDIR, modifiedAction.getPrepare().getMkdirs().get(0).getPath());
+
+        assertEquals(1024L, modifiedAction.getLauncher().getMemoryMb());
+        assertEquals(2L, modifiedAction.getLauncher().getVCores());
+        assertEquals(DEFAULT, modifiedAction.getLauncher().getQueue());
+        assertEquals(DEFAULT, modifiedAction.getLauncher().getSharelib());
+        assertEquals(DEFAULT, modifiedAction.getLauncher().getViewAcl());
+        assertEquals(DEFAULT, modifiedAction.getLauncher().getModifyAcl());
+
+        assertEquals(action.getCommand(), modifiedAction.getCommand());
+    }
+
+    @Test
+    public void testFromOtherAction() {
+        final ShellAction parent = ShellActionBuilder.create()
+                .withName("parent")
+                .build();
+
+        final ShellAction otherAction = ShellActionBuilder.createFromExistingAction(parent)
+                .withName("shell")
+                .withParent(parent)
+                .build();
+
+        final SqoopAction fromOtherAction = SqoopActionBuilder.createFromExistingAction(otherAction)
+                .withName("sqoop")
+                .build();
+
+        assertEquals("sqoop", fromOtherAction.getName());
+        assertEquals(parent, fromOtherAction.getParentsWithoutConditions().get(0));
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestSshActionBuilder.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestSshActionBuilder.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestSshActionBuilder.java
new file mode 100644
index 0000000..d1da8fb
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestSshActionBuilder.java
@@ -0,0 +1,147 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestSshActionBuilder extends TestNodeBuilderBaseImpl<SshAction, SshActionBuilder> {
+    private static final String NAME = "ssh-name";
+    private static final String[] ARGS = {"arg1", "arg2", "arg3"};
+    private static final String DEFAULT = "default";
+
+    @Override
+    protected SshActionBuilder getBuilderInstance() {
+        return SshActionBuilder.create();
+    }
+
+    @Override
+    protected SshActionBuilder getBuilderInstance(final SshAction action) {
+        return SshActionBuilder.createFromExistingAction(action);
+    }
+
+    @Test
+    public void testSeveralArgsAdded() {
+        final SshActionBuilder builder = getBuilderInstance();
+
+        for (final String arg : ARGS) {
+            builder.withArg(arg);
+        }
+
+        final SshAction action = builder.build();
+
+        final List<String> argList = action.getArgs();
+        assertEquals(ARGS.length, argList.size());
+
+        for (int i = 0; i < ARGS.length; ++i) {
+            assertEquals(ARGS[i], argList.get(i));
+        }
+    }
+
+    @Test
+    public void testRemoveArgs() {
+        final SshActionBuilder builder = getBuilderInstance();
+
+        for (final String file : ARGS) {
+            builder.withArg(file);
+        }
+
+        builder.withoutArg(ARGS[0]);
+
+        final SshAction action = builder.build();
+
+        final List<String> argList = action.getArgs();
+        final String[] remainingArgs = Arrays.copyOfRange(ARGS, 1, ARGS.length);
+        assertEquals(remainingArgs.length, argList.size());
+
+        for (int i = 0; i < remainingArgs.length; ++i) {
+            assertEquals(remainingArgs[i], argList.get(i));
+        }
+    }
+
+    @Test
+    public void testClearArgs() {
+        final SshActionBuilder builder = getBuilderInstance();
+
+        for (final String file : ARGS) {
+            builder.withArg(file);
+        }
+
+        builder.clearArgs();
+
+        final SshAction action = builder.build();
+
+        final List<String> argList = action.getArgs();
+        assertEquals(0, argList.size());
+    }
+
+    @Test
+    public void testFromExistingSshAction() {
+        final SshActionBuilder builder = getBuilderInstance();
+
+        builder.withName(NAME)
+                .withCommand(DEFAULT)
+                .withHost(DEFAULT)
+                .withArg(ARGS[0])
+                .withArg(ARGS[1])
+                .withCaptureOutput(true);
+
+        final SshAction action = builder.build();
+
+        final SshActionBuilder fromExistingBuilder = getBuilderInstance(action);
+
+        final String newName = "fromExisting_" + NAME;
+        fromExistingBuilder.withName(newName)
+                .withoutArg(ARGS[1])
+                .withArg(ARGS[2]);
+
+        final SshAction modifiedAction = fromExistingBuilder.build();
+
+        assertEquals(newName, modifiedAction.getName());
+
+        assertEquals(Arrays.asList(ARGS[0], ARGS[2]), modifiedAction.getArgs());
+
+        assertEquals(action.getCommand(), modifiedAction.getCommand());
+        assertEquals(action.getHost(), modifiedAction.getHost());
+        assertEquals(action.isCaptureOutput(), modifiedAction.isCaptureOutput());
+    }
+
+    @Test
+    public void testFromOtherAction() {
+        final ShellAction parent = ShellActionBuilder.create()
+                .withName("parent")
+                .build();
+
+        final ShellAction otherAction = ShellActionBuilder.createFromExistingAction(parent)
+                .withName("shell")
+                .withParent(parent)
+                .build();
+
+        final SshAction fromOtherAction = SshActionBuilder.createFromExistingAction(otherAction)
+                .withName("ssh")
+                .build();
+
+        assertEquals("ssh", fromOtherAction.getName());
+        assertEquals(parent, fromOtherAction.getParentsWithoutConditions().get(0));
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestStreamingBuilder.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestStreamingBuilder.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestStreamingBuilder.java
new file mode 100644
index 0000000..9102334
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestStreamingBuilder.java
@@ -0,0 +1,127 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import java.util.Arrays;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestStreamingBuilder {
+    @Rule
+    public final ExpectedException expectedException = ExpectedException.none();
+
+    @Test
+    public void testWithMapper() {
+        final String mapperName = "mapper-name.sh";
+
+        final StreamingBuilder builder = new StreamingBuilder();
+        builder.withMapper(mapperName);
+
+        final Streaming streaming = builder.build();
+        assertEquals(mapperName, streaming.getMapper());
+    }
+
+    @Test
+    public void testWithMapperCalledTwiceThrows() {
+        final String mapperName1 = "mapper-name-2.sh";
+        final String mapperName2 = "mapper-name-2.sh";
+
+        final StreamingBuilder builder = new StreamingBuilder();
+        builder.withMapper(mapperName1);
+
+        expectedException.expect(IllegalStateException.class);
+        builder.withMapper(mapperName2);
+    }
+
+    @Test
+    public void testWithReducer() {
+        final String reducerName = "reducer-name.sh";
+
+        final StreamingBuilder builder = new StreamingBuilder();
+        builder.withReducer(reducerName);
+
+        final Streaming streaming = builder.build();
+        assertEquals(reducerName, streaming.getReducer());
+    }
+
+    @Test
+    public void testWithReducerCalledTwiceThrows() {
+        final String reducerName1 = "reducer-name1.sh";
+        final String reducerName2 = "reducer-name2.sh";
+
+        final StreamingBuilder builder = new StreamingBuilder();
+        builder.withReducer(reducerName1);
+
+        expectedException.expect(IllegalStateException.class);
+        builder.withReducer(reducerName2);
+    }
+
+    @Test
+    public void testWithRecordReader() {
+        final String recordReaderName = "record-reader-name.sh";
+
+        final StreamingBuilder builder = new StreamingBuilder();
+        builder.withRecordReader(recordReaderName);
+
+        final Streaming streaming = builder.build();
+        assertEquals(recordReaderName, streaming.getRecordReader());
+    }
+
+    @Test
+    public void testWithRecordReaderCalledTwiceThrows() {
+        final String recordReaderName1 = "record-reader-name1.sh";
+        final String recordReaderName2 = "record-reader-name2.sh";
+
+        final StreamingBuilder builder = new StreamingBuilder();
+        builder.withRecordReader(recordReaderName1);
+
+        expectedException.expect(IllegalStateException.class);
+        builder.withRecordReader(recordReaderName2);
+    }
+
+    @Test
+    public void testWithRecordReaderMapping() {
+        final String mapping1 = "mapping1";
+        final String mapping2 = "mapping2";
+
+        final StreamingBuilder builder = new StreamingBuilder();
+        builder.withRecordReaderMapping(mapping1)
+               .withRecordReaderMapping(mapping2);
+
+        final Streaming streaming = builder.build();
+        assertEquals(Arrays.asList(mapping1, mapping2), streaming.getRecordReaderMappings());
+    }
+
+    @Test
+    public void testWithEnv() {
+        final String env1 = "env1";
+        final String env2 = "env2";
+
+        final StreamingBuilder builder = new StreamingBuilder();
+        builder.withEnv(env1)
+                .withEnv(env2);
+
+        final Streaming streaming = builder.build();
+        assertEquals(Arrays.asList(env1, env2), streaming.getEnvs());
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestSubWorkflowBuilder.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestSubWorkflowBuilder.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestSubWorkflowBuilder.java
new file mode 100644
index 0000000..d6a99a2
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestSubWorkflowBuilder.java
@@ -0,0 +1,185 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import com.google.common.collect.ImmutableMap;
+import org.junit.Test;
+
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestSubWorkflowBuilder extends TestNodeBuilderBaseImpl<SubWorkflowAction, SubWorkflowActionBuilder> {
+    private static final String MAPRED_JOB_QUEUE_NAME = "mapred.job.queue.name";
+    private static final String DEFAULT = "default";
+
+    private static final ImmutableMap<String, String> CONFIG_EXAMPLE = getConfigExample();
+
+    private static ImmutableMap<String, String> getConfigExample() {
+        final ImmutableMap.Builder<String, String> configExampleBuilder = new ImmutableMap.Builder<>();
+
+        final String[] keys = {"mapred.map.tasks", "mapred.input.dir", "mapred.output.dir"};
+        final String[] values = {"1", "${inputDir}", "${outputDir}"};
+
+        for (int i = 0; i < keys.length; ++i) {
+            configExampleBuilder.put(keys[i], values[i]);
+        }
+
+        return configExampleBuilder.build();
+    }
+
+    @Override
+    protected SubWorkflowActionBuilder getBuilderInstance() {
+        return SubWorkflowActionBuilder.create();
+    }
+
+    @Override
+    protected SubWorkflowActionBuilder getBuilderInstance(SubWorkflowAction action) {
+        return SubWorkflowActionBuilder.createFromExistingAction(action);
+    }
+
+    @Test
+    public void testWithAppPath() {
+        final String appPath = "/path/to/app";
+
+        final SubWorkflowActionBuilder builder = getBuilderInstance();
+        builder.withAppPath(appPath);
+
+        final SubWorkflowAction action = builder.build();
+        assertEquals(appPath, action.getAppPath());
+    }
+
+    @Test
+    public void testWithAppPathCalledTwiceThrows() {
+        final String appPath1 = "/path/to/app1";
+        final String appPath2 = "/path/to/app2";
+
+        final SubWorkflowActionBuilder builder = getBuilderInstance();
+        builder.withAppPath(appPath1);
+
+        expectedException.expect(IllegalStateException.class);
+        builder.withAppPath(appPath2);
+    }
+
+    @Test
+    public void testWithPropagatingConfiguration() {
+        final SubWorkflowActionBuilder builder = getBuilderInstance();
+        builder.withPropagatingConfiguration();
+
+        final SubWorkflowAction action = builder.build();
+        assertEquals(true, action.isPropagatingConfiguration());
+    }
+
+    @Test
+    public void testWithPropagatingConfigurationCalledTwiceThrows() {
+        final SubWorkflowActionBuilder builder = getBuilderInstance();
+        builder.withPropagatingConfiguration();
+
+        expectedException.expect(IllegalStateException.class);
+        builder.withPropagatingConfiguration();
+    }
+
+    @Test
+    public void testWithoutPropagatingConfiguration() {
+        final SubWorkflowActionBuilder builder = getBuilderInstance();
+        builder.withPropagatingConfiguration();
+
+        final SubWorkflowAction action = builder.build();
+
+        final SubWorkflowActionBuilder fromExistingBuilder = getBuilderInstance(action);
+
+        fromExistingBuilder.withoutPropagatingConfiguration();
+
+        final SubWorkflowAction modifiedAction = fromExistingBuilder.build();
+        assertEquals(false, modifiedAction.isPropagatingConfiguration());
+    }
+
+    @Test
+    public void testWithoutPropagatingConfigurationCalledTwiceThrows() {
+        final SubWorkflowActionBuilder builder = getBuilderInstance();
+        builder.withPropagatingConfiguration();
+
+        final SubWorkflowAction action = builder.build();
+
+        final SubWorkflowActionBuilder fromExistingBuilder = getBuilderInstance(action);
+
+        fromExistingBuilder.withoutPropagatingConfiguration();
+
+        expectedException.expect(IllegalStateException.class);
+        fromExistingBuilder.withoutPropagatingConfiguration();
+    }
+
+    @Test
+    public void testConfigPropertyAdded() {
+        final SubWorkflowActionBuilder builder = getBuilderInstance();
+        builder.withConfigProperty(MAPRED_JOB_QUEUE_NAME, DEFAULT);
+
+        final SubWorkflowAction action = builder.build();
+        assertEquals(DEFAULT, action.getConfiguration().get(MAPRED_JOB_QUEUE_NAME));
+    }
+
+    @Test
+    public void testSeveralConfigPropertiesAdded() {
+        final SubWorkflowActionBuilder builder = getBuilderInstance();
+
+        for (final Map.Entry<String, String> entry : CONFIG_EXAMPLE.entrySet()) {
+            builder.withConfigProperty(entry.getKey(), entry.getValue());
+        }
+
+        final SubWorkflowAction action = builder.build();
+
+        for (final Map.Entry<String, String> entry : CONFIG_EXAMPLE.entrySet()) {
+            assertEquals(entry.getValue(), action.getConfiguration().get(entry.getKey()));
+        }
+
+        assertEquals(CONFIG_EXAMPLE, action.getConfiguration());
+    }
+
+    @Test
+    public void testSameConfigPropertyAddedTwiceThrows() {
+        final SubWorkflowActionBuilder builder = getBuilderInstance();
+        builder.withConfigProperty(MAPRED_JOB_QUEUE_NAME, DEFAULT);
+
+        expectedException.expect(IllegalStateException.class);
+        builder.withConfigProperty(MAPRED_JOB_QUEUE_NAME, DEFAULT);
+    }
+
+    @Test
+    public void testFromExistiongSubWorkflowAction() {
+        final String appPath = "/path/to/app";
+
+        final SubWorkflowActionBuilder builder = getBuilderInstance();
+        builder.withAppPath(appPath)
+                .withPropagatingConfiguration()
+                .withConfigProperty(MAPRED_JOB_QUEUE_NAME, DEFAULT);
+
+        final SubWorkflowAction action = builder.build();
+
+        final SubWorkflowActionBuilder fromExistingBuilder = getBuilderInstance(action);
+
+        final SubWorkflowAction modifiedAction = fromExistingBuilder.build();
+        assertEquals(appPath, modifiedAction.getAppPath());
+        assertEquals(true, modifiedAction.isPropagatingConfiguration());
+
+        final Map<String, String> expectedConfiguration = new LinkedHashMap<>();
+        expectedConfiguration.put(MAPRED_JOB_QUEUE_NAME, DEFAULT);
+        assertEquals(expectedConfiguration, modifiedAction.getConfiguration());
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestTouchz.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestTouchz.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestTouchz.java
new file mode 100644
index 0000000..0bff3a4
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/action/TestTouchz.java
@@ -0,0 +1,33 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestTouchz {
+    @Test
+    public void testGetPath() {
+        final String path = "path";
+        final Touchz touchz = new Touchz(path);
+
+        assertEquals(path, touchz.getPath());
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/dag/TestDecision.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/dag/TestDecision.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/dag/TestDecision.java
new file mode 100644
index 0000000..9c12d7e
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/dag/TestDecision.java
@@ -0,0 +1,210 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+package org.apache.oozie.fluentjob.api.dag;
+
+import org.apache.oozie.fluentjob.api.Condition;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import java.util.Arrays;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class TestDecision extends TestNodeBase<Decision> {
+    @Rule
+    public final ExpectedException expectedException = ExpectedException.none();
+
+    @Override
+    protected Decision getInstance(final String name) {
+        return new Decision(name);
+    }
+
+    @Test
+    public void testAddParentWhenNoneAlreadyExists() {
+        final Start parent = new Start("parent");
+        final Decision instance = getInstance("instance");
+
+        instance.addParent(parent);
+        assertEquals(parent, instance.getParent());
+        assertEquals(instance, parent.getChild());
+    }
+
+    @Test
+    public void testAddParentWhenItAlreadyExistsThrows() {
+        final NodeBase parent1 = new ExplicitNode("parent1", null);
+        final NodeBase parent2 = new ExplicitNode("parent2", null);
+
+        final Decision instance = getInstance("instance");
+
+        instance.addParent(parent1);
+
+        expectedException.expect(IllegalStateException.class);
+        instance.addParent(parent2);
+    }
+
+    @Test
+    public void testRemoveExistingParent() {
+        final Start parent = new Start("parent");
+        final Decision instance = getInstance("instance");
+
+        instance.addParent(parent);
+
+        instance.removeParent(parent);
+        assertEquals(null, instance.getParent());
+        assertEquals(null, parent.getChild());
+    }
+
+    @Test
+    public void testRemoveNonexistentParentThrows() {
+        final Start parent = new Start("parent");
+        final Decision instance = getInstance("instance");
+
+        expectedException.expect(IllegalArgumentException.class);
+        instance.removeParent(parent);
+    }
+
+    @Test
+    public void testClearExistingParent() {
+        final Start parent = new Start("parent");
+        final Decision instance = getInstance("instance");
+
+        instance.addParent(parent);
+
+        instance.clearParents();
+        assertEquals(null, instance.getParent());
+        assertEquals(null, parent.getChild());
+    }
+
+    @Test
+    public void testClearNonExistentParent() {
+        final Start parent = new Start("parent");
+        final Decision instance = getInstance("instance");
+
+        instance.clearParents();
+        assertEquals(null, instance.getParent());
+        assertEquals(null, parent.getChild());
+    }
+
+    @Test
+    public void testChildrenWithConditionsAreCorrect() {
+        final NodeBase child1 = new ExplicitNode("child1", null);
+        final NodeBase child2 = new ExplicitNode("child2", null);
+
+        final Decision decision = getInstance("decision");
+
+        final Condition condition1 = Condition.actualCondition("condition1");
+        final Condition condition2 = Condition.actualCondition("condition2");
+
+        child1.addParentWithCondition(decision, condition1);
+        child2.addParentWithCondition(decision, condition2);
+
+        final List<DagNodeWithCondition> childrenWithConditions = decision.getChildrenWithConditions();
+
+        assertEquals(2, childrenWithConditions.size());
+
+        assertEquals(child1, childrenWithConditions.get(0).getNode());
+        assertEquals(condition1, childrenWithConditions.get(0).getCondition());
+
+        assertEquals(child2, childrenWithConditions.get(1).getNode());
+        assertEquals(condition2, childrenWithConditions.get(1).getCondition());
+    }
+
+    @Test
+    public void testDefaultChildIsCorrect() {
+        final NodeBase child1 = new ExplicitNode("child1", null);
+        final NodeBase defaultChild = new ExplicitNode("defaultChild", null);
+
+        final Decision decision = getInstance("decision");
+
+        final Condition condition1 = Condition.actualCondition("condition1");
+
+        child1.addParentWithCondition(decision, condition1);
+        defaultChild.addParentDefaultConditional(decision);
+
+        final List<DagNodeWithCondition> childrenWithConditions = decision.getChildrenWithConditions();
+
+        assertEquals(2, childrenWithConditions.size());
+
+        assertEquals(child1, childrenWithConditions.get(0).getNode());
+        assertEquals(condition1, childrenWithConditions.get(0).getCondition());
+
+        assertEquals(defaultChild, childrenWithConditions.get(1).getNode());
+        assertTrue(childrenWithConditions.get(1).getCondition().isDefault());
+
+        assertEquals(defaultChild, decision.getDefaultChild());
+    }
+
+    @Test
+    public void testMultipleDefaultChildAddedThrows() {
+        final NodeBase child1 = new ExplicitNode("child1", null);
+        final NodeBase defaultChild1 = new ExplicitNode("defaultChild1", null);
+        final NodeBase defaultChild2 = new ExplicitNode("defaultChild2", null);
+
+        final Decision decision = getInstance("decision");
+
+        final Condition condition1 = Condition.actualCondition("condition1");
+
+        child1.addParentWithCondition(decision, condition1);
+        defaultChild1.addParentDefaultConditional(decision);
+
+        expectedException.expect(IllegalStateException.class);
+        defaultChild2.addParentDefaultConditional(decision);
+    }
+
+    @Test
+    public void testDefaultChildRemovedAndAnotherOneAdded() {
+        final NodeBase child1 = new ExplicitNode("child1", null);
+        final NodeBase defaultChild1 = new ExplicitNode("defaultChild1", null);
+        final NodeBase defaultChild2 = new ExplicitNode("defaultChild2", null);
+
+        final Decision decision = getInstance("decision");
+
+        final Condition condition1 = Condition.actualCondition("condition1");
+
+        child1.addParentWithCondition(decision, condition1);
+        defaultChild1.addParentDefaultConditional(decision);
+
+        defaultChild1.removeParent(decision);
+        defaultChild2.addParentDefaultConditional(decision);
+    }
+
+    @Test
+    public void testDecisionRemovedAsParent() {
+        final Decision instance = getInstance("instance");
+        final NodeBase child1 = new ExplicitNode("child1", null);
+        final NodeBase child2 = new ExplicitNode("child2", null);
+        final NodeBase child3 = new ExplicitNode("child3", null);
+        final NodeBase child4 = new ExplicitNode("child4", null);
+        final NodeBase child5 = new ExplicitNode("child5", null);
+
+        child1.addParentWithCondition(instance, Condition.actualCondition("condition"));
+        child2.addParentWithCondition(instance, Condition.actualCondition("condition"));
+        child3.addParentWithCondition(instance, Condition.actualCondition("condition"));
+        child4.addParentWithCondition(instance, Condition.actualCondition("condition"));
+        child5.addParentWithCondition(instance, Condition.actualCondition("condition"));
+
+        child5.removeParent(instance);
+
+        assertEquals(Arrays.asList(child1, child2, child3, child4), instance.getChildren());
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/dag/TestDecisionJoin.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/dag/TestDecisionJoin.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/dag/TestDecisionJoin.java
new file mode 100644
index 0000000..c987d1e
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/dag/TestDecisionJoin.java
@@ -0,0 +1,36 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.dag;
+
+public class TestDecisionJoin extends TestJoiningNodeBase<Decision, DecisionJoin> {
+    @Override
+    protected Decision getBranchingInstance(final String name) {
+        return new Decision(name);
+    }
+
+    @Override
+    protected DecisionJoin getJoiningInstance(final String name, final Decision branchingPair) {
+        return new DecisionJoin(name, branchingPair);
+    }
+
+    @Override
+    protected JoiningNodeBase<Decision> getInstance(final String name) {
+        return getJoiningInstance(name);
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/dag/TestEnd.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/dag/TestEnd.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/dag/TestEnd.java
new file mode 100644
index 0000000..36adbe0
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/dag/TestEnd.java
@@ -0,0 +1,117 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.dag;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class TestEnd extends TestNodeBase<End> {
+    @Rule
+    public final ExpectedException expectedException = ExpectedException.none();
+
+    @Override
+    protected End getInstance(final String name) {
+        return new End(name);
+    }
+
+    @Test
+    public void testAddParentWhenNoneAlreadyExists() {
+        final Start parent = new Start("parent");
+        final End instance = getInstance("instance");
+
+        instance.addParent(parent);
+        assertEquals(parent, instance.getParent());
+        assertEquals(instance, parent.getChild());
+    }
+
+    @Test
+    public void testAddParentWhenItAlreadyExists() {
+        final Start parent1 = new Start("parent1");
+        final Start parent2 = new Start("parent2");
+        final End instance = getInstance("instance");
+
+        instance.addParent(parent1);
+
+        expectedException.expect(IllegalStateException.class);
+        instance.addParent(parent2);
+    }
+
+    @Test
+    public void testRemoveExistingParent() {
+        final Start parent = new Start("parent");
+        final End instance = getInstance("instance");
+
+        instance.addParent(parent);
+
+        instance.removeParent(parent);
+        assertEquals(null, instance.getParent());
+        assertEquals(null, parent.getChild());
+    }
+
+    @Test
+    public void testRemoveNonexistentParentThrows() {
+        final Start parent = new Start("parent");
+        final End instance = getInstance("instance");
+
+        expectedException.expect(IllegalArgumentException.class);
+        instance.removeParent(parent);
+    }
+
+    @Test
+    public void testClearExistingParent() {
+        final Start parent = new Start("parent");
+        final End instance = getInstance("instance");
+
+        instance.addParent(parent);
+
+        instance.clearParents();
+        assertEquals(null, instance.getParent());
+        assertEquals(null, parent.getChild());
+    }
+
+    @Test
+    public void testClearNonExistentParent() {
+        final Start parent = new Start("parent");
+        final End instance = getInstance("instance");
+
+        instance.clearParents();
+        assertEquals(null, instance.getParent());
+        assertEquals(null, parent.getChild());
+    }
+
+    @Test
+    public void testAddedAsParentThrows () {
+        final End instance = getInstance("instance");
+        final ExplicitNode child = new ExplicitNode("child", null);
+
+        expectedException.expect(IllegalStateException.class);
+        child.addParent(instance);
+    }
+
+    @Test
+    public void testGetChildren() {
+        final End instance = getInstance("end");
+
+        assertTrue(instance.getChildren().isEmpty());
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/dag/TestExplicitNode.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/dag/TestExplicitNode.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/dag/TestExplicitNode.java
new file mode 100644
index 0000000..5b7ff38
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/dag/TestExplicitNode.java
@@ -0,0 +1,156 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.dag;
+
+import org.apache.oozie.fluentjob.api.action.MapReduceActionBuilder;
+import org.apache.oozie.fluentjob.api.action.Node;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import java.util.Arrays;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestExplicitNode extends TestNodeBase<ExplicitNode> {
+    @Rule
+    public final ExpectedException expectedException = ExpectedException.none();
+
+    @Override
+    protected ExplicitNode getInstance(final String name) {
+        return new ExplicitNode(name, null);
+    }
+
+    @Test
+    public void testRealNode() {
+        final Node node = MapReduceActionBuilder.create().build();
+        final ExplicitNode instance = new ExplicitNode(NAME, node);
+
+        assertEquals(node, instance.getRealNode());
+    }
+
+    @Test
+    public void testAddParentWhenNoneAlreadyExists() {
+        final ExplicitNode parent = getInstance("parent");
+        final ExplicitNode instance = getInstance("instance");
+
+        instance.addParent(parent);
+        assertEquals(parent, instance.getParent());
+        assertEquals(instance, parent.getChild());
+    }
+
+    @Test
+    public void testAddingParentWhenItAlreadyExistsThrows() {
+        final NodeBase parent1 = getInstance("parent1");
+        final NodeBase parent2 = getInstance("parent2");
+
+        final ExplicitNode instance = getInstance("instance");
+
+        instance.addParent(parent1);
+
+        expectedException.expect(IllegalStateException.class);
+        instance.addParent(parent2);
+    }
+
+    @Test
+    public void testRemoveExistingParent() {
+        final ExplicitNode parent = getInstance("parent");
+        final ExplicitNode instance = getInstance("instance");
+
+        instance.addParent(parent);
+
+        instance.removeParent(parent);
+        assertEquals(null, instance.getParent());
+        assertEquals(null, parent.getChild());
+    }
+
+    @Test
+    public void testRemoveNonexistentParentThrows() {
+        final ExplicitNode parent = getInstance("parent");
+        final ExplicitNode instance = getInstance("instance");
+
+        expectedException.expect(IllegalArgumentException.class);
+        instance.removeParent(parent);
+    }
+
+    @Test
+    public void testClearExistingParent() {
+        final Start parent = new Start("parent");
+        final ExplicitNode instance = getInstance("instance");
+
+        instance.addParent(parent);
+
+        instance.clearParents();
+        assertEquals(null, instance.getParent());
+        assertEquals(null, parent.getChild());
+    }
+
+    @Test
+    public void testClearNonExistentParent() {
+        final Start parent = new Start("parent");
+        final ExplicitNode instance = getInstance("instance");
+
+        instance.clearParents();
+        assertEquals(null, instance.getParent());
+        assertEquals(null, parent.getChild());
+    }
+
+    @Test
+    public void testNormalAddedAsParentWhenItHasNoChild() {
+        final ExplicitNode instance = getInstance("start");
+        final NodeBase child = getInstance("child");
+
+        child.addParent(instance);
+
+        assertEquals(child, instance.getChild());
+    }
+
+    @Test
+    public void testNormalAddedAsParentWhenItAlreadyHasAChildThrows() {
+        final ExplicitNode instance = getInstance("instance");
+        final NodeBase child1 = new ExplicitNode("child1", null);
+        final NodeBase child2 = new ExplicitNode("child2", null);
+
+        child1.addParent(instance);
+
+        expectedException.expect(IllegalStateException.class);
+        child2.addParent(instance);
+    }
+
+    @Test
+    public void testNormalRemovedAsParent() {
+        final ExplicitNode instance = getInstance("instance");
+        final NodeBase child = getInstance("child");
+
+        child.addParent(instance);
+        child.removeParent(instance);
+
+        assertEquals(null, instance.getChild());
+    }
+
+    @Test
+    public void testGetChildren() {
+        final ExplicitNode instance = getInstance("start");
+        final NodeBase child = getInstance("child");
+
+        child.addParent(instance);
+
+        assertEquals(Arrays.asList(child), instance.getChildren());
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/dag/TestFork.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/dag/TestFork.java b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/dag/TestFork.java
new file mode 100644
index 0000000..f6a2666
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/test/java/org/apache/oozie/fluentjob/api/dag/TestFork.java
@@ -0,0 +1,156 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.dag;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import java.util.Arrays;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class TestFork extends TestNodeBase<Fork> {
+    @Rule
+    public final ExpectedException expectedException = ExpectedException.none();
+
+    @Override
+    protected Fork getInstance(final String name) {
+        return new Fork(name);
+    }
+
+    @Test
+    public void testAddParentWhenNoneAlreadyExists() {
+        final Start parent = new Start("parent");
+        final Fork instance = getInstance("instance");
+
+        instance.addParent(parent);
+        assertEquals(parent, instance.getParent());
+        assertEquals(instance, parent.getChild());
+    }
+
+    @Test
+    public void testAddParentWhenItAlreadyExistsThrows() {
+        final NodeBase parent1 = getInstance("parent1");
+        final NodeBase parent2 = getInstance("parent2");
+
+        final Fork instance = getInstance("instance");
+
+        instance.addParent(parent1);
+
+        expectedException.expect(IllegalStateException.class);
+        instance.addParent(parent2);
+    }
+
+    @Test
+    public void testRemoveExistingParent() {
+        final Start parent = new Start("parent");
+        final Fork instance = getInstance("instance");
+
+        instance.addParent(parent);
+
+        instance.removeParent(parent);
+        assertEquals(null, instance.getParent());
+        assertEquals(null, parent.getChild());
+    }
+
+    @Test
+    public void testRemoveNonexistentParentThrows() {
+        final Start parent = new Start("parent");
+        final Fork instance = getInstance("instance");
+
+        expectedException.expect(IllegalArgumentException.class);
+        instance.removeParent(parent);
+    }
+
+    @Test
+    public void testClearExistingParent() {
+        final Start parent = new Start("parent");
+        final Fork instance = getInstance("instance");
+
+        instance.addParent(parent);
+
+        instance.clearParents();
+        assertEquals(null, instance.getParent());
+        assertEquals(null, parent.getChild());
+    }
+
+    @Test
+    public void testClearNonExistentParent() {
+        final Start parent = new Start("parent");
+        final Fork instance = getInstance("instance");
+
+        instance.clearParents();
+        assertEquals(null, instance.getParent());
+        assertEquals(null, parent.getChild());
+    }
+
+    @Test
+    public void testForkAddedAsParentWhenItHasNoChild() {
+        final Fork instance = getInstance("instance");
+        final NodeBase child = getInstance("child");
+
+        child.addParent(instance);
+
+        assertEquals(Arrays.asList(child), instance.getChildren());
+    }
+
+    @Test
+    public void testForkAddedAsParentWhenItAlreadyHasAChild() {
+        final Fork instance = getInstance("instance");
+        final NodeBase child1 = new ExplicitNode("child1", null);
+        final NodeBase child2 = new ExplicitNode("child2", null);
+
+        child1.addParent(instance);
+        child2.addParent(instance);
+
+        assertEquals(Arrays.asList(child1, child2), instance.getChildren());
+    }
+
+    @Test
+    public void testForkRemovedAsParent() {
+        final Fork instance = getInstance("instance");
+        final NodeBase child1 = new ExplicitNode("child1", null);
+        final NodeBase child2 = new ExplicitNode("child2", null);
+        final NodeBase child3 = new ExplicitNode("child3", null);
+        final NodeBase child4 = new ExplicitNode("child4", null);
+        final NodeBase child5 = new ExplicitNode("child5", null);
+
+        child1.addParent(instance);
+        child2.addParent(instance);
+        child3.addParent(instance);
+        child4.addParent(instance);
+        child5.addParent(instance);
+
+        child5.removeParent(instance);
+
+        assertEquals(Arrays.asList(child1, child2, child3, child4), instance.getChildren());
+    }
+
+    @Test
+    public void testClose() {
+        final Fork instance = getInstance("instance");
+
+        final Join join = new Join("join", instance);
+
+        assertEquals(join, instance.getClosingJoin());
+        assertTrue(instance.isClosed());
+    }
+}


[10/16] oozie git commit: OOZIE-2339 [fluent-job] Minimum Viable Fluent Job API (daniel.becker, andras.piros via rkanter, gezapeti, pbacsko)

Posted by an...@apache.org.
http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/dag/Join.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/dag/Join.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/dag/Join.java
new file mode 100644
index 0000000..2a1d491
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/dag/Join.java
@@ -0,0 +1,36 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.dag;
+
+/**
+ * A class representing fork nodes in an Oozie workflow definition DAG. These nodes are generated automatically,
+ * the end user should not need to use this class directly.
+ */
+public class Join extends JoiningNodeBase<Fork> {
+
+    /**
+     * Create a new end node with the given name.
+     * @param name The name of the new end node.
+     * @param fork The fork that this {@link Join} node closes.
+     */
+    public Join(final String name, final Fork fork) {
+        super(name, fork);
+        fork.close(this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/dag/JoiningNodeBase.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/dag/JoiningNodeBase.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/dag/JoiningNodeBase.java
new file mode 100644
index 0000000..d5dc9fb
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/dag/JoiningNodeBase.java
@@ -0,0 +1,126 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.dag;
+
+import com.google.common.base.Preconditions;
+import org.apache.oozie.fluentjob.api.Condition;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * An abstract base class for {@link Join} and {@link DecisionJoin}.
+ * @param <T> The type of the node for which this class is the closing pair.
+ */
+public abstract class JoiningNodeBase<T> extends NodeBase {
+    private final List<NodeBase> parents;
+    private NodeBase child;
+
+    private final T branching;
+
+    JoiningNodeBase(final String name, final T branching) {
+        super(name);
+
+        this.parents = new ArrayList<>();
+        this.branching = branching;
+    }
+
+    /**
+     * Returns an unmodifiable list of the parents of this node.
+     * @return An unmodifiable list of the parents of this node.
+     */
+    public List<NodeBase> getParents() {
+        return Collections.unmodifiableList(parents);
+    }
+
+    @Override
+    public void addParent(final NodeBase parent) {
+        if (parent != null) {
+            parent.addChild(this);
+        }
+
+        parents.add(parent);
+    }
+
+    @Override
+    public void addParentWithCondition(final Decision parent, final Condition condition) {
+        if (parent != null) {
+            parent.addChildWithCondition(this,  condition);
+        }
+
+        parents.add(parent);
+    }
+
+    @Override
+    public void addParentDefaultConditional(final Decision parent) {
+        if (parent != null) {
+            parent.addDefaultChild(this);
+        }
+
+        parents.add(parent);
+    }
+
+    @Override
+    public void removeParent(final NodeBase parent) {
+        Preconditions.checkArgument(parents.remove(parent), "Trying to remove a nonexistent parent");
+
+        parent.removeChild(this);
+    }
+
+    @Override
+    public void clearParents() {
+        final List<NodeBase> oldParents = new ArrayList<>(parents);
+        for (final NodeBase parent : oldParents) {
+            removeParent(parent);
+        }
+    }
+
+    @Override
+    public List<NodeBase> getChildren() {
+        if (child == null) {
+            return Arrays.asList();
+        }
+
+        return Arrays.asList(child);
+    }
+
+    public NodeBase getChild() {
+        return child;
+    }
+
+    T getBranchingPair() {
+        return branching;
+    }
+
+    @Override
+    protected void addChild(final NodeBase child) {
+        Preconditions.checkState(this.child == null, "JoiningNodeBase nodes cannot have multiple children.");
+
+        this.child = child;
+    }
+
+    @Override
+    protected void removeChild(final NodeBase child) {
+        Preconditions.checkArgument(this.child == child, "Trying to remove a nonexistent child.");
+
+        this.child = null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/dag/NodeBase.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/dag/NodeBase.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/dag/NodeBase.java
new file mode 100644
index 0000000..f9a3235
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/dag/NodeBase.java
@@ -0,0 +1,104 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.dag;
+
+import org.apache.oozie.fluentjob.api.Condition;
+
+import java.util.List;
+
+/**
+ * An abstract base class for nodes in the intermediate graph representation of the workflow. Nodes in this graph may
+ * contain information about nodes of the API level graph ({@link ExplicitNode}) or be generated control nodes
+ * (for example {@link Fork}).
+ * <p>
+ * These nodes should not be used directly by the end user.
+ */
+public abstract class NodeBase {
+    private final String name;
+
+    /**
+     * Creates a new {@link NodeBase} object with the given name.
+     * @param name The name of the new {@link NodeBase} object.
+     */
+    protected NodeBase(final String name) {
+        this.name = name;
+    }
+
+    /**
+     * Returns the name of this {@link NodeBase} object.
+     * @return The name of this {@link NodeBase} object.
+     */
+    public String getName() {
+        return name;
+    }
+
+    /**
+     * Adds the provided node as a parent of this {@link NodeBase} object.
+     * @param parent The new parent of this {@link NodeBase} object.
+     */
+    public abstract void addParent(final NodeBase parent);
+
+    /**
+     * Adds the provided node as a conditional parent of this {@link NodeBase} object.
+     * @param parent The new conditional parent of this {@link NodeBase} object.
+     * @param condition The condition which must be true in addition the parent completing successfully for this node
+     *                  to be executed.
+     */
+    public abstract void addParentWithCondition(final Decision parent, final Condition condition);
+
+    /**
+     * Adds the provided node as the default conditional parent of this {@link NodeBase} object.
+     * @param parent The new conditional parent of this {@link NodeBase} object.
+     */
+    public abstract void addParentDefaultConditional(final Decision parent);
+
+    /**
+     * Removes a parent (whether or not conditional).
+     * @param parent The parent to remove.
+     * @throws IllegalStateException if {@code parent} is not the parent of this node.
+     */
+    public abstract void removeParent(final NodeBase parent);
+
+    /**
+     * Removes all parents (whether or not conditional) of this {@link NodeBase} object.
+     */
+    public abstract void clearParents();
+
+    /**
+     * Returns all the children (whether conditional or not) of this {@link NodeBase} object.
+     * @return All the children (whether conditional or not) of this {@link NodeBase} object.
+     */
+    public abstract List<NodeBase> getChildren();
+
+    /**
+     * Adds a child to this {@link NodeBase} object. This method should only be used in the implementation of the
+     * {@link NodeBase#addParent(NodeBase)} method of a subclass of {@link NodeBase}, so that whenever a node is assigned a new
+     * parent, the parent also knows about the child.
+     * @param child The node to be added as the child of this {@link NodeBase}.
+     */
+    protected abstract void addChild(final NodeBase child);
+
+    /**
+     * Removes the specified child from this {@link NodeBase} object. This method should only be used in the implementation of the
+     * {@link NodeBase#removeParent(NodeBase)} method of a subclass of {@link NodeBase}, so that whenever a child is removed from a
+     * node, the parent also knows about the child having been removed.
+     * @param child The node to be removed as the child of this {@link NodeBase}.
+     */
+    protected abstract void removeChild(final NodeBase child);
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/dag/Start.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/dag/Start.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/dag/Start.java
new file mode 100644
index 0000000..5b639a6
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/dag/Start.java
@@ -0,0 +1,120 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.dag;
+
+import com.google.common.base.Preconditions;
+import org.apache.oozie.fluentjob.api.Condition;
+
+import java.util.Arrays;
+import java.util.List;
+
+/**
+ * A class representing start nodes in an Oozie workflow definition DAG. These nodes are generated automatically,
+ * the end user should not need to use this class directly.
+ */
+public class Start extends NodeBase {
+    private NodeBase child;
+
+    /**
+     * Create a new start node with the given name.
+     * @param name The name of the new start node.
+     */
+    public Start(final String name) {
+        super(name);
+    }
+
+    /**
+     * Returns the child of this {@link Start} object.
+     * @return the child {@link NodeBase}
+     */
+    public NodeBase getChild() {
+        return child;
+    }
+
+    /**
+     * Throws {@link IllegalStateException} as start nodes cannot have parents.
+     * @param parent The new parent of this {@link NodeBase} object.
+     * @throws IllegalStateException always.
+     */
+    @Override
+    public void addParent(final NodeBase parent) {
+        throw new IllegalStateException("Start nodes cannot have parents.");
+    }
+
+    /**
+     * Throws {@link IllegalStateException} as start nodes cannot have parents.
+     * @param parent The new parent of this {@link NodeBase} object.
+     * @throws IllegalStateException always.
+     */
+    @Override
+    public void addParentWithCondition(final Decision parent, final Condition condition) {
+        throw new IllegalStateException("Start nodes cannot have parents.");
+    }
+
+    /**
+     * Throws {@link IllegalStateException} as start nodes cannot have parents.
+     * @param parent The new parent of this {@link NodeBase} object.
+     * @throws IllegalStateException always.
+     */
+    @Override
+    public void addParentDefaultConditional(Decision parent) {
+        throw new IllegalStateException("Start nodes cannot have parents.");
+    }
+
+    /**
+     * Throws {@link IllegalStateException} as start nodes cannot have parents.
+     * @param parent The new parent of this {@link NodeBase} object.
+     * @throws IllegalStateException always.
+     */
+    @Override
+    public void removeParent(NodeBase parent) {
+        throw new IllegalStateException("Start nodes cannot have parents.");
+    }
+
+    /**
+     * Does nothing as start nodes cannot have parents.
+     */
+    @Override
+    public void clearParents() {
+
+    }
+
+    @Override
+    public List<NodeBase> getChildren() {
+        if (child == null) {
+            return Arrays.asList();
+        }
+
+        return Arrays.asList(child);
+    }
+
+    @Override
+    protected void addChild(final NodeBase child) {
+        Preconditions.checkState(this.child == null, "Start nodes cannot have multiple children.");
+
+        this.child =  child;
+    }
+
+    @Override
+    protected void removeChild(final NodeBase child) {
+        Preconditions.checkArgument(this.child == child, "Trying to remove a nonexistent child.");
+
+        this.child = null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/factory/WorkflowFactory.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/factory/WorkflowFactory.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/factory/WorkflowFactory.java
new file mode 100644
index 0000000..3e11f52
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/factory/WorkflowFactory.java
@@ -0,0 +1,35 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.factory;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.oozie.fluentjob.api.workflow.Workflow;
+
+/**
+ * A marker interface telling callers the implementor can and will create a {@link Workflow} instance.
+ * <p>
+ * All classes meant to be used as an input for {@code OozieCLI}'s Jobs API {@code .jar} checking, submission
+ * or running methods must implement {@code WorkflowFactory}.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public interface WorkflowFactory {
+    Workflow create();
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/BooleanToFLAGConverter.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/BooleanToFLAGConverter.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/BooleanToFLAGConverter.java
new file mode 100644
index 0000000..6431aa2
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/BooleanToFLAGConverter.java
@@ -0,0 +1,50 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.mapping;
+
+import org.apache.oozie.fluentjob.api.generated.workflow.FLAG;
+import org.apache.oozie.fluentjob.api.generated.workflow.ObjectFactory;
+import org.dozer.DozerConverter;
+
+/**
+ * A {@link DozerConverter} converting from {@link Boolean} to JAXB {@link FLAG}.
+ */
+public class BooleanToFLAGConverter extends DozerConverter<Boolean, FLAG> {
+    public BooleanToFLAGConverter() {
+        super(Boolean.class, FLAG.class);
+    }
+
+    @Override
+    public FLAG convertTo(final Boolean source, final FLAG destination) {
+        if (source == null) {
+            return null;
+        }
+
+        if (source) {
+            return new ObjectFactory().createFLAG();
+        }
+
+        return null;
+    }
+
+    @Override
+    public Boolean convertFrom(final FLAG source, final Boolean destination) {
+        throw new UnsupportedOperationException("This mapping is not bidirectional.");
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/BooleanToShellFLAGConverter.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/BooleanToShellFLAGConverter.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/BooleanToShellFLAGConverter.java
new file mode 100644
index 0000000..c6a6e8b
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/BooleanToShellFLAGConverter.java
@@ -0,0 +1,50 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.mapping;
+
+import org.apache.oozie.fluentjob.api.generated.action.shell.FLAG;
+import org.apache.oozie.fluentjob.api.generated.action.shell.ObjectFactory;
+import org.dozer.DozerConverter;
+
+/**
+ * A {@link DozerConverter} converting from {@link Boolean} to JAXB {@link FLAG}.
+ */
+public class BooleanToShellFLAGConverter extends DozerConverter<Boolean, FLAG> {
+    public BooleanToShellFLAGConverter() {
+        super(Boolean.class, FLAG.class);
+    }
+
+    @Override
+    public FLAG convertTo(final Boolean source, final FLAG destination) {
+        if (source == null) {
+            return null;
+        }
+
+        if (source) {
+            return new ObjectFactory().createFLAG();
+        }
+
+        return null;
+    }
+
+    @Override
+    public Boolean convertFrom(final FLAG source, final Boolean destination) {
+        throw new UnsupportedOperationException("This mapping is not bidirectional.");
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/BooleanToSshFLAGConverter.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/BooleanToSshFLAGConverter.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/BooleanToSshFLAGConverter.java
new file mode 100644
index 0000000..8ae417d
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/BooleanToSshFLAGConverter.java
@@ -0,0 +1,50 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.mapping;
+
+import org.apache.oozie.fluentjob.api.generated.action.ssh.FLAG;
+import org.apache.oozie.fluentjob.api.generated.action.ssh.ObjectFactory;
+import org.dozer.DozerConverter;
+
+/**
+ * A {@link DozerConverter} converting from {@link Boolean} to JAXB {@link FLAG}.
+ */
+public class BooleanToSshFLAGConverter extends DozerConverter<Boolean, FLAG> {
+    public BooleanToSshFLAGConverter() {
+        super(Boolean.class, FLAG.class);
+    }
+
+    @Override
+    public FLAG convertTo(final Boolean source, final FLAG destination) {
+        if (source == null) {
+            return null;
+        }
+
+        if (source) {
+            return new ObjectFactory().createFLAG();
+        }
+
+        return null;
+    }
+
+    @Override
+    public Boolean convertFrom(final FLAG source, final Boolean destination) {
+        throw new UnsupportedOperationException("This mapping is not bidirectional.");
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/CredentialsConverter.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/CredentialsConverter.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/CredentialsConverter.java
new file mode 100644
index 0000000..df6a96b
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/CredentialsConverter.java
@@ -0,0 +1,94 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.mapping;
+
+import org.apache.oozie.fluentjob.api.generated.workflow.CREDENTIAL;
+import org.apache.oozie.fluentjob.api.generated.workflow.CREDENTIALS;
+import org.apache.oozie.fluentjob.api.generated.workflow.ObjectFactory;
+import org.apache.oozie.fluentjob.api.workflow.ConfigurationEntry;
+import org.apache.oozie.fluentjob.api.workflow.Credential;
+import org.apache.oozie.fluentjob.api.workflow.Credentials;
+import org.dozer.DozerConverter;
+
+/**
+ * A {@link DozerConverter} converting from {@link Credentials} to JAXB {@link CREDENTIALS}.
+ */
+public class CredentialsConverter extends DozerConverter<Credentials, CREDENTIALS> {
+    private static final ObjectFactory OBJECT_FACTORY = new ObjectFactory();
+
+    public CredentialsConverter() {
+        super(Credentials.class, CREDENTIALS.class);
+    }
+
+    @Override
+    public CREDENTIALS convertTo(final Credentials source, CREDENTIALS destination) {
+        if (source == null) {
+            return null;
+        }
+
+        destination = ensureDestination(destination);
+
+        mapCredentials(source, destination);
+
+        return destination;
+    }
+
+    private CREDENTIALS ensureDestination(final CREDENTIALS destination) {
+        if (destination == null) {
+            return OBJECT_FACTORY.createCREDENTIALS();
+        }
+
+        return destination;
+    }
+
+    private void mapCredentials(final Credentials source, final CREDENTIALS destination) {
+        if (source.getCredentials() == null) {
+            return;
+        }
+
+        for (final Credential credential : source.getCredentials()) {
+            final CREDENTIAL mappedCredential = OBJECT_FACTORY.createCREDENTIAL();
+            mappedCredential.setName(credential.getName());
+            mappedCredential.setType(credential.getType());
+            mapConfigurationEntries(credential, mappedCredential);
+
+            destination.getCredential().add(mappedCredential);
+        }
+    }
+
+    private void mapConfigurationEntries(final Credential source, final CREDENTIAL destination) {
+        if (source.getConfigurationEntries() == null) {
+            return;
+        }
+
+        for (final ConfigurationEntry configurationEntry : source.getConfigurationEntries()) {
+            final CREDENTIAL.Property mappedProperty = OBJECT_FACTORY.createCREDENTIALProperty();
+            mappedProperty.setName(configurationEntry.getName());
+            mappedProperty.setValue(configurationEntry.getValue());
+            mappedProperty.setDescription(configurationEntry.getDescription());
+
+            destination.getProperty().add(mappedProperty);
+        }
+    }
+
+    @Override
+    public Credentials convertFrom(final CREDENTIALS source, final Credentials destination) {
+        throw new UnsupportedOperationException("This mapping is not bidirectional.");
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/DecisionConverter.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/DecisionConverter.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/DecisionConverter.java
new file mode 100644
index 0000000..c28f082
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/DecisionConverter.java
@@ -0,0 +1,122 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.mapping;
+
+import com.google.common.base.Preconditions;
+import org.apache.oozie.fluentjob.api.Condition;
+import org.apache.oozie.fluentjob.api.generated.workflow.CASE;
+import org.apache.oozie.fluentjob.api.generated.workflow.DECISION;
+import org.apache.oozie.fluentjob.api.generated.workflow.DEFAULT;
+import org.apache.oozie.fluentjob.api.generated.workflow.ObjectFactory;
+import org.apache.oozie.fluentjob.api.generated.workflow.SWITCH;
+import org.apache.oozie.fluentjob.api.dag.DagNodeWithCondition;
+import org.apache.oozie.fluentjob.api.dag.Decision;
+import org.apache.oozie.fluentjob.api.dag.NodeBase;
+import org.dozer.DozerConverter;
+import org.dozer.Mapper;
+import org.dozer.MapperAware;
+
+import java.util.List;
+
+/**
+ * A {@link DozerConverter} converting from {@link Decision} to JAXB {@link DECISION}.
+ */
+public class DecisionConverter extends DozerConverter<Decision, DECISION> implements MapperAware {
+    private static final ObjectFactory OBJECT_FACTORY = new ObjectFactory();
+    private Mapper mapper;
+
+    public DecisionConverter() {
+        super(Decision.class, DECISION.class);
+    }
+
+    @Override
+    public DECISION convertTo(final Decision source, DECISION destination) {
+        destination = ensureDestination(destination);
+
+        mapName(source, destination);
+
+        mapTransitions(source, destination);
+
+        return destination;
+    }
+
+    @Override
+    public Decision convertFrom(final DECISION source, final Decision destination) {
+        throw new UnsupportedOperationException("This mapping is not bidirectional.");
+    }
+
+    private Mapper checkAndGetMapper() {
+        Preconditions.checkNotNull(mapper, "mapper should be set");
+        return mapper;
+    }
+
+    @Override
+    public void setMapper(final Mapper mapper) {
+        this.mapper = mapper;
+    }
+
+    private void mapName(final Decision source, final DECISION destination) {
+        final String name = source.getName();
+        destination.setName(name);
+    }
+
+    private void mapTransitions(final Decision source, final DECISION destination) {
+        final NodeBase defaultNode = source.getDefaultChild();
+
+        Preconditions.checkState(defaultNode != null, "No default transition found.");
+
+        final NodeBase realDefaultNode = RealChildLocator.findRealChild(defaultNode);
+
+        final DEFAULT defaultCase = OBJECT_FACTORY.createDEFAULT();
+        defaultCase.setTo(realDefaultNode.getName());
+        destination.getSwitch().setDefault(defaultCase);
+
+        final List<DagNodeWithCondition> childrenIncludingDefault = source.getChildrenWithConditions();
+
+        // The default child is the last on the list, we remove it as we have already handled that.
+        final List<DagNodeWithCondition> children = childrenIncludingDefault.subList(0, childrenIncludingDefault.size() - 1);
+        final List<CASE> cases = destination.getSwitch().getCase();
+
+        for (final DagNodeWithCondition childWithCondition : children) {
+            final NodeBase child = childWithCondition.getNode();
+            final NodeBase realChild = RealChildLocator.findRealChild(child);
+
+            final Condition condition = childWithCondition.getCondition();
+
+            final DagNodeWithCondition realChildWithCondition = new DagNodeWithCondition(realChild, condition);
+
+            final CASE mappedCase = checkAndGetMapper().map(realChildWithCondition, CASE.class);
+            cases.add(mappedCase);
+        }
+    }
+
+    private DECISION ensureDestination(final DECISION destination) {
+        DECISION result = destination;
+        if (result == null) {
+            result = OBJECT_FACTORY.createDECISION();
+        }
+
+        if (result.getSwitch() == null) {
+            final SWITCH _switch = OBJECT_FACTORY.createSWITCH();
+            result.setSwitch(_switch);
+        }
+
+        return result;
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/DistcpConfigurationConverter.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/DistcpConfigurationConverter.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/DistcpConfigurationConverter.java
new file mode 100644
index 0000000..fffb734
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/DistcpConfigurationConverter.java
@@ -0,0 +1,81 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.mapping;
+
+import org.apache.oozie.fluentjob.api.generated.action.distcp.CONFIGURATION;
+import org.apache.oozie.fluentjob.api.generated.action.distcp.ObjectFactory;
+import org.dozer.DozerConverter;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * A {@link DozerConverter} converting from {@link Map} to JAXB {@link CONFIGURATION}.
+ */
+public class DistcpConfigurationConverter extends DozerConverter<Map, CONFIGURATION> {
+    private static final ObjectFactory OBJECT_FACTORY = new ObjectFactory();
+
+    public DistcpConfigurationConverter() {
+        super(Map.class, CONFIGURATION.class);
+    }
+
+    @Override
+    public CONFIGURATION convertTo(final Map source, CONFIGURATION destination) {
+        if (source == null) {
+            return null;
+        }
+
+        destination = ensureDestination(destination);
+
+        mapEntries(source, destination);
+
+        return destination;
+    }
+
+    private CONFIGURATION ensureDestination(CONFIGURATION destination) {
+        if (destination == null) {
+            destination = OBJECT_FACTORY.createCONFIGURATION();
+        }
+
+        return destination;
+    }
+
+    private void mapEntries(final Map source, final CONFIGURATION destination) {
+        if (source != null) {
+            final List<CONFIGURATION.Property> targetProperties = new ArrayList<>();
+
+            for (final Object objectKey : source.keySet()) {
+                final String name = objectKey.toString();
+                final String value = source.get(name).toString();
+                final CONFIGURATION.Property targetProperty = OBJECT_FACTORY.createCONFIGURATIONProperty();
+                targetProperty.setName(name);
+                targetProperty.setValue(value);
+                targetProperties.add(targetProperty);
+            }
+
+            destination.setProperty(targetProperties);
+        }
+    }
+
+    @Override
+    public Map convertFrom(final CONFIGURATION source, final Map destination) {
+        throw new UnsupportedOperationException("This mapping is not bidirectional.");
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/DistcpPrepareConverter.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/DistcpPrepareConverter.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/DistcpPrepareConverter.java
new file mode 100644
index 0000000..64a7b35
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/DistcpPrepareConverter.java
@@ -0,0 +1,97 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.mapping;
+
+import org.apache.oozie.fluentjob.api.generated.action.distcp.DELETE;
+import org.apache.oozie.fluentjob.api.generated.action.distcp.MKDIR;
+import org.apache.oozie.fluentjob.api.generated.action.distcp.ObjectFactory;
+import org.apache.oozie.fluentjob.api.generated.action.distcp.PREPARE;
+import org.apache.oozie.fluentjob.api.action.Delete;
+import org.apache.oozie.fluentjob.api.action.Mkdir;
+import org.apache.oozie.fluentjob.api.action.Prepare;
+import org.dozer.DozerConverter;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * A {@link DozerConverter} converting from {@link Prepare} to JAXB {@link PREPARE}.
+ */
+public class DistcpPrepareConverter extends DozerConverter<Prepare, PREPARE> {
+    private static final ObjectFactory OBJECT_FACTORY = new ObjectFactory();
+
+    public DistcpPrepareConverter() {
+        super(Prepare.class, PREPARE.class);
+    }
+
+    @Override
+    public PREPARE convertTo(final Prepare source, PREPARE destination) {
+        if (source == null) {
+            return null;
+        }
+
+        destination = ensureDestination(destination);
+
+        mapDeletes(source, destination);
+
+        mapMkdirs(source, destination);
+
+        return destination;
+    }
+
+    private PREPARE ensureDestination(final PREPARE destination) {
+        if (destination == null) {
+            return OBJECT_FACTORY.createPREPARE();
+        }
+        return destination;
+    }
+
+    private void mapDeletes(final Prepare source, final PREPARE destination) {
+        if (source.getDeletes() != null) {
+            final List<DELETE> targetDeletes = new ArrayList<>();
+
+            for (final Delete sourceDelete : source.getDeletes()) {
+                final DELETE targetDelete = OBJECT_FACTORY.createDELETE();
+                targetDelete.setPath(sourceDelete.getPath());
+                targetDeletes.add(targetDelete);
+            }
+
+            destination.setDelete(targetDeletes);
+        }
+    }
+
+    private void mapMkdirs(final Prepare source, final PREPARE destination) {
+        if (source.getMkdirs() != null) {
+            final List<MKDIR> targetMkdirs = new ArrayList<>();
+
+            for (final Mkdir sourceMkDir: source.getMkdirs()) {
+                final MKDIR targetMkDir = OBJECT_FACTORY.createMKDIR();
+                targetMkDir.setPath(sourceMkDir.getPath());
+                targetMkdirs.add(targetMkDir);
+            }
+
+            destination.setMkdir(targetMkdirs);
+        }
+    }
+
+    @Override
+    public Prepare convertFrom(final PREPARE source, final Prepare destination) {
+        throw new UnsupportedOperationException("This mapping is not bidirectional.");
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/DozerBeanMapperSingleton.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/DozerBeanMapperSingleton.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/DozerBeanMapperSingleton.java
new file mode 100644
index 0000000..a08bd0e
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/DozerBeanMapperSingleton.java
@@ -0,0 +1,50 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.mapping;
+
+import org.dozer.DozerBeanMapper;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Instantiates and configures a {@link DozerBeanMapper} to use custom mapping files.
+ */
+public class DozerBeanMapperSingleton {
+    private static DozerBeanMapper mapper;
+
+    private static void init() {
+        mapper = new DozerBeanMapper();
+
+        final List<String> mappingFiles = new ArrayList<>();
+        mappingFiles.add("dozer_config.xml");
+        mappingFiles.add("mappingGraphToWORKFLOWAPP.xml");
+        mappingFiles.add("action_mappings.xml");
+
+        mapper.setMappingFiles(mappingFiles);
+    }
+
+    public static DozerBeanMapper instance() {
+        if (mapper == null) {
+            init();
+        }
+
+        return mapper;
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/ExplicitNodeConverter.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/ExplicitNodeConverter.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/ExplicitNodeConverter.java
new file mode 100644
index 0000000..7bb82e5
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/ExplicitNodeConverter.java
@@ -0,0 +1,296 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.mapping;
+
+import com.google.common.base.Strings;
+import com.google.common.collect.ImmutableMap;
+import org.apache.oozie.fluentjob.api.action.DistcpAction;
+import org.apache.oozie.fluentjob.api.action.EmailAction;
+import org.apache.oozie.fluentjob.api.action.FSAction;
+import org.apache.oozie.fluentjob.api.action.Hive2Action;
+import org.apache.oozie.fluentjob.api.action.HiveAction;
+import org.apache.oozie.fluentjob.api.action.JavaAction;
+import org.apache.oozie.fluentjob.api.action.MapReduceAction;
+import org.apache.oozie.fluentjob.api.action.Node;
+import org.apache.oozie.fluentjob.api.action.PigAction;
+import org.apache.oozie.fluentjob.api.action.ShellAction;
+import org.apache.oozie.fluentjob.api.action.SparkAction;
+import org.apache.oozie.fluentjob.api.action.SqoopAction;
+import org.apache.oozie.fluentjob.api.action.SshAction;
+import org.apache.oozie.fluentjob.api.action.SubWorkflowAction;
+import org.apache.oozie.fluentjob.api.generated.workflow.ACTION;
+import org.apache.oozie.fluentjob.api.generated.workflow.ACTIONTRANSITION;
+import org.apache.oozie.fluentjob.api.generated.workflow.FS;
+import org.apache.oozie.fluentjob.api.generated.workflow.JAVA;
+import org.apache.oozie.fluentjob.api.generated.workflow.MAPREDUCE;
+import org.apache.oozie.fluentjob.api.generated.workflow.ObjectFactory;
+import org.apache.oozie.fluentjob.api.generated.workflow.PIG;
+import org.apache.oozie.fluentjob.api.generated.workflow.SUBWORKFLOW;
+import org.apache.oozie.fluentjob.api.dag.DecisionJoin;
+import org.apache.oozie.fluentjob.api.dag.ExplicitNode;
+import org.apache.oozie.fluentjob.api.dag.NodeBase;
+import org.apache.oozie.fluentjob.api.workflow.Credential;
+import org.dozer.DozerConverter;
+import org.dozer.Mapper;
+import org.dozer.MapperAware;
+import com.google.common.base.Preconditions;
+
+import javax.xml.bind.JAXBElement;
+import java.util.Map;
+
+/**
+ * A {@link DozerConverter} converting from {@link ExplicitNode} to JAXB {@link ACTION}.
+ * <p>
+ * Being the main entry point to Dozer framework, this class finds out the type of the actual
+ * {@code ExplicitNode}, delegates to further {@code DozerConverter}s based on the type,
+ * and connects the resulting JAXB objects based on the transitions between {@code ExplicitNode}s.
+ */
+public class ExplicitNodeConverter extends DozerConverter<ExplicitNode, ACTION> implements MapperAware {
+    private static final ObjectFactory WORKFLOW_OBJECT_FACTORY = new ObjectFactory();
+
+    private static final Map<Class<? extends Node>, Class<? extends Object>> ACTION_CLASSES = initActionClasses();
+
+    private static Map<Class<? extends Node>, Class<? extends Object>> initActionClasses() {
+        final ImmutableMap.Builder<Class<? extends Node>, Class<? extends Object>> builder = new ImmutableMap.Builder<>();
+
+        builder.put(MapReduceAction.class, MAPREDUCE.class)
+                .put(SubWorkflowAction.class, SUBWORKFLOW.class)
+                .put(FSAction.class, FS.class)
+                .put(EmailAction.class, org.apache.oozie.fluentjob.api.generated.action.email.ACTION.class)
+                .put(DistcpAction.class, org.apache.oozie.fluentjob.api.generated.action.distcp.ACTION.class)
+                .put(HiveAction.class, org.apache.oozie.fluentjob.api.generated.action.hive.ACTION.class)
+                .put(Hive2Action.class, org.apache.oozie.fluentjob.api.generated.action.hive2.ACTION.class)
+                .put(JavaAction.class, JAVA.class)
+                .put(PigAction.class, PIG.class)
+                .put(ShellAction.class, org.apache.oozie.fluentjob.api.generated.action.shell.ACTION.class)
+                .put(SparkAction.class, org.apache.oozie.fluentjob.api.generated.action.spark.ACTION.class)
+                .put(SqoopAction.class, org.apache.oozie.fluentjob.api.generated.action.sqoop.ACTION.class)
+                .put(SshAction.class, org.apache.oozie.fluentjob.api.generated.action.ssh.ACTION.class);
+
+        return builder.build();
+    }
+
+    private Mapper mapper;
+
+    public ExplicitNodeConverter() {
+        super(ExplicitNode.class, ACTION.class);
+    }
+
+    @Override
+    public ACTION convertTo(final ExplicitNode source, ACTION destination) {
+        destination = ensureDestination(destination);
+
+        mapAttributes(source, destination);
+
+        mapTransitions(source, destination);
+
+        mapActionContent(source, destination);
+
+        return destination;
+    }
+
+    private ACTION ensureDestination(ACTION destination) {
+        if (destination == null) {
+            destination = WORKFLOW_OBJECT_FACTORY.createACTION();
+        }
+        return destination;
+    }
+
+    @Override
+    public ExplicitNode convertFrom(final ACTION source, final ExplicitNode destination) {
+        throw new UnsupportedOperationException("This mapping is not bidirectional.");
+    }
+
+    private Mapper checkAndGetMapper() {
+        Preconditions.checkNotNull(mapper, "mapper should be set");
+        return mapper;
+    }
+
+    @Override
+    public void setMapper(final Mapper mapper) {
+        this.mapper = mapper;
+    }
+
+    private void mapAttributes(final ExplicitNode source, final ACTION destination) {
+        destination.setName(source.getName());
+
+        final StringBuilder credBuilder = new StringBuilder();
+        for (final Credential credential : source.getRealNode().getCredentials()) {
+            if (credBuilder.length() > 0) {
+                credBuilder.append(",");
+            }
+            credBuilder.append(credential.getName());
+        }
+        if (!Strings.isNullOrEmpty(credBuilder.toString())) {
+            destination.setCred(credBuilder.toString());
+        }
+
+        final Integer retryInterval = source.getRealNode().getRetryInterval();
+        if (retryInterval != null) {
+            destination.setRetryInterval(retryInterval.toString());
+        }
+
+        final Integer retryMax = source.getRealNode().getRetryMax();
+        if (retryMax != null) {
+            destination.setRetryMax(retryMax.toString());
+        }
+
+        if (!Strings.isNullOrEmpty(source.getRealNode().getRetryPolicy())) {
+            destination.setRetryPolicy(source.getRealNode().getRetryPolicy());
+        }
+    }
+
+    private void mapTransitions(final ExplicitNode source, final ACTION destination) {
+        // Error transitions are handled at the level of converting the Graph object to a WORKFLOWAPP object.
+        final ACTIONTRANSITION ok = WORKFLOW_OBJECT_FACTORY.createACTIONTRANSITION();
+        final NodeBase child = findNonDecisionNodeDescendant(source);
+
+        ok.setTo(child == null ? "" : child.getName());
+
+        destination.setOk(ok);
+    }
+
+    private NodeBase findNonDecisionNodeDescendant(final ExplicitNode source) {
+        if (source.getChild() instanceof DecisionJoin) {
+            return ((DecisionJoin) source.getChild()).getFirstNonDecisionJoinDescendant();
+        }
+        return source.getChild();
+    }
+
+    private void mapActionContent(final ExplicitNode source, final ACTION destination) {
+        final Node realNode = source.getRealNode();
+
+        Object actionTypeObject = null;
+        if (ACTION_CLASSES.containsKey(realNode.getClass())) {
+            final Class<? extends Object> mappedClass = ACTION_CLASSES.get(realNode.getClass());
+            actionTypeObject = checkAndGetMapper().map(realNode, mappedClass);
+        }
+
+        Preconditions.checkNotNull(actionTypeObject, "actionTypeObject");
+
+        if (actionTypeObject instanceof MAPREDUCE) {
+            destination.setMapReduce((MAPREDUCE) actionTypeObject);
+        }
+        else if (actionTypeObject instanceof PIG) {
+            destination.setPig((PIG) actionTypeObject);
+        }
+        else if (actionTypeObject instanceof SUBWORKFLOW) {
+            destination.setSubWorkflow((SUBWORKFLOW) actionTypeObject);
+        }
+        else if (actionTypeObject instanceof FS) {
+            destination.setFs((FS) actionTypeObject);
+        }
+        else if (actionTypeObject instanceof JAVA) {
+            destination.setJava((JAVA) actionTypeObject);
+        }
+        else if (actionTypeObject instanceof org.apache.oozie.fluentjob.api.generated.action.email.ACTION) {
+            setEmail((org.apache.oozie.fluentjob.api.generated.action.email.ACTION) actionTypeObject, destination);
+        }
+        else if (actionTypeObject instanceof org.apache.oozie.fluentjob.api.generated.action.distcp.ACTION) {
+            setDistcp((org.apache.oozie.fluentjob.api.generated.action.distcp.ACTION) actionTypeObject, destination);
+        }
+        else if (actionTypeObject instanceof org.apache.oozie.fluentjob.api.generated.action.hive.ACTION) {
+            setHive((org.apache.oozie.fluentjob.api.generated.action.hive.ACTION) actionTypeObject, destination);
+        }
+        else if (actionTypeObject instanceof org.apache.oozie.fluentjob.api.generated.action.hive2.ACTION) {
+            setHive2((org.apache.oozie.fluentjob.api.generated.action.hive2.ACTION) actionTypeObject, destination);
+        }
+        else if (actionTypeObject instanceof org.apache.oozie.fluentjob.api.generated.action.shell.ACTION) {
+            setShell((org.apache.oozie.fluentjob.api.generated.action.shell.ACTION) actionTypeObject, destination);
+        }
+        else if (actionTypeObject instanceof org.apache.oozie.fluentjob.api.generated.action.spark.ACTION) {
+            setSpark((org.apache.oozie.fluentjob.api.generated.action.spark.ACTION) actionTypeObject, destination);
+        }
+        else if (actionTypeObject instanceof org.apache.oozie.fluentjob.api.generated.action.sqoop.ACTION) {
+            setSqoop((org.apache.oozie.fluentjob.api.generated.action.sqoop.ACTION) actionTypeObject, destination);
+        }
+        else if (actionTypeObject instanceof org.apache.oozie.fluentjob.api.generated.action.ssh.ACTION) {
+            setSsh((org.apache.oozie.fluentjob.api.generated.action.ssh.ACTION) actionTypeObject, destination);
+        }
+    }
+
+    private void setEmail(final org.apache.oozie.fluentjob.api.generated.action.email.ACTION source, final ACTION destination) {
+        final JAXBElement<?> jaxbElement =
+                new org.apache.oozie.fluentjob.api.generated.action.email.ObjectFactory().createEmail(source);
+        destination.setOther(jaxbElement);
+    }
+
+    private void setDistcp(final org.apache.oozie.fluentjob.api.generated.action.distcp.ACTION source, final ACTION destination) {
+        final JAXBElement<org.apache.oozie.fluentjob.api.generated.action.distcp.ACTION> jaxbElement =
+                new org.apache.oozie.fluentjob.api.generated.action.distcp.ObjectFactory().createDistcp(source);
+
+        destination.setOther(jaxbElement);
+    }
+
+    private void setHive(final org.apache.oozie.fluentjob.api.generated.action.hive.ACTION source, final ACTION destination) {
+        final JAXBElement<org.apache.oozie.fluentjob.api.generated.action.hive.ACTION> jaxbElement =
+                new org.apache.oozie.fluentjob.api.generated.action.hive.ObjectFactory().createHive(source);
+
+        destination.setOther(jaxbElement);
+    }
+
+    private void setHive2(final org.apache.oozie.fluentjob.api.generated.action.hive2.ACTION source, final ACTION destination) {
+        final JAXBElement<org.apache.oozie.fluentjob.api.generated.action.hive2.ACTION> jaxbElement =
+                new org.apache.oozie.fluentjob.api.generated.action.hive2.ObjectFactory().createHive2(source);
+
+        destination.setOther(jaxbElement);
+    }
+
+    private void setJava(final JAVA source, final ACTION destination) {
+        final JAXBElement<JAVA> jaxbElement =
+                new ObjectFactory().createJava(source);
+
+        destination.setOther(jaxbElement);
+    }
+
+    private void setPig(final PIG source, final ACTION destination) {
+        final JAXBElement<PIG> jaxbElement =
+                new ObjectFactory().createPig(source);
+
+        destination.setOther(jaxbElement);
+    }
+
+    private void setShell(final org.apache.oozie.fluentjob.api.generated.action.shell.ACTION source, final ACTION destination) {
+        final JAXBElement<org.apache.oozie.fluentjob.api.generated.action.shell.ACTION> jaxbElement =
+                new org.apache.oozie.fluentjob.api.generated.action.shell.ObjectFactory().createShell(source);
+
+        destination.setOther(jaxbElement);
+    }
+
+    private void setSpark(final org.apache.oozie.fluentjob.api.generated.action.spark.ACTION source, final ACTION destination) {
+        final JAXBElement<org.apache.oozie.fluentjob.api.generated.action.spark.ACTION> jaxbElement =
+                new org.apache.oozie.fluentjob.api.generated.action.spark.ObjectFactory().createSpark(source);
+
+        destination.setOther(jaxbElement);
+    }
+
+    private void setSqoop(final org.apache.oozie.fluentjob.api.generated.action.sqoop.ACTION source, final ACTION destination) {
+        final JAXBElement<org.apache.oozie.fluentjob.api.generated.action.sqoop.ACTION> jaxbElement =
+                new org.apache.oozie.fluentjob.api.generated.action.sqoop.ObjectFactory().createSqoop(source);
+
+        destination.setOther(jaxbElement);
+    }
+
+    private void setSsh(final org.apache.oozie.fluentjob.api.generated.action.ssh.ACTION source, final ACTION destination) {
+        final JAXBElement<org.apache.oozie.fluentjob.api.generated.action.ssh.ACTION> jaxbElement =
+                new org.apache.oozie.fluentjob.api.generated.action.ssh.ObjectFactory().createSsh(source);
+
+        destination.setOther(jaxbElement);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/ForkConverter.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/ForkConverter.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/ForkConverter.java
new file mode 100644
index 0000000..7ad6da9
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/ForkConverter.java
@@ -0,0 +1,76 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.mapping;
+
+import org.apache.oozie.fluentjob.api.generated.workflow.FORK;
+import org.apache.oozie.fluentjob.api.generated.workflow.FORKTRANSITION;
+import org.apache.oozie.fluentjob.api.generated.workflow.ObjectFactory;
+import org.apache.oozie.fluentjob.api.dag.Fork;
+import org.apache.oozie.fluentjob.api.dag.NodeBase;
+import org.dozer.DozerConverter;
+
+import java.util.List;
+
+/**
+ * A {@link DozerConverter} converting from {@link Fork} to JAXB {@link FORK}.
+ */
+public class ForkConverter extends DozerConverter<Fork, FORK> {
+    private static final ObjectFactory WORKFLOW_OBJECT_FACTORY = new ObjectFactory();
+
+    public ForkConverter() {
+        super(Fork.class, FORK.class);
+    }
+
+    @Override
+    public FORK convertTo(final Fork source, FORK destination) {
+        destination = ensureDestination(destination);
+
+        destination.setName(source.getName());
+
+        final List<FORKTRANSITION> transitions = destination.getPath();
+        for (final NodeBase child : source.getChildren()) {
+            final NodeBase realChild = RealChildLocator.findRealChild(child);
+            transitions.add(convertToFORKTRANSITION(realChild));
+        }
+
+        return destination;
+    }
+
+    private FORK ensureDestination(FORK destination) {
+        if (destination == null) {
+            destination = WORKFLOW_OBJECT_FACTORY.createFORK();
+        }
+        return destination;
+    }
+
+    @Override
+    public Fork convertFrom(final FORK source, final Fork destination) {
+        throw new UnsupportedOperationException("This mapping is not bidirectional.");
+    }
+
+    private FORKTRANSITION convertToFORKTRANSITION(final NodeBase source) {
+        final FORKTRANSITION destination = WORKFLOW_OBJECT_FACTORY.createFORKTRANSITION();
+
+        final NodeBase realChild = RealChildLocator.findRealChild(source);
+
+        destination.setStart(realChild.getName());
+
+        return destination;
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/GlobalConverter.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/GlobalConverter.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/GlobalConverter.java
new file mode 100644
index 0000000..80e8e57
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/GlobalConverter.java
@@ -0,0 +1,100 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.mapping;
+
+import com.google.common.base.Preconditions;
+import org.apache.oozie.fluentjob.api.generated.workflow.CONFIGURATION;
+import org.apache.oozie.fluentjob.api.generated.workflow.GLOBAL;
+import org.apache.oozie.fluentjob.api.generated.workflow.LAUNCHER;
+import org.apache.oozie.fluentjob.api.generated.workflow.ObjectFactory;
+import org.apache.oozie.fluentjob.api.workflow.Global;
+import org.dozer.DozerConverter;
+import org.dozer.Mapper;
+import org.dozer.MapperAware;
+
+/**
+ * A {@link DozerConverter} converting from {@link Global} to JAXB {@link GLOBAL}.
+ */
+public class GlobalConverter extends DozerConverter<Global, GLOBAL> implements MapperAware {
+    private static final ObjectFactory OBJECT_FACTORY = new ObjectFactory();
+
+    private Mapper mapper;
+
+    public GlobalConverter() {
+        super(Global.class, GLOBAL.class);
+    }
+
+    @Override
+    public GLOBAL convertTo(final Global source, GLOBAL destination) {
+        if (source == null) {
+            return null;
+        }
+
+        destination = ensureDestination(destination);
+
+        mapFields(source, destination);
+
+        return destination;
+    }
+
+    private GLOBAL ensureDestination(final GLOBAL destination) {
+        if (destination == null) {
+            return OBJECT_FACTORY.createGLOBAL();
+        }
+
+        return destination;
+    }
+
+    private void mapFields(final Global source, final GLOBAL destination) {
+        destination.setResourceManager(source.getResourceManager());
+        destination.setNameNode(source.getNameNode());
+        destination.getJobXml().addAll(source.getJobXmls());
+
+        mapLauncher(source, destination);
+
+        mapConfiguration(source, destination);
+    }
+
+    private void mapLauncher(final Global source, final GLOBAL destination) {
+        if (source.getLauncher() != null) {
+            destination.setLauncher(checkAndGetMapper().map(source.getLauncher(), LAUNCHER.class));
+        }
+    }
+
+    private void mapConfiguration(final Global source, final GLOBAL destination) {
+        if (source.getConfiguration() != null) {
+            destination.setConfiguration(checkAndGetMapper().map(source.getConfiguration(), CONFIGURATION.class));
+        }
+    }
+
+    @Override
+    public Global convertFrom(final GLOBAL source, final Global destination) {
+        throw new UnsupportedOperationException("This mapping is not bidirectional.");
+    }
+
+    private Mapper checkAndGetMapper() {
+        Preconditions.checkNotNull(mapper, "mapper should be set");
+        return mapper;
+    }
+
+    @Override
+    public void setMapper(final Mapper mapper) {
+        this.mapper = mapper;
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/GraphNodes.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/GraphNodes.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/GraphNodes.java
new file mode 100644
index 0000000..198998f
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/GraphNodes.java
@@ -0,0 +1,86 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.mapping;
+
+import org.apache.oozie.fluentjob.api.workflow.Credentials;
+import org.apache.oozie.fluentjob.api.workflow.Global;
+import org.apache.oozie.fluentjob.api.workflow.Parameters;
+import org.apache.oozie.fluentjob.api.dag.End;
+import org.apache.oozie.fluentjob.api.dag.NodeBase;
+import org.apache.oozie.fluentjob.api.dag.Start;
+
+import java.util.Collection;
+
+/**
+ * We use this class for better testability of the conversion from a Graph to {@code WORKFLOWAPP} - we don't have to build
+ * a Workflow to turn it into a {@code Graph}, we can generate the NodeBase's directly in the tests.
+ */
+public class GraphNodes {
+    private final String name;
+    private final Parameters parameters;
+    private final Global global;
+    private final Credentials credentials;
+    private final Start start;
+    private final End end;
+    private final Collection<NodeBase> nodes;
+
+    GraphNodes(final String name,
+               final Parameters parameters,
+               final Global global,
+               final Credentials credentials,
+               final Start start,
+               final End end,
+               final Collection<NodeBase> nodes) {
+        this.name = name;
+        this.parameters = parameters;
+        this.global = global;
+        this.credentials = credentials;
+        this.start = start;
+        this.end = end;
+        this.nodes = nodes;
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public Start getStart() {
+        return start;
+    }
+
+    public End getEnd() {
+        return end;
+    }
+
+    public Collection<NodeBase> getNodes() {
+        return nodes;
+    }
+
+    public Parameters getParameters() {
+        return parameters;
+    }
+
+    public Global getGlobal() {
+        return global;
+    }
+
+    public Credentials getCredentials() {
+        return credentials;
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/GraphNodesToWORKFLOWAPPConverter.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/GraphNodesToWORKFLOWAPPConverter.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/GraphNodesToWORKFLOWAPPConverter.java
new file mode 100644
index 0000000..da4691f
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/GraphNodesToWORKFLOWAPPConverter.java
@@ -0,0 +1,257 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.mapping;
+
+import com.google.common.base.Preconditions;
+import org.apache.oozie.fluentjob.api.generated.workflow.ACTION;
+import org.apache.oozie.fluentjob.api.generated.workflow.ACTIONTRANSITION;
+import org.apache.oozie.fluentjob.api.generated.workflow.CREDENTIALS;
+import org.apache.oozie.fluentjob.api.generated.workflow.DECISION;
+import org.apache.oozie.fluentjob.api.generated.workflow.END;
+import org.apache.oozie.fluentjob.api.generated.workflow.FORK;
+import org.apache.oozie.fluentjob.api.generated.workflow.GLOBAL;
+import org.apache.oozie.fluentjob.api.generated.workflow.JOIN;
+import org.apache.oozie.fluentjob.api.generated.workflow.KILL;
+import org.apache.oozie.fluentjob.api.generated.workflow.ObjectFactory;
+import org.apache.oozie.fluentjob.api.generated.workflow.PARAMETERS;
+import org.apache.oozie.fluentjob.api.generated.workflow.START;
+import org.apache.oozie.fluentjob.api.generated.workflow.WORKFLOWAPP;
+import org.apache.oozie.fluentjob.api.workflow.Credentials;
+import org.apache.oozie.fluentjob.api.workflow.Global;
+import org.apache.oozie.fluentjob.api.workflow.Parameters;
+import org.apache.oozie.fluentjob.api.action.ErrorHandler;
+import org.apache.oozie.fluentjob.api.action.Node;
+import org.apache.oozie.fluentjob.api.dag.Decision;
+import org.apache.oozie.fluentjob.api.dag.ExplicitNode;
+import org.apache.oozie.fluentjob.api.dag.Fork;
+import org.apache.oozie.fluentjob.api.dag.Join;
+import org.apache.oozie.fluentjob.api.dag.NodeBase;
+import org.dozer.DozerConverter;
+import org.dozer.Mapper;
+import org.dozer.MapperAware;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * A {@link DozerConverter} converting from {@link GraphNodes} to JAXB {@link WORKFLOWAPP}.
+ * <p>
+ * It performs tasks that are normally present when users write workflow XML files,
+ * but are hidden when using Jobs API:
+ * <ul>
+ *     <li>sets attributes</li>
+ *     <li>by delegating converts and sets JAXB objects of {@code <parameters>} section</li>
+ *     <li>by delegating converts and sets JAXB objects of {@code <global>} section</li>
+ *     <li>by delegating converts and sets JAXB objects of {@code <credentials>} section</li>
+ *     <li>generates and sets JAXB object of {@code <start>} node</li>
+ *     <li>generates and sets JAXB object of {@code <end>} node</li>
+ *     <li>generates and sets JAXB object of {@code <kill>} node</li>
+ *     <li>iterates through {@code GraphNodes} children, generates and sets child {@code <action>} instances</li>
+ * </ul>
+ */
+public class GraphNodesToWORKFLOWAPPConverter extends DozerConverter<GraphNodes, WORKFLOWAPP> implements MapperAware {
+    private static final ObjectFactory OBJECT_FACTORY = new ObjectFactory();
+
+    private Mapper mapper;
+
+    private final static Map<Class<? extends Object>, Class<? extends Object>> SOURCE_TARGET_CLASSES = new HashMap<>();
+    static {
+        SOURCE_TARGET_CLASSES.put(Decision.class, DECISION.class);
+        SOURCE_TARGET_CLASSES.put(Fork.class, FORK.class);
+        SOURCE_TARGET_CLASSES.put(Join.class, JOIN.class);
+        SOURCE_TARGET_CLASSES.put(ExplicitNode.class, ACTION.class);
+        SOURCE_TARGET_CLASSES.put(Parameters.class, PARAMETERS.class);
+        SOURCE_TARGET_CLASSES.put(Global.class, GLOBAL.class);
+        SOURCE_TARGET_CLASSES.put(Credentials.class, CREDENTIALS.class);
+    }
+
+    public GraphNodesToWORKFLOWAPPConverter() {
+        super(GraphNodes.class, WORKFLOWAPP.class);
+    }
+
+    @Override
+    public WORKFLOWAPP convertTo(final GraphNodes graphNodes, WORKFLOWAPP workflowapp) {
+        workflowapp = ensureWorkflowApp(workflowapp);
+
+        workflowapp.setName(graphNodes.getName());
+
+        mapParameters(graphNodes, workflowapp);
+
+        mapGlobal(graphNodes, workflowapp);
+
+        mapCredentials(graphNodes, workflowapp);
+
+        mapStart(graphNodes, workflowapp);
+
+        mapEnd(graphNodes, workflowapp);
+
+        final KILL kill = mapKill(workflowapp);
+
+        mapChildren(graphNodes, workflowapp, kill);
+
+        return workflowapp;
+    }
+
+    private WORKFLOWAPP ensureWorkflowApp(WORKFLOWAPP workflowapp) {
+        if (workflowapp == null) {
+            workflowapp = new ObjectFactory().createWORKFLOWAPP();
+        }
+        return workflowapp;
+    }
+
+    private void mapParameters(final GraphNodes graphNodes, final WORKFLOWAPP workflowapp) {
+        if (graphNodes.getParameters() == null) {
+            return;
+        }
+
+        final PARAMETERS mappedParameters = mapper.map(graphNodes.getParameters(), PARAMETERS.class);
+        workflowapp.setParameters(mappedParameters);
+    }
+
+    private void mapGlobal(final GraphNodes graphNodes, final WORKFLOWAPP workflowapp) {
+        if (graphNodes.getGlobal() == null) {
+            return;
+        }
+
+        final GLOBAL mappedGlobal = mapper.map(graphNodes.getGlobal(), GLOBAL.class);
+        workflowapp.setGlobal(mappedGlobal);
+    }
+
+    private void mapCredentials(final GraphNodes graphNodes, final WORKFLOWAPP workflowapp) {
+        if (graphNodes.getCredentials() == null) {
+            return;
+        }
+
+        final CREDENTIALS mappedCredentials = mapper.map(graphNodes.getCredentials(), CREDENTIALS.class);
+        workflowapp.setCredentials(mappedCredentials);
+    }
+
+    private void mapStart(final GraphNodes graphNodes, final WORKFLOWAPP workflowapp) {
+        final START start = mapper.map(graphNodes.getStart(), START.class);
+        workflowapp.setStart(start);
+    }
+
+    private void mapEnd(final GraphNodes graphNodes, final WORKFLOWAPP workflowapp) {
+        final END end = mapper.map(graphNodes.getEnd(), END.class);
+        workflowapp.setEnd(end);
+    }
+
+    private KILL mapKill(final WORKFLOWAPP workflowapp) {
+        final KILL kill = createKillNode();
+        workflowapp.getDecisionOrForkOrJoin().add(kill);
+        return kill;
+    }
+
+    private void mapChildren(final GraphNodes graphNodes, final WORKFLOWAPP workflowapp, final KILL kill) {
+        for (final NodeBase nodeBase : graphNodes.getNodes()) {
+            convertNode(nodeBase, workflowapp, kill);
+        }
+    }
+
+    @Override
+    public GraphNodes convertFrom(final WORKFLOWAPP workflowapp, final GraphNodes graphNodes) {
+        throw new UnsupportedOperationException("This mapping is not bidirectional.");
+    }
+
+    @Override
+    public void setMapper(final Mapper mapper) {
+        this.mapper = mapper;
+    }
+
+    private void convertNode(final NodeBase nodeBase, final WORKFLOWAPP workflowapp, final KILL kill) {
+        Preconditions.checkNotNull(nodeBase, "nodeBase");
+
+        final Class<?> sourceClass = nodeBase.getClass();
+        if (SOURCE_TARGET_CLASSES.containsKey(sourceClass)) {
+            final Object mappedObject = mapper.map(nodeBase, SOURCE_TARGET_CLASSES.get(sourceClass));
+
+            if (nodeBase instanceof ExplicitNode) {
+                final ACTION errorHandlerAction = addErrorTransition((ExplicitNode) nodeBase, (ACTION) mappedObject, kill);
+                if (errorHandlerAction != null) {
+                    workflowapp.getDecisionOrForkOrJoin().add(errorHandlerAction);
+                }
+            }
+
+            workflowapp.getDecisionOrForkOrJoin().add(mappedObject);
+        }
+    }
+
+    private KILL createKillNode() {
+        final KILL kill = OBJECT_FACTORY.createKILL();
+        kill.setName("kill");
+        kill.setMessage("Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]");
+
+        return kill;
+    }
+
+    private ACTION addErrorTransition(final ExplicitNode node, final ACTION action, final KILL kill) {
+        final ACTIONTRANSITION error = ensureError(action);
+
+        final ErrorHandler errorHandler = node.getRealNode().getErrorHandler();
+
+        if (errorHandler == null) {
+            error.setTo(kill.getName());
+
+            return null;
+        }
+        else {
+            final Node handlerNode = errorHandler.getHandlerNode();
+
+            final ACTION handlerAction = createErrorHandlerAction(handlerNode, kill);
+            error.setTo(handlerAction.getName());
+
+            return handlerAction;
+        }
+    }
+
+    private ACTIONTRANSITION ensureError(final ACTION action) {
+        ACTIONTRANSITION error = action.getError();
+
+        if (error == null) {
+            error = OBJECT_FACTORY.createACTIONTRANSITION();
+            action.setError(error);
+        }
+
+        return error;
+    }
+
+    private ACTIONTRANSITION ensureOk(final ACTION handlerAction) {
+        ACTIONTRANSITION ok = handlerAction.getOk();
+
+        if (ok == null) {
+            ok = OBJECT_FACTORY.createACTIONTRANSITION();
+            handlerAction.setOk(ok);
+        }
+
+        return ok;
+    }
+
+    private ACTION createErrorHandlerAction(final Node handlerNode, final KILL kill) {
+        final ExplicitNode explicitNode = new ExplicitNode(handlerNode.getName(), handlerNode);
+        final ACTION handlerAction = mapper.map(explicitNode, ACTION.class);
+
+        final ACTIONTRANSITION ok = ensureOk(handlerAction);
+        ok.setTo(kill.getName());
+
+        final ACTIONTRANSITION error = ensureError(handlerAction);
+        error.setTo(kill.getName());
+
+        return handlerAction;
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/GraphToWORKFLOWAPPConverter.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/GraphToWORKFLOWAPPConverter.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/GraphToWORKFLOWAPPConverter.java
new file mode 100644
index 0000000..5ed9412
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/GraphToWORKFLOWAPPConverter.java
@@ -0,0 +1,67 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.mapping;
+
+import com.google.common.base.Preconditions;
+import org.apache.oozie.fluentjob.api.generated.workflow.WORKFLOWAPP;
+import org.apache.oozie.fluentjob.api.dag.Graph;
+import org.dozer.DozerConverter;
+import org.dozer.Mapper;
+import org.dozer.MapperAware;
+
+/**
+ * A {@link DozerConverter} converting from {@link Graph} to JAXB {@link WORKFLOWAPP}.
+ * <p>
+ * Delegates to {@link GraphNodesToWORKFLOWAPPConverter}.
+ */
+public class GraphToWORKFLOWAPPConverter extends DozerConverter<Graph, WORKFLOWAPP> implements MapperAware {
+    private Mapper mapper;
+
+    public GraphToWORKFLOWAPPConverter() {
+        super(Graph.class, WORKFLOWAPP.class);
+    }
+
+    @Override
+    public WORKFLOWAPP convertTo(final Graph graph, final WORKFLOWAPP workflowapp) {
+        final GraphNodes graphNodes = new GraphNodes(graph.getName(),
+                graph.getParameters(),
+                graph.getGlobal(),
+                graph.getCredentials(),
+                graph.getStart(),
+                graph.getEnd(),
+                graph.getNodes());
+
+        return checkAndGetMapper().map(graphNodes, WORKFLOWAPP.class);
+    }
+
+    @Override
+    public Graph convertFrom(final WORKFLOWAPP workflowapp, final Graph graph) {
+        throw new UnsupportedOperationException("This mapping is not bidirectional.");
+    }
+
+    private Mapper checkAndGetMapper() {
+        Preconditions.checkNotNull(mapper, "mapper should be set");
+        return mapper;
+    }
+
+    @Override
+    public void setMapper(final Mapper mapper) {
+        this.mapper = mapper;
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/Hive2ConfigurationConverter.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/Hive2ConfigurationConverter.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/Hive2ConfigurationConverter.java
new file mode 100644
index 0000000..c67b5ae
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/Hive2ConfigurationConverter.java
@@ -0,0 +1,81 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.mapping;
+
+import org.apache.oozie.fluentjob.api.generated.action.hive2.CONFIGURATION;
+import org.apache.oozie.fluentjob.api.generated.action.hive2.ObjectFactory;
+import org.dozer.DozerConverter;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * A {@link DozerConverter} converting from {@link Map} to JAXB {@link CONFIGURATION}.
+ */
+public class Hive2ConfigurationConverter extends DozerConverter<Map, CONFIGURATION> {
+    private static final ObjectFactory OBJECT_FACTORY = new ObjectFactory();
+
+    public Hive2ConfigurationConverter() {
+        super(Map.class, CONFIGURATION.class);
+    }
+
+    @Override
+    public CONFIGURATION convertTo(final Map source, CONFIGURATION destination) {
+        if (source == null) {
+            return null;
+        }
+
+        destination = ensureDestination(destination);
+
+        mapEntries(source, destination);
+
+        return destination;
+    }
+
+    private CONFIGURATION ensureDestination(CONFIGURATION destination) {
+        if (destination == null) {
+            destination = OBJECT_FACTORY.createCONFIGURATION();
+        }
+
+        return destination;
+    }
+
+    private void mapEntries(final Map source, final CONFIGURATION destination) {
+        if (source != null) {
+            final List<CONFIGURATION.Property> targetProperties = new ArrayList<>();
+
+            for (final Object objectKey : source.keySet()) {
+                final String name = objectKey.toString();
+                final String value = source.get(name).toString();
+                final CONFIGURATION.Property targetProperty = OBJECT_FACTORY.createCONFIGURATIONProperty();
+                targetProperty.setName(name);
+                targetProperty.setValue(value);
+                targetProperties.add(targetProperty);
+            }
+
+            destination.setProperty(targetProperties);
+        }
+    }
+
+    @Override
+    public Map convertFrom(final CONFIGURATION source, final Map destination) {
+        throw new UnsupportedOperationException("This mapping is not bidirectional.");
+    }
+}


[15/16] oozie git commit: OOZIE-2339 [fluent-job] Minimum Viable Fluent Job API (daniel.becker, andras.piros via rkanter, gezapeti, pbacsko)

Posted by an...@apache.org.
http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/docs/src/site/twiki/DG_CustomActionExecutor.twiki
----------------------------------------------------------------------
diff --git a/docs/src/site/twiki/DG_CustomActionExecutor.twiki b/docs/src/site/twiki/DG_CustomActionExecutor.twiki
index 4acbf0d..7831484 100644
--- a/docs/src/site/twiki/DG_CustomActionExecutor.twiki
+++ b/docs/src/site/twiki/DG_CustomActionExecutor.twiki
@@ -66,6 +66,9 @@ Any configuration properties to be made available to this class should also be a
 
 The XML schema (XSD) for the new Actions should be added to oozie-site.xml, under the property 'oozie.service.WorkflowSchemaService.ext.schemas'. A comma separated list for multiple Action schemas.
 
+The XML schema (XSD) for the new action should be also added to Fluent Job API. Please refer to
+[[DG_FluentJobAPI#AE.C_Appendix_C_How_To_Extend][Fluent Job API :: How To Extend]] for details.
+
 The executor class should be placed along with the oozie webapp in the correct path. Once Oozie is restarted, the custom action node can be used in workflows.
 
 

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/docs/src/site/twiki/DG_Examples.twiki
----------------------------------------------------------------------
diff --git a/docs/src/site/twiki/DG_Examples.twiki b/docs/src/site/twiki/DG_Examples.twiki
index 13dfa28..5323a17 100644
--- a/docs/src/site/twiki/DG_Examples.twiki
+++ b/docs/src/site/twiki/DG_Examples.twiki
@@ -186,6 +186,13 @@ import java.util.Properties;
 Also asynchronous actions like FS action can be used / tested using =LocalOozie= / =OozieClient= API. Please see the module
 =oozie-mini= for details like =fs-decision.xml= workflow example.
 
+
+---++ Fluent Job API Examples
+
+There are some elaborate examples how to use the [[DG_FluentJobAPI][Fluent Job API]], under =examples/fluentjob/=. There are two
+simple examples covered under [[DG_FluentJobAPI#A_Simple_Example][Fluent Job API :: A Simple Example]] and
+[[DG_FluentJobAPI#A_More_Verbose_Example][Fluent Job API :: A More Verbose Example]].
+
 [[index][::Go back to Oozie Documentation Index::]]
 
 </noautolink>

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/docs/src/site/twiki/DG_FluentJobAPI.twiki
----------------------------------------------------------------------
diff --git a/docs/src/site/twiki/DG_FluentJobAPI.twiki b/docs/src/site/twiki/DG_FluentJobAPI.twiki
new file mode 100644
index 0000000..c8b764b
--- /dev/null
+++ b/docs/src/site/twiki/DG_FluentJobAPI.twiki
@@ -0,0 +1,376 @@
+<noautolink>
+
+[[index][::Go back to Oozie Documentation Index::]]
+
+---+!! Fluent Job API
+
+%TOC%
+
+---++ Introduction
+
+Oozie is a mature workflow scheduler system. XML is the standard way of defining workflow, coordinator, or bundle jobs.  For users
+who prefer an alternative, the Fluent Job API provides a Java interface instead.
+
+---+++ Motivation
+
+Prior to Oozie 5.1.0, the following ways were available to submit a workflow, coordinator, or bundle job: through Oozie CLI or via
+HTTP submit a generic workflow, coordinator, or bundle job, or submit a Pig, Hive, Sqoop, or MapReduce workflow job.
+
+As the generic way goes, the user has to have uploaded a workflow, coordinator, or bundle XML and all necessary dependencies like
+scripts, JAR or ZIP files, to HDFS beforehand, as well as have a =job.properties= file at command line and / or provide any
+missing parameters as part of the command.
+
+As the specific Pig, Hive, or Sqoop ways go, the user can provide all necessary parameters as part of the command issued. A
+ =workflow.xml= file will be generated with all the necessary details and stored to HDFS so that Oozie can grab it. Note that
+dependencies have to be uploaded to HDFS beforehand as well.
+
+There are some usability problems by using the XML job definition. XML is not an ideal way to express dependencies and a directed
+acyclic graph (DAG). We have to define a control flow, that is, which action follows the actual one. It's also necessary to build
+the whole control flow up front as XML is a declarative language that doesn't allow for dynamic evaluation. We have to define also
+boilerplate actions like start and end - those are present in every Oozie workflow, still need to explicitly define these.
+
+Apart from boilerplate actions, all the transitions between actions have also to be defined and taken care of. Furthermore, multiple
+similar actions cannot inherit common properties from each other. Again, the reason being workflows are defined in XML.
+
+Fork and join actions have to be defined in pairs, that is, there shouldn't be defined a join those incoming actions do not share
+the same ancestor fork. Such situations would result still in a DAG, but Oozie doesn't currently allow that. Note that with Fluent
+Job API new dependencies are introduced automatically when the DAG represented by API code couldn't have been expressed as
+fork / join pairs automatically.
+
+Either way, there were no programmatic ways to define workflow jobs. That doesn't mean users could not generate XML themselves -
+actually this is something HUE's Oozie UI also tries to target.
+
+---+++ Goals
+
+Fluent Job API aims to solve following from the user's perspective. It provides a Java API instead of declarative XML to define
+workflows. It defines dependencies across actions as opposed to defining a control flow. This is how data engineers and data
+scientists think. It eliminates all boilerplate actions and transitions. Only the necessary bits should be defined.
+
+Multiple similar actions can inherit from each other. In fact, since Fluent Job API is programmatic, it's possible to generate
+actions or even workflows using conditional, iterative, or recursive structures.
+
+Fluent Job API is backwards compatible with workflows defined as XML. That is, it should also be possible to have a Fluent Job API
+workflow rendered as XML, as well as coexist XML based and Fluent Job API based workflows in the same Oozie installation at the same
+time all workflow action types. When XSDs change, as few manual steps are necessary as possible both on API internal and public
+side.
+
+---+++ Non-goals
+
+The following points are not targeted for the initial release of Fluent Job API with Oozie 5.1.0. It doesn't provide API in any
+language other than Java. It doesn't provide a REPL. It doesn't allow for dynamic action instantiation depending on e.g. conditional
+logic. That is, using the API users still have to implement the whole workflow generation logic in advance.
+
+It has no support for programmatic coordinators and bundles, or even EL expressions created by API builders. Note that EL
+expressions for workflows can now be expressed the way these are used in XML workflow definitions, as strings in the right places.
+
+At the moment only the transformation from Fluent Job API to workflow definition is present. The other direction, from workflow
+definition to Fluent Job API JAR artifact, though sensible, is not supported.
+
+It's based only on latest XSDs. Older XSD versions, as well as conversion between XSD versions are not supported. Also no support
+for user-supplied custom actions / XSDs.
+
+Most of the non-goals may be targeted as enhancements of the Fluent Job API for future Oozie releases.
+
+---+++ Approach
+
+When using the Fluent Job API, the following points are different from the XML jobs definition. Instead of control flow (successor)
+definition, the user can define dependencies (parents of an action).
+
+All boilerplate (start, end, ...) has been eliminated, only nodes having useful actions have to be defined.
+
+Control flow and necessary boilerplate are generated automatically by keeping user defined dependencies, and possibly introducing
+new dependencies to keep Oozie workflow format of nested fork / join pairs. Note that not every dependency DAG can be expressed in
+the Oozie workflow format. When this is not possible, user is notified at build time.
+
+---++ How To Use
+
+---+++ A Simple Example
+
+The simplest thing to create using the Oozie Fluent Job API is a workflow consisting of only one action. Let's see how it goes, step
+by step.
+
+First, put the project =org.apache.oozie:oozie-fluent-job-api= to the build path. In case of a Maven managed build, create a new
+Maven project and declare a Maven dependency to =org.apache.oozie:oozie-fluent-job-api=.
+
+Then, create a class that =implements WorkflowFactory= and implement the method =WorkflowFactory#create()=. inside that method,
+create a =ShellAction= using =ShellActionBuilder=, fill in some attributes then create a =Workflow= using =WorkflowBuilder= using
+the =ShellAction= just built. Return the =Workflow=.
+
+Compile a Fluent Job API jar that has the =Main-Class= attribute set to the =WorkflowFactory= subclass just created,
+e.g. =shell-workflow.jar=.
+
+Moving on, [[DG_CommandLineTool#Checking_a_workflow_definition_generated_by_a_Fluent_Job_API_jar_file][check via command line]] that
+the compiled API JAR file is valid.
+
+As a finishing touch,
+[[DG_CommandLineTool#Running_a_workflow_definition_generated_by_a_Fluent_Job_API_jar_file][run via command line]] the Fluent Job API
+workflow.
+
+*For reference, a simplistic API JAR example consisting of a =Workflow= having only one =ShellAction=:*
+<verbatim>
+public class MyFirstWorkflowFactory implements WorkflowFactory {
+.
+    @Override
+    public Workflow create() {
+        final ShellAction shellAction = ShellActionBuilder.create()
+                .withName("shell-action")
+                .withResourceManager("${resourceManager}")
+                .withNameNode("${nameNode}")
+                .withConfigProperty("mapred.job.queue.name", "${queueName}")
+                .withExecutable("echo")
+                .withArgument("my_output=Hello Oozie")
+                .withCaptureOutput(true)
+                .build();
+.
+        final Workflow shellWorkflow = new WorkflowBuilder()
+                .withName("shell-workflow")
+                .withDagContainingNode(shellAction).build();
+.
+        return shellWorkflow;
+    }
+}
+</verbatim>
+
+*After check, the generated workflow XML looks like this:*
+<verbatim>
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+<workflow:workflow-app xmlns:workflow="uri:oozie:workflow:1.0"  xmlns:shell="uri:oozie:shell-action:1.0" name="shell-workflow">
+.
+    <workflow:start to="parent"/>
+.
+    <workflow:kill name="kill">
+        <workflow:message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</workflow:message>
+    </workflow:kill>
+.
+    <workflow:action name="shell-action">
+        <shell:shell>
+            <shell:resource-manager>${resourceManager}</shell:resource-manager>
+            <shell:name-node>${nameNode}</shell:name-node>
+            <shell:configuration>
+                <shell:property>
+                    <shell:name>mapred.job.queue.name</shell:name>
+                    <shell:value>${queueName}</shell:value>
+                </shell:property>
+            </shell:configuration>
+            <shell:exec>echo</shell:exec>
+            <shell:argument>my_output=Hello Oozie</shell:argument>
+            <shell:capture-output/>
+        </shell:shell>
+        <workflow:ok to="end"/>
+        <workflow:error to="kill"/>
+    </workflow:action>
+.
+    <workflow:end name="end"/>
+.
+</workflow:workflow-app>
+</verbatim>
+
+
+---+++ A More Verbose Example
+
+*Error handling*
+
+If you would like to provide some error handling in case of action failure, you should add an =ErrorHandler= to the =Node=
+representing the action. The error handler action will be added as the ="error-transition"= of the original action in the generated
+Oozie workflow XML. Both the ="ok-transition"= and the ="error-transition"= of the error handler action itself will lead to an
+autogenerated kill node.
+
+*Here you find an example consisting of a =Workflow= having three =ShellAction=s, an error handler =EmailAction=, and one =decision=
+to sort out which way to go:*
+<verbatim>
+public class MySecondWorkflowFactory implements WorkflowFactory {
+.
+    @Override
+    public Workflow create() {
+        final ShellAction parent = ShellActionBuilder.create()
+                .withName("parent")
+                .withResourceManager("${resourceManager}")
+                .withNameNode("${nameNode}")
+                .withConfigProperty("mapred.job.queue.name", "${queueName}")
+                .withExecutable("echo")
+                .withArgument("my_output=Hello Oozie")
+                .withCaptureOutput(true)
+                .withErrorHandler(ErrorHandler.buildAsErrorHandler(EmailActionBuilder.create()
+                        .withName("email-on-error")
+                        .withRecipient("somebody@apache.org")
+                        .withSubject("Workflow error")
+                        .withBody("Shell action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]")))
+                .build();
+.
+        ShellActionBuilder.createFromExistingAction(parent)
+                .withName("happy-path")
+                .withParentWithCondition(parent, "${wf:actionData('parent')['my_output'] eq 'Hello Oozie'}")
+                .withoutArgument("my_output=Hello Oozie")
+                .withArgument("Happy path")
+                .withCaptureOutput(null)
+                .build();
+.
+        ShellActionBuilder.createFromExistingAction(parent)
+                .withName("sad-path")
+                .withParentDefaultConditional(parent)
+                .withArgument("Sad path")
+                .build();
+.
+        final Workflow workflow = new WorkflowBuilder()
+                .withName("shell-example")
+                .withDagContainingNode(parent).build();
+.
+        return workflow;
+    }
+}
+</verbatim>
+
+*After check, the generated workflow XML looks like this:*
+<verbatim>
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+<workflow:workflow-app ... name="shell-example">
+.
+    <workflow:start to="parent"/>
+.
+    <workflow:kill name="kill">
+        <workflow:message>Action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</workflow:message>
+    </workflow:kill>
+.
+    <workflow:action name="email-on-error">
+        <email:email>
+            <email:to>somebody@apache.org</email:to>
+            <email:subject>Workflow error</email:subject>
+            <email:body>Shell action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</email:body>
+        </email:email>
+        <workflow:ok to="kill"/>
+        <workflow:error to="kill"/>
+    </workflow:action>
+.
+    <workflow:action name="parent">
+        <shell:shell>
+            <shell:resource-manager>${resourceManager}</shell:resource-manager>
+            <shell:name-node>${nameNode}</shell:name-node>
+            <shell:configuration>
+                <shell:property>
+                    <shell:name>mapred.job.queue.name</shell:name>
+                    <shell:value>${queueName}</shell:value>
+                </shell:property>
+            </shell:configuration>
+            <shell:exec>echo</shell:exec>
+            <shell:argument>my_output=Hello Oozie</shell:argument>
+            <shell:capture-output/>
+        </shell:shell>
+        <workflow:ok to="decision1"/>
+        <workflow:error to="email-on-error"/>
+    </workflow:action>
+.
+    <workflow:decision name="decision1">
+        <workflow:switch>
+            <workflow:case to="happy-path">${wf:actionData('parent')['my_output'] eq 'Hello Oozie'}</workflow:case>
+            <workflow:default to="sad-path"/>
+        </workflow:switch>
+    </workflow:decision>
+.
+    <workflow:action name="happy-path">
+        <shell:shell>
+            <shell:resource-manager>${resourceManager}</shell:resource-manager>
+            <shell:name-node>${nameNode}</shell:name-node>
+            <shell:configuration>
+                <shell:property>
+                    <shell:name>mapred.job.queue.name</shell:name>
+                    <shell:value>${queueName}</shell:value>
+                </shell:property>
+            </shell:configuration>
+            <shell:exec>echo</shell:exec>
+            <shell:argument>Happy path</shell:argument>
+        </shell:shell>
+        <workflow:ok to="end"/>
+        <workflow:error to="email-on-error"/>
+    </workflow:action>
+.
+    <workflow:action name="sad-path">
+        <shell:shell>
+            <shell:resource-manager>${resourceManager}</shell:resource-manager>
+            <shell:name-node>${nameNode}</shell:name-node>
+            <shell:configuration>
+                <shell:property>
+                    <shell:name>mapred.job.queue.name</shell:name>
+                    <shell:value>${queueName}</shell:value>
+                </shell:property>
+            </shell:configuration>
+            <shell:exec>echo</shell:exec>
+            <shell:argument>my_output=Hello Oozie</shell:argument>
+            <shell:argument>Sad path</shell:argument>
+            <shell:capture-output/>
+        </shell:shell>
+        <workflow:ok to="end"/>
+        <workflow:error to="email-on-error"/>
+    </workflow:action>
+.
+    <workflow:end name="end"/>
+.
+</workflow:workflow-app>
+</verbatim>
+
+---+++ Runtime Limitations
+
+Even if Fluent Job API tries to abstract away the task of assembly job descriptor XML files, there are some runtime
+limitations apart from the [[DG_FluentJobAPI#Non-goals][non-goals section]]. All such limitations are based on the current
+implementations and subject to further improvements and fixes.
+
+There is only one =kill= possibility in every =workflow=. That is, there can be defined only one =action= to be executed just before
+any other =action= turns to be =kill=ed. Furthermore, =kill= goes to =end= directly. That means, there cannot be defined an
+intricate network of =kill= nodes, cascading sometimes to other =action= nodes, avoiding going to =end= in the first place.
+
+There are places where =decision= node generation fails, throwing an =Exception=. The problem is that during the transformation,
+Fluent Job API reaches a state where there is a =fork= that transitions to two =decision= nodes, which in turn split into two paths
+each. One of the paths from the first =decision= joins a path from the other =decision=, but the remaining conditional paths never
+meet. Therefore, not all paths originating from the =fork= converge to the same =join=.
+
+---++ Appendixes
+
+---+++ AE.A Appendix A, API JAR format
+
+It's kept simple - all the necessary Java class files that are needed are packed into a JAR file, that has a =META-INF/MANIFEST.MF=
+with a single entry having the =Main-Class= attribute set to the fully qualified name of the entry class, the one that
+=implements WorkflowFactory=:
+<verbatim>
+Main-Class: org.apache.oozie.jobs.api.factory.MyFirstWorkflowFactory
+</verbatim>
+
+*An example of the command line assembly of such an API JAR:*
+<verbatim>
+jar cfe simple-workflow.jar org.apache.oozie.fluentjob.api.factory.MyFirstWorkflowFactory \
+-C /Users/forsage/Workspace/oozie/fluent-job/fluent-job-api/target/classes \
+org/apache/oozie/jobs/api/factory/MyFirstWorkflowFactory.class
+</verbatim>
+
+---+++ AE.B Appendix B, Some Useful Builder classes
+
+For a complete list of =Builder= classes, please have a look at =oozie-fluent-job-api= artifact's following packages:
+   * =org.apache.oozie.fluentjob.api.action= - =ActionBuilder= classes
+   * =org.apache.oozie.fluentjob.api.factory= - the single entry point, =WorkflowFactory= is here
+   * =org.apache.oozie.fluentjob.api.workflow= - workflow related =Builder= classes
+
+On examples how to use these please see =oozie-examples= artifact's =org.apache.oozie.example.fluentjob= package.
+
+---+++ AE.C Appendix C, How To Extend
+
+Sometimes there are new XSD versions of an existing custom or core workflow action, sometimes it's a new custom workflow action that
+gets introduced. In any case, Fluent Job API needs to keep up with the changes.
+
+Here are the steps needed:
+   * in =fluent-job-api/pom.xml= extend or modify =jaxb2-maven-plugin= section =sources= by a new =source=
+   * in =fluent-job-api/src/main/xjb/bindings.xml= extend by a new or modify an existing =jaxb:bindings=
+   * in =fluent-job-api=, =org.apache.oozie.fluentjob.api.mapping= package, introduce a new or modify an existing =DozerConverter=
+   * in =dozer_config.xml=, introduce a new or modify an existing =converter= inside =custom-converters=
+   * in =fluent-job-api=, =org.apache.oozie.fluentjob.api.action=, introduce a new =Action= and a new =Builder=
+   * write new / modify existing relevant unit and integration tests
+
+---+++ AE.D Appendix D, API compatibility guarantees
+
+Fluent Job API is available beginning version 5.1.0. It's marked =@InterfaceAudience.Private= (intended for use in Oozie itself) and
+=@InterfaceStability.Unstable= (no stability guarantees are provided across any level of release granularity) to indicate that for
+the next few minor releases it's bound to change a lot.
+
+Beginning from around 5.4.0 planning the next phase, =@InterfaceStability.Evolving= (compatibility breaking only between minors),
+and a few minor releases later, =@InterfaceAudience.Public= (safe to use outside of Oozie).
+
+[[index][::Go back to Oozie Documentation Index::]]
+
+</noautolink>

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/examples/pom.xml
----------------------------------------------------------------------
diff --git a/examples/pom.xml b/examples/pom.xml
index 420b4f8..680e3fb 100644
--- a/examples/pom.xml
+++ b/examples/pom.xml
@@ -87,6 +87,12 @@
         </dependency>
 
         <dependency>
+            <groupId>org.apache.oozie</groupId>
+            <artifactId>oozie-fluent-job-api</artifactId>
+            <scope>compile</scope>
+        </dependency>
+
+        <dependency>
             <groupId>org.slf4j</groupId>
             <artifactId>slf4j-log4j12</artifactId>
             <scope>test</scope>

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/examples/src/main/java/org/apache/oozie/example/fluentjob/CredentialsRetrying.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/oozie/example/fluentjob/CredentialsRetrying.java b/examples/src/main/java/org/apache/oozie/example/fluentjob/CredentialsRetrying.java
new file mode 100644
index 0000000..b450e75
--- /dev/null
+++ b/examples/src/main/java/org/apache/oozie/example/fluentjob/CredentialsRetrying.java
@@ -0,0 +1,93 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.example.fluentjob;
+
+import com.google.common.collect.Lists;
+import org.apache.oozie.fluentjob.api.action.*;
+import org.apache.oozie.fluentjob.api.factory.WorkflowFactory;
+import org.apache.oozie.fluentjob.api.workflow.*;
+
+/**
+ * This {@link WorkflowFactory} generates a workflow definition that has {@code credentials} section, and an action that has
+ * {@code retry-interval}, {@code retry-max}, and {@code retry-policy} attributes set.
+ * <p>
+ * Note that {@link WorkflowBuilder#withCredentials(Credentials)} doesn't necessarily have to be called, since if
+ * {@link WorkflowBuilder#credentialsBuilder} is emtpy by the time {@link WorkflowBuilder#build()} is called,
+ * {@link Workflow#credentials} is built based on all the {@link Node#getCredentials()} that have been added to that
+ * {@code Workflow} in beforehand.
+ * <p>
+ * Note also that when {@code WorkflowBuilder#withCredentials(Credentials)} is explicitly called, the {@code <workflowapp />}'s
+ * {@code <credential />} section is generated only by using the {@code Credentials} defined on the {@code Workflow} level.
+ * <p>
+ * This way, users can, if they want to, omit calling {@code WorkflowBuilder#withCredentials(Credentials)} by default, but can
+ * also override the autogenerated {@code <credentials />} section of {@code <workflowapp />} by explicitly calling that method
+ * after another call to {@link CredentialsBuilder#build()}.
+ * {@link CredentialsBuilder#build()}.
+ */
+public class CredentialsRetrying implements WorkflowFactory {
+    @Override
+    public Workflow create() {
+        final Credential hbaseCredential = CredentialBuilder.create()
+                .withName("hbase")
+                .withType("hbase")
+                .build();
+
+        final Credential hcatalogCredential = CredentialBuilder.create()
+                .withName("hcatalog")
+                .withType("hcatalog")
+                .withConfigurationEntry("hcat.metastore.uri", "thrift://<host>:<port>")
+                .withConfigurationEntry("hcat.metastore.principal", "hive/<host>@<realm>")
+                .build();
+
+        final Credential hive2Credential = CredentialBuilder.create()
+                .withName("hive2")
+                .withType("hive2")
+                .withConfigurationEntry("jdbcUrl", "jdbc://<host>/<database>")
+                .build();
+
+        final ShellAction shellActionWithHBase = ShellActionBuilder
+                .create()
+                .withName("shell-with-hbase-credential")
+                .withCredential(hbaseCredential)
+                .withResourceManager("${resourceManager}")
+                .withNameNode("${nameNode}")
+                .withExecutable("call-hbase.sh")
+                .build();
+
+        Hive2ActionBuilder
+                .createFromExistingAction(shellActionWithHBase)
+                .withParent(shellActionWithHBase)
+                .withName("hive2-action-with-hcatalog-and-hive2-credentials")
+                .clearCredentials()
+                .withCredential(hcatalogCredential)
+                .withCredential(hive2Credential)
+                .withRetryInterval(1)
+                .withRetryMax(3)
+                .withRetryPolicy("exponential")
+                .withScript("call-hive2.sql")
+                .build();
+
+        final Workflow workflow = new WorkflowBuilder()
+                .withName("workflow-with-credentials")
+                .withDagContainingNode(shellActionWithHBase)
+                .build();
+
+        return workflow;
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/examples/src/main/java/org/apache/oozie/example/fluentjob/Global.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/oozie/example/fluentjob/Global.java b/examples/src/main/java/org/apache/oozie/example/fluentjob/Global.java
new file mode 100644
index 0000000..c50fda0
--- /dev/null
+++ b/examples/src/main/java/org/apache/oozie/example/fluentjob/Global.java
@@ -0,0 +1,49 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.example.fluentjob;
+
+import org.apache.oozie.fluentjob.api.action.LauncherBuilder;
+import org.apache.oozie.fluentjob.api.factory.WorkflowFactory;
+import org.apache.oozie.fluentjob.api.workflow.GlobalBuilder;
+import org.apache.oozie.fluentjob.api.workflow.Workflow;
+import org.apache.oozie.fluentjob.api.workflow.WorkflowBuilder;
+
+/**
+ * This {@link WorkflowFactory} generates a workflow definition that has global section.
+ */
+public class Global implements WorkflowFactory {
+    @Override
+    public Workflow create() {
+        final Workflow workflow = new WorkflowBuilder()
+                .withName("workflow-with-global")
+                .withGlobal(GlobalBuilder.create()
+                        .withResourceManager("${resourceManager}")
+                        .withNameNode("${nameNode}")
+                        .withJobXml("job.xml")
+                        .withConfigProperty("key1", "value1")
+                        .withLauncher(new LauncherBuilder()
+                                .withMemoryMb(1024L)
+                                .withVCores(1L)
+                                .build())
+                        .build())
+                .build();
+
+        return workflow;
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/examples/src/main/java/org/apache/oozie/example/fluentjob/JavaMain.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/oozie/example/fluentjob/JavaMain.java b/examples/src/main/java/org/apache/oozie/example/fluentjob/JavaMain.java
new file mode 100644
index 0000000..a8948fd
--- /dev/null
+++ b/examples/src/main/java/org/apache/oozie/example/fluentjob/JavaMain.java
@@ -0,0 +1,56 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.example.fluentjob;
+
+import org.apache.oozie.fluentjob.api.action.EmailActionBuilder;
+import org.apache.oozie.fluentjob.api.action.ErrorHandler;
+import org.apache.oozie.fluentjob.api.action.JavaAction;
+import org.apache.oozie.fluentjob.api.action.JavaActionBuilder;
+import org.apache.oozie.fluentjob.api.factory.WorkflowFactory;
+import org.apache.oozie.fluentjob.api.workflow.Workflow;
+import org.apache.oozie.fluentjob.api.workflow.WorkflowBuilder;
+
+/**
+ * This {@link WorkflowFactory} generates the exact same workflow definition as {@code apps/java-main/workflow.xml}.
+ */
+public class JavaMain implements WorkflowFactory {
+    @Override
+    public Workflow create() {
+        final JavaAction parent = JavaActionBuilder.create()
+                .withName("java-main")
+                .withResourceManager("${resourceManager}")
+                .withNameNode("${nameNode}")
+                .withConfigProperty("mapred.job.queue.name", "${queueName}")
+                .withMainClass("org.apache.oozie.example.DemoJavaMain")
+                .withArg("Hello")
+                .withArg("Oozie!")
+                .withErrorHandler(ErrorHandler.buildAsErrorHandler(EmailActionBuilder.create()
+                        .withName("email-on-error")
+                        .withRecipient("somebody@apache.org")
+                        .withSubject("Workflow error")
+                        .withBody("Shell action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]")))
+                .build();
+
+        final Workflow workflow = new WorkflowBuilder()
+                .withName("java-main-example")
+                .withDagContainingNode(parent).build();
+
+        return workflow;
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/examples/src/main/java/org/apache/oozie/example/fluentjob/MultipleShellActions.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/oozie/example/fluentjob/MultipleShellActions.java b/examples/src/main/java/org/apache/oozie/example/fluentjob/MultipleShellActions.java
new file mode 100644
index 0000000..e1185b1
--- /dev/null
+++ b/examples/src/main/java/org/apache/oozie/example/fluentjob/MultipleShellActions.java
@@ -0,0 +1,79 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.example.fluentjob;
+
+import org.apache.oozie.fluentjob.api.action.EmailActionBuilder;
+import org.apache.oozie.fluentjob.api.action.ErrorHandler;
+import org.apache.oozie.fluentjob.api.factory.WorkflowFactory;
+import org.apache.oozie.fluentjob.api.workflow.Workflow;
+import org.apache.oozie.fluentjob.api.workflow.WorkflowBuilder;
+import org.apache.oozie.fluentjob.api.action.ShellAction;
+import org.apache.oozie.fluentjob.api.action.ShellActionBuilder;
+
+/**
+ * An easily understandable {@link WorkflowFactory} that creates a {@link Workflow} instance consisting of
+ * multiple {@link ShellAction}s, the latter depending conditionally on the output of the former.
+ * <p>
+ * It demonstrates how the Jobs API can be used to create dynamic {@code Workflow} artifacts, as well as
+ * serves as an input for {@code TestOozieCLI} methods that check, submit or run Jobs API {@code .jar} files.
+ */
+public class MultipleShellActions implements WorkflowFactory {
+
+    @Override
+    public Workflow create() {
+        final ShellAction parent = ShellActionBuilder.create()
+                .withName("parent")
+                .withResourceManager("${resourceManager}")
+                .withNameNode("${nameNode}")
+                .withConfigProperty("mapred.job.queue.name", "${queueName}")
+                .withArgument("my_output=Hello Oozie")
+                .withExecutable("echo")
+                .withCaptureOutput(true)
+                .withErrorHandler(ErrorHandler.buildAsErrorHandler(EmailActionBuilder.create()
+                        .withName("email-on-error")
+                        .withRecipient("somebody@apache.org")
+                        .withSubject("Workflow error")
+                        .withBody("Shell action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]")))
+                .build();
+
+        ShellAction next = parent;
+
+        for (int ixShellPair = 0; ixShellPair < 5; ixShellPair++) {
+            final ShellAction happyPath = ShellActionBuilder.createFromExistingAction(parent)
+                    .withName("happy-path-" + ixShellPair)
+                    .withParentWithCondition(next, "${wf:actionData('" + next.getName() + "')['my_output'] eq 'Hello Oozie'}")
+                    .build();
+
+            ShellActionBuilder.createFromExistingAction(parent)
+                    .withName("sad-path-" + ixShellPair)
+                    .withParentDefaultConditional(next)
+                    .withArgument("Sad path " + ixShellPair)
+                    .withCaptureOutput(null)
+                    .build();
+
+            next = happyPath;
+        }
+
+        final Workflow workflow = new WorkflowBuilder()
+                .withName("shell-example")
+                .withDagContainingNode(parent).build();
+
+        return workflow;
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/examples/src/main/java/org/apache/oozie/example/fluentjob/Parameters.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/oozie/example/fluentjob/Parameters.java b/examples/src/main/java/org/apache/oozie/example/fluentjob/Parameters.java
new file mode 100644
index 0000000..18311be
--- /dev/null
+++ b/examples/src/main/java/org/apache/oozie/example/fluentjob/Parameters.java
@@ -0,0 +1,39 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.example.fluentjob;
+
+import org.apache.oozie.fluentjob.api.factory.WorkflowFactory;
+import org.apache.oozie.fluentjob.api.workflow.Workflow;
+import org.apache.oozie.fluentjob.api.workflow.WorkflowBuilder;
+
+/**
+ * This {@link WorkflowFactory} generates a workflow definition that has parameters section.
+ */
+public class Parameters implements WorkflowFactory {
+    @Override
+    public Workflow create() {
+        final Workflow workflow = new WorkflowBuilder()
+                .withName("workflow-with-parameters")
+                .withParameter("name1", "value1")
+                .withParameter("name2", "value2", "description2")
+                .build();
+
+        return workflow;
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/examples/src/main/java/org/apache/oozie/example/fluentjob/Shell.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/oozie/example/fluentjob/Shell.java b/examples/src/main/java/org/apache/oozie/example/fluentjob/Shell.java
new file mode 100644
index 0000000..b20a92a
--- /dev/null
+++ b/examples/src/main/java/org/apache/oozie/example/fluentjob/Shell.java
@@ -0,0 +1,67 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.example.fluentjob;
+
+import org.apache.oozie.fluentjob.api.action.*;
+import org.apache.oozie.fluentjob.api.factory.WorkflowFactory;
+import org.apache.oozie.fluentjob.api.workflow.Workflow;
+import org.apache.oozie.fluentjob.api.workflow.WorkflowBuilder;
+
+/**
+ * This {@link WorkflowFactory} generates a similar workflow definition to {@code apps/shell/workflow.xml}.
+ */
+public class Shell implements WorkflowFactory {
+    @Override
+    public Workflow create() {
+        final ShellAction parent = ShellActionBuilder.create()
+                .withName("parent")
+                .withResourceManager("${resourceManager}")
+                .withNameNode("${nameNode}")
+                .withConfigProperty("mapred.job.queue.name", "${queueName}")
+                .withArgument("my_output=Hello Oozie")
+                .withExecutable("echo")
+                .withCaptureOutput(true)
+                .withErrorHandler(ErrorHandler.buildAsErrorHandler(EmailActionBuilder.create()
+                        .withName("email-on-error")
+                        .withRecipient("somebody@apache.org")
+                        .withSubject("Workflow error")
+                        .withBody("Shell action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]")))
+                .build();
+
+        ShellActionBuilder.createFromExistingAction(parent)
+                .withName("happy-path")
+                .withParentWithCondition(parent, "${wf:actionData('parent')['my_output'] eq 'Hello Oozie'}")
+                .withoutArgument("my_output=Hello Oozie")
+                .withArgument("Happy path")
+                .withCaptureOutput(null)
+                .build();
+
+        ShellActionBuilder.createFromExistingAction(parent)
+                .withName("sad-path")
+                .withParentDefaultConditional(parent)
+                .withArgument("Sad path")
+                .build();
+
+        final Workflow workflow = new WorkflowBuilder()
+                .withName("shell-example")
+                .withDagContainingNode(parent).build();
+
+        return workflow;
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/examples/src/main/java/org/apache/oozie/example/fluentjob/Spark.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/oozie/example/fluentjob/Spark.java b/examples/src/main/java/org/apache/oozie/example/fluentjob/Spark.java
new file mode 100644
index 0000000..6065933
--- /dev/null
+++ b/examples/src/main/java/org/apache/oozie/example/fluentjob/Spark.java
@@ -0,0 +1,61 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.example.fluentjob;
+
+import org.apache.oozie.fluentjob.api.action.*;
+import org.apache.oozie.fluentjob.api.factory.WorkflowFactory;
+import org.apache.oozie.fluentjob.api.workflow.Workflow;
+import org.apache.oozie.fluentjob.api.workflow.WorkflowBuilder;
+
+/**
+ * This {@link WorkflowFactory} generates the exact same workflow definition as {@code apps/spark/workflow.xml}.
+ */
+public class Spark implements WorkflowFactory {
+    @Override
+    public Workflow create() {
+        final Prepare prepare = new PrepareBuilder()
+                .withDelete("${nameNode}/user/${wf:user()}/${examplesRoot}/output-data/spark")
+                .build();
+
+        final SparkAction parent = SparkActionBuilder.create()
+                .withName("spark-file-copy")
+                .withResourceManager("${resourceManager}")
+                .withNameNode("${nameNode}")
+                .withPrepare(prepare)
+                .withConfigProperty("mapred.job.queue.name", "${queueName}")
+                .withArg("${nameNode}/user/${wf:user()}/${examplesRoot}/input-data/text/data.txt")
+                .withArg("${nameNode}/user/${wf:user()}/${examplesRoot}/output-data/spark")
+                .withMaster("${master}")
+                .withActionName("Spark File Copy Example")
+                .withActionClass("org.apache.oozie.example.SparkFileCopy")
+                .withJar("${nameNode}/user/${wf:user()}/${examplesRoot}/apps/spark/lib/oozie-examples.jar")
+                .withErrorHandler(ErrorHandler.buildAsErrorHandler(EmailActionBuilder.create()
+                        .withName("email-on-error")
+                        .withRecipient("somebody@apache.org")
+                        .withSubject("Workflow error")
+                        .withBody("Shell action failed, error message[${wf:errorMessage(wf:lastErrorNode())}]")))
+                .build();
+
+        final Workflow workflow = new WorkflowBuilder()
+                .withName("spark-file-copy")
+                .withDagContainingNode(parent).build();
+
+        return workflow;
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/findbugs-filter.xml
----------------------------------------------------------------------
diff --git a/findbugs-filter.xml b/findbugs-filter.xml
index 03ee4d1..133178f 100644
--- a/findbugs-filter.xml
+++ b/findbugs-filter.xml
@@ -29,4 +29,16 @@
        <Method name="toString" />
        <Bug pattern="WMI_WRONG_MAP_ITERATOR" />
      </Match>
+
+    <!-- Don't have much control on generated JAXB2 classes -->
+    <Match>
+        <Package name="~org\.apache\.oozie\.fluentjob\.api\.generated.*" />
+        <Bug pattern="EQ_UNUSUAL" />
+    </Match>
+
+    <!-- Directory name is private static final -->
+    <Match>
+        <Class name="org.apache.oozie.fluentjob.api.GraphVisualization" />
+        <Bug pattern="WEAK_FILENAMEUTILS" />
+    </Match>
 </FindBugsFilter>

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/pom.xml
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/pom.xml b/fluent-job/fluent-job-api/pom.xml
new file mode 100644
index 0000000..4c9b853
--- /dev/null
+++ b/fluent-job/fluent-job-api/pom.xml
@@ -0,0 +1,212 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you under the Apache License, Version 2.0 (the
+  "License"); you may not use this file except in compliance
+  with the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.oozie</groupId>
+        <artifactId>oozie-fluent-job</artifactId>
+        <version>5.1.0-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>oozie-fluent-job-api</artifactId>
+    <version>5.1.0-SNAPSHOT</version>
+    <description>Apache Oozie Fluent Job API</description>
+    <name>Apache Oozie Fluent Job API</name>
+    <packaging>jar</packaging>
+
+    <dependencies>
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.mockito</groupId>
+            <artifactId>mockito-all</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.commons</groupId>
+            <artifactId>commons-lang3</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>guru.nidi</groupId>
+            <artifactId>graphviz-java</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>net.sf.dozer</groupId>
+            <artifactId>dozer</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>com.google.guava</groupId>
+            <artifactId>guava</artifactId>
+            <scope>compile</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.jvnet.jaxb2_commons</groupId>
+            <artifactId>jaxb2-basics</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-annotations</artifactId>
+        </dependency>
+    </dependencies>
+
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>org.apache.rat</groupId>
+                <artifactId>apache-rat-plugin</artifactId>
+                <configuration>
+                    <excludeSubProjects>false</excludeSubProjects>
+                    <excludes>
+                        <!-- excluding all as the root POM does the full check-->
+                        <exclude>**</exclude>
+                    </excludes>
+                </configuration>
+            </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-jar-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <goals>
+                            <goal>test-jar</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-assembly-plugin</artifactId>
+                <configuration>
+                    <descriptors>
+                        <descriptor>../../src/main/assemblies/tools.xml</descriptor>
+                    </descriptors>
+                </configuration>
+            </plugin>
+            <plugin>
+                <groupId>org.codehaus.mojo</groupId>
+                <artifactId>jaxb2-maven-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <id>xjc</id>
+                        <goals>
+                            <goal>xjc</goal>
+                        </goals>
+                    </execution>
+                </executions>
+                <configuration>
+                    <arguments>
+                        <argument>-XhashCode</argument>
+                        <argument>-Xequals</argument>
+                        <argument>-Xnamespace-prefix</argument>
+                        <argument>-Xsetters</argument>
+                    </arguments>
+
+                    <sources>
+                        <source>../../client/src/main/resources/distcp-action-1.0.xsd</source>
+                        <source>../../client/src/main/resources/email-action-0.2.xsd</source>
+                        <source>../../client/src/main/resources/hive2-action-1.0.xsd</source>
+                        <source>../../client/src/main/resources/hive-action-1.0.xsd</source>
+                        <source>../../client/src/main/resources/oozie-sla-0.2.xsd</source>
+                        <source>../../client/src/main/resources/oozie-workflow-1.0.xsd</source>
+                        <source>../../client/src/main/resources/shell-action-1.0.xsd</source>
+                        <source>../../client/src/main/resources/spark-action-1.0.xsd</source>
+                        <source>../../client/src/main/resources/sqoop-action-1.0.xsd</source>
+                        <source>../../client/src/main/resources/ssh-action-0.2.xsd</source>
+                    </sources>
+                </configuration>
+
+                <dependencies>
+                    <dependency>
+                        <groupId>org.jvnet.jaxb2_commons</groupId>
+                        <artifactId>jaxb2-basics</artifactId>
+                        <version>1.11.1</version>
+                    </dependency>
+                    <dependency>
+                        <groupId>org.jvnet.jaxb2_commons</groupId>
+                        <artifactId>jaxb2-namespace-prefix</artifactId>
+                        <version>1.3</version>
+                    </dependency>
+                </dependencies>
+            </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-javadoc-plugin</artifactId>
+                <configuration>
+                    <excludePackageNames>org.apache.oozie.fluentjob.api.generated.*</excludePackageNames>
+                </configuration>
+            </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-checkstyle-plugin</artifactId>
+                <configuration>
+                    <excludes>**/generated/**/*</excludes>
+                </configuration>
+            </plugin>
+        </plugins>
+    </build>
+
+    <profiles>
+        <profile>
+            <id>generateDocs</id>
+            <activation>
+                <activeByDefault>false</activeByDefault>
+                <property>
+                    <name>generateDocs</name>
+                </property>
+            </activation>
+            <build>
+                <plugins>
+                    <plugin>
+                        <groupId>org.apache.maven.plugins</groupId>
+                        <artifactId>maven-javadoc-plugin</artifactId>
+                        <configuration>
+                            <linksource>true</linksource>
+                            <quiet>true</quiet>
+                            <verbose>false</verbose>
+                            <source>${maven.compile.source}</source>
+                            <charset>${maven.compile.encoding}</charset>
+                            <groups>
+                                <group>
+                                    <title>Fluent Job API</title>
+                                    <packages>
+                                        org.apache.oozie.fluentjob.api
+                                    </packages>
+                                </group>
+                            </groups>
+                            <excludePackageNames>org.apache.oozie.fluentjob.api.generated.*</excludePackageNames>
+                        </configuration>
+                        <executions>
+                            <execution>
+                                <goals>
+                                    <goal>javadoc</goal>
+                                </goals>
+                                <phase>package</phase>
+                            </execution>
+                        </executions>
+                    </plugin>
+                </plugins>
+            </build>
+        </profile>
+    </profiles>
+</project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/Condition.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/Condition.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/Condition.java
new file mode 100644
index 0000000..dd18f7a
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/Condition.java
@@ -0,0 +1,102 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * A class representing a condition in the "switch statement" of an Oozie decision node.
+ */
+public class Condition {
+    private final String condition;
+    private final boolean isDefault;
+
+    private Condition(final String condition, final boolean isDefault) {
+        final boolean bothFieldsSet = condition == null && !isDefault;
+        final boolean bothFieldsUnset = condition != null && isDefault;
+        Preconditions.checkArgument(!bothFieldsSet && !bothFieldsUnset,
+                "Exactly one of 'condition' and 'isDefault' must be non-null or true (respectively).");
+
+        this.condition = condition;
+        this.isDefault = isDefault;
+    }
+
+    /**
+     * Creates an actual condition (as opposed to a default condition).
+     * @param condition The string defining the condition.
+     * @return A new actual condition.
+     */
+    public static Condition actualCondition(final String condition) {
+        Preconditions.checkArgument(condition != null, "The argument 'condition' must not be null.");
+
+        return new Condition(condition, false);
+    }
+
+    /**
+     * Creates a new default condition. Every decision node must have a default path which is chosen if no other
+     * condition is true.
+     * @return A new default condition.
+     */
+    public static Condition defaultCondition() {
+        return new Condition(null, true);
+    }
+
+    /**
+     * Returns the string defining the condition or {@code null} if this is a default condition.
+     * @return The string defining the condition or {@code null} if this is a default condition.
+     */
+    public String getCondition() {
+        return condition;
+    }
+
+    /**
+     * Returns whether this condition is a default condition.
+     * @return {@code true} if this condition is a default condition, {@code false} otherwise.
+     */
+    public boolean isDefault() {
+        return isDefault;
+    }
+
+    @Override
+    public boolean equals(final Object o) {
+        if (this == o) {
+            return true;
+        }
+
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+
+        final Condition other = (Condition) o;
+
+        if (isDefault != other.isDefault) {
+            return false;
+        }
+
+        return condition != null ? condition.equals(other.condition) : other.condition == null;
+    }
+
+    @Override
+    public int hashCode() {
+        int result = condition != null ? condition.hashCode() : 0;
+        result = 31 * result + (isDefault ? 1 : 0);
+
+        return result;
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/GraphVisualization.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/GraphVisualization.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/GraphVisualization.java
new file mode 100644
index 0000000..437d6b3
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/GraphVisualization.java
@@ -0,0 +1,117 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api;
+
+import guru.nidi.graphviz.engine.Format;
+import guru.nidi.graphviz.engine.Graphviz;
+import guru.nidi.graphviz.model.MutableGraph;
+import guru.nidi.graphviz.parse.Parser;
+import org.apache.commons.io.FilenameUtils;
+import org.apache.oozie.fluentjob.api.action.Node;
+import org.apache.oozie.fluentjob.api.dag.Decision;
+import org.apache.oozie.fluentjob.api.dag.Graph;
+import org.apache.oozie.fluentjob.api.dag.NodeBase;
+import org.apache.oozie.fluentjob.api.workflow.Workflow;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.List;
+
+/**
+ * Given a {@link Graph} or a {@link Workflow} instance, creates a visually appealing output
+ * using {@code nidi-graphviz} library in either {@code .dot} or {@code .png} format into {@link #PARENT_FOLDER_NAME}.
+ * <p>
+ * Applying memory constraints, the width of the resulting {@code .png} is limited to {@link #PNG_WIDTH}.
+ */
+public class GraphVisualization {
+    private static final String PARENT_FOLDER_NAME = "target/graphviz";
+    private static final int PNG_WIDTH = 1024;
+
+    public static String graphToDot(final Graph graph) {
+        return nodeBasesToDot(graph.getNodes());
+    }
+
+    private static String nodeBasesToDot(final Collection<NodeBase> nodes) {
+        final StringBuilder builder = new StringBuilder();
+        builder.append("digraph {\n");
+        for (final NodeBase node : nodes) {
+            final List<NodeBase> children = node.getChildren();
+
+            String style = "";
+            if (node instanceof Decision) {
+                style = "[style=dashed];";
+            }
+
+            for (final NodeBase child : children) {
+                final String s = String.format("\t\"%s\" -> \"%s\"%s%n", node.getName(), child.getName(), style);
+                builder.append(s);
+            }
+        }
+
+        builder.append("}");
+
+        return builder.toString();
+    }
+
+    private static String workflowToDot(final Workflow workflow) {
+        return nodesToDot(workflow.getNodes());
+    }
+
+    private static String nodesToDot(final Collection<Node> nodes) {
+        final StringBuilder builder = new StringBuilder();
+        builder.append("digraph {\n");
+        for (final Node node : nodes) {
+            final List<Node> children = node.getAllChildren();
+
+            String style = "";
+            if (!node.getChildrenWithConditions().isEmpty()) {
+                style = "[style=dashed];";
+            }
+
+            for (final Node child : children) {
+                builder.append(String.format("\t\"%s\" -> \"%s\"%s%n", node.getName(), child.getName(), style));
+            }
+        }
+
+        builder.append("}");
+
+        return builder.toString();
+    }
+
+    public static void graphToPng(final Graph graph, final String fileName) throws IOException {
+        final MutableGraph mg = Parser.read(graphToDot(graph));
+        mg.setLabel(fileName);
+
+        Graphviz.fromGraph(mg)
+                .width(PNG_WIDTH)
+                .render(Format.PNG)
+                .toFile(new File(PARENT_FOLDER_NAME, FilenameUtils.getName(fileName)));
+    }
+
+    public static void workflowToPng(final Workflow workflow, final String fileName) throws IOException {
+        final MutableGraph mg = Parser.read(workflowToDot(workflow));
+        mg.setLabel(fileName);
+
+        Graphviz.fromGraph(mg)
+                .width(PNG_WIDTH)
+                .render(Format.PNG)
+                .toFile(new File(PARENT_FOLDER_NAME, FilenameUtils.getName(fileName)));
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/ModifyOnce.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/ModifyOnce.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/ModifyOnce.java
new file mode 100644
index 0000000..9561372
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/ModifyOnce.java
@@ -0,0 +1,67 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * A generic wrapper class for a value that can be modified once after construction, but only once.
+ * @param <T> the generic type that can be modified once, but only once
+ */
+public class ModifyOnce<T> {
+    private T data;
+    private boolean modified;
+
+    /**
+     * Creates a new {@link ModifyOnce} object initialized to {@code null}.
+     */
+    public ModifyOnce() {
+        this(null);
+    }
+
+    /**
+     * Creates a new {@link ModifyOnce} object initialized to {@code defaultData}.
+     * @param defaultData The initial value of this {@link ModifyOnce} object.
+     */
+    public ModifyOnce(final T defaultData) {
+        this.data = defaultData;
+        this.modified = false;
+    }
+
+    /**
+     * Returns the wrapped value.
+     * @return The wrapped value.
+     */
+    public T get() {
+        return data;
+    }
+
+    /**
+     * Sets the wrapped value. If it is not the first modification attempt, {@link IllegalStateException} is thrown.
+     * @param data The new data to store.
+     *
+     * @throws IllegalStateException if this is not the first modification attempt.
+     */
+    public void set(final T data) {
+        Preconditions.checkState(!modified, "Has already been modified once.");
+
+        this.data = data;
+        this.modified = true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/ActionAttributes.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/ActionAttributes.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/ActionAttributes.java
new file mode 100644
index 0000000..692eafb
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/ActionAttributes.java
@@ -0,0 +1,258 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.action;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+import java.util.List;
+import java.util.Map;
+
+/**
+ * An immutable class holding data that is used by several actions. It should be constructed by using an
+ * {@link ActionAttributesBuilder}.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class ActionAttributes {
+    private final String resourceManager;
+    private final String nameNode;
+    private final Prepare prepare;
+    private final Streaming streaming;
+    private final Pipes pipes;
+    private final ImmutableList<String> jobXmls;
+    private final ImmutableMap<String, String> configuration;
+    private final String configClass;
+    private final ImmutableList<String> files;
+    private final ImmutableList<String> archives;
+    private final ImmutableList<Delete> deletes;
+    private final ImmutableList<Mkdir> mkdirs;
+    private final ImmutableList<Move> moves;
+    private final ImmutableList<Chmod> chmods;
+    private final ImmutableList<Touchz> touchzs;
+    private final ImmutableList<Chgrp> chgrps;
+    private final String javaOpts;
+    private final ImmutableList<String> args;
+    private final Launcher launcher;
+    private final Boolean captureOutput;
+
+    ActionAttributes(final String resourceManager,
+                     final String nameNode,
+                     final Prepare prepare,
+                     final Streaming streaming,
+                     final Pipes pipes,
+                     final ImmutableList<String> jobXmls,
+                     final ImmutableMap<String, String> configuration,
+                     final String configClass,
+                     final ImmutableList<String> files,
+                     final ImmutableList<String> archives,
+                     final ImmutableList<Delete> deletes,
+                     final ImmutableList<Mkdir> mkdirs,
+                     final ImmutableList<Move> moves,
+                     final ImmutableList<Chmod> chmods,
+                     final ImmutableList<Touchz> touchzs,
+                     final ImmutableList<Chgrp> chgrps,
+                     final String javaOpts,
+                     final ImmutableList<String> args,
+                     final Launcher launcher,
+                     final Boolean captureOutput) {
+        this.resourceManager = resourceManager;
+        this.nameNode = nameNode;
+        this.prepare = prepare;
+        this.streaming = streaming;
+        this.pipes = pipes;
+        this.jobXmls = jobXmls;
+        this.configuration = configuration;
+        this.configClass = configClass;
+        this.files = files;
+        this.archives = archives;
+        this.deletes = deletes;
+        this.mkdirs = mkdirs;
+        this.moves = moves;
+        this.chmods = chmods;
+        this.touchzs = touchzs;
+        this.chgrps = chgrps;
+        this.javaOpts = javaOpts;
+        this.args = args;
+        this.launcher = launcher;
+        this.captureOutput = captureOutput;
+    }
+
+    /**
+     * Returns the resource manager address
+     * @return the resource manager address
+     */
+    public String getResourceManager() {
+        return resourceManager;
+    }
+
+    /**
+     * Returns the name node stored in this {@link ActionAttributes} object.
+     * @return The name node stored in this {@link ActionAttributes} object.
+     */
+    public String getNameNode() {
+        return nameNode;
+    }
+
+    /**
+     * Returns the {@link Prepare} object stored in this {@link ActionAttributes} object.
+     * @return The {@link Prepare} object stored in this {@link ActionAttributes} object.
+     */
+    public Prepare getPrepare() {
+        return prepare;
+    }
+
+    /**
+     * Returns the {@link Streaming} object stored in this {@link ActionAttributes} object.
+     * @return The {@link Streaming} object stored in this {@link ActionAttributes} object.
+     */
+    public Streaming getStreaming() {
+        return streaming;
+    }
+
+    /**
+     * Returns the {@link Pipes} object stored in this {@link ActionAttributes} object.
+     * @return The {@link Pipes} object stored in this {@link ActionAttributes} object.
+     */
+    public Pipes getPipes() {
+        return pipes;
+    }
+
+    /**
+     * Returns the list of job XMLs stored in this {@link ActionAttributes} object.
+     * @return The list of job XMLs stored in this {@link ActionAttributes} object.
+     */
+    public List<String> getJobXmls() {
+        return jobXmls;
+    }
+
+    /**
+     * Returns a map of the configuration key-value pairs stored in this {@link ActionAttributes} object.
+     * @return A map of the configuration key-value pairs stored in this {@link ActionAttributes} object.
+     */
+    public Map<String, String> getConfiguration() {
+        return configuration;
+    }
+
+    /**
+     * Returns the configuration class property of this {@link ActionAttributes} object.
+     * @return The configuration class property of this {@link ActionAttributes} object.
+     */
+    public String getConfigClass() {
+        return configClass;
+    }
+
+    /**
+     * Returns a list of the names of the files associated with this {@link ActionAttributes} object.
+     * @return A list of the names of the files associated with this {@link ActionAttributes} object.
+     */
+    public List<String> getFiles() {
+        return files;
+    }
+
+    /**
+     * Returns a list of the names of the archives associated with this {@link ActionAttributes} object.
+     * @return A list of the names of the archives associated with this {@link ActionAttributes} object.
+     */
+    public List<String> getArchives() {
+        return archives;
+    }
+
+    /**
+     * Returns a list of the {@link Delete} objects stored in this {@link ActionAttributes} object.
+     * @return A list of the {@link Delete} objects stored in this {@link ActionAttributes} object.
+     */
+    public List<Delete> getDeletes() {
+        return deletes;
+    }
+
+    /**
+     * Returns a list of the {@link Mkdir} objects stored in this {@link ActionAttributes} object.
+     * @return A list of the {@link Mkdir} objects stored in this {@link ActionAttributes} object.
+     */
+    public List<Mkdir> getMkdirs() {
+        return mkdirs;
+    }
+
+    /**
+     * Returns a list of the {@link Move} objects stored in this {@link ActionAttributes} object.
+     * @return A list of the {@link Move} objects stored in this {@link ActionAttributes} object.
+     */
+    public List<Move> getMoves() {
+        return moves;
+    }
+
+    /**
+     * Returns a list of the {@link Chmod} objects stored in this {@link ActionAttributes} object.
+     * @return A list of the {@link Chmod} objects stored in this {@link ActionAttributes} object.
+     */
+    public List<Chmod> getChmods() {
+        return chmods;
+    }
+
+    /**
+     * Returns a list of the {@link Touchz} objects stored in this {@link ActionAttributes} object.
+     * @return A list of the {@link Touchz} objects stored in this {@link ActionAttributes} object.
+     */
+    public List<Touchz> getTouchzs() {
+        return touchzs;
+    }
+
+    /**
+     * Returns a list of the {@link Chgrp} objects stored in this {@link ActionAttributes} object.
+     * @return A list of the {@link Delete} objects stored in this {@link ActionAttributes} object.
+     */
+    public List<Chgrp> getChgrps() {
+        return chgrps;
+    }
+
+    /**
+     * Get the java options.
+     * @return the java options
+     */
+    public String getJavaOpts() {
+        return javaOpts;
+    }
+
+    /**
+     * Get all the arguments.
+     * @return the argument list
+     */
+    public List<String> getArgs() {
+        return args;
+    }
+
+    /**
+     * Get the {@link Launcher}
+     * @return the {@link Launcher}
+     */
+    public Launcher getLauncher() {
+        return launcher;
+    }
+
+    /**
+     * Tells the caller whether to capture output or not.
+     * @return {@code true} when capturing output
+     */
+    public boolean isCaptureOutput() {
+        return captureOutput == null ? false : captureOutput;
+    }
+}


[09/16] oozie git commit: OOZIE-2339 [fluent-job] Minimum Viable Fluent Job API (daniel.becker, andras.piros via rkanter, gezapeti, pbacsko)

Posted by an...@apache.org.
http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/Hive2LauncherConverter.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/Hive2LauncherConverter.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/Hive2LauncherConverter.java
new file mode 100644
index 0000000..34b11f8
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/Hive2LauncherConverter.java
@@ -0,0 +1,74 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.mapping;
+
+import org.apache.oozie.fluentjob.api.generated.action.hive2.LAUNCHER;
+import org.apache.oozie.fluentjob.api.generated.action.hive2.ObjectFactory;
+import org.apache.oozie.fluentjob.api.action.Launcher;
+import org.dozer.DozerConverter;
+
+/**
+ * A {@link DozerConverter} converting from {@link Launcher} to JAXB {@link LAUNCHER}.
+ */
+public class Hive2LauncherConverter extends DozerConverter<Launcher, LAUNCHER> {
+    private static final ObjectFactory OBJECT_FACTORY = new ObjectFactory();
+
+    public Hive2LauncherConverter() {
+        super(Launcher.class, LAUNCHER.class);
+    }
+
+    @Override
+    public LAUNCHER convertTo(final Launcher source, LAUNCHER destination) {
+        if (source == null) {
+            return null;
+        }
+
+        destination = ensureDestination(destination);
+
+        mapAttributes(source, destination);
+
+        return destination;
+    }
+
+    private LAUNCHER ensureDestination(final LAUNCHER destination) {
+        if (destination == null) {
+            return OBJECT_FACTORY.createLAUNCHER();
+        }
+
+        return destination;
+    }
+
+    private void mapAttributes(final Launcher source, final LAUNCHER destination) {
+        if (source == null) {
+            return;
+        }
+
+        destination.getMemoryMbOrVcoresOrJavaOpts().add(OBJECT_FACTORY.createLAUNCHERMemoryMb(source.getMemoryMb()));
+        destination.getMemoryMbOrVcoresOrJavaOpts().add(OBJECT_FACTORY.createLAUNCHERVcores(source.getVCores()));
+        destination.getMemoryMbOrVcoresOrJavaOpts().add(OBJECT_FACTORY.createLAUNCHERQueue(source.getQueue()));
+        destination.getMemoryMbOrVcoresOrJavaOpts().add(OBJECT_FACTORY.createLAUNCHERSharelib(source.getSharelib()));
+        destination.getMemoryMbOrVcoresOrJavaOpts().add(OBJECT_FACTORY.createLAUNCHERViewAcl(source.getViewAcl()));
+        destination.getMemoryMbOrVcoresOrJavaOpts().add(OBJECT_FACTORY.createLAUNCHERModifyAcl(source.getModifyAcl()));
+    }
+
+    @Override
+    public Launcher convertFrom(final LAUNCHER source, final Launcher destination) {
+        throw new UnsupportedOperationException("This mapping is not bidirectional.");
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/Hive2PrepareConverter.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/Hive2PrepareConverter.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/Hive2PrepareConverter.java
new file mode 100644
index 0000000..105f996
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/Hive2PrepareConverter.java
@@ -0,0 +1,97 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.mapping;
+
+import org.apache.oozie.fluentjob.api.generated.action.hive2.DELETE;
+import org.apache.oozie.fluentjob.api.generated.action.hive2.MKDIR;
+import org.apache.oozie.fluentjob.api.generated.action.hive2.ObjectFactory;
+import org.apache.oozie.fluentjob.api.generated.action.hive2.PREPARE;
+import org.apache.oozie.fluentjob.api.action.Delete;
+import org.apache.oozie.fluentjob.api.action.Mkdir;
+import org.apache.oozie.fluentjob.api.action.Prepare;
+import org.dozer.DozerConverter;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * A {@link DozerConverter} converting from {@link Prepare} to JAXB {@link PREPARE}.
+ */
+public class Hive2PrepareConverter extends DozerConverter<Prepare, PREPARE> {
+    private static final ObjectFactory OBJECT_FACTORY = new ObjectFactory();
+
+    public Hive2PrepareConverter() {
+        super(Prepare.class, PREPARE.class);
+    }
+
+    @Override
+    public PREPARE convertTo(final Prepare source, PREPARE destination) {
+        if (source == null) {
+            return null;
+        }
+
+        destination = ensureDestination(destination);
+
+        mapDeletes(source, destination);
+
+        mapMkdirs(source, destination);
+
+        return destination;
+    }
+
+    private PREPARE ensureDestination(final PREPARE destination) {
+        if (destination == null) {
+            return OBJECT_FACTORY.createPREPARE();
+        }
+        return destination;
+    }
+
+    private void mapDeletes(final Prepare source, final PREPARE destination) {
+        if (source.getDeletes() != null) {
+            final List<DELETE> targetDeletes = new ArrayList<>();
+
+            for (final Delete sourceDelete : source.getDeletes()) {
+                final DELETE targetDelete = OBJECT_FACTORY.createDELETE();
+                targetDelete.setPath(sourceDelete.getPath());
+                targetDeletes.add(targetDelete);
+            }
+
+            destination.setDelete(targetDeletes);
+        }
+    }
+
+    private void mapMkdirs(final Prepare source, final PREPARE destination) {
+        if (source.getMkdirs() != null) {
+            final List<MKDIR> targetMkdirs = new ArrayList<>();
+
+            for (final Mkdir sourceMkDir: source.getMkdirs()) {
+                final MKDIR targetMkDir = OBJECT_FACTORY.createMKDIR();
+                targetMkDir.setPath(sourceMkDir.getPath());
+                targetMkdirs.add(targetMkDir);
+            }
+
+            destination.setMkdir(targetMkdirs);
+        }
+    }
+
+    @Override
+    public Prepare convertFrom(final PREPARE source, final Prepare destination) {
+        throw new UnsupportedOperationException("This mapping is not bidirectional.");
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/HiveConfigurationConverter.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/HiveConfigurationConverter.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/HiveConfigurationConverter.java
new file mode 100644
index 0000000..5f9a2b1
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/HiveConfigurationConverter.java
@@ -0,0 +1,81 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.mapping;
+
+import org.apache.oozie.fluentjob.api.generated.action.hive.CONFIGURATION;
+import org.apache.oozie.fluentjob.api.generated.action.hive.ObjectFactory;
+import org.dozer.DozerConverter;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * A {@link DozerConverter} converting from {@link Map} to JAXB {@link CONFIGURATION}.
+ */
+public class HiveConfigurationConverter extends DozerConverter<Map, CONFIGURATION> {
+    private static final ObjectFactory OBJECT_FACTORY = new ObjectFactory();
+
+    public HiveConfigurationConverter() {
+        super(Map.class, CONFIGURATION.class);
+    }
+
+    @Override
+    public CONFIGURATION convertTo(final Map source, CONFIGURATION destination) {
+        if (source == null) {
+            return null;
+        }
+
+        destination = ensureDestination(destination);
+
+        mapEntries(source, destination);
+
+        return destination;
+    }
+
+    private CONFIGURATION ensureDestination(CONFIGURATION destination) {
+        if (destination == null) {
+            destination = OBJECT_FACTORY.createCONFIGURATION();
+        }
+
+        return destination;
+    }
+
+    private void mapEntries(final Map source, final CONFIGURATION destination) {
+        if (source != null) {
+            final List<CONFIGURATION.Property> targetProperties = new ArrayList<>();
+
+            for (final Object objectKey : source.keySet()) {
+                final String name = objectKey.toString();
+                final String value = source.get(name).toString();
+                final CONFIGURATION.Property targetProperty = OBJECT_FACTORY.createCONFIGURATIONProperty();
+                targetProperty.setName(name);
+                targetProperty.setValue(value);
+                targetProperties.add(targetProperty);
+            }
+
+            destination.setProperty(targetProperties);
+        }
+    }
+
+    @Override
+    public Map convertFrom(final CONFIGURATION source, final Map destination) {
+        throw new UnsupportedOperationException("This mapping is not bidirectional.");
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/HiveLauncherConverter.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/HiveLauncherConverter.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/HiveLauncherConverter.java
new file mode 100644
index 0000000..ad231f5
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/HiveLauncherConverter.java
@@ -0,0 +1,74 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.mapping;
+
+import org.apache.oozie.fluentjob.api.generated.action.hive.LAUNCHER;
+import org.apache.oozie.fluentjob.api.generated.action.hive.ObjectFactory;
+import org.apache.oozie.fluentjob.api.action.Launcher;
+import org.dozer.DozerConverter;
+
+/**
+ * A {@link DozerConverter} converting from {@link Launcher} to JAXB {@link LAUNCHER}.
+ */
+public class HiveLauncherConverter extends DozerConverter<Launcher, LAUNCHER> {
+    private static final ObjectFactory OBJECT_FACTORY = new ObjectFactory();
+
+    public HiveLauncherConverter() {
+        super(Launcher.class, LAUNCHER.class);
+    }
+
+    @Override
+    public LAUNCHER convertTo(final Launcher source, LAUNCHER destination) {
+        if (source == null) {
+            return null;
+        }
+
+        destination = ensureDestination(destination);
+
+        mapAttributes(source, destination);
+
+        return destination;
+    }
+
+    private LAUNCHER ensureDestination(final LAUNCHER destination) {
+        if (destination == null) {
+            return OBJECT_FACTORY.createLAUNCHER();
+        }
+
+        return destination;
+    }
+
+    private void mapAttributes(final Launcher source, final LAUNCHER destination) {
+        if (source == null) {
+            return;
+        }
+
+        destination.getMemoryMbOrVcoresOrJavaOpts().add(OBJECT_FACTORY.createLAUNCHERMemoryMb(source.getMemoryMb()));
+        destination.getMemoryMbOrVcoresOrJavaOpts().add(OBJECT_FACTORY.createLAUNCHERVcores(source.getVCores()));
+        destination.getMemoryMbOrVcoresOrJavaOpts().add(OBJECT_FACTORY.createLAUNCHERQueue(source.getQueue()));
+        destination.getMemoryMbOrVcoresOrJavaOpts().add(OBJECT_FACTORY.createLAUNCHERSharelib(source.getSharelib()));
+        destination.getMemoryMbOrVcoresOrJavaOpts().add(OBJECT_FACTORY.createLAUNCHERViewAcl(source.getViewAcl()));
+        destination.getMemoryMbOrVcoresOrJavaOpts().add(OBJECT_FACTORY.createLAUNCHERModifyAcl(source.getModifyAcl()));
+    }
+
+    @Override
+    public Launcher convertFrom(final LAUNCHER source, final Launcher destination) {
+        throw new UnsupportedOperationException("This mapping is not bidirectional.");
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/HivePrepareConverter.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/HivePrepareConverter.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/HivePrepareConverter.java
new file mode 100644
index 0000000..9b668d4
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/HivePrepareConverter.java
@@ -0,0 +1,97 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.mapping;
+
+import org.apache.oozie.fluentjob.api.generated.action.hive.DELETE;
+import org.apache.oozie.fluentjob.api.generated.action.hive.MKDIR;
+import org.apache.oozie.fluentjob.api.generated.action.hive.ObjectFactory;
+import org.apache.oozie.fluentjob.api.generated.action.hive.PREPARE;
+import org.apache.oozie.fluentjob.api.action.Delete;
+import org.apache.oozie.fluentjob.api.action.Mkdir;
+import org.apache.oozie.fluentjob.api.action.Prepare;
+import org.dozer.DozerConverter;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * A {@link DozerConverter} converting from {@link Prepare} to JAXB {@link PREPARE}.
+ */
+public class HivePrepareConverter extends DozerConverter<Prepare, PREPARE> {
+    private static final ObjectFactory OBJECT_FACTORY = new ObjectFactory();
+
+    public HivePrepareConverter() {
+        super(Prepare.class, PREPARE.class);
+    }
+
+    @Override
+    public PREPARE convertTo(final Prepare source, PREPARE destination) {
+        if (source == null) {
+            return null;
+        }
+
+        destination = ensureDestination(destination);
+
+        mapDeletes(source, destination);
+
+        mapMkdirs(source, destination);
+
+        return destination;
+    }
+
+    private PREPARE ensureDestination(final PREPARE destination) {
+        if (destination == null) {
+            return OBJECT_FACTORY.createPREPARE();
+        }
+        return destination;
+    }
+
+    private void mapDeletes(final Prepare source, final PREPARE destination) {
+        if (source.getDeletes() != null) {
+            final List<DELETE> targetDeletes = new ArrayList<>();
+
+            for (final Delete sourceDelete : source.getDeletes()) {
+                final DELETE targetDelete = OBJECT_FACTORY.createDELETE();
+                targetDelete.setPath(sourceDelete.getPath());
+                targetDeletes.add(targetDelete);
+            }
+
+            destination.setDelete(targetDeletes);
+        }
+    }
+
+    private void mapMkdirs(final Prepare source, final PREPARE destination) {
+        if (source.getMkdirs() != null) {
+            final List<MKDIR> targetMkdirs = new ArrayList<>();
+
+            for (final Mkdir sourceMkDir: source.getMkdirs()) {
+                final MKDIR targetMkDir = OBJECT_FACTORY.createMKDIR();
+                targetMkDir.setPath(sourceMkDir.getPath());
+                targetMkdirs.add(targetMkDir);
+            }
+
+            destination.setMkdir(targetMkdirs);
+        }
+    }
+
+    @Override
+    public Prepare convertFrom(final PREPARE source, final Prepare destination) {
+        throw new UnsupportedOperationException("This mapping is not bidirectional.");
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/InlineWorkflowConfigurationConverter.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/InlineWorkflowConfigurationConverter.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/InlineWorkflowConfigurationConverter.java
new file mode 100644
index 0000000..b1e17c9
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/InlineWorkflowConfigurationConverter.java
@@ -0,0 +1,81 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.mapping;
+
+import org.apache.oozie.fluentjob.api.generated.workflow.CONFIGURATION;
+import org.apache.oozie.fluentjob.api.generated.workflow.ObjectFactory;
+import org.dozer.DozerConverter;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * A {@link DozerConverter} converting from {@link Map} to JAXB {@link CONFIGURATION}.
+ */
+public class InlineWorkflowConfigurationConverter extends DozerConverter<Map, CONFIGURATION> {
+    private static final ObjectFactory OBJECT_FACTORY = new ObjectFactory();
+
+    public InlineWorkflowConfigurationConverter() {
+        super(Map.class, CONFIGURATION.class);
+    }
+
+    @Override
+    public CONFIGURATION convertTo(final Map source, CONFIGURATION destination) {
+        if (source == null) {
+            return null;
+        }
+
+        destination = ensureDestination(destination);
+
+        mapEntries(source, destination);
+
+        return destination;
+    }
+
+    private CONFIGURATION ensureDestination(CONFIGURATION destination) {
+        if (destination == null) {
+            destination = OBJECT_FACTORY.createCONFIGURATION();
+        }
+
+        return destination;
+    }
+
+    private void mapEntries(final Map source, final CONFIGURATION destination) {
+        if (source != null) {
+            final List<CONFIGURATION.Property> targetProperties = new ArrayList<>();
+
+            for (final Object objectKey : source.keySet()) {
+                final String name = objectKey.toString();
+                final String value = source.get(name).toString();
+                final CONFIGURATION.Property targetProperty = OBJECT_FACTORY.createCONFIGURATIONProperty();
+                targetProperty.setName(name);
+                targetProperty.setValue(value);
+                targetProperties.add(targetProperty);
+            }
+
+            destination.setProperty(targetProperties);
+        }
+    }
+
+    @Override
+    public Map convertFrom(final CONFIGURATION source, final Map destination) {
+        throw new UnsupportedOperationException("This mapping is not bidirectional.");
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/InlineWorkflowLauncherConverter.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/InlineWorkflowLauncherConverter.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/InlineWorkflowLauncherConverter.java
new file mode 100644
index 0000000..8d32160
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/InlineWorkflowLauncherConverter.java
@@ -0,0 +1,74 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.mapping;
+
+import org.apache.oozie.fluentjob.api.generated.workflow.LAUNCHER;
+import org.apache.oozie.fluentjob.api.generated.workflow.ObjectFactory;
+import org.apache.oozie.fluentjob.api.action.Launcher;
+import org.dozer.DozerConverter;
+
+/**
+ * A {@link DozerConverter} converting from {@link Launcher} to JAXB {@link LAUNCHER}.
+ */
+public class InlineWorkflowLauncherConverter extends DozerConverter<Launcher, LAUNCHER> {
+    private static final ObjectFactory OBJECT_FACTORY = new ObjectFactory();
+
+    public InlineWorkflowLauncherConverter() {
+        super(Launcher.class, LAUNCHER.class);
+    }
+
+    @Override
+    public LAUNCHER convertTo(final Launcher source, LAUNCHER destination) {
+        if (source == null) {
+            return null;
+        }
+
+        destination = ensureDestination(destination);
+
+        mapAttributes(source, destination);
+
+        return destination;
+    }
+
+    private LAUNCHER ensureDestination(final LAUNCHER destination) {
+        if (destination == null) {
+            return OBJECT_FACTORY.createLAUNCHER();
+        }
+
+        return destination;
+    }
+
+    private void mapAttributes(final Launcher source, final LAUNCHER destination) {
+        if (source == null) {
+            return;
+        }
+
+        destination.getMemoryMbOrVcoresOrJavaOpts().add(OBJECT_FACTORY.createLAUNCHERMemoryMb(source.getMemoryMb()));
+        destination.getMemoryMbOrVcoresOrJavaOpts().add(OBJECT_FACTORY.createLAUNCHERVcores(source.getVCores()));
+        destination.getMemoryMbOrVcoresOrJavaOpts().add(OBJECT_FACTORY.createLAUNCHERQueue(source.getQueue()));
+        destination.getMemoryMbOrVcoresOrJavaOpts().add(OBJECT_FACTORY.createLAUNCHERSharelib(source.getSharelib()));
+        destination.getMemoryMbOrVcoresOrJavaOpts().add(OBJECT_FACTORY.createLAUNCHERViewAcl(source.getViewAcl()));
+        destination.getMemoryMbOrVcoresOrJavaOpts().add(OBJECT_FACTORY.createLAUNCHERModifyAcl(source.getModifyAcl()));
+    }
+
+    @Override
+    public Launcher convertFrom(final LAUNCHER source, final Launcher destination) {
+        throw new UnsupportedOperationException("This mapping is not bidirectional.");
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/InlineWorkflowPrepareConverter.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/InlineWorkflowPrepareConverter.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/InlineWorkflowPrepareConverter.java
new file mode 100644
index 0000000..5f2bf7b
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/InlineWorkflowPrepareConverter.java
@@ -0,0 +1,97 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.mapping;
+
+import org.apache.oozie.fluentjob.api.generated.workflow.DELETE;
+import org.apache.oozie.fluentjob.api.generated.workflow.MKDIR;
+import org.apache.oozie.fluentjob.api.generated.workflow.ObjectFactory;
+import org.apache.oozie.fluentjob.api.generated.workflow.PREPARE;
+import org.apache.oozie.fluentjob.api.action.Delete;
+import org.apache.oozie.fluentjob.api.action.Mkdir;
+import org.apache.oozie.fluentjob.api.action.Prepare;
+import org.dozer.DozerConverter;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * A {@link DozerConverter} converting from {@link Prepare} to JAXB {@link PREPARE}.
+ */
+public class InlineWorkflowPrepareConverter extends DozerConverter<Prepare, PREPARE> {
+    private static final ObjectFactory OBJECT_FACTORY = new ObjectFactory();
+
+    public InlineWorkflowPrepareConverter() {
+        super(Prepare.class, PREPARE.class);
+    }
+
+    @Override
+    public PREPARE convertTo(final Prepare source, PREPARE destination) {
+        if (source == null) {
+            return null;
+        }
+
+        destination = ensureDestination(destination);
+
+        mapDeletes(source, destination);
+
+        mapMkdirs(source, destination);
+
+        return destination;
+    }
+
+    private PREPARE ensureDestination(final PREPARE destination) {
+        if (destination == null) {
+            return OBJECT_FACTORY.createPREPARE();
+        }
+        return destination;
+    }
+
+    private void mapDeletes(final Prepare source, final PREPARE destination) {
+        if (source.getDeletes() != null) {
+            final List<DELETE> targetDeletes = new ArrayList<>();
+
+            for (final Delete sourceDelete : source.getDeletes()) {
+                final DELETE targetDelete = OBJECT_FACTORY.createDELETE();
+                targetDelete.setPath(sourceDelete.getPath());
+                targetDeletes.add(targetDelete);
+            }
+
+            destination.setDelete(targetDeletes);
+        }
+    }
+
+    private void mapMkdirs(final Prepare source, final PREPARE destination) {
+        if (source.getMkdirs() != null) {
+            final List<MKDIR> targetMkdirs = new ArrayList<>();
+
+            for (final Mkdir sourceMkDir: source.getMkdirs()) {
+                final MKDIR targetMkDir = OBJECT_FACTORY.createMKDIR();
+                targetMkDir.setPath(sourceMkDir.getPath());
+                targetMkdirs.add(targetMkDir);
+            }
+
+            destination.setMkdir(targetMkdirs);
+        }
+    }
+
+    @Override
+    public Prepare convertFrom(final PREPARE source, final Prepare destination) {
+        throw new UnsupportedOperationException("This mapping is not bidirectional.");
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/JoinConverter.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/JoinConverter.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/JoinConverter.java
new file mode 100644
index 0000000..e9495b6
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/JoinConverter.java
@@ -0,0 +1,63 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.mapping;
+
+import org.apache.oozie.fluentjob.api.generated.workflow.JOIN;
+import org.apache.oozie.fluentjob.api.generated.workflow.ObjectFactory;
+import org.apache.oozie.fluentjob.api.dag.Join;
+import org.apache.oozie.fluentjob.api.dag.NodeBase;
+import org.dozer.DozerConverter;
+
+/**
+ * A {@link DozerConverter} converting from {@link Join} to JAXB {@link JOIN}.
+ */
+public class JoinConverter extends DozerConverter<Join, JOIN> {
+
+    private static final ObjectFactory WORKFLOW_OBJECT_FACTORY = new ObjectFactory();
+
+    public JoinConverter() {
+        super(Join.class, JOIN.class);
+    }
+
+    @Override
+    public JOIN convertTo(final Join source, JOIN destination) {
+        destination = ensureDestination(destination);
+
+        destination.setName(source.getName());
+
+        final NodeBase child  = source.getChild();
+        final NodeBase realChild = RealChildLocator.findRealChild(child);
+
+        destination.setTo(realChild.getName());
+
+        return destination;
+    }
+
+    private JOIN ensureDestination(JOIN destination) {
+        if (destination == null) {
+            destination = WORKFLOW_OBJECT_FACTORY.createJOIN();
+        }
+        return destination;
+    }
+
+    @Override
+    public Join convertFrom(JOIN source, Join destination) {
+        throw new UnsupportedOperationException("This mapping is not bidirectional.");
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/MapToConfigurationPropertyConverter.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/MapToConfigurationPropertyConverter.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/MapToConfigurationPropertyConverter.java
new file mode 100644
index 0000000..c86e066
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/MapToConfigurationPropertyConverter.java
@@ -0,0 +1,75 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.mapping;
+
+import org.apache.oozie.fluentjob.api.generated.workflow.CONFIGURATION;
+import org.apache.oozie.fluentjob.api.generated.workflow.ObjectFactory;
+import org.dozer.DozerConverter;
+
+import java.util.Map;
+
+/**
+ * A {@link DozerConverter} converting from {@link Map} to JAXB {@link CONFIGURATION}.
+ */
+public class MapToConfigurationPropertyConverter extends DozerConverter<Map, CONFIGURATION> {
+    private static final ObjectFactory OBJECT_FACTORY = new ObjectFactory();
+
+    public MapToConfigurationPropertyConverter() {
+        super(Map.class, CONFIGURATION.class);
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public CONFIGURATION convertTo(final Map source, CONFIGURATION destination) {
+        destination = ensureConfiguration(destination);
+
+        for (final Object entryObject : source.entrySet()) {
+            final Map.Entry<String, String> entry = (Map.Entry<String, String>) entryObject;
+            final String key = entry.getKey();
+            final String value = entry.getValue();
+
+            final CONFIGURATION.Property property = createProperty(key, value);
+
+            destination.getProperty().add(property);
+        }
+
+        return destination;
+    }
+
+    private CONFIGURATION ensureConfiguration(CONFIGURATION destination) {
+        if (destination == null) {
+            destination = OBJECT_FACTORY.createCONFIGURATION();
+        }
+        return destination;
+    }
+
+    private CONFIGURATION.Property createProperty(final String key, final String value) {
+        final CONFIGURATION.Property property = OBJECT_FACTORY.createCONFIGURATIONProperty();
+
+        property.setName(key);
+        property.setValue(value);
+
+        return property;
+    }
+
+    @Override
+    public Map convertFrom(final CONFIGURATION source, final Map destination) {
+        throw new UnsupportedOperationException("This mapping is not bidirectional.");
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/ParametersConverter.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/ParametersConverter.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/ParametersConverter.java
new file mode 100644
index 0000000..f3677e5
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/ParametersConverter.java
@@ -0,0 +1,73 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.mapping;
+
+import org.apache.oozie.fluentjob.api.generated.workflow.ObjectFactory;
+import org.apache.oozie.fluentjob.api.generated.workflow.PARAMETERS;
+import org.apache.oozie.fluentjob.api.workflow.Parameter;
+import org.apache.oozie.fluentjob.api.workflow.Parameters;
+import org.dozer.DozerConverter;
+
+/**
+ * A {@link DozerConverter} converting from {@link Parameters} to JAXB {@link PARAMETERS}.
+ */
+public class ParametersConverter extends DozerConverter<Parameters, PARAMETERS> {
+    private static final ObjectFactory OBJECT_FACTORY = new ObjectFactory();
+
+    public ParametersConverter() {
+        super(Parameters.class, PARAMETERS.class);
+    }
+
+    @Override
+    public PARAMETERS convertTo(final Parameters source, PARAMETERS destination) {
+        if (source == null) {
+            return null;
+        }
+
+        destination = ensureDestination(destination);
+
+        mapParameters(source, destination);
+
+        return destination;
+    }
+
+    private PARAMETERS ensureDestination(final PARAMETERS destination) {
+        if (destination == null) {
+            return OBJECT_FACTORY.createPARAMETERS();
+        }
+
+        return destination;
+    }
+
+    private void mapParameters(final Parameters source, final PARAMETERS destination) {
+        for (final Parameter parameter : source.getParameters()) {
+            final PARAMETERS.Property property = OBJECT_FACTORY.createPARAMETERSProperty();
+            property.setName(parameter.getName());
+            property.setValue(parameter.getValue());
+            property.setDescription(parameter.getDescription());
+
+            destination.getProperty().add(property);
+        }
+    }
+
+    @Override
+    public Parameters convertFrom(final PARAMETERS source, final Parameters destination) {
+        throw new UnsupportedOperationException("This mapping is not bidirectional.");
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/RealChildLocator.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/RealChildLocator.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/RealChildLocator.java
new file mode 100644
index 0000000..a1073a0
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/RealChildLocator.java
@@ -0,0 +1,35 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.mapping;
+
+import org.apache.oozie.fluentjob.api.dag.DecisionJoin;
+import org.apache.oozie.fluentjob.api.dag.NodeBase;
+
+/**
+ * Finds the real child among list of {@link NodeBase} children, that is, one that isn't a {@link DecisionJoin}.
+ */
+class RealChildLocator {
+    static NodeBase findRealChild(final NodeBase originalChild) {
+        if (originalChild instanceof DecisionJoin) {
+            return findRealChild(((DecisionJoin) originalChild).getChild());
+        }
+
+        return originalChild;
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/ShellConfigurationConverter.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/ShellConfigurationConverter.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/ShellConfigurationConverter.java
new file mode 100644
index 0000000..ab73dfd
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/ShellConfigurationConverter.java
@@ -0,0 +1,81 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.mapping;
+
+import org.apache.oozie.fluentjob.api.generated.action.shell.CONFIGURATION;
+import org.apache.oozie.fluentjob.api.generated.action.shell.ObjectFactory;
+import org.dozer.DozerConverter;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * A {@link DozerConverter} converting from {@link Map} to JAXB {@link CONFIGURATION}.
+ */
+public class ShellConfigurationConverter extends DozerConverter<Map, CONFIGURATION> {
+    private static final ObjectFactory OBJECT_FACTORY = new ObjectFactory();
+
+    public ShellConfigurationConverter() {
+        super(Map.class, CONFIGURATION.class);
+    }
+
+    @Override
+    public CONFIGURATION convertTo(final Map source, CONFIGURATION destination) {
+        if (source == null) {
+            return null;
+        }
+
+        destination = ensureDestination(destination);
+
+        mapEntries(source, destination);
+
+        return destination;
+    }
+
+    private CONFIGURATION ensureDestination(CONFIGURATION destination) {
+        if (destination == null) {
+            destination = OBJECT_FACTORY.createCONFIGURATION();
+        }
+
+        return destination;
+    }
+
+    private void mapEntries(final Map source, final CONFIGURATION destination) {
+        if (source != null) {
+            final List<CONFIGURATION.Property> targetProperties = new ArrayList<>();
+
+            for (final Object objectKey : source.keySet()) {
+                final String name = objectKey.toString();
+                final String value = source.get(name).toString();
+                final CONFIGURATION.Property targetProperty = OBJECT_FACTORY.createCONFIGURATIONProperty();
+                targetProperty.setName(name);
+                targetProperty.setValue(value);
+                targetProperties.add(targetProperty);
+            }
+
+            destination.setProperty(targetProperties);
+        }
+    }
+
+    @Override
+    public Map convertFrom(final CONFIGURATION source, final Map destination) {
+        throw new UnsupportedOperationException("This mapping is not bidirectional.");
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/ShellLauncherConverter.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/ShellLauncherConverter.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/ShellLauncherConverter.java
new file mode 100644
index 0000000..d933499
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/ShellLauncherConverter.java
@@ -0,0 +1,74 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.mapping;
+
+import org.apache.oozie.fluentjob.api.generated.action.shell.LAUNCHER;
+import org.apache.oozie.fluentjob.api.generated.action.shell.ObjectFactory;
+import org.apache.oozie.fluentjob.api.action.Launcher;
+import org.dozer.DozerConverter;
+
+/**
+ * A {@link DozerConverter} converting from {@link Launcher} to JAXB {@link LAUNCHER}.
+ */
+public class ShellLauncherConverter extends DozerConverter<Launcher, LAUNCHER> {
+    private static final ObjectFactory OBJECT_FACTORY = new ObjectFactory();
+
+    public ShellLauncherConverter() {
+        super(Launcher.class, LAUNCHER.class);
+    }
+
+    @Override
+    public LAUNCHER convertTo(final Launcher source, LAUNCHER destination) {
+        if (source == null) {
+            return null;
+        }
+
+        destination = ensureDestination(destination);
+
+        mapAttributes(source, destination);
+
+        return destination;
+    }
+
+    private LAUNCHER ensureDestination(final LAUNCHER destination) {
+        if (destination == null) {
+            return OBJECT_FACTORY.createLAUNCHER();
+        }
+
+        return destination;
+    }
+
+    private void mapAttributes(final Launcher source, final LAUNCHER destination) {
+        if (source == null) {
+            return;
+        }
+
+        destination.getMemoryMbOrVcoresOrJavaOpts().add(OBJECT_FACTORY.createLAUNCHERMemoryMb(source.getMemoryMb()));
+        destination.getMemoryMbOrVcoresOrJavaOpts().add(OBJECT_FACTORY.createLAUNCHERVcores(source.getVCores()));
+        destination.getMemoryMbOrVcoresOrJavaOpts().add(OBJECT_FACTORY.createLAUNCHERQueue(source.getQueue()));
+        destination.getMemoryMbOrVcoresOrJavaOpts().add(OBJECT_FACTORY.createLAUNCHERSharelib(source.getSharelib()));
+        destination.getMemoryMbOrVcoresOrJavaOpts().add(OBJECT_FACTORY.createLAUNCHERViewAcl(source.getViewAcl()));
+        destination.getMemoryMbOrVcoresOrJavaOpts().add(OBJECT_FACTORY.createLAUNCHERModifyAcl(source.getModifyAcl()));
+    }
+
+    @Override
+    public Launcher convertFrom(final LAUNCHER source, final Launcher destination) {
+        throw new UnsupportedOperationException("This mapping is not bidirectional.");
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/ShellPrepareConverter.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/ShellPrepareConverter.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/ShellPrepareConverter.java
new file mode 100644
index 0000000..30fcf7c
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/ShellPrepareConverter.java
@@ -0,0 +1,97 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.mapping;
+
+import org.apache.oozie.fluentjob.api.generated.action.shell.DELETE;
+import org.apache.oozie.fluentjob.api.generated.action.shell.MKDIR;
+import org.apache.oozie.fluentjob.api.generated.action.shell.ObjectFactory;
+import org.apache.oozie.fluentjob.api.generated.action.shell.PREPARE;
+import org.apache.oozie.fluentjob.api.action.Delete;
+import org.apache.oozie.fluentjob.api.action.Mkdir;
+import org.apache.oozie.fluentjob.api.action.Prepare;
+import org.dozer.DozerConverter;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * A {@link DozerConverter} converting from {@link Prepare} to JAXB {@link PREPARE}.
+ */
+public class ShellPrepareConverter extends DozerConverter<Prepare, PREPARE> {
+    private static final ObjectFactory OBJECT_FACTORY = new ObjectFactory();
+
+    public ShellPrepareConverter() {
+        super(Prepare.class, PREPARE.class);
+    }
+
+    @Override
+    public PREPARE convertTo(final Prepare source, PREPARE destination) {
+        if (source == null) {
+            return null;
+        }
+
+        destination = ensureDestination(destination);
+
+        mapDeletes(source, destination);
+
+        mapMkdirs(source, destination);
+
+        return destination;
+    }
+
+    private PREPARE ensureDestination(final PREPARE destination) {
+        if (destination == null) {
+            return OBJECT_FACTORY.createPREPARE();
+        }
+        return destination;
+    }
+
+    private void mapDeletes(final Prepare source, final PREPARE destination) {
+        if (source.getDeletes() != null) {
+            final List<DELETE> targetDeletes = new ArrayList<>();
+
+            for (final Delete sourceDelete : source.getDeletes()) {
+                final DELETE targetDelete = OBJECT_FACTORY.createDELETE();
+                targetDelete.setPath(sourceDelete.getPath());
+                targetDeletes.add(targetDelete);
+            }
+
+            destination.setDelete(targetDeletes);
+        }
+    }
+
+    private void mapMkdirs(final Prepare source, final PREPARE destination) {
+        if (source.getMkdirs() != null) {
+            final List<MKDIR> targetMkdirs = new ArrayList<>();
+
+            for (final Mkdir sourceMkDir: source.getMkdirs()) {
+                final MKDIR targetMkDir = OBJECT_FACTORY.createMKDIR();
+                targetMkDir.setPath(sourceMkDir.getPath());
+                targetMkdirs.add(targetMkDir);
+            }
+
+            destination.setMkdir(targetMkdirs);
+        }
+    }
+
+    @Override
+    public Prepare convertFrom(final PREPARE source, final Prepare destination) {
+        throw new UnsupportedOperationException("This mapping is not bidirectional.");
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/SparkConfigurationConverter.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/SparkConfigurationConverter.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/SparkConfigurationConverter.java
new file mode 100644
index 0000000..8827769
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/SparkConfigurationConverter.java
@@ -0,0 +1,81 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.mapping;
+
+import org.apache.oozie.fluentjob.api.generated.action.spark.CONFIGURATION;
+import org.apache.oozie.fluentjob.api.generated.action.spark.ObjectFactory;
+import org.dozer.DozerConverter;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * A {@link DozerConverter} converting from {@link Map} to JAXB {@link CONFIGURATION}.
+ */
+public class SparkConfigurationConverter extends DozerConverter<Map, CONFIGURATION> {
+    private static final ObjectFactory OBJECT_FACTORY = new ObjectFactory();
+
+    public SparkConfigurationConverter() {
+        super(Map.class, CONFIGURATION.class);
+    }
+
+    @Override
+    public CONFIGURATION convertTo(final Map source, CONFIGURATION destination) {
+        if (source == null) {
+            return null;
+        }
+
+        destination = ensureDestination(destination);
+
+        mapEntries(source, destination);
+
+        return destination;
+    }
+
+    private CONFIGURATION ensureDestination(CONFIGURATION destination) {
+        if (destination == null) {
+            destination = OBJECT_FACTORY.createCONFIGURATION();
+        }
+
+        return destination;
+    }
+
+    private void mapEntries(final Map source, final CONFIGURATION destination) {
+        if (source != null) {
+            final List<CONFIGURATION.Property> targetProperties = new ArrayList<>();
+
+            for (final Object objectKey : source.keySet()) {
+                final String name = objectKey.toString();
+                final String value = source.get(name).toString();
+                final CONFIGURATION.Property targetProperty = OBJECT_FACTORY.createCONFIGURATIONProperty();
+                targetProperty.setName(name);
+                targetProperty.setValue(value);
+                targetProperties.add(targetProperty);
+            }
+
+            destination.setProperty(targetProperties);
+        }
+    }
+
+    @Override
+    public Map convertFrom(final CONFIGURATION source, final Map destination) {
+        throw new UnsupportedOperationException("This mapping is not bidirectional.");
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/SparkLauncherConverter.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/SparkLauncherConverter.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/SparkLauncherConverter.java
new file mode 100644
index 0000000..be7436a
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/SparkLauncherConverter.java
@@ -0,0 +1,74 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.mapping;
+
+import org.apache.oozie.fluentjob.api.generated.action.spark.LAUNCHER;
+import org.apache.oozie.fluentjob.api.generated.action.spark.ObjectFactory;
+import org.apache.oozie.fluentjob.api.action.Launcher;
+import org.dozer.DozerConverter;
+
+/**
+ * A {@link DozerConverter} converting from {@link Launcher} to JAXB {@link LAUNCHER}.
+ */
+public class SparkLauncherConverter extends DozerConverter<Launcher, LAUNCHER> {
+    private static final ObjectFactory OBJECT_FACTORY = new ObjectFactory();
+
+    public SparkLauncherConverter() {
+        super(Launcher.class, LAUNCHER.class);
+    }
+
+    @Override
+    public LAUNCHER convertTo(final Launcher source, LAUNCHER destination) {
+        if (source == null) {
+            return null;
+        }
+
+        destination = ensureDestination(destination);
+
+        mapAttributes(source, destination);
+
+        return destination;
+    }
+
+    private LAUNCHER ensureDestination(final LAUNCHER destination) {
+        if (destination == null) {
+            return OBJECT_FACTORY.createLAUNCHER();
+        }
+
+        return destination;
+    }
+
+    private void mapAttributes(final Launcher source, final LAUNCHER destination) {
+        if (source == null) {
+            return;
+        }
+
+        destination.getMemoryMbOrVcoresOrJavaOpts().add(OBJECT_FACTORY.createLAUNCHERMemoryMb(source.getMemoryMb()));
+        destination.getMemoryMbOrVcoresOrJavaOpts().add(OBJECT_FACTORY.createLAUNCHERVcores(source.getVCores()));
+        destination.getMemoryMbOrVcoresOrJavaOpts().add(OBJECT_FACTORY.createLAUNCHERQueue(source.getQueue()));
+        destination.getMemoryMbOrVcoresOrJavaOpts().add(OBJECT_FACTORY.createLAUNCHERSharelib(source.getSharelib()));
+        destination.getMemoryMbOrVcoresOrJavaOpts().add(OBJECT_FACTORY.createLAUNCHERViewAcl(source.getViewAcl()));
+        destination.getMemoryMbOrVcoresOrJavaOpts().add(OBJECT_FACTORY.createLAUNCHERModifyAcl(source.getModifyAcl()));
+    }
+
+    @Override
+    public Launcher convertFrom(final LAUNCHER source, final Launcher destination) {
+        throw new UnsupportedOperationException("This mapping is not bidirectional.");
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/SparkPrepareConverter.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/SparkPrepareConverter.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/SparkPrepareConverter.java
new file mode 100644
index 0000000..812e53d
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/SparkPrepareConverter.java
@@ -0,0 +1,97 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.mapping;
+
+import org.apache.oozie.fluentjob.api.generated.action.spark.DELETE;
+import org.apache.oozie.fluentjob.api.generated.action.spark.MKDIR;
+import org.apache.oozie.fluentjob.api.generated.action.spark.ObjectFactory;
+import org.apache.oozie.fluentjob.api.generated.action.spark.PREPARE;
+import org.apache.oozie.fluentjob.api.action.Delete;
+import org.apache.oozie.fluentjob.api.action.Mkdir;
+import org.apache.oozie.fluentjob.api.action.Prepare;
+import org.dozer.DozerConverter;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * A {@link DozerConverter} converting from {@link Prepare} to JAXB {@link PREPARE}.
+ */
+public class SparkPrepareConverter extends DozerConverter<Prepare, PREPARE> {
+    private static final ObjectFactory OBJECT_FACTORY = new ObjectFactory();
+
+    public SparkPrepareConverter() {
+        super(Prepare.class, PREPARE.class);
+    }
+
+    @Override
+    public PREPARE convertTo(final Prepare source, PREPARE destination) {
+        if (source == null) {
+            return null;
+        }
+
+        destination = ensureDestination(destination);
+
+        mapDeletes(source, destination);
+
+        mapMkdirs(source, destination);
+
+        return destination;
+    }
+
+    private PREPARE ensureDestination(final PREPARE destination) {
+        if (destination == null) {
+            return OBJECT_FACTORY.createPREPARE();
+        }
+        return destination;
+    }
+
+    private void mapDeletes(final Prepare source, final PREPARE destination) {
+        if (source.getDeletes() != null) {
+            final List<DELETE> targetDeletes = new ArrayList<>();
+
+            for (final Delete sourceDelete : source.getDeletes()) {
+                final DELETE targetDelete = OBJECT_FACTORY.createDELETE();
+                targetDelete.setPath(sourceDelete.getPath());
+                targetDeletes.add(targetDelete);
+            }
+
+            destination.setDelete(targetDeletes);
+        }
+    }
+
+    private void mapMkdirs(final Prepare source, final PREPARE destination) {
+        if (source.getMkdirs() != null) {
+            final List<MKDIR> targetMkdirs = new ArrayList<>();
+
+            for (final Mkdir sourceMkDir: source.getMkdirs()) {
+                final MKDIR targetMkDir = OBJECT_FACTORY.createMKDIR();
+                targetMkDir.setPath(sourceMkDir.getPath());
+                targetMkdirs.add(targetMkDir);
+            }
+
+            destination.setMkdir(targetMkdirs);
+        }
+    }
+
+    @Override
+    public Prepare convertFrom(final PREPARE source, final Prepare destination) {
+        throw new UnsupportedOperationException("This mapping is not bidirectional.");
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/SqoopConfigurationConverter.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/SqoopConfigurationConverter.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/SqoopConfigurationConverter.java
new file mode 100644
index 0000000..1d4f615
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/SqoopConfigurationConverter.java
@@ -0,0 +1,81 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.mapping;
+
+import org.apache.oozie.fluentjob.api.generated.action.sqoop.CONFIGURATION;
+import org.apache.oozie.fluentjob.api.generated.action.sqoop.ObjectFactory;
+import org.dozer.DozerConverter;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * A {@link DozerConverter} converting from {@link Map} to JAXB {@link CONFIGURATION}.
+ */
+public class SqoopConfigurationConverter extends DozerConverter<Map, CONFIGURATION> {
+    private static final ObjectFactory OBJECT_FACTORY = new ObjectFactory();
+
+    public SqoopConfigurationConverter() {
+        super(Map.class, CONFIGURATION.class);
+    }
+
+    @Override
+    public CONFIGURATION convertTo(final Map source, CONFIGURATION destination) {
+        if (source == null) {
+            return null;
+        }
+
+        destination = ensureDestination(destination);
+
+        mapEntries(source, destination);
+
+        return destination;
+    }
+
+    private CONFIGURATION ensureDestination(CONFIGURATION destination) {
+        if (destination == null) {
+            destination = OBJECT_FACTORY.createCONFIGURATION();
+        }
+
+        return destination;
+    }
+
+    private void mapEntries(final Map source, final CONFIGURATION destination) {
+        if (source != null) {
+            final List<CONFIGURATION.Property> targetProperties = new ArrayList<>();
+
+            for (final Object objectKey : source.keySet()) {
+                final String name = objectKey.toString();
+                final String value = source.get(name).toString();
+                final CONFIGURATION.Property targetProperty = OBJECT_FACTORY.createCONFIGURATIONProperty();
+                targetProperty.setName(name);
+                targetProperty.setValue(value);
+                targetProperties.add(targetProperty);
+            }
+
+            destination.setProperty(targetProperties);
+        }
+    }
+
+    @Override
+    public Map convertFrom(final CONFIGURATION source, final Map destination) {
+        throw new UnsupportedOperationException("This mapping is not bidirectional.");
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/SqoopLauncherConverter.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/SqoopLauncherConverter.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/SqoopLauncherConverter.java
new file mode 100644
index 0000000..556cb7b
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/SqoopLauncherConverter.java
@@ -0,0 +1,74 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.mapping;
+
+import org.apache.oozie.fluentjob.api.generated.action.sqoop.LAUNCHER;
+import org.apache.oozie.fluentjob.api.generated.action.sqoop.ObjectFactory;
+import org.apache.oozie.fluentjob.api.action.Launcher;
+import org.dozer.DozerConverter;
+
+/**
+ * A {@link DozerConverter} converting from {@link Launcher} to JAXB {@link LAUNCHER}.
+ */
+public class SqoopLauncherConverter extends DozerConverter<Launcher, LAUNCHER> {
+    private static final ObjectFactory OBJECT_FACTORY = new ObjectFactory();
+
+    public SqoopLauncherConverter() {
+        super(Launcher.class, LAUNCHER.class);
+    }
+
+    @Override
+    public LAUNCHER convertTo(final Launcher source, LAUNCHER destination) {
+        if (source == null) {
+            return null;
+        }
+
+        destination = ensureDestination(destination);
+
+        mapAttributes(source, destination);
+
+        return destination;
+    }
+
+    private LAUNCHER ensureDestination(final LAUNCHER destination) {
+        if (destination == null) {
+            return OBJECT_FACTORY.createLAUNCHER();
+        }
+
+        return destination;
+    }
+
+    private void mapAttributes(final Launcher source, final LAUNCHER destination) {
+        if (source == null) {
+            return;
+        }
+
+        destination.getMemoryMbOrVcoresOrJavaOpts().add(OBJECT_FACTORY.createLAUNCHERMemoryMb(source.getMemoryMb()));
+        destination.getMemoryMbOrVcoresOrJavaOpts().add(OBJECT_FACTORY.createLAUNCHERVcores(source.getVCores()));
+        destination.getMemoryMbOrVcoresOrJavaOpts().add(OBJECT_FACTORY.createLAUNCHERQueue(source.getQueue()));
+        destination.getMemoryMbOrVcoresOrJavaOpts().add(OBJECT_FACTORY.createLAUNCHERSharelib(source.getSharelib()));
+        destination.getMemoryMbOrVcoresOrJavaOpts().add(OBJECT_FACTORY.createLAUNCHERViewAcl(source.getViewAcl()));
+        destination.getMemoryMbOrVcoresOrJavaOpts().add(OBJECT_FACTORY.createLAUNCHERModifyAcl(source.getModifyAcl()));
+    }
+
+    @Override
+    public Launcher convertFrom(final LAUNCHER source, final Launcher destination) {
+        throw new UnsupportedOperationException("This mapping is not bidirectional.");
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/SqoopPrepareConverter.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/SqoopPrepareConverter.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/SqoopPrepareConverter.java
new file mode 100644
index 0000000..5062c08
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/SqoopPrepareConverter.java
@@ -0,0 +1,97 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.mapping;
+
+import org.apache.oozie.fluentjob.api.generated.action.sqoop.DELETE;
+import org.apache.oozie.fluentjob.api.generated.action.sqoop.MKDIR;
+import org.apache.oozie.fluentjob.api.generated.action.sqoop.ObjectFactory;
+import org.apache.oozie.fluentjob.api.generated.action.sqoop.PREPARE;
+import org.apache.oozie.fluentjob.api.action.Delete;
+import org.apache.oozie.fluentjob.api.action.Mkdir;
+import org.apache.oozie.fluentjob.api.action.Prepare;
+import org.dozer.DozerConverter;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * A {@link DozerConverter} converting from {@link Prepare} to JAXB {@link PREPARE}.
+ */
+public class SqoopPrepareConverter extends DozerConverter<Prepare, PREPARE> {
+    private static final ObjectFactory OBJECT_FACTORY = new ObjectFactory();
+
+    public SqoopPrepareConverter() {
+        super(Prepare.class, PREPARE.class);
+    }
+
+    @Override
+    public PREPARE convertTo(final Prepare source, PREPARE destination) {
+        if (source == null) {
+            return null;
+        }
+
+        destination = ensureDestination(destination);
+
+        mapDeletes(source, destination);
+
+        mapMkdirs(source, destination);
+
+        return destination;
+    }
+
+    private PREPARE ensureDestination(final PREPARE destination) {
+        if (destination == null) {
+            return OBJECT_FACTORY.createPREPARE();
+        }
+        return destination;
+    }
+
+    private void mapDeletes(final Prepare source, final PREPARE destination) {
+        if (source.getDeletes() != null) {
+            final List<DELETE> targetDeletes = new ArrayList<>();
+
+            for (final Delete sourceDelete : source.getDeletes()) {
+                final DELETE targetDelete = OBJECT_FACTORY.createDELETE();
+                targetDelete.setPath(sourceDelete.getPath());
+                targetDeletes.add(targetDelete);
+            }
+
+            destination.setDelete(targetDeletes);
+        }
+    }
+
+    private void mapMkdirs(final Prepare source, final PREPARE destination) {
+        if (source.getMkdirs() != null) {
+            final List<MKDIR> targetMkdirs = new ArrayList<>();
+
+            for (final Mkdir sourceMkDir: source.getMkdirs()) {
+                final MKDIR targetMkDir = OBJECT_FACTORY.createMKDIR();
+                targetMkDir.setPath(sourceMkDir.getPath());
+                targetMkdirs.add(targetMkDir);
+            }
+
+            destination.setMkdir(targetMkdirs);
+        }
+    }
+
+    @Override
+    public Prepare convertFrom(final PREPARE source, final Prepare destination) {
+        throw new UnsupportedOperationException("This mapping is not bidirectional.");
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/StartConverter.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/StartConverter.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/StartConverter.java
new file mode 100644
index 0000000..fe4d55e
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/StartConverter.java
@@ -0,0 +1,50 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.mapping;
+
+import org.apache.oozie.fluentjob.api.generated.workflow.ObjectFactory;
+import org.apache.oozie.fluentjob.api.generated.workflow.START;
+import org.apache.oozie.fluentjob.api.dag.NodeBase;
+import org.apache.oozie.fluentjob.api.dag.Start;
+import org.dozer.DozerConverter;
+
+/**
+ * A {@link DozerConverter} converting from {@link Start} to JAXB {@link START}.
+ */
+public class StartConverter extends DozerConverter<Start, START> {
+    public StartConverter() {
+        super(Start.class, START.class);
+    }
+
+    @Override
+    public START convertTo(Start source, START destination) {
+        if (destination == null) {
+            destination = new ObjectFactory().createSTART();
+        }
+
+        final NodeBase realChild = RealChildLocator.findRealChild(source.getChild());
+        destination.setTo(realChild.getName());
+        return destination;
+    }
+
+    @Override
+    public Start convertFrom(START source, Start destination) {
+        throw new UnsupportedOperationException("This mapping is not bidirectional.");
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/serialization/WorkflowMarshaller.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/serialization/WorkflowMarshaller.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/serialization/WorkflowMarshaller.java
new file mode 100644
index 0000000..ec56554
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/serialization/WorkflowMarshaller.java
@@ -0,0 +1,105 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.serialization;
+
+import org.apache.oozie.fluentjob.api.action.Node;
+import org.apache.oozie.fluentjob.api.generated.workflow.ObjectFactory;
+import org.apache.oozie.fluentjob.api.generated.workflow.WORKFLOWAPP;
+import org.apache.oozie.fluentjob.api.mapping.DozerBeanMapperSingleton;
+import org.apache.oozie.fluentjob.api.dag.Graph;
+import org.apache.oozie.fluentjob.api.workflow.Workflow;
+
+import javax.xml.bind.JAXBContext;
+import javax.xml.bind.JAXBElement;
+import javax.xml.bind.JAXBException;
+import javax.xml.bind.Marshaller;
+import java.io.ByteArrayOutputStream;
+import java.io.UnsupportedEncodingException;
+import java.nio.charset.Charset;
+import java.util.Locale;
+
+/**
+ * Similar to JAXB {@link Marshaller} / {@link javax.xml.bind.Unmarshaller}, this class translates between Jobs API {@link Workflow}
+ * and JAXB {@link WORKFLOWAPP} by using the appropriate Dozer converters.
+ */
+public class WorkflowMarshaller {
+
+    public static String marshal(final Workflow workflow) throws JAXBException, UnsupportedEncodingException {
+        final Graph graph = new Graph(workflow);
+        final WORKFLOWAPP workflowapp = DozerBeanMapperSingleton.instance().map(graph, WORKFLOWAPP.class);
+        final String filteredPackages = filterPackages(workflow);
+
+        return marshal(workflowapp, filteredPackages);
+    }
+
+    private static String marshal(final WORKFLOWAPP workflowapp, final String filteredPackages)
+            throws JAXBException, UnsupportedEncodingException {
+        final JAXBElement<?> wfElement = new ObjectFactory().createWorkflowApp(workflowapp);
+
+        final JAXBContext jc = JAXBContext.newInstance(filteredPackages);
+        final Marshaller m =  jc.createMarshaller();
+        m.setProperty(Marshaller.JAXB_FORMATTED_OUTPUT, Boolean.TRUE);
+        final ByteArrayOutputStream out = new ByteArrayOutputStream();
+        m.marshal(wfElement, out);
+
+        return out.toString(Charset.defaultCharset().name());
+    }
+
+    private static String filterPackages(final Workflow workflow) {
+        final StringBuilder filteredPackages = new StringBuilder();
+
+        filteredPackages.append("org.apache.oozie.fluentjob.api.generated.workflow");
+        appendIfPresent(workflow, filteredPackages, "distcp");
+        appendIfPresent(workflow, filteredPackages, "email");
+        appendIfPresent(workflow, filteredPackages, "hive2");
+        appendIfPresent(workflow, filteredPackages, "hive");
+        appendIfPresent(workflow, filteredPackages, "sla");
+        appendIfPresent(workflow, filteredPackages, "shell");
+        appendIfPresent(workflow, filteredPackages, "spark");
+        appendIfPresent(workflow, filteredPackages, "sqoop");
+        appendIfPresent(workflow, filteredPackages, "ssh");
+
+        return filteredPackages.toString();
+    }
+
+    private static void appendIfPresent(final Workflow workflow, final StringBuilder filteredPackages, final String nodeType) {
+        if (containsNodeType(workflow, nodeType)) {
+            filteredPackages.append(":org.apache.oozie.fluentjob.api.generated.action.").append(nodeType);
+        }
+    }
+
+    private static boolean containsNodeType(final Workflow workflow, final String nodeType) {
+        final String actionType = nodeType + "action";
+        for (final Node node : workflow.getAllNodes()) {
+            final String nodeSimpleName = node.getClass().getSimpleName();
+            if (nodeSimpleName.toLowerCase(Locale.getDefault()).startsWith(actionType.toLowerCase(Locale.getDefault()))) {
+                return true;
+            }
+            if (node.getErrorHandler() != null) {
+                final String errorHandlerSimpleName = node.getErrorHandler().getHandlerNode().getClass().getSimpleName();
+                if (errorHandlerSimpleName.toLowerCase(Locale.getDefault())
+                        .startsWith(actionType.toLowerCase(Locale.getDefault()))) {
+                    return true;
+                }
+            }
+        }
+
+        return false;
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/8a0a6487/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/workflow/ConfigurationEntry.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/workflow/ConfigurationEntry.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/workflow/ConfigurationEntry.java
new file mode 100644
index 0000000..72b2d5a
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/workflow/ConfigurationEntry.java
@@ -0,0 +1,61 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.oozie.fluentjob.api.workflow;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Represents the {@code <configuration>} element and its siblings inside workflow XML / XSD.
+ * <p>
+ * By assigning non-{@code null} field values, the resulting parent {@code <workflow>} will have its
+ * optional {@code <configuration>} element and its siblings filled.
+ * <p>
+ * This class is used only as part of a {@link org.apache.oozie.fluentjob.api.workflow.Workflow}, isn't
+ * to be used alone with Jobs API.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class ConfigurationEntry {
+    private final String name;
+    private final String value;
+    private final String description;
+
+    public ConfigurationEntry(final String name, final String description) {
+        this(name, description, null);
+    }
+
+    public ConfigurationEntry(final String name, final String value, final String description) {
+        this.name = name;
+        this.value = value;
+        this.description = description;
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public String getValue() {
+        return value;
+    }
+
+    public String getDescription() {
+        return description;
+    }
+}