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/09/04 13:48:14 UTC

[2/2] oozie git commit: OOZIE-2877 Git action (clayb, andras.piros via pbacsko, gezapeti)

OOZIE-2877 Git action (clayb, andras.piros via pbacsko, gezapeti)


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

Branch: refs/heads/master
Commit: fe2da6e57d5ae48d50d5cac4780c887de1d46afe
Parents: 6323a8e
Author: Andras Piros <an...@cloudera.com>
Authored: Tue Sep 4 15:32:08 2018 +0200
Committer: Andras Piros <an...@cloudera.com>
Committed: Tue Sep 4 15:32:08 2018 +0200

----------------------------------------------------------------------
 client/src/main/resources/git-action-1.0.xsd    |  37 +++
 .../oozie/action/hadoop/GitActionExecutor.java  | 228 +++++++++++++++++++
 core/src/main/resources/oozie-default.xml       |   6 +-
 .../java/org/apache/oozie/test/XTestCase.java   |   1 +
 .../src/site/twiki/WorkflowFunctionalSpec.twiki | 115 ++++++++++
 examples/src/main/apps/git/job.properties       |  26 +++
 examples/src/main/apps/git/workflow.xml         |  34 +++
 .../org/apache/oozie/example/fluentjob/Git.java |  45 ++++
 fluent-job/fluent-job-api/pom.xml               |   1 +
 .../oozie/fluentjob/api/action/GitAction.java   | 101 ++++++++
 .../fluentjob/api/action/GitActionBuilder.java  | 168 ++++++++++++++
 .../mapping/DistcpConfigurationConverter.java   |   2 +-
 .../api/mapping/ExplicitNodeConverter.java      |  14 +-
 .../api/mapping/GitConfigurationConverter.java  |  82 +++++++
 .../api/mapping/GitPrepareConverter.java        |  97 ++++++++
 .../mapping/Hive2ConfigurationConverter.java    |   2 +-
 .../api/mapping/HiveConfigurationConverter.java |   2 +-
 .../InlineWorkflowConfigurationConverter.java   |   2 +-
 .../mapping/ShellConfigurationConverter.java    |   2 +-
 .../mapping/SparkConfigurationConverter.java    |   2 +-
 .../mapping/SqoopConfigurationConverter.java    |   2 +-
 .../api/serialization/WorkflowMarshaller.java   |   1 +
 .../src/main/resources/action_mappings.xml      |  45 ++++
 .../fluent-job-api/src/main/xjb/bindings.xml    |  13 +-
 .../jobs/client/minitest/TestGitAction.java     |  60 +++++
 pom.xml                                         |  14 +-
 release-log.txt                                 |   1 +
 sharelib/git/pom.xml                            | 172 ++++++++++++++
 .../org/apache/oozie/action/hadoop/GitMain.java | 197 ++++++++++++++++
 .../oozie/action/hadoop/GitOperations.java      | 154 +++++++++++++
 .../apache/oozie/action/hadoop/GitServer.java   | 155 +++++++++++++
 .../action/hadoop/TestGitActionExecutor.java    | 213 +++++++++++++++++
 .../apache/oozie/action/hadoop/TestGitMain.java | 119 ++++++++++
 .../TestIntegrationGitActionExecutor.java       | 133 +++++++++++
 sharelib/pom.xml                                |   1 +
 src/main/assemblies/sharelib.xml                |   4 +
 webapp/pom.xml                                  |   4 +
 37 files changed, 2243 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/oozie/blob/fe2da6e5/client/src/main/resources/git-action-1.0.xsd
----------------------------------------------------------------------
diff --git a/client/src/main/resources/git-action-1.0.xsd b/client/src/main/resources/git-action-1.0.xsd
new file mode 100644
index 0000000..d8738e8
--- /dev/null
+++ b/client/src/main/resources/git-action-1.0.xsd
@@ -0,0 +1,37 @@
+<?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.
+-->
+<xs:schema xmlns:xs="http://www.w3.org/2001/XMLSchema"
+           xmlns:git="uri:oozie:git-action:1.0"
+           elementFormDefault="qualified"
+           targetNamespace="uri:oozie:git-action:1.0">
+  <xs:include schemaLocation="oozie-common-1.0.xsd"/>
+  <xs:element name="git" type="git:ACTION"/>
+  <xs:complexType name="ACTION">
+    <xs:sequence>
+      <xs:element name="resource-manager" type="xs:string" minOccurs="0" maxOccurs="1"/>
+      <xs:element name="name-node" type="xs:string" minOccurs="1" maxOccurs="1"/>
+      <xs:element name="prepare" type="git:PREPARE" minOccurs="0" maxOccurs="1"/>
+      <xs:element name="git-uri" type="xs:string" minOccurs="1" maxOccurs="1"/>
+      <xs:element name="branch" type="xs:string" minOccurs="0" maxOccurs="1"/>
+      <xs:element name="key-path" type="xs:string" minOccurs="0" maxOccurs="1"/>
+      <xs:element name="destination-uri" type="xs:string" minOccurs="1" maxOccurs="1"/>
+      <xs:element name="configuration" type="git:CONFIGURATION" minOccurs="0" maxOccurs="1"/>
+    </xs:sequence>
+  </xs:complexType>
+</xs:schema>

