Author: tucu
Date: Thu Mar  1 14:02:02 2012
New Revision: 1295566

URL: http://svn.apache.org/viewvc?rev=1295566&view=rev
Log:
OOZIE-701 Oozie notification URLs don't get replaced with the taken transition 
(tucu)

Modified:
    
incubator/oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/ActionEndCommand.java
    
incubator/oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/ActionEndXCommand.java
    
incubator/oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/SignalCommand.java
    
incubator/oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/SignalXCommand.java
    incubator/oozie/trunk/core/src/test/java/org/apache/oozie/TestDagEngine.java
    incubator/oozie/trunk/release-log.txt

Modified: 
incubator/oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/ActionEndCommand.java
URL: 
http://svn.apache.org/viewvc/incubator/oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/ActionEndCommand.java?rev=1295566&r1=1295565&r2=1295566&view=diff
==============================================================================
--- 
incubator/oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/ActionEndCommand.java
 (original)
+++ 
incubator/oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/ActionEndCommand.java
 Thu Mar  1 14:02:02 2012
@@ -132,7 +132,6 @@ public class ActionEndCommand extends Ac
                         }
                         SLADbOperations.writeStausEvent(action.getSlaXml(), 
action.getId(), store, slaStatus,
                                                         
SlaAppType.WORKFLOW_ACTION);
-                        queueCallable(new NotificationCommand(workflow, 
action));
                         XLog.getLog(getClass()).debug(
                                 "Queuing commands for action=" + id + ", 
status=" + action.getStatus()
                                         + ", Set pending=" + 
action.getPending());

Modified: 
incubator/oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/ActionEndXCommand.java
URL: 
http://svn.apache.org/viewvc/incubator/oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/ActionEndXCommand.java?rev=1295566&r1=1295565&r2=1295566&view=diff
==============================================================================
--- 
incubator/oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/ActionEndXCommand.java
 (original)
+++ 
incubator/oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/ActionEndXCommand.java
 Thu Mar  1 14:02:02 2012
@@ -193,7 +193,6 @@ public class ActionEndXCommand extends A
             }
             if (!shouldHandleUserRetry || !handleUserRetry(wfAction)) {
                 SLADbXOperations.writeStausEvent(wfAction.getSlaXml(), 
wfAction.getId(), slaStatus, SlaAppType.WORKFLOW_ACTION);
-                queue(new NotificationXCommand(wfJob, wfAction));
                 LOG.debug(
                         "Queuing commands for action=" + actionId + ", 
status=" + wfAction.getStatus()
                         + ", Set pending=" + wfAction.getPending());

Modified: 
incubator/oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/SignalCommand.java
URL: 
http://svn.apache.org/viewvc/incubator/oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/SignalCommand.java?rev=1295566&r1=1295565&r2=1295566&view=diff
==============================================================================
--- 
incubator/oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/SignalCommand.java
 (original)
+++ 
incubator/oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/SignalCommand.java
 Thu Mar  1 14:02:02 2012
@@ -125,6 +125,7 @@ public class SignalCommand extends Workf
                         action.resetPending();
                         if (!skipAction) {
                             
action.setTransition(workflowInstance.getTransition(action.getName()));
+                            queueCallable(new NotificationCommand(workflow, 
action));
                         }
                         store.updateAction(action);
                     }
@@ -142,6 +143,7 @@ public class SignalCommand extends Workf
                             WorkflowActionBean actionToFail = 
store.getAction(actionToFailId, false);
                             actionToFail.resetPending();
                             
actionToFail.setStatus(WorkflowActionBean.Status.FAILED);
+                            queueCallable(new NotificationCommand(workflow, 
actionToFail));
                             
SLADbOperations.writeStausEvent(action.getSlaXml(), action.getId(), store, 
Status.FAILED,
                                                             
SlaAppType.WORKFLOW_ACTION);
                             store.updateAction(actionToFail);

Modified: 
incubator/oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/SignalXCommand.java
URL: 
http://svn.apache.org/viewvc/incubator/oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/SignalXCommand.java?rev=1295566&r1=1295565&r2=1295566&view=diff
==============================================================================
--- 
incubator/oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/SignalXCommand.java
 (original)
+++ 
incubator/oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/SignalXCommand.java
 Thu Mar  1 14:02:02 2012
@@ -168,6 +168,7 @@ public class SignalXCommand extends Work
             wfAction.resetPending();
             if (!skipAction) {
                 
wfAction.setTransition(workflowInstance.getTransition(wfAction.getName()));
+                queue(new NotificationXCommand(wfJob, wfAction));
             }
             try {
                 jpaService.execute(new 
WorkflowActionUpdateJPAExecutor(wfAction));
@@ -195,6 +196,7 @@ public class SignalXCommand extends Work
                             actionToFailId));
                     actionToFail.resetPending();
                     actionToFail.setStatus(WorkflowActionBean.Status.FAILED);
