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/24 07:59:15 UTC

oozie git commit: OOZIE-3346 [examples] [action] Fix Git example. PrepareActionsHandler should support XML namespace prefixes (asalamon74, andras.piros)

Repository: oozie
Updated Branches:
  refs/heads/master 3cbb64882 -> f21074a0e


OOZIE-3346 [examples] [action] Fix Git example. PrepareActionsHandler should support XML namespace prefixes (asalamon74, andras.piros)


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

Branch: refs/heads/master
Commit: f21074a0ef9eae9becec67fd90035d73df1b5195
Parents: 3cbb648
Author: Andras Piros <an...@cloudera.com>
Authored: Mon Sep 24 09:58:37 2018 +0200
Committer: Andras Piros <an...@cloudera.com>
Committed: Mon Sep 24 09:58:37 2018 +0200

----------------------------------------------------------------------
 examples/src/main/apps/git/workflow.xml         |  7 +-
 .../org/apache/oozie/example/fluentjob/Git.java | 12 ++-
 release-log.txt                                 |  1 +
 .../apache/oozie/action/hadoop/LauncherAM.java  |  2 +-
 .../hadoop/LauncherURIHandlerFactory.java       |  6 ++
 .../action/hadoop/PrepareActionsDriver.java     |  6 +-
 .../action/hadoop/PrepareActionsHandler.java    | 14 +++-
 .../hadoop/TestPrepareActionsHandler.java       | 81 ++++++++++++++++++++
 8 files changed, 115 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/oozie/blob/f21074a0/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
index b1b0858..bbdac95 100644
--- a/examples/src/main/apps/git/workflow.xml
+++ b/examples/src/main/apps/git/workflow.xml
@@ -18,11 +18,14 @@
 <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">
+        <git xmlns="uri:oozie:git-action:1.0">
             <resource-manager>${resourceManager}</resource-manager>
             <name-node>${nameNode}</name-node>
+            <prepare>
+                <delete path="${nameNode}/user/${wf:user()}/${examplesRoot}/output-data/git/oozie"/>
+            </prepare>
             <git-uri>https://github.com/apache/oozie</git-uri>
-            <destination-uri>/user/${wf:user()}/${examplesRoot}/output-data/git/oozie</destination-uri>
+            <destination-uri>${nameNode}/user/${wf:user()}/${examplesRoot}/output-data/git/oozie</destination-uri>
         </git>
         <ok to="end"/>
         <error to="fail"/>

http://git-wip-us.apache.org/repos/asf/oozie/blob/f21074a0/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
index 2548d39..8d036cf 100644
--- a/examples/src/main/java/org/apache/oozie/example/fluentjob/Git.java
+++ b/examples/src/main/java/org/apache/oozie/example/fluentjob/Git.java
@@ -18,7 +18,10 @@
 
 package org.apache.oozie.example.fluentjob;
 
-import org.apache.oozie.fluentjob.api.action.*;
+import org.apache.oozie.fluentjob.api.action.GitAction;
+import org.apache.oozie.fluentjob.api.action.GitActionBuilder;
+import org.apache.oozie.fluentjob.api.action.Prepare;
+import org.apache.oozie.fluentjob.api.action.PrepareBuilder;
 import org.apache.oozie.fluentjob.api.factory.WorkflowFactory;
 import org.apache.oozie.fluentjob.api.workflow.Workflow;
 import org.apache.oozie.fluentjob.api.workflow.WorkflowBuilder;
