You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@oozie.apache.org by rk...@apache.org on 2015/08/11 23:45:56 UTC

oozie git commit: OOZIE-2324 A syntax error in the kill node causes the workflow to get stuck and other problems (rkanter)

Repository: oozie
Updated Branches:
  refs/heads/master 2d270148f -> a6d640d58


OOZIE-2324 A syntax error in the kill node causes the workflow to get stuck and other problems (rkanter)


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

Branch: refs/heads/master
Commit: a6d640d58d20998ebb7a948534f1bfc15659dcd7
Parents: 2d27014
Author: Robert Kanter <rk...@cloudera.com>
Authored: Tue Aug 11 14:45:59 2015 -0700
Committer: Robert Kanter <rk...@cloudera.com>
Committed: Tue Aug 11 14:45:59 2015 -0700

----------------------------------------------------------------------
 .../main/java/org/apache/oozie/ErrorCode.java   |  1 +
 .../org/apache/oozie/WorkflowActionBean.java    |  2 +-
 .../apache/oozie/command/wf/SignalXCommand.java | 17 ++++----
 .../jpa/WorkflowActionQueryExecutor.java        |  1 +
 .../oozie/command/wf/TestActionErrors.java      | 45 ++++++++++++++++++--
 .../jpa/TestWorkflowActionQueryExecutor.java    |  1 +
 .../wf-test-kill-node-message-error.xml         | 42 ++++++++++++++++++
 .../wf-test-kill-node-message-error2.xml        | 42 ++++++++++++++++++
 release-log.txt                                 |  1 +
 9 files changed, 140 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/oozie/blob/a6d640d5/core/src/main/java/org/apache/oozie/ErrorCode.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/oozie/ErrorCode.java b/core/src/main/java/org/apache/oozie/ErrorCode.java
index 4e9f62b..6c1e399 100644
--- a/core/src/main/java/org/apache/oozie/ErrorCode.java
+++ b/core/src/main/java/org/apache/oozie/ErrorCode.java
@@ -152,6 +152,7 @@ public enum ErrorCode {
     E0743(XLog.STD, "Multiple \"ok to\" transitions to the same node, [{0}], are not allowed"),
     E0744(XLog.STD, "A fork, [{0}], is not allowed to have multiple transitions to the same node, [{1}]"),
     E0755(XLog.STD, "Workflow Job Rerun Error: {0}"),
+    E0756(XLog.STD, "Exception parsing Kill node message [{0}]"),
 
     E0800(XLog.STD, "Action it is not running its in [{1}] state, action [{0}]"),
     E0801(XLog.STD, "Workflow already running, workflow [{0}]"),

http://git-wip-us.apache.org/repos/asf/oozie/blob/a6d640d5/core/src/main/java/org/apache/oozie/WorkflowActionBean.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/oozie/WorkflowActionBean.java b/core/src/main/java/org/apache/oozie/WorkflowActionBean.java
index 57ae3f5..974c52b 100644
--- a/core/src/main/java/org/apache/oozie/WorkflowActionBean.java
+++ b/core/src/main/java/org/apache/oozie/WorkflowActionBean.java
@@ -74,7 +74,7 @@ import org.json.simple.JSONObject;
 
     @NamedQuery(name = "UPDATE_ACTION_PENDING_TRANS", query = "update WorkflowActionBean a set a.pending = :pending, a.pendingAgeTimestamp = :pendingAge, a.transition = :transition where a.id = :id"),
 
-    @NamedQuery(name = "UPDATE_ACTION_PENDING_TRANS_ERROR", query = "update WorkflowActionBean a set a.pending = :pending, a.pendingAgeTimestamp = :pendingAge, a.transition = :transition, a.errorCode = :errorCode, a.errorMessage = :errorMessage where a.id = :id"),
+    @NamedQuery(name = "UPDATE_ACTION_PENDING_TRANS_ERROR", query = "update WorkflowActionBean a set a.pending = :pending, a.pendingAgeTimestamp = :pendingAge, a.transition = :transition, a.errorCode = :errorCode, a.errorMessage = :errorMessage, a.statusStr = :status where a.id = :id"),
 
     @NamedQuery(name = "DELETE_ACTION", query = "delete from WorkflowActionBean a where a.id IN (:id)"),
 

http://git-wip-us.apache.org/repos/asf/oozie/blob/a6d640d5/core/src/main/java/org/apache/oozie/command/wf/SignalXCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/oozie/command/wf/SignalXCommand.java b/core/src/main/java/org/apache/oozie/command/wf/SignalXCommand.java
index 1b4b0b6..d1fcd1a 100644
--- a/core/src/main/java/org/apache/oozie/command/wf/SignalXCommand.java
+++ b/core/src/main/java/org/apache/oozie/command/wf/SignalXCommand.java
@@ -297,23 +297,24 @@ public class SignalXCommand extends WorkflowXCommand<Void> {
                     InstrumentUtils.incrJobCounter(INSTR_KILLED_JOBS_COUNTER_NAME, 1, getInstrumentation());
                     try {
                         String tmpNodeConf = nodeDef.getConf();
-                        String actionConf = context.getELEvaluator().evaluate(tmpNodeConf, String.class);
+                        String message = context.getELEvaluator().evaluate(tmpNodeConf, String.class);
                         LOG.debug(
                                 "Try to resolve KillNode message for jobid [{0}], actionId [{1}], before resolve [{2}], "
-                                        + "after resolve [{3}]", jobId, actionId, tmpNodeConf, actionConf);
+                                        + "after resolve [{3}]", jobId, actionId, tmpNodeConf, message);
                         if (wfAction.getErrorCode() != null) {
-                            wfAction.setErrorInfo(wfAction.getErrorCode(), actionConf);
+                            wfAction.setErrorInfo(wfAction.getErrorCode(), message);
                         }
                         else {
-                            wfAction.setErrorInfo(ErrorCode.E0729.toString(), actionConf);
+                            wfAction.setErrorInfo(ErrorCode.E0729.toString(), message);
                         }
-                        updateList.add(new UpdateEntry<WorkflowActionQuery>(
-                                WorkflowActionQuery.UPDATE_ACTION_PENDING_TRANS_ERROR, wfAction));
                     }
                     catch (Exception ex) {
-                        LOG.warn("Exception in SignalXCommand ", ex.getMessage(), ex);
-                        throw new CommandException(ErrorCode.E0729, wfAction.getName(), ex);
+                        LOG.warn("Exception in SignalXCommand when processing Kill node message: {0}", ex.getMessage(), ex);
+                        wfAction.setErrorInfo(ErrorCode.E0756.toString(), ErrorCode.E0756.format(ex.getMessage()));
+                        wfAction.setStatus(WorkflowAction.Status.ERROR);
                     }
+                    updateList.add(new UpdateEntry<WorkflowActionQuery>(
+                            WorkflowActionQuery.UPDATE_ACTION_PENDING_TRANS_ERROR, wfAction));
                 }
             }
 

