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 2012/07/31 04:26:36 UTC

svn commit: r1367394 - in /incubator/oozie/trunk: ./ core/src/main/java/org/apache/oozie/command/wf/ core/src/main/java/org/apache/oozie/executor/jpa/ core/src/test/java/org/apache/oozie/executor/jpa/

Author: virag
Date: Tue Jul 31 02:26:36 2012
New Revision: 1367394

URL: http://svn.apache.org/viewvc?rev=1367394&view=rev
Log:
OOZIE-903 Workflow action status Ok but the workflow job remains in RUNNING (virag)

Added:
    incubator/oozie/trunk/core/src/main/java/org/apache/oozie/executor/jpa/BulkUpdateInsertJPAExecutor.java
    incubator/oozie/trunk/core/src/test/java/org/apache/oozie/executor/jpa/TestBulkUpdateInsertJPAExecutor.java
Modified:
    incubator/oozie/trunk/core/src/main/java/org/apache/oozie/command/wf/SignalXCommand.java
    incubator/oozie/trunk/release-log.txt

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=1367394&r1=1367393&r2=1367394&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 Tue Jul 31 02:26:36 2012
@@ -6,9 +6,9 @@
  * 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.
@@ -21,6 +21,7 @@ import org.apache.hadoop.conf.Configurat
 import org.apache.oozie.client.WorkflowJob;
 import org.apache.oozie.client.SLAEvent.SlaAppType;
 import org.apache.oozie.client.SLAEvent.Status;
+import org.apache.oozie.client.rest.JsonBean;
 import org.apache.oozie.WorkflowActionBean;
 import org.apache.oozie.WorkflowJobBean;
 import org.apache.oozie.ErrorCode;
@@ -29,12 +30,10 @@ import org.apache.oozie.command.CommandE
 import org.apache.oozie.command.PreconditionException;
 import org.apache.oozie.command.coord.CoordActionUpdateXCommand;
 import org.apache.oozie.command.wf.ActionXCommand.ActionExecutorContext;
+import org.apache.oozie.executor.jpa.BulkUpdateInsertJPAExecutor;
 import org.apache.oozie.executor.jpa.JPAExecutorException;
 import org.apache.oozie.executor.jpa.WorkflowActionGetJPAExecutor;
-import org.apache.oozie.executor.jpa.WorkflowActionInsertJPAExecutor;
-import org.apache.oozie.executor.jpa.WorkflowActionUpdateJPAExecutor;
 import org.apache.oozie.executor.jpa.WorkflowJobGetJPAExecutor;
-import org.apache.oozie.executor.jpa.WorkflowJobUpdateJPAExecutor;
 import org.apache.oozie.service.ELService;
 import org.apache.oozie.service.JPAService;
 import org.apache.oozie.service.SchemaService;
@@ -56,6 +55,7 @@ import org.jdom.Element;
 import org.jdom.Namespace;
 
 import java.io.StringReader;
+import java.util.ArrayList;
 import java.util.Date;
 import java.util.List;
 import java.util.Map;
@@ -69,6 +69,9 @@ public class SignalXCommand extends Work
     private String actionId;
     private WorkflowJobBean wfJob;
     private WorkflowActionBean wfAction;
+    private List<JsonBean> updateList = new ArrayList<JsonBean>();
+    private List<JsonBean> insertList = new ArrayList<JsonBean>();
+
 
     public SignalXCommand(String name, int priority, String jobId) {
         super(name, name, priority);
@@ -170,12 +173,7 @@ public class SignalXCommand extends Work
                 wfAction.setTransition(workflowInstance.getTransition(wfAction.getName()));
                 queue(new NotificationXCommand(wfJob, wfAction));
             }
