You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@oozie.apache.org by sa...@apache.org on 2017/09/20 16:58:08 UTC

oozie git commit: OOZIE-3048 Check El Functions for the coordinator action (satishsaley)

Repository: oozie
Updated Branches:
  refs/heads/master 5e4d28843 -> a5f9aa54a


OOZIE-3048 Check El Functions for the coordinator action (satishsaley)


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

Branch: refs/heads/master
Commit: a5f9aa54a553bb5d981538c321ace2cefe0e5e28
Parents: 5e4d288
Author: satishsaley <sa...@apache.org>
Authored: Wed Sep 20 09:53:50 2017 -0700
Committer: satishsaley <sa...@apache.org>
Committed: Wed Sep 20 09:53:50 2017 -0700

----------------------------------------------------------------------
 .../coord/CoordActionInputCheckXCommand.java    | 38 +++++++++---
 .../coord/CoordPushDependencyCheckXCommand.java |  6 ++
 .../apache/oozie/coord/CoordELFunctions.java    |  2 +-
 .../org/apache/oozie/coord/ElException.java     | 32 ++++++++++
 .../TestCoordActionInputCheckXCommand.java      | 35 +++++++++++
 .../TestCoordPushDependencyCheckXCommand.java   | 38 ++++++++++++
 .../coord-hcatinput-invalid-elfunction.xml      | 62 +++++++++++++++++++
 .../coord-hdfsinput-invalid-elfunction.xml      | 64 ++++++++++++++++++++
 release-log.txt                                 |  1 +
 9 files changed, 270 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/oozie/blob/a5f9aa54/core/src/main/java/org/apache/oozie/command/coord/CoordActionInputCheckXCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/oozie/command/coord/CoordActionInputCheckXCommand.java b/core/src/main/java/org/apache/oozie/command/coord/CoordActionInputCheckXCommand.java
index 650e7f8..401b2c7 100644
--- a/core/src/main/java/org/apache/oozie/command/coord/CoordActionInputCheckXCommand.java
+++ b/core/src/main/java/org/apache/oozie/command/coord/CoordActionInputCheckXCommand.java
@@ -34,6 +34,7 @@ import org.apache.oozie.command.CommandException;
 import org.apache.oozie.command.PreconditionException;
 import org.apache.oozie.coord.CoordELEvaluator;
 import org.apache.oozie.coord.CoordELFunctions;
+import org.apache.oozie.coord.ElException;
 import org.apache.oozie.coord.input.dependency.CoordInputDependency;
 import org.apache.oozie.executor.jpa.CoordActionGetForInputCheckJPAExecutor;
 import org.apache.oozie.executor.jpa.CoordActionQueryExecutor;