http://git-wip-us.apache.org/repos/asf/oozie/blob/a6d640d5/core/src/main/java/org/apache/oozie/executor/jpa/WorkflowActionQueryExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/oozie/executor/jpa/WorkflowActionQueryExecutor.java b/core/src/main/java/org/apache/oozie/executor/jpa/WorkflowActionQueryExecutor.java
index 4dec9da..76f9adc 100644
--- a/core/src/main/java/org/apache/oozie/executor/jpa/WorkflowActionQueryExecutor.java
+++ b/core/src/main/java/org/apache/oozie/executor/jpa/WorkflowActionQueryExecutor.java
@@ -131,6 +131,7 @@ public class WorkflowActionQueryExecutor extends
                 query.setParameter("pendingAge", actionBean.getPendingAgeTimestamp());
                 query.setParameter("errorCode", actionBean.getErrorCode());
                 query.setParameter("errorMessage", actionBean.getErrorMessage());
+                query.setParameter("status", actionBean.getStatusStr());
                 query.setParameter("id", actionBean.getId());
                 break;
             case UPDATE_ACTION_START:

http://git-wip-us.apache.org/repos/asf/oozie/blob/a6d640d5/core/src/test/java/org/apache/oozie/command/wf/TestActionErrors.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/oozie/command/wf/TestActionErrors.java b/core/src/test/java/org/apache/oozie/command/wf/TestActionErrors.java
index 8eb98af..dadebb5 100644
--- a/core/src/test/java/org/apache/oozie/command/wf/TestActionErrors.java
+++ b/core/src/test/java/org/apache/oozie/command/wf/TestActionErrors.java
@@ -31,6 +31,7 @@ import org.apache.oozie.DagEngine;
 import org.apache.oozie.ForTestingActionExecutor;
 import org.apache.oozie.WorkflowActionBean;
 import org.apache.oozie.WorkflowJobBean;
