You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@oozie.apache.org by vi...@apache.org on 2013/09/06 18:25:40 UTC

svn commit: r1520625 - in /oozie/trunk: ./ core/src/main/java/org/apache/oozie/command/wf/ core/src/test/java/org/apache/oozie/command/wf/ core/src/test/resources/

Author: virag
Date: Fri Sep  6 16:25:39 2013
New Revision: 1520625

URL: http://svn.apache.org/r1520625
Log:
OOZIE-1522 SignalX may try to insert transition for a forked node twice (virag)

Added:
    oozie/trunk/core/src/test/resources/wf-fork.xml
Modified:
    oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/KillXCommand.java
    oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/SignalXCommand.java
    oozie/trunk/core/src/test/java/org/apache/oozie/command/wf/TestSignalXCommand.java
    oozie/trunk/release-log.txt

Modified: oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/KillXCommand.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/KillXCommand.java?rev=1520625&r1=1520624&r2=1520625&view=diff
==============================================================================
--- oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/KillXCommand.java (original)
+++ oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/KillXCommand.java Fri Sep  6 16:25:39 2013
@@ -150,7 +150,8 @@ public class KillXCommand extends Workfl
                         || action.getStatus() == WorkflowActionBean.Status.START_RETRY
                         || action.getStatus() == WorkflowActionBean.Status.START_MANUAL
                         || action.getStatus() == WorkflowActionBean.Status.END_RETRY
