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/08/03 09:38:01 UTC

svn commit: r1368811 [3/3] - in /incubator/oozie/trunk: ./ core/src/main/java/org/apache/oozie/ core/src/main/java/org/apache/oozie/command/ core/src/main/java/org/apache/oozie/command/bundle/ core/src/main/java/org/apache/oozie/command/coord/ core/src...

Modified: incubator/oozie/trunk/core/src/test/java/org/apache/oozie/service/TestStatusTransitService.java
URL: http://svn.apache.org/viewvc/incubator/oozie/trunk/core/src/test/java/org/apache/oozie/service/TestStatusTransitService.java?rev=1368811&r1=1368810&r2=1368811&view=diff
==============================================================================
--- incubator/oozie/trunk/core/src/test/java/org/apache/oozie/service/TestStatusTransitService.java (original)
+++ incubator/oozie/trunk/core/src/test/java/org/apache/oozie/service/TestStatusTransitService.java Fri Aug  3 07:38:00 2012
@@ -45,6 +45,7 @@ import org.apache.oozie.executor.jpa.Coo
 import org.apache.oozie.executor.jpa.JPAExecutorException;
 import org.apache.oozie.executor.jpa.WorkflowJobGetJPAExecutor;
 import org.apache.oozie.executor.jpa.WorkflowJobInsertJPAExecutor;
+import org.apache.oozie.service.StatusTransitService.StatusTransitRunnable;
 import org.apache.oozie.test.XDataTestCase;
 import org.apache.oozie.util.DateUtils;
 import org.apache.oozie.workflow.WorkflowApp;
@@ -52,7 +53,6 @@ import org.apache.oozie.workflow.Workflo
 import org.apache.oozie.workflow.lite.EndNodeDef;
 import org.apache.oozie.workflow.lite.LiteWorkflowApp;
 import org.apache.oozie.workflow.lite.StartNodeDef;