+import org.apache.oozie.action.control.KillActionExecutor;
 import org.apache.oozie.client.CoordinatorJob;
 import org.apache.oozie.client.CoordinatorAction;
 import org.apache.oozie.client.OozieClient;
@@ -244,8 +245,42 @@ public class TestActionErrors extends XDataTestCase {
      * @throws Exception
      */
     public void testKillNodeErrorMessage() throws Exception {
+        WorkflowActionBean killAction = _testKillNodeErrorMessage("wf-test-kill-node-message.xml");
+        assertEquals("E0729", killAction.getErrorCode());
+        assertEquals("[end]", killAction.getErrorMessage());
+        assertEquals(WorkflowAction.Status.OK, killAction.getStatus());
+    }
+
+    /**
+     * Provides functionality to test kill node message when there's an error processing the message itself
+     *
+     * @throws Exception
+     */
+    public void testKillNodeErrorMessageError() throws Exception {
+        WorkflowActionBean killAction = _testKillNodeErrorMessage("wf-test-kill-node-message-error.xml");
+        assertEquals("E0756", killAction.getErrorCode());
+        assertEquals("E0756: Exception parsing Kill node message [Encountered \"{\", expected one of [<INTEGER_LITERAL>, " +
+                        "<FLOATING_POINT_LITERAL>, <STRING_LITERAL>, \"true\", \"false\", \"null\", \"(\", \"-\", \"not\", " +
+                        "\"!\", \"empty\", <IDENTIFIER>]]", killAction.getErrorMessage());
+        assertEquals(WorkflowAction.Status.ERROR, killAction.getStatus());
+    }
+
+    /**
+     * Provides functionality to test kill node message when there's an error processing the message itself
+     *
+     * @throws Exception
+     */
+    public void testKillNodeErrorMessageError2() throws Exception {
+        WorkflowActionBean killAction = _testKillNodeErrorMessage("wf-test-kill-node-message-error2.xml");
+        assertEquals("E0756", killAction.getErrorCode());
+        assertEquals("E0756: Exception parsing Kill node message [variable [bar] cannot be resolved]",
+                killAction.getErrorMessage());
+        assertEquals(WorkflowAction.Status.ERROR, killAction.getStatus());
+    }
+
+    private WorkflowActionBean _testKillNodeErrorMessage(String workflowXmlFile) throws Exception {
         String workflowPath = getTestCaseFileUri("workflow.xml");
-        Reader reader = IOUtils.getResourceAsReader("wf-test-kill-node-message.xml", -1);
+        Reader reader = IOUtils.getResourceAsReader(workflowXmlFile, -1);
         Writer writer = new FileWriter(new File(getTestCaseDir(), "workflow.xml"));
         IOUtils.copyCharStream(reader, writer);
 
@@ -277,18 +312,22 @@ public class TestActionErrors extends XDataTestCase {
         WorkflowActionsGetForJobJPAExecutor wfActionsGetCmd = new WorkflowActionsGetForJobJPAExecutor(jobId);
         List<WorkflowActionBean> actions = jpaService.execute(wfActionsGetCmd);
 
-        int n = actions.size();
         WorkflowActionBean action = null;
+        WorkflowActionBean killAction = null;
         for (WorkflowActionBean bean : actions) {
             if (bean.getType().equals("test")) {
                 action = bean;
-                break;
+            }
+            if (bean.getType().equals(KillActionExecutor.TYPE)) {
+                killAction = bean;
             }
         }
         assertNotNull(action);
         assertEquals("TEST_ERROR", action.getErrorCode());
         assertEquals("end", action.getErrorMessage());
         assertEquals(WorkflowAction.Status.ERROR, action.getStatus());
+        assertNotNull(killAction);
+        return killAction;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/oozie/blob/a6d640d5/core/src/test/java/org/apache/oozie/executor/jpa/TestWorkflowActionQueryExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/oozie/executor/jpa/TestWorkflowActionQueryExecutor.java b/core/src/test/java/org/apache/oozie/executor/jpa/TestWorkflowActionQueryExecutor.java
index 0099f01..47542af 100644
--- a/core/src/test/java/org/apache/oozie/executor/jpa/TestWorkflowActionQueryExecutor.java
+++ b/core/src/test/java/org/apache/oozie/executor/jpa/TestWorkflowActionQueryExecutor.java
@@ -121,6 +121,7 @@ public class TestWorkflowActionQueryExecutor extends XDataTestCase {
         assertEquals(query.getParameterValue("transition"), bean.getTransition());
         assertEquals(query.getParameterValue("errorCode"), bean.getErrorCode());
         assertEquals(query.getParameterValue("errorMessage"), bean.getErrorMessage());
+        assertEquals(query.getParameterValue("status"), bean.getStatusStr());
         assertEquals(query.getParameterValue("id"), bean.getId());
         // UPDATE_ACTION_START
         query = WorkflowActionQueryExecutor.getInstance().getUpdateQuery(WorkflowActionQuery.UPDATE_ACTION_START, bean,

http://git-wip-us.apache.org/repos/asf/oozie/blob/a6d640d5/core/src/test/resources/wf-test-kill-node-message-error.xml
----------------------------------------------------------------------
diff --git a/core/src/test/resources/wf-test-kill-node-message-error.xml b/core/src/test/resources/wf-test-kill-node-message-error.xml
new file mode 100644
index 0000000..56ea4dd
--- /dev/null
+++ b/core/src/test/resources/wf-test-kill-node-message-error.xml
@@ -0,0 +1,42 @@
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you under the Apache License, Version 2.0 (the
+  "License"); you may not use this file except in compliance
+  with the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT 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:0.1" name="test-wf">
+
+    <start to="a"/>
+
+    <action name="a">
+        <test xmlns="uri:test">
+            <signal-value>${wf:conf('signal-value')}</signal-value>
+            <external-status>${wf:conf('external-status')}</external-status>
+            <error>${wf:conf('error')}</error>
+            <avoid-set-execution-data>${wf:conf('avoid-set-execution-data')}</avoid-set-execution-data>
+            <avoid-set-end-data>${wf:conf('avoid-set-end-data')}</avoid-set-end-data>
+            <running-mode>${wf:conf('running-mode')}</running-mode>
+        </test>
+        <ok to="end"/>
+        <error to="kill"/>
+    </action>
+
+    <kill name="kill">
+        <!-- the message purposefully has an EL syntax error -->
+        <message>${{wf:user()}</message>
+    </kill>
+
+    <end name="end"/>
+
+</workflow-app>

http://git-wip-us.apache.org/repos/asf/oozie/blob/a6d640d5/core/src/test/resources/wf-test-kill-node-message-error2.xml
----------------------------------------------------------------------
diff --git a/core/src/test/resources/wf-test-kill-node-message-error2.xml b/core/src/test/resources/wf-test-kill-node-message-error2.xml
new file mode 100644
index 0000000..7a1e257
--- /dev/null
+++ b/core/src/test/resources/wf-test-kill-node-message-error2.xml
@@ -0,0 +1,42 @@
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you under the Apache License, Version 2.0 (the
+  "License"); you may not use this file except in compliance
+  with the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT 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:0.1" name="test-wf">
+
+    <start to="a"/>
+
+    <action name="a">
+        <test xmlns="uri:test">
+            <signal-value>${wf:conf('signal-value')}</signal-value>
+            <external-status>${wf:conf('external-status')}</external-status>
+            <error>${wf:conf('error')}</error>
+            <avoid-set-execution-data>${wf:conf('avoid-set-execution-data')}</avoid-set-execution-data>
+            <avoid-set-end-data>${wf:conf('avoid-set-end-data')}</avoid-set-end-data>
+            <running-mode>${wf:conf('running-mode')}</running-mode>
+        </test>
+        <ok to="end"/>
+        <error to="kill"/>
+    </action>
+
+    <kill name="kill">
+        <!-- the message purposefully has an undefined EL variable -->
+        <message>${bar}</message>
+    </kill>
+
+    <end name="end"/>
+
+</workflow-app>

http://git-wip-us.apache.org/repos/asf/oozie/blob/a6d640d5/release-log.txt
----------------------------------------------------------------------
diff --git a/release-log.txt b/release-log.txt
index 0339017..39aa295 100644
--- a/release-log.txt
+++ b/release-log.txt
@@ -1,5 +1,6 @@
 -- Oozie 4.3.0 release (trunk - unreleased)
 
+OOZIE-2324 A syntax error in the kill node causes the workflow to get stuck and other problems (rkanter)
 OOZIE-2309 Enable the coord:dateOffset() function in /coordinator-app/datasets/dataset/@initial-instance (kailongs via rohini)
 OOZIE-2305 Compile Oozie with Hive-1.2.0 (raviprak via shwethags)
 OOZIE-2320 TestZKXLogStreamingService.testStreamingWithMultipleOozieServers_coordActionList is failing (rkanter)