-            try {
-                jpaService.execute(new WorkflowActionUpdateJPAExecutor(wfAction));
-            }
-            catch (JPAExecutorException je) {
-                throw new CommandException(je);
-            }
+            updateList.add(wfAction);
         }
 
         if (completed) {
@@ -187,7 +185,7 @@ public class SignalXCommand extends Work
 
                     actionToKill.setPending();
                     actionToKill.setStatus(WorkflowActionBean.Status.KILLED);
-                    jpaService.execute(new WorkflowActionUpdateJPAExecutor(actionToKill));
+                    updateList.add(actionToKill);
                     queue(new ActionKillXCommand(actionToKill.getId(), actionToKill.getType()));
                 }
 
@@ -199,7 +197,7 @@ public class SignalXCommand extends Work
                     queue(new NotificationXCommand(wfJob, actionToFail));
                     SLADbXOperations.writeStausEvent(wfAction.getSlaXml(), wfAction.getId(), Status.FAILED,
                             SlaAppType.WORKFLOW_ACTION);
-                    jpaService.execute(new WorkflowActionUpdateJPAExecutor(actionToFail));
+                    updateList.add(actionToFail);
                 }
             }
             catch (JPAExecutorException je) {
@@ -248,10 +246,7 @@ public class SignalXCommand extends Work
                         else {
                             wfAction.setErrorInfo(ErrorCode.E0729.toString(), actionConf);
                         }
-                        jpaService.execute(new WorkflowActionUpdateJPAExecutor(wfAction));
-                    }
-                    catch (JPAExecutorException je) {
-                        throw new CommandException(je);
+                        updateList.add(wfAction);
                     }
                     catch (Exception ex) {
                         LOG.warn("Exception in SignalXCommand ", ex.getMessage(), ex);
@@ -276,7 +271,7 @@ public class SignalXCommand extends Work
                         oldAction = jpaService.execute(new WorkflowActionGetJPAExecutor(newAction.getId()));
 
                         oldAction.setPending();
-                        jpaService.execute(new WorkflowActionUpdateJPAExecutor(oldAction));
+                        updateList.add(oldAction);
 
                         queue(new SignalXCommand(jobId, oldAction.getId()));
                     }
@@ -285,7 +280,7 @@ public class SignalXCommand extends Work
                         String actionSlaXml = getActionSLAXml(newAction.getName(), workflowInstance.getApp()
                                 .getDefinition(), wfJob.getConf());
                         newAction.setSlaXml(actionSlaXml);
-                        jpaService.execute(new WorkflowActionInsertJPAExecutor(newAction));
+                        insertList.add(newAction);
                         LOG.debug("SignalXCommand: Name: "+ newAction.getName() + ", Id: " +newAction.getId() + ", Authcode:" + newAction.getCred());
                         queue(new ActionStartXCommand(newAction.getId(), newAction.getType()));
                     }