-                        || action.getStatus() == WorkflowActionBean.Status.END_MANUAL) {
+                        || action.getStatus() == WorkflowActionBean.Status.END_MANUAL
+                        || action.getStatus() == WorkflowActionBean.Status.USER_RETRY) {
 
                     action.setStatus(WorkflowActionBean.Status.KILLED);
                     action.resetPending();

Modified: oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/SignalXCommand.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/SignalXCommand.java?rev=1520625&r1=1520624&r2=1520625&view=diff
==============================================================================
--- oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/SignalXCommand.java (original)
+++ oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/SignalXCommand.java Fri Sep  6 16:25:39 2013
@@ -306,6 +306,14 @@ public class SignalXCommand extends Work
                         queue(new SignalXCommand(jobId, oldAction.getId()));
                     }
                     else {
+                        try {
+                            // Make sure that transition node for a forked action
+                            // is inserted only once
+                            jpaService.execute(new WorkflowActionGetJPAExecutor(newAction.getId()));
+                            continue;
+                        }
+                        catch (JPAExecutorException jee) {
+                        }
                         checkForSuspendNode(newAction);
                         newAction.setPending();
                         String actionSlaXml = getActionSLAXml(newAction.getName(), workflowInstance.getApp()

Modified: oozie/trunk/core/src/test/java/org/apache/oozie/command/wf/TestSignalXCommand.java
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/test/java/org/apache/oozie/command/wf/TestSignalXCommand.java?rev=1520625&r1=1520624&r2=1520625&view=diff
==============================================================================
--- oozie/trunk/core/src/test/java/org/apache/oozie/command/wf/TestSignalXCommand.java (original)
+++ oozie/trunk/core/src/test/java/org/apache/oozie/command/wf/TestSignalXCommand.java Fri Sep  6 16:25:39 2013
@@ -17,6 +17,7 @@
  */
 package org.apache.oozie.command.wf;
 
+import java.io.ByteArrayOutputStream;
 import java.io.File;
 import java.io.OutputStreamWriter;
 import java.io.Reader;
@@ -24,28 +25,75 @@ import java.io.Writer;
 import java.util.Properties;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.log4j.Appender;
+import org.apache.log4j.Layout;
+import org.apache.log4j.Logger;
+import org.apache.log4j.SimpleLayout;
+import org.apache.log4j.WriterAppender;
+import org.apache.oozie.DagEngine;
 import org.apache.oozie.client.OozieClient;
 import org.apache.oozie.client.WorkflowAction;
 import org.apache.oozie.client.WorkflowJob;
 import org.apache.oozie.local.LocalOozie;
+import org.apache.oozie.service.Services;
 import org.apache.oozie.test.XDataTestCase;
 import org.apache.oozie.util.IOUtils;
+import org.apache.oozie.util.XConfiguration;
+import org.apache.oozie.workflow.lite.LiteWorkflowAppParser;
 
 public class TestSignalXCommand extends XDataTestCase {
 
+    private Services services;
     @Override
     protected void setUp() throws Exception {
         super.setUp();
-        LocalOozie.start();
+        services = new Services();
+        services.getConf().setBoolean(LiteWorkflowAppParser.VALIDATE_FORK_JOIN, false);
+        services.init();
+
     }
 
     @Override
     protected void tearDown() throws Exception {
-        LocalOozie.stop();
+        services.destroy();
         super.tearDown();
     }
 
+    public void testJoinFail() throws Exception {
+        Logger logger = Logger.getLogger(SignalXCommand.class);
+        ByteArrayOutputStream out = new ByteArrayOutputStream();
+        Layout layout = new SimpleLayout();
+        Appender appender = new WriterAppender(layout, out);
+        logger.addAppender(appender);
+
+        FileSystem fs = getFileSystem();
+        Path appPath = new Path(getFsTestCaseDir(), "app");
+        fs.mkdirs(appPath);
+        Reader reader = IOUtils.getResourceAsReader("wf-fork.xml", -1);
+        Writer writer = new OutputStreamWriter(fs.create(new Path(appPath, "workflow.xml")));
+        IOUtils.copyCharStream(reader, writer);
+        writer.close();
+        reader.close();
+
+        final DagEngine engine = new DagEngine("u");
+
+        XConfiguration conf = new XConfiguration();
+        conf.set(OozieClient.APP_PATH, appPath.toString() + File.separator + "workflow.xml");
+        conf.set(OozieClient.USER_NAME, getTestUser());
+
+        final String jobId = engine.submitJob(conf, false);
+
+
+        assertNotNull(jobId);
+        engine.start(jobId);
+
+        Thread.sleep(2000);
+        assertFalse(out.toString().contains("EntityExistsException"));
+    }
+
     public void testSuspendPoints() throws Exception {
+        services.destroy();
+        LocalOozie.start();
         FileSystem fs = getFileSystem();
         Path appPath = new Path(getFsTestCaseDir(), "app");
         fs.mkdirs(appPath);
@@ -103,9 +151,12 @@ public class TestSignalXCommand extends 
                 new String[]{},
                 new String[]{":start:", "action1", "action2", "decision1", "action3", "fork1", "action4a", "action4b", "action4c",
                              "join1", "end"});
+        LocalOozie.stop();
     }
 
     public void testSuspendPointsAll() throws Exception {
+        services.destroy();
+        LocalOozie.start();
         FileSystem fs = getFileSystem();
         Path appPath = new Path(getFsTestCaseDir(), "app");
         fs.mkdirs(appPath);
@@ -179,6 +230,7 @@ public class TestSignalXCommand extends 
                 new String[]{},
                 new String[]{":start:", "action1", "action2", "decision1", "action3", "fork1", "action4a", "action4b", "action4c",
                              "join1", "end"});
+        LocalOozie.stop();
     }
 
     private void checkSuspendActions(WorkflowJob wf, final OozieClient oc, final String jobId, final WorkflowJob.Status status,

Added: oozie/trunk/core/src/test/resources/wf-fork.xml
URL: http://svn.apache.org/viewvc/oozie/trunk/core/src/test/resources/wf-fork.xml?rev=1520625&view=auto
==============================================================================
--- oozie/trunk/core/src/test/resources/wf-fork.xml (added)
+++ oozie/trunk/core/src/test/resources/wf-fork.xml Fri Sep  6 16:25:39 2013
@@ -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.4" name="wf-fork">
+    <start to="fork1"/>
+
+     <fork name="fork1">
+        <path start="action1"/>
+        <path start="action2"/>
+    </fork>
+    <action name="action1">
+        <fs>
+        </fs>
+        <ok to="kill"/>
+        <error to="kill"/>
+    </action>
+    <action name="action2">
+        <fs>
+        </fs>
+        <ok to="kill"/>
+        <error to="kill"/>
+    </action>
+    <join name="join1" to="end"/>
+    <kill name="kill">
+        <message>killed</message>
+    </kill>
+    <end name="end"/>
+</workflow-app>

Modified: oozie/trunk/release-log.txt
URL: http://svn.apache.org/viewvc/oozie/trunk/release-log.txt?rev=1520625&r1=1520624&r2=1520625&view=diff
==============================================================================
--- oozie/trunk/release-log.txt (original)
+++ oozie/trunk/release-log.txt Fri Sep  6 16:25:39 2013
@@ -1,5 +1,6 @@
 -- Oozie 4.1.0 release (trunk - unreleased)
 
+OOZIE-1522 SignalX may try to insert transition for a forked node twice (virag)
 OOZIE-1461 provide an option to auto-deploy launcher jar onto HDFS system libpath (ryota,virag via virag)
 OOZIE-1520 Sequencefile Reader fails to use doas for reading action data file (rohini,mona via mona)
 OOZIE-1513 Workflow stays in running if Fork/join validation or loop detection fails (mona)