http://git-wip-us.apache.org/repos/asf/oozie/blob/fe2da6e5/core/src/main/java/org/apache/oozie/action/hadoop/GitActionExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/oozie/action/hadoop/GitActionExecutor.java b/core/src/main/java/org/apache/oozie/action/hadoop/GitActionExecutor.java
new file mode 100644
index 0000000..fc4d65b
--- /dev/null
+++ b/core/src/main/java/org/apache/oozie/action/hadoop/GitActionExecutor.java
@@ -0,0 +1,228 @@
+/**
+ * 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.action.hadoop;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.net.ConnectException;
+import java.net.UnknownHostException;
+import java.net.URISyntaxException;
+import java.util.List;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.util.DiskChecker;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.AclEntryType;
+import org.apache.hadoop.security.AccessControlException;
+import org.apache.oozie.action.ActionExecutorException;
+import org.apache.oozie.action.ActionExecutorException.ErrorType;
+import org.apache.oozie.service.HadoopAccessorException;
+import org.apache.oozie.util.XLog;
+import org.jdom.Element;
+import org.jdom.JDOMException;
+import org.jdom.Namespace;
+
+public class GitActionExecutor extends JavaActionExecutor {
+
+    private static final String GIT_MAIN_CLASS_NAME =
+            "org.apache.oozie.action.hadoop.GitMain";
+    public static final String GIT_ACTION_TYPE = "git";
+
+    static final String APP_NAME = "oozie.app.name";
+    static final String WORKFLOW_ID = "oozie.workflow.id";
+    static final String CALLBACK_URL = "oozie.callback.url";
+    static final String RESOURCE_MANAGER = "oozie.resource.manager";
+    static final String NAME_NODE = "oozie.name.node";
+    static final String GIT_URI = "oozie.git.source.uri";
+    static final String GIT_BRANCH = "oozie.git.branch";
+    static final String DESTINATION_URI = "oozie.git.destination.uri";
+    static final String KEY_PATH = "oozie.git.key.path";
+    static final String ACTION_TYPE = "oozie.action.type";
+    static final String ACTION_NAME = "oozie.action.name";
+
+    public GitActionExecutor() {
+        super(GIT_ACTION_TYPE);
+    }
+
+    @Override
+    public List<Class<?>> getLauncherClasses() {
+        List<Class<?>> classes = new ArrayList<Class<?>>();
+        try {
+            classes.add(Class.forName(GIT_MAIN_CLASS_NAME));
+        }
+        catch (ClassNotFoundException e) {
+            throw new RuntimeException("Class not found", e);
+        }
+        return classes;
+    }
+
+    @Override
+    protected String getLauncherMain(Configuration launcherConf, Element actionXml) {
+        return launcherConf.get(LauncherAMUtils.CONF_OOZIE_ACTION_MAIN_CLASS,
+                GIT_MAIN_CLASS_NAME);
+    }
+
+    @Override
+    public void initActionType() {
+        super.initActionType();
+
+        registerErrors();
+    }
+
+    private void registerErrors() {
+        registerError(UnknownHostException.class.getName(), ErrorType.TRANSIENT, "GIT001");
+        registerError(AccessControlException.class.getName(), ErrorType.NON_TRANSIENT,
+                "GIT002");
+        registerError(DiskChecker.DiskOutOfSpaceException.class.getName(),
+                ErrorType.NON_TRANSIENT, "GIT003");
+        registerError(org.apache.hadoop.hdfs.protocol.QuotaExceededException.class.getName(),
+                ErrorType.NON_TRANSIENT, "GIT004");
+        registerError(org.apache.hadoop.hdfs.server.namenode.SafeModeException.class.getName(),
+                ErrorType.NON_TRANSIENT, "GIT005");
+        registerError(ConnectException.class.getName(), ErrorType.TRANSIENT, "GIT006");
+        registerError(JDOMException.class.getName(), ErrorType.ERROR, "GIT007");
+        registerError(FileNotFoundException.class.getName(), ErrorType.ERROR, "GIT008");
+        registerError(IOException.class.getName(), ErrorType.TRANSIENT, "GIT009");
+        registerError(NullPointerException.class.getName(), ErrorType.ERROR, "GIT010");
+    }
+
+    @Override
+    Configuration setupActionConf(Configuration actionConf, Context context,
+                                  Element actionXml, Path appPath) throws ActionExecutorException {
+        super.setupActionConf(actionConf, context, actionXml, appPath);
+
+        Namespace ns = actionXml.getNamespace();
+
+        ActionConfVerifier confChecker = new ActionConfVerifier(actionConf);
+
+        confChecker.checkTrimAndSet(GitActionExecutor.NAME_NODE, actionXml.getChild("name-node", ns));
+
+        confChecker.checkTrimAndSet(GitActionExecutor.DESTINATION_URI,
+                actionXml.getChild("destination-uri", ns));
+
+        confChecker.checkTrimAndSet(GitActionExecutor.GIT_URI, actionXml.getChild("git-uri", ns));
+
+        confChecker.trimAndSet(KEY_PATH, actionXml.getChild("key-path", ns));
+        String keyPath = actionConf.get(KEY_PATH);
+        if (keyPath != null && keyPath.length() > 0) {
+            try {
+                confChecker.verifyKeyPermissions(context.getAppFileSystem(), new Path(keyPath));
+            } catch (HadoopAccessorException|URISyntaxException|IOException e){
+                throw new ActionExecutorException(ActionExecutorException.ErrorType.ERROR, "GIT011", XLog
+                        .format("Not able to verify permissions on key file {0}", keyPath), e);
+            }
+        }
+
+        confChecker.trimAndSet(GIT_BRANCH, actionXml.getChild("branch", ns));
+
+        actionConf.set(ACTION_TYPE, getType());
+        actionConf.set(ACTION_NAME, GIT_ACTION_TYPE);
+
+        return actionConf;
+    }
+
+    @Override
+    protected String getDefaultShareLibName(Element actionXml) {
+        return GIT_ACTION_TYPE;
+    }
+
+    static class ActionConfVerifier {
+        private final Configuration actionConf;
+
+        /**
+         * Create ActionConfVerifier checker which will set action conf values and throw
+         * ActionExecutorException's with the exception code provided
+         *  @param  actionConf       the actionConf in which to set values
+         *
+         */
+        ActionConfVerifier(Configuration actionConf) {
+            this.actionConf = actionConf;
+        }
+
+        /**
+         * Validates Git key permissions are secure on disk and throw an exception if not.
+         * Otherwise exit out gracefully
+         */
+        void verifyKeyPermissions(FileSystem fs, Path keyPath) throws IOException, ActionExecutorException{
+            String failedPermsWarning = "The permissions on the access key {0} are considered insecure: {1}";
+            FileStatus status = fs.getFileStatus(keyPath);
+            FsPermission perms = status.getPermission();
+            // check standard permissioning for other's read access
+            if (perms.getOtherAction().and(FsAction.READ) == FsAction.READ) {
+                throw new ActionExecutorException(ActionExecutorException.ErrorType.ERROR, "GIT012", XLog
+                        .format(failedPermsWarning, keyPath, perms.toString()));
+            }
+            // check if any ACLs have been specified which allow others read access
+            if (perms.getAclBit()) {
+                List<AclEntry> aclEntries = new ArrayList<>(fs.getAclStatus(keyPath).getEntries());
+                for (AclEntry acl: aclEntries) {
+                    if (acl.getType() == AclEntryType.OTHER && acl.getPermission().and(FsAction.READ) == FsAction.READ) {
+                        throw new ActionExecutorException(ActionExecutorException.ErrorType.ERROR, "GIT013", XLog
+                                .format(failedPermsWarning, keyPath, perms.toString()));
+                    }
+                }
+            }
+        }
+
+        /**
+         * Calls helper function to verify value not null and throw an exception if so.
+         * Otherwise, set actionConf value displayName to XML trimmed text value
+         */
+        void checkTrimAndSet(String displayName, Element value) {
+            Preconditions.checkNotNull(value, "Action Configuration does not have [%s] property", displayName);
+            actionConf.set(displayName, value.getTextTrim());
+        }
+
+        /**
+        * Calls helper function to verify value not null and throw an exception if so.
+        * Otherwise, set actionConf value displayName to value
+        */
+        void checkAndSet(String displayName, String value) {
+            Preconditions.checkNotNull(value, "Action Configuration does not have [%s] property", displayName);
+            actionConf.set(displayName, value);
+        }
+
+        /**
+         * *f value is null, does nothing
+         * If value is not null, sets actionConf value displayName to XML trimmed text value
+         */
+        void trimAndSet(String displayName, Element value) {
+            if (value != null) {
+                actionConf.set(displayName, value.getTextTrim());
+            }
+        }
+
+        /**
+         * Calls helper function to verify name not null and throw an exception if so.
+         * Otherwise, returns actionConf value
+         * @param name - actionConf value to return
+         */
+        String checkAndGetTrimmed(String name) {
+            Preconditions.checkNotNull(actionConf.getTrimmed(name), "Action Configuration does not have [%s] property", name);
+            return actionConf.getTrimmed(name);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/fe2da6e5/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 b69d2c9..c354f02 100644
--- a/core/src/main/resources/oozie-default.xml
+++ b/core/src/main/resources/oozie-default.xml
@@ -1648,7 +1648,8 @@ will be the requeue interval for the actions which are waiting for a long time w
             distcp-action-0.1.xsd,distcp-action-0.2.xsd,distcp-action-1.0.xsd,
             oozie-sla-0.1.xsd,oozie-sla-0.2.xsd,
             hive2-action-0.1.xsd,hive2-action-0.2.xsd,hive2-action-1.0.xsd,
-            spark-action-0.1.xsd,spark-action-0.2.xsd,spark-action-1.0.xsd
+            spark-action-0.1.xsd,spark-action-0.2.xsd,spark-action-1.0.xsd,
+            git-action-1.0.xsd
         </value>
         <description>
             List of schemas for workflows (separated by commas).
@@ -1786,7 +1787,8 @@ will be the requeue interval for the actions which are waiting for a long time w
             org.apache.oozie.action.ssh.SshActionExecutor,
             org.apache.oozie.action.oozie.SubWorkflowActionExecutor,
             org.apache.oozie.action.email.EmailActionExecutor,
-            org.apache.oozie.action.hadoop.SparkActionExecutor
+            org.apache.oozie.action.hadoop.SparkActionExecutor,
+            org.apache.oozie.action.hadoop.GitActionExecutor
         </value>
         <description>
             List of ActionExecutors classes (separated by commas).

http://git-wip-us.apache.org/repos/asf/oozie/blob/fe2da6e5/core/src/test/java/org/apache/oozie/test/XTestCase.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/oozie/test/XTestCase.java b/core/src/test/java/org/apache/oozie/test/XTestCase.java
index 661970d..3048dda 100644
--- a/core/src/test/java/org/apache/oozie/test/XTestCase.java
+++ b/core/src/test/java/org/apache/oozie/test/XTestCase.java
@@ -1060,6 +1060,7 @@ public abstract class XTestCase extends TestCase {
       conf.set("dfs.block.access.token.enable", "false");
       conf.set("dfs.permissions", "true");
       conf.set("hadoop.security.authentication", "simple");
+      conf.setBoolean("dfs.namenode.acls.enabled", true);
 
       //Doing this because Hadoop 1.x does not support '*' if the value is '*,127.0.0.1'
       StringBuilder sb = new StringBuilder();

http://git-wip-us.apache.org/repos/asf/oozie/blob/fe2da6e5/docs/src/site/twiki/WorkflowFunctionalSpec.twiki
----------------------------------------------------------------------
diff --git a/docs/src/site/twiki/WorkflowFunctionalSpec.twiki b/docs/src/site/twiki/WorkflowFunctionalSpec.twiki
index 76cbe21..46a454c 100644
--- a/docs/src/site/twiki/WorkflowFunctionalSpec.twiki
+++ b/docs/src/site/twiki/WorkflowFunctionalSpec.twiki
@@ -1595,6 +1595,121 @@ the main-class element has priority.
 existing Main to see how it works before creating your own.  In fact, its probably simplest to just subclass the existing Main and
 add/modify/overwrite any behavior you want to change.
 
+#GitAction
+---++++ 3.2.7 Git action
+
+The =git= action allows one to clone a Git repository into HDFS. The supported options are =git-uri=, =branch=, =key-path=
+and =destination-uri=.
+
+The =git clone= action is executed asynchronously by one of the YARN containers assigned to run on the cluster. If an SSH key is
+specified it will be created on the file system in a YARN container's local directory, relying on YARN NodeManager to remove the
+file after the action has run.
+
+Path names specified in the =git= action should be able to be parameterized (templatized) using EL expressions,
+e.g. =${wf:user()}= . Path name should be specified as an absolute path. Each file path must specify the file system URI.
+
+*Syntax:*
+
+<verbatim>
+<workflow-app name="[WF-DEF-NAME]" xmlns="uri:oozie:workflow:1.0">
+    ...
+    <action name="[NODE-NAME]">
+        <git>
+            <git-uri>[SOURCE-URI]</git-uri>
+            ...
+            <branch>[BRANCH]</branch>
+            ...
+            <key-path>[HDFS-PATH]</key-path>
+            ...
+            <destination-uri>[HDFS-PATH]</destination-uri>
+        </git>
+        <ok to="[NODE-NAME]"/>
+        <error to="[NODE-NAME]"/>
+    </action>
+    ...
+</workflow-app>
+</verbatim>
+
+*Example:*
+
+<verbatim>
+<workflow-app name="sample-wf" xmlns="uri:oozie:workflow:0.1">
+    ...
+    <action name="clone_oozie">
+        <git>
+            <git-uri>https://github.com/apache/oozie</git-uri>
+            <destination-uri>hdfs://my_git_repo_directory</destination-uri>
+        </git>
+        <ok to="myotherjob"/>
+        <error to="errorcleanup"/>
+    </action>
+    ...
+</workflow-app>
+</verbatim>
+
+In the above example, a Git repository on e.g. GitHub.com is cloned to the HDFS directory =my_git_repo_directory= which should not
+exist previously on the filesystem. Note that repository addresses outside of GitHub.com but accessible to the YARN container
+running the Git action may also be used.
+
+If a =name-node= element is specified, then it is not necessary for any of the paths to start with the file system URI as it is
+taken from the =name-node= element.
+
+The =resource-manager= (Oozie 5.x) element has to be specified to name the YARN ResourceManager address.
+
+If any of the paths need to be served from another HDFS namenode, its address has to be part of
+that filesystem URI prefix:
+<verbatim>
+<workflow-app name="[WF-DEF-NAME]" xmlns="uri:oozie:workflow:1.0">
+    ...
+    <action name="[NODE-NAME]">
+        <git>
+            ...
+            <name-node>hdfs://name-node.first.company.com:8020</name-node>
+            ...
+            <key-path>hdfs://name-node.second.company.com:8020/[HDFS-PATH]</key-path>
+            ...
+        </git>
+        ...
+    </action>
+    ...
+</workflow-app>
+</verbatim>
+
+This is also true if the name-node is specified in the global section (see
+[[WorkflowFunctionalSpec#GlobalConfigurations][Global Configurations]]).
+
+Be aware that =key-path= might point to a secure object store location other than the current =fs.defaultFS=. In that case,
+appropriate file permissions are still necessary (readable by submitting user), credentials provided, etc.
+
+As of workflow schema 1.0, zero or more =job-xml= elements can be specified; these must refer to Hadoop JobConf =job.xml= formatted
+files bundled in the workflow application. They can be used to set additional properties for the =FileSystem= instance.
+
+As of schema workflow schema 1.0, if a =configuration= element is specified, then it will also be used to set additional =JobConf=
+properties for the =FileSystem= instance. Properties specified in the =configuration= element are overridden by properties
+specified in the files specified by any =job-xml= elements.
+
+*Example:*
+
+<workflow-app name="[WF-DEF-NAME]" xmlns="uri:oozie:workflow:1.0">
+    ...
+    <action name="[NODE-NAME]">
+        <git>
+            ...
+            <name-node>hdfs://foo:8020</name-node>
+            <job-xml>fs-info.xml</job-xml>
+            <configuration>
+                <property>
+                    <name>some.property</name>
+                    <value>some.value</value>
+                </property>
+            </configuration>
+        </git>
+        ...
+    </action>
+    ...
+</workflow>
+</verbatim>
+
 #WorkflowParameterization
 ---++ 4 Parameterization of Workflows
 

http://git-wip-us.apache.org/repos/asf/oozie/blob/fe2da6e5/examples/src/main/apps/git/job.properties
----------------------------------------------------------------------
diff --git a/examples/src/main/apps/git/job.properties b/examples/src/main/apps/git/job.properties
new file mode 100644
index 0000000..f4dcf81
--- /dev/null
+++ b/examples/src/main/apps/git/job.properties
@@ -0,0 +1,26 @@
+#
+# 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.
+#
+
+nameNode=hdfs://localhost:8020
+resourceManager=localhost:8032
+queueName=default
+examplesRoot=examples
+
+oozie.use.system.libpath=true
+
+oozie.wf.application.path=${nameNode}/user/${user.name}/${examplesRoot}/apps/git

http://git-wip-us.apache.org/repos/asf/oozie/blob/fe2da6e5/examples/src/main/apps/git/workflow.xml
----------------------------------------------------------------------
diff --git a/examples/src/main/apps/git/workflow.xml b/examples/src/main/apps/git/workflow.xml
new file mode 100644
index 0000000..b1b0858
--- /dev/null
+++ b/examples/src/main/apps/git/workflow.xml
@@ -0,0 +1,34 @@
+<!--
+  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="git-wf">
+    <start to="git-node"/>
+    <action name="git-node">
+        <git xmlns="uri:oozie:git-action:0.1">
+            <resource-manager>${resourceManager}</resource-manager>
+            <name-node>${nameNode}</name-node>
+            <git-uri>https://github.com/apache/oozie</git-uri>
+            <destination-uri>/user/${wf:user()}/${examplesRoot}/output-data/git/oozie</destination-uri>
+        </git>
+        <ok to="end"/>
+        <error to="fail"/>
+    </action>
+    <kill name="fail">
+        <message>Git failed, error message[${wf:errorMessage(wf:lastErrorNode())}]</message>
+    </kill>
+    <end name="end"/>
+</workflow-app>

http://git-wip-us.apache.org/repos/asf/oozie/blob/fe2da6e5/examples/src/main/java/org/apache/oozie/example/fluentjob/Git.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/oozie/example/fluentjob/Git.java b/examples/src/main/java/org/apache/oozie/example/fluentjob/Git.java
new file mode 100644
index 0000000..2548d39
--- /dev/null
+++ b/examples/src/main/java/org/apache/oozie/example/fluentjob/Git.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.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/git/workflow.xml}.
+ */
+public class Git implements WorkflowFactory {
+    @Override
+    public Workflow create() {
+        final GitAction parent = GitActionBuilder.create()
+                .withResourceManager("${resourceManager}")
+                .withNameNode("${nameNode}")
+                .withDestinationUri("/user/${wf:user()}/${examplesRoot}/output-data/git/oozie")
+                .withGitUri("https://github.com/apache/oozie")
+                .build();
+
+        final Workflow workflow = new WorkflowBuilder()
+                .withName("git-example")
+                .withDagContainingNode(parent).build();
+
+        return workflow;
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/fe2da6e5/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
index cefd43c..a41ca25 100644
--- a/fluent-job/fluent-job-api/pom.xml
+++ b/fluent-job/fluent-job-api/pom.xml
@@ -141,6 +141,7 @@
                         <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>
+                        <source>../../client/src/main/resources/git-action-1.0.xsd</source>
                     </sources>
                 </configuration>
 

http://git-wip-us.apache.org/repos/asf/oozie/blob/fe2da6e5/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/GitAction.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/GitAction.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/GitAction.java
new file mode 100644
index 0000000..18dd7e0
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/GitAction.java
@@ -0,0 +1,101 @@
+/**
+ * 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 Git action.
+ * Instances of this class should be built using the builder {@link GitActionBuilder}.
+ *
+ * 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 GitActionBuilder#build} either.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class GitAction extends Node implements HasAttributes {
+    protected final ActionAttributes attributes;
+    protected final String gitUri;
+    protected final String branch;
+    protected final String keyPath;
+    protected final String destinationUri;
+
+    GitAction(final ConstructionData constructionData,
+              final ActionAttributes attributes,
+              final String gitUri,
+              final String branch,
+              final String keyPath,
+              final String destinationUri) {
+        super(constructionData);
+
+        this.attributes = attributes;
+        this.gitUri = gitUri;
+        this.branch = branch;
+        this.keyPath = keyPath;
+        this.destinationUri = destinationUri;
+    }
+
+    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 getGitUri() {
+        return gitUri;
+    }
+
+    public String getBranch() {
+        return branch;
+    }
+
+    public String getKeyPath() {
+        return keyPath;
+    }
+
+    public String getDestinationUri() {
+        return destinationUri;
+    }
+
+    public ActionAttributes getAttributes() {
+        return attributes;
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/fe2da6e5/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/GitActionBuilder.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/GitActionBuilder.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/GitActionBuilder.java
new file mode 100644
index 0000000..3222723
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/action/GitActionBuilder.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 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 GitAction}.
+ *
+ * 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 GitActionBuilder#build} either.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class GitActionBuilder extends NodeBuilderBaseImpl<GitActionBuilder> implements Builder<GitAction> {
+    private final ActionAttributesBuilder attributesBuilder;
+    private final ModifyOnce<String> gitUri;
+    private final ModifyOnce<String> branch;
+    private final ModifyOnce<String> keyPath;
+    private final ModifyOnce<String> destinationUri;
+
+    public static GitActionBuilder create() {
+        final ActionAttributesBuilder builder = ActionAttributesBuilder.create();
+        final ModifyOnce<String> gitUri = new ModifyOnce<>();
+        final ModifyOnce<String> branch = new ModifyOnce<>();
+        final ModifyOnce<String> keyPath = new ModifyOnce<>();
+        final ModifyOnce<String> destinationUri = new ModifyOnce<>();
+
+        return new GitActionBuilder(
+                null,
+                builder,
+                gitUri,
+                branch,
+                keyPath,
+                destinationUri);
+    }
+
+    public static GitActionBuilder createFromExistingAction(final GitAction action) {
+        final ActionAttributesBuilder builder = ActionAttributesBuilder.createFromExisting(action.getAttributes());
+        final ModifyOnce<String> gitUri = new ModifyOnce<>(action.getGitUri());
+        final ModifyOnce<String> branch = new ModifyOnce<>(action.getBranch());
+        final ModifyOnce<String> keyPath = new ModifyOnce<>(action.getKeyPath());
+        final ModifyOnce<String> destinationUri = new ModifyOnce<>(action.getDestinationUri());
+
+        return new GitActionBuilder(
+                action,
+                builder,
+                gitUri,
+                branch,
+                keyPath,
+                destinationUri);
+    }
+
+    public static GitActionBuilder createFromExistingAction(final Node action) {
+        final ActionAttributesBuilder builder = ActionAttributesBuilder.createFromAction(action);
+        final ModifyOnce<String> gitUri = new ModifyOnce<>();
+        final ModifyOnce<String> branch = new ModifyOnce<>();
+        final ModifyOnce<String> keyPath = new ModifyOnce<>();
+        final ModifyOnce<String> destinationUri = new ModifyOnce<>();
+
+        return new GitActionBuilder(
+                action,
+                builder,
+                gitUri,
+                branch,
+                keyPath,
+                destinationUri);
+    }
+
+    private GitActionBuilder(final Node action,
+                     final ActionAttributesBuilder attributesBuilder,
+                     final ModifyOnce<String> gitUri,
+                     final ModifyOnce<String> branch,
+                     final ModifyOnce<String> keyPath,
+                     final ModifyOnce<String> destinationUri) {
+        super(action);
+
+        this.attributesBuilder = attributesBuilder;
+        this.gitUri = gitUri;
+        this.branch = branch;
+        this.keyPath = keyPath;
+        this.destinationUri = destinationUri;
+    }
+
+    public GitActionBuilder withResourceManager(final String resourceManager) {
+        this.attributesBuilder.withResourceManager(resourceManager);
+        return this;
+    }
+
+    public GitActionBuilder withNameNode(final String nameNode) {
+        this.attributesBuilder.withNameNode(nameNode);
+        return this;
+    }
+
+    public GitActionBuilder withPrepare(final Prepare prepare) {
+        this.attributesBuilder.withPrepare(prepare);
+        return this;
+    }
+
+    public GitActionBuilder withConfigProperty(final String key, final String value) {
+        this.attributesBuilder.withConfigProperty(key, value);
+        return this;
+    }
+
+    public GitActionBuilder withGitUri(final String gitUri) {
+        this.gitUri.set(gitUri);
+        return this;
+    }
+
+    public GitActionBuilder withBranch(final String branch) {
+        this.branch.set(branch);
+        return this;
+    }
+
+    public GitActionBuilder withKeyPath(final String keyPath) {
+        this.keyPath.set(keyPath);
+        return this;
+    }
+
+    public GitActionBuilder withDestinationUri(final String destinationUri) {
+        this.destinationUri.set(destinationUri);
+        return this;
+    }
+
+    @Override
+    public GitAction build() {
+        final Node.ConstructionData constructionData = getConstructionData();
+
+        final GitAction instance = new GitAction(
+                constructionData,
+                attributesBuilder.build(),
+                gitUri.get(),
+                branch.get(),
+                keyPath.get(),
+                destinationUri.get());
+
+        addAsChildToAllParents(instance);
+
+        return instance;
+    }
+
+    @Override
+    protected GitActionBuilder getRuntimeSelfReference() {
+        return this;
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/fe2da6e5/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
index fffb734..568c099 100644
--- 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
@@ -38,7 +38,7 @@ public class DistcpConfigurationConverter extends DozerConverter<Map, CONFIGURAT
 
     @Override
     public CONFIGURATION convertTo(final Map source, CONFIGURATION destination) {
-        if (source == null) {
+        if (source == null || source.isEmpty()) {
             return null;
         }
 

http://git-wip-us.apache.org/repos/asf/oozie/blob/fe2da6e5/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
index 7bb82e5..e5f8e07 100644
--- 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
@@ -23,6 +23,7 @@ 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.GitAction;
 import org.apache.oozie.fluentjob.api.action.Hive2Action;
 import org.apache.oozie.fluentjob.api.action.HiveAction;
 import org.apache.oozie.fluentjob.api.action.JavaAction;
@@ -75,6 +76,7 @@ public class ExplicitNodeConverter extends DozerConverter<ExplicitNode, ACTION>
                 .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(GitAction.class, org.apache.oozie.fluentjob.api.generated.action.git.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)
@@ -205,6 +207,9 @@ public class ExplicitNodeConverter extends DozerConverter<ExplicitNode, ACTION>
         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.git.ACTION) {
+            setGit((org.apache.oozie.fluentjob.api.generated.action.git.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);
         }
@@ -238,6 +243,13 @@ public class ExplicitNodeConverter extends DozerConverter<ExplicitNode, ACTION>
         destination.setOther(jaxbElement);
     }
 
+    private void setGit(final org.apache.oozie.fluentjob.api.generated.action.git.ACTION source, final ACTION destination) {
+        final JAXBElement<org.apache.oozie.fluentjob.api.generated.action.git.ACTION> jaxbElement =
+                new org.apache.oozie.fluentjob.api.generated.action.git.ObjectFactory().createGit(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);
@@ -293,4 +305,4 @@ public class ExplicitNodeConverter extends DozerConverter<ExplicitNode, ACTION>
 
         destination.setOther(jaxbElement);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/fe2da6e5/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/GitConfigurationConverter.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/GitConfigurationConverter.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/GitConfigurationConverter.java
new file mode 100644
index 0000000..a2f2f86
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/GitConfigurationConverter.java
@@ -0,0 +1,82 @@
+/**
+ * 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.git.CONFIGURATION;
+import org.apache.oozie.fluentjob.api.generated.action.git.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}.
+ */
+@SuppressWarnings("unchecked")
+public class GitConfigurationConverter extends DozerConverter<Map, CONFIGURATION> {
+    private static final ObjectFactory OBJECT_FACTORY = new ObjectFactory();
+
+    public GitConfigurationConverter() {
+        super(Map.class, CONFIGURATION.class);
+    }
+
+    @Override
+    public CONFIGURATION convertTo(final Map source, CONFIGURATION destination) {
+        if (source == null || source.isEmpty()) {
+            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/fe2da6e5/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/GitPrepareConverter.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/GitPrepareConverter.java b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/GitPrepareConverter.java
new file mode 100644
index 0000000..c6e270e
--- /dev/null
+++ b/fluent-job/fluent-job-api/src/main/java/org/apache/oozie/fluentjob/api/mapping/GitPrepareConverter.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.git.DELETE;
+import org.apache.oozie.fluentjob.api.generated.action.git.MKDIR;
+import org.apache.oozie.fluentjob.api.generated.action.git.ObjectFactory;
+import org.apache.oozie.fluentjob.api.generated.action.git.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 GitPrepareConverter extends DozerConverter<Prepare, PREPARE> {
+    private static final ObjectFactory OBJECT_FACTORY = new ObjectFactory();
+
+    public GitPrepareConverter() {
+        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/fe2da6e5/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
index c67b5ae..9ac343a 100644
--- 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
@@ -38,7 +38,7 @@ public class Hive2ConfigurationConverter extends DozerConverter<Map, CONFIGURATI
 
     @Override
     public CONFIGURATION convertTo(final Map source, CONFIGURATION destination) {
-        if (source == null) {
+        if (source == null || source.isEmpty()) {
             return null;
         }
 

http://git-wip-us.apache.org/repos/asf/oozie/blob/fe2da6e5/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
index 5f9a2b1..60f3654 100644
--- 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
@@ -38,7 +38,7 @@ public class HiveConfigurationConverter extends DozerConverter<Map, CONFIGURATIO
 
     @Override
     public CONFIGURATION convertTo(final Map source, CONFIGURATION destination) {
-        if (source == null) {
+        if (source == null || source.isEmpty()) {
             return null;
         }
 

http://git-wip-us.apache.org/repos/asf/oozie/blob/fe2da6e5/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
index b1e17c9..759dcdc 100644
--- 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
@@ -38,7 +38,7 @@ public class InlineWorkflowConfigurationConverter extends DozerConverter<Map, CO
 
     @Override
     public CONFIGURATION convertTo(final Map source, CONFIGURATION destination) {
-        if (source == null) {
+        if (source == null || source.isEmpty()) {
             return null;
         }
 

http://git-wip-us.apache.org/repos/asf/oozie/blob/fe2da6e5/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
index ab73dfd..0d5eae3 100644
--- 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
@@ -38,7 +38,7 @@ public class ShellConfigurationConverter extends DozerConverter<Map, CONFIGURATI
 
     @Override
     public CONFIGURATION convertTo(final Map source, CONFIGURATION destination) {
-        if (source == null) {
+        if (source == null || source.isEmpty()) {
             return null;
         }
 

http://git-wip-us.apache.org/repos/asf/oozie/blob/fe2da6e5/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
index 8827769..9976039 100644
--- 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
@@ -38,7 +38,7 @@ public class SparkConfigurationConverter extends DozerConverter<Map, CONFIGURATI
 
     @Override
     public CONFIGURATION convertTo(final Map source, CONFIGURATION destination) {
-        if (source == null) {
+        if (source == null || source.isEmpty()) {
             return null;
         }
 

http://git-wip-us.apache.org/repos/asf/oozie/blob/fe2da6e5/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
index 1d4f615..05dcf12 100644
--- 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
@@ -38,7 +38,7 @@ public class SqoopConfigurationConverter extends DozerConverter<Map, CONFIGURATI
 
     @Override
     public CONFIGURATION convertTo(final Map source, CONFIGURATION destination) {
-        if (source == null) {
+        if (source == null || source.isEmpty()) {
             return null;
         }
 

http://git-wip-us.apache.org/repos/asf/oozie/blob/fe2da6e5/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
index ec56554..69cb3e3 100644
--- 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
@@ -67,6 +67,7 @@ public class WorkflowMarshaller {
         filteredPackages.append("org.apache.oozie.fluentjob.api.generated.workflow");
         appendIfPresent(workflow, filteredPackages, "distcp");
         appendIfPresent(workflow, filteredPackages, "email");
+        appendIfPresent(workflow, filteredPackages, "git");
         appendIfPresent(workflow, filteredPackages, "hive2");
         appendIfPresent(workflow, filteredPackages, "hive");
         appendIfPresent(workflow, filteredPackages, "sla");

http://git-wip-us.apache.org/repos/asf/oozie/blob/fe2da6e5/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
index a5f890e..8cf3a0f 100644
--- a/fluent-job/fluent-job-api/src/main/resources/action_mappings.xml
+++ b/fluent-job/fluent-job-api/src/main/resources/action_mappings.xml
@@ -818,4 +818,49 @@
             <b>configuration</b>
         </field>
     </mapping>
+
+    <mapping type="one-way">
+        <class-a>org.apache.oozie.fluentjob.api.action.GitAction</class-a>
+        <class-b>org.apache.oozie.fluentjob.api.generated.action.git.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.GitPrepareConverter">
+            <a>prepare</a>
+            <b>prepare</b>
+        </field>
+
+        <field custom-converter="org.apache.oozie.fluentjob.api.mapping.GitConfigurationConverter">
+            <a>configuration</a>
+            <b>configuration</b>
+        </field>
+
+        <field>
+            <a>gitUri</a>
+            <b>gitUri</b>
+        </field>
+
+        <field>
+            <a>branch</a>
+            <b>branch</b>
+        </field>
+
+        <field>
+            <a>keyPath</a>
+            <b>keyPath</b>
+        </field>
+
+        <field>
+            <a>destinationUri</a>
+            <b>destinationUri</b>
+        </field>
+    </mapping>
 </mappings>

http://git-wip-us.apache.org/repos/asf/oozie/blob/fe2da6e5/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
index 48f6890..ca6fd13 100644
--- a/fluent-job/fluent-job-api/src/main/xjb/bindings.xml
+++ b/fluent-job/fluent-job-api/src/main/xjb/bindings.xml
@@ -163,4 +163,15 @@
 
     </jaxb:bindings>
 
-</jaxb:bindings>
\ No newline at end of file
+    <jaxb:bindings schemaLocation = "../../../../../client/src/main/resources/git-action-1.0.xsd">
+
+        <jaxb:schemaBindings>
+            <jaxb:package name = "org.apache.oozie.fluentjob.api.generated.action.git"/>
+        </jaxb:schemaBindings>
+
+        <jaxb:bindings>
+            <namespace:prefix name="git" />
+        </jaxb:bindings>
+
+    </jaxb:bindings>
+</jaxb:bindings>

http://git-wip-us.apache.org/repos/asf/oozie/blob/fe2da6e5/fluent-job/fluent-job-client/src/test/java/org/apache/oozie/jobs/client/minitest/TestGitAction.java
----------------------------------------------------------------------
diff --git a/fluent-job/fluent-job-client/src/test/java/org/apache/oozie/jobs/client/minitest/TestGitAction.java b/fluent-job/fluent-job-client/src/test/java/org/apache/oozie/jobs/client/minitest/TestGitAction.java
new file mode 100644
index 0000000..315ca0e
--- /dev/null
+++ b/fluent-job/fluent-job-client/src/test/java/org/apache/oozie/jobs/client/minitest/TestGitAction.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.jobs.client.minitest;
+
+import org.apache.oozie.client.OozieClientException;
+import org.apache.oozie.fluentjob.api.GraphVisualization;
+import org.apache.oozie.fluentjob.api.action.*;
+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 TestGitAction extends WorkflowTestCase {
+    public void testGitAction() throws IOException, JAXBException, OozieClientException {
+        final GitAction parent = GitActionBuilder.create()
+                .withResourceManager(getJobTrackerUri())
+                .withNameNode(getNameNodeUri())
+                .withDestinationUri("/user/${wf:user()}/${examplesRoot}/output-data/git/oozie")
+                .withGitUri("https://github.com/apache/oozie")
+                .build();
+
+        final Workflow workflow = new WorkflowBuilder()
+                .withName("simple-git-example")
+                .withDagContainingNode(parent).build();
+
+        final String xml = WorkflowMarshaller.marshal(workflow);
+
+        System.out.println(xml);
+
+        GraphVisualization.workflowToPng(workflow, "simple-git-example-workflow.png");
+
+        final Graph intermediateGraph = new Graph(workflow);
+
+        GraphVisualization.graphToPng(intermediateGraph, "simple-git-example-graph.png");
+
+        log.debug("Workflow XML is:\n{0}", xml);
+
+        validate(xml);
+    }
+}

http://git-wip-us.apache.org/repos/asf/oozie/blob/fe2da6e5/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index e0dbe85..423d19d 100644
--- a/pom.xml
+++ b/pom.xml
@@ -118,6 +118,7 @@
          <httpclient.version>4.3.6</httpclient.version>
          <kyro.version>2.22</kyro.version>
          <javax.servlet.api.version>3.0.1</javax.servlet.api.version>
+         <jgit.version>5.0.1.201806211838-r</jgit.version>
     </properties>
 
     <modules>
@@ -253,6 +254,11 @@
             </dependency>
             <dependency>
                 <groupId>org.apache.oozie</groupId>
+                <artifactId>oozie-sharelib-git</artifactId>
+                <version>${project.version}</version>
+            </dependency>
+            <dependency>
+                <groupId>org.apache.oozie</groupId>
                 <artifactId>oozie-docs</artifactId>
                 <version>${project.version}</version>
                 <type>war</type>
@@ -1589,7 +1595,13 @@
             <dependency>
                 <groupId>org.eclipse.jgit</groupId>
                 <artifactId>org.eclipse.jgit</artifactId>
-                <version>3.3.1.201403241930-r</version>
+                <version>${jgit.version}</version>
+            </dependency>
+
+            <dependency>
+                <groupId>org.eclipse.jgit</groupId>
+                <artifactId>org.eclipse.jgit.http.server</artifactId>
+                <version>${jgit.version}</version>
             </dependency>
 
             <dependency>

http://git-wip-us.apache.org/repos/asf/oozie/blob/fe2da6e5/release-log.txt
----------------------------------------------------------------------
diff --git a/release-log.txt b/release-log.txt
index caacb1f..78f25e7 100644
--- a/release-log.txt
+++ b/release-log.txt
@@ -1,5 +1,6 @@
 -- Oozie 5.1.0 release (trunk - unreleased)
 
+OOZIE-2877 Git action (clayb, andras.piros via pbacsko, gezapeti)
 OOZIE-3061 Kill only those child jobs which are not already killed (matijhs via gezapeti, andras.piros)
 OOZIE-3155 [ui] Job DAG is not refreshed when a job is finished (asalamon74 via andras.piros)
 OOZIE-3334 Don't use org.apache.hadoop.hbase.security.User in HDFSCredentials (gezapeti)

http://git-wip-us.apache.org/repos/asf/oozie/blob/fe2da6e5/sharelib/git/pom.xml
----------------------------------------------------------------------
diff --git a/sharelib/git/pom.xml b/sharelib/git/pom.xml
new file mode 100644
index 0000000..85234ac
--- /dev/null
+++ b/sharelib/git/pom.xml
@@ -0,0 +1,172 @@
+<?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-main</artifactId>
+        <version>5.1.0-SNAPSHOT</version>
+        <relativePath>../..</relativePath>
+    </parent>
+    <groupId>org.apache.oozie</groupId>
+    <artifactId>oozie-sharelib-git</artifactId>
+    <version>5.1.0-SNAPSHOT</version>
+    <description>Apache Oozie Share Lib Git</description>
+    <name>Apache Oozie Share Lib Git</name>
+    <packaging>jar</packaging>
+
+    <properties>
+        <sharelib.action.postfix>git</sharelib.action.postfix>
+        <sharelib.transitive.filtering>true</sharelib.transitive.filtering>
+    </properties>
+
+    <dependencies>
+        <dependency>
+            <groupId>junit</groupId>
+            <artifactId>junit</artifactId>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.oozie</groupId>
+            <artifactId>oozie-core</artifactId>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.oozie</groupId>
+            <artifactId>oozie-core</artifactId>
+            <classifier>tests</classifier>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.oozie</groupId>
+            <artifactId>oozie-client</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-client</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.eclipse.jgit</groupId>
+            <artifactId>org.eclipse.jgit</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.eclipse.jgit</groupId>
+            <artifactId>org.eclipse.jgit.http.server</artifactId>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-minicluster</artifactId>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.commons</groupId>
+            <artifactId>commons-lang3</artifactId>
+        </dependency>
+
+        <dependency>
+            <groupId>com.google.code.findbugs</groupId>
+            <artifactId>annotations</artifactId>
+            <scope>provided</scope>
+        </dependency>
+    </dependencies>
+
+    <build>
+        <resources>
+            <resource>
+                <directory>src/main/resources</directory>
+                <filtering>true</filtering>
+            </resource>
+        </resources>
+        <plugins>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-dependency-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <id>gen-classpath</id>
+                        <phase>generate-test-resources</phase>
+                        <goals>
+                            <goal>build-classpath</goal>
+                        </goals>
+                        <configuration>
+                            <includeScope>compile</includeScope>
+                            <outputFile>${project.build.directory}/classpath</outputFile>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-antrun-plugin</artifactId>
+                <version>1.6</version>
+                <executions>
+                    <execution>
+                        <configuration>
+                            <target>
+                                <!-- needed to include Main class in classpath for mini yarn cluster for unit tests -->
+                                <echo file="${project.build.directory}/test-classes/mrapp-generated-classpath"
+                                      append="true"
+                                      message=":${project.build.directory}/classes"/>
+                            </target>
+                        </configuration>
+                        <goals>
+                            <goal>run</goal>
+                        </goals>
+                        <phase>generate-test-resources</phase>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-deploy-plugin</artifactId>
+                <configuration>
+                    <skip>true</skip>
+                </configuration>
+            </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-assembly-plugin</artifactId>
+                <configuration>
+                    <finalName>partial-sharelib</finalName>
+                    <appendAssemblyId>false</appendAssemblyId>
+                    <descriptors>
+                        <descriptor>../../src/main/assemblies/partial-sharelib.xml</descriptor>
+                    </descriptors>
+                </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>

http://git-wip-us.apache.org/repos/asf/oozie/blob/fe2da6e5/sharelib/git/src/main/java/org/apache/oozie/action/hadoop/GitMain.java
----------------------------------------------------------------------
diff --git a/sharelib/git/src/main/java/org/apache/oozie/action/hadoop/GitMain.java b/sharelib/git/src/main/java/org/apache/oozie/action/hadoop/GitMain.java
new file mode 100644
index 0000000..8c0e62c
--- /dev/null
+++ b/sharelib/git/src/main/java/org/apache/oozie/action/hadoop/GitMain.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.action.hadoop;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.nio.file.attribute.PosixFilePermissions;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+
+import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+import org.apache.oozie.action.hadoop.GitOperations.GitOperationsException;
+
+import com.google.common.annotations.VisibleForTesting;
+
+public class GitMain extends LauncherMain {
+
+    private static final String OOZIE_ACTION_CONF = "oozie.action.conf.xml";
+
+    private String nameNode;
+    private String keyPath;
+    private String destinationUri;
+    private String gitUri;
+    private String gitBranch;
+    public static void main(final String[] args) throws Exception {
+        run(GitMain.class, args);
+    }
+
+    @VisibleForTesting
+    void setNameNode(final String nameNode) {
+        this.nameNode = nameNode;
+    }
+
+    @Override
+    protected void run(final String[] args) throws Exception {
+        System.out.println("=============================================");
+        System.out.println("Oozie Git Action Configuration");
+        System.out.println("=============================================");
+
+        final Configuration actionConf = prepareActionConf();
+        parseActionConfiguration(actionConf);
+        final File localKey = getLocalKeyFile();
+        final GitOperations gitRepo = new GitOperations(new URI(gitUri), gitBranch, localKey);
+
+        try {
+            gitRepo.cloneRepoToFS(new Path(destinationUri));
+        }
+        catch (final IOException | GitOperationsException e){
+            System.err.println(e.getMessage());
+            throw e;
+        }
+    }
+
+    @SuppressFBWarnings(value = "PATH_TRAVERSAL_IN", justification = "File created without user input")
+    private Configuration prepareActionConf() {
+        final Configuration actionConf = new Configuration(false);
+
+        final String actionXml = System.getProperty(OOZIE_ACTION_CONF);
+        if (actionXml == null) {
+            throw new RuntimeException(
+                    "Missing Java System Property [" + OOZIE_ACTION_CONF + "]");
+        }
+        if (!new File(actionXml).exists()) {
+            throw new RuntimeException("Action Configuration XML file ["
+                    + actionXml + "] does not exist");
+        }
+
+        actionConf.addResource(new Path("file:///", actionXml));
+        return actionConf;
+    }
+
+    private File getLocalKeyFile() throws IOException, URISyntaxException {
+        File localKey = null;
+
+        if (keyPath != null) {
+            localKey = getKeyFromFS(new Path(keyPath));
+        }
+
+        return localKey;
+    }
+
+    /**
+     * Gathers the Git authentication key from a FileSystem and copies it to a local
+     * filesystem location
+     *
+     * @param location where the key is located (an HDFS URI)
+     * @return the location to where the key was saved
+     */
+    @VisibleForTesting
+    @SuppressFBWarnings(value = "PATH_TRAVERSAL_IN", justification = "File created without user input")
+    File getKeyFromFS(final Path location) throws IOException, URISyntaxException {
+        final String keyCopyMsg = "Copied keys to local container!";
+
+        final Configuration conf = new Configuration();
+        final FileSystem fs = FileSystem.newInstance(new URI(nameNode), conf);
+
+        final File key = createTempDir("git");
+
+        fs.copyToLocalFile(location, new Path("file:///" +
+            key.getAbsolutePath() + "/privkey"));
+        System.out.println(keyCopyMsg);
+
+        return new File(key.getAbsolutePath() + "/privkey");
+    }
+
+    /**
+     * Create a local temporary directory
+     *
+     * @param prefix string to use as a prefix to the directory
+     * @return file path of temp. directory (will be set to delete on exit)
+     */
+    @SuppressFBWarnings(value = "PATH_TRAVERSAL_IN", justification = "File created without user input")
+    static File createTempDir(final String prefix) throws IOException {
+        final File tempDir = new File(Files.createTempDirectory(
+            Paths.get("."),
+            prefix + "_" + Long.toString(System.nanoTime()),
+            PosixFilePermissions
+                .asFileAttribute(PosixFilePermissions
+                .fromString("rwx------")))
+            .toString());
+        tempDir.deleteOnExit();
+
+        final String localMkdirMsg = "Local mkdir called creating temp. dir at: " + tempDir.getAbsolutePath();
+        System.out.println(localMkdirMsg);
+
+        return tempDir;
+    }
+
+    /**
+     * Validate a URI is well formed and has a scheme
+     *
+     * @param testUri URI string to test
+     * @return URI from string
+     * @throws OozieActionConfiguratorException if the <code>testUri</code> fails any validity checks
+     */
+    private URI isValidUri(final String testUri) throws OozieActionConfiguratorException {
+        final URI uri;
+        try {
+            uri = new URI(testUri);
+        } catch (final URISyntaxException e) {
+            throw new OozieActionConfiguratorException("Action Configuration does not have "
+                    + "a proper URI: " + testUri + " exception "
+                    + e.toString());
+        }
+        if (uri.getScheme() == null) {
+            throw new OozieActionConfiguratorException("Action Configuration does not have "
+                    + "a proper URI " + testUri + " null scheme.");
+        }
+        return uri;
+    }
+
+    /**
+     * Parse action configuration and set configuration variables
+     *
+     * @param actionConf Oozie action configuration
+     * @throws OozieActionConfiguratorException upon any required properties missing
+     */
+    private void parseActionConfiguration(final Configuration actionConf) throws OozieActionConfiguratorException {
+        final GitActionExecutor.ActionConfVerifier confChecker = new GitActionExecutor.ActionConfVerifier(actionConf);
+
+        nameNode = confChecker.checkAndGetTrimmed(GitActionExecutor.NAME_NODE);
+        destinationUri = confChecker.checkAndGetTrimmed(GitActionExecutor.DESTINATION_URI);
+        try {
+            final FileSystem fs = FileSystem.get(isValidUri(destinationUri), actionConf);
+            destinationUri = fs.makeQualified(new Path(destinationUri)).toString();
+        } catch (final IOException e) {
+            throw new OozieActionConfiguratorException("Action Configuration does not have "
+                    + "a valid filesystem for URI " + GitActionExecutor.DESTINATION_URI + "exception "
+                    + e.toString());
+        }
+        gitUri = isValidUri(confChecker.checkAndGetTrimmed(GitActionExecutor.GIT_URI)).toString();
+        gitBranch = actionConf.get(GitActionExecutor.GIT_BRANCH);
+        keyPath = actionConf.get(GitActionExecutor.KEY_PATH);
+    }
+}