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:47:00 UTC

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

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.");
+    }
+}