@@ -29,10 +32,15 @@ import org.apache.oozie.fluentjob.api.workflow.WorkflowBuilder;
 public class Git implements WorkflowFactory {
     @Override
     public Workflow create() {
+        final Prepare prepare = new PrepareBuilder()
+                .withDelete("${nameNode}/user/${wf:user()}/${examplesRoot}/output-data/git/oozie")
+                .build();
+
         final GitAction parent = GitActionBuilder.create()
                 .withResourceManager("${resourceManager}")
                 .withNameNode("${nameNode}")
-                .withDestinationUri("/user/${wf:user()}/${examplesRoot}/output-data/git/oozie")
+                .withPrepare(prepare)
+                .withDestinationUri("${nameNode}/user/${wf:user()}/${examplesRoot}/output-data/git/oozie")
                 .withGitUri("https://github.com/apache/oozie")
                 .build();
 

http://git-wip-us.apache.org/repos/asf/oozie/blob/f21074a0/release-log.txt
----------------------------------------------------------------------
diff --git a/release-log.txt b/release-log.txt
index 37ad2ac..330cee6 100644
--- a/release-log.txt
+++ b/release-log.txt
@@ -1,5 +1,6 @@
 -- Oozie 5.1.0 release (trunk - unreleased)
 
+OOZIE-3346 [examples] [action] Fix Git example. PrepareActionsHandler should support XML namespace prefixes (asalamon74, andras.piros)
 OOZIE-3347 [examples] Fix Fluent Job global example (asalamon74 via andras.piros)
 OOZIE-3160 amend PriorityDelayQueue put()/take() can cause significant CPU load due to busy waiting (pbacsko)
 OOZIE-3342 [examples] Missing archive tag from Fluent Job examples (asalamon74 via andras.piros)

http://git-wip-us.apache.org/repos/asf/oozie/blob/f21074a0/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/LauncherAM.java
----------------------------------------------------------------------
diff --git a/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/LauncherAM.java b/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/LauncherAM.java
index 4a41941..63afd91 100644
--- a/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/LauncherAM.java
+++ b/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/LauncherAM.java
@@ -145,7 +145,7 @@ public class LauncherAM {
                         new AMRMCallBackHandler(),
                         new HdfsOperations(new SequenceFileWriterFactory()),
                         new LocalFsOperations(),
-                        new PrepareActionsHandler(),
+                        new PrepareActionsHandler(new LauncherURIHandlerFactory(null)),
                         new LauncherAMCallbackNotifierFactory(),
                         new LauncherSecurityManager(),
                         System.getenv(ApplicationConstants.Environment.CONTAINER_ID.name()),

http://git-wip-us.apache.org/repos/asf/oozie/blob/f21074a0/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/LauncherURIHandlerFactory.java
----------------------------------------------------------------------
diff --git a/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/LauncherURIHandlerFactory.java b/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/LauncherURIHandlerFactory.java
index 04c9175..bb12bfa 100644
--- a/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/LauncherURIHandlerFactory.java
+++ b/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/LauncherURIHandlerFactory.java
@@ -19,6 +19,8 @@
 package org.apache.oozie.action.hadoop;
 
 import java.net.URI;
+
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.util.ReflectionUtils;
 
@@ -62,4 +64,8 @@ public class LauncherURIHandlerFactory {
         return handler;
     }
 
+    @VisibleForTesting
+    LauncherURIHandler getURIHandler(final URI uri, final Configuration configuration) throws LauncherException {
+        return new LauncherURIHandlerFactory(configuration).getURIHandler(uri);
+    }
 }

http://git-wip-us.apache.org/repos/asf/oozie/blob/f21074a0/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/PrepareActionsDriver.java
----------------------------------------------------------------------
diff --git a/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/PrepareActionsDriver.java b/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/PrepareActionsDriver.java
index cb5b1ac..e730f0f 100644
--- a/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/PrepareActionsDriver.java
+++ b/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/PrepareActionsDriver.java
@@ -21,14 +21,10 @@ package org.apache.oozie.action.hadoop;
 import java.io.ByteArrayInputStream;
 import java.io.IOException;
 import java.io.InputStream;
-import java.net.URI;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
 import org.xml.sax.SAXException;
 import org.w3c.dom.Document;
-import org.w3c.dom.Node;
-import org.w3c.dom.NodeList;
 
 import javax.xml.parsers.DocumentBuilderFactory;
 import javax.xml.parsers.DocumentBuilder;
@@ -40,7 +36,7 @@ import javax.xml.parsers.ParserConfigurationException;
  */
 @Deprecated
 public class PrepareActionsDriver {
-    private static final PrepareActionsHandler prepareHandler = new PrepareActionsHandler();
+    private static final PrepareActionsHandler prepareHandler = new PrepareActionsHandler(new LauncherURIHandlerFactory(null));
 
     /**
      * Method to parse the prepare XML and execute the corresponding prepare actions

http://git-wip-us.apache.org/repos/asf/oozie/blob/f21074a0/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/PrepareActionsHandler.java
----------------------------------------------------------------------
diff --git a/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/PrepareActionsHandler.java b/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/PrepareActionsHandler.java
index 923b085..93731ad 100644
--- a/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/PrepareActionsHandler.java
+++ b/sharelib/oozie/src/main/java/org/apache/oozie/action/hadoop/PrepareActionsHandler.java
@@ -26,6 +26,7 @@ import javax.xml.parsers.DocumentBuilder;
 import javax.xml.parsers.DocumentBuilderFactory;
 import javax.xml.parsers.ParserConfigurationException;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.w3c.dom.Document;
@@ -34,6 +35,12 @@ import org.w3c.dom.NodeList;
 import org.xml.sax.SAXException;
 
 public class PrepareActionsHandler {
+    private final LauncherURIHandlerFactory factory;
+
+    @VisibleForTesting
+    PrepareActionsHandler(final LauncherURIHandlerFactory factory) {
+        this.factory = factory;
+    }
 
     /**
      * Method to parse the prepare XML and execute the corresponding prepare actions
@@ -45,25 +52,24 @@ public class PrepareActionsHandler {
      * @throws ParserConfigurationException if the parser is not well configured
      * @throws LauncherException in case of error
      */
-    public void prepareAction(String prepareXML, Configuration conf)
+    void prepareAction(String prepareXML, Configuration conf)
             throws IOException, SAXException, ParserConfigurationException, LauncherException {
         Document doc = getDocumentFromXML(prepareXML);
         doc.getDocumentElement().normalize();
 
         // Get the list of child nodes, basically, each one corresponding to a separate action
         NodeList nl = doc.getDocumentElement().getChildNodes();
-        LauncherURIHandlerFactory factory = new LauncherURIHandlerFactory(conf);
 
         for (int i = 0; i < nl.getLength(); ++i) {
             Node n = nl.item(i);
-            String operation = n.getNodeName();
+            String operation = n.getLocalName();
             if (n.getAttributes() == null || n.getAttributes().getNamedItem("path") == null) {
                 continue;
             }
             String pathStr = n.getAttributes().getNamedItem("path").getNodeValue().trim();
             // use Path to avoid URIsyntax error caused by square bracket in glob
             URI uri = new Path(pathStr).toUri();
-            LauncherURIHandler handler = factory.getURIHandler(uri);
+            LauncherURIHandler handler = factory.getURIHandler(uri, conf);
             execute(operation, uri, handler, conf);
         }
     }

http://git-wip-us.apache.org/repos/asf/oozie/blob/f21074a0/sharelib/oozie/src/test/java/org/apache/oozie/action/hadoop/TestPrepareActionsHandler.java
----------------------------------------------------------------------
diff --git a/sharelib/oozie/src/test/java/org/apache/oozie/action/hadoop/TestPrepareActionsHandler.java b/sharelib/oozie/src/test/java/org/apache/oozie/action/hadoop/TestPrepareActionsHandler.java
new file mode 100644
index 0000000..2547350
--- /dev/null
+++ b/sharelib/oozie/src/test/java/org/apache/oozie/action/hadoop/TestPrepareActionsHandler.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.action.hadoop;
+
+import org.apache.hadoop.conf.Configuration;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
+import org.mockito.runners.MockitoJUnitRunner;
+import org.xml.sax.SAXException;
+
+import javax.xml.parsers.ParserConfigurationException;
+import java.io.IOException;
+import java.net.URI;
+
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.verifyNoMoreInteractions;
+import static org.mockito.Mockito.when;
+
+@RunWith(MockitoJUnitRunner.class)
+public class TestPrepareActionsHandler {
+    private PrepareActionsHandler prepareActionsHandler;
+
+    @Mock
+    private LauncherURIHandlerFactory launcherURIHandlerFactory;
+
+    @Mock
+    private LauncherURIHandler launcherURIHandler;
+
+    @Before
+    public void setUp() throws LauncherException {
+        prepareActionsHandler = new PrepareActionsHandler(launcherURIHandlerFactory);
+
+        when(launcherURIHandlerFactory.getURIHandler(any(URI.class), any(Configuration.class))).thenReturn(launcherURIHandler);
+    }
+
+    @Test
+    public void whenPrepareActionWithoutPrefixExecutedSuccessfully()
+            throws SAXException, ParserConfigurationException, LauncherException, IOException {
+        final String prepareXmlWithoutPrefix = "<prepare>\n" +
+                "    <delete path=\"${nameNode}/user/${wf:user()}/${examplesRoot}/output-data/git/oozie\"/>\n" +
+                "</prepare>";
+
+        callAndVerifyPrepareAction(prepareXmlWithoutPrefix);
+    }
+
+    private void callAndVerifyPrepareAction(final String prepareXml)
+            throws IOException, SAXException, ParserConfigurationException, LauncherException {
+        prepareActionsHandler.prepareAction(prepareXml, null);
+
+        verify(launcherURIHandler).delete(any(), any());
+        verifyNoMoreInteractions(launcherURIHandler);
+    }
+
+    @Test
+    public void whenPrepareActionWithGitPrefixExecutedSuccessfully()
+            throws SAXException, ParserConfigurationException, LauncherException, IOException {
+        final String prepareXmlWithGitPrefix = "<git:prepare xmlns:git=\"uri:oozie:git-action:1.0\">\n" +
+                "    <git:delete path=\"${nameNode}/user/${wf:user()}/${examplesRoot}/output-data/git/oozie\"/>\n" +
+                "</git:prepare>";
+
+        callAndVerifyPrepareAction(prepareXmlWithGitPrefix);
+    }
+}
\ No newline at end of file