+                    queue(new NotificationXCommand(wfJob, actionToFail));
                     SLADbXOperations.writeStausEvent(wfAction.getSlaXml(), 
wfAction.getId(), Status.FAILED,
                             SlaAppType.WORKFLOW_ACTION);
                     jpaService.execute(new 
WorkflowActionUpdateJPAExecutor(actionToFail));

Modified: 
incubator/oozie/trunk/core/src/test/java/org/apache/oozie/TestDagEngine.java
URL: 
http://svn.apache.org/viewvc/incubator/oozie/trunk/core/src/test/java/org/apache/oozie/TestDagEngine.java?rev=1295566&r1=1295565&r2=1295566&view=diff
==============================================================================
--- 
incubator/oozie/trunk/core/src/test/java/org/apache/oozie/TestDagEngine.java 
(original)
+++ 
incubator/oozie/trunk/core/src/test/java/org/apache/oozie/TestDagEngine.java 
Thu Mar  1 14:02:02 2012
@@ -24,12 +24,18 @@ import org.apache.oozie.service.ActionSe
 import org.apache.oozie.service.SchemaService;
 import org.apache.oozie.service.WorkflowStoreService;
 import org.apache.oozie.service.Services;
+import org.apache.oozie.test.EmbeddedServletContainer;
 import org.apache.oozie.test.XTestCase;
 import org.apache.oozie.util.IOUtils;
 import org.apache.oozie.util.XConfiguration;
 
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
 import java.io.File;
 import java.io.FileWriter;
+import java.io.IOException;
 import java.io.Reader;
 import java.io.Writer;
 import java.io.StringReader;
@@ -38,10 +44,36 @@ import java.io.FileOutputStream;
 import java.util.List;
 
 public class TestDagEngine extends XTestCase {
+    private EmbeddedServletContainer container;
     private Services services;
 
+    public static class CallbackServlet extends HttpServlet {
+        public static volatile String JOB_ID = null;
+        public static String NODE_NAME = null;
+        public static String STATUS = null;
+
+        public static void reset() {
+            JOB_ID = null;
+            NODE_NAME = null;
+            STATUS = null;
+        }
+
+        @Override
+        protected void doGet(HttpServletRequest req, HttpServletResponse resp) 
throws ServletException, IOException {
+            JOB_ID = req.getParameter("jobId");
+            NODE_NAME = req.getParameter("nodeName");
+            STATUS = req.getParameter("status");
+            resp.setStatus(HttpServletResponse.SC_OK);
+        }
+
+    }
     protected void setUp() throws Exception {
         super.setUp();
+        CallbackServlet.reset();
+        container = new EmbeddedServletContainer("oozie");
+        container.addServletEndpoint("/callback", CallbackServlet.class);
+        container.start();
+
         setSystemProperty(SchemaService.WF_CONF_EXT_SCHEMAS, 
"wf-ext-schema.xsd");
         services = new Services();
         cleanUpDB(services.getConf());
@@ -51,6 +83,7 @@ public class TestDagEngine extends XTest
 
     protected void tearDown() throws Exception {
         services.destroy();
+        container.stop();
         super.tearDown();
     }
 
@@ -74,6 +107,8 @@ public class TestDagEngine extends XTest
         conf.set(OozieClient.GROUP_NAME, getTestGroup());
         injectKerberosInfo(conf);
         conf.set(OozieClient.LOG_TOKEN, "t");
+        conf.set(OozieClient.ACTION_NOTIFICATION_URL, 
container.getServletURL("/callback") +
+                                                      
"?jobId=$jobId&status=$status&nodeName=$nodeName");
         conf.set("signal-value", "OK");
         conf.set("external-status", "ok");
         conf.set("error", "end.error");
@@ -100,6 +135,14 @@ public class TestDagEngine extends XTest
             }
         });
         assertEquals(WorkflowJob.Status.KILLED, 
engine.getJob(jobId1).getStatus());
+        waitFor(5000, new Predicate() {
+            public boolean evaluate() throws Exception {
+                return CallbackServlet.JOB_ID != null;
+            }
+        });
+        assertEquals(wf.getId(), CallbackServlet.JOB_ID);
+        assertEquals("a", CallbackServlet.NODE_NAME);
+        assertEquals("T:kill", CallbackServlet.STATUS);
     }
 
     public void testJobDefinition() throws Exception {

Modified: incubator/oozie/trunk/release-log.txt
URL: 
http://svn.apache.org/viewvc/incubator/oozie/trunk/release-log.txt?rev=1295566&r1=1295565&r2=1295566&view=diff
==============================================================================
--- incubator/oozie/trunk/release-log.txt (original)
+++ incubator/oozie/trunk/release-log.txt Thu Mar  1 14:02:02 2012
@@ -1,5 +1,6 @@
 -- Oozie 3.2.0 release
 
+OOZIE-701 Oozie notification URLs don't get replaced with the taken transition 
(tucu)
 OOZIE-724 TestClassUtils fails as looks for hadoop-core (tucu)
 OOZIE-725 Increase JVM MaxPermSize value for the testcases (tucu)
 OOZIE-727 make test timeout value configurable via -D (tucu)


Reply via email to