@@ -297,7 +292,10 @@ public class SignalXCommand extends Work
         }
 
         try {
-            jpaService.execute(new WorkflowJobUpdateJPAExecutor(wfJob));
+            wfJob.setLastModifiedTime(new Date());
+            updateList.add(wfJob);
+            // call JPAExecutor to do the bulk writes
+            jpaService.execute(new BulkUpdateInsertJPAExecutor(updateList, insertList));
         }
         catch (JPAExecutorException je) {
             throw new CommandException(je);

Added: incubator/oozie/trunk/core/src/main/java/org/apache/oozie/executor/jpa/BulkUpdateInsertJPAExecutor.java
URL: http://svn.apache.org/viewvc/incubator/oozie/trunk/core/src/main/java/org/apache/oozie/executor/jpa/BulkUpdateInsertJPAExecutor.java?rev=1367394&view=auto
==============================================================================
--- incubator/oozie/trunk/core/src/main/java/org/apache/oozie/executor/jpa/BulkUpdateInsertJPAExecutor.java (added)
+++ incubator/oozie/trunk/core/src/main/java/org/apache/oozie/executor/jpa/BulkUpdateInsertJPAExecutor.java Tue Jul 31 02:26:36 2012
@@ -0,0 +1,103 @@
+/**
+ * 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.executor.jpa;
+
+import java.util.Collection;
+
+import javax.persistence.EntityManager;
+import org.apache.oozie.ErrorCode;
+import org.apache.oozie.FaultInjection;
+import org.apache.oozie.client.rest.JsonBean;
+import org.apache.oozie.util.ParamChecker;
+
+/**
+ * Class for inserting and updating beans in bulk
+ * @param <T>
+*/
+public class BulkUpdateInsertJPAExecutor implements JPAExecutor<Void> {
+
+    private Collection<JsonBean> updateList;
+    private Collection<JsonBean> insertList;
+
+    /**
+     * Initialize the JPAExecutor using the insert and update list of JSON beans
+     * @param updateList
+     * @param insertList
+     */
+    public BulkUpdateInsertJPAExecutor(Collection<JsonBean> updateList, Collection<JsonBean> insertList) {
+        this.updateList = updateList;
+        this.insertList = insertList;
+    }
+
+    public BulkUpdateInsertJPAExecutor() {
+    }
+
+    /**
+     * Sets the update list for JSON bean
+     * @param updateList
+     */
+    public void setUpdateList(Collection<JsonBean> updateList) {
+        this.updateList = updateList;
+    }
+
+    /**
+     * Sets the insert list for JSON bean
+     * @param insertList
+     */
+    public void setInsertList(Collection<JsonBean> insertList) {
+        this.insertList = insertList;
+    }
+
+
+    /* (non-Javadoc)
+     * @see org.apache.oozie.executor.jpa.JPAExecutor#getName()
+     */
+    @Override
+    public String getName() {
+        return "BulkUpdateInsertJPAExecutor";
+    }
+
+    /* (non-Javadoc)
+     * @see org.apache.oozie.executor.jpa.JPAExecutor#execute(javax.persistence.EntityManager)
+     */
+    @Override
+    public Void execute(EntityManager em) throws JPAExecutorException {
+        try {
+            if (updateList!= null){
+                for (JsonBean entity: updateList){
+                    ParamChecker.notNull(entity, "JsonBean");
+                    em.merge(entity);
+                }
+            }
+            // Only used by test cases to check for rollback of transaction
+            FaultInjection.activate("org.apache.oozie.command.SkipCommitFaultInjection");
+            if (insertList!= null){
+                for (JsonBean entity: insertList){
+                    ParamChecker.notNull(entity, "JsonBean");
+                    em.persist(entity);
+                }
+            }
+            return null;
+        }
+        catch (Exception e) {
+            throw new JPAExecutorException(ErrorCode.E0603, e);
+        }
+    }
+
+
+}

Added: incubator/oozie/trunk/core/src/test/java/org/apache/oozie/executor/jpa/TestBulkUpdateInsertJPAExecutor.java
URL: http://svn.apache.org/viewvc/incubator/oozie/trunk/core/src/test/java/org/apache/oozie/executor/jpa/TestBulkUpdateInsertJPAExecutor.java?rev=1367394&view=auto
==============================================================================
--- incubator/oozie/trunk/core/src/test/java/org/apache/oozie/executor/jpa/TestBulkUpdateInsertJPAExecutor.java (added)
+++ incubator/oozie/trunk/core/src/test/java/org/apache/oozie/executor/jpa/TestBulkUpdateInsertJPAExecutor.java Tue Jul 31 02:26:36 2012
@@ -0,0 +1,254 @@
+/**
+ * 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.executor.jpa;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.oozie.CoordinatorJobBean;
+import org.apache.oozie.ErrorCode;
+import org.apache.oozie.FaultInjection;
+import org.apache.oozie.WorkflowActionBean;
+import org.apache.oozie.WorkflowJobBean;
+import org.apache.oozie.client.CoordinatorJob;
+import org.apache.oozie.client.Job;
+import org.apache.oozie.client.OozieClient;
+import org.apache.oozie.client.WorkflowAction;
+import org.apache.oozie.client.WorkflowJob;
+import org.apache.oozie.client.rest.JsonBean;
+import org.apache.oozie.command.SkipCommitFaultInjection;
+import org.apache.oozie.executor.jpa.WorkflowJobGetJPAExecutor;
+import org.apache.oozie.service.JPAService;
+import org.apache.oozie.service.Services;
+import org.apache.oozie.test.XDataTestCase;
+import org.apache.oozie.workflow.WorkflowApp;
+import org.apache.oozie.workflow.WorkflowInstance;
+import org.apache.oozie.workflow.lite.EndNodeDef;
+import org.apache.oozie.workflow.lite.LiteWorkflowApp;
+import org.apache.oozie.workflow.lite.StartNodeDef;
+
+public class TestBulkUpdateInsertJPAExecutor extends XDataTestCase {
+    Services services;
+
+    @Override
+    protected void setUp() throws Exception {
+        super.setUp();
+        services = new Services();
+        services.init();
+        cleanUpDBTables();
+    }
+
+    @Override
+    protected void tearDown() throws Exception {
+        services.destroy();
+        super.tearDown();
+    }
+
+    /**
+     * Test bulk updates by updating coordinator job, workflow job and workflow action
+     * @throws Exception
+     */
+    public void testUpdates() throws Exception {
+        CoordinatorJobBean coordJob = addRecordToCoordJobTable(CoordinatorJob.Status.PREP, true, true);
+        WorkflowJobBean wfJob = addRecordToWfJobTable(WorkflowJob.Status.PREP, WorkflowInstance.Status.PREP);
+        WorkflowActionBean action = addRecordToWfActionTable(wfJob.getId(), "1", WorkflowAction.Status.PREP);
+        JPAService jpaService = Services.get().get(JPAService.class);
+        assertNotNull(jpaService);
+
+        // update the status
+        coordJob.setStatus(CoordinatorJob.Status.RUNNING);
+        wfJob.setStatus(WorkflowJob.Status.SUCCEEDED);
+        action.setStatus(WorkflowAction.Status.RUNNING);
+
+        List<JsonBean> updateList = new ArrayList<JsonBean>();
+        // update the list for doing bulk writes
+        updateList.add(coordJob);
+        updateList.add(wfJob);
+        updateList.add(action);
+        BulkUpdateInsertJPAExecutor bulkUpdateCmd = new BulkUpdateInsertJPAExecutor();
+        bulkUpdateCmd.setUpdateList(updateList);
+        jpaService.execute(bulkUpdateCmd);
+
+        // check for expected status after running bulkUpdateJPA
+        coordJob = jpaService.execute(new CoordJobGetJPAExecutor(coordJob.getId()));
+        assertEquals("RUNNING", coordJob.getStatusStr());
+
+        wfJob = jpaService.execute(new WorkflowJobGetJPAExecutor(wfJob.getId()));
+        assertEquals("SUCCEEDED", wfJob.getStatusStr());
+
+        WorkflowActionBean action2 = jpaService.execute(new WorkflowActionGetJPAExecutor(action.getId()));
+        assertEquals(WorkflowAction.Status.RUNNING, action2.getStatus());
+
+    }
+
+    /**
+     * Test bulk inserts by inserting a workflow job and two workflow actions
+     * @throws Exception
+     */
+    public void testInserts() throws Exception{
+        WorkflowApp app = new LiteWorkflowApp("testApp", "<workflow-app/>", new StartNodeDef("end"))
+                .addNode(new EndNodeDef("end"));
+        Configuration conf = new Configuration();
+        Path appUri = new Path(getAppPath(), "workflow.xml");
+        conf.set(OozieClient.APP_PATH, appUri.toString());
+        conf.set(OozieClient.LOG_TOKEN, "testToken");
+        conf.set(OozieClient.USER_NAME, getTestUser());
+
+        WorkflowJobBean job = createWorkflow(app, conf, "auth", WorkflowJob.Status.PREP, WorkflowInstance.Status.PREP);
+        WorkflowActionBean action1 = createWorkflowAction(job.getId(), "1", WorkflowAction.Status.PREP);
+        WorkflowActionBean action2 = createWorkflowAction(job.getId(), "2", WorkflowAction.Status.PREP);
+
+        List<JsonBean> insertList = new ArrayList<JsonBean>();
+        // insert one workflow job and two actions
+        insertList.add(action1);
+        insertList.add(action2);
+        insertList.add(job);
+
+        JPAService jpaService = Services.get().get(JPAService.class);
+        assertNotNull(jpaService);
+        BulkUpdateInsertJPAExecutor bulkInsertCmd = new BulkUpdateInsertJPAExecutor();
+        bulkInsertCmd.setInsertList(insertList);
+        jpaService.execute(bulkInsertCmd);
+
+        // check for expected status after running bulkUpdateJPA
+        WorkflowActionGetJPAExecutor actionGetCmd = new WorkflowActionGetJPAExecutor(action1.getId());
+        action1 = jpaService.execute(actionGetCmd);
+        assertEquals("PREP", action1.getStatusStr());
+
+        actionGetCmd = new WorkflowActionGetJPAExecutor(action2.getId());
+        action2 = jpaService.execute(actionGetCmd);
+        assertEquals("PREP", action2.getStatusStr());
+
+        WorkflowJobGetJPAExecutor wfGetCmd = new WorkflowJobGetJPAExecutor(job.getId());
+        job = jpaService.execute(wfGetCmd);
+        assertEquals("PREP", job.getStatusStr());
+
+    }
+
+    /**
+     * Test bulk inserts and updates by inserting wf actions and updating
+     * coordinator and workflow jobs
+     *
+     * @throws Exception
+     */
+    public void testBulkInsertUpdates() throws Exception{
+        CoordinatorJobBean coordJob = addRecordToCoordJobTable(CoordinatorJob.Status.PREP, true, true);
+        WorkflowJobBean job = addRecordToWfJobTable(WorkflowJob.Status.PREP, WorkflowInstance.Status.PREP);
+        WorkflowActionBean action1 = createWorkflowAction(job.getId(), "1", WorkflowAction.Status.PREP);
+        WorkflowActionBean action2 = createWorkflowAction(job.getId(), "2", WorkflowAction.Status.PREP);
+
+        job.setStatus(WorkflowJob.Status.RUNNING);
+        coordJob.setStatus(Job.Status.SUCCEEDED);
+        List<JsonBean> insertList = new ArrayList<JsonBean>();
+        // Add two actions to insert list
+        insertList.add(action1);
+        insertList.add(action2);
+
+        List<JsonBean> updateList = new ArrayList<JsonBean>();
+        //Add two jobs to update list
+        updateList.add(coordJob);
+        updateList.add(job);
+
+        JPAService jpaService = Services.get().get(JPAService.class);
+        assertNotNull(jpaService);
+
+        BulkUpdateInsertJPAExecutor bulkUpdateCmd = new BulkUpdateInsertJPAExecutor(updateList, insertList);
+        jpaService.execute(bulkUpdateCmd);
+
+        coordJob = jpaService.execute(new CoordJobGetJPAExecutor(coordJob.getId()));
+        assertEquals("SUCCEEDED", coordJob.getStatusStr());
+
+        WorkflowJobGetJPAExecutor wfGetCmd = new WorkflowJobGetJPAExecutor(job.getId());
+        WorkflowJobBean wfBean = jpaService.execute(wfGetCmd);
+        assertEquals("RUNNING", wfBean.getStatusStr());
+
+        WorkflowActionGetJPAExecutor actionGetCmd = new WorkflowActionGetJPAExecutor(action1.getId());
+        action1 = jpaService.execute(actionGetCmd);
+        assertEquals("PREP", action1.getStatusStr());
+
+        actionGetCmd = new WorkflowActionGetJPAExecutor(action2.getId());
+        action2 = jpaService.execute(actionGetCmd);
+        assertEquals("PREP", action2.getStatusStr());
+    }
+
+    /**
+     * Test bulk inserts and updates rollback
+     *
+     * @throws Exception
+     */
+    public void testBulkInsertUpdatesRollback() throws Exception{
+        WorkflowJobBean job = addRecordToWfJobTable(WorkflowJob.Status.PREP, WorkflowInstance.Status.PREP);
+        WorkflowActionBean action1 = createWorkflowAction(job.getId(), "1", WorkflowAction.Status.PREP);
+        WorkflowActionBean action2 = createWorkflowAction(job.getId(), "2", WorkflowAction.Status.PREP);
+
+        job.setStatus(WorkflowJob.Status.RUNNING);
+        List<JsonBean> insertList = new ArrayList<JsonBean>();
+        // Add two actions to insert list
+        insertList.add(action1);
+        insertList.add(action2);
+
+        List<JsonBean> updateList = new ArrayList<JsonBean>();
+        // Add to update list
+        updateList.add(job);
+
+        JPAService jpaService = Services.get().get(JPAService.class);
+        assertNotNull(jpaService);
+
+        BulkUpdateInsertJPAExecutor wfUpdateCmd1 = new BulkUpdateInsertJPAExecutor(updateList, insertList);
+
+        // set fault injection to true, so transaction is roll backed
+        setSystemProperty(FaultInjection.FAULT_INJECTION, "true");
+        setSystemProperty(SkipCommitFaultInjection.ACTION_FAILOVER_FAULT_INJECTION, "true");
+        try {
+            jpaService.execute(wfUpdateCmd1);
+            fail("Expected exception due to commit failure but didn't get any");
+        }
+        catch (Exception e) {
+        }
+        FaultInjection.deactivate("org.apache.oozie.command.SkipCommitFaultInjection");
+
+        // Check whether transactions are rolled back or not
+        WorkflowJobGetJPAExecutor wfGetCmd = new WorkflowJobGetJPAExecutor(job.getId());
+        WorkflowJobBean wfBean = jpaService.execute(wfGetCmd);
+        // status should not be RUNNING
+        assertEquals("PREP", wfBean.getStatusStr());
+
+        WorkflowActionGetJPAExecutor actionGetCmd = new WorkflowActionGetJPAExecutor(action1.getId());
+        try {
+            action1 = jpaService.execute(actionGetCmd);
+            fail("Expected exception but didnt get any");
+        }
+        catch (JPAExecutorException jpaee) {
+            assertEquals(ErrorCode.E0605, jpaee.getErrorCode());
+        }
+
+
+        actionGetCmd = new WorkflowActionGetJPAExecutor(action2.getId());
+        try {
+            action2 = jpaService.execute(actionGetCmd);
+            fail("Expected exception but didnt get any");
+        }
+        catch (JPAExecutorException jpaee) {
+            assertEquals(ErrorCode.E0605, jpaee.getErrorCode());
+        }
+
+    }
+
+}

Modified: incubator/oozie/trunk/release-log.txt
URL: http://svn.apache.org/viewvc/incubator/oozie/trunk/release-log.txt?rev=1367394&r1=1367393&r2=1367394&view=diff
==============================================================================
--- incubator/oozie/trunk/release-log.txt (original)
+++ incubator/oozie/trunk/release-log.txt Tue Jul 31 02:26:36 2012
@@ -1,5 +1,6 @@
 -- Oozie 3.3.0 release (trunk - unreleased)
 
+OOZIE-903 Workflow action status 'Ok' but the workflow job remains in 'RUNNING' (virag)
 OOZIE-928 Clarify the documentation for submitting coordinator jobs using web services API (rkanter via virag)
 OOZIE-920 Incorrect error message for multiple start instances in coordinator xml (bcyr via tucu)
 OOZIE-925 Change default logging level for oozie to INFO (rkanter via tucu)