@@ -182,13 +183,7 @@ public class CoordActionInputCheckXCommand extends CoordinatorXCommand<Void> {
             isChangeInDependency = isChangeInDependency(nonExistList, missingDependencies, nonResolvedList, status);
 
             if (status && isPushDependenciesMet) {
-                String newActionXml = resolveCoordConfiguration(actionXml, actionConf, actionId,
-                        coordPullInputDependency, coordPushInputDependency);
-                actionXml.replace(0, actionXml.length(), newActionXml);
-                coordAction.setActionXml(actionXml.toString());
-                coordAction.setStatus(CoordinatorAction.Status.READY);
-                updateCoordAction(coordAction, true);
-                new CoordActionReadyXCommand(coordAction.getJobId()).call();
+                moveCoordActionToReady(actionXml, actionConf, coordPullInputDependency, coordPushInputDependency);
             }
             else if (!isTimeout(currentTime)) {
                 if (!status) {
@@ -429,6 +424,35 @@ public class CoordActionInputCheckXCommand extends CoordinatorXCommand<Void> {
     }
 
     /**
+     * Resolves coordinator configuration and moves CoordAction to READY state
+     *
+     * @param actionXml
+     * @param actionConf
+     * @param coordPullInputDependency
+     * @param coordPushInputDependency
+     * @throws Exception
+     */
+    private void moveCoordActionToReady(StringBuilder actionXml, Configuration actionConf,
+            CoordInputDependency coordPullInputDependency, CoordInputDependency coordPushInputDependency)
+            throws Exception {
+        String newActionXml = null;
+        try {
+            newActionXml = resolveCoordConfiguration(actionXml, actionConf, actionId, coordPullInputDependency,
+                    coordPushInputDependency);
+        }
+        catch (ElException e) {
+            coordAction.setStatus(CoordinatorAction.Status.FAILED);
+            updateCoordAction(coordAction, true);
+            throw e;
+        }
+        actionXml.replace(0, actionXml.length(), newActionXml);
+        coordAction.setActionXml(actionXml.toString());
+        coordAction.setStatus(CoordinatorAction.Status.READY);
+        updateCoordAction(coordAction, true);
+        new CoordActionReadyXCommand(coordAction.getJobId()).call();
+    }
+
+    /**
      * getting the error code of the coord action. (used mainly for unit testing)
      */
     protected String getCoordActionErrorCode() {

http://git-wip-us.apache.org/repos/asf/oozie/blob/a5f9aa54/core/src/main/java/org/apache/oozie/command/coord/CoordPushDependencyCheckXCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/oozie/command/coord/CoordPushDependencyCheckXCommand.java b/core/src/main/java/org/apache/oozie/command/coord/CoordPushDependencyCheckXCommand.java
index 0ad3970..1e864ea 100644
--- a/core/src/main/java/org/apache/oozie/command/coord/CoordPushDependencyCheckXCommand.java
+++ b/core/src/main/java/org/apache/oozie/command/coord/CoordPushDependencyCheckXCommand.java
@@ -34,6 +34,7 @@ import org.apache.oozie.client.Job;
 import org.apache.oozie.client.OozieClient;
 import org.apache.oozie.command.CommandException;
 import org.apache.oozie.command.PreconditionException;
+import org.apache.oozie.coord.ElException;
 import org.apache.oozie.coord.input.dependency.CoordInputDependency;
 import org.apache.oozie.dependency.ActionDependency;
 import org.apache.oozie.dependency.DependencyChecker;
@@ -263,6 +264,11 @@ public class CoordPushDependencyCheckXCommand extends CoordinatorXCommand<Void>
             actionXml.replace(0, actionXml.length(), newActionXml);
             return actionXml.toString();
         }
+        catch (ElException e) {
+            coordAction.setStatus(CoordinatorAction.Status.FAILED);
+            updateCoordAction(coordAction, true);
+            throw new CommandException(ErrorCode.E1021, e.getMessage(), e);
+        }
         catch (Exception e) {
             throw new CommandException(ErrorCode.E1021, e.getMessage(), e);
         }

http://git-wip-us.apache.org/repos/asf/oozie/blob/a5f9aa54/core/src/main/java/org/apache/oozie/coord/CoordELFunctions.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/oozie/coord/CoordELFunctions.java b/core/src/main/java/org/apache/oozie/coord/CoordELFunctions.java
index fd2f92d..76d837c 100644
--- a/core/src/main/java/org/apache/oozie/coord/CoordELFunctions.java
+++ b/core/src/main/java/org/apache/oozie/coord/CoordELFunctions.java
@@ -784,7 +784,7 @@ public class CoordELFunctions {
             }
         }
         catch (Exception e) {
-            throw new Exception("Unable to evaluate :" + expr + ":\n", e);
+            throw new ElException(ErrorCode.E1004, "Unable to evaluate :" + expr + ":\n", e);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/oozie/blob/a5f9aa54/core/src/main/java/org/apache/oozie/coord/ElException.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/oozie/coord/ElException.java b/core/src/main/java/org/apache/oozie/coord/ElException.java
new file mode 100644
index 0000000..c1d2b35
--- /dev/null
+++ b/core/src/main/java/org/apache/oozie/coord/ElException.java
@@ -0,0 +1,32 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.coord;
+
+import org.apache.oozie.ErrorCode;
+import org.apache.oozie.XException;
+
+/**
+ * Exception thrown while evaluating el functions
+ */
+public class ElException extends XException {
+    private static final long serialVersionUID = 1L;
+
+    public ElException(ErrorCode errorCode, Object... params) {
+        super(errorCode, params);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/oozie/blob/a5f9aa54/core/src/test/java/org/apache/oozie/command/coord/TestCoordActionInputCheckXCommand.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/oozie/command/coord/TestCoordActionInputCheckXCommand.java b/core/src/test/java/org/apache/oozie/command/coord/TestCoordActionInputCheckXCommand.java
index bbe9ea7..9f20942 100644
--- a/core/src/test/java/org/apache/oozie/command/coord/TestCoordActionInputCheckXCommand.java
+++ b/core/src/test/java/org/apache/oozie/command/coord/TestCoordActionInputCheckXCommand.java
@@ -873,6 +873,41 @@ public class TestCoordActionInputCheckXCommand extends XDataTestCase {
 
     }
 
+    public void testExceptionOnInvalidElFunction() {
+        try {
+            CoordinatorJobBean job = addRecordToCoordJobTableForWaiting("coord-hdfsinput-invalid-elfunction.xml",
+                    CoordinatorJob.Status.RUNNING, false, true);
+
+            CoordinatorActionBean action = addRecordToCoordActionTableForWaiting(job.getId(), 1,
+                    CoordinatorAction.Status.WAITING, "coord-hdfsinput-invalid-elfunction.xml");
+
+            createTestCaseSubDir("2009/01/29/_SUCCESS".split("/"));
+            createTestCaseSubDir("2009/01/22/_SUCCESS".split("/"));
+            createTestCaseSubDir("2009/01/15/_SUCCESS".split("/"));
+            createTestCaseSubDir("2009/01/08/_SUCCESS".split("/"));
+            sleep(3000);
+            final String actionId = action.getId();
+            try {
+                new CoordActionInputCheckXCommand(action.getId(), job.getId()).call();
+                waitFor(6000, new Predicate() {
+                    @Override
+                    public boolean evaluate() throws Exception {
+                        CoordinatorActionBean action = CoordActionQueryExecutor.getInstance()
+                                .get(CoordActionQueryExecutor.CoordActionQuery.GET_COORD_ACTION, actionId);
+                        return action.getStatus() == CoordinatorAction.Status.FAILED;
+                    }
+                });
+                fail("Should throw an exception");
+            }
+            catch (Exception e) {
+                assertTrue(e.getMessage().contains("Coord Action Input Check Error"));
+            }
+        }
+        catch (Exception e) {
+            e.printStackTrace(System.out);
+            fail("Unexpected exception");
+        }
+    }
     protected CoordinatorActionBean addRecordToCoordActionTableForWaiting(String jobId, int actionNum,
             CoordinatorAction.Status status, String resourceXmlName) throws Exception {
         CoordinatorActionBean action = createCoordAction(jobId, actionNum, status, resourceXmlName, 0, TZ, null);

http://git-wip-us.apache.org/repos/asf/oozie/blob/a5f9aa54/core/src/test/java/org/apache/oozie/command/coord/TestCoordPushDependencyCheckXCommand.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/oozie/command/coord/TestCoordPushDependencyCheckXCommand.java b/core/src/test/java/org/apache/oozie/command/coord/TestCoordPushDependencyCheckXCommand.java
index 25f250c..6cb6485 100644
--- a/core/src/test/java/org/apache/oozie/command/coord/TestCoordPushDependencyCheckXCommand.java
+++ b/core/src/test/java/org/apache/oozie/command/coord/TestCoordPushDependencyCheckXCommand.java
@@ -33,6 +33,7 @@ import org.apache.oozie.client.CoordinatorJob;
 import org.apache.oozie.coord.CoordELFunctions;
 import org.apache.oozie.dependency.DependencyChecker;
 import org.apache.oozie.executor.jpa.CoordActionGetJPAExecutor;
+import org.apache.oozie.executor.jpa.CoordActionQueryExecutor;
 import org.apache.oozie.executor.jpa.JPAExecutorException;
 import org.apache.oozie.service.CallableQueueService;
 import org.apache.oozie.service.HCatAccessorService;
@@ -426,6 +427,43 @@ public class TestCoordPushDependencyCheckXCommand extends XDataTestCase {
         assertTrue(out.toString().contains("ACTION[" + actionId2 + "]"));
     }
 
+    @Test
+    public void testExceptionOnInvalidElFunction() throws Exception {
+        String db = "default";
+        String table = "tablename";
+        String newHCatDependency1 = "hcat://" + server + "/" + db + "/" + table + "/dt=20120412;country=brazil";
+        String newHCatDependency2 = "hcat://" + server + "/" + db + "/" + table + "/dt=20120430;country=usa";
+        String newHCatDependency = newHCatDependency1 + CoordELFunctions.INSTANCE_SEPARATOR + newHCatDependency2;
+        populateTable(db, table);
+
+        CoordinatorJobBean job = addRecordToCoordJobTableForWaiting("coord-hcatinput-invalid-elfunction.xml",
+                CoordinatorJob.Status.RUNNING, false, true);
+
+        CoordinatorActionBean action = addRecordToCoordActionTableForWaiting(job.getId(), 1,
+                CoordinatorAction.Status.WAITING, "coord-hcatinput-invalid-elfunction.xml", null, newHCatDependency,
+                "Z");
+
+        final String actionId = action.getId();
+        checkCoordAction(actionId, newHCatDependency, CoordinatorAction.Status.WAITING);
+
+        try {
+            new CoordPushDependencyCheckXCommand(actionId).call();
+            waitFor(6000, new Predicate() {
+                @Override
+                public boolean evaluate() throws Exception {
+                    CoordinatorActionBean action = CoordActionQueryExecutor.getInstance()
+                            .get(CoordActionQueryExecutor.CoordActionQuery.GET_COORD_ACTION, actionId);
+                    return action.getStatus() == CoordinatorAction.Status.FAILED;
+                }
+            });
+            fail("Should throw an exception");
+        }
+        catch (Exception e) {
+            assertTrue(e.getMessage().contains("Coord Action Input Check Error"));
+        }
+
+    }
+
     private void populateTable(String db, String table) throws Exception {
         dropTable(db, table, true);
         dropDatabase(db, true);

http://git-wip-us.apache.org/repos/asf/oozie/blob/a5f9aa54/core/src/test/resources/coord-hcatinput-invalid-elfunction.xml
----------------------------------------------------------------------
diff --git a/core/src/test/resources/coord-hcatinput-invalid-elfunction.xml b/core/src/test/resources/coord-hcatinput-invalid-elfunction.xml
new file mode 100644
index 0000000..7ca2060
--- /dev/null
+++ b/core/src/test/resources/coord-hcatinput-invalid-elfunction.xml
@@ -0,0 +1,62 @@
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you under the Apache License, Version 2.0 (the
+  "License"); you may not use this file except in compliance
+  with the License.  You may obtain a copy of the License at
+       http://www.apache.org/licenses/LICENSE-2.0
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+<coordinator-app xmlns="uri:oozie:coordinator:0.2" name="NAME" frequency="1" timezone="UTC" freq_timeunit="DAY"
+end_of_duration="NONE" instance-number="1" action-nominal-time="2009-02-01T23:59${TZ}" action-actual-time="2010-10-01T00:00${TZ}">
+  <controls>
+    <timeout>10</timeout>
+    <concurrency>2</concurrency>
+    <execution>LIFO</execution>
+  </controls>
+  <input-events>
+    <data-in name="A" dataset="a">
+      <dataset name="a" frequency="7" initial-instance="2009-01-01T01:00${TZ}" timezone="UTC" freq_timeunit="DAY"
+      end_of_duration="NONE">
+        <uri-template>hcat://dummyhcat:1000/db1/table1/ds=${YEAR}-${DAY}</uri-template>
+      </dataset>
+      <uris>hcat://dummyhcat:1000/db1/table1/ds=/2009-29#hcat://dummyhcat:1000/db1/table1/ds=/2009-29#
+      hcat://dummyhcat:1000/db1/table1/ds=/2009-29</uris>
+    </data-in>
+  </input-events>
+  <output-events>
+    <data-out name="LOCAL_A" dataset="local_a">
+      <dataset name="local_a" frequency="7" initial-instance="2009-01-01T01:00${TZ}" timezone="UTC" freq_timeunit="DAY"
+      end_of_duration="NONE">
+        <uri-template>file://#testDir/${YEAR}/${DAY}</uri-template>
+      </dataset>
+      <uris>hcat://dummyhcat:1000/db1/table1/ds=/2009-29</uris>
+      <start-instance>${coord:current(-3)}</start-instance>
+    </data-out>
+  </output-events>
+  <action>
+    <workflow>
+      <app-path>hdfs:///tmp/workflows/</app-path>
+      <configuration>
+        <property>
+          <name>inputA</name>
+          <value>${coord:dataIn('A')}</value>
+        </property>
+        <property>
+          <name>inputB</name>
+          <value>${coord:dataOut('LOCAL_A')}</value>
+        </property>
+        <property>
+          <name>yyyymmdd</name>
+          <value>${coord:formatTime(coord:nominalTime(), 'DAY')}</value>
+        </property>
+      </configuration>
+    </workflow>
+  </action>
+</coordinator-app>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/oozie/blob/a5f9aa54/core/src/test/resources/coord-hdfsinput-invalid-elfunction.xml
----------------------------------------------------------------------
diff --git a/core/src/test/resources/coord-hdfsinput-invalid-elfunction.xml b/core/src/test/resources/coord-hdfsinput-invalid-elfunction.xml
new file mode 100644
index 0000000..3b1c881
--- /dev/null
+++ b/core/src/test/resources/coord-hdfsinput-invalid-elfunction.xml
@@ -0,0 +1,64 @@
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you under the Apache License, Version 2.0 (the
+  "License"); you may not use this file except in compliance
+  with the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+<coordinator-app xmlns="uri:oozie:coordinator:0.2" name="NAME" frequency="1" timezone="UTC"
+freq_timeunit="DAY" end_of_duration="NONE" instance-number="1" action-nominal-time="2009-02-01T23:59${TZ}"
+action-actual-time="2010-10-01T00:00${TZ}">
+  <controls>
+    <timeout>10</timeout>
+    <concurrency>2</concurrency>
+    <execution>LIFO</execution>
+  </controls>
+  <input-events>
+    <data-in name="A" dataset="a">
+      <dataset name="a" frequency="7" initial-instance="2009-01-01T01:00${TZ}" timezone="UTC"
+      freq_timeunit="DAY" end_of_duration="NONE">
+        <uri-template>file://#testDir/${YEAR}/${DAY}</uri-template>
+      </dataset>
+      <uris>file://#testDir/2009/29#file://#testDir/2009/22#file://#testDir/2009/15#file://#testDir/2009/08</uris>
+    </data-in>
+  </input-events>
+  <output-events>
+    <data-out name="LOCAL_A" dataset="local_a">
+      <dataset name="local_a" frequency="7" initial-instance="2009-01-01T01:00${TZ}" timezone="UTC" freq_timeunit="DAY"
+       end_of_duration="NONE">
+        <uri-template>file://#testDir/${YEAR}/${DAY}</uri-template>
+      </dataset>
+      <uris>file://#testDir/2009/29</uris>
+      <start-instance>${coord:current(-3)}</start-instance>
+    </data-out>
+  </output-events>
+  <action>
+    <workflow>
+      <app-path>hdfs:///tmp/workflows/</app-path>
+      <configuration>
+        <property>
+          <name>inputA</name>
+          <value>${coord:dataIn('A')}</value>
+        </property>
+        <property>
+          <name>inputB</name>
+          <value>${coord:dataOut('LOCAL_A')}</value>
+        </property>
+        <property>
+          <name>yyyymmdd</name>
+          <value>${coord:formatTime(coord:nominalTime(), 'DAY')}</value>
+        </property>
+      </configuration>
+    </workflow>
+  </action>
+</coordinator-app>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/oozie/blob/a5f9aa54/release-log.txt
----------------------------------------------------------------------
diff --git a/release-log.txt b/release-log.txt
index 0d6deb4..803da2f 100644
--- a/release-log.txt
+++ b/release-log.txt
@@ -1,5 +1,6 @@
 -- Oozie 5.0.0 release (trunk - unreleased)
 
+OOZIE-3048 Check El Functions for the coordinator action (satishsaley)
 OOZIE-3058 nocleanup option is missing in oozie-coordinator-0.5.xsd (satishsaley)
 OOZIE-2909 LauncherAM: rewrite UGI calls (gezapeti)
 OOZIE-2687 Create XML schema for launcher configurations (asasvari)