-import org.apache.oozie.service.StatusTransitService.StatusTransitRunnable;
 
 public class TestStatusTransitService extends XDataTestCase {
     private Services services;
@@ -257,7 +257,7 @@ public class TestStatusTransitService ex
     public void testCoordStatusTransitServiceSuspendedByUser() throws Exception {
         Date start = DateUtils.parseDateUTC("2009-02-01T01:00Z");
         Date end = DateUtils.parseDateUTC("2009-02-02T23:59Z");
-        CoordinatorJobBean job = addRecordToCoordJobTable(CoordinatorJob.Status.SUSPENDED, start, end, true, false, 3);
+        CoordinatorJobBean job = addRecordToCoordJobTable(CoordinatorJob.Status.SUSPENDED, start, end, true, true, 3);
         addRecordToCoordActionTable(job.getId(), 1, CoordinatorAction.Status.SUCCEEDED, "coord-action-get.xml", 0);
         addRecordToCoordActionTable(job.getId(), 2, CoordinatorAction.Status.SUCCEEDED, "coord-action-get.xml", 0);
         addRecordToCoordActionTable(job.getId(), 3, CoordinatorAction.Status.SUCCEEDED, "coord-action-get.xml", 0);
@@ -268,7 +268,7 @@ public class TestStatusTransitService ex
 
         Runnable runnable = new StatusTransitRunnable();
         runnable.run();
-        waitFor(5 * 1000, new Predicate() {
+        waitFor(10 * 1000, new Predicate() {
             public boolean evaluate() throws Exception {
                 CoordinatorJobBean coordJob = jpaService.execute(new CoordJobGetJPAExecutor(jobId));
                 return coordJob.isPending() == false;
@@ -278,20 +278,23 @@ public class TestStatusTransitService ex
         CoordJobGetJPAExecutor coordGetCmd = new CoordJobGetJPAExecutor(job.getId());
         job = jpaService.execute(coordGetCmd);
         assertFalse(job.isPending());
+        assertEquals(Job.Status.SUCCEEDED, job.getStatus());
     }
 
     /**
-     * Test : coord job suspended by all coord actions are suspended - pending update to false
+     * Test : coord actions suspended and 1 succeeded - check status change to SUSPENDED and pending update to false
      *
      * @throws Exception
      */
     public void testCoordStatusTransitServiceSuspendedBottomUp() throws Exception {
         Date start = DateUtils.parseDateUTC("2009-02-01T01:00Z");
         Date end = DateUtils.parseDateUTC("2009-02-02T23:59Z");
-        CoordinatorJobBean job = addRecordToCoordJobTable(CoordinatorJob.Status.RUNNING, start, end, true, true, 3);
+        CoordinatorJobBean job = addRecordToCoordJobTable(CoordinatorJob.Status.RUNNING, start, end, true, true, 4);
         addRecordToCoordActionTable(job.getId(), 1, CoordinatorAction.Status.SUSPENDED, "coord-action-get.xml", 0);
         addRecordToCoordActionTable(job.getId(), 2, CoordinatorAction.Status.SUSPENDED, "coord-action-get.xml", 0);
         addRecordToCoordActionTable(job.getId(), 3, CoordinatorAction.Status.SUSPENDED, "coord-action-get.xml", 0);
+        addRecordToCoordActionTable(job.getId(), 4, CoordinatorAction.Status.SUCCEEDED, "coord-action-get.xml", 0);
+
 
         final String jobId = job.getId();
         final JPAService jpaService = Services.get().get(JPAService.class);
@@ -314,6 +317,44 @@ public class TestStatusTransitService ex
     }
 
     /**
+     * Test : all coord actions suspended except one which is killed - check status change to SUSPENDEDWITHERROR
+     *
+     * @throws Exception
+     */
+    public void testCoordStatusTransitServiceSuspendedWithError() throws Exception {
+        Services.get().destroy();
+        setSystemProperty(StatusTransitService.CONF_BACKWARD_SUPPORT_FOR_STATES_WITHOUT_ERROR, "false");
+        new Services().init();
+        Date start = DateUtils.parseDateUTC("2009-02-01T01:00Z");
+        Date end = DateUtils.parseDateUTC("2009-02-02T23:59Z");
+        CoordinatorJobBean job = addRecordToCoordJobTable(CoordinatorJob.Status.RUNNING, start, end, true, true, 4);
+        addRecordToCoordActionTable(job.getId(), 1, CoordinatorAction.Status.SUSPENDED, "coord-action-get.xml", 0);
+        addRecordToCoordActionTable(job.getId(), 2, CoordinatorAction.Status.SUSPENDED, "coord-action-get.xml", 0);
+        addRecordToCoordActionTable(job.getId(), 3, CoordinatorAction.Status.SUSPENDED, "coord-action-get.xml", 0);
+        addRecordToCoordActionTable(job.getId(), 4, CoordinatorAction.Status.KILLED, "coord-action-get.xml", 0);
+
+
+        final String jobId = job.getId();
+        final JPAService jpaService = Services.get().get(JPAService.class);
+        assertNotNull(jpaService);
+
+        Runnable runnable = new StatusTransitRunnable();
+        runnable.run();
+        // Keeping wait time to 10s to ensure status is updated
+        waitFor(15 * 1000, new Predicate() {
+            public boolean evaluate() throws Exception {
+                CoordinatorJobBean coordJob = jpaService.execute(new CoordJobGetJPAExecutor(jobId));
+                return coordJob.getStatus() == CoordinatorJob.Status.SUSPENDEDWITHERROR;
+            }
+        });
+
+        CoordJobGetJPAExecutor coordGetCmd = new CoordJobGetJPAExecutor(job.getId());
+        job = jpaService.execute(coordGetCmd);
+        assertEquals(CoordinatorJob.Status.SUSPENDEDWITHERROR, job.getStatus());
+        assertFalse(job.isPending());
+    }
+
+    /**
      * Test : Suspend and resume a coordinator job which has finished materialization and all actions are succeeded.
      * </p>
      * Coordinator job changes to succeeded after resume
@@ -353,7 +394,7 @@ public class TestStatusTransitService ex
         Runnable runnable = new StatusTransitRunnable();
         runnable.run();
 
-        waitFor(5 * 1000, new Predicate() {
+        waitFor(10 * 1000, new Predicate() {
             public boolean evaluate() throws Exception {
                 CoordinatorJobBean job = jpaService.execute(new CoordJobGetJPAExecutor(coordJobId));
                 return job.getStatus().equals(Job.Status.SUCCEEDED);
@@ -365,6 +406,7 @@ public class TestStatusTransitService ex
         assertEquals(Job.Status.SUCCEEDED, coordJob1.getStatus());
     }
 
+
     /**
      * Test : all coord actions are running, job pending is reset
      *
@@ -394,20 +436,97 @@ public class TestStatusTransitService ex
         CoordJobGetJPAExecutor coordGetCmd = new CoordJobGetJPAExecutor(job.getId());
         job = jpaService.execute(coordGetCmd);
         assertFalse(job.isPending());
+        assertEquals(job.getStatus(), Job.Status.RUNNING);
     }
 
+
     /**
-     * Test : all coord actions are running, job pending is reset
+     * Test : 2 coord actions are running, 1 suspended and 1 succeeded, check job pending is reset
+     * and status changed to RUNNING
      *
      * @throws Exception
      */
     public void testCoordStatusTransitServiceRunning2() throws Exception {
         Date start = DateUtils.parseDateUTC("2009-02-01T01:00Z");
         Date end = DateUtils.parseDateUTC("2009-02-02T23:59Z");
-        CoordinatorJobBean job = addRecordToCoordJobTable(CoordinatorJob.Status.RUNNING, start, end, true, false, 3);
+        CoordinatorJobBean job = addRecordToCoordJobTable(CoordinatorJob.Status.RUNNINGWITHERROR, start, end, true, false, 4);
         addRecordToCoordActionTable(job.getId(), 1, CoordinatorAction.Status.SUCCEEDED, "coord-action-get.xml", 0);
         addRecordToCoordActionTable(job.getId(), 2, CoordinatorAction.Status.RUNNING, "coord-action-get.xml", 0);
         addRecordToCoordActionTable(job.getId(), 3, CoordinatorAction.Status.RUNNING, "coord-action-get.xml", 0);
+        addRecordToCoordActionTable(job.getId(), 4, CoordinatorAction.Status.SUSPENDED, "coord-action-get.xml", 0);
+
+        final String jobId = job.getId();
+        final JPAService jpaService = Services.get().get(JPAService.class);
+        assertNotNull(jpaService);
+
+        Runnable runnable = new StatusTransitRunnable();
+        runnable.run();
+        waitFor(10 * 1000, new Predicate() {
+            public boolean evaluate() throws Exception {
+                CoordinatorJobBean coordJob = jpaService.execute(new CoordJobGetJPAExecutor(jobId));
+                return coordJob.getStatus() == Job.Status.RUNNING;
+            }
+        });
+
+        CoordJobGetJPAExecutor coordGetCmd = new CoordJobGetJPAExecutor(job.getId());
+        job = jpaService.execute(coordGetCmd);
+        assertFalse(job.isPending());
+        assertEquals(job.getStatus(), Job.Status.RUNNING);
+    }
+
+    /**
+     * Test : Keep the backward support for states on. 2 coord actions are running, 1 killed, check if job pending is reset and state changed to
+     * RUNNING. Make sure the status is not RUNNINGWITHERROR
+     *
+     * @throws Exception
+     */
+    public void testCoordStatusTransitServiceBackwardSupport() throws Exception {
+        Services.get().destroy();
+        setSystemProperty(StatusTransitService.CONF_BACKWARD_SUPPORT_FOR_STATES_WITHOUT_ERROR, "true");
+        new Services().init();
+        Date start = DateUtils.parseDateUTC("2009-02-01T01:00Z");
+        Date end = DateUtils.parseDateUTC("2009-02-02T23:59Z");
+        CoordinatorJobBean job = addRecordToCoordJobTable(CoordinatorJob.Status.RUNNING, start, end, true, false, 3);
+        addRecordToCoordActionTable(job.getId(), 1, CoordinatorAction.Status.KILLED, "coord-action-get.xml", 0);
+        addRecordToCoordActionTable(job.getId(), 2, CoordinatorAction.Status.RUNNING, "coord-action-get.xml", 0);
+        addRecordToCoordActionTable(job.getId(), 3, CoordinatorAction.Status.RUNNING, "coord-action-get.xml", 0);
+
+        final String jobId = job.getId();
+        final JPAService jpaService = Services.get().get(JPAService.class);
+        assertNotNull(jpaService);
+
+        Runnable runnable = new StatusTransitRunnable();
+        runnable.run();
+        waitFor(5 * 1000, new Predicate() {
+            public boolean evaluate() throws Exception {
+                CoordinatorJobBean coordJob = jpaService.execute(new CoordJobGetJPAExecutor(jobId));
+                return coordJob.isPending() == false;
+            }
+        });
+
+        CoordJobGetJPAExecutor coordGetCmd = new CoordJobGetJPAExecutor(job.getId());
+        job = jpaService.execute(coordGetCmd);
+        assertFalse(job.isPending());
+        assertEquals(job.getStatus(), Job.Status.RUNNING);
+    }
+
+
+    /**
+     * Test : 2 coord actions are running, 1 killed, check if job pending is reset and state changed to
+     * RUNNINGWITHERROR
+     *
+     * @throws Exception
+     */
+    public void testCoordStatusTransitServiceRunning3() throws Exception {
+        Services.get().destroy();
+        setSystemProperty(StatusTransitService.CONF_BACKWARD_SUPPORT_FOR_STATES_WITHOUT_ERROR, "false");
+        new Services().init();
+        Date start = DateUtils.parseDateUTC("2009-02-01T01:00Z");
+        Date end = DateUtils.parseDateUTC("2009-02-02T23:59Z");
+        CoordinatorJobBean job = addRecordToCoordJobTable(CoordinatorJob.Status.RUNNING, start, end, true, false, 3);
+        addRecordToCoordActionTable(job.getId(), 1, CoordinatorAction.Status.KILLED, "coord-action-get.xml", 0);
+        addRecordToCoordActionTable(job.getId(), 2, CoordinatorAction.Status.RUNNING, "coord-action-get.xml", 0);
+        addRecordToCoordActionTable(job.getId(), 3, CoordinatorAction.Status.RUNNING, "coord-action-get.xml", 0);
 
         final String jobId = job.getId();
         final JPAService jpaService = Services.get().get(JPAService.class);
@@ -425,8 +544,80 @@ public class TestStatusTransitService ex
         CoordJobGetJPAExecutor coordGetCmd = new CoordJobGetJPAExecutor(job.getId());
         job = jpaService.execute(coordGetCmd);
         assertFalse(job.isPending());
+        assertEquals(job.getStatus(), Job.Status.RUNNINGWITHERROR);
     }
 
+
+
+    /**
+     * Test : all coord actions are running, job pending is reset
+     *
+     * @throws Exception
+     */
+    public void testCoordStatusTransitServicePaused() throws Exception {
+        Services.get().destroy();
+        setSystemProperty(StatusTransitService.CONF_BACKWARD_SUPPORT_FOR_STATES_WITHOUT_ERROR, "false");
+        new Services().init();
+        Date start = DateUtils.parseDateUTC("2009-02-01T01:00Z");
+        Date end = DateUtils.parseDateUTC("2009-02-02T23:59Z");
+        CoordinatorJobBean job = addRecordToCoordJobTable(CoordinatorJob.Status.PAUSED, start, end, true, false, 3);
+        addRecordToCoordActionTable(job.getId(), 1, CoordinatorAction.Status.KILLED, "coord-action-get.xml", 0);
+        addRecordToCoordActionTable(job.getId(), 2, CoordinatorAction.Status.RUNNING, "coord-action-get.xml", 0);
+        addRecordToCoordActionTable(job.getId(), 3, CoordinatorAction.Status.RUNNING, "coord-action-get.xml", 0);
+
+        final String jobId = job.getId();
+        final JPAService jpaService = Services.get().get(JPAService.class);
+        assertNotNull(jpaService);
+
+        Runnable runnable = new StatusTransitRunnable();
+        runnable.run();
+        waitFor(5 * 1000, new Predicate() {
+            public boolean evaluate() throws Exception {
+                CoordinatorJobBean coordJob = jpaService.execute(new CoordJobGetJPAExecutor(jobId));
+                return coordJob.isPending() == false;
+            }
+        });
+
+        CoordJobGetJPAExecutor coordGetCmd = new CoordJobGetJPAExecutor(job.getId());
+        job = jpaService.execute(coordGetCmd);
+        assertFalse(job.isPending());
+        assertEquals(job.getStatus(), Job.Status.PAUSEDWITHERROR);
+    }
+
+    /**
+     * Test : all coord actions are running, job pending is reset
+     *
+     * @throws Exception
+     */
+    public void testCoordStatusTransitServicePausedWithError() throws Exception {
+        Services.get().destroy();
+        setSystemProperty(StatusTransitService.CONF_BACKWARD_SUPPORT_FOR_STATES_WITHOUT_ERROR, "false");
+        new Services().init();
+        Date start = DateUtils.parseDateUTC("2009-02-01T01:00Z");
+        Date end = DateUtils.parseDateUTC("2009-02-02T23:59Z");
+        CoordinatorJobBean job = addRecordToCoordJobTable(CoordinatorJob.Status.PAUSEDWITHERROR, start, end, true, false, 3);
+        addRecordToCoordActionTable(job.getId(), 1, CoordinatorAction.Status.SUCCEEDED, "coord-action-get.xml", 0);
+        addRecordToCoordActionTable(job.getId(), 2, CoordinatorAction.Status.RUNNING, "coord-action-get.xml", 0);
+        addRecordToCoordActionTable(job.getId(), 3, CoordinatorAction.Status.RUNNING, "coord-action-get.xml", 0);
+
+        final String jobId = job.getId();
+        final JPAService jpaService = Services.get().get(JPAService.class);
+        assertNotNull(jpaService);
+
+        Runnable runnable = new StatusTransitRunnable();
+        runnable.run();
+        waitFor(5 * 1000, new Predicate() {
+            public boolean evaluate() throws Exception {
+                CoordinatorJobBean coordJob = jpaService.execute(new CoordJobGetJPAExecutor(jobId));
+                return coordJob.isPending() == false;
+            }
+        });
+
+        CoordJobGetJPAExecutor coordGetCmd = new CoordJobGetJPAExecutor(job.getId());
+        job = jpaService.execute(coordGetCmd);
+        assertFalse(job.isPending());
+        assertEquals(job.getStatus(), Job.Status.PAUSED);
+    }
     /**
      * Tests functionality of the StatusTransitService Runnable command. </p> Insert a coordinator job with RUNNING and
      * pending true and coordinator actions with TIMEDOUT state. Then, runs the StatusTransitService runnable and
@@ -574,8 +765,7 @@ public class TestStatusTransitService ex
         new BundleJobResumeXCommand(bundleId).call();
 
         job = jpaService.execute(bundleJobGetCmd);
-
-        assertTrue(job.getStatus() == Job.Status.RUNNING || job.getStatus() == Job.Status.SUCCEEDED);
+        assertTrue(job.getStatus() == Job.Status.SUCCEEDED || job.getStatus() == Job.Status.RUNNING);
 
         Thread.sleep(3000);
 
@@ -610,8 +800,8 @@ public class TestStatusTransitService ex
         addRecordToBundleActionTable(bundleId, "action1", 1, Job.Status.KILLED);
         addRecordToBundleActionTable(bundleId, "action2", 1, Job.Status.KILLED);
 
-        addRecordToCoordJobTableWithBundle(bundleId, "action1", CoordinatorJob.Status.RUNNING, false, false, 2);
-        addRecordToCoordJobTableWithBundle(bundleId, "action2", CoordinatorJob.Status.RUNNING, false, false, 2);
+        addRecordToCoordJobTableWithBundle(bundleId, "action1", CoordinatorJob.Status.RUNNING, false, true, 2);
+        addRecordToCoordJobTableWithBundle(bundleId, "action2", CoordinatorJob.Status.RUNNING, false, true, 2);
 
         final CoordinatorActionBean coordAction1_1 = addRecordToCoordActionTable("action1", 1,
                 CoordinatorAction.Status.RUNNING, "coord-action-get.xml", 0);
@@ -655,18 +845,103 @@ public class TestStatusTransitService ex
 
         bundleJob = jpaService.execute(new BundleJobGetJPAExecutor(bundleId));
         assertFalse(bundleJob.isPending());
+        assertEquals(Job.Status.KILLED, bundleJob.getStatus());
 
         BundleActionBean bundleAction1 = jpaService.execute(new BundleActionGetJPAExecutor(bundleId, "action1"));
         assertFalse(bundleAction1.isPending());
+        assertEquals(Job.Status.KILLED, bundleAction1.getStatus());
 
         CoordinatorJobBean coordJob1 = jpaService.execute(new CoordJobGetJPAExecutor("action1"));
         assertFalse(coordJob1.isPending());
+        assertEquals(Job.Status.KILLED, coordJob1.getStatus());
 
         BundleActionBean bundleAction2 = jpaService.execute(new BundleActionGetJPAExecutor(bundleId, "action2"));
         assertFalse(bundleAction2.isPending());
+        assertEquals(Job.Status.KILLED, bundleAction2.getStatus());
 
         CoordinatorJobBean coordJob2 = jpaService.execute(new CoordJobGetJPAExecutor("action2"));
         assertFalse(coordJob2.isPending());
+        assertEquals(Job.Status.KILLED, coordJob2.getStatus());
+    }
+
+    /**
+     * Test : kill one coord job and keep the other running. Check whether the bundle job's status
+     * is updated to RUNNINGWITHERROR
+     * @throws Exception
+     */
+    public void testBundleStatusTransitServiceRunningWithError() throws Exception {
+        Services.get().destroy();
+        setSystemProperty(StatusTransitService.CONF_BACKWARD_SUPPORT_FOR_STATES_WITHOUT_ERROR, "false");
+        new Services().init();
+        BundleJobBean bundleJob = this.addRecordToBundleJobTable(Job.Status.RUNNING, true);
+        final JPAService jpaService = Services.get().get(JPAService.class);
+        assertNotNull(jpaService);
+
+        final String bundleId = bundleJob.getId();
+        addRecordToBundleActionTable(bundleId, "action1", 1, Job.Status.RUNNING);
+        addRecordToBundleActionTable(bundleId, "action2", 1, Job.Status.RUNNING);
+
+        addRecordToCoordJobTableWithBundle(bundleId, "action1", CoordinatorJob.Status.RUNNING, false, true, 2);
+        addRecordToCoordJobTableWithBundle(bundleId, "action2", CoordinatorJob.Status.RUNNING, false, false, 2);
+
+        final CoordinatorActionBean coordAction1_1 = addRecordToCoordActionTable("action1", 1,
+                CoordinatorAction.Status.RUNNING, "coord-action-get.xml", 0);
+        final CoordinatorActionBean coordAction1_2 = addRecordToCoordActionTable("action1", 2,
+                CoordinatorAction.Status.RUNNING, "coord-action-get.xml", 0);
+
+        final CoordinatorActionBean coordAction1_3 = addRecordToCoordActionTable("action2", 1,
+                CoordinatorAction.Status.RUNNING, "coord-action-get.xml", 0);
+        final CoordinatorActionBean coordAction1_4 = addRecordToCoordActionTable("action2", 2,
+                CoordinatorAction.Status.RUNNING, "coord-action-get.xml", 0);
+
+        this.addRecordToWfJobTable(coordAction1_1.getExternalId(), WorkflowJob.Status.RUNNING,
+                WorkflowInstance.Status.RUNNING);
+        this.addRecordToWfJobTable(coordAction1_2.getExternalId(), WorkflowJob.Status.RUNNING,
+                WorkflowInstance.Status.RUNNING);
+        this.addRecordToWfJobTable(coordAction1_3.getExternalId(), WorkflowJob.Status.RUNNING,
+                WorkflowInstance.Status.RUNNING);
+        this.addRecordToWfJobTable(coordAction1_4.getExternalId(), WorkflowJob.Status.RUNNING,
+                WorkflowInstance.Status.RUNNING);
+
+        new CoordKillXCommand("action1").call();
+
+        waitFor(5 * 1000, new Predicate() {
+            public boolean evaluate() throws Exception {
+                WorkflowJobBean wfJob = jpaService
+                        .execute(new WorkflowJobGetJPAExecutor(coordAction1_4.getExternalId()));
+                return wfJob.getStatus().equals(Job.Status.KILLED);
+            }
+        });
+
+        Runnable runnable = new StatusTransitRunnable();
+        runnable.run();
+
+        waitFor(5 * 1000, new Predicate() {
+            public boolean evaluate() throws Exception {
+                BundleJobBean bundle = jpaService.execute(new BundleJobGetJPAExecutor(bundleId));
+                return bundle.isPending() == false;
+            }
+        });
+
+        bundleJob = jpaService.execute(new BundleJobGetJPAExecutor(bundleId));
+        assertTrue(bundleJob.isPending());
+        assertEquals(Job.Status.RUNNINGWITHERROR, bundleJob.getStatus());
+
+        BundleActionBean bundleAction1 = jpaService.execute(new BundleActionGetJPAExecutor(bundleId, "action1"));
+        assertFalse(bundleAction1.isPending());
+        assertEquals(Job.Status.KILLED, bundleAction1.getStatus());
+
+        CoordinatorJobBean coordJob1 = jpaService.execute(new CoordJobGetJPAExecutor("action1"));
+        assertFalse(coordJob1.isPending());
+        assertEquals(Job.Status.KILLED, coordJob1.getStatus());
+
+        BundleActionBean bundleAction2 = jpaService.execute(new BundleActionGetJPAExecutor(bundleId, "action2"));
+        assertTrue(bundleAction2.isPending());
+        assertEquals(Job.Status.RUNNING, bundleAction2.getStatus());
+
+        CoordinatorJobBean coordJob2 = jpaService.execute(new CoordJobGetJPAExecutor("action2"));
+        assertTrue(coordJob2.isPending());
+        assertEquals(Job.Status.RUNNING, coordJob2.getStatus());
     }
 
     /**
@@ -731,20 +1006,122 @@ public class TestStatusTransitService ex
 
         bundleJob = jpaService.execute(new BundleJobGetJPAExecutor(bundleId));
         assertFalse(bundleJob.isPending());
+        assertEquals(Job.Status.SUSPENDED, bundleJob.getStatus());
 
         BundleActionBean bundleAction1 = jpaService.execute(new BundleActionGetJPAExecutor(bundleId, "action1"));
         assertFalse(bundleAction1.isPending());
+        assertEquals(Job.Status.SUSPENDED, bundleAction1.getStatus());
 
         CoordinatorJobBean coordJob1 = jpaService.execute(new CoordJobGetJPAExecutor("action1"));
         assertFalse(coordJob1.isPending());
+        assertEquals(Job.Status.SUSPENDED, coordJob1.getStatus());
 
         BundleActionBean bundleAction2 = jpaService.execute(new BundleActionGetJPAExecutor(bundleId, "action2"));
         assertFalse(bundleAction2.isPending());
+        assertEquals(Job.Status.SUSPENDED, bundleAction2.getStatus());
 
         CoordinatorJobBean coordJob2 = jpaService.execute(new CoordJobGetJPAExecutor("action2"));
         assertFalse(coordJob2.isPending());
+        assertEquals(Job.Status.SUSPENDED, coordJob2.getStatus());
     }
 
+
+    /**
+     * Test : Check the transition of a bundle job from RUNNING TO SUSPENDEDWITHERROR
+     * @throws Exception
+     */
+    public void testBundleStatusTransitServiceSuspendedWithError() throws Exception {
+        Services.get().destroy();
+        setSystemProperty(StatusTransitService.CONF_BACKWARD_SUPPORT_FOR_STATES_WITHOUT_ERROR, "false");
+        new Services().init();
+        BundleJobBean bundleJob = this.addRecordToBundleJobTable(Job.Status.RUNNING, true);
+        final JPAService jpaService = Services.get().get(JPAService.class);
+        assertNotNull(jpaService);
+
+        final String bundleId = bundleJob.getId();
+        addRecordToBundleActionTable(bundleId, "action1", 0, Job.Status.SUSPENDED);
+        addRecordToBundleActionTable(bundleId, "action2", 0, Job.Status.SUSPENDEDWITHERROR);
+
+        Runnable runnable = new StatusTransitRunnable();
+        runnable.run();
+
+        waitFor(5 * 1000, new Predicate() {
+            public boolean evaluate() throws Exception {
+                BundleJobBean bundle = jpaService.execute(new BundleJobGetJPAExecutor(bundleId));
+                return bundle.isPending() == false;
+            }
+        });
+
+        bundleJob = jpaService.execute(new BundleJobGetJPAExecutor(bundleId));
+        assertFalse(bundleJob.isPending());
+        assertEquals(Job.Status.SUSPENDEDWITHERROR, bundleJob.getStatus());
+
+    }
+
+    /**
+     * Test : Check the transition of a PAUSED bundle job to PAUSEDWITHERROR
+     * @throws Exception
+     */
+    public void testBundleStatusTransitServicePausedWithError() throws Exception {
+        Services.get().destroy();
+        setSystemProperty(StatusTransitService.CONF_BACKWARD_SUPPORT_FOR_STATES_WITHOUT_ERROR, "false");
+        new Services().init();
+        BundleJobBean bundleJob = this.addRecordToBundleJobTable(Job.Status.PAUSED, true);
+        final JPAService jpaService = Services.get().get(JPAService.class);
+        assertNotNull(jpaService);
+
+        final String bundleId = bundleJob.getId();
+        addRecordToBundleActionTable(bundleId, "action1", 1, Job.Status.PAUSED);
+        addRecordToBundleActionTable(bundleId, "action2", 1, Job.Status.PAUSED);
+        addRecordToBundleActionTable(bundleId, "action3", 0, Job.Status.FAILED);
+
+        Runnable runnable = new StatusTransitRunnable();
+        runnable.run();
+
+        waitFor(5 * 1000, new Predicate() {
+            public boolean evaluate() throws Exception {
+                BundleJobBean bundle = jpaService.execute(new BundleJobGetJPAExecutor(bundleId));
+                return bundle.getStatus() == Job.Status.PAUSEDWITHERROR;
+            }
+        });
+
+        bundleJob = jpaService.execute(new BundleJobGetJPAExecutor(bundleId));
+        assertEquals(Job.Status.PAUSEDWITHERROR, bundleJob.getStatus());
+    }
+
+
+    /**
+     * Test : Check the transition of a PAUSEDWITHERROR bundle job to PAUSED
+     * @throws Exception
+     */
+    public void testBundleStatusTransitServicePaused() throws Exception {
+        Services.get().destroy();
+        setSystemProperty(StatusTransitService.CONF_BACKWARD_SUPPORT_FOR_STATES_WITHOUT_ERROR, "false");
+        new Services().init();
+        BundleJobBean bundleJob = this.addRecordToBundleJobTable(Job.Status.PAUSEDWITHERROR, true);
+        final JPAService jpaService = Services.get().get(JPAService.class);
+        assertNotNull(jpaService);
+
+        final String bundleId = bundleJob.getId();
+        addRecordToBundleActionTable(bundleId, "action1", 1, Job.Status.PAUSED);
+        addRecordToBundleActionTable(bundleId, "action2", 1, Job.Status.PAUSED);
+        addRecordToBundleActionTable(bundleId, "action3", 0, Job.Status.SUCCEEDED);
+
+        Runnable runnable = new StatusTransitRunnable();
+        runnable.run();
+
+        waitFor(5 * 1000, new Predicate() {
+            public boolean evaluate() throws Exception {
+                BundleJobBean bundle = jpaService.execute(new BundleJobGetJPAExecutor(bundleId));
+                return bundle.getStatus() == Job.Status.PAUSED;
+            }
+        });
+
+        bundleJob = jpaService.execute(new BundleJobGetJPAExecutor(bundleId));
+        assertEquals(Job.Status.PAUSED, bundleJob.getStatus());
+    }
+
+
     protected WorkflowJobBean addRecordToWfJobTable(String wfId, WorkflowJob.Status jobStatus,
             WorkflowInstance.Status instanceStatus) throws Exception {
         WorkflowApp app = new LiteWorkflowApp("testApp", "<workflow-app/>",

Modified: incubator/oozie/trunk/docs/src/site/twiki/BundleFunctionalSpec.twiki
URL: http://svn.apache.org/viewvc/incubator/oozie/trunk/docs/src/site/twiki/BundleFunctionalSpec.twiki?rev=1368811&r1=1368810&r2=1368811&view=diff
==============================================================================
--- incubator/oozie/trunk/docs/src/site/twiki/BundleFunctionalSpec.twiki (original)
+++ incubator/oozie/trunk/docs/src/site/twiki/BundleFunctionalSpec.twiki Fri Aug  3 07:38:00 2012
@@ -45,39 +45,44 @@ EL expressions can be used in XML attrib
 
 ---+++ 4.1. Bundle Job Status
 
-At any time, a bundle job is in one of the following status: *PREP, RUNNING, SUSPENDED, PREPSUSPENDED, PAUSED, PREPPAUSED, SUCCEEDED, DONEWITHERROR, KILLED, FAILED*.
+At any time, a bundle job is in one of the following status: *PREP, RUNNING, RUNNINGWITHERROR, SUSPENDED, PREPSUSPENDED, SUSPENDEDWITHERROR, PAUSED, PAUSEDWITHERROR, PREPPAUSED, SUCCEEDED, DONEWITHERROR, KILLED, FAILED*.
 
 ---+++ 4.2. Transitions of Bundle Job Status 
 
-At any time, a bundle job is in one of the following status: PREP, RUNNING, PREPSUSPENDED, SUSPENDED, PREPPAUSED, PAUSED, SUCCEEDED, DONWITHERROR, KILLED, FAILED.
-
 Valid bundle job status transitions are:
 
    * *PREP --> PREPSUSPENDED | PREPPAUSED | RUNNING | KILLED*
-   * *RUNNING --> SUSPENDED | PAUSED | SUCCEEDED | DONWITHERROR | KILLED | FAILED*
+   * *RUNNING --> RUNNINGWITHERROR | SUSPENDED | PAUSED | SUCCEEDED | KILLED*
+   * *RUNNINGWITHERROR --> RUNNING | SUSPENDEDWITHERROR | PAUSEDWITHERROR | DONEWITHERROR | FAILED | KILLED*
    * *PREPSUSPENDED --> PREP | KILLED*
    * *SUSPENDED --> RUNNING | KILLED*
+   * *SUSPENDEDWITHERROR --> RUNNINGWITHERROR | KILLED*
    * *PREPPAUSED --> PREP | KILLED*
    * *PAUSED --> SUSPENDED | RUNNING | KILLED*
+   * *PAUSEDWITHERROR --> SUSPENDEDWITHERROR | RUNNINGWITHERROR | KILLED*
 
 ---+++ 4.3. Details of Status Transitions
 When a bundle job is submitted, oozie parses the bundle job XML. Oozie then creates a record for the bundle with status *PREP* and returns a unique ID.
 
-When a user requests to suspend a bundle job that is in *PREP* state, oozie puts the job in status *PREPSUSPEND*. Similarly, when pause time reaches for a bundle job with *PREP* status, oozie puts the job in status *PREPPAUSED*.
+When a user requests to suspend a bundle job that is in *PREP* state, oozie puts the job in status *PREPSUSPENDED*. Similarly, when pause time reaches for a bundle job with *PREP* status, oozie puts the job in status *PREPPAUSED*.
 
-Conversely, when a user requests to resume a PREPSUSPENDED bundle job, oozie puts the job in status *PREP*. And when pause time is reset for a bundle job that is in PREPPAUSED state, oozie puts the job in status *PREP*.
+Conversely, when a user requests to resume a *PREPSUSPENDED* bundle job, oozie puts the job in status *PREP*. And when pause time is reset for a bundle job that is in *PREPPAUSED* state, oozie puts the job in status *PREP*.
 
 There are two ways a bundle job could be started.
+
      * If =kick-off-time= (defined in the bundle xml) reaches. The default value is null which means starts coordinators NOW.
-    * If user sends a start request to START the bundle.
 
-When a bundle job starts, oozie puts the job in status *RUNNING* and it submits the all coordinator jobs.
+     * If user sends a start request to START the bundle.
+
+When a bundle job starts, oozie puts the job in status *RUNNING* and it submits all the coordinator jobs. If any coordinator job goes to *FAILED/KILLED/DONEWITHERROR* state, the bundle job is put in *RUNNINGWITHERROR*
 
 When a user requests to kill a bundle job, oozie puts the job in status *KILLED* and it sends kill to all submitted coordinator jobs.
 
-When a user requests to suspend a bundle job that is not in *PREP* status, oozie puts the job in status *SUSPEND* and it suspends all submitted coordinator jobs.
+When a user requests to suspend a bundle job that is in *RUNNING* status, oozie puts the job in status *SUSPENDED* and it suspends all submitted coordinator jobs. Similarly, when a user requests to suspend a bundle job that is in *RUNNINGWITHERROR* status, oozie puts the job in status *SUSPENDEDWITHERROR* and it suspends all submitted coordinator jobs.
+
+When pause time reaches for a bundle job that is in *RUNNING* status, oozie puts the job in status *PAUSED*. When pause time reaches for a bundle job that is in *RUNNINGWITHERROR* status, oozie puts the job in status *PAUSEDWITHERROR*.
 
-When pause time reaches for a bundle job that is not in *PREP* status, oozie puts the job in status *PAUSED*. When the paused time is reset, Oozie puts back the job in status *RUNNING*.
+Conversely, when a user requests to resume a *SUSPENDED* bundle job, oozie puts the job in status *RUNNING*. Similarly, when a user requests to resume a *SUSPENDEDWITHERROR* bundle job, oozie puts the job in status *RUNNINGWITHERROR*. And when pause time is reset for a bundle job and job status is *PAUSED*, oozie puts the job in status *RUNNING*. Similarly, when the pause time is reset for a bundle job and job status is *PAUSEDWITHERROR*, oozie puts the job in status *RUNNINGWITHERROR*
 
 When all the coordinator jobs finish, oozie updates the bundle status accordingly. If all coordinators reaches to the _same_ terminal state, bundle job status also move to the same status. For example, if all coordinators are *SUCCEEDED*, oozie puts the bundle job into *SUCCEEDED* status. However, if all coordinator jobs don't finish with the same status, oozie puts the bundle job into *DONEWITHERROR*.
     

Modified: incubator/oozie/trunk/docs/src/site/twiki/CoordinatorFunctionalSpec.twiki
URL: http://svn.apache.org/viewvc/incubator/oozie/trunk/docs/src/site/twiki/CoordinatorFunctionalSpec.twiki?rev=1368811&r1=1368810&r2=1368811&view=diff
==============================================================================
--- incubator/oozie/trunk/docs/src/site/twiki/CoordinatorFunctionalSpec.twiki (original)
+++ incubator/oozie/trunk/docs/src/site/twiki/CoordinatorFunctionalSpec.twiki Fri Aug  3 07:38:00 2012
@@ -610,36 +610,39 @@ To create a coordinator job, a job confi
 
 A coordinator job is a running instance of a coordinator application running from a start time to an end time.
 
-At any time, a coordinator job is in one of the following status: *PREP, RUNNING, PREPSUSPENDED, SUSPENDED, PREPPAUSED, PAUSED, SUCCEEDED, DONWITHERROR, KILLED, FAILED*.
+At any time, a coordinator job is in one of the following status: *PREP, RUNNING, RUNNINGWITHERROR, PREPSUSPENDED, SUSPENDED, SUSPENDEDWITHERROR, PREPPAUSED, PAUSED, PAUSEDWITHERROR, SUCCEEDED, DONEWITHERROR, KILLED, FAILED*.
 
 Valid coordinator job status transitions are:
 
    * *PREP --> PREPSUSPENDED | PREPPAUSED | RUNNING | KILLED*
-   * *RUNNING --> SUSPENDED | PAUSED | SUCCEEDED | DONWITHERROR | KILLED | FAILED*
+   * *RUNNING --> RUNNINGWITHERROR | SUSPENDED | PAUSED | SUCCEEDED | KILLED*
+   * *RUNNINGWITHERROR --> RUNNING | SUSPENDEDWITHERROR | PAUSEDWITHERROR | DONEWITHERROR | KILLED | FAILED*
    * *PREPSUSPENDED --> PREP | KILLED*
    * *SUSPENDED --> RUNNING | KILLED*
+   * *SUSPENDEDWITHERROR --> RUNNINGWITHERROR | KILLED*
    * *PREPPAUSED --> PREP | KILLED*
    * *PAUSED --> SUSPENDED | RUNNING | KILLED*
+   * *PAUSEDWITHERROR --> SUSPENDEDWITHERROR | RUNNINGWITHERROR | KILLED*
 
 When a coordinator job is submitted, oozie parses the coordinator job XML. Oozie then creates a record for the coordinator with status *PREP* and returns a unique ID. The coordinator is also started immediately if pause time is not set.
 
-When a user requests to suspend a coordinator job that is in *PREP* state, oozie puts the job in status *PREPSUSPEND*. Similarly, when pause time reaches for a coordinator job with *PREP* status, oozie puts the job in status *PREPPAUSED*.
+When a user requests to suspend a coordinator job that is in *PREP* state, oozie puts the job in status *PREPSUSPENDED*. Similarly, when pause time reaches for a coordinator job with *PREP* status, oozie puts the job in status *PREPPAUSED*.
 
-Conversely, when a user requests to resume a *PREPSUSPEND* coordinator job, oozie puts the job in status *PREP*. And when pause time is reset for a coordinator job and job status is *PREPPAUSED*, oozie puts the job in status *PREP*.
+Conversely, when a user requests to resume a *PREPSUSPENDED* coordinator job, oozie puts the job in status *PREP*. And when pause time is reset for a coordinator job and job status is *PREPPAUSED*, oozie puts the job in status *PREP*.
 
-When a coordinator job starts, oozie puts the job in status *RUNNING* and start materializing workflow jobs based on job frequency.
+When a coordinator job starts, oozie puts the job in status *RUNNING* and start materializing workflow jobs based on job frequency. If any workflow job goes to *FAILED/KILLED/TIMEDOUT* state, the coordinator job is put in *RUNNINGWITHERROR*
 
-When a user requests to kill a coordinator job, oozie puts the job in status *KILLED* and it sends kill to all submitted workflow jobs. If any coordinator action finishes with not *KILLED*, oozie puts the coordinator job into *DONEWITHERROR*.
+When a user requests to kill a coordinator job, oozie puts the job in status *KILLED* and it sends kill to all submitted workflow jobs.
 
-When a user requests to suspend a coordinator job that is in *RUNNING* status, oozie puts the job in status *SUSPEND* and it suspends all submitted workflow jobs.
+When a user requests to suspend a coordinator job that is in *RUNNING* status, oozie puts the job in status *SUSPENDED* and it suspends all submitted workflow jobs. Similarly, when a user requests to suspend a coordinator job that is in *RUNNINGWITHERROR* status, oozie puts the job in status *SUSPENDEDWITHERROR* and it suspends all submitted workflow jobs.
 
-When pause time reaches for a coordinator job that is in *RUNNING* status, oozie puts the job in status *PAUSED*.
+When pause time reaches for a coordinator job that is in *RUNNING* status, oozie puts the job in status *PAUSED*. Similarly, when pause time reaches for a coordinator job that is in *RUNNINGWITHERROR* status, oozie puts the job in status *PAUSEDWITHERROR*.
 
-Conversely, when a user requests to resume a *SUSPEND* coordinator job, oozie puts the job in status *RUNNING*. And when pause time is reset for a coordinator job and job status is *PAUSED*, oozie puts the job in status *RUNNING*.
+Conversely, when a user requests to resume a *SUSPENDED* coordinator job, oozie puts the job in status *RUNNING*. Also,  when a user requests to resume a *SUSPENDEDWITHERROR* coordinator job, oozie puts the job in status *RUNNINGWITHERROR*. And when pause time is reset for a coordinator job and job status is *PAUSED*, oozie puts the job in status *RUNNING*. Also, when the pause time is reset for a coordinator job and job status is *PAUSEDWITHERROR*, oozie puts the job in status *RUNNINGWITHERROR*
 
 A coordinator job creates workflow jobs (commonly coordinator actions) only for the duration of the coordinator job and only if the coordinator job is in *RUNNING* status. If the coordinator job has been suspended, when resumed it will create all the coordinator actions that should have been created during the time it was suspended, actions will not be lost, they will delayed.
 
-When the coordinator job materialization finishs and all workflow jobs finish, oozie updates the coordinator status accordingly. For example, if all workflows are *SUCCEEDED*, oozie put the coordinator job into *SUCCEEDED* status. However, if any workflow job finishes with not *SUCCEEDED* (e.g. *KILLED* or *FAILED* or *TIMEOUT*), oozie puts the coordinator job into *DONEWITHERROR*. If all coordinator actions are *TIMEDOUT*, oozie puts the coordinator job into *DONEWITHERROR*.
+When the coordinator job materialization finishs and all workflow jobs finish, oozie updates the coordinator status accordingly. For example, if all workflows are *SUCCEEDED*, oozie puts the coordinator job into *SUCCEEDED* status. If all workflows are *FAILED*, oozie puts the coordinator job into *FAILED* status. If all workflows are *KILLED*, the coordinator job status changes to KILLED. However, if any workflow job finishes with not *SUCCEEDED* and combination of *KILLED*, *FAILED* or *TIMEOUT*, oozie puts the coordinator job into *DONEWITHERROR*. If all coordinator actions are *TIMEDOUT*, oozie puts the coordinator job into *DONEWITHERROR*.
 
 
 ---++++ 6.1.3. Coordinator Action

Modified: incubator/oozie/trunk/release-log.txt
URL: http://svn.apache.org/viewvc/incubator/oozie/trunk/release-log.txt?rev=1368811&r1=1368810&r2=1368811&view=diff
==============================================================================
--- incubator/oozie/trunk/release-log.txt (original)
+++ incubator/oozie/trunk/release-log.txt Fri Aug  3 07:38:00 2012
@@ -1,10 +1,11 @@
 -- Oozie 3.3.0 release (trunk - unreleased)
 
+OOZIE-708 Update the parent entity status dynamically (virag)
 OOZIE-243 Workflow nodes START/END/KILL/FORK/JOIN should create rows in the action DB table (tucu)
 OOZIE-913 Add Name Node, job-xml, and configuration Elements to FS action (rkanter via tucu)
 OOZIE-937 Mention how to use the sharelib in the Quick Start page of the docs (rkanter via tucu)
 OOZIE-886 Display request header info when trace is enabled (jay7306 via tucu)
-OOZIE-932 space needed between "Created" and "Nominal time" in oozieCLI output (egashira via virag) 
+OOZIE-932 space needed between "Created" and "Nominal time" in oozieCLI output (egashira via virag)
 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)